KAFKA-16333: remove deprecated join method with named param (#16764)

Reviewers: Matthias J. Sax <matthias@confluent.io>
This commit is contained in:
Murali Basani 2024-08-22 22:20:56 +02:00 committed by GitHub
parent a3aa6372ea
commit a2f89f5412
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
3 changed files with 0 additions and 209 deletions

View File

@ -2111,29 +2111,6 @@ public interface KTable<K, V> {
final Function<V, KO> foreignKeyExtractor,
final ValueJoiner<V, VO, VR> joiner);
/**
* Join records of this {@code KTable} with another {@code KTable} using non-windowed inner join.
* <p>
* This is a foreign key join, where the joining key is determined by the {@code foreignKeyExtractor}.
*
* @param other the other {@code KTable} to be joined with this {@code KTable}. Keyed by KO.
* @param foreignKeyExtractor a {@link Function} that extracts the key (KO) from this table's value (V). If the
* result is null, the update is ignored as invalid.
* @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records
* @param named a {@link Named} config used to name the processor in the topology
* @param <VR> the value type of the result {@code KTable}
* @param <KO> the key type of the other {@code KTable}
* @param <VO> the value type of the other {@code KTable}
* @return a {@code KTable} that contains the result of joining this table with {@code other}
*
* @deprecated since 3.1, removal planned for 4.0. Use {@link #join(KTable, Function, ValueJoiner, TableJoined)} instead.
*/
@Deprecated
<VR, KO, VO> KTable<K, VR> join(final KTable<KO, VO> other,
final Function<V, KO> foreignKeyExtractor,
final ValueJoiner<V, VO, VR> joiner,
final Named named);
/**
* Join records of this {@code KTable} with another {@code KTable} using non-windowed inner join,
* using the {@link TableJoined} instance for optional configurations including
@ -2178,32 +2155,6 @@ public interface KTable<K, V> {
final ValueJoiner<V, VO, VR> joiner,
final Materialized<K, VR, KeyValueStore<Bytes, byte[]>> materialized);
/**
* Join records of this {@code KTable} with another {@code KTable} using non-windowed inner join.
* <p>
* This is a foreign key join, where the joining key is determined by the {@code foreignKeyExtractor}.
*
* @param other the other {@code KTable} to be joined with this {@code KTable}. Keyed by KO.
* @param foreignKeyExtractor a {@link Function} that extracts the key (KO) from this table's value (V). If the
* result is null, the update is ignored as invalid.
* @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records
* @param named a {@link Named} config used to name the processor in the topology
* @param materialized a {@link Materialized} that describes how the {@link StateStore} for the resulting {@code KTable}
* should be materialized. Cannot be {@code null}
* @param <VR> the value type of the result {@code KTable}
* @param <KO> the key type of the other {@code KTable}
* @param <VO> the value type of the other {@code KTable}
* @return a {@code KTable} that contains the result of joining this table with {@code other}
*
* @deprecated since 3.1, removal planned for 4.0. Use {@link #join(KTable, Function, ValueJoiner, TableJoined, Materialized)} instead.
*/
@Deprecated
<VR, KO, VO> KTable<K, VR> join(final KTable<KO, VO> other,
final Function<V, KO> foreignKeyExtractor,
final ValueJoiner<V, VO, VR> joiner,
final Named named,
final Materialized<K, VR, KeyValueStore<Bytes, byte[]>> materialized);
/**
* Join records of this {@code KTable} with another {@code KTable} using non-windowed inner join,
* using the {@link TableJoined} instance for optional configurations including
@ -2248,29 +2199,6 @@ public interface KTable<K, V> {
final Function<V, KO> foreignKeyExtractor,
final ValueJoiner<V, VO, VR> joiner);
/**
* Join records of this {@code KTable} with another {@code KTable} using non-windowed left join.
* <p>
* This is a foreign key join, where the joining key is determined by the {@code foreignKeyExtractor}.
*
* @param other the other {@code KTable} to be joined with this {@code KTable}. Keyed by KO.
* @param foreignKeyExtractor a {@link Function} that extracts the key (KO) from this table's value (V). If the
* extract is null, then the right hand side of the result will be null.
* @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records
* @param named a {@link Named} config used to name the processor in the topology
* @param <VR> the value type of the result {@code KTable}
* @param <KO> the key type of the other {@code KTable}
* @param <VO> the value type of the other {@code KTable}
* @return a {@code KTable} that contains the result of joining this table with {@code other}
*
* @deprecated since 3.1, removal planned for 4.0. Use {@link #leftJoin(KTable, Function, ValueJoiner, TableJoined)} instead.
*/
@Deprecated
<VR, KO, VO> KTable<K, VR> leftJoin(final KTable<KO, VO> other,
final Function<V, KO> foreignKeyExtractor,
final ValueJoiner<V, VO, VR> joiner,
final Named named);
/**
* Join records of this {@code KTable} with another {@code KTable} using non-windowed left join,
* using the {@link TableJoined} instance for optional configurations including
@ -2314,32 +2242,6 @@ public interface KTable<K, V> {
final ValueJoiner<V, VO, VR> joiner,
final Materialized<K, VR, KeyValueStore<Bytes, byte[]>> materialized);
/**
* Join records of this {@code KTable} with another {@code KTable} using non-windowed left join.
* <p>
* This is a foreign key join, where the joining key is determined by the {@code foreignKeyExtractor}.
*
* @param other the other {@code KTable} to be joined with this {@code KTable}. Keyed by KO.
* @param foreignKeyExtractor a {@link Function} that extracts the key (KO) from this table's value (V). If the
* extract is null, then the right hand side of the result will be null.
* @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records
* @param named a {@link Named} config used to name the processor in the topology
* @param materialized a {@link Materialized} that describes how the {@link StateStore} for the resulting {@code KTable}
* should be materialized. Cannot be {@code null}
* @param <VR> the value type of the result {@code KTable}
* @param <KO> the key type of the other {@code KTable}
* @param <VO> the value type of the other {@code KTable}
* @return a {@code KTable} that contains the result of joining this table with {@code other}
*
* @deprecated since 3.1, removal planned for 4.0. Use {@link #leftJoin(KTable, Function, ValueJoiner, TableJoined, Materialized)} instead.
*/
@Deprecated
<VR, KO, VO> KTable<K, VR> leftJoin(final KTable<KO, VO> other,
final Function<V, KO> foreignKeyExtractor,
final ValueJoiner<V, VO, VR> joiner,
final Named named,
final Materialized<K, VR, KeyValueStore<Bytes, byte[]>> materialized);
/**
* Join records of this {@code KTable} with another {@code KTable} using non-windowed left join,
* using the {@link TableJoined} instance for optional configurations including

View File

@ -916,22 +916,6 @@ public class KTableImpl<K, S, V> extends AbstractStream<K, V> implements KTable<
);
}
@SuppressWarnings("deprecation")
@Override
public <VR, KO, VO> KTable<K, VR> join(final KTable<KO, VO> other,
final Function<V, KO> foreignKeyExtractor,
final ValueJoiner<V, VO, VR> joiner,
final Named named) {
return doJoinOnForeignKey(
other,
foreignKeyExtractor,
joiner,
TableJoined.as(new NamedInternal(named).name()),
Materialized.with(null, null),
false
);
}
@Override
public <VR, KO, VO> KTable<K, VR> join(final KTable<KO, VO> other,
final Function<V, KO> foreignKeyExtractor,
@ -955,23 +939,6 @@ public class KTableImpl<K, S, V> extends AbstractStream<K, V> implements KTable<
return doJoinOnForeignKey(other, foreignKeyExtractor, joiner, TableJoined.with(null, null), materialized, false);
}
@SuppressWarnings("deprecation")
@Override
public <VR, KO, VO> KTable<K, VR> join(final KTable<KO, VO> other,
final Function<V, KO> foreignKeyExtractor,
final ValueJoiner<V, VO, VR> joiner,
final Named named,
final Materialized<K, VR, KeyValueStore<Bytes, byte[]>> materialized) {
return doJoinOnForeignKey(
other,
foreignKeyExtractor,
joiner,
TableJoined.as(new NamedInternal(named).name()),
materialized,
false
);
}
@Override
public <VR, KO, VO> KTable<K, VR> join(final KTable<KO, VO> other,
final Function<V, KO> foreignKeyExtractor,
@ -1002,22 +969,6 @@ public class KTableImpl<K, S, V> extends AbstractStream<K, V> implements KTable<
);
}
@SuppressWarnings("deprecation")
@Override
public <VR, KO, VO> KTable<K, VR> leftJoin(final KTable<KO, VO> other,
final Function<V, KO> foreignKeyExtractor,
final ValueJoiner<V, VO, VR> joiner,
final Named named) {
return doJoinOnForeignKey(
other,
foreignKeyExtractor,
joiner,
TableJoined.as(new NamedInternal(named).name()),
Materialized.with(null, null),
true
);
}
@Override
public <VR, KO, VO> KTable<K, VR> leftJoin(final KTable<KO, VO> other,
final Function<V, KO> foreignKeyExtractor,
@ -1033,22 +984,6 @@ public class KTableImpl<K, S, V> extends AbstractStream<K, V> implements KTable<
);
}
@SuppressWarnings("deprecation")
@Override
public <VR, KO, VO> KTable<K, VR> leftJoin(final KTable<KO, VO> other,
final Function<V, KO> foreignKeyExtractor,
final ValueJoiner<V, VO, VR> joiner,
final Named named,
final Materialized<K, VR, KeyValueStore<Bytes, byte[]>> materialized) {
return doJoinOnForeignKey(
other,
foreignKeyExtractor,
joiner,
TableJoined.as(new NamedInternal(named).name()),
materialized,
true);
}
@Override
public <VR, KO, VO> KTable<K, VR> leftJoin(final KTable<KO, VO> other,
final Function<V, KO> foreignKeyExtractor,

View File

@ -643,29 +643,6 @@ class KTable[K, V](val inner: KTableJ[K, V]) {
): KTable[K, VR] =
new KTable(inner.join(other.inner, keyExtractor.asJavaFunction, joiner, materialized))
/**
* Join records of this [[KTable]] with another [[KTable]]'s records using non-windowed inner join. Records from this
* table are joined according to the result of keyExtractor on the other KTable.
*
* @param other the other [[KTable]] to be joined with this [[KTable]], keyed on the value obtained from keyExtractor
* @param keyExtractor a function that extracts the foreign key from this table's value
* @param joiner a function that computes the join result for a pair of matching records
* @param named a [[Named]] config used to name the processor in the topology
* @param materialized a `Materialized` that describes how the `StateStore` for the resulting [[KTable]]
* should be materialized.
* @return a [[KTable]] that contains join-records for each key and values computed by the given joiner,
* one for each matched record-pair with the same key
*/
@deprecated("Use join(KTable, Function, ValueJoiner, TableJoined, Materialized) instead", since = "3.1")
def join[VR, KO, VO](
other: KTable[KO, VO],
keyExtractor: Function[V, KO],
joiner: ValueJoiner[V, VO, VR],
named: Named,
materialized: Materialized[K, VR, KeyValueStore[Bytes, Array[Byte]]]
): KTable[K, VR] =
new KTable(inner.join(other.inner, keyExtractor.asJavaFunction, joiner, named, materialized))
/**
* Join records of this [[KTable]] with another [[KTable]]'s records using non-windowed inner join. Records from this
* table are joined according to the result of keyExtractor on the other KTable.
@ -709,29 +686,6 @@ class KTable[K, V](val inner: KTableJ[K, V]) {
): KTable[K, VR] =
new KTable(inner.leftJoin(other.inner, keyExtractor.asJavaFunction, joiner, materialized))
/**
* Join records of this [[KTable]] with another [[KTable]]'s records using non-windowed left join. Records from this
* table are joined according to the result of keyExtractor on the other KTable.
*
* @param other the other [[KTable]] to be joined with this [[KTable]], keyed on the value obtained from keyExtractor
* @param keyExtractor a function that extracts the foreign key from this table's value
* @param joiner a function that computes the join result for a pair of matching records
* @param named a [[Named]] config used to name the processor in the topology
* @param materialized a `Materialized` that describes how the `StateStore` for the resulting [[KTable]]
* should be materialized.
* @return a [[KTable]] that contains join-records for each key and values computed by the given joiner,
* one for each matched record-pair with the same key
*/
@deprecated("Use leftJoin(KTable, Function, ValueJoiner, TableJoined, Materialized) instead", since = "3.1")
def leftJoin[VR, KO, VO](
other: KTable[KO, VO],
keyExtractor: Function[V, KO],
joiner: ValueJoiner[V, VO, VR],
named: Named,
materialized: Materialized[K, VR, KeyValueStore[Bytes, Array[Byte]]]
): KTable[K, VR] =
new KTable(inner.leftJoin(other.inner, keyExtractor.asJavaFunction, joiner, named, materialized))
/**
* Join records of this [[KTable]] with another [[KTable]]'s records using non-windowed left join. Records from this
* table are joined according to the result of keyExtractor on the other KTable.