Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
40 changes: 34 additions & 6 deletions streams/src/main/java/org/apache/kafka/streams/kstream/Joined.java
Original file line number Diff line number Diff line change
Expand Up @@ -22,13 +22,12 @@
* The {@code Joined} class represents optional params that can be passed to
* {@link KStream#join}, {@link KStream#leftJoin}, and {@link KStream#outerJoin} operations.
*/
public class Joined<K, V, VO> {

private final Serde<K> keySerde;
private final Serde<V> valueSerde;
private final Serde<VO> otherValueSerde;
private final String name;
public class Joined<K, V, VO> implements NamedOperation<Joined<K, V, VO>> {

protected final Serde<K> keySerde;
protected final Serde<V> valueSerde;
protected final Serde<VO> otherValueSerde;
protected final String name;

private Joined(final Serde<K> keySerde,
final Serde<V> valueSerde,
Expand All @@ -40,6 +39,10 @@ private Joined(final Serde<K> keySerde,
this.name = name;
}

protected Joined(final Joined<K, V, VO> joined) {
this(joined.keySerde, joined.valueSerde, joined.otherValueSerde, joined.name);
}

/**
* Create an instance of {@code Joined} with key, value, and otherValue {@link Serde} instances.
* {@code null} values are accepted and will be replaced by the default serdes as defined in config.
Expand Down Expand Up @@ -135,11 +138,30 @@ public static <K, V, VO> Joined<K, V, VO> otherValueSerde(final Serde<VO> otherV
* @param <V> value type
* @param <VO> other value type
* @return new {@code Joined} instance configured with the name
*
* @deprecated use {@link #as(String)} instead
*/
@Deprecated
public static <K, V, VO> Joined<K, V, VO> named(final String name) {
return new Joined<>(null, null, null, name);
}

/**
* Create an instance of {@code Joined} with base name for all components of the join, this may
* include any repartition topics created to complete the join.
*
* @param name the name used as the base for naming components of the join including any
* repartition topics
* @param <K> key type
* @param <V> value type
* @param <VO> other value type
* @return new {@code Joined} instance configured with the name
*
*/
public static <K, V, VO> Joined<K, V, VO> as(final String name) {
return new Joined<>(null, null, null, name);
}


/**
* Set the key {@link Serde} to be used. Null values are accepted and will be replaced by the default
Expand Down Expand Up @@ -182,6 +204,7 @@ public Joined<K, V, VO> withOtherValueSerde(final Serde<VO> otherValueSerde) {
* repartition topics
* @return new {@code Joined} instance configured with the {@code name}
*/
@Override
public Joined<K, V, VO> withName(final String name) {
return new Joined<>(keySerde, valueSerde, otherValueSerde, name);
}
Expand All @@ -198,7 +221,12 @@ public Serde<VO> otherValueSerde() {
return otherValueSerde;
}

/**
* @deprecated this method will be removed in a in a future release
*/
@Deprecated
public String name() {
return name;
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.kstream;

/**
* Default interface which can be used to personalized the named of operations, internal topics or store.
*/
interface NamedOperation<T extends NamedOperation<T>> {

/**
* Sets the name to be used for an operation.
*
* @param name the name to use.
* @return an instance of {@link NamedOperation}
*/
T withName(final String name);

}
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@

import java.time.Duration;

public interface Suppressed<K> {
public interface Suppressed<K> extends NamedOperation<Suppressed<K>> {
Comment thread
fhussonnois marked this conversation as resolved.
Outdated

/**
* Marker interface for a buffer configuration that is "strict" in the sense that it will strictly
Expand Down Expand Up @@ -163,5 +163,6 @@ static <K> Suppressed<K> untilTimeLimit(final Duration timeToWaitForMoreEvents,
* @param name The name to be used for the suppression node and changelog topic
* @return The same configuration with the addition of the given {@code name}.
*/
@Override
Suppressed<K> withName(final String name);
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,45 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.kstream.internals;

import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.streams.kstream.Joined;

public class JoinedInternal<K, V, VO> extends Joined<K, V, VO> {

JoinedInternal(final Joined<K, V, VO> joined) {
super(joined);
}

public Serde<K> keySerde() {
return keySerde;
}

public Serde<V> valueSerde() {
return valueSerde;
}

public Serde<VO> otherValueSerde() {
return otherValueSerde;
}

@Override // TODO remove annotation when super.name() is removed
@SuppressWarnings("deprecation") // this method should not be removed if super.name() is removed
public String name() {
return name;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -567,13 +567,15 @@ private <VO, VR> KStream<K, VR> doJoin(final KStream<K, VO> other,
KStreamImpl<K, V> joinThis = this;
KStreamImpl<K, VO> joinOther = (KStreamImpl<K, VO>) other;

final JoinedInternal<K, V, VO> joinedInternal = new JoinedInternal<>(joined);
final String name = joinedInternal.name();
if (joinThis.repartitionRequired) {
final String leftJoinRepartitionTopicName = joined.name() != null ? joined.name() + "-left" : joinThis.name;
final String leftJoinRepartitionTopicName = name != null ? name + "-left" : joinThis.name;
joinThis = joinThis.repartitionForJoin(leftJoinRepartitionTopicName, joined.keySerde(), joined.valueSerde());
}

if (joinOther.repartitionRequired) {
final String rightJoinRepartitionTopicName = joined.name() != null ? joined.name() + "-right" : joinOther.name;
final String rightJoinRepartitionTopicName = name != null ? name + "-right" : joinOther.name;
joinOther = joinOther.repartitionForJoin(rightJoinRepartitionTopicName, joined.keySerde(), joined.otherValueSerde());
}

Expand Down Expand Up @@ -679,9 +681,12 @@ public <VO, VR> KStream<K, VR> join(final KTable<K, VO> other,
Objects.requireNonNull(other, "other can't be null");
Objects.requireNonNull(joiner, "joiner can't be null");
Objects.requireNonNull(joined, "joined can't be null");

final JoinedInternal<K, V, VO> joinedInternal = new JoinedInternal<>(joined);
final String name = joinedInternal.name();
if (repartitionRequired) {
final KStreamImpl<K, V> thisStreamRepartitioned = repartitionForJoin(
joined.name() != null ? joined.name() : name,
name != null ? name : this.name,
joined.keySerde(),
joined.valueSerde()
);
Expand All @@ -703,9 +708,11 @@ public <VO, VR> KStream<K, VR> leftJoin(final KTable<K, VO> other,
Objects.requireNonNull(other, "other can't be null");
Objects.requireNonNull(joiner, "joiner can't be null");
Objects.requireNonNull(joined, "joined can't be null");
final JoinedInternal<K, V, VO> joinedInternal = new JoinedInternal<>(joined);
final String internalName = joinedInternal.name();
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: should this be name (similar to above) only?

if (repartitionRequired) {
final KStreamImpl<K, V> thisStreamRepartitioned = repartitionForJoin(
joined.name() != null ? joined.name() : name,
internalName != null ? internalName : name,
joined.keySerde(),
joined.valueSerde()
);
Expand Down