Skip to content

Commit

Permalink
[FLINK-34549][API] Introduces and implements ProcessConfigurable
Browse files Browse the repository at this point in the history
  • Loading branch information
reswqa committed May 22, 2024
1 parent 1d73510 commit 6691dc6
Show file tree
Hide file tree
Showing 17 changed files with 726 additions and 90 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,8 @@
import org.apache.flink.annotation.Experimental;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.datastream.api.function.TwoInputBroadcastStreamProcessFunction;
import org.apache.flink.datastream.api.stream.KeyedPartitionStream.ProcessConfigurableAndKeyedPartitionStream;
import org.apache.flink.datastream.api.stream.NonKeyedPartitionStream.ProcessConfigurableAndNonKeyedPartitionStream;

/** This interface represents a stream that each parallel task processes the same data. */
@Experimental
Expand All @@ -39,7 +41,7 @@ public interface BroadcastStream<T> extends DataStream {
* @param processFunction to perform operation.
* @return new stream with this operation.
*/
<K, T_OTHER, OUT> NonKeyedPartitionStream<OUT> connectAndProcess(
<K, T_OTHER, OUT> ProcessConfigurableAndNonKeyedPartitionStream<OUT> connectAndProcess(
KeyedPartitionStream<K, T_OTHER> other,
TwoInputBroadcastStreamProcessFunction<T_OTHER, T, OUT> processFunction);

Expand All @@ -50,7 +52,7 @@ <K, T_OTHER, OUT> NonKeyedPartitionStream<OUT> connectAndProcess(
* @param processFunction to perform operation.
* @return new stream with this operation.
*/
<T_OTHER, OUT> NonKeyedPartitionStream<OUT> connectAndProcess(
<T_OTHER, OUT> ProcessConfigurableAndNonKeyedPartitionStream<OUT> connectAndProcess(
NonKeyedPartitionStream<T_OTHER> other,
TwoInputBroadcastStreamProcessFunction<T_OTHER, T, OUT> processFunction);

Expand All @@ -69,7 +71,7 @@ <T_OTHER, OUT> NonKeyedPartitionStream<OUT> connectAndProcess(
* @param newKeySelector to select the key after process.
* @return new {@link KeyedPartitionStream} with this operation.
*/
<K, T_OTHER, OUT> KeyedPartitionStream<K, OUT> connectAndProcess(
<K, T_OTHER, OUT> ProcessConfigurableAndKeyedPartitionStream<K, OUT> connectAndProcess(
KeyedPartitionStream<K, T_OTHER> other,
TwoInputBroadcastStreamProcessFunction<T_OTHER, T, OUT> processFunction,
KeySelector<OUT, K> newKeySelector);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,8 @@ public interface GlobalStream<T> extends DataStream {
* @param processFunction to perform operation.
* @return new stream with this operation.
*/
<OUT> GlobalStream<OUT> process(OneInputStreamProcessFunction<T, OUT> processFunction);
<OUT> ProcessConfigurableAndGlobalStream<OUT> process(
OneInputStreamProcessFunction<T, OUT> processFunction);

/**
* Apply a two output operation to this {@link GlobalStream}.
Expand All @@ -52,7 +53,7 @@ <OUT1, OUT2> TwoGlobalStreams<OUT1, OUT2> process(
* @param processFunction to perform operation.
* @return new stream with this operation.
*/
<T_OTHER, OUT> GlobalStream<OUT> connectAndProcess(
<T_OTHER, OUT> ProcessConfigurableAndGlobalStream<OUT> connectAndProcess(
GlobalStream<T_OTHER> other,
TwoInputNonBroadcastStreamProcessFunction<T, T_OTHER, OUT> processFunction);

Expand All @@ -79,7 +80,12 @@ <T_OTHER, OUT> GlobalStream<OUT> connectAndProcess(
*/
BroadcastStream<T> broadcast();

void toSink(Sink<T> sink);
ProcessConfigurable<?> toSink(Sink<T> sink);

/** This interface represents a configurable {@link GlobalStream}. */
@Experimental
interface ProcessConfigurableAndGlobalStream<T>
extends GlobalStream<T>, ProcessConfigurable<ProcessConfigurableAndGlobalStream<T>> {}

/**
* This class represents a combination of two {@link GlobalStream}. It will be used as the
Expand All @@ -88,9 +94,9 @@ <T_OTHER, OUT> GlobalStream<OUT> connectAndProcess(
@Experimental
interface TwoGlobalStreams<T1, T2> {
/** Get the first stream. */
GlobalStream<T1> getFirst();
ProcessConfigurableAndGlobalStream<T1> getFirst();

/** Get the second stream. */
GlobalStream<T2> getSecond();
ProcessConfigurableAndGlobalStream<T2> getSecond();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import org.apache.flink.datastream.api.function.TwoInputBroadcastStreamProcessFunction;
import org.apache.flink.datastream.api.function.TwoInputNonBroadcastStreamProcessFunction;
import org.apache.flink.datastream.api.function.TwoOutputStreamProcessFunction;
import org.apache.flink.datastream.api.stream.NonKeyedPartitionStream.ProcessConfigurableAndNonKeyedPartitionStream;
import org.apache.flink.datastream.api.stream.NonKeyedPartitionStream.TwoNonKeyedPartitionStreams;

/**
Expand All @@ -45,7 +46,7 @@ public interface KeyedPartitionStream<K, T> extends DataStream {
* @param newKeySelector to select the key after process.
* @return new {@link KeyedPartitionStream} with this operation.
*/
<OUT> KeyedPartitionStream<K, OUT> process(
<OUT> ProcessConfigurableAndKeyedPartitionStream<K, OUT> process(
OneInputStreamProcessFunction<T, OUT> processFunction,
KeySelector<OUT, K> newKeySelector);

Expand All @@ -61,7 +62,7 @@ <OUT> KeyedPartitionStream<K, OUT> process(
* @param processFunction to perform operation.
* @return new {@link NonKeyedPartitionStream} with this operation.
*/
<OUT> NonKeyedPartitionStream<OUT> process(
<OUT> ProcessConfigurableAndNonKeyedPartitionStream<OUT> process(
OneInputStreamProcessFunction<T, OUT> processFunction);

/**
Expand Down Expand Up @@ -105,7 +106,7 @@ <OUT1, OUT2> TwoNonKeyedPartitionStreams<OUT1, OUT2> process(
* @param processFunction to perform operation.
* @return new {@link NonKeyedPartitionStream} with this operation.
*/
<T_OTHER, OUT> NonKeyedPartitionStream<OUT> connectAndProcess(
<T_OTHER, OUT> ProcessConfigurableAndNonKeyedPartitionStream<OUT> connectAndProcess(
KeyedPartitionStream<K, T_OTHER> other,
TwoInputNonBroadcastStreamProcessFunction<T, T_OTHER, OUT> processFunction);

Expand All @@ -123,7 +124,7 @@ <T_OTHER, OUT> NonKeyedPartitionStream<OUT> connectAndProcess(
* @param newKeySelector to select the key after process.
* @return new {@link KeyedPartitionStream} with this operation.
*/
<T_OTHER, OUT> KeyedPartitionStream<K, OUT> connectAndProcess(
<T_OTHER, OUT> ProcessConfigurableAndKeyedPartitionStream<K, OUT> connectAndProcess(
KeyedPartitionStream<K, T_OTHER> other,
TwoInputNonBroadcastStreamProcessFunction<T, T_OTHER, OUT> processFunction,
KeySelector<OUT, K> newKeySelector);
Expand All @@ -141,7 +142,7 @@ <T_OTHER, OUT> KeyedPartitionStream<K, OUT> connectAndProcess(
* @param processFunction to perform operation.
* @return new stream with this operation.
*/
<T_OTHER, OUT> NonKeyedPartitionStream<OUT> connectAndProcess(
<T_OTHER, OUT> ProcessConfigurableAndNonKeyedPartitionStream<OUT> connectAndProcess(
BroadcastStream<T_OTHER> other,
TwoInputBroadcastStreamProcessFunction<T, T_OTHER, OUT> processFunction);

Expand All @@ -160,7 +161,7 @@ <T_OTHER, OUT> NonKeyedPartitionStream<OUT> connectAndProcess(
* @param newKeySelector to select the key after process.
* @return new {@link KeyedPartitionStream} with this operation.
*/
<T_OTHER, OUT> KeyedPartitionStream<K, OUT> connectAndProcess(
<T_OTHER, OUT> ProcessConfigurableAndKeyedPartitionStream<K, OUT> connectAndProcess(
BroadcastStream<T_OTHER> other,
TwoInputBroadcastStreamProcessFunction<T, T_OTHER, OUT> processFunction,
KeySelector<OUT, K> newKeySelector);
Expand Down Expand Up @@ -195,7 +196,13 @@ <T_OTHER, OUT> KeyedPartitionStream<K, OUT> connectAndProcess(
*/
BroadcastStream<T> broadcast();

void toSink(Sink<T> sink);
ProcessConfigurable<?> toSink(Sink<T> sink);

/** This interface represents a configurable {@link KeyedPartitionStream}. */
@Experimental
interface ProcessConfigurableAndKeyedPartitionStream<K, T>
extends KeyedPartitionStream<K, T>,
ProcessConfigurable<ProcessConfigurableAndKeyedPartitionStream<K, T>> {}

/**
* This class represents a combination of two {@link KeyedPartitionStream}. It will be used as
Expand All @@ -204,9 +211,9 @@ <T_OTHER, OUT> KeyedPartitionStream<K, OUT> connectAndProcess(
@Experimental
interface TwoKeyedPartitionStreams<K, T1, T2> {
/** Get the first stream. */
KeyedPartitionStream<K, T1> getFirst();
ProcessConfigurableAndKeyedPartitionStream<K, T1> getFirst();

/** Get the second stream. */
KeyedPartitionStream<K, T2> getSecond();
ProcessConfigurableAndKeyedPartitionStream<K, T2> getSecond();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ public interface NonKeyedPartitionStream<T> extends DataStream {
* @param processFunction to perform operation.
* @return new stream with this operation.
*/
<OUT> NonKeyedPartitionStream<OUT> process(
<OUT> ProcessConfigurableAndNonKeyedPartitionStream<OUT> process(
OneInputStreamProcessFunction<T, OUT> processFunction);

/**
Expand All @@ -57,7 +57,7 @@ <OUT1, OUT2> TwoNonKeyedPartitionStreams<OUT1, OUT2> process(
* @param processFunction to perform operation.
* @return new stream with this operation.
*/
<T_OTHER, OUT> NonKeyedPartitionStream<OUT> connectAndProcess(
<T_OTHER, OUT> ProcessConfigurableAndNonKeyedPartitionStream<OUT> connectAndProcess(
NonKeyedPartitionStream<T_OTHER> other,
TwoInputNonBroadcastStreamProcessFunction<T, T_OTHER, OUT> processFunction);

Expand All @@ -67,7 +67,7 @@ <T_OTHER, OUT> NonKeyedPartitionStream<OUT> connectAndProcess(
* @param processFunction to perform operation.
* @return new stream with this operation.
*/
<T_OTHER, OUT> NonKeyedPartitionStream<OUT> connectAndProcess(
<T_OTHER, OUT> ProcessConfigurableAndNonKeyedPartitionStream<OUT> connectAndProcess(
BroadcastStream<T_OTHER> other,
TwoInputBroadcastStreamProcessFunction<T, T_OTHER, OUT> processFunction);

Expand Down Expand Up @@ -101,7 +101,13 @@ <T_OTHER, OUT> NonKeyedPartitionStream<OUT> connectAndProcess(
*/
BroadcastStream<T> broadcast();

void toSink(Sink<T> sink);
ProcessConfigurable<?> toSink(Sink<T> sink);

/** This interface represents a configurable {@link NonKeyedPartitionStream}. */
@Experimental
interface ProcessConfigurableAndNonKeyedPartitionStream<T>
extends NonKeyedPartitionStream<T>,
ProcessConfigurable<ProcessConfigurableAndNonKeyedPartitionStream<T>> {}

/**
* This interface represents a combination of two {@link NonKeyedPartitionStream}. It will be
Expand All @@ -110,9 +116,9 @@ <T_OTHER, OUT> NonKeyedPartitionStream<OUT> connectAndProcess(
@Experimental
interface TwoNonKeyedPartitionStreams<T1, T2> {
/** Get the first stream. */
NonKeyedPartitionStream<T1> getFirst();
ProcessConfigurableAndNonKeyedPartitionStream<T1> getFirst();

/** Get the second stream. */
NonKeyedPartitionStream<T2> getSecond();
ProcessConfigurableAndNonKeyedPartitionStream<T2> getSecond();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,84 @@
/*
* 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.flink.datastream.api.stream;

import org.apache.flink.annotation.Experimental;
import org.apache.flink.api.common.SlotSharingGroup;

/**
* This represents the configuration handle of processing. Many processing-related properties can be
* set through the `withYYY` method provided by this interface, such as withName, withUid, etc.
*/
@Experimental
public interface ProcessConfigurable<T extends ProcessConfigurable<T>> {
/**
* Sets an ID for this operator.
*
* <p>The specified ID is used to assign the same process operator ID across job submissions
* (for example when starting a job from a savepoint).
*
* <p><strong>Important</strong>: this ID needs to be unique per transformation and job.
* Otherwise, job submission will fail.
*
* @param uid The unique user-specified ID of this transformation.
* @return The operator with the specified ID.
*/
T withUid(String uid);

/**
* Sets the name of the current data stream. This name is used by the visualization and logging
* during runtime.
*
* @return The named operator.
*/
T withName(String name);

/**
* Sets the parallelism for this process operator.
*
* @param parallelism The parallelism for this operator.
* @return The operator with set parallelism.
*/
T withParallelism(int parallelism);

/**
* Sets the maximum parallelism of this operator.
*
* <p>The maximum parallelism specifies the upper bound for dynamic scaling. It also defines the
* number of key groups used for partitioned state.
*
* @param maxParallelism Maximum parallelism
* @return The operator with set maximum parallelism
*/
T withMaxParallelism(int maxParallelism);

/**
* Sets the slot sharing group of this operation. Parallel instances of operations that are in
* the same slot sharing group will be co-located in the same TaskManager slot, if possible.
*
* <p>Operations inherit the slot sharing group of input operations if all input operations are
* in the same slot sharing group and no slot sharing group was explicitly specified.
*
* <p>Initially an operation is in the default slot sharing group. An operation can be put into
* the default group explicitly by setting the slot sharing group with name {@code "default"}.
*
* @param slotSharingGroup Which contains name and its resource spec.
*/
T withSlotSharingGroup(SlotSharingGroup slotSharingGroup);
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,9 @@
import org.apache.flink.datastream.api.function.TwoInputBroadcastStreamProcessFunction;
import org.apache.flink.datastream.api.stream.BroadcastStream;
import org.apache.flink.datastream.api.stream.KeyedPartitionStream;
import org.apache.flink.datastream.api.stream.KeyedPartitionStream.ProcessConfigurableAndKeyedPartitionStream;
import org.apache.flink.datastream.api.stream.NonKeyedPartitionStream;
import org.apache.flink.datastream.api.stream.NonKeyedPartitionStream.ProcessConfigurableAndNonKeyedPartitionStream;
import org.apache.flink.datastream.impl.ExecutionEnvironmentImpl;
import org.apache.flink.datastream.impl.operators.KeyedTwoInputBroadcastProcessOperator;
import org.apache.flink.datastream.impl.operators.TwoInputBroadcastProcessOperator;
Expand All @@ -48,7 +50,7 @@ private BroadcastStreamImpl(
}

@Override
public <K, T_OTHER, OUT> NonKeyedPartitionStream<OUT> connectAndProcess(
public <K, T_OTHER, OUT> ProcessConfigurableAndNonKeyedPartitionStream<OUT> connectAndProcess(
KeyedPartitionStream<K, T_OTHER> other,
TwoInputBroadcastStreamProcessFunction<T_OTHER, T, OUT> processFunction) {
TypeInformation<OUT> outTypeInfo =
Expand All @@ -67,11 +69,12 @@ public <K, T_OTHER, OUT> NonKeyedPartitionStream<OUT> connectAndProcess(
outTypeInfo,
processOperator);
environment.addOperator(outTransformation);
return new NonKeyedPartitionStreamImpl<>(environment, outTransformation);
return StreamUtils.wrapWithConfigureHandle(
new NonKeyedPartitionStreamImpl<>(environment, outTransformation));
}

@Override
public <T_OTHER, OUT> NonKeyedPartitionStream<OUT> connectAndProcess(
public <T_OTHER, OUT> ProcessConfigurableAndNonKeyedPartitionStream<OUT> connectAndProcess(
NonKeyedPartitionStream<T_OTHER> other,
TwoInputBroadcastStreamProcessFunction<T_OTHER, T, OUT> processFunction) {
TypeInformation<OUT> outTypeInfo =
Expand All @@ -90,11 +93,12 @@ public <T_OTHER, OUT> NonKeyedPartitionStream<OUT> connectAndProcess(
outTypeInfo,
processOperator);
environment.addOperator(outTransformation);
return new NonKeyedPartitionStreamImpl<>(environment, outTransformation);
return StreamUtils.wrapWithConfigureHandle(
new NonKeyedPartitionStreamImpl<>(environment, outTransformation));
}

@Override
public <K, T_OTHER, OUT> KeyedPartitionStream<K, OUT> connectAndProcess(
public <K, T_OTHER, OUT> ProcessConfigurableAndKeyedPartitionStream<K, OUT> connectAndProcess(
KeyedPartitionStream<K, T_OTHER> other,
TwoInputBroadcastStreamProcessFunction<T_OTHER, T, OUT> processFunction,
KeySelector<OUT, K> newKeySelector) {
Expand All @@ -118,10 +122,11 @@ public <K, T_OTHER, OUT> KeyedPartitionStream<K, OUT> connectAndProcess(
new NonKeyedPartitionStreamImpl<>(environment, outTransformation);
environment.addOperator(outTransformation);
// Construct a keyed stream directly without partitionTransformation to avoid shuffle.
return new KeyedPartitionStreamImpl<>(
outputStream,
outTransformation,
newKeySelector,
TypeExtractor.getKeySelectorTypes(newKeySelector, outputStream.getType()));
return StreamUtils.wrapWithConfigureHandle(
new KeyedPartitionStreamImpl<>(
outputStream,
outTransformation,
newKeySelector,
TypeExtractor.getKeySelectorTypes(newKeySelector, outputStream.getType())));
}
}
Loading

0 comments on commit 6691dc6

Please sign in to comment.