forked from apache/kafka
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
KAFKA-16362: Fix type-unsafety in KStreamKStreamJoin caused by isLeft…
…Side (apache#15601) The introduced changes provide a cleaner definition of the join side in KStreamKStreamJoin. Before, this was done by using a Boolean flag, which led to returning a raw LeftOrRightValue without generic arguments because the generic type arguments depended on the boolean input. Reviewers: Greg Harris <[email protected]>, Bruno Cadonna <[email protected]>
- Loading branch information
Showing
9 changed files
with
244 additions
and
114 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
71 changes: 71 additions & 0 deletions
71
.../src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinLeftSide.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,71 @@ | ||
/* | ||
* 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.streams.kstream.ValueJoinerWithKey; | ||
import org.apache.kafka.streams.kstream.internals.KStreamImplJoin.TimeTrackerSupplier; | ||
import org.apache.kafka.streams.processor.api.Processor; | ||
import org.apache.kafka.streams.state.internals.LeftOrRightValue; | ||
import org.apache.kafka.streams.state.internals.TimestampedKeyAndJoinSide; | ||
|
||
import java.util.Optional; | ||
|
||
class KStreamKStreamJoinLeftSide<K, VLeft, VRight, VOut> extends KStreamKStreamJoin<K, VLeft, VRight, VOut, VLeft, VRight> { | ||
|
||
KStreamKStreamJoinLeftSide(final String otherWindowName, | ||
final JoinWindowsInternal windows, | ||
final ValueJoinerWithKey<? super K, ? super VLeft, ? super VRight, ? extends VOut> joiner, | ||
final boolean outer, | ||
final Optional<String> outerJoinWindowName, | ||
final TimeTrackerSupplier sharedTimeTrackerSupplier) { | ||
super(otherWindowName, windows, joiner, outer, outerJoinWindowName, windows.beforeMs, windows.afterMs, | ||
sharedTimeTrackerSupplier); | ||
} | ||
|
||
@Override | ||
public Processor<K, VLeft, K, VOut> get() { | ||
return new KStreamKStreamJoinLeftProcessor(); | ||
} | ||
|
||
private class KStreamKStreamJoinLeftProcessor extends KStreamKStreamJoinProcessor { | ||
|
||
@Override | ||
public TimestampedKeyAndJoinSide<K> makeThisKey(final K key, final long timestamp) { | ||
return TimestampedKeyAndJoinSide.makeLeft(key, timestamp); | ||
} | ||
|
||
@Override | ||
public LeftOrRightValue<VLeft, VRight> makeThisValue(final VLeft thisValue) { | ||
return LeftOrRightValue.makeLeftValue(thisValue); | ||
} | ||
|
||
@Override | ||
public TimestampedKeyAndJoinSide<K> makeOtherKey(final K key, final long timestamp) { | ||
return TimestampedKeyAndJoinSide.makeRight(key, timestamp); | ||
} | ||
|
||
@Override | ||
public VLeft getThisValue(final LeftOrRightValue<? extends VLeft, ? extends VRight> leftOrRightValue) { | ||
return leftOrRightValue.getLeftValue(); | ||
} | ||
|
||
@Override | ||
public VRight getOtherValue(final LeftOrRightValue<? extends VLeft, ? extends VRight> leftOrRightValue) { | ||
return leftOrRightValue.getRightValue(); | ||
} | ||
} | ||
} |
Oops, something went wrong.