All Downloads are FREE. Search and download functionalities are using the official Maven repository.

org.apache.flink.streaming.api.datastream.KeyedStream Maven / Gradle / Ivy

There is a newer version: 1.14.6
Show newest version
/*
 * 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.streaming.api.datastream;

import org.apache.flink.annotation.Internal;
import org.apache.flink.annotation.Public;
import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.api.common.InvalidProgramException;
import org.apache.flink.api.common.functions.FlatMapFunction;
import org.apache.flink.api.common.functions.FoldFunction;
import org.apache.flink.api.common.functions.ReduceFunction;
import org.apache.flink.api.common.state.FoldingStateDescriptor;
import org.apache.flink.api.common.state.ReducingStateDescriptor;
import org.apache.flink.api.common.state.ValueStateDescriptor;
import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.Utils;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo;
import org.apache.flink.api.java.typeutils.PojoTypeInfo;
import org.apache.flink.api.java.typeutils.TupleTypeInfoBase;
import org.apache.flink.api.java.typeutils.TypeExtractor;
import org.apache.flink.streaming.api.TimeCharacteristic;
import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
import org.apache.flink.streaming.api.functions.ProcessFunction;
import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction;
import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator;
import org.apache.flink.streaming.api.functions.aggregation.SumAggregator;
import org.apache.flink.streaming.api.functions.query.QueryableAppendingStateOperator;
import org.apache.flink.streaming.api.functions.query.QueryableValueStateOperator;
import org.apache.flink.streaming.api.functions.sink.SinkFunction;
import org.apache.flink.streaming.api.graph.StreamGraphGenerator;
import org.apache.flink.streaming.api.operators.KeyedProcessOperator;
import org.apache.flink.streaming.api.operators.LegacyKeyedProcessOperator;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.operators.StreamGroupedFold;
import org.apache.flink.streaming.api.operators.StreamGroupedReduce;
import org.apache.flink.streaming.api.transformations.OneInputTransformation;
import org.apache.flink.streaming.api.transformations.PartitionTransformation;
import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
import org.apache.flink.streaming.api.windowing.assigners.SlidingEventTimeWindows;
import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows;
import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows;
import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
import org.apache.flink.streaming.api.windowing.evictors.CountEvictor;
import org.apache.flink.streaming.api.windowing.time.Time;
import org.apache.flink.streaming.api.windowing.triggers.CountTrigger;
import org.apache.flink.streaming.api.windowing.triggers.PurgingTrigger;
import org.apache.flink.streaming.api.windowing.windows.GlobalWindow;
import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
import org.apache.flink.streaming.api.windowing.windows.Window;
import org.apache.flink.streaming.runtime.partitioner.KeyGroupStreamPartitioner;
import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;

import org.apache.commons.lang3.StringUtils;

import java.util.ArrayList;
import java.util.List;
import java.util.Stack;
import java.util.UUID;

/**
 * A {@link KeyedStream} represents a {@link DataStream} on which operator state is
 * partitioned by key using a provided {@link KeySelector}. Typical operations supported by a
 * {@code DataStream} are also possible on a {@code KeyedStream}, with the exception of
 * partitioning methods such as shuffle, forward and keyBy.
 *
 * 

Reduce-style operations, such as {@link #reduce}, {@link #sum} and {@link #fold} work on * elements that have the same key. * * @param The type of the elements in the Keyed Stream. * @param The type of the key in the Keyed Stream. */ @Public public class KeyedStream extends DataStream { /** * The key selector that can get the key by which the stream if partitioned from the elements. */ private final KeySelector keySelector; /** The type of the key by which the stream is partitioned. */ private final TypeInformation keyType; /** * Creates a new {@link KeyedStream} using the given {@link KeySelector} * to partition operator state by key. * * @param dataStream * Base stream of data * @param keySelector * Function for determining state partitions */ public KeyedStream(DataStream dataStream, KeySelector keySelector) { this(dataStream, keySelector, TypeExtractor.getKeySelectorTypes(keySelector, dataStream.getType())); } /** * Creates a new {@link KeyedStream} using the given {@link KeySelector} * to partition operator state by key. * * @param dataStream * Base stream of data * @param keySelector * Function for determining state partitions */ public KeyedStream(DataStream dataStream, KeySelector keySelector, TypeInformation keyType) { this( dataStream, new PartitionTransformation<>( dataStream.getTransformation(), new KeyGroupStreamPartitioner<>(keySelector, StreamGraphGenerator.DEFAULT_LOWER_BOUND_MAX_PARALLELISM)), keySelector, keyType); } /** * Creates a new {@link KeyedStream} using the given {@link KeySelector} and {@link TypeInformation} * to partition operator state by key, where the partitioning is defined by a {@link PartitionTransformation}. * * @param stream * Base stream of data * @param partitionTransformation * Function that determines how the keys are distributed to downstream operator(s) * @param keySelector * Function to extract keys from the base stream * @param keyType * Defines the type of the extracted keys */ @Internal KeyedStream( DataStream stream, PartitionTransformation partitionTransformation, KeySelector keySelector, TypeInformation keyType) { super(stream.getExecutionEnvironment(), partitionTransformation); this.keySelector = clean(keySelector); this.keyType = validateKeyType(keyType); } /** * Validates that a given type of element (as encoded by the provided {@link TypeInformation}) can be * used as a key in the {@code DataStream.keyBy()} operation. This is done by searching depth-first the * key type and checking if each of the composite types satisfies the required conditions * (see {@link #validateKeyTypeIsHashable(TypeInformation)}). * * @param keyType The {@link TypeInformation} of the key. */ private TypeInformation validateKeyType(TypeInformation keyType) { Stack> stack = new Stack<>(); stack.push(keyType); List> unsupportedTypes = new ArrayList<>(); while (!stack.isEmpty()) { TypeInformation typeInfo = stack.pop(); if (!validateKeyTypeIsHashable(typeInfo)) { unsupportedTypes.add(typeInfo); } if (typeInfo instanceof TupleTypeInfoBase) { for (int i = 0; i < typeInfo.getArity(); i++) { stack.push(((TupleTypeInfoBase) typeInfo).getTypeAt(i)); } } } if (!unsupportedTypes.isEmpty()) { throw new InvalidProgramException("Type " + keyType + " cannot be used as key. Contained " + "UNSUPPORTED key types: " + StringUtils.join(unsupportedTypes, ", ") + ". Look " + "at the keyBy() documentation for the conditions a type has to satisfy in order to be " + "eligible for a key."); } return keyType; } /** * Validates that a given type of element (as encoded by the provided {@link TypeInformation}) can be * used as a key in the {@code DataStream.keyBy()} operation. * * @param type The {@link TypeInformation} of the type to check. * @return {@code false} if: *

    *
  1. it is a POJO type but does not override the {@link #hashCode()} method and relies on * the {@link Object#hashCode()} implementation.
  2. *
  3. it is an array of any type (see {@link PrimitiveArrayTypeInfo}, {@link BasicArrayTypeInfo}, * {@link ObjectArrayTypeInfo}).
  4. *
, * {@code true} otherwise. */ private boolean validateKeyTypeIsHashable(TypeInformation type) { try { return (type instanceof PojoTypeInfo) ? !type.getTypeClass().getMethod("hashCode").getDeclaringClass().equals(Object.class) : !(type instanceof PrimitiveArrayTypeInfo || type instanceof BasicArrayTypeInfo || type instanceof ObjectArrayTypeInfo); } catch (NoSuchMethodException ignored) { // this should never happen as we are just searching for the hashCode() method. } return false; } // ------------------------------------------------------------------------ // properties // ------------------------------------------------------------------------ /** * Gets the key selector that can get the key by which the stream if partitioned from the elements. * @return The key selector for the key. */ @Internal public KeySelector getKeySelector() { return this.keySelector; } /** * Gets the type of the key by which the stream is partitioned. * @return The type of the key by which the stream is partitioned. */ @Internal public TypeInformation getKeyType() { return keyType; } @Override protected DataStream setConnectionType(StreamPartitioner partitioner) { throw new UnsupportedOperationException("Cannot override partitioning for KeyedStream."); } // ------------------------------------------------------------------------ // basic transformations // ------------------------------------------------------------------------ @Override @PublicEvolving public SingleOutputStreamOperator transform(String operatorName, TypeInformation outTypeInfo, OneInputStreamOperator operator) { SingleOutputStreamOperator returnStream = super.transform(operatorName, outTypeInfo, operator); // inject the key selector and key type OneInputTransformation transform = (OneInputTransformation) returnStream.getTransformation(); transform.setStateKeySelector(keySelector); transform.setStateKeyType(keyType); return returnStream; } @Override public DataStreamSink addSink(SinkFunction sinkFunction) { DataStreamSink result = super.addSink(sinkFunction); result.getTransformation().setStateKeySelector(keySelector); result.getTransformation().setStateKeyType(keyType); return result; } /** * Applies the given {@link ProcessFunction} on the input stream, thereby creating a transformed output stream. * *

The function will be called for every element in the input streams and can produce zero * or more output elements. Contrary to the {@link DataStream#flatMap(FlatMapFunction)} * function, this function can also query the time and set timers. When reacting to the firing * of set timers the function can directly emit elements and/or register yet more timers. * * @param processFunction The {@link ProcessFunction} that is called for each element * in the stream. * * @param The type of elements emitted by the {@code ProcessFunction}. * * @return The transformed {@link DataStream}. * * @deprecated Use {@link KeyedStream#process(KeyedProcessFunction)} */ @Deprecated @Override @PublicEvolving public SingleOutputStreamOperator process(ProcessFunction processFunction) { TypeInformation outType = TypeExtractor.getUnaryOperatorReturnType( processFunction, ProcessFunction.class, 0, 1, TypeExtractor.NO_INDEX, TypeExtractor.NO_INDEX, getType(), Utils.getCallLocationName(), true); return process(processFunction, outType); } /** * Applies the given {@link ProcessFunction} on the input stream, thereby creating a transformed output stream. * *

The function will be called for every element in the input streams and can produce zero * or more output elements. Contrary to the {@link DataStream#flatMap(FlatMapFunction)} * function, this function can also query the time and set timers. When reacting to the firing * of set timers the function can directly emit elements and/or register yet more timers. * * @param processFunction The {@link ProcessFunction} that is called for each element * in the stream. * @param outputType {@link TypeInformation} for the result type of the function. * * @param The type of elements emitted by the {@code ProcessFunction}. * * @return The transformed {@link DataStream}. * * @deprecated Use {@link KeyedStream#process(KeyedProcessFunction, TypeInformation)} */ @Deprecated @Override @Internal public SingleOutputStreamOperator process( ProcessFunction processFunction, TypeInformation outputType) { LegacyKeyedProcessOperator operator = new LegacyKeyedProcessOperator<>(clean(processFunction)); return transform("Process", outputType, operator); } /** * Applies the given {@link KeyedProcessFunction} on the input stream, thereby creating a transformed output stream. * *

The function will be called for every element in the input streams and can produce zero * or more output elements. Contrary to the {@link DataStream#flatMap(FlatMapFunction)} * function, this function can also query the time and set timers. When reacting to the firing * of set timers the function can directly emit elements and/or register yet more timers. * * @param keyedProcessFunction The {@link KeyedProcessFunction} that is called for each element in the stream. * * @param The type of elements emitted by the {@code KeyedProcessFunction}. * * @return The transformed {@link DataStream}. */ @PublicEvolving public SingleOutputStreamOperator process(KeyedProcessFunction keyedProcessFunction) { TypeInformation outType = TypeExtractor.getUnaryOperatorReturnType( keyedProcessFunction, KeyedProcessFunction.class, 1, 2, TypeExtractor.NO_INDEX, TypeExtractor.NO_INDEX, getType(), Utils.getCallLocationName(), true); return process(keyedProcessFunction, outType); } /** * Applies the given {@link KeyedProcessFunction} on the input stream, thereby creating a transformed output stream. * *

The function will be called for every element in the input streams and can produce zero * or more output elements. Contrary to the {@link DataStream#flatMap(FlatMapFunction)} * function, this function can also query the time and set timers. When reacting to the firing * of set timers the function can directly emit elements and/or register yet more timers. * * @param keyedProcessFunction The {@link KeyedProcessFunction} that is called for each element in the stream. * * @param outputType {@link TypeInformation} for the result type of the function. * * @param The type of elements emitted by the {@code KeyedProcessFunction}. * * @return The transformed {@link DataStream}. */ @Internal public SingleOutputStreamOperator process( KeyedProcessFunction keyedProcessFunction, TypeInformation outputType) { KeyedProcessOperator operator = new KeyedProcessOperator<>(clean(keyedProcessFunction)); return transform("KeyedProcess", outputType, operator); } // ------------------------------------------------------------------------ // Windowing // ------------------------------------------------------------------------ /** * Windows this {@code KeyedStream} into tumbling time windows. * *

This is a shortcut for either {@code .window(TumblingEventTimeWindows.of(size))} or * {@code .window(TumblingProcessingTimeWindows.of(size))} depending on the time characteristic * set using * {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#setStreamTimeCharacteristic(org.apache.flink.streaming.api.TimeCharacteristic)} * * @param size The size of the window. */ public WindowedStream timeWindow(Time size) { if (environment.getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime) { return window(TumblingProcessingTimeWindows.of(size)); } else { return window(TumblingEventTimeWindows.of(size)); } } /** * Windows this {@code KeyedStream} into sliding time windows. * *

This is a shortcut for either {@code .window(SlidingEventTimeWindows.of(size, slide))} or * {@code .window(SlidingProcessingTimeWindows.of(size, slide))} depending on the time * characteristic set using * {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#setStreamTimeCharacteristic(org.apache.flink.streaming.api.TimeCharacteristic)} * * @param size The size of the window. */ public WindowedStream timeWindow(Time size, Time slide) { if (environment.getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime) { return window(SlidingProcessingTimeWindows.of(size, slide)); } else { return window(SlidingEventTimeWindows.of(size, slide)); } } /** * Windows this {@code KeyedStream} into tumbling count windows. * * @param size The size of the windows in number of elements. */ public WindowedStream countWindow(long size) { return window(GlobalWindows.create()).trigger(PurgingTrigger.of(CountTrigger.of(size))); } /** * Windows this {@code KeyedStream} into sliding count windows. * * @param size The size of the windows in number of elements. * @param slide The slide interval in number of elements. */ public WindowedStream countWindow(long size, long slide) { return window(GlobalWindows.create()) .evictor(CountEvictor.of(size)) .trigger(CountTrigger.of(slide)); } /** * Windows this data stream to a {@code WindowedStream}, which evaluates windows * over a key grouped stream. Elements are put into windows by a {@link WindowAssigner}. The * grouping of elements is done both by key and by window. * *

A {@link org.apache.flink.streaming.api.windowing.triggers.Trigger} can be defined to * specify when windows are evaluated. However, {@code WindowAssigners} have a default * {@code Trigger} that is used if a {@code Trigger} is not specified. * * @param assigner The {@code WindowAssigner} that assigns elements to windows. * @return The trigger windows data stream. */ @PublicEvolving public WindowedStream window(WindowAssigner assigner) { return new WindowedStream<>(this, assigner); } // ------------------------------------------------------------------------ // Non-Windowed aggregation operations // ------------------------------------------------------------------------ /** * Applies a reduce transformation on the grouped data stream grouped on by * the given key position. The {@link ReduceFunction} will receive input * values based on the key value. Only input values with the same key will * go to the same reducer. * * @param reducer * The {@link ReduceFunction} that will be called for every * element of the input values with the same key. * @return The transformed DataStream. */ public SingleOutputStreamOperator reduce(ReduceFunction reducer) { return transform("Keyed Reduce", getType(), new StreamGroupedReduce( clean(reducer), getType().createSerializer(getExecutionConfig()))); } /** * Applies a fold transformation on the grouped data stream grouped on by * the given key position. The {@link FoldFunction} will receive input * values based on the key value. Only input values with the same key will * go to the same folder. * * @param folder * The {@link FoldFunction} that will be called for every element * of the input values with the same key. * @param initialValue * The initialValue passed to the folders for each key. * @return The transformed DataStream. * * @deprecated will be removed in a future version */ @Deprecated public SingleOutputStreamOperator fold(R initialValue, FoldFunction folder) { TypeInformation outType = TypeExtractor.getFoldReturnTypes( clean(folder), getType(), Utils.getCallLocationName(), true); return transform("Keyed Fold", outType, new StreamGroupedFold<>(clean(folder), initialValue)); } /** * Applies an aggregation that gives a rolling sum of the data stream at the * given position grouped by the given key. An independent aggregate is kept * per key. * * @param positionToSum * The field position in the data points to sum. This is applicable to * Tuple types, basic and primitive array types, Scala case classes, * and primitive types (which is considered as having one field). * @return The transformed DataStream. */ public SingleOutputStreamOperator sum(int positionToSum) { return aggregate(new SumAggregator<>(positionToSum, getType(), getExecutionConfig())); } /** * Applies an aggregation that gives the current sum of the data * stream at the given field by the given key. An independent * aggregate is kept per key. * * @param field * In case of a POJO, Scala case class, or Tuple type, the * name of the (public) field on which to perform the aggregation. * Additionally, a dot can be used to drill down into nested * objects, as in {@code "field1.fieldxy" }. * Furthermore "*" can be specified in case of a basic type * (which is considered as having only one field). * @return The transformed DataStream. */ public SingleOutputStreamOperator sum(String field) { return aggregate(new SumAggregator<>(field, getType(), getExecutionConfig())); } /** * Applies an aggregation that gives the current minimum of the data * stream at the given position by the given key. An independent aggregate * is kept per key. * * @param positionToMin * The field position in the data points to minimize. This is applicable to * Tuple types, Scala case classes, and primitive types (which is considered * as having one field). * @return The transformed DataStream. */ public SingleOutputStreamOperator min(int positionToMin) { return aggregate(new ComparableAggregator<>(positionToMin, getType(), AggregationFunction.AggregationType.MIN, getExecutionConfig())); } /** * Applies an aggregation that gives the current minimum of the * data stream at the given field expression by the given key. An * independent aggregate is kept per key. A field expression is either the * name of a public field or a getter method with parentheses of the * {@link DataStream}'s underlying type. A dot can be used to drill down into * objects, as in {@code "field1.fieldxy" }. * * @param field * In case of a POJO, Scala case class, or Tuple type, the * name of the (public) field on which to perform the aggregation. * Additionally, a dot can be used to drill down into nested * objects, as in {@code "field1.fieldxy" }. * Furthermore "*" can be specified in case of a basic type * (which is considered as having only one field). * @return The transformed DataStream. */ public SingleOutputStreamOperator min(String field) { return aggregate(new ComparableAggregator<>(field, getType(), AggregationFunction.AggregationType.MIN, false, getExecutionConfig())); } /** * Applies an aggregation that gives the current maximum of the data stream * at the given position by the given key. An independent aggregate is kept * per key. * * @param positionToMax * The field position in the data points to minimize. This is applicable to * Tuple types, Scala case classes, and primitive types (which is considered * as having one field). * @return The transformed DataStream. */ public SingleOutputStreamOperator max(int positionToMax) { return aggregate(new ComparableAggregator<>(positionToMax, getType(), AggregationFunction.AggregationType.MAX, getExecutionConfig())); } /** * Applies an aggregation that gives the current maximum of the * data stream at the given field expression by the given key. An * independent aggregate is kept per key. A field expression is either the * name of a public field or a getter method with parentheses of the * {@link DataStream}'s underlying type. A dot can be used to drill down into * objects, as in {@code "field1.fieldxy" }. * * @param field * In case of a POJO, Scala case class, or Tuple type, the * name of the (public) field on which to perform the aggregation. * Additionally, a dot can be used to drill down into nested * objects, as in {@code "field1.fieldxy" }. * Furthermore "*" can be specified in case of a basic type * (which is considered as having only one field). * @return The transformed DataStream. */ public SingleOutputStreamOperator max(String field) { return aggregate(new ComparableAggregator<>(field, getType(), AggregationFunction.AggregationType.MAX, false, getExecutionConfig())); } /** * Applies an aggregation that gives the current minimum element of the * data stream by the given field expression by the given key. An * independent aggregate is kept per key. A field expression is either the * name of a public field or a getter method with parentheses of the * {@link DataStream}'s underlying type. A dot can be used to drill down into * objects, as in {@code "field1.fieldxy" }. * * @param field * In case of a POJO, Scala case class, or Tuple type, the * name of the (public) field on which to perform the aggregation. * Additionally, a dot can be used to drill down into nested * objects, as in {@code "field1.fieldxy" }. * Furthermore "*" can be specified in case of a basic type * (which is considered as having only one field). * @param first * If True then in case of field equality the first object will * be returned * @return The transformed DataStream. */ @SuppressWarnings({ "rawtypes", "unchecked" }) public SingleOutputStreamOperator minBy(String field, boolean first) { return aggregate(new ComparableAggregator(field, getType(), AggregationFunction.AggregationType.MINBY, first, getExecutionConfig())); } /** * Applies an aggregation that gives the current maximum element of the * data stream by the given field expression by the given key. An * independent aggregate is kept per key. A field expression is either the * name of a public field or a getter method with parentheses of the * {@link DataStream}'s underlying type. A dot can be used to drill down into * objects, as in {@code "field1.fieldxy" }. * * @param field * In case of a POJO, Scala case class, or Tuple type, the * name of the (public) field on which to perform the aggregation. * Additionally, a dot can be used to drill down into nested * objects, as in {@code "field1.fieldxy" }. * Furthermore "*" can be specified in case of a basic type * (which is considered as having only one field). * @param first * If True then in case of field equality the first object will * be returned * @return The transformed DataStream. */ public SingleOutputStreamOperator maxBy(String field, boolean first) { return aggregate(new ComparableAggregator<>(field, getType(), AggregationFunction.AggregationType.MAXBY, first, getExecutionConfig())); } /** * Applies an aggregation that gives the current element with the * minimum value at the given position by the given key. An independent * aggregate is kept per key. If more elements have the minimum value at the * given position, the operator returns the first one by default. * * @param positionToMinBy * The field position in the data points to minimize. This is applicable to * Tuple types, Scala case classes, and primitive types (which is considered * as having one field). * @return The transformed DataStream. */ public SingleOutputStreamOperator minBy(int positionToMinBy) { return this.minBy(positionToMinBy, true); } /** * Applies an aggregation that gives the current element with the * minimum value at the given position by the given key. An independent * aggregate is kept per key. If more elements have the minimum value at the * given position, the operator returns the first one by default. * * @param positionToMinBy * In case of a POJO, Scala case class, or Tuple type, the * name of the (public) field on which to perform the aggregation. * Additionally, a dot can be used to drill down into nested * objects, as in {@code "field1.fieldxy" }. * Furthermore "*" can be specified in case of a basic type * (which is considered as having only one field). * @return The transformed DataStream. */ public SingleOutputStreamOperator minBy(String positionToMinBy) { return this.minBy(positionToMinBy, true); } /** * Applies an aggregation that gives the current element with the * minimum value at the given position by the given key. An independent * aggregate is kept per key. If more elements have the minimum value at the * given position, the operator returns either the first or last one, * depending on the parameter set. * * @param positionToMinBy * The field position in the data points to minimize. This is applicable to * Tuple types, Scala case classes, and primitive types (which is considered * as having one field). * @param first * If true, then the operator return the first element with the * minimal value, otherwise returns the last * @return The transformed DataStream. */ public SingleOutputStreamOperator minBy(int positionToMinBy, boolean first) { return aggregate(new ComparableAggregator(positionToMinBy, getType(), AggregationFunction.AggregationType.MINBY, first, getExecutionConfig())); } /** * Applies an aggregation that gives the current element with the * maximum value at the given position by the given key. An independent * aggregate is kept per key. If more elements have the maximum value at the * given position, the operator returns the first one by default. * * @param positionToMaxBy * The field position in the data points to minimize. This is applicable to * Tuple types, Scala case classes, and primitive types (which is considered * as having one field). * @return The transformed DataStream. */ public SingleOutputStreamOperator maxBy(int positionToMaxBy) { return this.maxBy(positionToMaxBy, true); } /** * Applies an aggregation that gives the current element with the * maximum value at the given position by the given key. An independent * aggregate is kept per key. If more elements have the maximum value at the * given position, the operator returns the first one by default. * * @param positionToMaxBy * In case of a POJO, Scala case class, or Tuple type, the * name of the (public) field on which to perform the aggregation. * Additionally, a dot can be used to drill down into nested * objects, as in {@code "field1.fieldxy" }. * Furthermore "*" can be specified in case of a basic type * (which is considered as having only one field). * @return The transformed DataStream. */ public SingleOutputStreamOperator maxBy(String positionToMaxBy) { return this.maxBy(positionToMaxBy, true); } /** * Applies an aggregation that gives the current element with the * maximum value at the given position by the given key. An independent * aggregate is kept per key. If more elements have the maximum value at the * given position, the operator returns either the first or last one, * depending on the parameter set. * * @param positionToMaxBy * The field position in the data points to minimize. This is applicable to * Tuple types, Scala case classes, and primitive types (which is considered * as having one field). * @param first * If true, then the operator return the first element with the * maximum value, otherwise returns the last * @return The transformed DataStream. */ public SingleOutputStreamOperator maxBy(int positionToMaxBy, boolean first) { return aggregate(new ComparableAggregator<>(positionToMaxBy, getType(), AggregationFunction.AggregationType.MAXBY, first, getExecutionConfig())); } protected SingleOutputStreamOperator aggregate(AggregationFunction aggregate) { StreamGroupedReduce operator = new StreamGroupedReduce( clean(aggregate), getType().createSerializer(getExecutionConfig())); return transform("Keyed Aggregation", getType(), operator); } /** * Publishes the keyed stream as queryable ValueState instance. * * @param queryableStateName Name under which to the publish the queryable state instance * @return Queryable state instance */ @PublicEvolving public QueryableStateStream asQueryableState(String queryableStateName) { ValueStateDescriptor valueStateDescriptor = new ValueStateDescriptor( UUID.randomUUID().toString(), getType()); return asQueryableState(queryableStateName, valueStateDescriptor); } /** * Publishes the keyed stream as a queryable ValueState instance. * * @param queryableStateName Name under which to the publish the queryable state instance * @param stateDescriptor State descriptor to create state instance from * @return Queryable state instance */ @PublicEvolving public QueryableStateStream asQueryableState( String queryableStateName, ValueStateDescriptor stateDescriptor) { transform("Queryable state: " + queryableStateName, getType(), new QueryableValueStateOperator<>(queryableStateName, stateDescriptor)); stateDescriptor.initializeSerializerUnlessSet(getExecutionConfig()); return new QueryableStateStream<>( queryableStateName, stateDescriptor, getKeyType().createSerializer(getExecutionConfig())); } /** * Publishes the keyed stream as a queryable FoldingState instance. * * @param queryableStateName Name under which to the publish the queryable state instance * @param stateDescriptor State descriptor to create state instance from * @return Queryable state instance * * @deprecated will be removed in a future version */ @PublicEvolving @Deprecated public QueryableStateStream asQueryableState( String queryableStateName, FoldingStateDescriptor stateDescriptor) { transform("Queryable state: " + queryableStateName, getType(), new QueryableAppendingStateOperator<>(queryableStateName, stateDescriptor)); stateDescriptor.initializeSerializerUnlessSet(getExecutionConfig()); return new QueryableStateStream<>( queryableStateName, stateDescriptor, getKeyType().createSerializer(getExecutionConfig())); } /** * Publishes the keyed stream as a queryable ReducingState instance. * * @param queryableStateName Name under which to the publish the queryable state instance * @param stateDescriptor State descriptor to create state instance from * @return Queryable state instance */ @PublicEvolving public QueryableStateStream asQueryableState( String queryableStateName, ReducingStateDescriptor stateDescriptor) { transform("Queryable state: " + queryableStateName, getType(), new QueryableAppendingStateOperator<>(queryableStateName, stateDescriptor)); stateDescriptor.initializeSerializerUnlessSet(getExecutionConfig()); return new QueryableStateStream<>( queryableStateName, stateDescriptor, getKeyType().createSerializer(getExecutionConfig())); } }





© 2015 - 2024 Weber Informatics LLC | Privacy Policy