From 029d578e098e2b02281d5c320da55c06e239b2b9 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Tue, 11 Feb 2020 08:25:12 +0100 Subject: [PATCH 001/885] [FLINK-16033][table-api] Introduced Java Table API Expression DSL --- .../apache/flink/table/api/ApiExpression.java | 66 + .../apache/flink/table/api/Expressions.java | 549 +++++++ .../apache/flink/table/api/GroupWindow.java | 5 +- .../flink/table/api/SessionWithGap.java | 3 +- .../flink/table/api/SessionWithGapOnTime.java | 5 +- .../api/SessionWithGapOnTimeWithAlias.java | 3 +- .../apache/flink/table/api/SlideWithSize.java | 3 +- .../table/api/SlideWithSizeAndSlide.java | 5 +- .../api/SlideWithSizeAndSlideOnTime.java | 13 +- .../SlideWithSizeAndSlideOnTimeWithAlias.java | 13 +- .../flink/table/api/TumbleWithSize.java | 3 +- .../flink/table/api/TumbleWithSizeOnTime.java | 5 +- .../api/TumbleWithSizeOnTimeWithAlias.java | 3 +- .../table/api/internal/BaseExpressions.java | 1284 +++++++++++++++++ .../flink/table/api/internal/TableImpl.java | 37 +- .../table/expressions/ApiExpressionUtils.java | 39 +- .../expressions/LookupCallExpression.java | 3 +- .../expressions/UnresolvedCallExpression.java | 7 +- .../resolver/ExpressionResolver.java | 2 + .../resolver/LookupCallResolver.java | 12 + .../rules/OverWindowResolverRule.java | 2 +- .../resolver/rules/ResolverRules.java | 6 + .../rules/UnwrapApiExpressionRule.java | 40 + .../flink/table/typeutils/FieldInfoUtils.java | 14 +- .../flink/table/api/expressionDsl.scala | 1045 ++------------ .../PlannerExpressionConverter.scala | 10 +- .../PlannerExpressionConverter.scala | 8 +- 27 files changed, 2230 insertions(+), 955 deletions(-) create mode 100644 flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/ApiExpression.java create mode 100644 flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java create mode 100644 flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java create mode 100644 flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/rules/UnwrapApiExpressionRule.java diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/ApiExpression.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/ApiExpression.java new file mode 100644 index 0000000000..0e2027d075 --- /dev/null +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/ApiExpression.java @@ -0,0 +1,66 @@ +/* + * 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.table.api; + +import org.apache.flink.table.api.internal.BaseExpressions; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.ExpressionVisitor; + +import java.util.List; + +/** + * Java API class that gives access to expression operations. + * + * @see BaseExpressions + */ +public final class ApiExpression extends BaseExpressions implements Expression { + private final Expression wrappedExpression; + + @Override + public String asSummaryString() { + return wrappedExpression.asSummaryString(); + } + + ApiExpression(Expression wrappedExpression) { + if (wrappedExpression instanceof ApiExpression) { + throw new UnsupportedOperationException("This is a bug. Please file an issue."); + } + this.wrappedExpression = wrappedExpression; + } + + @Override + public Expression toExpr() { + return wrappedExpression; + } + + @Override + protected ApiExpression toApiSpecificExpression(Expression expression) { + return new ApiExpression(expression); + } + + @Override + public List getChildren() { + return wrappedExpression.getChildren(); + } + + @Override + public R accept(ExpressionVisitor visitor) { + return wrappedExpression.accept(visitor); + } +} diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java new file mode 100644 index 0000000000..9e0c10ed15 --- /dev/null +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java @@ -0,0 +1,549 @@ +/* + * 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.table.api; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.table.expressions.ApiExpressionUtils; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.expressions.TimePointUnit; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.flink.table.types.utils.ValueDataTypeConverter; + +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.table.expressions.ApiExpressionUtils.objectToExpression; +import static org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedCall; +import static org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedRef; +import static org.apache.flink.table.expressions.ApiExpressionUtils.valueLiteral; + +/** + * Entry point of the Table API Expression DSL such as: {@code $("myField").plus(10).abs()} + * + *

This class contains static methods for referencing table columns, creating literals, + * and building more complex {@link Expression} chains. {@link ApiExpression ApiExpressions} are + * pure API entities that are further translated into {@link ResolvedExpression ResolvedExpressions} + * under the hood. + * + *

For fluent definition of expressions and easier readability, we recommend to add a + * star import to the methods of this class: + * + *

+ * import static org.apache.flink.table.api.Expressions.*;
+ * 
+ * + *

Check the documentation for more programming language specific APIs, for example, by using + * Scala implicits. + */ +@PublicEvolving +public final class Expressions { + /** + * Creates an unresolved reference to a table's field. + * + *

Example: + *

{@code
+	 *   tab.select($("key"), $("value"))
+	 * }
+	 * 
+ */ + //CHECKSTYLE.OFF: MethodName + public static ApiExpression $(String name) { + return new ApiExpression(unresolvedRef(name)); + } + //CHECKSTYLE.ON: MethodName + + /** + * Creates a SQL literal. + * + *

The data type is derived from the object's class and its value. + * + *

For example: + *

    + *
  • {@code lit(12)} leads to {@code INT}
  • + *
  • {@code lit("abc")} leads to {@code CHAR(3)}
  • + *
  • {@code lit(new BigDecimal("123.45"))} leads to {@code DECIMAL(5, 2)}
  • + *
+ * + *

See {@link ValueDataTypeConverter} for a list of supported literal values. + */ + public static ApiExpression lit(Object v) { + return new ApiExpression(valueLiteral(v)); + } + + /** + * Creates a SQL literal of a given {@link DataType}. + * + *

The method {@link #lit(Object)} is preferred as it extracts the {@link DataType} automatically. + * Use this method only when necessary. The class of {@code v} must be supported according to the + * {@link org.apache.flink.table.types.logical.LogicalType#supportsInputConversion(Class)}. + */ + public static ApiExpression lit(Object v, DataType dataType) { + return new ApiExpression(valueLiteral(v, dataType)); + } + + /** + * Indicates a range from 'start' to 'end', which can be used in columns + * selection. + * + *

Example: + *

{@code
+	 * Table table = ...
+	 * table.select(withColumns(range(b, c)))
+	 * }
+ * + * @see #withColumns(Object, Object...) + * @see #withoutColumns(Object, Object...) + */ + public static ApiExpression range(String start, String end) { + return apiCall(BuiltInFunctionDefinitions.RANGE_TO, unresolvedRef(start), unresolvedRef(end)); + } + + /** + * Indicates an index based range, which can be used in columns selection. + * + *

Example: + *

{@code
+	 * Table table = ...
+	 * table.select(withColumns(range(3, 4)))
+	 * }
+ * + * @see #withColumns(Object, Object...) + * @see #withoutColumns(Object, Object...) + */ + public static ApiExpression range(int start, int end) { + return apiCall(BuiltInFunctionDefinitions.RANGE_TO, valueLiteral(start), valueLiteral(end)); + } + + /** + * Boolean AND in three-valued logic. + */ + public static ApiExpression and(Object predicate0, Object predicate1, Object... predicates) { + return apiCallAtLeastTwoArgument(BuiltInFunctionDefinitions.AND, predicate0, predicate1, predicates); + } + + /** + * Boolean OR in three-valued logic. + */ + public static ApiExpression or(Object predicate0, Object predicate1, Object... predicates) { + return apiCallAtLeastTwoArgument(BuiltInFunctionDefinitions.OR, predicate0, predicate1, predicates); + } + + /** + * Offset constant to be used in the {@code preceding} clause of unbounded {@code Over} windows. Use this + * constant for a time interval. Unbounded over windows start with the first row of a partition. + */ + public static final ApiExpression UNBOUNDED_ROW = apiCall(BuiltInFunctionDefinitions.UNBOUNDED_ROW); + + /** + * Offset constant to be used in the {@code preceding} clause of unbounded {@link Over} windows. Use this + * constant for a row-count interval. Unbounded over windows start with the first row of a + * partition. + */ + public static final ApiExpression UNBOUNDED_RANGE = apiCall(BuiltInFunctionDefinitions.UNBOUNDED_RANGE); + + /** + * Offset constant to be used in the {@code following} clause of {@link Over} windows. Use this for setting + * the upper bound of the window to the current row. + */ + public static final ApiExpression CURRENT_ROW = apiCall(BuiltInFunctionDefinitions.CURRENT_ROW); + + /** + * Offset constant to be used in the {@code following} clause of {@link Over} windows. Use this for setting + * the upper bound of the window to the sort key of the current row, i.e., all rows with the same + * sort key as the current row are included in the window. + */ + public static final ApiExpression CURRENT_RANGE = apiCall(BuiltInFunctionDefinitions.CURRENT_RANGE); + + /** + * Returns the current SQL date in UTC time zone. + */ + public static ApiExpression currentDate() { + return apiCall(BuiltInFunctionDefinitions.CURRENT_DATE); + } + + /** + * Returns the current SQL time in UTC time zone. + */ + public static ApiExpression currentTime() { + return apiCall(BuiltInFunctionDefinitions.CURRENT_TIME); + } + + /** + * Returns the current SQL timestamp in UTC time zone. + */ + public static ApiExpression currentTimestamp() { + return apiCall(BuiltInFunctionDefinitions.CURRENT_TIMESTAMP); + } + + /** + * Returns the current SQL time in local time zone. + */ + public static ApiExpression localTime() { + return apiCall(BuiltInFunctionDefinitions.LOCAL_TIME); + } + + /** + * Returns the current SQL timestamp in local time zone. + */ + public static ApiExpression localTimestamp() { + return apiCall(BuiltInFunctionDefinitions.LOCAL_TIMESTAMP); + } + + /** + * Determines whether two anchored time intervals overlap. Time point and temporal are + * transformed into a range defined by two time points (start, end). The function + * evaluates leftEnd >= rightStart && rightEnd >= leftStart. + * + *

It evaluates: leftEnd >= rightStart && rightEnd >= leftStart + * + *

e.g. + *

{@code
+	 * temporalOverlaps(
+	 *      lit("2:55:00").toTime(),
+	 *      interval(Duration.ofHour(1)),
+	 *      lit("3:30:00").toTime(),
+	 *      interval(Duration.ofHour(2))
+	 * }
+ * leads to true + */ + public static ApiExpression temporalOverlaps( + Object leftTimePoint, + Object leftTemporal, + Object rightTimePoint, + Object rightTemporal) { + return apiCall( + BuiltInFunctionDefinitions.TEMPORAL_OVERLAPS, + leftTimePoint, + leftTemporal, + rightTimePoint, + rightTemporal); + } + + /** + * Formats a timestamp as a string using a specified format. + * The format must be compatible with MySQL's date formatting syntax as used by the + * date_parse function. + * + *

For example {@code dataFormat($("time"), "%Y, %d %M")} results in strings formatted as "2017, 05 May". + * + * @param timestamp The timestamp to format as string. + * @param format The format of the string. + * @return The formatted timestamp as string. + */ + public static ApiExpression dateFormat( + Object timestamp, + Object format) { + return apiCall(BuiltInFunctionDefinitions.DATE_FORMAT, timestamp, format); + } + + /** + * Returns the (signed) number of {@link TimePointUnit} between timePoint1 and timePoint2. + * + *

For example, {@code timestampDiff(TimePointUnit.DAY, lit("2016-06-15").toDate(), lit("2016-06-18").toDate()} + * leads to 3. + * + * @param timePointUnit The unit to compute diff. + * @param timePoint1 The first point in time. + * @param timePoint2 The second point in time. + * @return The number of intervals as integer value. + */ + public static ApiExpression timestampDiff( + TimePointUnit timePointUnit, + Object timePoint1, + Object timePoint2) { + return apiCall(BuiltInFunctionDefinitions.TIMESTAMP_DIFF, valueLiteral(timePointUnit), timePoint1, timePoint2); + } + + /** + * Creates an array of literals. + */ + public static ApiExpression array(Object head, Object... tail) { + return apiCallAtLeastOneArgument(BuiltInFunctionDefinitions.ARRAY, head, tail); + } + + /** + * Creates a row of expressions. + */ + public static ApiExpression row(Object head, Object... tail) { + return apiCallAtLeastOneArgument(BuiltInFunctionDefinitions.ROW, head, tail); + } + + /** + * Creates a map of expressions. + * + *

{@code
+	 *  table.select(
+	 *      map(
+	 *          "key1", 1,
+	 *          "key2", 2,
+	 *          "key3", 3
+	 *      ))
+	 * }
+ * + *

Note keys and values should have the same types for all entries. + */ + public static ApiExpression map(Object key, Object value, Object... tail) { + return apiCallAtLeastTwoArgument(BuiltInFunctionDefinitions.MAP, key, value, tail); + } + + /** + * Creates an interval of rows. + * + * @see Table#window(GroupWindow) + * @see Table#window(OverWindow...) + */ + public static ApiExpression rowInterval(Long rows) { + return new ApiExpression(valueLiteral(rows)); + } + + /** + * Returns a value that is closer than any other value to pi. + */ + public static ApiExpression pi() { + return apiCall(BuiltInFunctionDefinitions.PI); + } + + /** + * Returns a value that is closer than any other value to e. + */ + public static ApiExpression e() { + return apiCall(BuiltInFunctionDefinitions.E); + } + + /** + * Returns a pseudorandom double value between 0.0 (inclusive) and 1.0 (exclusive). + */ + public static ApiExpression rand() { + return apiCall(BuiltInFunctionDefinitions.RAND); + } + + /** + * Returns a pseudorandom double value between 0.0 (inclusive) and 1.0 (exclusive) with a + * initial seed. Two rand() functions will return identical sequences of numbers if they + * have same initial seed. + */ + public static ApiExpression rand(Object seed) { + return apiCall(BuiltInFunctionDefinitions.RAND, objectToExpression(seed)); + } + + /** + * Returns a pseudorandom integer value between 0.0 (inclusive) and the specified + * value (exclusive). + */ + public static ApiExpression randInteger(Object bound) { + return apiCall(BuiltInFunctionDefinitions.RAND_INTEGER, objectToExpression(bound)); + } + + /** + * Returns a pseudorandom integer value between 0.0 (inclusive) and the specified value + * (exclusive) with a initial seed. Two randInteger() functions will return identical sequences + * of numbers if they have same initial seed and same bound. + */ + public static ApiExpression randInteger(Object seed, Object bound) { + return apiCall(BuiltInFunctionDefinitions.RAND_INTEGER, objectToExpression(seed), objectToExpression(bound)); + } + + /** + * Returns the string that results from concatenating the arguments. + * Returns NULL if any argument is NULL. + */ + public static ApiExpression concat(Object string, Object... strings) { + return apiCallAtLeastOneArgument(BuiltInFunctionDefinitions.CONCAT, string, strings); + } + + /** + * Calculates the arc tangent of a given coordinate. + */ + public static ApiExpression atan2(Object y, Object x) { + return apiCallAtLeastOneArgument(BuiltInFunctionDefinitions.ATAN2, y, x); + } + + /** + * Returns negative numeric. + */ + public static ApiExpression negative(Object v) { + return apiCall(BuiltInFunctionDefinitions.MINUS_PREFIX, v); + } + + /** + * Returns the string that results from concatenating the arguments and separator. + * Returns NULL If the separator is NULL. + * + *

Note: this function does not skip empty strings. However, it does skip any NULL + * values after the separator argument. + */ + public static ApiExpression concatWs(Object separator, Object string, Object... strings) { + return apiCallAtLeastTwoArgument(BuiltInFunctionDefinitions.CONCAT_WS, separator, string, strings); + } + + /** + * Returns an UUID (Universally Unique Identifier) string (e.g., + * "3d3c68f7-f608-473f-b60c-b0c44ad4cc4e") according to RFC 4122 type 4 (pseudo randomly + * generated) UUID. The UUID is generated using a cryptographically strong pseudo random number + * generator. + */ + public static ApiExpression uuid() { + return apiCall(BuiltInFunctionDefinitions.UUID); + } + + /** + * Returns a null literal value of a given data type. + * + *

e.g. {@code nullOf(DataTypes.INT())} + */ + public static ApiExpression nullOf(DataType dataType) { + return new ApiExpression(valueLiteral(null, dataType)); + } + + /** + * @deprecated This method will be removed in future versions as it uses the old type system. + * It is recommended to use {@link #nullOf(DataType)} instead which uses the new type + * system based on {@link DataTypes}. Please make sure to use either the old or the new + * type system consistently to avoid unintended behavior. See the website + * documentation for more information. + */ + public static ApiExpression nullOf(TypeInformation typeInfo) { + return nullOf(TypeConversions.fromLegacyInfoToDataType(typeInfo)); + } + + /** + * Calculates the logarithm of the given value. + */ + public static ApiExpression log(Object value) { + return apiCall(BuiltInFunctionDefinitions.LOG, value); + } + + /** + * Calculates the logarithm of the given value to the given base. + */ + public static ApiExpression log(Object base, Object value) { + return apiCall(BuiltInFunctionDefinitions.LOG, base, value); + } + + /** + * Ternary conditional operator that decides which of two other expressions should be evaluated + * based on a evaluated boolean condition. + * + *

e.g. ifThenElse($("f0") > 5, "A", "B") leads to "A" + * + * @param condition boolean condition + * @param ifTrue expression to be evaluated if condition holds + * @param ifFalse expression to be evaluated if condition does not hold + */ + public static ApiExpression ifThenElse(Object condition, Object ifTrue, Object ifFalse) { + return apiCall(BuiltInFunctionDefinitions.IF, condition, ifTrue, ifFalse); + } + + /** + * Creates an expression that selects a range of columns. It can be used wherever an array of + * expression is accepted such as function calls, projections, or groupings. + * + *

A range can either be index-based or name-based. Indices start at 1 and boundaries are + * inclusive. + * + *

e.g. withColumns(range("b", "c")) or withoutColumns($("*")) + */ + public static ApiExpression withColumns(Object head, Object... tail) { + return apiCallAtLeastOneArgument(BuiltInFunctionDefinitions.WITH_COLUMNS, head, tail); + } + + /** + * Creates an expression that selects all columns except for the given range of columns. It can + * be used wherever an array of expression is accepted such as function calls, projections, or + * groupings. + * + *

A range can either be index-based or name-based. Indices start at 1 and boundaries are + * inclusive. + * + *

e.g. withoutColumns(range("b", "c")) or withoutColumns($("c")) + */ + public static ApiExpression withoutColumns(Object head, Object... tail) { + return apiCallAtLeastOneArgument(BuiltInFunctionDefinitions.WITHOUT_COLUMNS, head, tail); + } + + /** + * A call to a function that will be looked up in a catalog. There are two kinds of functions: + *

    + *
  • System functions - which are identified with one part names
  • + *
  • Catalog functions - which are identified always with three parts names + * (catalog, database, function)
  • + *
+ * + *

Moreover each function can either be a temporary function or permanent one + * (which is stored in an external catalog). + * + *

Based on that two properties the resolution order for looking up a function based on + * the provided {@code functionName} is following: + *

    + *
  • Temporary system function
  • + *
  • System function
  • + *
  • Temporary catalog function
  • + *
  • Catalog function
  • + *
+ * + * @see TableEnvironment#useCatalog(String) + * @see TableEnvironment#useDatabase(String) + * @see TableEnvironment#createTemporaryFunction + * @see TableEnvironment#createTemporarySystemFunction + */ + public static ApiExpression call(String path, Object... params) { + return new ApiExpression(ApiExpressionUtils.lookupCall( + path, + Arrays.stream(params).map(ApiExpressionUtils::objectToExpression).toArray(Expression[]::new))); + } + + private static ApiExpression apiCall(FunctionDefinition functionDefinition, Object... args) { + List arguments = + Stream.of(args) + .map(ApiExpressionUtils::objectToExpression) + .collect(Collectors.toList()); + return new ApiExpression(unresolvedCall(functionDefinition, arguments)); + } + + private static ApiExpression apiCallAtLeastOneArgument(FunctionDefinition functionDefinition, + Object arg0, + Object... args) { + List arguments = Stream.concat( + Stream.of(arg0), + Stream.of(args) + ).map(ApiExpressionUtils::objectToExpression) + .collect(Collectors.toList()); + return new ApiExpression(unresolvedCall(functionDefinition, arguments)); + } + + private static ApiExpression apiCallAtLeastTwoArgument( + FunctionDefinition functionDefinition, + Object arg0, + Object arg1, + Object... args) { + List arguments = Stream.concat( + Stream.of(arg0, arg1), + Stream.of(args) + ).map(ApiExpressionUtils::objectToExpression) + .collect(Collectors.toList()); + return new ApiExpression(unresolvedCall(functionDefinition, arguments)); + } +} diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/GroupWindow.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/GroupWindow.java index 4601e75ecb..36cd8554cf 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/GroupWindow.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/GroupWindow.java @@ -19,6 +19,7 @@ package org.apache.flink.table.api; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.table.expressions.ApiExpressionUtils; import org.apache.flink.table.expressions.Expression; /** @@ -41,8 +42,8 @@ public abstract class GroupWindow { private final Expression timeField; GroupWindow(Expression alias, Expression timeField) { - this.alias = alias; - this.timeField = timeField; + this.alias = ApiExpressionUtils.unwrapFromApi(alias); + this.timeField = ApiExpressionUtils.unwrapFromApi(timeField); } public Expression getAlias() { diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/SessionWithGap.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/SessionWithGap.java index 6e646f7bd9..0d9ee6ded3 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/SessionWithGap.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/SessionWithGap.java @@ -19,6 +19,7 @@ package org.apache.flink.table.api; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.table.expressions.ApiExpressionUtils; import org.apache.flink.table.expressions.Expression; import org.apache.flink.table.expressions.ExpressionParser; @@ -36,7 +37,7 @@ public final class SessionWithGap { private final Expression gap; SessionWithGap(Expression gap) { - this.gap = gap; + this.gap = ApiExpressionUtils.unwrapFromApi(gap); } /** diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/SessionWithGapOnTime.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/SessionWithGapOnTime.java index 53c9c4fc51..ef63ddea42 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/SessionWithGapOnTime.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/SessionWithGapOnTime.java @@ -19,6 +19,7 @@ package org.apache.flink.table.api; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.table.expressions.ApiExpressionUtils; import org.apache.flink.table.expressions.Expression; import org.apache.flink.table.expressions.ExpressionParser; @@ -32,8 +33,8 @@ public final class SessionWithGapOnTime { private final Expression gap; SessionWithGapOnTime(Expression timeField, Expression gap) { - this.timeField = timeField; - this.gap = gap; + this.timeField = ApiExpressionUtils.unwrapFromApi(timeField); + this.gap = ApiExpressionUtils.unwrapFromApi(gap); } /** diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/SessionWithGapOnTimeWithAlias.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/SessionWithGapOnTimeWithAlias.java index 891ea7ca86..b88a93ec65 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/SessionWithGapOnTimeWithAlias.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/SessionWithGapOnTimeWithAlias.java @@ -19,6 +19,7 @@ package org.apache.flink.table.api; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.table.expressions.ApiExpressionUtils; import org.apache.flink.table.expressions.Expression; /** @@ -31,7 +32,7 @@ public final class SessionWithGapOnTimeWithAlias extends GroupWindow { SessionWithGapOnTimeWithAlias(Expression alias, Expression timeField, Expression gap) { super(alias, timeField); - this.gap = gap; + this.gap = ApiExpressionUtils.unwrapFromApi(gap); } public Expression getGap() { diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/SlideWithSize.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/SlideWithSize.java index 44470f9b4a..983d07d460 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/SlideWithSize.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/SlideWithSize.java @@ -19,6 +19,7 @@ package org.apache.flink.table.api; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.table.expressions.ApiExpressionUtils; import org.apache.flink.table.expressions.Expression; import org.apache.flink.table.expressions.ExpressionParser; @@ -32,7 +33,7 @@ public final class SlideWithSize { private final Expression size; SlideWithSize(Expression size) { - this.size = size; + this.size = ApiExpressionUtils.unwrapFromApi(size); } /** diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/SlideWithSizeAndSlide.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/SlideWithSizeAndSlide.java index 4d509d1741..d4d83c10dc 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/SlideWithSizeAndSlide.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/SlideWithSizeAndSlide.java @@ -19,6 +19,7 @@ package org.apache.flink.table.api; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.table.expressions.ApiExpressionUtils; import org.apache.flink.table.expressions.Expression; import org.apache.flink.table.expressions.ExpressionParser; @@ -37,8 +38,8 @@ public final class SlideWithSizeAndSlide { private final Expression slide; SlideWithSizeAndSlide(Expression size, Expression slide) { - this.size = size; - this.slide = slide; + this.size = ApiExpressionUtils.unwrapFromApi(size); + this.slide = ApiExpressionUtils.unwrapFromApi(slide); } /** diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/SlideWithSizeAndSlideOnTime.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/SlideWithSizeAndSlideOnTime.java index 604b9cd4d7..c74529afd3 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/SlideWithSizeAndSlideOnTime.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/SlideWithSizeAndSlideOnTime.java @@ -19,6 +19,7 @@ package org.apache.flink.table.api; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.table.expressions.ApiExpressionUtils; import org.apache.flink.table.expressions.Expression; import org.apache.flink.table.expressions.ExpressionParser; @@ -33,12 +34,12 @@ public final class SlideWithSizeAndSlideOnTime { private final Expression slide; SlideWithSizeAndSlideOnTime( - Expression timeField, - Expression size, - Expression slide) { - this.timeField = timeField; - this.size = size; - this.slide = slide; + Expression timeField, + Expression size, + Expression slide) { + this.timeField = ApiExpressionUtils.unwrapFromApi(timeField); + this.size = ApiExpressionUtils.unwrapFromApi(size); + this.slide = ApiExpressionUtils.unwrapFromApi(slide); } /** diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/SlideWithSizeAndSlideOnTimeWithAlias.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/SlideWithSizeAndSlideOnTimeWithAlias.java index 50b3692a69..8b9c692ff3 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/SlideWithSizeAndSlideOnTimeWithAlias.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/SlideWithSizeAndSlideOnTimeWithAlias.java @@ -19,6 +19,7 @@ package org.apache.flink.table.api; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.table.expressions.ApiExpressionUtils; import org.apache.flink.table.expressions.Expression; /** @@ -31,13 +32,13 @@ public final class SlideWithSizeAndSlideOnTimeWithAlias extends GroupWindow { private final Expression slide; SlideWithSizeAndSlideOnTimeWithAlias( - Expression alias, - Expression timeField, - Expression size, - Expression slide) { + Expression alias, + Expression timeField, + Expression size, + Expression slide) { super(alias, timeField); - this.size = size; - this.slide = slide; + this.size = ApiExpressionUtils.unwrapFromApi(size); + this.slide = ApiExpressionUtils.unwrapFromApi(slide); } public Expression getSize() { diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TumbleWithSize.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TumbleWithSize.java index 600e3a1d8d..bbc1825311 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TumbleWithSize.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TumbleWithSize.java @@ -19,6 +19,7 @@ package org.apache.flink.table.api; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.table.expressions.ApiExpressionUtils; import org.apache.flink.table.expressions.Expression; import org.apache.flink.table.expressions.ExpressionParser; @@ -36,7 +37,7 @@ public final class TumbleWithSize { private Expression size; TumbleWithSize(Expression size) { - this.size = size; + this.size = ApiExpressionUtils.unwrapFromApi(size); } /** diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TumbleWithSizeOnTime.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TumbleWithSizeOnTime.java index f635ba5814..07616d1bb7 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TumbleWithSizeOnTime.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TumbleWithSizeOnTime.java @@ -19,6 +19,7 @@ package org.apache.flink.table.api; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.table.expressions.ApiExpressionUtils; import org.apache.flink.table.expressions.Expression; import org.apache.flink.table.expressions.ExpressionParser; @@ -32,8 +33,8 @@ public final class TumbleWithSizeOnTime { private final Expression size; TumbleWithSizeOnTime(Expression time, Expression size) { - this.time = time; - this.size = size; + this.time = ApiExpressionUtils.unwrapFromApi(time); + this.size = ApiExpressionUtils.unwrapFromApi(size); } /** diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TumbleWithSizeOnTimeWithAlias.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TumbleWithSizeOnTimeWithAlias.java index 5180d33f10..4e25a4f53d 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TumbleWithSizeOnTimeWithAlias.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TumbleWithSizeOnTimeWithAlias.java @@ -19,6 +19,7 @@ package org.apache.flink.table.api; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.table.expressions.ApiExpressionUtils; import org.apache.flink.table.expressions.Expression; /** @@ -31,7 +32,7 @@ public final class TumbleWithSizeOnTimeWithAlias extends GroupWindow { TumbleWithSizeOnTimeWithAlias(Expression alias, Expression timeField, Expression size) { super(alias, timeField); - this.size = size; + this.size = ApiExpressionUtils.unwrapFromApi(size); } public Expression getSize() { diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java new file mode 100644 index 0000000000..41cb93ccea --- /dev/null +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java @@ -0,0 +1,1284 @@ +/* + * 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.table.api.internal; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.table.api.Expressions; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.expressions.ApiExpressionUtils; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.TimeIntervalUnit; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.flink.table.types.DataType; + +import java.util.Arrays; +import java.util.stream.Stream; + +import static org.apache.flink.table.expressions.ApiExpressionUtils.MILLIS_PER_DAY; +import static org.apache.flink.table.expressions.ApiExpressionUtils.MILLIS_PER_HOUR; +import static org.apache.flink.table.expressions.ApiExpressionUtils.MILLIS_PER_MINUTE; +import static org.apache.flink.table.expressions.ApiExpressionUtils.MILLIS_PER_SECOND; +import static org.apache.flink.table.expressions.ApiExpressionUtils.objectToExpression; +import static org.apache.flink.table.expressions.ApiExpressionUtils.tableRef; +import static org.apache.flink.table.expressions.ApiExpressionUtils.toMilliInterval; +import static org.apache.flink.table.expressions.ApiExpressionUtils.toMonthInterval; +import static org.apache.flink.table.expressions.ApiExpressionUtils.typeLiteral; +import static org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedCall; +import static org.apache.flink.table.expressions.ApiExpressionUtils.valueLiteral; +import static org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType; + +//CHECKSTYLE.OFF: AvoidStarImport|ImportOrder +import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.*; +//CHECKSTYLE.ON: AvoidStarImport|ImportOrder + +/** + * These are Java and Scala common operations that can be used to construct an {@link Expression} AST for + * expression operations. + * + * @param The accepted type of input expressions, it is {@code Expression} for Scala and + * {@code Object} for Java. Generally the expression DSL works on expressions, the + * reason why Java accepts Object is to remove cumbersome call to {@code lit()} for + * literals. Scala alleviates this problem via implicit conversions. + * @param The produced type of the DSL. It is {@code ApiExpression} for Java and {@code Expression} + * for Scala. In Scala the infix operations are included via implicit conversions. In Java + * we introduced a wrapper that enables the operations without pulling them through the whole stack. + */ +@PublicEvolving +public abstract class BaseExpressions { + protected abstract Expression toExpr(); + + protected abstract OutType toApiSpecificExpression(Expression expression); + + /** + * Specifies a name for an expression i.e. a field. + * + * @param name name for one field + * @param extraNames additional names if the expression expands to multiple fields + */ + public OutType as(String name, String... extraNames) { + return toApiSpecificExpression(ApiExpressionUtils.unresolvedCall( + BuiltInFunctionDefinitions.AS, + Stream.concat( + Stream.of(toExpr(), ApiExpressionUtils.valueLiteral(name)), + Stream.of(extraNames).map(ApiExpressionUtils::valueLiteral) + ).toArray(Expression[]::new))); + } + + /** + * Boolean AND in three-valued logic. This is an infix notation. See also + * {@link Expressions#and(Object, Object, Object...)} for prefix notation with multiple arguments. + * + * @see Expressions#and(Object, Object, Object...) + */ + public OutType and(InType other) { + return toApiSpecificExpression(unresolvedCall(AND, toExpr(), objectToExpression(other))); + } + + /** + * Boolean OR in three-valued logic. This is an infix notation. See also + * {@link Expressions#or(Object, Object, Object...)} for prefix notation with multiple arguments. + * + * @see Expressions#or(Object, Object, Object...) + */ + public OutType or(InType other) { + return toApiSpecificExpression(unresolvedCall(OR, toExpr(), objectToExpression(other))); + } + + /** + * Greater than. + */ + public OutType isGreater(InType other) { + return toApiSpecificExpression(unresolvedCall(GREATER_THAN, toExpr(), objectToExpression(other))); + } + + /** + * Greater than or equal. + */ + public OutType isGreaterOrEqual(InType other) { + return toApiSpecificExpression(unresolvedCall(GREATER_THAN_OR_EQUAL, toExpr(), objectToExpression(other))); + } + + /** + * Less than. + */ + public OutType isLess(InType other) { + return toApiSpecificExpression(unresolvedCall(LESS_THAN, toExpr(), objectToExpression(other))); + } + + /** + * Less than or equal. + */ + public OutType isLessOrEqual(InType other) { + return toApiSpecificExpression(unresolvedCall(LESS_THAN_OR_EQUAL, toExpr(), objectToExpression(other))); + } + + /** + * Equals. + */ + public OutType isEqual(InType other) { + return toApiSpecificExpression(unresolvedCall(EQUALS, toExpr(), objectToExpression(other))); + } + + /** + * Not equal. + */ + public OutType isNotEqual(InType other) { + return toApiSpecificExpression(unresolvedCall(NOT_EQUALS, toExpr(), objectToExpression(other))); + } + + /** + * Returns left plus right. + */ + public OutType plus(InType other) { + return toApiSpecificExpression(unresolvedCall(PLUS, toExpr(), objectToExpression(other))); + } + + /** + * Returns left minus right. + */ + public OutType minus(InType other) { + return toApiSpecificExpression(unresolvedCall(MINUS, toExpr(), objectToExpression(other))); + } + + /** + * Returns left divided by right. + */ + public OutType dividedBy(InType other) { + return toApiSpecificExpression(unresolvedCall(DIVIDE, toExpr(), objectToExpression(other))); + } + + /** + * Returns left multiplied by right. + */ + public OutType times(InType other) { + return toApiSpecificExpression(unresolvedCall(TIMES, toExpr(), objectToExpression(other))); + } + + /** + * Returns true if the given expression is between lowerBound and upperBound (both inclusive). + * False otherwise. The parameters must be numeric types or identical comparable types. + * + * @param lowerBound numeric or comparable expression + * @param upperBound numeric or comparable expression + */ + public OutType between(InType lowerBound, InType upperBound) { + return toApiSpecificExpression(unresolvedCall( + BETWEEN, + toExpr(), + objectToExpression(lowerBound), + objectToExpression(upperBound))); + } + + /** + * Returns true if the given expression is not between lowerBound and upperBound (both + * inclusive). False otherwise. The parameters must be numeric types or identical + * comparable types. + * + * @param lowerBound numeric or comparable expression + * @param upperBound numeric or comparable expression + */ + public OutType notBetween(InType lowerBound, InType upperBound) { + return toApiSpecificExpression(unresolvedCall( + NOT_BETWEEN, + toExpr(), + objectToExpression(lowerBound), + objectToExpression(upperBound))); + } + + /** + * Ternary conditional operator that decides which of two other expressions should be evaluated + * based on a evaluated boolean condition. + * + *

e.g. lit(42).isGreater(5).then("A", "B") leads to "A" + * + * @param ifTrue expression to be evaluated if condition holds + * @param ifFalse expression to be evaluated if condition does not hold + */ + public OutType then(InType ifTrue, InType ifFalse) { + return toApiSpecificExpression(unresolvedCall( + IF, + toExpr(), + objectToExpression(ifTrue), + objectToExpression(ifFalse))); + } + + /** + * Returns true if the given expression is null. + */ + public OutType isNull() { + return toApiSpecificExpression(unresolvedCall(IS_NULL, toExpr())); + } + + /** + * Returns true if the given expression is not null. + */ + public OutType isNotNull() { + return toApiSpecificExpression(unresolvedCall(IS_NOT_NULL, toExpr())); + } + + /** + * Returns true if given boolean expression is true. False otherwise (for null and false). + */ + public OutType isTrue() { + return toApiSpecificExpression(unresolvedCall(IS_TRUE, toExpr())); + } + + /** + * Returns true if given boolean expression is false. False otherwise (for null and true). + */ + public OutType isFalse() { + return toApiSpecificExpression(unresolvedCall(IS_FALSE, toExpr())); + } + + /** + * Returns true if given boolean expression is not true (for null and false). False otherwise. + */ + public OutType isNotTrue() { + return toApiSpecificExpression(unresolvedCall(IS_NOT_TRUE, toExpr())); + } + + /** + * Returns true if given boolean expression is not false (for null and true). False otherwise. + */ + public OutType isNotFalse() { + return toApiSpecificExpression(unresolvedCall(IS_NOT_FALSE, toExpr())); + } + + /** + * Similar to a SQL distinct aggregation clause such as COUNT(DISTINCT a), declares that an + * aggregation function is only applied on distinct input values. + * + *

For example: + *

+	 * {@code
+	 * orders
+	 *  .groupBy($("a"))
+	 *  .select($("a"), $("b").sum().distinct().as("d"))
+	 * }
+	 * 
+ */ + public OutType distinct() { + return toApiSpecificExpression(unresolvedCall(DISTINCT, toExpr())); + } + + /** + * Returns the sum of the numeric field across all input values. + * If all values are null, null is returned. + */ + public OutType sum() { + return toApiSpecificExpression(unresolvedCall(SUM, toExpr())); + } + + /** + * Returns the sum of the numeric field across all input values. + * If all values are null, 0 is returned. + */ + public OutType sum0() { + return toApiSpecificExpression(unresolvedCall(SUM0, toExpr())); + } + + /** + * Returns the minimum value of field across all input values. + */ + public OutType min() { + return toApiSpecificExpression(unresolvedCall(MIN, toExpr())); + } + + /** + * Returns the maximum value of field across all input values. + */ + public OutType max() { + return toApiSpecificExpression(unresolvedCall(MAX, toExpr())); + } + + /** + * Returns the number of input rows for which the field is not null. + */ + public OutType count() { + return toApiSpecificExpression(unresolvedCall(COUNT, toExpr())); + } + + /** + * Returns the average (arithmetic mean) of the numeric field across all input values. + */ + public OutType avg() { + return toApiSpecificExpression(unresolvedCall(AVG, toExpr())); + } + + /** + * Returns the population standard deviation of an expression (the square root of varPop()). + */ + public OutType stddevPop() { + return toApiSpecificExpression(unresolvedCall(STDDEV_POP, toExpr())); + } + + /** + * Returns the sample standard deviation of an expression (the square root of varSamp()). + */ + public OutType stddevSamp() { + return toApiSpecificExpression(unresolvedCall(STDDEV_SAMP, toExpr())); + } + + /** + * Returns the population standard variance of an expression. + */ + public OutType varPop() { + return toApiSpecificExpression(unresolvedCall(VAR_POP, toExpr())); + } + + /** + * Returns the sample variance of a given expression. + */ + public OutType varSamp() { + return toApiSpecificExpression(unresolvedCall(VAR_SAMP, toExpr())); + } + + /** + * Returns multiset aggregate of a given expression. + */ + public OutType collect() { + return toApiSpecificExpression(unresolvedCall(COLLECT, toExpr())); + } + + /** + * Converts a value to a given data type. + * + *

e.g. "42".cast(DataTypes.INT()) leads to 42. + */ + public OutType cast(DataType toType) { + return toApiSpecificExpression(unresolvedCall(CAST, toExpr(), typeLiteral(toType))); + } + + /** + * @deprecated This method will be removed in future versions as it uses the old type system. It + * is recommended to use {@link #cast(DataType)} instead which uses the new type system + * based on {@link org.apache.flink.table.api.DataTypes}. Please make sure to use either the old + * or the new type system consistently to avoid unintended behavior. See the website documentation + * for more information. + */ + @Deprecated + public OutType cast(TypeInformation toType) { + return toApiSpecificExpression(unresolvedCall(CAST, toExpr(), typeLiteral(fromLegacyInfoToDataType(toType)))); + } + + /** + * Specifies ascending order of an expression i.e. a field for orderBy unresolvedCall. + */ + public OutType asc() { + return toApiSpecificExpression(unresolvedCall(ORDER_ASC, toExpr())); + } + + /** + * Specifies descending order of an expression i.e. a field for orderBy unresolvedCall. + */ + public OutType desc() { + return toApiSpecificExpression(unresolvedCall(ORDER_DESC, toExpr())); + } + + /** + * Returns true if an expression exists in a given list of expressions. This is a shorthand + * for multiple OR conditions. + * + *

If the testing set contains null, the result will be null if the element can not be found + * and true if it can be found. If the element is null, the result is always null. + * + *

e.g. lit("42").in(1, 2, 3) leads to false. + */ + @SafeVarargs + public final OutType in(InType... elements) { + Expression[] args = Stream.concat( + Stream.of(toExpr()), + Arrays.stream(elements).map(ApiExpressionUtils::objectToExpression)) + .toArray(Expression[]::new); + return toApiSpecificExpression(unresolvedCall(IN, args)); + } + + /** + * Returns true if an expression exists in a given table sub-query. The sub-query table + * must consist of one column. This column must have the same data type as the expression. + * + *

Note: This operation is not supported in a streaming environment yet. + */ + public OutType in(Table table) { + return toApiSpecificExpression(unresolvedCall(IN, toExpr(), tableRef(table.toString(), table))); + } + + /** + * Returns the start time (inclusive) of a window when applied on a window reference. + */ + public OutType start() { + return toApiSpecificExpression(unresolvedCall(WINDOW_START, toExpr())); + } + + /** + * Returns the end time (exclusive) of a window when applied on a window reference. + * + *

e.g. if a window ends at 10:59:59.999 this property will return 11:00:00.000. + */ + public OutType end() { + return toApiSpecificExpression(unresolvedCall(WINDOW_END, toExpr())); + } + + /** + * Calculates the remainder of division the given number by another one. + */ + public OutType mod(InType other) { + return toApiSpecificExpression(unresolvedCall(MOD, toExpr(), objectToExpression(other))); + } + + /** + * Calculates the Euler's number raised to the given power. + */ + public OutType exp() { + return toApiSpecificExpression(unresolvedCall(EXP, toExpr())); + } + + /** + * Calculates the base 10 logarithm of the given value. + */ + public OutType log10() { + return toApiSpecificExpression(unresolvedCall(LOG10, toExpr())); + } + + /** + * Calculates the base 2 logarithm of the given value. + */ + public OutType log2() { + return toApiSpecificExpression(unresolvedCall(LOG2, toExpr())); + } + + /** + * Calculates the natural logarithm of the given value. + */ + public OutType ln() { + return toApiSpecificExpression(unresolvedCall(LN, toExpr())); + } + + /** + * Calculates the natural logarithm of the given value. + */ + public OutType log() { + return toApiSpecificExpression(unresolvedCall(LOG, toExpr())); + } + + /** + * Calculates the logarithm of the given value to the given base. + */ + public OutType log(InType base) { + return toApiSpecificExpression(unresolvedCall(LOG, objectToExpression(base), toExpr())); + } + + /** + * Calculates the given number raised to the power of the other value. + */ + public OutType power(InType other) { + return toApiSpecificExpression(unresolvedCall(POWER, toExpr(), objectToExpression(other))); + } + + /** + * Calculates the hyperbolic cosine of a given value. + */ + public OutType cosh() { + return toApiSpecificExpression(unresolvedCall(COSH, toExpr())); + } + + /** + * Calculates the square root of a given value. + */ + public OutType sqrt() { + return toApiSpecificExpression(unresolvedCall(SQRT, toExpr())); + } + + /** + * Calculates the absolute value of given value. + */ + public OutType abs() { + return toApiSpecificExpression(unresolvedCall(ABS, toExpr())); + } + + /** + * Calculates the largest integer less than or equal to a given number. + */ + public OutType floor() { + return toApiSpecificExpression(unresolvedCall(FLOOR, toExpr())); + } + + /** + * Calculates the hyperbolic sine of a given value. + */ + public OutType sinh() { + return toApiSpecificExpression(unresolvedCall(SINH, toExpr())); + } + + /** + * Calculates the smallest integer greater than or equal to a given number. + */ + public OutType ceil() { + return toApiSpecificExpression(unresolvedCall(CEIL, toExpr())); + } + + /** + * Calculates the sine of a given number. + */ + public OutType sin() { + return toApiSpecificExpression(unresolvedCall(SIN, toExpr())); + } + + /** + * Calculates the cosine of a given number. + */ + public OutType cos() { + return toApiSpecificExpression(unresolvedCall(COS, toExpr())); + } + + /** + * Calculates the tangent of a given number. + */ + public OutType tan() { + return toApiSpecificExpression(unresolvedCall(TAN, toExpr())); + } + + /** + * Calculates the cotangent of a given number. + */ + public OutType cot() { + return toApiSpecificExpression(unresolvedCall(COT, toExpr())); + } + + /** + * Calculates the arc sine of a given number. + */ + public OutType asin() { + return toApiSpecificExpression(unresolvedCall(ASIN, toExpr())); + } + + /** + * Calculates the arc cosine of a given number. + */ + public OutType acos() { + return toApiSpecificExpression(unresolvedCall(ACOS, toExpr())); + } + + /** + * Calculates the arc tangent of a given number. + */ + public OutType atan() { + return toApiSpecificExpression(unresolvedCall(ATAN, toExpr())); + } + + /** + * Calculates the hyperbolic tangent of a given number. + */ + public OutType tanh() { + return toApiSpecificExpression(unresolvedCall(TANH, toExpr())); + } + + /** + * Converts numeric from radians to degrees. + */ + public OutType degrees() { + return toApiSpecificExpression(unresolvedCall(DEGREES, toExpr())); + } + + /** + * Converts numeric from degrees to radians. + */ + public OutType radians() { + return toApiSpecificExpression(unresolvedCall(RADIANS, toExpr())); + } + + /** + * Calculates the signum of a given number. + */ + public OutType sign() { + return toApiSpecificExpression(unresolvedCall(SIGN, toExpr())); + } + + /** + * Rounds the given number to integer places right to the decimal point. + */ + public OutType round(InType places) { + return toApiSpecificExpression(unresolvedCall(ROUND, toExpr(), objectToExpression(places))); + } + + /** + * Returns a string representation of an integer numeric value in binary format. Returns null if + * numeric is null. E.g. "4" leads to "100", "12" leads to "1100". + */ + public OutType bin() { + return toApiSpecificExpression(unresolvedCall(BIN, toExpr())); + } + + /** + * Returns a string representation of an integer numeric value or a string in hex format. Returns + * null if numeric or string is null. + * + *

E.g. a numeric 20 leads to "14", a numeric 100 leads to "64", and a string "hello,world" leads + * to "68656c6c6f2c776f726c64". + */ + public OutType hex() { + return toApiSpecificExpression(unresolvedCall(HEX, toExpr())); + } + + /** + * Returns a number of truncated to n decimal places. + * If n is 0,the result has no decimal point or fractional part. + * n can be negative to cause n digits left of the decimal point of the value to become zero. + * E.g. truncate(42.345, 2) to 42.34. + */ + public OutType truncate(InType n) { + return toApiSpecificExpression(unresolvedCall(TRUNCATE, toExpr(), objectToExpression(n))); + } + + /** + * Returns a number of truncated to 0 decimal places. + * E.g. truncate(42.345) to 42.0. + */ + public OutType truncate() { + return toApiSpecificExpression(unresolvedCall(TRUNCATE, toExpr())); + } + + // String operations + + /** + * Creates a substring of the given string at given index for a given length. + * + * @param beginIndex first character of the substring (starting at 1, inclusive) + * @param length number of characters of the substring + */ + public OutType substring(InType beginIndex, InType length) { + return toApiSpecificExpression(unresolvedCall(SUBSTRING, toExpr(), objectToExpression(beginIndex), objectToExpression(length))); + } + + /** + * Creates a substring of the given string beginning at the given index to the end. + * + * @param beginIndex first character of the substring (starting at 1, inclusive) + */ + public OutType substring(InType beginIndex) { + return toApiSpecificExpression(unresolvedCall(SUBSTRING, toExpr(), objectToExpression(beginIndex))); + } + + /** + * Removes leading space characters from the given string. + */ + public OutType trimLeading() { + return toApiSpecificExpression(unresolvedCall( + TRIM, + valueLiteral(true), + valueLiteral(false), + valueLiteral(" "), + toExpr())); + } + + /** + * Removes leading characters from the given string. + * + * @param character string containing the character + */ + public OutType trimLeading(InType character) { + return toApiSpecificExpression(unresolvedCall( + TRIM, + valueLiteral(true), + valueLiteral(false), + objectToExpression(character), + toExpr())); + } + + /** + * Removes trailing space characters from the given string. + */ + public OutType trimTrailing() { + return toApiSpecificExpression(unresolvedCall( + TRIM, + valueLiteral(false), + valueLiteral(true), + valueLiteral(" "), + toExpr())); + } + + /** + * Removes trailing characters from the given string. + * + * @param character string containing the character + */ + public OutType trimTrailing(InType character) { + return toApiSpecificExpression(unresolvedCall( + TRIM, + valueLiteral(false), + valueLiteral(true), + objectToExpression(character), + toExpr())); + } + + /** + * Removes leading and trailing space characters from the given string. + */ + public OutType trim() { + return toApiSpecificExpression(unresolvedCall( + TRIM, + valueLiteral(true), + valueLiteral(true), + valueLiteral(" "), + toExpr())); + } + + /** + * Removes leading and trailing characters from the given string. + * + * @param character string containing the character + */ + public OutType trim(InType character) { + return toApiSpecificExpression(unresolvedCall( + TRIM, + valueLiteral(true), + valueLiteral(true), + objectToExpression(character), + toExpr())); + } + + /** + * Returns a new string which replaces all the occurrences of the search target + * with the replacement string (non-overlapping). + */ + public OutType replace(InType search, InType replacement) { + return toApiSpecificExpression(unresolvedCall(REPLACE, toExpr(), objectToExpression(search), objectToExpression(replacement))); + } + + /** + * Returns the length of a string. + */ + public OutType charLength() { + return toApiSpecificExpression(unresolvedCall(CHAR_LENGTH, toExpr())); + } + + /** + * Returns all of the characters in a string in upper case using the rules of + * the default locale. + */ + public OutType upperCase() { + return toApiSpecificExpression(unresolvedCall(UPPER, toExpr())); + } + + /** + * Returns all of the characters in a string in lower case using the rules of + * the default locale. + */ + public OutType lowerCase() { + return toApiSpecificExpression(unresolvedCall(LOWER, toExpr())); + } + + /** + * Converts the initial letter of each word in a string to uppercase. + * Assumes a string containing only [A-Za-z0-9], everything else is treated as whitespace. + */ + public OutType initCap() { + return toApiSpecificExpression(unresolvedCall(INIT_CAP, toExpr())); + } + + /** + * Returns true, if a string matches the specified LIKE pattern. + * + *

e.g. "Jo_n%" matches all strings that start with "Jo(arbitrary letter)n" + */ + public OutType like(InType pattern) { + return toApiSpecificExpression(unresolvedCall(LIKE, toExpr(), objectToExpression(pattern))); + } + + /** + * Returns true, if a string matches the specified SQL regex pattern. + * + *

e.g. "A+" matches all strings that consist of at least one A + */ + public OutType similar(InType pattern) { + return toApiSpecificExpression(unresolvedCall(SIMILAR, toExpr(), objectToExpression(pattern))); + } + + /** + * Returns the position of string in an other string starting at 1. + * Returns 0 if string could not be found. + * + *

e.g. lit("a").position("bbbbba") leads to 6 + */ + public OutType position(InType haystack) { + return toApiSpecificExpression(unresolvedCall(POSITION, toExpr(), objectToExpression(haystack))); + } + + /** + * Returns a string left-padded with the given pad string to a length of len characters. If + * the string is longer than len, the return value is shortened to len characters. + * + *

e.g. lit("hi").lpad(4, "??") returns "??hi", lit("hi").lpad(1, '??') returns "h" + */ + public OutType lpad(InType len, InType pad) { + return toApiSpecificExpression(unresolvedCall(LPAD, toExpr(), objectToExpression(len), objectToExpression(pad))); + } + + /** + * Returns a string right-padded with the given pad string to a length of len characters. If + * the string is longer than len, the return value is shortened to len characters. + * + *

e.g. lit("hi").rpad(4, "??") returns "hi??", lit("hi").rpad(1, '??') returns "h" + */ + public OutType rpad(InType len, InType pad) { + return toApiSpecificExpression(unresolvedCall(RPAD, toExpr(), objectToExpression(len), objectToExpression(pad))); + } + + /** + * Defines an aggregation to be used for a previously specified over window. + * + *

For example: + * + *

+	 * {@code
+	 * table
+	 *   .window(Over partitionBy 'c orderBy 'rowtime preceding 2.rows following CURRENT_ROW as 'w)
+	 *   .select('c, 'a, 'a.count over 'w, 'a.sum over 'w)
+	 * }
+ */ + public OutType over(InType alias) { + return toApiSpecificExpression(unresolvedCall(OVER, toExpr(), objectToExpression(alias))); + } + + /** + * Replaces a substring of string with a string starting at a position (starting at 1). + * + *

e.g. lit("xxxxxtest").overlay("xxxx", 6) leads to "xxxxxxxxx" + */ + public OutType overlay(InType newString, InType starting) { + return toApiSpecificExpression(unresolvedCall( + OVERLAY, + toExpr(), + objectToExpression(newString), + objectToExpression(starting))); + } + + /** + * Replaces a substring of string with a string starting at a position (starting at 1). + * The length specifies how many characters should be removed. + * + *

e.g. lit("xxxxxtest").overlay("xxxx", 6, 2) leads to "xxxxxxxxxst" + */ + public OutType overlay(InType newString, InType starting, InType length) { + return toApiSpecificExpression(unresolvedCall( + OVERLAY, + toExpr(), + objectToExpression(newString), + objectToExpression(starting), + objectToExpression(length))); + } + + /** + * Returns a string with all substrings that match the regular expression consecutively + * being replaced. + */ + public OutType regexpReplace(InType regex, InType replacement) { + return toApiSpecificExpression(unresolvedCall( + REGEXP_REPLACE, + toExpr(), + objectToExpression(regex), + objectToExpression(replacement))); + } + + /** + * Returns a string extracted with a specified regular expression and a regex match group + * index. + */ + public OutType regexpExtract(InType regex, InType extractIndex) { + return toApiSpecificExpression(unresolvedCall( + REGEXP_EXTRACT, + toExpr(), + objectToExpression(regex), + objectToExpression(extractIndex))); + } + + /** + * Returns a string extracted with a specified regular expression. + */ + public OutType regexpExtract(InType regex) { + return toApiSpecificExpression(unresolvedCall(REGEXP_EXTRACT, toExpr(), objectToExpression(regex))); + } + + /** + * Returns the base string decoded with base64. + */ + public OutType fromBase64() { + return toApiSpecificExpression(unresolvedCall(FROM_BASE64, toExpr())); + } + + /** + * Returns the base64-encoded result of the input string. + */ + public OutType toBase64() { + return toApiSpecificExpression(unresolvedCall(TO_BASE64, toExpr())); + } + + /** + * Returns a string that removes the left whitespaces from the given string. + */ + public OutType ltrim() { + return toApiSpecificExpression(unresolvedCall(LTRIM, toExpr())); + } + + /** + * Returns a string that removes the right whitespaces from the given string. + */ + public OutType rtrim() { + return toApiSpecificExpression(unresolvedCall(RTRIM, toExpr())); + } + + /** + * Returns a string that repeats the base string n times. + */ + public OutType repeat(InType n) { + return toApiSpecificExpression(unresolvedCall(REPEAT, toExpr(), objectToExpression(n))); + } + + // Temporal operations + + /** + * Parses a date string in the form "yyyy-MM-dd" to a SQL Date. + */ + public OutType toDate() { + return toApiSpecificExpression(unresolvedCall( + CAST, + toExpr(), + typeLiteral(fromLegacyInfoToDataType(SqlTimeTypeInfo.DATE)))); + } + + /** + * Parses a time string in the form "HH:mm:ss" to a SQL Time. + */ + public OutType toTime() { + return toApiSpecificExpression(unresolvedCall( + CAST, + toExpr(), + typeLiteral(fromLegacyInfoToDataType(SqlTimeTypeInfo.TIME)))); + } + + /** + * Parses a timestamp string in the form "yyyy-MM-dd HH:mm:ss[.SSS]" to a SQL Timestamp. + */ + public OutType toTimestamp() { + return toApiSpecificExpression(unresolvedCall( + CAST, + toExpr(), + typeLiteral(fromLegacyInfoToDataType(SqlTimeTypeInfo.TIMESTAMP)))); + } + + /** + * Extracts parts of a time point or time interval. Returns the part as a long value. + * + *

e.g. lit("2006-06-05").toDate().extract(DAY) leads to 5 + */ + public OutType extract(TimeIntervalUnit timeIntervalUnit) { + return toApiSpecificExpression(unresolvedCall(EXTRACT, valueLiteral(timeIntervalUnit), toExpr())); + } + + /** + * Rounds down a time point to the given unit. + * + *

e.g. lit("12:44:31").toDate().floor(MINUTE) leads to 12:44:00 + */ + public OutType floor(TimeIntervalUnit timeIntervalUnit) { + return toApiSpecificExpression(unresolvedCall(FLOOR, valueLiteral(timeIntervalUnit), toExpr())); + } + + /** + * Rounds up a time point to the given unit. + * + *

e.g. lit("12:44:31").toDate().ceil(MINUTE) leads to 12:45:00 + */ + public OutType ceil(TimeIntervalUnit timeIntervalUnit) { + return toApiSpecificExpression(unresolvedCall( + CEIL, + valueLiteral(timeIntervalUnit), + toExpr())); + } + + // Advanced type helper functions + + /** + * Accesses the field of a Flink composite type (such as Tuple, POJO, etc.) by name and + * returns it's value. + * + * @param name name of the field (similar to Flink's field expressions) + */ + public OutType get(String name) { + return toApiSpecificExpression(unresolvedCall(GET, toExpr(), valueLiteral(name))); + } + + /** + * Accesses the field of a Flink composite type (such as Tuple, POJO, etc.) by index and + * returns it's value. + * + * @param index position of the field + */ + public OutType get(int index) { + return toApiSpecificExpression(unresolvedCall(GET, toExpr(), valueLiteral(index))); + } + + /** + * Converts a Flink composite type (such as Tuple, POJO, etc.) and all of its direct subtypes + * into a flat representation where every subtype is a separate field. + */ + public OutType flatten() { + return toApiSpecificExpression(unresolvedCall(FLATTEN, toExpr())); + } + + /** + * Accesses the element of an array or map based on a key or an index (starting at 1). + * + * @param index key or position of the element (array index starting at 1) + */ + public OutType at(InType index) { + return toApiSpecificExpression(unresolvedCall(AT, toExpr(), objectToExpression(index))); + } + + /** + * Returns the number of elements of an array or number of entries of a map. + */ + public OutType cardinality() { + return toApiSpecificExpression(unresolvedCall(CARDINALITY, toExpr())); + } + + /** + * Returns the sole element of an array with a single element. Returns null if the array is + * empty. Throws an exception if the array has more than one element. + */ + public OutType element() { + return toApiSpecificExpression(unresolvedCall(ARRAY_ELEMENT, toExpr())); + } + + // Time definition + + /** + * Declares a field as the rowtime attribute for indicating, accessing, and working in + * Flink's event time. + */ + public OutType rowtime() { + return toApiSpecificExpression(unresolvedCall(ROWTIME, toExpr())); + } + + /** + * Declares a field as the proctime attribute for indicating, accessing, and working in + * Flink's processing time. + */ + public OutType proctime() { + return toApiSpecificExpression(unresolvedCall(PROCTIME, toExpr())); + } + + /** + * Creates an interval of the given number of years. + * + *

The produced expression is of type {@code DataTypes.INTERVAL} + */ + public OutType year() { + return toApiSpecificExpression(toMonthInterval(toExpr(), 12)); + } + + /** + * Creates an interval of the given number of years. + */ + public OutType years() { + return year(); + } + + /** + * Creates an interval of the given number of quarters. + */ + public OutType quarter() { + return toApiSpecificExpression(toMonthInterval(toExpr(), 3)); + } + + /** + * Creates an interval of the given number of quarters. + */ + public OutType quarters() { + return quarter(); + } + + /** + * Creates an interval of the given number of months. + */ + public OutType month() { + return toApiSpecificExpression(toMonthInterval(toExpr(), 1)); + } + + /** + * Creates an interval of the given number of months. + */ + public OutType months() { + return month(); + } + + /** + * Creates an interval of the given number of weeks. + */ + public OutType week() { + return toApiSpecificExpression(toMilliInterval(toExpr(), 7 * MILLIS_PER_DAY)); + } + + /** + * Creates an interval of the given number of weeks. + */ + public OutType weeks() { + return week(); + } + + /** + * Creates an interval of the given number of days. + */ + public OutType day() { + return toApiSpecificExpression(toMilliInterval(toExpr(), MILLIS_PER_DAY)); + } + + /** + * Creates an interval of the given number of days. + */ + public OutType days() { + return day(); + } + + /** + * Creates an interval of the given number of hours. + */ + public OutType hour() { + return toApiSpecificExpression(toMilliInterval(toExpr(), MILLIS_PER_HOUR)); + } + + /** + * Creates an interval of the given number of hours. + */ + public OutType hours() { + return hour(); + } + + /** + * Creates an interval of the given number of minutes. + */ + public OutType minute() { + return toApiSpecificExpression(toMilliInterval(toExpr(), MILLIS_PER_MINUTE)); + } + + /** + * Creates an interval of the given number of minutes. + */ + public OutType minutes() { + return minute(); + } + + /** + * Creates an interval of the given number of seconds. + */ + public OutType second() { + return toApiSpecificExpression(toMilliInterval(toExpr(), MILLIS_PER_SECOND)); + } + + /** + * Creates an interval of the given number of seconds. + */ + public OutType seconds() { + return second(); + } + + /** + * Creates an interval of the given number of milliseconds. + */ + public OutType milli() { + return toApiSpecificExpression(toMilliInterval(toExpr(), 1)); + } + + /** + * Creates an interval of the given number of milliseconds. + */ + public OutType millis() { + return milli(); + } + + // Hash functions + + /** + * Returns the MD5 hash of the string argument; null if string is null. + * + * @return string of 32 hexadecimal digits or null + */ + public OutType md5() { + return toApiSpecificExpression(unresolvedCall(MD5, toExpr())); + } + + /** + * Returns the SHA-1 hash of the string argument; null if string is null. + * + * @return string of 40 hexadecimal digits or null + */ + public OutType sha1() { + return toApiSpecificExpression(unresolvedCall(SHA1, toExpr())); + } + + /** + * Returns the SHA-224 hash of the string argument; null if string is null. + * + * @return string of 56 hexadecimal digits or null + */ + public OutType sha224() { + return toApiSpecificExpression(unresolvedCall(SHA224, toExpr())); + } + + /** + * Returns the SHA-256 hash of the string argument; null if string is null. + * + * @return string of 64 hexadecimal digits or null + */ + public OutType sha256() { + return toApiSpecificExpression(unresolvedCall(SHA256, toExpr())); + } + + /** + * Returns the SHA-384 hash of the string argument; null if string is null. + * + * @return string of 96 hexadecimal digits or null + */ + public OutType sha384() { + return toApiSpecificExpression(unresolvedCall(SHA384, toExpr())); + } + + /** + * Returns the SHA-512 hash of the string argument; null if string is null. + * + * @return string of 128 hexadecimal digits or null + */ + public OutType sha512() { + return toApiSpecificExpression(unresolvedCall(SHA512, toExpr())); + } + + /** + * Returns the hash for the given string expression using the SHA-2 family of hash + * functions (SHA-224, SHA-256, SHA-384, or SHA-512). + * + * @param hashLength bit length of the result (either 224, 256, 384, or 512) + * @return string or null if one of the arguments is null. + */ + public OutType sha2(InType hashLength) { + return toApiSpecificExpression(unresolvedCall(SHA2, toExpr(), objectToExpression(hashLength))); + } +} + diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableImpl.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableImpl.java index 2b545abd73..2333ffaf3d 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableImpl.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableImpl.java @@ -119,9 +119,7 @@ public class TableImpl implements Table { @Override public Table select(Expression... fields) { - List expressionsWithResolvedCalls = Arrays.stream(fields) - .map(f -> f.accept(lookupResolver)) - .collect(Collectors.toList()); + List expressionsWithResolvedCalls = preprocessExpressions(fields); CategorizedExpressions extracted = OperationExpressionsUtils.extractAggregationsAndProperties( expressionsWithResolvedCalls ); @@ -464,9 +462,7 @@ public class TableImpl implements Table { } private Table addColumnsOperation(boolean replaceIfExist, List fields) { - List expressionsWithResolvedCalls = fields.stream() - .map(f -> f.accept(lookupResolver)) - .collect(Collectors.toList()); + List expressionsWithResolvedCalls = preprocessExpressions(fields); CategorizedExpressions extracted = OperationExpressionsUtils.extractAggregationsAndProperties( expressionsWithResolvedCalls ); @@ -563,6 +559,16 @@ public class TableImpl implements Table { return new TableImpl(tableEnvironment, operation, operationTreeBuilder, lookupResolver); } + private List preprocessExpressions(List expressions) { + return preprocessExpressions(expressions.toArray(new Expression[0])); + } + + private List preprocessExpressions(Expression[] expressions) { + return Arrays.stream(expressions) + .map(f -> f.accept(lookupResolver)) + .collect(Collectors.toList()); + } + private static final class GroupedTableImpl implements GroupedTable { private final TableImpl table; @@ -582,9 +588,7 @@ public class TableImpl implements Table { @Override public Table select(Expression... fields) { - List expressionsWithResolvedCalls = Arrays.stream(fields) - .map(f -> f.accept(table.lookupResolver)) - .collect(Collectors.toList()); + List expressionsWithResolvedCalls = table.preprocessExpressions(fields); CategorizedExpressions extracted = OperationExpressionsUtils.extractAggregationsAndProperties( expressionsWithResolvedCalls ); @@ -716,7 +720,8 @@ public class TableImpl implements Table { @Override public WindowGroupedTable groupBy(Expression... fields) { - List fieldsWithoutWindow = Arrays.stream(fields) + List fieldsWithoutWindow = table.preprocessExpressions(fields) + .stream() .filter(f -> !window.getAlias().equals(f)) .collect(Collectors.toList()); if (fields.length != fieldsWithoutWindow.size() + 1) { @@ -749,9 +754,7 @@ public class TableImpl implements Table { @Override public Table select(Expression... fields) { - List expressionsWithResolvedCalls = Arrays.stream(fields) - .map(f -> f.accept(table.lookupResolver)) - .collect(Collectors.toList()); + List expressionsWithResolvedCalls = table.preprocessExpressions(fields); CategorizedExpressions extracted = OperationExpressionsUtils.extractAggregationsAndProperties( expressionsWithResolvedCalls ); @@ -816,9 +819,7 @@ public class TableImpl implements Table { @Override public Table select(Expression... fields) { - List expressionsWithResolvedCalls = Arrays.stream(fields) - .map(f -> f.accept(table.lookupResolver)) - .collect(Collectors.toList()); + List expressionsWithResolvedCalls = table.preprocessExpressions(fields); CategorizedExpressions extracted = OperationExpressionsUtils.extractAggregationsAndProperties( expressionsWithResolvedCalls ); @@ -873,9 +874,7 @@ public class TableImpl implements Table { @Override public Table select(Expression... fields) { - List expressionsWithResolvedCalls = Arrays.stream(fields) - .map(f -> f.accept(table.lookupResolver)) - .collect(Collectors.toList()); + List expressionsWithResolvedCalls = table.preprocessExpressions(fields); CategorizedExpressions extracted = OperationExpressionsUtils.extractAggregationsAndProperties( expressionsWithResolvedCalls ); diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/ApiExpressionUtils.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/ApiExpressionUtils.java index ec57fd5d27..92f8d83f4e 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/ApiExpressionUtils.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/ApiExpressionUtils.java @@ -19,6 +19,7 @@ package org.apache.flink.table.expressions; import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.ApiExpression; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.ValidationException; @@ -31,6 +32,7 @@ import org.apache.flink.table.types.DataType; import java.util.Arrays; import java.util.List; +import java.util.stream.Collectors; /** * Utilities for API-specific {@link Expression}s. @@ -50,6 +52,24 @@ public final class ApiExpressionUtils { // private } + public static Expression objectToExpression(Object expression) { + if (expression instanceof ApiExpression) { + return ((ApiExpression) expression).toExpr(); + } else if (expression instanceof Expression) { + return (Expression) expression; + } else { + return valueLiteral(expression); + } + } + + public static Expression unwrapFromApi(Expression expression) { + if (expression instanceof ApiExpression) { + return ((ApiExpression) expression).toExpr(); + } else { + return expression; + } + } + public static LocalReferenceExpression localRef(String name, DataType dataType) { return new LocalReferenceExpression(name, dataType); } @@ -74,14 +94,17 @@ public final class ApiExpressionUtils { FunctionIdentifier functionIdentifier, FunctionDefinition functionDefinition, Expression... args) { - return new UnresolvedCallExpression(functionIdentifier, functionDefinition, Arrays.asList(args)); + return unresolvedCall(functionIdentifier, functionDefinition, Arrays.asList(args)); } public static UnresolvedCallExpression unresolvedCall( FunctionIdentifier functionIdentifier, FunctionDefinition functionDefinition, List args) { - return new UnresolvedCallExpression(functionIdentifier, functionDefinition, args); + return new UnresolvedCallExpression(functionIdentifier, functionDefinition, + args.stream() + .map(ApiExpressionUtils::unwrapFromApi) + .collect(Collectors.toList())); } public static UnresolvedCallExpression unresolvedCall(FunctionDefinition functionDefinition, Expression... args) { @@ -89,7 +112,11 @@ public final class ApiExpressionUtils { } public static UnresolvedCallExpression unresolvedCall(FunctionDefinition functionDefinition, List args) { - return new UnresolvedCallExpression(functionDefinition, args); + return new UnresolvedCallExpression( + functionDefinition, + args.stream() + .map(ApiExpressionUtils::unwrapFromApi) + .collect(Collectors.toList())); } public static TableReferenceExpression tableRef(String name, Table table) { @@ -101,7 +128,11 @@ public final class ApiExpressionUtils { } public static LookupCallExpression lookupCall(String name, Expression... args) { - return new LookupCallExpression(name, Arrays.asList(args)); + return new LookupCallExpression( + name, + Arrays.stream(args) + .map(ApiExpressionUtils::unwrapFromApi) + .collect(Collectors.toList())); } public static Expression toMonthInterval(Expression e, int multiplier) { diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/LookupCallExpression.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/LookupCallExpression.java index d30efd5522..ef0663ad5a 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/LookupCallExpression.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/LookupCallExpression.java @@ -22,7 +22,6 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.table.functions.FunctionDefinition; import org.apache.flink.util.Preconditions; -import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Objects; @@ -43,7 +42,7 @@ public final class LookupCallExpression implements Expression { LookupCallExpression(String unresolvedFunction, List args) { this.unresolvedName = Preconditions.checkNotNull(unresolvedFunction); - this.args = Collections.unmodifiableList(new ArrayList<>(Preconditions.checkNotNull(args))); + this.args = Collections.unmodifiableList(Preconditions.checkNotNull(args)); } public String getUnresolvedName() { diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/UnresolvedCallExpression.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/UnresolvedCallExpression.java index 7fdf57d78c..a280d0ea97 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/UnresolvedCallExpression.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/UnresolvedCallExpression.java @@ -26,7 +26,6 @@ import org.apache.flink.util.Preconditions; import javax.annotation.Nullable; -import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Objects; @@ -61,8 +60,7 @@ public final class UnresolvedCallExpression implements Expression { Preconditions.checkNotNull(functionIdentifier, "Function identifier must not be null."); this.functionDefinition = Preconditions.checkNotNull(functionDefinition, "Function definition must not be null."); - this.args = Collections.unmodifiableList( - new ArrayList<>(Preconditions.checkNotNull(args, "Arguments must not be null."))); + this.args = Collections.unmodifiableList(Preconditions.checkNotNull(args, "Arguments must not be null.")); } UnresolvedCallExpression( @@ -71,8 +69,7 @@ public final class UnresolvedCallExpression implements Expression { this.functionIdentifier = null; this.functionDefinition = Preconditions.checkNotNull(functionDefinition, "Function definition must not be null."); - this.args = Collections.unmodifiableList( - new ArrayList<>(Preconditions.checkNotNull(args, "Arguments must not be null."))); + this.args = Collections.unmodifiableList(Preconditions.checkNotNull(args, "Arguments must not be null.")); } public Optional getFunctionIdentifier() { diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/ExpressionResolver.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/ExpressionResolver.java index 434697da12..59ef852bcc 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/ExpressionResolver.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/ExpressionResolver.java @@ -77,6 +77,7 @@ public class ExpressionResolver { */ public static List getExpandingResolverRules() { return Arrays.asList( + ResolverRules.UNWRAP_API_EXPRESSION, ResolverRules.LOOKUP_CALL_BY_NAME, ResolverRules.FLATTEN_STAR_REFERENCE, ResolverRules.EXPAND_COLUMN_FUNCTIONS); @@ -87,6 +88,7 @@ public class ExpressionResolver { */ public static List getAllResolverRules() { return Arrays.asList( + ResolverRules.UNWRAP_API_EXPRESSION, ResolverRules.LOOKUP_CALL_BY_NAME, ResolverRules.FLATTEN_STAR_REFERENCE, ResolverRules.EXPAND_COLUMN_FUNCTIONS, diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/LookupCallResolver.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/LookupCallResolver.java index 93471ef593..3071fa6e78 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/LookupCallResolver.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/LookupCallResolver.java @@ -19,6 +19,7 @@ package org.apache.flink.table.expressions.resolver; import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.ApiExpression; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.FunctionLookup; import org.apache.flink.table.expressions.Expression; @@ -64,6 +65,17 @@ public class LookupCallResolver extends ApiExpressionDefaultVisitor .collect(Collectors.toList()); } + @Override + public Expression visitNonApiExpression(Expression other) { + // LookupCallResolver might be called outside of ExpressionResolver, thus we need to additionally + // handle the ApiExpressions here + if (other instanceof ApiExpression) { + return ((ApiExpression) other).toExpr().accept(this); + } else { + return defaultMethod(other); + } + } + @Override protected Expression defaultMethod(Expression expression) { return expression; diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/rules/OverWindowResolverRule.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/rules/OverWindowResolverRule.java index 42dc075763..3acb5afdc8 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/rules/OverWindowResolverRule.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/rules/OverWindowResolverRule.java @@ -55,7 +55,7 @@ final class OverWindowResolverRule implements ResolverRule { .collect(Collectors.toList()); } - private class ExpressionResolverVisitor extends RuleExpressionVisitor { + private static class ExpressionResolverVisitor extends RuleExpressionVisitor { ExpressionResolverVisitor(ResolutionContext context) { super(context); diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/rules/ResolverRules.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/rules/ResolverRules.java index 671b1e1966..aab227222b 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/rules/ResolverRules.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/rules/ResolverRules.java @@ -19,6 +19,7 @@ package org.apache.flink.table.expressions.resolver.rules; import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.ApiExpression; import org.apache.flink.table.expressions.UnresolvedReferenceExpression; /** @@ -62,6 +63,11 @@ public final class ResolverRules { */ public static final ResolverRule QUALIFY_BUILT_IN_FUNCTIONS = new QualifyBuiltInFunctionsRule(); + /** + * Unwraps all {@link ApiExpression}. + */ + public static final ResolverRule UNWRAP_API_EXPRESSION = new UnwrapApiExpressionRule(); + private ResolverRules() { } } diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/rules/UnwrapApiExpressionRule.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/rules/UnwrapApiExpressionRule.java new file mode 100644 index 0000000000..1ef374b3d0 --- /dev/null +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/rules/UnwrapApiExpressionRule.java @@ -0,0 +1,40 @@ +/* + * 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.table.expressions.resolver.rules; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.ApiExpression; +import org.apache.flink.table.expressions.ApiExpressionUtils; +import org.apache.flink.table.expressions.Expression; + +import java.util.List; +import java.util.stream.Collectors; + +/** + * Unwraps all {@link ApiExpression}. + */ +@Internal +final class UnwrapApiExpressionRule implements ResolverRule { + @Override + public List apply( + List expression, + ResolutionContext context) { + return expression.stream().map(ApiExpressionUtils::unwrapFromApi).collect(Collectors.toList()); + } +} diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/typeutils/FieldInfoUtils.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/typeutils/FieldInfoUtils.java index 4877bc40a6..3a3b0a0462 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/typeutils/FieldInfoUtils.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/typeutils/FieldInfoUtils.java @@ -28,6 +28,7 @@ import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.Types; import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.expressions.ApiExpressionUtils; import org.apache.flink.table.expressions.Expression; import org.apache.flink.table.expressions.ExpressionUtils; import org.apache.flink.table.expressions.UnresolvedCallExpression; @@ -154,7 +155,7 @@ public class FieldInfoUtils { * used if the input type has a defined field order (tuple, case class, Row) and no of fields * references a field of the input type. */ - public static boolean isReferenceByPosition(CompositeType ct, Expression[] fields) { + private static boolean isReferenceByPosition(CompositeType ct, Expression[] fields) { if (!(ct instanceof TupleTypeInfoBase)) { return false; } @@ -224,7 +225,10 @@ public class FieldInfoUtils { public static TypeInfoSchema getFieldsInfo(TypeInformation inputType, Expression[] expressions) { validateInputTypeInfo(inputType); - final List fieldInfos = extractFieldInformation(inputType, expressions); + final List fieldInfos = extractFieldInformation( + inputType, + Arrays.stream(expressions).map(ApiExpressionUtils::unwrapFromApi).toArray(Expression[]::new) + ); validateNoStarReference(fieldInfos); boolean isRowtimeAttribute = checkIfRowtimeAttribute(fieldInfos); @@ -244,8 +248,8 @@ public class FieldInfoUtils { } private static List extractFieldInformation( - TypeInformation inputType, - Expression[] exprs) { + TypeInformation inputType, + Expression[] exprs) { final List fieldInfos; if (inputType instanceof GenericTypeInfo && inputType.getTypeClass() == Row.class) { throw new ValidationException( @@ -367,7 +371,7 @@ public class FieldInfoUtils { final TypeInformation[] fieldTypes; if (inputType instanceof CompositeType) { int arity = inputType.getArity(); - CompositeType ct = (CompositeType) inputType; + CompositeType ct = (CompositeType) inputType; fieldTypes = IntStream.range(0, arity).mapToObj(ct::getTypeAt).toArray(TypeInformation[]::new); } else { fieldTypes = new TypeInformation[]{inputType}; diff --git a/flink-table/flink-table-api-scala/src/main/scala/org/apache/flink/table/api/expressionDsl.scala b/flink-table/flink-table-api-scala/src/main/scala/org/apache/flink/table/api/expressionDsl.scala index cdf4042ae0..cb71dc7d02 100644 --- a/flink-table/flink-table-api-scala/src/main/scala/org/apache/flink/table/api/expressionDsl.scala +++ b/flink-table/flink-table-api-scala/src/main/scala/org/apache/flink/table/api/expressionDsl.scala @@ -17,105 +17,82 @@ */ package org.apache.flink.table.api -import java.lang.{Boolean => JBoolean, Byte => JByte, Double => JDouble, Float => JFloat, Integer => JInteger, Long => JLong, Short => JShort} -import java.math.{BigDecimal => JBigDecimal} -import java.sql.{Date, Time, Timestamp} -import java.time.{LocalDate, LocalDateTime, LocalTime} - import org.apache.flink.annotation.PublicEvolving -import org.apache.flink.api.common.typeinfo.{SqlTimeTypeInfo, TypeInformation} +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.table.api.internal.BaseExpressions +import org.apache.flink.table.expressions.ApiExpressionUtils._ import org.apache.flink.table.expressions._ -import ApiExpressionUtils._ import org.apache.flink.table.functions.BuiltInFunctionDefinitions._ import org.apache.flink.table.functions.{ScalarFunction, TableFunction, UserDefinedAggregateFunction, UserDefinedFunctionHelper, _} import org.apache.flink.table.types.DataType -import org.apache.flink.table.types.utils.TypeConversions -import org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType + +import java.lang.{Boolean => JBoolean, Byte => JByte, Double => JDouble, Float => JFloat, Integer => JInteger, Long => JLong, Short => JShort} +import java.math.{BigDecimal => JBigDecimal} +import java.sql.{Date, Time, Timestamp} +import java.time.{LocalDate, LocalDateTime, LocalTime} import _root_.scala.language.implicitConversions /** * These are all the operations that can be used to construct an [[Expression]] AST for * expression operations. - * - * These operations must be kept in sync with the parser in - * [[org.apache.flink.table.expressions.ExpressionParser]]. */ @PublicEvolving -trait ImplicitExpressionOperations { +trait ImplicitExpressionOperations extends BaseExpressions[Expression, Expression] { private[flink] def expr: Expression + override def toExpr: Expression = expr + + override protected def toApiSpecificExpression(expression: Expression): Expression = expression + /** - * Enables literals on left side of binary expressions. - * - * e.g. 12.toExpr % 'a - * - * @return expression - */ - def toExpr: Expression = expr + * Specifies a name for an expression i.e. a field. + * + * @param name name for one field + * @param extraNames additional names if the expression expands to multiple fields + * @return field with an alias + */ + def as(name: Symbol, extraNames: Symbol*): Expression = as(name.name, extraNames.map(_.name): _*) /** * Boolean AND in three-valued logic. */ - def && (other: Expression): Expression = unresolvedCall(AND, expr, other) + def && (other: Expression): Expression = and(other) /** * Boolean OR in three-valued logic. */ - def || (other: Expression): Expression = unresolvedCall(OR, expr, other) + def || (other: Expression): Expression = or(other) /** * Greater than. */ - def > (other: Expression): Expression = unresolvedCall(GREATER_THAN, expr, other) + def > (other: Expression): Expression = isGreater(other) /** * Greater than or equal. */ - def >= (other: Expression): Expression = unresolvedCall(GREATER_THAN_OR_EQUAL, expr, other) + def >= (other: Expression): Expression = isGreaterOrEqual(other) /** * Less than. */ - def < (other: Expression): Expression = unresolvedCall(LESS_THAN, expr, other) + def < (other: Expression): Expression = isLess(other) /** * Less than or equal. */ - def <= (other: Expression): Expression = unresolvedCall(LESS_THAN_OR_EQUAL, expr, other) + def <= (other: Expression): Expression = isLessOrEqual(other) /** * Equals. */ - def === (other: Expression): Expression = unresolvedCall(EQUALS, expr, other) + def === (other: Expression): Expression = isEqual(other) /** * Not equal. */ - def !== (other: Expression): Expression = unresolvedCall(NOT_EQUALS, expr, other) - - /** - * Returns true if the given expression is between lowerBound and upperBound (both inclusive). - * False otherwise. The parameters must be numeric types or identical comparable types. - * - * @param lowerBound numeric or comparable expression - * @param upperBound numeric or comparable expression - * @return boolean or null - */ - def between(lowerBound: Expression, upperBound: Expression): Expression = - unresolvedCall(BETWEEN, expr, lowerBound, upperBound) - - /** - * Returns true if the given expression is not between lowerBound and upperBound (both - * inclusive). False otherwise. The parameters must be numeric types or identical - * comparable types. - * - * @param lowerBound numeric or comparable expression - * @param upperBound numeric or comparable expression - * @return boolean or null - */ - def notBetween(lowerBound: Expression, upperBound: Expression): Expression = - unresolvedCall(NOT_BETWEEN, expr, lowerBound, upperBound) + def !== (other: Expression): Expression = isNotEqual(other) /** * Whether boolean expression is not true; returns null if boolean is null. @@ -125,62 +102,32 @@ trait ImplicitExpressionOperations { /** * Returns negative numeric. */ - def unary_- : Expression = unresolvedCall(MINUS_PREFIX, expr) + def unary_- : Expression = Expressions.negative(expr) /** * Returns numeric. */ def unary_+ : Expression = expr - /** - * Returns true if the given expression is null. - */ - def isNull: Expression = unresolvedCall(IS_NULL, expr) - - /** - * Returns true if the given expression is not null. - */ - def isNotNull: Expression = unresolvedCall(IS_NOT_NULL, expr) - - /** - * Returns true if given boolean expression is true. False otherwise (for null and false). - */ - def isTrue: Expression = unresolvedCall(IS_TRUE, expr) - - /** - * Returns true if given boolean expression is false. False otherwise (for null and true). - */ - def isFalse: Expression = unresolvedCall(IS_FALSE, expr) - - /** - * Returns true if given boolean expression is not true (for null and false). False otherwise. - */ - def isNotTrue: Expression = unresolvedCall(IS_NOT_TRUE, expr) - - /** - * Returns true if given boolean expression is not false (for null and true). False otherwise. - */ - def isNotFalse: Expression = unresolvedCall(IS_NOT_FALSE, expr) - /** * Returns left plus right. */ - def + (other: Expression): Expression = unresolvedCall(PLUS, expr, other) + def + (other: Expression): Expression = plus(other) /** * Returns left minus right. */ - def - (other: Expression): Expression = unresolvedCall(MINUS, expr, other) + def - (other: Expression): Expression = minus(other) /** * Returns left divided by right. */ - def / (other: Expression): Expression = unresolvedCall(DIVIDE, expr, other) + def / (other: Expression): Expression = dividedBy(other) /** * Returns left multiplied by right. */ - def * (other: Expression): Expression = unresolvedCall(TIMES, expr, other) + def * (other: Expression): Expression = times(other) /** * Returns the remainder (modulus) of left divided by right. @@ -194,351 +141,22 @@ trait ImplicitExpressionOperations { * * e.g. withColumns(1 to 3) */ - def to (other: Expression): Expression = unresolvedCall(RANGE_TO, expr, other) - - /** - * Similar to a SQL distinct aggregation clause such as COUNT(DISTINCT a), declares that an - * aggregation function is only applied on distinct input values. - * - * For example: - * - * {{{ - * orders - * .groupBy('a) - * .select('a, 'b.sum.distinct as 'd) - * }}} - */ - def distinct: Expression = unresolvedCall(DISTINCT, expr) - - /** - * Returns the sum of the numeric field across all input values. - * If all values are null, null is returned. - */ - def sum: Expression = unresolvedCall(SUM, expr) - - /** - * Returns the sum of the numeric field across all input values. - * If all values are null, 0 is returned. - */ - def sum0: Expression = unresolvedCall(SUM0, expr) - - /** - * Returns the minimum value of field across all input values. - */ - def min: Expression = unresolvedCall(MIN, expr) - - /** - * Returns the maximum value of field across all input values. - */ - def max: Expression = unresolvedCall(MAX, expr) - - /** - * Returns the number of input rows for which the field is not null. - */ - def count: Expression = unresolvedCall(COUNT, expr) - - /** - * Returns the average (arithmetic mean) of the numeric field across all input values. - */ - def avg: Expression = unresolvedCall(AVG, expr) - - /** - * Returns the population standard deviation of an expression (the square root of varPop()). - */ - def stddevPop: Expression = unresolvedCall(STDDEV_POP, expr) - - /** - * Returns the sample standard deviation of an expression (the square root of varSamp()). - */ - def stddevSamp: Expression = unresolvedCall(STDDEV_SAMP, expr) - - /** - * Returns the population standard variance of an expression. - */ - def varPop: Expression = unresolvedCall(VAR_POP, expr) - - /** - * Returns the sample variance of a given expression. - */ - def varSamp: Expression = unresolvedCall(VAR_SAMP, expr) - - /** - * Returns multiset aggregate of a given expression. - */ - def collect: Expression = unresolvedCall(COLLECT, expr) - - /** - * Converts a value to a given data type. - * - * e.g. "42".cast(DataTypes.INT()) leads to 42. - * - * @return casted expression - */ - def cast(toType: DataType): Expression = - unresolvedCall(CAST, expr, typeLiteral(toType)) - - /** - * @deprecated This method will be removed in future versions as it uses the old type system. It - * is recommended to use [[cast(DataType)]] instead which uses the new type system - * based on [[DataTypes]]. Please make sure to use either the old or the new type - * system consistently to avoid unintended behavior. See the website documentation - * for more information. - */ - @deprecated - def cast(toType: TypeInformation[_]): Expression = - unresolvedCall(CAST, expr, typeLiteral(fromLegacyInfoToDataType(toType))) - - /** - * Specifies a name for an expression i.e. a field. - * - * @param name name for one field - * @param extraNames additional names if the expression expands to multiple fields - * @return field with an alias - */ - def as(name: Symbol, extraNames: Symbol*): Expression = - unresolvedCall( - AS, - expr +: valueLiteral(name.name) +: extraNames.map(name => valueLiteral(name.name)): _*) - - /** - * Specifies ascending order of an expression i.e. a field for orderBy call. - * - * @return ascend expression - */ - def asc: Expression = unresolvedCall(ORDER_ASC, expr) - - /** - * Specifies descending order of an expression i.e. a field for orderBy call. - * - * @return descend expression - */ - def desc: Expression = unresolvedCall(ORDER_DESC, expr) - - /** - * Returns true if an expression exists in a given list of expressions. This is a shorthand - * for multiple OR conditions. - * - * If the testing set contains null, the result will be null if the element can not be found - * and true if it can be found. If the element is null, the result is always null. - * - * e.g. "42".in(1, 2, 3) leads to false. - */ - def in(elements: Expression*): Expression = unresolvedCall(IN, expr +: elements: _*) - - /** - * Returns true if an expression exists in a given table sub-query. The sub-query table - * must consist of one column. This column must have the same data type as the expression. - * - * Note: This operation is not supported in a streaming environment yet. - */ - def in(table: Table): Expression = unresolvedCall(IN, expr, tableRef(table.toString, table)) - - /** - * Returns the start time (inclusive) of a window when applied on a window reference. - */ - def start: Expression = unresolvedCall(WINDOW_START, expr) - - /** - * Returns the end time (exclusive) of a window when applied on a window reference. - * - * e.g. if a window ends at 10:59:59.999 this property will return 11:00:00.000. - */ - def end: Expression = unresolvedCall(WINDOW_END, expr) + def to (other: Expression): Expression = unresolvedCall(RANGE_TO, expr, objectToExpression(other)) /** * Ternary conditional operator that decides which of two other expressions should be * based on a evaluated boolean condition. * - * e.g. (42 > 5).?("A", "B") leads to "A" + * e.g. ($"f0" > 5).?("A", "B") leads to "A" * * @param ifTrue expression to be evaluated if condition holds * @param ifFalse expression to be evaluated if condition does not hold */ def ?(ifTrue: Expression, ifFalse: Expression): Expression = - unresolvedCall(IF, expr, ifTrue, ifFalse) + Expressions.ifThenElse(expr, ifTrue, ifFalse).toExpr // scalar functions - /** - * Calculates the remainder of division the given number by another one. - */ - def mod(other: Expression): Expression = unresolvedCall(MOD, expr, other) - - /** - * Calculates the Euler's number raised to the given power. - */ - def exp(): Expression = unresolvedCall(EXP, expr) - - /** - * Calculates the base 10 logarithm of the given value. - */ - def log10(): Expression = unresolvedCall(LOG10, expr) - - /** - * Calculates the base 2 logarithm of the given value. - */ - def log2(): Expression = unresolvedCall(LOG2, expr) - - /** - * Calculates the natural logarithm of the given value. - */ - def ln(): Expression = unresolvedCall(LN, expr) - - /** - * Calculates the natural logarithm of the given value. - */ - def log(): Expression = unresolvedCall(LOG, expr) - - /** - * Calculates the logarithm of the given value to the given base. - */ - def log(base: Expression): Expression = unresolvedCall(LOG, base, expr) - - /** - * Calculates the given number raised to the power of the other value. - */ - def power(other: Expression): Expression = unresolvedCall(POWER, expr, other) - - /** - * Calculates the hyperbolic cosine of a given value. - */ - def cosh(): Expression = unresolvedCall(COSH, expr) - - /** - * Calculates the square root of a given value. - */ - def sqrt(): Expression = unresolvedCall(SQRT, expr) - - /** - * Calculates the absolute value of given value. - */ - def abs(): Expression = unresolvedCall(ABS, expr) - - /** - * Calculates the largest integer less than or equal to a given number. - */ - def floor(): Expression = unresolvedCall(FLOOR, expr) - - /** - * Calculates the hyperbolic sine of a given value. - */ - def sinh(): Expression = unresolvedCall(SINH, expr) - - /** - * Calculates the smallest integer greater than or equal to a given number. - */ - def ceil(): Expression = unresolvedCall(CEIL, expr) - - /** - * Calculates the sine of a given number. - */ - def sin(): Expression = unresolvedCall(SIN, expr) - - /** - * Calculates the cosine of a given number. - */ - def cos(): Expression = unresolvedCall(COS, expr) - - /** - * Calculates the tangent of a given number. - */ - def tan(): Expression = unresolvedCall(TAN, expr) - - /** - * Calculates the cotangent of a given number. - */ - def cot(): Expression = unresolvedCall(COT, expr) - - /** - * Calculates the arc sine of a given number. - */ - def asin(): Expression = unresolvedCall(ASIN, expr) - - /** - * Calculates the arc cosine of a given number. - */ - def acos(): Expression = unresolvedCall(ACOS, expr) - - /** - * Calculates the arc tangent of a given number. - */ - def atan(): Expression = unresolvedCall(ATAN, expr) - - /** - * Calculates the hyperbolic tangent of a given number. - */ - def tanh(): Expression = unresolvedCall(TANH, expr) - - /** - * Converts numeric from radians to degrees. - */ - def degrees(): Expression = unresolvedCall(DEGREES, expr) - - /** - * Converts numeric from degrees to radians. - */ - def radians(): Expression = unresolvedCall(RADIANS, expr) - - /** - * Calculates the signum of a given number. - */ - def sign(): Expression = unresolvedCall(SIGN, expr) - - /** - * Rounds the given number to integer places right to the decimal point. - */ - def round(places: Expression): Expression = unresolvedCall(ROUND, expr, places) - - /** - * Returns a string representation of an integer numeric value in binary format. Returns null if - * numeric is null. E.g. "4" leads to "100", "12" leads to "1100". - */ - def bin(): Expression = unresolvedCall(BIN, expr) - - /** - * Returns a string representation of an integer numeric value or a string in hex format. Returns - * null if numeric or string is null. - * - * E.g. a numeric 20 leads to "14", a numeric 100 leads to "64", and a string "hello,world" leads - * to "68656c6c6f2c776f726c64". - */ - def hex(): Expression = unresolvedCall(HEX, expr) - - /** - * Returns a number of truncated to n decimal places. - * If n is 0,the result has no decimal point or fractional part. - * n can be negative to cause n digits left of the decimal point of the value to become zero. - * E.g. truncate(42.345, 2) to 42.34. - */ - def truncate(n: Expression): Expression = unresolvedCall(TRUNCATE, expr, n) - - /** - * Returns a number of truncated to 0 decimal places. - * E.g. truncate(42.345) to 42.0. - */ - def truncate(): Expression = unresolvedCall(TRUNCATE, expr) - - // String operations - - /** - * Creates a substring of the given string at given index for a given length. - * - * @param beginIndex first character of the substring (starting at 1, inclusive) - * @param length number of characters of the substring - * @return substring - */ - def substring(beginIndex: Expression, length: Expression): Expression = - unresolvedCall(SUBSTRING, expr, beginIndex, length) - - /** - * Creates a substring of the given string beginning at the given index to the end. - * - * @param beginIndex first character of the substring (starting at 1, inclusive) - * @return substring - */ - def substring(beginIndex: Expression): Expression = - unresolvedCall(SUBSTRING, expr, beginIndex) - /** * Removes leading and/or trailing characters from the given string. * @@ -552,324 +170,14 @@ trait ImplicitExpressionOperations { removeTrailing: Boolean = true, character: Expression = valueLiteral(" ")) : Expression = { - unresolvedCall(TRIM, valueLiteral(removeLeading), valueLiteral(removeTrailing), character, expr) + unresolvedCall( + TRIM, + valueLiteral(removeLeading), + valueLiteral(removeTrailing), + ApiExpressionUtils.objectToExpression(character), + expr) } - /** - * Returns a new string which replaces all the occurrences of the search target - * with the replacement string (non-overlapping). - */ - def replace(search: Expression, replacement: Expression): Expression = - unresolvedCall(REPLACE, expr, search, replacement) - - /** - * Returns the length of a string. - */ - def charLength(): Expression = unresolvedCall(CHAR_LENGTH, expr) - - /** - * Returns all of the characters in a string in upper case using the rules of - * the default locale. - */ - def upperCase(): Expression = unresolvedCall(UPPER, expr) - - /** - * Returns all of the characters in a string in lower case using the rules of - * the default locale. - */ - def lowerCase(): Expression = unresolvedCall(LOWER, expr) - - /** - * Converts the initial letter of each word in a string to uppercase. - * Assumes a string containing only [A-Za-z0-9], everything else is treated as whitespace. - */ - def initCap(): Expression = unresolvedCall(INIT_CAP, expr) - - /** - * Returns true, if a string matches the specified LIKE pattern. - * - * e.g. "Jo_n%" matches all strings that start with "Jo(arbitrary letter)n" - */ - def like(pattern: Expression): Expression = unresolvedCall(LIKE, expr, pattern) - - /** - * Returns true, if a string matches the specified SQL regex pattern. - * - * e.g. "A+" matches all strings that consist of at least one A - */ - def similar(pattern: Expression): Expression = unresolvedCall(SIMILAR, expr, pattern) - - /** - * Returns the position of string in an other string starting at 1. - * Returns 0 if string could not be found. - * - * e.g. "a".position("bbbbba") leads to 6 - */ - def position(haystack: Expression): Expression = unresolvedCall(POSITION, expr, haystack) - - /** - * Returns a string left-padded with the given pad string to a length of len characters. If - * the string is longer than len, the return value is shortened to len characters. - * - * e.g. "hi".lpad(4, '??') returns "??hi", "hi".lpad(1, '??') returns "h" - */ - def lpad(len: Expression, pad: Expression): Expression = unresolvedCall(LPAD, expr, len, pad) - - /** - * Returns a string right-padded with the given pad string to a length of len characters. If - * the string is longer than len, the return value is shortened to len characters. - * - * e.g. "hi".rpad(4, '??') returns "hi??", "hi".rpad(1, '??') returns "h" - */ - def rpad(len: Expression, pad: Expression): Expression = unresolvedCall(RPAD, expr, len, pad) - - /** - * Defines an aggregation to be used for a previously specified over window. - * - * For example: - * - * {{{ - * table - * .window(Over partitionBy 'c orderBy 'rowtime preceding 2.rows following CURRENT_ROW as 'w) - * .select('c, 'a, 'a.count over 'w, 'a.sum over 'w) - * }}} - */ - def over(alias: Expression): Expression = unresolvedCall(OVER, expr, alias) - - /** - * Replaces a substring of string with a string starting at a position (starting at 1). - * - * e.g. "xxxxxtest".overlay("xxxx", 6) leads to "xxxxxxxxx" - */ - def overlay(newString: Expression, starting: Expression): Expression = - unresolvedCall(OVERLAY, expr, newString, starting) - - /** - * Replaces a substring of string with a string starting at a position (starting at 1). - * The length specifies how many characters should be removed. - * - * e.g. "xxxxxtest".overlay("xxxx", 6, 2) leads to "xxxxxxxxxst" - */ - def overlay(newString: Expression, starting: Expression, length: Expression): Expression = - unresolvedCall(OVERLAY, expr, newString, starting, length) - - /** - * Returns a string with all substrings that match the regular expression consecutively - * being replaced. - */ - def regexpReplace(regex: Expression, replacement: Expression): Expression = - unresolvedCall(REGEXP_REPLACE, expr, regex, replacement) - - /** - * Returns a string extracted with a specified regular expression and a regex match group - * index. - */ - def regexpExtract(regex: Expression, extractIndex: Expression): Expression = - unresolvedCall(REGEXP_EXTRACT, expr, regex, extractIndex) - - /** - * Returns a string extracted with a specified regular expression. - */ - def regexpExtract(regex: Expression): Expression = - unresolvedCall(REGEXP_EXTRACT, expr, regex) - - /** - * Returns the base string decoded with base64. - */ - def fromBase64(): Expression = unresolvedCall(FROM_BASE64, expr) - - /** - * Returns the base64-encoded result of the input string. - */ - def toBase64(): Expression = unresolvedCall(TO_BASE64, expr) - - /** - * Returns a string that removes the left whitespaces from the given string. - */ - def ltrim(): Expression = unresolvedCall(LTRIM, expr) - - /** - * Returns a string that removes the right whitespaces from the given string. - */ - def rtrim(): Expression = unresolvedCall(RTRIM, expr) - - /** - * Returns a string that repeats the base string n times. - */ - def repeat(n: Expression): Expression = unresolvedCall(REPEAT, expr, n) - - // Temporal operations - - /** - * Parses a date string in the form "yyyy-MM-dd" to a SQL Date. - */ - def toDate: Expression = - unresolvedCall(CAST, expr, typeLiteral(fromLegacyInfoToDataType(SqlTimeTypeInfo.DATE))) - - /** - * Parses a time string in the form "HH:mm:ss" to a SQL Time. - */ - def toTime: Expression = - unresolvedCall(CAST, expr, typeLiteral(fromLegacyInfoToDataType(SqlTimeTypeInfo.TIME))) - - /** - * Parses a timestamp string in the form "yyyy-MM-dd HH:mm:ss[.SSS]" to a SQL Timestamp. - */ - def toTimestamp: Expression = - unresolvedCall(CAST, expr, typeLiteral(fromLegacyInfoToDataType(SqlTimeTypeInfo.TIMESTAMP))) - - /** - * Extracts parts of a time point or time interval. Returns the part as a long value. - * - * e.g. "2006-06-05".toDate.extract(DAY) leads to 5 - */ - def extract(timeIntervalUnit: TimeIntervalUnit): Expression = - unresolvedCall(EXTRACT, valueLiteral(timeIntervalUnit), expr) - - /** - * Rounds down a time point to the given unit. - * - * e.g. "12:44:31".toDate.floor(MINUTE) leads to 12:44:00 - */ - def floor(timeIntervalUnit: TimeIntervalUnit): Expression = - unresolvedCall(FLOOR, valueLiteral(timeIntervalUnit), expr) - - /** - * Rounds up a time point to the given unit. - * - * e.g. "12:44:31".toDate.ceil(MINUTE) leads to 12:45:00 - */ - def ceil(timeIntervalUnit: TimeIntervalUnit): Expression = - unresolvedCall(CEIL, valueLiteral(timeIntervalUnit), expr) - - // Interval types - - /** - * Creates an interval of the given number of years. - * - * @return interval of months - */ - def year: Expression = toMonthInterval(expr, 12) - - /** - * Creates an interval of the given number of years. - * - * @return interval of months - */ - def years: Expression = year - - /** - * Creates an interval of the given number of quarters. - * - * @return interval of months - */ - def quarter: Expression = toMonthInterval(expr, 3) - - /** - * Creates an interval of the given number of quarters. - * - * @return interval of months - */ - def quarters: Expression = quarter - - /** - * Creates an interval of the given number of months. - * - * @return interval of months - */ - def month: Expression = toMonthInterval(expr, 1) - - /** - * Creates an interval of the given number of months. - * - * @return interval of months - */ - def months: Expression = month - - /** - * Creates an interval of the given number of weeks. - * - * @return interval of milliseconds - */ - def week: Expression = toMilliInterval(expr, 7 * MILLIS_PER_DAY) - - /** - * Creates an interval of the given number of weeks. - * - * @return interval of milliseconds - */ - def weeks: Expression = week - - /** - * Creates an interval of the given number of days. - * - * @return interval of milliseconds - */ - def day: Expression = toMilliInterval(expr, MILLIS_PER_DAY) - - /** - * Creates an interval of the given number of days. - * - * @return interval of milliseconds - */ - def days: Expression = day - - /** - * Creates an interval of the given number of hours. - * - * @return interval of milliseconds - */ - def hour: Expression = toMilliInterval(expr, MILLIS_PER_HOUR) - - /** - * Creates an interval of the given number of hours. - * - * @return interval of milliseconds - */ - def hours: Expression = hour - - /** - * Creates an interval of the given number of minutes. - * - * @return interval of milliseconds - */ - def minute: Expression = toMilliInterval(expr, MILLIS_PER_MINUTE) - - /** - * Creates an interval of the given number of minutes. - * - * @return interval of milliseconds - */ - def minutes: Expression = minute - - /** - * Creates an interval of the given number of seconds. - * - * @return interval of milliseconds - */ - def second: Expression = toMilliInterval(expr, MILLIS_PER_SECOND) - - /** - * Creates an interval of the given number of seconds. - * - * @return interval of milliseconds - */ - def seconds: Expression = second - - /** - * Creates an interval of the given number of milliseconds. - * - * @return interval of milliseconds - */ - def milli: Expression = toMilliInterval(expr, 1) - - /** - * Creates an interval of the given number of milliseconds. - * - * @return interval of milliseconds - */ - def millis: Expression = milli - // Row interval type /** @@ -879,121 +187,6 @@ trait ImplicitExpressionOperations { */ def rows: Expression = toRowInterval(expr) - // Advanced type helper functions - - /** - * Accesses the field of a Flink composite type (such as Tuple, POJO, etc.) by name and - * returns it's value. - * - * @param name name of the field (similar to Flink's field expressions) - * @return value of the field - */ - def get(name: String): Expression = unresolvedCall(GET, expr, valueLiteral(name)) - - /** - * Accesses the field of a Flink composite type (such as Tuple, POJO, etc.) by index and - * returns it's value. - * - * @param index position of the field - * @return value of the field - */ - def get(index: Int): Expression = unresolvedCall(GET, expr, valueLiteral(index)) - - /** - * Converts a Flink composite type (such as Tuple, POJO, etc.) and all of its direct subtypes - * into a flat representation where every subtype is a separate field. - */ - def flatten(): Expression = unresolvedCall(FLATTEN, expr) - - /** - * Accesses the element of an array or map based on a key or an index (starting at 1). - * - * @param index key or position of the element (array index starting at 1) - * @return value of the element - */ - def at(index: Expression): Expression = unresolvedCall(AT, expr, index) - - /** - * Returns the number of elements of an array or number of entries of a map. - * - * @return number of elements or entries - */ - def cardinality(): Expression = unresolvedCall(CARDINALITY, expr) - - /** - * Returns the sole element of an array with a single element. Returns null if the array is - * empty. Throws an exception if the array has more than one element. - * - * @return the first and only element of an array with a single element - */ - def element(): Expression = unresolvedCall(ARRAY_ELEMENT, expr) - - // Time definition - - /** - * Declares a field as the rowtime attribute for indicating, accessing, and working in - * Flink's event time. - */ - def rowtime: Expression = unresolvedCall(ROWTIME, expr) - - /** - * Declares a field as the proctime attribute for indicating, accessing, and working in - * Flink's processing time. - */ - def proctime: Expression = unresolvedCall(PROCTIME, expr) - - // Hash functions - - /** - * Returns the MD5 hash of the string argument; null if string is null. - * - * @return string of 32 hexadecimal digits or null - */ - def md5(): Expression = unresolvedCall(MD5, expr) - - /** - * Returns the SHA-1 hash of the string argument; null if string is null. - * - * @return string of 40 hexadecimal digits or null - */ - def sha1(): Expression = unresolvedCall(SHA1, expr) - - /** - * Returns the SHA-224 hash of the string argument; null if string is null. - * - * @return string of 56 hexadecimal digits or null - */ - def sha224(): Expression = unresolvedCall(SHA224, expr) - - /** - * Returns the SHA-256 hash of the string argument; null if string is null. - * - * @return string of 64 hexadecimal digits or null - */ - def sha256(): Expression = unresolvedCall(SHA256, expr) - - /** - * Returns the SHA-384 hash of the string argument; null if string is null. - * - * @return string of 96 hexadecimal digits or null - */ - def sha384(): Expression = unresolvedCall(SHA384, expr) - - /** - * Returns the SHA-512 hash of the string argument; null if string is null. - * - * @return string of 128 hexadecimal digits or null - */ - def sha512(): Expression = unresolvedCall(SHA512, expr) - - /** - * Returns the hash for the given string expression using the SHA-2 family of hash - * functions (SHA-224, SHA-256, SHA-384, or SHA-512). - * - * @param hashLength bit length of the result (either 224, 256, 384, or 512) - * @return string or null if one of the arguments is null. - */ - def sha2(hashLength: Expression): Expression = unresolvedCall(SHA2, expr, hashLength) } /** @@ -1109,7 +302,9 @@ trait ImplicitExpressionConversions { * Calls a scalar function for the given parameters. */ def apply(params: Expression*): Expression = { - unresolvedCall(new ScalarFunctionDefinition(s.getClass.getName, s), params:_*) + unresolvedCall( + new ScalarFunctionDefinition(s.getClass.getName, s), + params.map(ApiExpressionUtils.objectToExpression): _*) } } @@ -1121,7 +316,9 @@ trait ImplicitExpressionConversions { def apply(params: Expression*): Expression = { val resultTypeInfo: TypeInformation[T] = UserDefinedFunctionHelper .getReturnTypeOfTableFunction(t, implicitly[TypeInformation[T]]) - unresolvedCall(new TableFunctionDefinition(t.getClass.getName, t, resultTypeInfo), params: _*) + unresolvedCall( + new TableFunctionDefinition(t.getClass.getName, t, resultTypeInfo), + params.map(ApiExpressionUtils.objectToExpression): _*) } } @@ -1149,7 +346,9 @@ trait ImplicitExpressionConversions { * Calls an aggregate function for the given parameters. */ def apply(params: Expression*): Expression = { - unresolvedCall(createFunctionDefinition(), params: _*) + unresolvedCall( + createFunctionDefinition(), + params.map(ApiExpressionUtils.objectToExpression): _*) } /** @@ -1160,6 +359,25 @@ trait ImplicitExpressionConversions { } } + + /** + * Extends Scala's StringContext with a method for creating an unresolved reference via + * string interpolation. + */ + implicit class FieldExpression(val sc: StringContext) { + + /** + * Creates an unresolved reference to a table's field. + * + * Example: + * ``` + * tab.select($"key", $"value") + * ``` + * + */ + def $(args: Any*): Expression = unresolvedRef(sc.s(args: _*)) + } + implicit def tableSymbolToExpression(sym: TableSymbol): Expression = valueLiteral(sym) @@ -1169,7 +387,7 @@ trait ImplicitExpressionConversions { implicit def scalaRange2RangeExpression(range: Range.Inclusive): Expression = { val startExpression = valueLiteral(range.start) val endExpression = valueLiteral(range.end) - startExpression to endExpression + unresolvedCall(RANGE_TO, startExpression, endExpression) } implicit def byte2Literal(b: Byte): Expression = valueLiteral(b) @@ -1284,47 +502,77 @@ trait ImplicitExpressionConversions { * @see TableEnvironment#createTemporaryFunction * @see TableEnvironment#createTemporarySystemFunction */ - def call(path: String, params: Expression*): Expression = { - lookupCall(path, params: _*) - } + def call(path: String, params: Expression*): Expression = Expressions.call(path, params: _*) // ---------------------------------------------------------------------------------------------- // Implicit expressions in prefix notation // ---------------------------------------------------------------------------------------------- + /** + * Creates a SQL literal. + * + * The data type is derived from the object's class and its value. + * + * For example: + * + * - `lit(12)`` leads to `INT` + * - `lit("abc")`` leads to `CHAR(3)` + * - `lit(new java.math.BigDecimal("123.45"))` leads to `DECIMAL(5, 2)` + * + * See [[org.apache.flink.table.types.utils.ValueDataTypeConverter]] for a list of supported + * literal values. + */ + def lit(v: Any): Expression = Expressions.lit(v) + + /** + * Creates a SQL literal of a given [[DataType]]. + * + * The method [[lit(Object)]] is preferred as it extracts the [[DataType]] + * automatically. The class of `v` must be supported according to the + * [[org.apache.flink.table.types.logical.LogicalType#supportsInputConversion(Class)]]. + */ + def lit(v: Any, dataType: DataType): Expression = Expressions.lit(v, dataType) + + /** + * Returns negative numeric. + */ + def negative(v: Expression): Expression = { + Expressions.negative(v) + } + /** * Returns the current SQL date in UTC time zone. */ def currentDate(): Expression = { - unresolvedCall(CURRENT_DATE) + Expressions.currentDate() } /** * Returns the current SQL time in UTC time zone. */ def currentTime(): Expression = { - unresolvedCall(CURRENT_TIME) + Expressions.currentTime() } /** * Returns the current SQL timestamp in UTC time zone. */ def currentTimestamp(): Expression = { - unresolvedCall(CURRENT_TIMESTAMP) + Expressions.currentTimestamp() } /** * Returns the current SQL time in local time zone. */ def localTime(): Expression = { - unresolvedCall(LOCAL_TIME) + Expressions.localTime() } /** * Returns the current SQL timestamp in local time zone. */ def localTimestamp(): Expression = { - unresolvedCall(LOCAL_TIMESTAMP) + Expressions.localTimestamp() } /** @@ -1342,7 +590,7 @@ trait ImplicitExpressionConversions { rightTimePoint: Expression, rightTemporal: Expression) : Expression = { - unresolvedCall(TEMPORAL_OVERLAPS, leftTimePoint, leftTemporal, rightTimePoint, rightTemporal) + Expressions.temporalOverlaps(leftTimePoint, leftTemporal, rightTimePoint, rightTemporal) } /** @@ -1360,7 +608,7 @@ trait ImplicitExpressionConversions { timestamp: Expression, format: Expression) : Expression = { - unresolvedCall(DATE_FORMAT, timestamp, format) + Expressions.dateFormat(timestamp, format) } /** @@ -1379,49 +627,49 @@ trait ImplicitExpressionConversions { timePoint1: Expression, timePoint2: Expression) : Expression = { - unresolvedCall(TIMESTAMP_DIFF, timePointUnit, timePoint1, timePoint2) + Expressions.timestampDiff(timePointUnit, timePoint1, timePoint2) } /** * Creates an array of literals. */ def array(head: Expression, tail: Expression*): Expression = { - unresolvedCall(ARRAY, head +: tail: _*) + Expressions.array(head, tail: _*) } /** * Creates a row of expressions. */ def row(head: Expression, tail: Expression*): Expression = { - unresolvedCall(ROW, head +: tail: _*) + Expressions.row(head, tail: _*) } /** * Creates a map of expressions. */ def map(key: Expression, value: Expression, tail: Expression*): Expression = { - unresolvedCall(MAP, key +: value +: tail: _*) + Expressions.map(key, value, tail: _*) } /** * Returns a value that is closer than any other value to pi. */ def pi(): Expression = { - unresolvedCall(PI) + Expressions.pi() } /** * Returns a value that is closer than any other value to e. */ def e(): Expression = { - unresolvedCall(E) + Expressions.e() } /** * Returns a pseudorandom double value between 0.0 (inclusive) and 1.0 (exclusive). */ def rand(): Expression = { - unresolvedCall(RAND) + Expressions.rand() } /** @@ -1430,7 +678,7 @@ trait ImplicitExpressionConversions { * have same initial seed. */ def rand(seed: Expression): Expression = { - unresolvedCall(RAND, seed) + Expressions.rand(seed) } /** @@ -1438,7 +686,7 @@ trait ImplicitExpressionConversions { * value (exclusive). */ def randInteger(bound: Expression): Expression = { - unresolvedCall(RAND_INTEGER, bound) + Expressions.randInteger(bound) } /** @@ -1447,7 +695,7 @@ trait ImplicitExpressionConversions { * of numbers if they have same initial seed and same bound. */ def randInteger(seed: Expression, bound: Expression): Expression = { - unresolvedCall(RAND_INTEGER, seed, bound) + Expressions.randInteger(seed, bound) } /** @@ -1455,25 +703,38 @@ trait ImplicitExpressionConversions { * Returns NULL if any argument is NULL. */ def concat(string: Expression, strings: Expression*): Expression = { - unresolvedCall(CONCAT, string +: strings: _*) + Expressions.concat(string, strings: _*) } /** * Calculates the arc tangent of a given coordinate. */ def atan2(y: Expression, x: Expression): Expression = { - unresolvedCall(ATAN2, y, x) + Expressions.atan2(y, x) } /** * Returns the string that results from concatenating the arguments and separator. * Returns NULL If the separator is NULL. * - * Note: this user-defined function does not skip empty strings. However, it does skip any NULL + * Note: This function does not skip empty strings. However, it does skip any NULL * values after the separator argument. + * @deprecated use [[ImplicitExpressionConversions.concatWs()]] **/ + @deprecated def concat_ws(separator: Expression, string: Expression, strings: Expression*): Expression = { - unresolvedCall(CONCAT_WS, separator +: string +: strings: _*) + concatWs(separator, string, strings: _*) + } + + /** + * Returns the string that results from concatenating the arguments and separator. + * Returns NULL If the separator is NULL. + * + * Note: this user-defined function does not skip empty strings. However, it does skip any NULL + * values after the separator argument. + **/ + def concatWs(separator: Expression, string: Expression, strings: Expression*): Expression = { + Expressions.concatWs(separator, string, strings: _*) } /** @@ -1483,7 +744,7 @@ trait ImplicitExpressionConversions { * generator. */ def uuid(): Expression = { - unresolvedCall(UUID) + Expressions.uuid() } /** @@ -1492,7 +753,7 @@ trait ImplicitExpressionConversions { * e.g. nullOf(DataTypes.INT()) */ def nullOf(dataType: DataType): Expression = { - valueLiteral(null, dataType) + Expressions.nullOf(dataType) } /** @@ -1503,21 +764,21 @@ trait ImplicitExpressionConversions { * documentation for more information. */ def nullOf(typeInfo: TypeInformation[_]): Expression = { - nullOf(TypeConversions.fromLegacyInfoToDataType(typeInfo)) + Expressions.nullOf(typeInfo) } /** * Calculates the logarithm of the given value. */ def log(value: Expression): Expression = { - unresolvedCall(LOG, value) + Expressions.log(value) } /** * Calculates the logarithm of the given value to the given base. */ def log(base: Expression, value: Expression): Expression = { - unresolvedCall(LOG, base, value) + Expressions.log(base, value) } /** @@ -1531,7 +792,7 @@ trait ImplicitExpressionConversions { * @param ifFalse expression to be evaluated if condition does not hold */ def ifThenElse(condition: Expression, ifTrue: Expression, ifFalse: Expression): Expression = { - unresolvedCall(IF, condition, ifTrue, ifFalse) + Expressions.ifThenElse(condition, ifTrue, ifFalse) } /** @@ -1544,7 +805,7 @@ trait ImplicitExpressionConversions { * e.g. withColumns('b to 'c) or withColumns('*) */ def withColumns(head: Expression, tail: Expression*): Expression = { - unresolvedCall(WITH_COLUMNS, head +: tail: _*) + Expressions.withColumns(head, tail: _*) } /** @@ -1558,6 +819,20 @@ trait ImplicitExpressionConversions { * e.g. withoutColumns('b to 'c) or withoutColumns('c) */ def withoutColumns(head: Expression, tail: Expression*): Expression = { - unresolvedCall(WITHOUT_COLUMNS, head +: tail: _*) + Expressions.withoutColumns(head, tail: _*) + } + + /** + * Boolean AND in three-valued logic. + */ + def and(predicate0: Expression, predicate1: Expression, predicates: Expression*): Expression = { + Expressions.and(predicate0, predicate1, predicates: _*) + } + + /** + * Boolean OR in three-valued logic. + */ + def or(predicate0: Expression, predicate1: Expression, predicates: Expression*): Expression = { + Expressions.or(predicate0, predicate1, predicates: _*) } } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala index 06a9b5fe99..fadf85439f 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala @@ -26,7 +26,7 @@ import org.apache.flink.table.functions._ import org.apache.flink.table.planner.expressions.{E => PlannerE, UUID => PlannerUUID} import org.apache.flink.table.planner.functions.InternalFunctionDefinitions.THROW_EXCEPTION import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter.fromDataTypeToTypeInfo -import org.apache.flink.table.types.logical.LogicalTypeRoot.{CHAR, DECIMAL, SYMBOL, TIMESTAMP_WITHOUT_TIME_ZONE} +import org.apache.flink.table.types.logical.LogicalTypeRoot.{CHAR, DECIMAL, SYMBOL} import org.apache.flink.table.types.logical.utils.LogicalTypeChecks._ import _root_.scala.collection.JavaConverters._ @@ -151,12 +151,12 @@ class PlannerExpressionConverter private extends ApiExpressionVisitor[PlannerExp expr case AND => - assert(args.size == 2) - And(args.head, args.last) + assert(args.size >= 2) + args.reduceLeft(And) case OR => - assert(args.size == 2) - Or(args.head, args.last) + assert(args.size >= 2) + args.reduceLeft(Or) case NOT => assert(args.size == 1) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/expressions/PlannerExpressionConverter.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/expressions/PlannerExpressionConverter.scala index cb9f6b19e4..7b2c999254 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/expressions/PlannerExpressionConverter.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/expressions/PlannerExpressionConverter.scala @@ -133,12 +133,12 @@ class PlannerExpressionConverter private extends ApiExpressionVisitor[PlannerExp expr case AND => - assert(args.size == 2) - And(args.head, args.last) + assert(args.size >= 2) + args.reduceLeft(And) case OR => - assert(args.size == 2) - Or(args.head, args.last) + assert(args.size >= 2) + args.reduceLeft(Or) case NOT => assert(args.size == 1) -- Gitee From eb13a150576ef461bd48c53f9db0122afb82f4d9 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Tue, 11 Feb 2020 10:25:45 +0100 Subject: [PATCH 002/885] [FLINK-16033][table-api, test] Added test that check scala/java APIs completeness The test should check that methods are accessible from both JAVA and SCALA APIs or have equivalents. Checks are based on methods names. --- .../api/ExpressionsConsistencyCheckTest.scala | 293 ++++++++++++++++++ 1 file changed, 293 insertions(+) create mode 100644 flink-table/flink-table-api-scala/src/test/scala/org/apache/flink/table/api/ExpressionsConsistencyCheckTest.scala diff --git a/flink-table/flink-table-api-scala/src/test/scala/org/apache/flink/table/api/ExpressionsConsistencyCheckTest.scala b/flink-table/flink-table-api-scala/src/test/scala/org/apache/flink/table/api/ExpressionsConsistencyCheckTest.scala new file mode 100644 index 0000000000..f76009e954 --- /dev/null +++ b/flink-table/flink-table-api-scala/src/test/scala/org/apache/flink/table/api/ExpressionsConsistencyCheckTest.scala @@ -0,0 +1,293 @@ +/* + * 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.table.api + +import org.apache.flink.table.api.Expressions._ +import org.apache.flink.table.expressions.ApiExpressionUtils._ +import org.apache.flink.table.expressions.Expression +import org.apache.flink.table.functions.BuiltInFunctionDefinitions.{EQUALS, PLUS, TRIM} + +import org.hamcrest.CoreMatchers +import org.hamcrest.collection.IsEmptyIterable +import org.junit.Assert._ +import org.junit.Test + +import java.lang.reflect.Modifier + +import scala.collection.JavaConverters._ + +/** + * We test that all methods are either available or have equivalents in both Scala and Java + * expression DSL's + * + * If there are methods that do not map exactly in both APIs but have equivalent + * methods add those to `explicitScalaToJavaStaticMethodsMapping`(for static methods + * [[ImplicitExpressionConversions]]/[[Expressions]]) or `explicitScalaToJavaMapping` + * (for infix methods [[ApiExpression]]/[[ImplicitExpressionOperations]]). + * If equally named methods are not found the test will check if a mapping exists. + * This is a bidirectional mapping. + * + * If there are methods that should not have an equivalent in the other API add those to a + * corresponding list of exclude (`excludedStaticScalaMethods`, `excludedScalaMethods`, + * `excludedStaticJavaMethods`, `excludedJavaMethods`). + */ +class ExpressionsConsistencyCheckTest { + + // we cannot get class of package object + class Conversions extends ImplicitExpressionConversions {} + + // static methods from ImplicitExpressionConversions + val explicitScalaToJavaStaticMethodsMapping = Map( + "FieldExpression" -> "$", + "UnresolvedFieldExpression" -> "$", + "UserDefinedAggregateFunctionCall" -> "call", + "ScalarFunctionCall" -> "call", + "TableFunctionCall" -> "call", + "concat_ws" -> "concatWs" + ) + + // methods from WithOperations + val explicitScalaToJavaMapping = Map( + "$bang$eq$eq" -> "isNotEqual", // !== + "$eq$eq$eq" -> "isEqual", // === + "$less$eq" -> "isLessOrEqual", // <= + "$greater$eq" -> "isGreaterOrEqual", // >= + "$less" -> "isLess", // < + "$greater" -> "isGreater", // > + "$amp$amp" -> "and", // && + "$bar$bar" -> "or", // || + "$times" -> "times", // * + "$div" -> "dividedBy", // / + "$plus" -> "plus", // + + "$minus" -> "minus", // - + "$percent" -> "mod", // % + + // in scala trim has default values + "trim$default$1" -> "trimLeading", + "trim$default$2" -> "trimTrailing", + "trim$default$3" -> "trim" + ) + + val excludedStaticScalaMethods = Set( + + //----------------------------------------------------------------------------------- + // Scala implicit conversions to ImplicitExpressionOperations + //----------------------------------------------------------------------------------- + "WithOperations", + "apiExpressionToExpression", + "LiteralScalaDecimalExpression", + "LiteralJavaDecimalExpression", + "LiteralShortExpression", + "LiteralFloatExpression", + "LiteralSqlDateExpression", + "LiteralBooleanExpression", + "LiteralStringExpression", + "LiteralByteExpression", + "LiteralSqlTimestampExpression", + "LiteralLongExpression", + "LiteralDoubleExpression", + "LiteralIntExpression", + "LiteralSqlTimeExpression", + + //----------------------------------------------------------------------------------- + // Scala implicit conversions to Expressions + //----------------------------------------------------------------------------------- + "scalaRange2RangeExpression", + "scalaDec2Literal", + "double2Literal", + "sqlTime2Literal", + "symbol2FieldExpression", + "sqlTimestamp2Literal", + "localDateTime2Literal", + "localTime2Literal", + "javaDec2Literal", + "byte2Literal", + "int2Literal", + "long2Literal", + "short2Literal", + "string2Literal", + "sqlDate2Literal", + "boolean2Literal", + "localDate2Literal", + "float2Literal", + "array2ArrayConstructor", + "tableSymbolToExpression", + + //----------------------------------------------------------------------------------- + // Internal methods + //----------------------------------------------------------------------------------- + "org$apache$flink$table$api$ImplicitExpressionConversions$_setter_$CURRENT_RANGE_$eq", + "org$apache$flink$table$api$ImplicitExpressionConversions$_setter_$CURRENT_ROW_$eq", + "org$apache$flink$table$api$ImplicitExpressionConversions$_setter_$UNBOUNDED_ROW_$eq", + "org$apache$flink$table$api$ImplicitExpressionConversions$_setter_$UNBOUNDED_RANGE_$eq", + "org$apache$flink$table$api$ExpressionsConsistencyCheckTest$Conversions$$$outer" + ) + + val excludedScalaMethods = Set( + // in java we can use only static ifThenElse + "$qmark", // ? + + // in java we can use only static not + "unary_$bang", // unary_! + + // in java we can use only static range + "to", + + // in java we can use only static rowInterval + "rows", + + // users in java should use static negative() + "unary_$minus", // unary_- + + // not supported in java + "unary_$plus", // unary_+ + + //----------------------------------------------------------------------------------- + // Internal methods + //----------------------------------------------------------------------------------- + "expr", + "org$apache$flink$table$api$ImplicitExpressionConversions$WithOperations$$$outer", + "toApiSpecificExpression" + ) + + val excludedStaticJavaMethods = Set( + // in scala users should use "A" to "B" + "range", + + // users should use 1.rows, 123.millis, 3.years + "rowInterval", + + // users should use unary_- + "negative" + ) + + val excludedJavaMethods = Set( + //----------------------------------------------------------------------------------- + // Methods from Expression.java + //----------------------------------------------------------------------------------- + "accept", + "asSummaryString", + "getChildren" + ) + + @Test + def testScalaStaticMethodsAvailableInJava(): Unit = { + val scalaMethodNames = classOf[Conversions] + .getMethods + .map(_.getName) + .toSet + val javaMethodNames = classOf[Expressions].getMethods.map(_.getName).toSet ++ + classOf[Expressions].getFields.filter(f => Modifier.isStatic(f.getModifiers)).map(_.getName) + + checkMethodsMatch( + scalaMethodNames, + javaMethodNames, + explicitScalaToJavaStaticMethodsMapping, + excludedStaticScalaMethods) + } + + @Test + def testScalaExpressionMethodsAvailableInJava(): Unit = { + val scalaMethodNames = classOf[ImplicitExpressionConversions#WithOperations] + .getMethods + .map(_.getName) + .toSet + val javaMethodNames = classOf[ApiExpression].getMethods.map(_.getName).toSet + + checkMethodsMatch( + scalaMethodNames, + javaMethodNames, + explicitScalaToJavaMapping, + excludedScalaMethods) + } + + @Test + def testJavaStaticMethodsAvailableInScala(): Unit = { + val scalaMethodNames = classOf[Conversions].getMethods.map(_.getName).toSet + val javaMethodNames = classOf[Expressions].getMethods.map(_.getName).toSet + + checkMethodsMatch( + javaMethodNames, + scalaMethodNames, + explicitScalaToJavaStaticMethodsMapping.map(_.swap), + excludedStaticJavaMethods) + } + + @Test + def testJavaExpressionMethodsAvailableInScala(): Unit = { + val scalaMethodNames = classOf[ImplicitExpressionConversions#WithOperations] + .getMethods + .map(_.getName) + .toSet + val javaMethodNames = classOf[ApiExpression].getMethods.map(_.getName).toSet + + checkMethodsMatch( + javaMethodNames, + scalaMethodNames, + explicitScalaToJavaMapping, + excludedJavaMethods) + } + + @Test + def testInteroperability(): Unit = { + // In most cases it should be just fine to mix the two APIs. + // It should be discouraged though as it might have unforeseen side effects + object Conversions extends ImplicitExpressionConversions + import Conversions._ + val expr = lit("ABC") === $"f0".plus($("f1")).trim() + + assertThat( + expr, + CoreMatchers.equalTo[Expression]( + unresolvedCall( + EQUALS, + valueLiteral("ABC"), + unresolvedCall( + TRIM, + valueLiteral(true), + valueLiteral(true), + valueLiteral(" "), + unresolvedCall( + PLUS, + unresolvedRef("f0"), + unresolvedRef("f1") + ) + ) + ) + ) + ) + } + + private def checkMethodsMatch( + checkedMethods: Set[String], + methodsBeingCheckedAgainst: Set[String], + methodsMapping: Map[String, String], + excludedMethods: Set[String]) + : Unit = { + val missingMethods = (checkedMethods -- methodsBeingCheckedAgainst) + .filterNot( + scalaName => { + val mappedName = methodsMapping.getOrElse(scalaName, scalaName) + methodsBeingCheckedAgainst.contains(mappedName) + } + ).diff(excludedMethods) + + assertThat(missingMethods.asJava, IsEmptyIterable.emptyIterableOf(classOf[String])) + } +} -- Gitee From 344477b923e54dc25955d1ae517d002abd8250d3 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Tue, 11 Feb 2020 10:49:36 +0100 Subject: [PATCH 003/885] [FLINK-16033][table-api] Updated a test to use java expression dsl --- .../table/JavaTableEnvironmentITCase.java | 27 ++++++++++--------- 1 file changed, 14 insertions(+), 13 deletions(-) diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/batch/table/JavaTableEnvironmentITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/batch/table/JavaTableEnvironmentITCase.java index 177cf1a678..e362027c48 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/batch/table/JavaTableEnvironmentITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/batch/table/JavaTableEnvironmentITCase.java @@ -52,6 +52,7 @@ import java.util.Collection; import java.util.HashMap; import java.util.List; +import static org.apache.flink.table.api.Expressions.$; import static org.junit.Assert.assertTrue; /** @@ -103,7 +104,7 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase tableEnv.registerDataSet(tableName, ds); Table t = tableEnv.scan(tableName); - Table result = t.select("f0, f1"); + Table result = t.select($("f0"), $("f1")); DataSet resultSet = tableEnv.toDataSet(result, Row.class); List results = resultSet.collect(); @@ -123,7 +124,7 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase tableEnv.registerDataSet(tableName, ds, "a, b, c"); Table t = tableEnv.scan(tableName); - Table result = t.select("a, b, c"); + Table result = t.select($("a"), $("b"), $("c")); DataSet resultSet = tableEnv.toDataSet(result, Row.class); List results = resultSet.collect(); @@ -168,7 +169,7 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase DataSet> ds = CollectionDataSets.get3TupleDataSet(env); Table t = tableEnv.fromDataSet(ds); tableEnv.registerTable(tableName, t); - Table result = tableEnv.scan(tableName).select("f0, f1").filter("f0 > 7"); + Table result = tableEnv.scan(tableName).select($("f0"), $("f1")).filter($("f0").isGreater(7)); DataSet resultSet = tableEnv.toDataSet(result, Row.class); List results = resultSet.collect(); @@ -196,7 +197,7 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase Table table = tableEnv .fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a, b, c") - .select("a, b, c"); + .select($("a"), $("b"), $("c")); DataSet ds = tableEnv.toDataSet(table, Row.class); List results = ds.collect(); @@ -236,7 +237,7 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase Table table = tableEnv .fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a, b, c") - .select("a, b, c"); + .select($("a"), $("b"), $("c")); TypeInformation ti = new TupleTypeInfo>( BasicTypeInfo.INT_TYPE_INFO, @@ -268,7 +269,7 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase Table table = tableEnv .fromDataSet(env.fromCollection(data), "q, w, e, r") - .select("q as a, w as b, e as c, r as d"); + .select($("q").as("a"), $("w").as("b"), $("e").as("c"), $("r").as("d")); DataSet ds = tableEnv.toDataSet(table, SmallPojo2.class); List results = ds.collect(); @@ -293,7 +294,7 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase "salary AS c, " + "name AS d," + "roles as e") - .select("a, b, c, d, e"); + .select($("a"), $("b"), $("c"), $("d"), $("e")); DataSet ds = tableEnv.toDataSet(table, Row.class); List results = ds.collect(); @@ -344,7 +345,7 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase Table table = tableEnv .fromDataSet(env.fromCollection(data), "name AS d") - .select("d"); + .select($("d")); DataSet ds = tableEnv.toDataSet(table, Row.class); List results = ds.collect(); @@ -371,7 +372,7 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase "age AS b, " + "salary AS c, " + "name AS d") - .select("a, b, c, d"); + .select($("a"), $("b"), $("c"), $("d")); DataSet ds = tableEnv.toDataSet(table, Row.class); List results = ds.collect(); @@ -399,7 +400,7 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase "salary AS c, " + "name AS d," + "roles AS e") - .select("a, b, c, d, e"); + .select($("a"), $("b"), $("c"), $("d"), $("e")); DataSet ds = tableEnv.toDataSet(table, SmallPojo2.class); List results = ds.collect(); @@ -426,7 +427,7 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase "age AS b, " + "salary AS c, " + "name AS d") - .select("a, b, c, d"); + .select($("a"), $("b"), $("c"), $("d")); DataSet ds = tableEnv.toDataSet(table, PrivateSmallPojo2.class); List results = ds.collect(); @@ -457,8 +458,8 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase "age AS b, " + "generic AS c, " + "generic2 AS d") - .select("a, b, c, c as c2, d") - .select("a, b, c, c === c2, d"); + .select($("a"), $("b"), $("c"), $("c").as("c2"), $("d")) + .select($("a"), $("b"), $("c"), $("c").isEqual($("c2")), $("d")); DataSet ds = tableEnv.toDataSet(table, Row.class); List results = ds.collect(); -- Gitee From 50dcae2cd1af369521441508950a19d29566bcb8 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Thu, 20 Feb 2020 12:27:54 +0100 Subject: [PATCH 004/885] [FLINK-16033][table-api] Added test for expression resolution. The added test uses expressionDsl to create unresolved expressions that are later being resolved using ExpressionResolver. It covers constructing expected expressions as well as resolver rules. This closes #11081 --- .../resolver/ExpressionResolverTest.java | 339 ++++++++++++++++++ 1 file changed, 339 insertions(+) create mode 100644 flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/expressions/resolver/ExpressionResolverTest.java diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/expressions/resolver/ExpressionResolverTest.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/expressions/resolver/ExpressionResolverTest.java new file mode 100644 index 0000000000..02cb57f3ad --- /dev/null +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/expressions/resolver/ExpressionResolverTest.java @@ -0,0 +1,339 @@ +/* + * 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.table.expressions.resolver; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableConfig; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.FunctionLookup; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.UnresolvedIdentifier; +import org.apache.flink.table.delegation.PlannerTypeInferenceUtil; +import org.apache.flink.table.expressions.CallExpression; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.FieldReferenceExpression; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.expressions.ValueLiteralExpression; +import org.apache.flink.table.functions.BuiltInFunctionDefinition; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.flink.table.functions.FunctionIdentifier; +import org.apache.flink.table.functions.ScalarFunction; +import org.apache.flink.table.functions.ScalarFunctionDefinition; +import org.apache.flink.table.operations.CatalogQueryOperation; +import org.apache.flink.table.operations.QueryOperation; +import org.apache.flink.table.types.inference.TypeInferenceUtil; +import org.apache.flink.table.types.inference.utils.DataTypeFactoryMock; +import org.apache.flink.table.types.utils.TypeConversions; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +import static org.apache.flink.table.api.Expressions.$; +import static org.apache.flink.table.api.Expressions.call; +import static org.apache.flink.table.api.Expressions.range; +import static org.apache.flink.table.api.Expressions.withColumns; +import static org.apache.flink.table.expressions.ApiExpressionUtils.valueLiteral; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.junit.Assert.assertThat; + +/** + * This test supports only a subset of builtin functions because those functions still depend on + * planner expressions for argument validation and type inference. Supported builtin functions are: + * + *

- BuiltinFunctionDefinitions.EQUALS + * - BuiltinFunctionDefinitions.IS_NULL + * + *

Pseudo functions that are executed during expression resolution e.g.: + * - BuiltinFunctionDefinitions.WITH_COLUMNS + * - BuiltinFunctionDefinitions.WITHOUT_COLUMNS + * - BuiltinFunctionDefinitions.RANGE_TO + * - BuiltinFunctionDefinitions.FLATTEN + * + *

This test supports only a simplified identifier parsing logic. It does not support escaping. + * It just naively splits on dots. The proper logic comes with a planner implementation which is not + * available in the API module. + */ +@RunWith(Parameterized.class) +public class ExpressionResolverTest { + + @Parameterized.Parameters(name = "{0}") + public static Collection parameters() { + return Arrays.asList( + TestSpec.test("Columns range") + .inputSchemas( + TableSchema.builder() + .field("f0", DataTypes.BIGINT()) + .field("f1", DataTypes.STRING()) + .field("f2", DataTypes.SMALLINT()) + .build() + ) + .select(withColumns(range("f1", "f2")), withColumns(range(1, 2))) + .equalTo( + new FieldReferenceExpression("f1", DataTypes.STRING(), 0, 1), + new FieldReferenceExpression("f2", DataTypes.SMALLINT(), 0, 2), + new FieldReferenceExpression("f0", DataTypes.BIGINT(), 0, 0), + new FieldReferenceExpression("f1", DataTypes.STRING(), 0, 1) + ), + + TestSpec.test("Flatten call") + .inputSchemas( + TableSchema.builder() + .field("f0", DataTypes.ROW( + DataTypes.FIELD("n0", DataTypes.BIGINT()), + DataTypes.FIELD("n1", DataTypes.STRING()) + )) + .build() + ) + .select($("f0").flatten()) + .equalTo( + new CallExpression( + FunctionIdentifier.of("get"), + BuiltInFunctionDefinitions.GET, + Arrays.asList( + new FieldReferenceExpression("f0", DataTypes.ROW( + DataTypes.FIELD("n0", DataTypes.BIGINT()), + DataTypes.FIELD("n1", DataTypes.STRING()) + ), 0, 0), + new ValueLiteralExpression("n0") + ), + DataTypes.BIGINT() + ), + new CallExpression( + FunctionIdentifier.of("get"), + BuiltInFunctionDefinitions.GET, + Arrays.asList( + new FieldReferenceExpression("f0", DataTypes.ROW( + DataTypes.FIELD("n0", DataTypes.BIGINT()), + DataTypes.FIELD("n1", DataTypes.STRING()) + ), 0, 0), + new ValueLiteralExpression("n1") + ), + DataTypes.STRING() + )), + + TestSpec.test("Builtin function calls") + .inputSchemas( + TableSchema.builder() + .field("f0", DataTypes.INT()) + .field("f1", DataTypes.STRING()) + .build() + ) + .select($("f0").isEqual($("f1"))) + .equalTo(new CallExpression( + FunctionIdentifier.of("equals"), + BuiltInFunctionDefinitions.EQUALS, + Arrays.asList( + new FieldReferenceExpression("f0", DataTypes.INT(), 0, 0), + new FieldReferenceExpression("f1", DataTypes.STRING(), 0, 1) + ), + DataTypes.BOOLEAN() + )), + + TestSpec.test("Lookup system function call") + .inputSchemas( + TableSchema.builder() + .field("f0", DataTypes.INT()) + .build() + ) + .lookupFunction("func", new ScalarFunctionDefinition("func", new ScalarFunc())) + .select(call("func", 1, $("f0"))) + .equalTo(new CallExpression( + FunctionIdentifier.of("func"), + new ScalarFunctionDefinition("func", new ScalarFunc()), + Arrays.asList(valueLiteral(1), new FieldReferenceExpression("f0", DataTypes.INT(), 0, 0)), + DataTypes.INT().bridgedTo(Integer.class) + ))); + } + + @Parameterized.Parameter + public TestSpec testSpec; + + @Test + public void testResolvingExpressions() { + List resolvedExpressions = testSpec.getResolver() + .resolve(Arrays.asList(testSpec.expressions)); + assertThat( + resolvedExpressions, + equalTo(testSpec.expectedExpressions)); + } + + /** + * Test scalar function that uses legacy type inference logic. + */ + public static class ScalarFunc extends ScalarFunction { + public int eval(Object... any) { + return 0; + } + + @Override + public TypeInformation getResultType(Class[] signature) { + return Types.INT; + } + + @Override + public int hashCode() { + return 0; + } + + @Override + public boolean equals(Object obj) { + return obj instanceof ScalarFunc; + } + } + + private static class TestSpec { + private final String description; + private TableSchema[] schemas; + private Expression[] expressions; + private List expectedExpressions; + private Map functions = new HashMap<>(); + + private TestSpec(String description) { + this.description = description; + } + + public static TestSpec test(String description) { + return new TestSpec(description); + } + + public TestSpec inputSchemas(TableSchema... schemas) { + this.schemas = schemas; + return this; + } + + public TestSpec lookupFunction(String name, FunctionDefinition functionDefinition) { + functions.put(FunctionIdentifier.of(name), functionDefinition); + return this; + } + + public TestSpec lookupFunction(ObjectIdentifier identifier, FunctionDefinition functionDefinition) { + functions.put(FunctionIdentifier.of(identifier), functionDefinition); + return this; + } + + public TestSpec select(Expression... expressions) { + this.expressions = expressions; + return this; + } + + public TestSpec equalTo(ResolvedExpression... resolvedExpressions) { + this.expectedExpressions = Arrays.asList(resolvedExpressions); + return this; + } + + public ExpressionResolver getResolver() { + FunctionLookup functionLookup = new FunctionLookup() { + @Override + public Optional lookupFunction(String stringIdentifier) { + // this is a simplified version for the test + return lookupFunction(UnresolvedIdentifier.of(stringIdentifier.split("\\."))); + } + + @Override + public Optional lookupFunction(UnresolvedIdentifier identifier) { + final FunctionIdentifier functionIdentifier; + if (identifier.getCatalogName().isPresent() && identifier.getDatabaseName().isPresent()) { + functionIdentifier = FunctionIdentifier.of( + ObjectIdentifier.of( + identifier.getCatalogName().get(), + identifier.getDatabaseName().get(), + identifier.getObjectName())); + } else { + functionIdentifier = FunctionIdentifier.of(identifier.getObjectName()); + } + + return Optional.ofNullable(functions.get(functionIdentifier)) + .map(func -> new Result(functionIdentifier, func)); + } + + @Override + public Result lookupBuiltInFunction(BuiltInFunctionDefinition definition) { + return new Result( + FunctionIdentifier.of(definition.getName()), + definition + ); + } + + @Override + public PlannerTypeInferenceUtil getPlannerTypeInferenceUtil() { + return (unresolvedCall, resolvedArgs) -> { + FunctionDefinition functionDefinition = unresolvedCall.getFunctionDefinition(); + if (functionDefinition.equals(BuiltInFunctionDefinitions.EQUALS)) { + return new TypeInferenceUtil.Result( + resolvedArgs.stream() + .map(ResolvedExpression::getOutputDataType) + .collect(Collectors.toList()), + null, + DataTypes.BOOLEAN() + ); + } else if (functionDefinition.equals(BuiltInFunctionDefinitions.IS_NULL)) { + return new TypeInferenceUtil.Result( + resolvedArgs.stream() + .map(ResolvedExpression::getOutputDataType) + .collect(Collectors.toList()), + null, + DataTypes.BOOLEAN() + ); + } else if (functionDefinition instanceof ScalarFunctionDefinition) { + return new TypeInferenceUtil.Result( + resolvedArgs.stream() + .map(ResolvedExpression::getOutputDataType) + .collect(Collectors.toList()), + null, + // We do not support a full legacy type inference here. We support only a static result + // type + TypeConversions.fromLegacyInfoToDataType(((ScalarFunctionDefinition) functionDefinition) + .getScalarFunction() + .getResultType(null))); + } + + throw new IllegalArgumentException( + "Unsupported builtin function in the test: " + unresolvedCall); + }; + } + }; + return ExpressionResolver.resolverFor( + new TableConfig(), + name -> Optional.empty(), + functionLookup, + new DataTypeFactoryMock(), + Arrays.stream(schemas) + .map(schema -> (QueryOperation) new CatalogQueryOperation(ObjectIdentifier.of("", "", ""), schema)) + .toArray(QueryOperation[]::new) + ).build(); + } + + @Override + public String toString() { + return description; + } + } +} -- Gitee From 569468f729e118ab93547d84327ec846cc2646c8 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Wed, 26 Feb 2020 13:20:59 +0800 Subject: [PATCH 005/885] [FLINK-16285][network] Refactor SingleInputGate#setInputChannel to remove IntermediateResultPartitionID argument The IntermediateResultPartitionID info can be got directly from the respective InputChannel, so we can remove it from the arguments of SingleInputGate#setInputChannel to cleanup the codes. It is also helpful to simplify the unit tests and avoid passing the inconsistent IntermediateResultPartitionID with the internal ResultPartitionID that the respective InputChannel maintains. --- .../partition/consumer/SingleInputGate.java | 6 +++-- .../consumer/SingleInputGateFactory.java | 3 +-- .../consumer/InputChannelBuilder.java | 6 ++--- .../IteratorWrappingTestSingleInputGate.java | 3 +-- .../consumer/LocalInputChannelTest.java | 2 +- .../consumer/SingleInputGateTest.java | 22 +++++++------------ .../partition/consumer/TestInputChannel.java | 3 +-- .../consumer/TestSingleInputGate.java | 4 +--- .../consumer/UnionInputGateTest.java | 10 ++++----- .../consumer/StreamTestSingleInputGate.java | 4 +--- 10 files changed, 25 insertions(+), 38 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java index f03e347078..677b320630 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java @@ -318,9 +318,11 @@ public class SingleInputGate extends InputGate { } } - public void setInputChannel(IntermediateResultPartitionID partitionId, InputChannel inputChannel) { + public void setInputChannel(InputChannel inputChannel) { + checkNotNull(inputChannel); + IntermediateResultPartitionID partitionId = inputChannel.getPartitionId().getPartitionId(); synchronized (requestLock) { - if (inputChannels.put(checkNotNull(partitionId), checkNotNull(inputChannel)) == null + if (inputChannels.put(partitionId, inputChannel) == null && inputChannel instanceof UnknownInputChannel) { numberOfUninitializedChannels++; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java index dca505d727..a6d36418c8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java @@ -156,8 +156,7 @@ public class SingleInputGateFactory { shuffleDescriptors[i], channelStatistics, metrics); - ResultPartitionID resultPartitionID = inputChannels[i].getPartitionId(); - inputGate.setInputChannel(resultPartitionID.getPartitionId(), inputChannels[i]); + inputGate.setInputChannel(inputChannels[i]); } LOG.debug("{}: Created {} input channels ({}).", diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelBuilder.java index d8e9cc23ce..5709ffc51c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelBuilder.java @@ -120,7 +120,7 @@ public class InputChannelBuilder { maxBackoff, metrics, memorySegmentProvider); - inputGate.setInputChannel(partitionId.getPartitionId(), channel); + inputGate.setInputChannel(channel); return channel; } @@ -134,7 +134,7 @@ public class InputChannelBuilder { initialBackoff, maxBackoff, metrics); - inputGate.setInputChannel(partitionId.getPartitionId(), channel); + inputGate.setInputChannel(channel); return channel; } @@ -149,7 +149,7 @@ public class InputChannelBuilder { maxBackoff, metrics, memorySegmentProvider); - inputGate.setInputChannel(partitionId.getPartitionId(), channel); + inputGate.setInputChannel(channel); return channel; } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/IteratorWrappingTestSingleInputGate.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/IteratorWrappingTestSingleInputGate.java index 4cbc6c8b23..7de40343a6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/IteratorWrappingTestSingleInputGate.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/IteratorWrappingTestSingleInputGate.java @@ -27,7 +27,6 @@ import org.apache.flink.runtime.io.network.buffer.BufferBuilder; import org.apache.flink.runtime.io.network.buffer.BufferConsumer; import org.apache.flink.runtime.io.network.partition.consumer.InputChannel.BufferAndAvailability; import org.apache.flink.runtime.io.network.partition.consumer.TestInputChannel.BufferAndAvailabilityProvider; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.MutableObjectIterator; @@ -96,7 +95,7 @@ public class IteratorWrappingTestSingleInputGate e inputChannel.addBufferAndAvailability(answer); - inputGate.setInputChannel(new IntermediateResultPartitionID(), inputChannel); + inputGate.setInputChannel(inputChannel); return this; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java index 44d3e23993..9eca591473 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java @@ -286,7 +286,7 @@ public class LocalInputChannelTest { final LocalInputChannel localChannel = createLocalInputChannel( inputGate, new ResultPartitionManager(), 1, 1); - inputGate.setInputChannel(localChannel.getPartitionId().getPartitionId(), localChannel); + inputGate.setInputChannel(localChannel); localChannel.requestSubpartition(0); // The timer should be initialized at the first time of retriggering partition request. diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java index 3de27eb8cf..5606f9e927 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java @@ -96,11 +96,8 @@ public class SingleInputGateTest extends InputGateTestBase { new TestInputChannel(inputGate, 1) }; - inputGate.setInputChannel( - new IntermediateResultPartitionID(), inputChannels[0]); - - inputGate.setInputChannel( - new IntermediateResultPartitionID(), inputChannels[1]); + inputGate.setInputChannel(inputChannels[0]); + inputGate.setInputChannel(inputChannels[1]); // Test inputChannels[0].readBuffer(); @@ -149,7 +146,7 @@ public class SingleInputGateTest extends InputGateTestBase { assertTrue(compressedBuffer.isCompressed()); inputChannel.read(compressedBuffer); - inputGate.setInputChannel(new IntermediateResultPartitionID(), inputChannel); + inputGate.setInputChannel(inputChannel); inputGate.notifyChannelNonEmpty(inputChannel); Optional bufferOrEvent = inputGate.getNext(); @@ -166,7 +163,7 @@ public class SingleInputGateTest extends InputGateTestBase { public void testIsAvailable() throws Exception { final SingleInputGate inputGate = createInputGate(1); TestInputChannel inputChannel = new TestInputChannel(inputGate, 0); - inputGate.setInputChannel(new IntermediateResultPartitionID(), inputChannel); + inputGate.setInputChannel(inputChannel); testIsAvailable(inputGate, inputGate, inputChannel); } @@ -175,7 +172,7 @@ public class SingleInputGateTest extends InputGateTestBase { public void testIsAvailableAfterFinished() throws Exception { final SingleInputGate inputGate = createInputGate(1); TestInputChannel inputChannel = new TestInputChannel(inputGate, 0); - inputGate.setInputChannel(new IntermediateResultPartitionID(), inputChannel); + inputGate.setInputChannel(inputChannel); testIsAvailableAfterFinished( inputGate, @@ -195,11 +192,8 @@ public class SingleInputGateTest extends InputGateTestBase { new TestInputChannel(inputGate, 1) }; - inputGate.setInputChannel( - new IntermediateResultPartitionID(), inputChannels[0]); - - inputGate.setInputChannel( - new IntermediateResultPartitionID(), inputChannels[1]); + inputGate.setInputChannel(inputChannels[0]); + inputGate.setInputChannel(inputChannels[1]); // Test inputChannels[0].readBuffer(); @@ -627,7 +621,7 @@ public class SingleInputGateTest extends InputGateTestBase { final LocalInputChannel localChannel = createLocalInputChannel(inputGate, new ResultPartitionManager()); final ResultPartitionID partitionId = localChannel.getPartitionId(); - inputGate.setInputChannel(partitionId.getPartitionId(), localChannel); + inputGate.setInputChannel(localChannel); localChannel.setError(new PartitionNotFoundException(partitionId)); try { inputGate.getNext(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java index 111dae8aae..3e163273ef 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java @@ -24,7 +24,6 @@ import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import java.io.IOException; import java.util.ArrayList; @@ -113,7 +112,7 @@ public class TestInputChannel extends InputChannel { for (int i = 0; i < numberOfInputChannels; i++) { mocks[i] = new TestInputChannel(inputGate, i); - inputGate.setInputChannel(new IntermediateResultPartitionID(), mocks[i]); + inputGate.setInputChannel(mocks[i]); } return mocks; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestSingleInputGate.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestSingleInputGate.java index f60cdb9073..60d6aa6a02 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestSingleInputGate.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestSingleInputGate.java @@ -18,8 +18,6 @@ package org.apache.flink.runtime.io.network.partition.consumer; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; - import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createSingleInputGate; import static org.apache.flink.util.Preconditions.checkArgument; @@ -41,7 +39,7 @@ public class TestSingleInputGate { if (initialize) { for (int i = 0; i < numberOfInputChannels; i++) { inputChannels[i] = new TestInputChannel(inputGate, i); - inputGate.setInputChannel(new IntermediateResultPartitionID(), inputChannels[i]); + inputGate.setInputChannel(inputChannels[i]); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java index 1892836307..16ad4cc849 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java @@ -18,8 +18,6 @@ package org.apache.flink.runtime.io.network.partition.consumer; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; - import org.junit.Test; import static org.apache.flink.runtime.io.network.partition.consumer.SingleInputGateTest.verifyBufferOrEvent; @@ -107,11 +105,11 @@ public class UnionInputGateTest extends InputGateTestBase { public void testIsAvailable() throws Exception { final SingleInputGate inputGate1 = createInputGate(1); TestInputChannel inputChannel1 = new TestInputChannel(inputGate1, 0); - inputGate1.setInputChannel(new IntermediateResultPartitionID(), inputChannel1); + inputGate1.setInputChannel(inputChannel1); final SingleInputGate inputGate2 = createInputGate(1); TestInputChannel inputChannel2 = new TestInputChannel(inputGate2, 0); - inputGate2.setInputChannel(new IntermediateResultPartitionID(), inputChannel2); + inputGate2.setInputChannel(inputChannel2); testIsAvailable(new UnionInputGate(inputGate1, inputGate2), inputGate1, inputChannel1); } @@ -120,11 +118,11 @@ public class UnionInputGateTest extends InputGateTestBase { public void testIsAvailableAfterFinished() throws Exception { final SingleInputGate inputGate1 = createInputGate(1); TestInputChannel inputChannel1 = new TestInputChannel(inputGate1, 0); - inputGate1.setInputChannel(new IntermediateResultPartitionID(), inputChannel1); + inputGate1.setInputChannel(inputChannel1); final SingleInputGate inputGate2 = createInputGate(1); TestInputChannel inputChannel2 = new TestInputChannel(inputGate2, 0); - inputGate2.setInputChannel(new IntermediateResultPartitionID(), inputChannel2); + inputGate2.setInputChannel(inputChannel2); testIsAvailableAfterFinished( new UnionInputGate(inputGate1, inputGate2), diff --git a/flink-streaming-java/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java b/flink-streaming-java/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java index 4ec6b42de1..c35906021d 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java @@ -30,7 +30,6 @@ import org.apache.flink.runtime.io.network.buffer.BufferBuilder; import org.apache.flink.runtime.io.network.buffer.BufferConsumer; import org.apache.flink.runtime.io.network.partition.consumer.InputChannel.BufferAndAvailability; import org.apache.flink.runtime.io.network.partition.consumer.TestInputChannel.BufferAndAvailabilityProvider; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.plugable.SerializationDelegate; import org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer; @@ -124,8 +123,7 @@ public class StreamTestSingleInputGate extends TestSingleInputGate { inputChannels[channelIndex].addBufferAndAvailability(answer); - inputGate.setInputChannel(new IntermediateResultPartitionID(), - inputChannels[channelIndex]); + inputGate.setInputChannel(inputChannels[channelIndex]); } } -- Gitee From d4ca6ea2b0061c758f49c1edef562348eb96c9d0 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Thu, 27 Feb 2020 12:18:12 +0800 Subject: [PATCH 006/885] [hotfix][tests] Remove the dead codes of StreamTestSingleInputGate and TestInputChannel --- .../network/partition/consumer/TestInputChannel.java | 12 ------------ .../consumer/StreamTestSingleInputGate.java | 8 -------- 2 files changed, 20 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java index 3e163273ef..d6a2976c5e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java @@ -120,7 +120,6 @@ public class TestInputChannel extends InputChannel { @Override void requestSubpartition(int subpartitionIndex) throws IOException, InterruptedException { - } @Override @@ -141,7 +140,6 @@ public class TestInputChannel extends InputChannel { @Override void sendTaskEvent(TaskEvent event) throws IOException { - } @Override @@ -155,26 +153,16 @@ public class TestInputChannel extends InputChannel { @Override void releaseAllResources() throws IOException { - } @Override protected void notifyChannelNonEmpty() { - - } - - public void assertReturnedDataBuffersAreRecycled() { - assertReturnedBuffersAreRecycled(true, false); } public void assertReturnedEventsAreRecycled() { assertReturnedBuffersAreRecycled(false, true); } - public void assertAllReturnedBuffersAreRecycled() { - assertReturnedBuffersAreRecycled(true, true); - } - private void assertReturnedBuffersAreRecycled(boolean assertBuffers, boolean assertEvents) { for (Buffer b : allReturnedBuffers) { if (b.isBuffer() && assertBuffers && !b.isRecycled()) { diff --git a/flink-streaming-java/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java b/flink-streaming-java/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java index c35906021d..8f297cf063 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java @@ -157,14 +157,6 @@ public class StreamTestSingleInputGate extends TestSingleInputGate { * Returns true iff all input queues are empty. */ public boolean allQueuesEmpty() { -// for (int i = 0; i < numInputChannels; i++) { -// synchronized (inputQueues[i]) { -// inputQueues[i].add(InputValue.event(new DummyEvent())); -// inputQueues[i].notifyAll(); -// inputGate.onAvailableBuffer(inputChannels[i].getInputChannel()); -// } -// } - for (int i = 0; i < numInputChannels; i++) { if (inputQueues[i].size() > 0) { return false; -- Gitee From da7a6888cbee26f3e7ebc4957ea8d9993c0b53f8 Mon Sep 17 00:00:00 2001 From: testfixer <60407054+testfixer@users.noreply.github.com> Date: Mon, 2 Mar 2020 00:12:52 -0600 Subject: [PATCH 007/885] [FLINK-16352][table-common] Changing HashMap to LinkedHashMap for deterministic iterations in ExpressionTest (#11269) --- .../org/apache/flink/table/expressions/ExpressionTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/expressions/ExpressionTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/expressions/ExpressionTest.java index 3a0abc8a4e..730ee6ee5b 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/expressions/ExpressionTest.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/expressions/ExpressionTest.java @@ -32,7 +32,7 @@ import java.nio.charset.StandardCharsets; import java.sql.Timestamp; import java.time.LocalDateTime; import java.time.Period; -import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.Map; import static java.util.Arrays.asList; @@ -112,7 +112,7 @@ public class ExpressionTest { DataTypes.ARRAY(DataTypes.ARRAY(DataTypes.STRING()))) .toString()); - final Map map = new HashMap<>(); + final Map map = new LinkedHashMap<>(); map.put("key1", 1); map.put("key2", 2); map.put("key3", 3); -- Gitee From d9b9aad6c92bc4f65bae04821a041082b3ed6cf8 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Mon, 2 Mar 2020 17:02:54 +0800 Subject: [PATCH 008/885] [FLINK-16360][orc] Flink STRING data type should map to ORC STRING type (#11277) Hive 2.0 ORC not support schema evolution from STRING to VARCHAR. We need produce STRING in ORC for VarcharType(MAX_LENGHT) in Flink. --- .../main/java/org/apache/flink/orc/OrcSplitReaderUtil.java | 7 ++++++- .../java/org/apache/flink/orc/OrcSplitReaderUtilTest.java | 3 ++- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/OrcSplitReaderUtil.java b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/OrcSplitReaderUtil.java index 2a82979766..aafc1bef1c 100644 --- a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/OrcSplitReaderUtil.java +++ b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/OrcSplitReaderUtil.java @@ -140,7 +140,12 @@ public class OrcSplitReaderUtil { case CHAR: return TypeDescription.createChar().withMaxLength(((CharType) type).getLength()); case VARCHAR: - return TypeDescription.createVarchar().withMaxLength(((VarCharType) type).getLength()); + int len = ((VarCharType) type).getLength(); + if (len == VarCharType.MAX_LENGTH) { + return TypeDescription.createString(); + } else { + return TypeDescription.createVarchar().withMaxLength(len); + } case BOOLEAN: return TypeDescription.createBoolean(); case VARBINARY: diff --git a/flink-formats/flink-orc/src/test/java/org/apache/flink/orc/OrcSplitReaderUtilTest.java b/flink-formats/flink-orc/src/test/java/org/apache/flink/orc/OrcSplitReaderUtilTest.java index d6f1459313..7b84f1e9cc 100644 --- a/flink-formats/flink-orc/src/test/java/org/apache/flink/orc/OrcSplitReaderUtilTest.java +++ b/flink-formats/flink-orc/src/test/java/org/apache/flink/orc/OrcSplitReaderUtilTest.java @@ -36,6 +36,7 @@ public class OrcSplitReaderUtilTest { test("boolean", DataTypes.BOOLEAN()); test("char(123)", DataTypes.CHAR(123)); test("varchar(123)", DataTypes.VARCHAR(123)); + test("string", DataTypes.STRING()); test("binary", DataTypes.BYTES()); test("tinyint", DataTypes.TINYINT()); test("smallint", DataTypes.SMALLINT()); @@ -47,7 +48,7 @@ public class OrcSplitReaderUtilTest { test("timestamp", DataTypes.TIMESTAMP()); test("array", DataTypes.ARRAY(DataTypes.FLOAT())); test("map", DataTypes.MAP(DataTypes.FLOAT(), DataTypes.BIGINT())); - test("struct>", + test("struct>", DataTypes.ROW( DataTypes.FIELD("int0", DataTypes.INT()), DataTypes.FIELD("str1", DataTypes.STRING()), -- Gitee From f0bdf3179c1deff41eaaccb81544f19150519c30 Mon Sep 17 00:00:00 2001 From: huangxingbo Date: Thu, 27 Feb 2020 21:24:59 +0800 Subject: [PATCH 009/885] [FLINK-16007][table-planner][table-planner-blink][python] Add PythonCorrelateSplitRule to push down the Java Calls contained in Python Correlate node This closes #11242. --- .../logical/PythonCorrelateSplitRule.java | 238 +++++++++++++++++ .../plan/rules/FlinkStreamRuleSets.scala | 2 + .../stream/StreamExecCorrelateRule.scala | 5 +- .../logical/PythonCorrelateSplitRuleTest.xml | 41 +++ .../PythonCorrelateSplitRuleTest.scala | 66 +++++ .../logical/PythonCorrelateSplitRule.java | 245 ++++++++++++++++++ .../table/plan/rules/FlinkRuleSets.scala | 2 + .../flink/table/plan/util/CorrelateUtil.scala | 5 +- .../plan/PythonCorrelateSplitRuleTest.scala | 62 +++++ 9 files changed, 664 insertions(+), 2 deletions(-) create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PythonCorrelateSplitRule.java create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PythonCorrelateSplitRuleTest.xml create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCorrelateSplitRuleTest.scala create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/plan/rules/logical/PythonCorrelateSplitRule.java create mode 100644 flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/PythonCorrelateSplitRuleTest.scala diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PythonCorrelateSplitRule.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PythonCorrelateSplitRule.java new file mode 100644 index 0000000000..cf6421029a --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PythonCorrelateSplitRule.java @@ -0,0 +1,238 @@ +/* + * 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.table.planner.plan.rules.logical; + +import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalCalc; +import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalCorrelate; +import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalTableFunctionScan; +import org.apache.flink.table.planner.plan.rules.physical.stream.StreamExecCorrelateRule; +import org.apache.flink.table.planner.plan.utils.PythonUtil; + +import org.apache.calcite.plan.RelOptRule; +import org.apache.calcite.plan.RelOptRuleCall; +import org.apache.calcite.plan.hep.HepRelVertex; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rex.RexBuilder; +import org.apache.calcite.rex.RexCall; +import org.apache.calcite.rex.RexInputRef; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexProgram; +import org.apache.calcite.rex.RexProgramBuilder; +import org.apache.calcite.sql.validate.SqlValidatorUtil; + +import java.util.LinkedList; +import java.util.List; +import java.util.stream.Collectors; + +import scala.collection.Iterator; +import scala.collection.mutable.ArrayBuffer; + +/** + * Rule will split the Python {@link FlinkLogicalTableFunctionScan} which includes java calls into a + * {@link FlinkLogicalCalc} which will be the left input of the new {@link FlinkLogicalCorrelate} + * and a new {@link FlinkLogicalTableFunctionScan} without java calls. + */ +public class PythonCorrelateSplitRule extends RelOptRule { + public static final PythonCorrelateSplitRule INSTANCE = new PythonCorrelateSplitRule(); + + private PythonCorrelateSplitRule() { + super(operand(FlinkLogicalCorrelate.class, any()), "PythonCorrelateSplitRule"); + } + + private FlinkLogicalTableFunctionScan createNewScan( + FlinkLogicalTableFunctionScan scan, + ScalarFunctionSplitter splitter) { + RexCall rightRexCall = (RexCall) scan.getCall(); + // extract java funcs. + List rightCalcProjects = rightRexCall + .getOperands() + .stream() + .map(x -> x.accept(splitter)) + .collect(Collectors.toList()); + + RexCall newRightRexCall = rightRexCall.clone(rightRexCall.getType(), rightCalcProjects); + return new FlinkLogicalTableFunctionScan( + scan.getCluster(), + scan.getTraitSet(), + scan.getInputs(), + newRightRexCall, + scan.getElementType(), + scan.getRowType(), + scan.getColumnMappings()); + } + + @Override + public boolean matches(RelOptRuleCall call) { + FlinkLogicalCorrelate correlate = call.rel(0); + RelNode right = ((HepRelVertex) correlate.getRight()).getCurrentRel(); + FlinkLogicalTableFunctionScan pythonTableFuncScan; + if (right instanceof FlinkLogicalTableFunctionScan) { + pythonTableFuncScan = (FlinkLogicalTableFunctionScan) right; + } else if (right instanceof FlinkLogicalCalc) { + pythonTableFuncScan = StreamExecCorrelateRule.getTableScan((FlinkLogicalCalc) right); + } else { + return false; + } + RexNode rexNode = pythonTableFuncScan.getCall(); + if (rexNode instanceof RexCall) { + return PythonUtil.isPythonCall(rexNode) && PythonUtil.containsNonPythonCall(rexNode); + } + return false; + } + + private List createNewFiledNames( + RelDataType rowType, + RexBuilder rexBuilder, + int primitiveFieldCount, + ArrayBuffer extractedJavaRexCalls, + List calcProjects) { + for (int i = 0; i < primitiveFieldCount; i++) { + calcProjects.add(RexInputRef.of(i, rowType)); + } + // add the fields of the extracted java rex calls. + Iterator iterator = extractedJavaRexCalls.iterator(); + while (iterator.hasNext()) { + calcProjects.add(iterator.next()); + } + + List nameList = new LinkedList<>(); + for (int i = 0; i < primitiveFieldCount; i++) { + nameList.add(rowType.getFieldNames().get(i)); + } + Iterator indicesIterator = extractedJavaRexCalls.indices().iterator(); + while (indicesIterator.hasNext()) { + nameList.add("f" + indicesIterator.next()); + } + return SqlValidatorUtil.uniquify( + nameList, + rexBuilder.getTypeFactory().getTypeSystem().isSchemaCaseSensitive()); + } + + private FlinkLogicalCalc createNewLeftCalc( + RelNode left, + RexBuilder rexBuilder, + ArrayBuffer extractedJavaRexCalls, + FlinkLogicalCorrelate correlate) { + // add the fields of the primitive left input. + List leftCalcProjects = new LinkedList<>(); + RelDataType leftRowType = left.getRowType(); + List leftCalcCalcFieldNames = createNewFiledNames( + leftRowType, + rexBuilder, + leftRowType.getFieldCount(), + extractedJavaRexCalls, + leftCalcProjects); + + // create a new java calc + return new FlinkLogicalCalc( + correlate.getCluster(), + correlate.getTraitSet(), + left, + RexProgram.create( + leftRowType, + leftCalcProjects, + null, + leftCalcCalcFieldNames, + rexBuilder)); + } + + private FlinkLogicalCalc createTopCalc( + int primitiveLeftFieldCount, + RexBuilder rexBuilder, + ArrayBuffer extractedJavaRexCalls, + RelDataType calcRowType, + FlinkLogicalCorrelate newCorrelate) { + RexProgram rexProgram = new RexProgramBuilder(newCorrelate.getRowType(), rexBuilder).getProgram(); + int offset = extractedJavaRexCalls.size() + primitiveLeftFieldCount; + + // extract correlate output RexNode. + List newTopCalcProjects = rexProgram + .getExprList() + .stream() + .filter(x -> x instanceof RexInputRef) + .filter(x -> { + int index = ((RexInputRef) x).getIndex(); + return index < primitiveLeftFieldCount || index >= offset; + }) + .collect(Collectors.toList()); + + return new FlinkLogicalCalc( + newCorrelate.getCluster(), + newCorrelate.getTraitSet(), + newCorrelate, + RexProgram.create( + newCorrelate.getRowType(), + newTopCalcProjects, + null, + calcRowType, + rexBuilder)); + } + + @Override + public void onMatch(RelOptRuleCall call) { + FlinkLogicalCorrelate correlate = call.rel(0); + RexBuilder rexBuilder = call.builder().getRexBuilder(); + RelNode left = ((HepRelVertex) correlate.getLeft()).getCurrentRel(); + RelNode right = ((HepRelVertex) correlate.getRight()).getCurrentRel(); + int primitiveLeftFieldCount = left.getRowType().getFieldCount(); + ArrayBuffer extractedJavaRexCalls = new ArrayBuffer<>(); + ScalarFunctionSplitter splitter = new ScalarFunctionSplitter( + primitiveLeftFieldCount, + extractedJavaRexCalls, + false + ); + + RelNode rightNewInput; + if (right instanceof FlinkLogicalTableFunctionScan) { + FlinkLogicalTableFunctionScan scan = (FlinkLogicalTableFunctionScan) right; + rightNewInput = createNewScan(scan, splitter); + } else { + FlinkLogicalCalc calc = (FlinkLogicalCalc) right; + FlinkLogicalTableFunctionScan scan = StreamExecCorrelateRule.getTableScan(calc); + FlinkLogicalCalc mergedCalc = StreamExecCorrelateRule.getMergedCalc(calc); + FlinkLogicalTableFunctionScan newScan = createNewScan(scan, splitter); + rightNewInput = mergedCalc.copy(mergedCalc.getTraitSet(), newScan, mergedCalc.getProgram()); + } + + FlinkLogicalCalc leftCalc = createNewLeftCalc( + left, + rexBuilder, + extractedJavaRexCalls, + correlate); + + FlinkLogicalCorrelate newCorrelate = new FlinkLogicalCorrelate( + correlate.getCluster(), + correlate.getTraitSet(), + leftCalc, + rightNewInput, + correlate.getCorrelationId(), + correlate.getRequiredColumns(), + correlate.getJoinType()); + + FlinkLogicalCalc newTopCalc = createTopCalc( + primitiveLeftFieldCount, + rexBuilder, + extractedJavaRexCalls, + correlate.getRowType(), + newCorrelate); + + call.transformTo(newTopCalc); + } +} diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala index a77e836418..48e9e28880 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala @@ -342,6 +342,8 @@ object FlinkStreamRuleSets { // Rule that splits python ScalarFunctions from // java/scala ScalarFunctions in correlate conditions SplitPythonConditionFromCorrelateRule.INSTANCE, + // Rule that splits java calls from python TableFunction + PythonCorrelateSplitRule.INSTANCE, // merge calc after calc transpose FlinkCalcMergeRule.INSTANCE, // Rule that splits python ScalarFunctions from java/scala ScalarFunctions. diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecCorrelateRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecCorrelateRule.scala index d3630127e8..776d81a64a 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecCorrelateRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecCorrelateRule.scala @@ -123,7 +123,10 @@ object StreamExecCorrelateRule { } def getTableScan(calc: FlinkLogicalCalc): FlinkLogicalTableFunctionScan = { - val child = calc.getInput.asInstanceOf[RelSubset].getOriginal + val child = calc.getInput match { + case relSubset: RelSubset => relSubset.getOriginal + case hepRelVertex: HepRelVertex => hepRelVertex.getCurrentRel + } child match { case scan: FlinkLogicalTableFunctionScan => scan case calc: FlinkLogicalCalc => getTableScan(calc) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PythonCorrelateSplitRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PythonCorrelateSplitRuleTest.xml new file mode 100644 index 0000000000..a0f416b0e0 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PythonCorrelateSplitRuleTest.xml @@ -0,0 +1,41 @@ + + + + + + + + + + + + + + + diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCorrelateSplitRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCorrelateSplitRuleTest.scala new file mode 100644 index 0000000000..f75f783329 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCorrelateSplitRuleTest.scala @@ -0,0 +1,66 @@ +/* + * 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.table.planner.plan.rules.logical + +import org.apache.calcite.plan.hep.HepMatchOrder +import org.apache.flink.api.scala._ +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.planner.plan.nodes.FlinkConventions +import org.apache.flink.table.planner.plan.optimize.program._ +import org.apache.flink.table.planner.plan.rules.FlinkStreamRuleSets +import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.PythonScalarFunction +import org.apache.flink.table.planner.utils.{MockPythonTableFunction, TableTestBase} +import org.junit.{Before, Test} + +class PythonCorrelateSplitRuleTest extends TableTestBase { + private val util = streamTestUtil() + + @Before + def setup(): Unit = { + val programs = new FlinkChainedProgram[StreamOptimizeContext]() + // query decorrelation + programs.addLast("decorrelate", new FlinkDecorrelateProgram) + programs.addLast( + "logical", + FlinkVolcanoProgramBuilder.newBuilder + .add(FlinkStreamRuleSets.LOGICAL_OPT_RULES) + .setRequiredOutputTraits(Array(FlinkConventions.LOGICAL)) + .build()) + programs.addLast( + "logical_rewrite", + FlinkHepRuleSetProgramBuilder.newBuilder + .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_SEQUENCE) + .setHepMatchOrder(HepMatchOrder.BOTTOM_UP) + .add(FlinkStreamRuleSets.LOGICAL_REWRITE) + .build()) + util.replaceStreamProgram(programs) + + util.addTableSource[(Int, Int, Int)]("MyTable", 'a, 'b, 'c) + util.addFunction("func", new MockPythonTableFunction) + util.addFunction("pyFunc", new PythonScalarFunction("pyFunc")) + } + + @Test + def testPythonTableFunctionWithJavaFunc(): Unit = { + val sqlQuery = "SELECT a, b, c, x, y FROM MyTable, " + + "LATERAL TABLE(func(a * a, pyFunc(b, c))) AS T(x, y)" + util.verifyPlan(sqlQuery) + } + +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/plan/rules/logical/PythonCorrelateSplitRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/plan/rules/logical/PythonCorrelateSplitRule.java new file mode 100644 index 0000000000..b498d7d3fd --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/plan/rules/logical/PythonCorrelateSplitRule.java @@ -0,0 +1,245 @@ +/* + * 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.table.plan.rules.logical; + +import org.apache.flink.table.plan.nodes.logical.FlinkLogicalCalc; +import org.apache.flink.table.plan.nodes.logical.FlinkLogicalCorrelate; +import org.apache.flink.table.plan.nodes.logical.FlinkLogicalTableFunctionScan; +import org.apache.flink.table.plan.util.CorrelateUtil; +import org.apache.flink.table.plan.util.PythonUtil; + +import org.apache.calcite.plan.RelOptRule; +import org.apache.calcite.plan.RelOptRuleCall; +import org.apache.calcite.plan.hep.HepRelVertex; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rex.RexBuilder; +import org.apache.calcite.rex.RexCall; +import org.apache.calcite.rex.RexInputRef; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexProgram; +import org.apache.calcite.rex.RexProgramBuilder; +import org.apache.calcite.sql.validate.SqlValidatorUtil; + +import java.util.LinkedList; +import java.util.List; +import java.util.stream.Collectors; + +import scala.Option; +import scala.collection.Iterator; +import scala.collection.mutable.ArrayBuffer; + +/** + * Rule will split the Python {@link FlinkLogicalTableFunctionScan} which includes java calls into a + * {@link FlinkLogicalCalc} which will be the left input of the new {@link FlinkLogicalCorrelate} + * and a new {@link FlinkLogicalTableFunctionScan} without java calls. + */ +public class PythonCorrelateSplitRule extends RelOptRule { + + public static final PythonCorrelateSplitRule INSTANCE = new PythonCorrelateSplitRule(); + + private PythonCorrelateSplitRule() { + super(operand(FlinkLogicalCorrelate.class, any()), "PythonCorrelateSplitRule"); + } + + private FlinkLogicalTableFunctionScan createNewScan( + FlinkLogicalTableFunctionScan scan, + ScalarFunctionSplitter splitter) { + RexCall rightRexCall = (RexCall) scan.getCall(); + // extract java funcs. + List rightCalcProjects = rightRexCall + .getOperands() + .stream() + .map(x -> x.accept(splitter)) + .collect(Collectors.toList()); + + RexCall newRightRexCall = rightRexCall.clone(rightRexCall.getType(), rightCalcProjects); + return new FlinkLogicalTableFunctionScan( + scan.getCluster(), + scan.getTraitSet(), + scan.getInputs(), + newRightRexCall, + scan.getElementType(), + scan.getRowType(), + scan.getColumnMappings()); + } + + @Override + public boolean matches(RelOptRuleCall call) { + FlinkLogicalCorrelate correlate = call.rel(0); + RelNode right = ((HepRelVertex) correlate.getRight()).getCurrentRel(); + FlinkLogicalTableFunctionScan pythonTableFuncScan; + if (right instanceof FlinkLogicalTableFunctionScan) { + pythonTableFuncScan = (FlinkLogicalTableFunctionScan) right; + } else if (right instanceof FlinkLogicalCalc) { + Option scan = CorrelateUtil + .getTableFunctionScan((FlinkLogicalCalc) right); + if (scan.isEmpty()) { + return false; + } + pythonTableFuncScan = scan.get(); + } else { + return false; + } + RexNode rexNode = pythonTableFuncScan.getCall(); + if (rexNode instanceof RexCall) { + return PythonUtil.isPythonCall(rexNode) && PythonUtil.containsNonPythonCall(rexNode); + } + return false; + } + + private List createNewFiledNames( + RelDataType rowType, + RexBuilder rexBuilder, + int primitiveFieldCount, + ArrayBuffer extractedJavaRexCalls, + List calcProjects) { + for (int i = 0; i < primitiveFieldCount; i++) { + calcProjects.add(RexInputRef.of(i, rowType)); + } + // add the fields of the extracted java rex calls. + Iterator iterator = extractedJavaRexCalls.iterator(); + while (iterator.hasNext()) { + calcProjects.add(iterator.next()); + } + + List nameList = new LinkedList<>(); + for (int i = 0; i < primitiveFieldCount; i++) { + nameList.add(rowType.getFieldNames().get(i)); + } + Iterator indicesIterator = extractedJavaRexCalls.indices().iterator(); + while (indicesIterator.hasNext()) { + nameList.add("f" + indicesIterator.next()); + } + return SqlValidatorUtil.uniquify( + nameList, + rexBuilder.getTypeFactory().getTypeSystem().isSchemaCaseSensitive()); + } + + private FlinkLogicalCalc createNewLeftCalc( + RelNode left, + RexBuilder rexBuilder, + ArrayBuffer extractedJavaRexCalls, + FlinkLogicalCorrelate correlate) { + // add the fields of the primitive left input. + List leftCalcProjects = new LinkedList<>(); + RelDataType leftRowType = left.getRowType(); + List leftCalcCalcFieldNames = createNewFiledNames( + leftRowType, + rexBuilder, + leftRowType.getFieldCount(), + extractedJavaRexCalls, + leftCalcProjects); + + // create a new java calc + return new FlinkLogicalCalc( + correlate.getCluster(), + correlate.getTraitSet(), + left, + RexProgram.create( + leftRowType, + leftCalcProjects, + null, + leftCalcCalcFieldNames, + rexBuilder)); + } + + private FlinkLogicalCalc createTopCalc( + int primitiveLeftFieldCount, + RexBuilder rexBuilder, + ArrayBuffer extractedJavaRexCalls, + RelDataType calcRowType, + FlinkLogicalCorrelate newCorrelate) { + RexProgram rexProgram = new RexProgramBuilder(newCorrelate.getRowType(), rexBuilder).getProgram(); + int offset = extractedJavaRexCalls.size() + primitiveLeftFieldCount; + + // extract correlate output RexNode. + List newTopCalcProjects = rexProgram + .getExprList() + .stream() + .filter(x -> x instanceof RexInputRef) + .filter(x -> { + int index = ((RexInputRef) x).getIndex(); + return index < primitiveLeftFieldCount || index >= offset; + }) + .collect(Collectors.toList()); + + return new FlinkLogicalCalc( + newCorrelate.getCluster(), + newCorrelate.getTraitSet(), + newCorrelate, + RexProgram.create( + newCorrelate.getRowType(), + newTopCalcProjects, + null, + calcRowType, + rexBuilder)); + } + + @Override + public void onMatch(RelOptRuleCall call) { + FlinkLogicalCorrelate correlate = call.rel(0); + RexBuilder rexBuilder = call.builder().getRexBuilder(); + RelNode left = ((HepRelVertex) correlate.getLeft()).getCurrentRel(); + RelNode right = ((HepRelVertex) correlate.getRight()).getCurrentRel(); + int primitiveLeftFieldCount = left.getRowType().getFieldCount(); + ArrayBuffer extractedJavaRexCalls = new ArrayBuffer<>(); + ScalarFunctionSplitter splitter = new ScalarFunctionSplitter( + primitiveLeftFieldCount, + extractedJavaRexCalls, + false + ); + + RelNode rightNewInput; + if (right instanceof FlinkLogicalTableFunctionScan) { + FlinkLogicalTableFunctionScan scan = (FlinkLogicalTableFunctionScan) right; + rightNewInput = createNewScan(scan, splitter); + } else { + FlinkLogicalCalc calc = (FlinkLogicalCalc) right; + FlinkLogicalTableFunctionScan scan = CorrelateUtil.getTableFunctionScan(calc).get(); + FlinkLogicalCalc mergedCalc = CorrelateUtil.getMergedCalc(calc); + FlinkLogicalTableFunctionScan newScan = createNewScan(scan, splitter); + rightNewInput = mergedCalc.copy(mergedCalc.getTraitSet(), newScan, mergedCalc.getProgram()); + } + + FlinkLogicalCalc leftCalc = createNewLeftCalc( + left, + rexBuilder, + extractedJavaRexCalls, + correlate); + + FlinkLogicalCorrelate newCorrelate = new FlinkLogicalCorrelate( + correlate.getCluster(), + correlate.getTraitSet(), + leftCalc, + rightNewInput, + correlate.getCorrelationId(), + correlate.getRequiredColumns(), + correlate.getJoinType()); + + FlinkLogicalCalc newTopCalc = createTopCalc( + primitiveLeftFieldCount, + rexBuilder, + extractedJavaRexCalls, + correlate.getRowType(), + newCorrelate); + + call.transformTo(newTopCalc); + } +} diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala index 6dd9c8b5e0..8419e44aa5 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala @@ -152,6 +152,8 @@ object FlinkRuleSets { // Rule that splits python ScalarFunctions from // java/scala ScalarFunctions in correlate conditions SplitPythonConditionFromCorrelateRule.INSTANCE, + // Rule that splits java calls from python TableFunction + PythonCorrelateSplitRule.INSTANCE, CalcMergeRule.INSTANCE, PythonCalcSplitRule.SPLIT_CONDITION, PythonCalcSplitRule.SPLIT_PROJECT, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/util/CorrelateUtil.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/util/CorrelateUtil.scala index 8844f49ba8..06d3b0016d 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/util/CorrelateUtil.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/util/CorrelateUtil.scala @@ -32,7 +32,10 @@ object CorrelateUtil { * function at the end. */ def getTableFunctionScan(calc: FlinkLogicalCalc): Option[FlinkLogicalTableFunctionScan] = { - val child = calc.getInput.asInstanceOf[RelSubset].getOriginal + val child = calc.getInput match { + case relSubset: RelSubset => relSubset.getOriginal + case hepRelVertex: HepRelVertex => hepRelVertex.getCurrentRel + } child match { case scan: FlinkLogicalTableFunctionScan => Some(scan) case calc: FlinkLogicalCalc => getTableFunctionScan(calc) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/PythonCorrelateSplitRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/PythonCorrelateSplitRuleTest.scala new file mode 100644 index 0000000000..4b4261cec4 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/PythonCorrelateSplitRuleTest.scala @@ -0,0 +1,62 @@ +/* + * 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.table.plan + +import org.apache.flink.api.scala._ +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.runtime.utils.JavaUserDefinedScalarFunctions.PythonScalarFunction +import org.apache.flink.table.utils.TableTestUtil.{streamTableNode, term, unaryNode} +import org.apache.flink.table.utils.{MockPythonTableFunction, TableTestBase} +import org.junit.Test + +class PythonCorrelateSplitRuleTest extends TableTestBase { + + @Test + def testPythonTableFunctionWithJavaFunc(): Unit = { + val util = streamTestUtil() + val table = util.addTable[(Int, Int, Int)]("MyTable", 'a, 'b, 'c) + val pyFunc = new PythonScalarFunction("pyFunc") + val tableFunc = new MockPythonTableFunction + + val resultTable = table.joinLateral(tableFunc('a * 'a, pyFunc('b, 'c)) as('x, 'y)) + + val expected = unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamPythonCorrelate", + unaryNode( + "DataStreamCalc", + streamTableNode(table), + term("select", "a", "b", "c", "*(a, a) AS f0")), + term("invocation", + s"${tableFunc.functionIdentifier}($$3, " + + s"${pyFunc.functionIdentifier}($$1, $$2))"), + term("correlate", s"table(${tableFunc.getClass.getSimpleName}(f0, " + + s"${pyFunc.toString}(b, c)))"), + term("select", "a", "b", "c", "f0", "x", "y"), + term("rowType", + "RecordType(INTEGER a, INTEGER b, INTEGER c, INTEGER f0, INTEGER x, INTEGER y)"), + term("joinType", "INNER") + ), + term("select", "a", "b", "c", "x", "y") + ) + + util.verifyTable(resultTable, expected) + } +} -- Gitee From 0cfbe1fb2212f6490b5738bc265dd422bcecbd5c Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Tue, 18 Feb 2020 17:21:34 +0100 Subject: [PATCH 010/885] [hotfix][hdfs] Fixed error message when using HadoopRecoverableWriter on non hdfs. --- .../apache/flink/runtime/fs/hdfs/HadoopRecoverableWriter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopRecoverableWriter.java b/flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopRecoverableWriter.java index e81e4d1c43..d325f2cfc5 100644 --- a/flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopRecoverableWriter.java +++ b/flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopRecoverableWriter.java @@ -59,7 +59,7 @@ public class HadoopRecoverableWriter implements RecoverableWriter { // This writer is only supported on a subset of file systems if (!"hdfs".equalsIgnoreCase(fs.getScheme())) { throw new UnsupportedOperationException( - "Recoverable writers on Hadoop are only supported for HDFS and for Hadoop version 2.7 or newer"); + "Recoverable writers on Hadoop are only supported for HDFS"); } // Part of functionality depends on specific versions. We check these schemes and versions eagerly for -- Gitee From 91399fe2cd23850ba59b9d157863188cc194962c Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Wed, 19 Feb 2020 13:00:22 +0100 Subject: [PATCH 011/885] [FLINK-16015][filesystems]Throw an error when a plugin for a known scheme is missing. The error also avoid Hadoop fallback being used for s3 or other directly supported schemes. Also added a config to allow overridden this check for specific schemes. --- .../common_miscellaneous_section.html | 6 ++ .../generated/core_configuration.html | 6 ++ .../flink/configuration/CoreOptions.java | 8 +++ .../org/apache/flink/core/fs/FileSystem.java | 52 +++++++++++++-- .../apache/flink/core/fs/FileSystemTest.java | 64 +++++++++++++++++++ 5 files changed, 131 insertions(+), 5 deletions(-) diff --git a/docs/_includes/generated/common_miscellaneous_section.html b/docs/_includes/generated/common_miscellaneous_section.html index 0a4e15af8f..547a05b254 100644 --- a/docs/_includes/generated/common_miscellaneous_section.html +++ b/docs/_includes/generated/common_miscellaneous_section.html @@ -8,6 +8,12 @@ + +
fs.allowed-fallback-filesystems
+ (none) + String + A (semicolon-separated) list of file schemes, for which Hadoop can be used instead of an appropriate Flink plugin. (example: s3;wasb) +
fs.default-scheme
(none) diff --git a/docs/_includes/generated/core_configuration.html b/docs/_includes/generated/core_configuration.html index cf68ddf656..a9e7a7d980 100644 --- a/docs/_includes/generated/core_configuration.html +++ b/docs/_includes/generated/core_configuration.html @@ -26,6 +26,12 @@ String Defines the class resolution strategy when loading classes from user code, meaning whether to first check the user code jar ("child-first") or the application classpath ("parent-first"). The default settings indicate to load classes first from the user code jar, which means that user code jars can include and load different dependencies than Flink uses (transitively). + +
fs.allowed-fallback-filesystems
+ (none) + String + A (semicolon-separated) list of file schemes, for which Hadoop can be used instead of an appropriate Flink plugin. (example: s3;wasb) +
fs.default-scheme
(none) diff --git a/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java index 484f604d39..22509e885e 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java @@ -272,6 +272,14 @@ public class CoreOptions { .withDescription("The default filesystem scheme, used for paths that do not declare a scheme explicitly." + " May contain an authority, e.g. host:port in case of an HDFS NameNode."); + @Documentation.Section(Documentation.Sections.COMMON_MISCELLANEOUS) + public static final ConfigOption ALLOWED_FALLBACK_FILESYSTEMS = ConfigOptions + .key("fs.allowed-fallback-filesystems") + .stringType() + .defaultValue("") + .withDescription("A (semicolon-separated) list of file schemes, for which Hadoop can be used instead " + + "of an appropriate Flink plugin. (example: s3;wasb)"); + /** * Specifies whether file output writers should overwrite existing files by default. */ diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java index 744d18bb90..ba5c0c0350 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java @@ -35,7 +35,10 @@ import org.apache.flink.core.plugin.PluginManager; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TemporaryClassLoaderContext; +import org.apache.flink.shaded.guava18.com.google.common.base.Splitter; +import org.apache.flink.shaded.guava18.com.google.common.collect.ImmutableMultimap; import org.apache.flink.shaded.guava18.com.google.common.collect.Iterators; +import org.apache.flink.shaded.guava18.com.google.common.collect.Multimap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,9 +54,11 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.ServiceLoader; +import java.util.Set; import java.util.concurrent.locks.ReentrantLock; import java.util.function.Supplier; @@ -230,6 +235,23 @@ public abstract class FileSystem { /** The default factory that is used when no scheme matches. */ private static final FileSystemFactory FALLBACK_FACTORY = loadHadoopFsFactory(); + /** All known plugins for a given scheme, do not fallback for those. */ + private static final Multimap DIRECTLY_SUPPORTED_FILESYSTEM = + ImmutableMultimap.builder() + .put("wasb", "flink-fs-azure-hadoop") + .put("wasbs", "flink-fs-azure-hadoop") + .put("oss", "flink-oss-fs-hadoop") + .put("s3", "flink-s3-fs-hadoop") + .put("s3", "flink-s3-fs-presto") + .put("s3a", "flink-s3-fs-hadoop") + .put("s3p", "flink-s3-fs-presto") + .put("swift", "flink-swift-fs-hadoop") + // mapr deliberately omitted for now (no dedicated plugin) + .build(); + + /** Exceptions for DIRECTLY_SUPPORTED_FILESYSTEM. */ + private static final Set ALLOWED_FALLBACK_FILESYSTEMS = new HashSet<>(); + /** The default filesystem scheme to be used, configured during process-wide initialization. * This value defaults to the local file systems scheme {@code 'file:///'} or {@code 'file:/'}. */ private static URI defaultScheme; @@ -328,6 +350,11 @@ public abstract class FileSystem { CoreOptions.DEFAULT_FILESYSTEM_SCHEME + "') is invalid: " + stringifiedUri, e); } } + + ALLOWED_FALLBACK_FILESYSTEMS.clear(); + final Iterable allowedFallbackFilesystems = Splitter.on(';').omitEmptyStrings().trimResults() + .split(config.getString(CoreOptions.ALLOWED_FALLBACK_FILESYSTEMS)); + allowedFallbackFilesystems.forEach(ALLOWED_FALLBACK_FILESYSTEMS::add); } finally { LOCK.unlock(); @@ -440,16 +467,31 @@ public abstract class FileSystem { try (TemporaryClassLoaderContext ignored = TemporaryClassLoaderContext.of(classLoader)) { fs = factory.create(uri); } - } - else { + } else if (!ALLOWED_FALLBACK_FILESYSTEMS.contains(uri.getScheme()) && + DIRECTLY_SUPPORTED_FILESYSTEM.containsKey(uri.getScheme())) { + final Collection plugins = DIRECTLY_SUPPORTED_FILESYSTEM.get(uri.getScheme()); + throw new UnsupportedFileSystemSchemeException(String.format( + "Could not find a file system implementation for scheme '%s'. The scheme is " + + "directly supported by Flink through the following plugin%s: %s. Please ensure that each " + + "plugin resides within its own subfolder within the plugins directory. See https://ci.apache" + + ".org/projects/flink/flink-docs-stable/ops/plugins.html for more information. If you want to " + + "use a Hadoop file system for that scheme, please add the scheme to the configuration fs" + + ".allowed-fallback-filesystems. For a full list of supported file systems, " + + "please see https://ci.apache.org/projects/flink/flink-docs-stable/ops/filesystems/.", + uri.getScheme(), + plugins.size() == 1 ? "" : "s", + String.join(", ", plugins) + )); + } else { try { fs = FALLBACK_FACTORY.create(uri); } catch (UnsupportedFileSystemSchemeException e) { throw new UnsupportedFileSystemSchemeException( - "Could not find a file system implementation for scheme '" + uri.getScheme() + - "'. The scheme is not directly supported by Flink and no Hadoop file " + - "system to support this scheme could be loaded.", e); + "Could not find a file system implementation for scheme '" + uri.getScheme() + + "'. The scheme is not directly supported by Flink and no Hadoop file system to " + + "support this scheme could be loaded. For a full list of supported file systems, " + + "please see https://ci.apache.org/projects/flink/flink-docs-stable/ops/filesystems/.", e); } } diff --git a/flink-core/src/test/java/org/apache/flink/core/fs/FileSystemTest.java b/flink-core/src/test/java/org/apache/flink/core/fs/FileSystemTest.java index f6d3f45a44..061eb27a3e 100644 --- a/flink-core/src/test/java/org/apache/flink/core/fs/FileSystemTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/fs/FileSystemTest.java @@ -18,17 +18,25 @@ package org.apache.flink.core.fs; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; import org.apache.flink.core.fs.local.LocalFileSystem; import org.apache.flink.util.WrappingProxy; import org.apache.flink.util.WrappingProxyUtil; +import org.apache.flink.util.function.ThrowingRunnable; +import org.hamcrest.Matchers; import org.junit.Test; import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.not; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; /** * Tests for the {@link FileSystem} base class. @@ -56,6 +64,62 @@ public class FileSystemTest { assertTrue(getFileSystemWithoutSafetyNet("test/test") instanceof LocalFileSystem); } + @Test + public void testUnsupportedFS() throws Exception { + Exception e = assertThatCode(() -> getFileSystemWithoutSafetyNet("unknownfs://authority/")); + assertThat(e, Matchers.instanceOf(UnsupportedFileSystemSchemeException.class)); + } + + @Test + public void testKnownFSWithoutPlugins() throws Exception { + Exception e = assertThatCode(() -> getFileSystemWithoutSafetyNet("s3://authority/")); + assertThat(e, Matchers.instanceOf(UnsupportedFileSystemSchemeException.class)); + /* + exception should be: + org.apache.flink.core.fs.UnsupportedFileSystemSchemeException: Could not find a file + system implementation for scheme 's3'. The scheme is directly supported by Flink through the following + plugins: flink-s3-fs-hadoop, flink-s3-fs-presto. Please ensure that each plugin resides within its own + subfolder within the plugins directory. See https://ci.apache + .org/projects/flink/flink-docs-master/ops/plugins.html for more information. */ + assertThat(e.getMessage(), not(containsString("not directly supported"))); + assertThat(e.getMessage(), containsString("flink-s3-fs-hadoop")); + assertThat(e.getMessage(), containsString("flink-s3-fs-presto")); + } + + @Test + public void testKnownFSWithoutPluginsAndException() throws Exception { + try { + final Configuration config = new Configuration(); + config.set(CoreOptions.ALLOWED_FALLBACK_FILESYSTEMS, "s3;wasb"); + FileSystem.initialize(config); + + Exception e = assertThatCode(() -> getFileSystemWithoutSafetyNet("s3://authority/")); + assertThat(e, Matchers.instanceOf(UnsupportedFileSystemSchemeException.class)); + /* + exception should be: + org.apache.flink.core.fs.UnsupportedFileSystemSchemeException: Could not find a file system implementation + for scheme 's3'. The scheme is not directly supported by Flink and no Hadoop file system to support this + scheme could be loaded. */ + assertThat(e.getMessage(), containsString("not directly supported")); + } finally { + FileSystem.initialize(new Configuration()); + } + } + + private static E assertThatCode(ThrowingRunnable runnable) throws E { + try { + runnable.run(); + fail("No exception thrown"); + return null; + } catch (Throwable e) { + try { + return (E) e; + } catch (ClassCastException c) { + throw e; + } + } + } + private static FileSystem getFileSystemWithoutSafetyNet(final String uri) throws URISyntaxException, IOException { final FileSystem fileSystem = FileSystem.get(new URI(uri)); -- Gitee From 2489d0452dbdaf1e2ebf73f69bd022bf914696ae Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Thu, 20 Feb 2020 15:40:55 +0100 Subject: [PATCH 012/885] [hotfix][runtime] Removed duplicate isChainable. --- .../api/graph/StreamGraphHasherV2.java | 20 +------------------ 1 file changed, 1 insertion(+), 19 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphHasherV2.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphHasherV2.java index e7474e7a27..8fbbbc1d26 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphHasherV2.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphHasherV2.java @@ -20,10 +20,7 @@ package org.apache.flink.streaming.api.graph; import org.apache.flink.api.dag.Transformation; import org.apache.flink.runtime.jobgraph.JobVertexID; -import org.apache.flink.streaming.api.operators.ChainingStrategy; -import org.apache.flink.streaming.api.operators.StreamOperatorFactory; import org.apache.flink.streaming.api.operators.UdfStreamOperatorFactory; -import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner; import org.apache.flink.shaded.guava18.com.google.common.hash.HashFunction; import org.apache.flink.shaded.guava18.com.google.common.hash.Hasher; @@ -281,21 +278,6 @@ public class StreamGraphHasherV2 implements StreamGraphHasher { } private boolean isChainable(StreamEdge edge, boolean isChainingEnabled, StreamGraph streamGraph) { - StreamNode upStreamVertex = streamGraph.getSourceVertex(edge); - StreamNode downStreamVertex = streamGraph.getTargetVertex(edge); - - StreamOperatorFactory headOperator = upStreamVertex.getOperatorFactory(); - StreamOperatorFactory outOperator = downStreamVertex.getOperatorFactory(); - - return downStreamVertex.getInEdges().size() == 1 - && outOperator != null - && headOperator != null - && upStreamVertex.isSameSlotSharingGroup(downStreamVertex) - && outOperator.getChainingStrategy() == ChainingStrategy.ALWAYS - && (headOperator.getChainingStrategy() == ChainingStrategy.HEAD || - headOperator.getChainingStrategy() == ChainingStrategy.ALWAYS) - && (edge.getPartitioner() instanceof ForwardPartitioner) - && upStreamVertex.getParallelism() == downStreamVertex.getParallelism() - && isChainingEnabled; + return isChainingEnabled && StreamingJobGraphGenerator.isChainable(edge, streamGraph); } } -- Gitee From 1034b6abcdb0e1c37a25176859b0c9235a4f225d Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Wed, 26 Feb 2020 08:37:08 +0100 Subject: [PATCH 013/885] [hotfix][runtime] Unified variable names in StreamingJobGraphGenerator. Removed confusing headOperators in isChainable. --- .../api/graph/StreamingJobGraphGenerator.java | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java index f99ed79549..85674f206c 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java @@ -541,10 +541,10 @@ public class StreamingJobGraphGenerator { physicalEdgesInOrder.add(edge); - Integer downStreamvertexID = edge.getTargetId(); + Integer downStreamVertexID = edge.getTargetId(); JobVertex headVertex = jobVertices.get(headOfChain); - JobVertex downStreamVertex = jobVertices.get(downStreamvertexID); + JobVertex downStreamVertex = jobVertices.get(downStreamVertexID); StreamConfig downStreamConfig = new StreamConfig(downStreamVertex.getConfiguration()); @@ -586,7 +586,7 @@ public class StreamingJobGraphGenerator { if (LOG.isDebugEnabled()) { LOG.debug("CONNECTED: {} - {} -> {}", partitioner.getClass().getSimpleName(), - headOfChain, downStreamvertexID); + headOfChain, downStreamVertexID); } } @@ -594,16 +594,16 @@ public class StreamingJobGraphGenerator { StreamNode upStreamVertex = streamGraph.getSourceVertex(edge); StreamNode downStreamVertex = streamGraph.getTargetVertex(edge); - StreamOperatorFactory headOperator = upStreamVertex.getOperatorFactory(); - StreamOperatorFactory outOperator = downStreamVertex.getOperatorFactory(); + StreamOperatorFactory upStreamOperator = upStreamVertex.getOperatorFactory(); + StreamOperatorFactory downStreamOperator = downStreamVertex.getOperatorFactory(); return downStreamVertex.getInEdges().size() == 1 - && outOperator != null - && headOperator != null + && downStreamOperator != null + && upStreamOperator != null && upStreamVertex.isSameSlotSharingGroup(downStreamVertex) - && outOperator.getChainingStrategy() == ChainingStrategy.ALWAYS - && (headOperator.getChainingStrategy() == ChainingStrategy.HEAD || - headOperator.getChainingStrategy() == ChainingStrategy.ALWAYS) + && downStreamOperator.getChainingStrategy() == ChainingStrategy.ALWAYS + && (upStreamOperator.getChainingStrategy() == ChainingStrategy.HEAD || + upStreamOperator.getChainingStrategy() == ChainingStrategy.ALWAYS) && (edge.getPartitioner() instanceof ForwardPartitioner) && edge.getShuffleMode() != ShuffleMode.BATCH && upStreamVertex.getParallelism() == downStreamVertex.getParallelism() -- Gitee From ae823c8b621b23db02283878a1173d84b01615fb Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Thu, 20 Feb 2020 15:41:58 +0100 Subject: [PATCH 014/885] [FLINK-16219][runtime] Disallow chaining of legacy source and yielding operator. This change allows yielding operators to be eagerly chained whenever possible, except after legacy sources. Yielding operators do not properly work when processInput is called from another thread, but are usually fine in any other chain. --- .../api/graph/StreamingJobGraphGenerator.java | 45 ++++++++-- .../graph/StreamingJobGraphGeneratorTest.java | 82 +++++++++++++++++++ 2 files changed, 119 insertions(+), 8 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java index 85674f206c..ef11df043e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.api.graph; import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.operators.ResourceSpec; @@ -52,6 +53,7 @@ import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.InputSelectable; import org.apache.flink.streaming.api.operators.StreamOperatorFactory; import org.apache.flink.streaming.api.operators.UdfStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.YieldingOperatorFactory; import org.apache.flink.streaming.api.transformations.ShuffleMode; import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner; import org.apache.flink.streaming.runtime.partitioner.RescalePartitioner; @@ -594,22 +596,49 @@ public class StreamingJobGraphGenerator { StreamNode upStreamVertex = streamGraph.getSourceVertex(edge); StreamNode downStreamVertex = streamGraph.getTargetVertex(edge); - StreamOperatorFactory upStreamOperator = upStreamVertex.getOperatorFactory(); - StreamOperatorFactory downStreamOperator = downStreamVertex.getOperatorFactory(); - return downStreamVertex.getInEdges().size() == 1 - && downStreamOperator != null - && upStreamOperator != null && upStreamVertex.isSameSlotSharingGroup(downStreamVertex) - && downStreamOperator.getChainingStrategy() == ChainingStrategy.ALWAYS - && (upStreamOperator.getChainingStrategy() == ChainingStrategy.HEAD || - upStreamOperator.getChainingStrategy() == ChainingStrategy.ALWAYS) + && areOperatorsChainable(upStreamVertex, downStreamVertex, streamGraph) && (edge.getPartitioner() instanceof ForwardPartitioner) && edge.getShuffleMode() != ShuffleMode.BATCH && upStreamVertex.getParallelism() == downStreamVertex.getParallelism() && streamGraph.isChainingEnabled(); } + @VisibleForTesting + static boolean areOperatorsChainable( + StreamNode upStreamVertex, + StreamNode downStreamVertex, + StreamGraph streamGraph) { + StreamOperatorFactory upStreamOperator = upStreamVertex.getOperatorFactory(); + StreamOperatorFactory downStreamOperator = downStreamVertex.getOperatorFactory(); + if (downStreamOperator == null || upStreamOperator == null) { + return false; + } + + if (upStreamOperator.getChainingStrategy() == ChainingStrategy.NEVER || + downStreamOperator.getChainingStrategy() != ChainingStrategy.ALWAYS) { + return false; + } + + // yielding operators cannot be chained to legacy sources + if (downStreamOperator instanceof YieldingOperatorFactory) { + // unfortunately the information that vertices have been chained is not preserved at this point + return !getHeadOperator(upStreamVertex, streamGraph).isStreamSource(); + } + return true; + } + + /** + * Backtraces the head of an operator chain. + */ + private static StreamOperatorFactory getHeadOperator(StreamNode upStreamVertex, StreamGraph streamGraph) { + if (upStreamVertex.getInEdges().size() == 1 && isChainable(upStreamVertex.getInEdges().get(0), streamGraph)) { + return getHeadOperator(streamGraph.getSourceVertex(upStreamVertex.getInEdges().get(0)), streamGraph); + } + return upStreamVertex.getOperatorFactory(); + } + private void setSlotSharingAndCoLocation() { setSlotSharing(); setCoLocation(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java index e942aebce7..7ce05c406a 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGeneratorTest.java @@ -51,9 +51,15 @@ import org.apache.flink.streaming.api.datastream.IterativeStream; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.environment.CheckpointConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.DiscardingSink; import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.api.functions.source.InputFormatSourceFunction; import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; +import org.apache.flink.streaming.api.operators.MailboxExecutor; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamMap; +import org.apache.flink.streaming.api.operators.YieldingOperatorFactory; import org.apache.flink.streaming.api.transformations.PartitionTransformation; import org.apache.flink.streaming.api.transformations.ShuffleMode; import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner; @@ -65,6 +71,7 @@ import org.apache.flink.util.TestLogger; import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables; +import org.junit.Assert; import org.junit.Test; import javax.annotation.Nullable; @@ -74,10 +81,13 @@ import java.math.BigDecimal; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; +import static org.apache.flink.streaming.api.graph.StreamingJobGraphGenerator.areOperatorsChainable; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.junit.Assert.assertEquals; @@ -644,6 +654,66 @@ public class StreamingJobGraphGeneratorTest extends TestLogger { assertEquals(ResultPartitionType.PIPELINED_BOUNDED, printVertex.getInputs().get(0).getSource().getResultType()); } + @Test + public void testYieldingOperatorNotChainableToTaskChainedToLegacySource() { + StreamExecutionEnvironment chainEnv = StreamExecutionEnvironment.createLocalEnvironment(1); + + chainEnv.fromElements(1) + .map((x) -> x) + // not chainable because of YieldingOperatorFactory and legacy source + .transform("test", BasicTypeInfo.INT_TYPE_INFO, new YieldingTestOperatorFactory<>()); + + final StreamGraph streamGraph = chainEnv.getStreamGraph(); + + final List streamNodes = streamGraph.getStreamNodes().stream() + .sorted(Comparator.comparingInt(StreamNode::getId)) + .collect(Collectors.toList()); + assertTrue(areOperatorsChainable(streamNodes.get(0), streamNodes.get(1), streamGraph)); + assertFalse(areOperatorsChainable(streamNodes.get(1), streamNodes.get(2), streamGraph)); + } + + @Test + public void testYieldingOperatorChainableToTaskNotChainedToLegacySource() { + StreamExecutionEnvironment chainEnv = StreamExecutionEnvironment.createLocalEnvironment(1); + + chainEnv.fromElements(1).disableChaining() + .map((x) -> x) + .transform("test", BasicTypeInfo.INT_TYPE_INFO, new YieldingTestOperatorFactory<>()); + + final StreamGraph streamGraph = chainEnv.getStreamGraph(); + + final List streamNodes = streamGraph.getStreamNodes().stream() + .sorted(Comparator.comparingInt(StreamNode::getId)) + .collect(Collectors.toList()); + assertFalse(areOperatorsChainable(streamNodes.get(0), streamNodes.get(1), streamGraph)); + assertTrue(areOperatorsChainable(streamNodes.get(1), streamNodes.get(2), streamGraph)); + } + + /** + * Tests that {@link org.apache.flink.streaming.api.operators.YieldingOperatorFactory} are not chained to legacy + * sources, see FLINK-16219. + */ + @Test + public void testYieldingOperatorProperlyChained() { + StreamExecutionEnvironment chainEnv = StreamExecutionEnvironment.createLocalEnvironment(1); + + chainEnv.fromElements(1) + .map((x) -> x) + // should automatically break chain here + .transform("test", BasicTypeInfo.INT_TYPE_INFO, new YieldingTestOperatorFactory<>()) + .map((x) -> x) + .transform("test", BasicTypeInfo.INT_TYPE_INFO, new YieldingTestOperatorFactory<>()) + .map((x) -> x) + .addSink(new DiscardingSink<>()); + + final JobGraph jobGraph = chainEnv.getStreamGraph().getJobGraph(); + + final List vertices = jobGraph.getVerticesSortedTopologicallyFromSources(); + Assert.assertEquals(2, vertices.size()); + assertEquals(2, vertices.get(0).getOperatorIDs().size()); + assertEquals(5, vertices.get(1).getOperatorIDs().size()); + } + @Test(expected = UnsupportedOperationException.class) public void testNotSupportInputSelectableOperatorIfCheckpointing() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); @@ -873,4 +943,16 @@ public class StreamingJobGraphGeneratorTest extends TestLogger { setResourcesMethod.setAccessible(true); return setResourcesMethod; } + + private static class YieldingTestOperatorFactory extends SimpleOperatorFactory implements + YieldingOperatorFactory, OneInputStreamOperatorFactory { + private YieldingTestOperatorFactory() { + super(new StreamMap(x -> x)); + } + + @Override + public void setMailboxExecutor(MailboxExecutor mailboxExecutor) { + } + } + } -- Gitee From 3a04e179e09224b09c4ee656d31558844da83a26 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Thu, 20 Feb 2020 15:44:02 +0100 Subject: [PATCH 015/885] [FLINK-16219][runtime] Made AsyncWaitOperator chainable to non-sources. AsyncWaitOperator is not thread-safe when chained to legacy sources, but works well in a chained fashion in all other cases. Moved test case to StreamingJobGraphGenerator. --- .../operators/async/AsyncWaitOperator.java | 4 +-- .../async/AsyncWaitOperatorFactory.java | 2 +- .../async/AsyncWaitOperatorTest.java | 26 ------------------- 3 files changed, 2 insertions(+), 30 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java index 8071c90cdb..be2d520174 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java @@ -114,9 +114,7 @@ public class AsyncWaitOperator @Nonnull MailboxExecutor mailboxExecutor) { super(asyncFunction); - // TODO this is a temporary fix for the problems described under FLINK-13063 at the cost of breaking chains for - // AsyncOperators. - setChainingStrategy(ChainingStrategy.HEAD); + setChainingStrategy(ChainingStrategy.ALWAYS); Preconditions.checkArgument(capacity > 0, "The number of concurrent async operation should be greater than 0."); this.capacity = capacity; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorFactory.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorFactory.java index 922507da14..c30fb2fba5 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorFactory.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorFactory.java @@ -52,7 +52,7 @@ public class AsyncWaitOperatorFactory extends AbstractStreamOperatorFac this.timeout = timeout; this.capacity = capacity; this.outputMode = outputMode; - this.chainingStrategy = ChainingStrategy.HEAD; + this.chainingStrategy = ChainingStrategy.ALWAYS; } @Override diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java index f3d45489e1..3e0ad2a587 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java @@ -364,32 +364,6 @@ public class AsyncWaitOperatorTest extends TestLogger { } } - /** - * Test for the temporary fix to FLINK-13063. - */ - @Test - public void testAsyncOperatorIsNeverChained() { - StreamExecutionEnvironment chainEnv = StreamExecutionEnvironment.getExecutionEnvironment(); - - DataStream input = chainEnv.fromElements(1); - input = AsyncDataStream.orderedWait( - input, - new LazyAsyncFunction(), - TIMEOUT, - TimeUnit.MILLISECONDS, - 6).map((x) -> x); - AsyncDataStream.unorderedWait( - input, - new MyAsyncFunction(), - TIMEOUT, - TimeUnit.MILLISECONDS, - 3).map((x) -> x).addSink(new DiscardingSink<>()); - - final JobGraph jobGraph = chainEnv.getStreamGraph().getJobGraph(); - - Assert.assertEquals(3, jobGraph.getVerticesSortedTopologicallyFromSources().size()); - } - /** * Tests that the AsyncWaitOperator works together with chaining. */ -- Gitee From 447deaa09e581a299ca32023067106c599c1e45d Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Wed, 19 Feb 2020 11:38:26 +0100 Subject: [PATCH 016/885] [hotfix][build] Remove redundant javadoc-plugin versions --- pom.xml | 2 -- 1 file changed, 2 deletions(-) diff --git a/pom.xml b/pom.xml index a03c9da8b4..345dd8976d 100644 --- a/pom.xml +++ b/pom.xml @@ -1138,7 +1138,6 @@ under the License. org.apache.maven.plugins maven-javadoc-plugin - 2.9.1 true @@ -1222,7 +1221,6 @@ under the License. org.apache.maven.plugins maven-javadoc-plugin - 2.9.1 attach-javadocs -- Gitee From 8cdef5283ebf00bd44adf8853222a7675b9c62a6 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Wed, 19 Feb 2020 11:39:11 +0100 Subject: [PATCH 017/885] [FLINK-15633][build] Bump javadoc-plugin to 3.1.1 --- pom.xml | 15 +++++++++++++-- 1 file changed, 13 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 345dd8976d..f8e8315d57 100644 --- a/pom.xml +++ b/pom.xml @@ -930,6 +930,15 @@ under the License. org.apache.flink.testutils.junit.FailsOnJava11 + + org.apache.maven.plugins + maven-javadoc-plugin + + + --add-exports=java.base/sun.net.util=ALL-UNNAMED + + + @@ -1725,11 +1734,13 @@ under the License. org.apache.maven.plugins maven-javadoc-plugin - 2.9.1 + 3.1.1 true - -Xdoclint:none false + + -Xdoclint:none + -- Gitee From cc75e01abcb6b6eed50591c94db9e2a26f179518 Mon Sep 17 00:00:00 2001 From: GuoWei Ma Date: Thu, 27 Feb 2020 16:12:27 +0800 Subject: [PATCH 018/885] [FLINK-2336][DataStream API] Fix ArrayIndexOufOBoundsException in TypeExtractor for type erased lambdas This closes #11234 --- .../org/apache/flink/api/java/typeutils/TypeExtractor.java | 5 +++++ .../flink/api/java/typeutils/LambdaExtractionTest.java | 6 ++++++ 2 files changed, 11 insertions(+) diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java index 81cd4d2ea3..acb1ec94c8 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java @@ -953,6 +953,11 @@ public class TypeExtractor { break; } } + + if (inputTypeHierarchy.size() == 0) { + return null; + } + ParameterizedType baseClass = (ParameterizedType) inputTypeHierarchy.get(inputTypeHierarchy.size() - 1); TypeInformation info = null; diff --git a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/LambdaExtractionTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/LambdaExtractionTest.java index 1d5cf22a5c..36d3637443 100644 --- a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/LambdaExtractionTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/LambdaExtractionTest.java @@ -231,6 +231,12 @@ public class LambdaExtractionTest { assertTrue(ti instanceof MissingTypeInfo); } + @Test + public void testLambdaWithoutTypeErasure() { + TypeInformation ti = TypeExtractor.getMapReturnTypes(Tuple1::of, BasicTypeInfo.STRING_TYPE_INFO, null, true); + assertTrue(ti instanceof MissingTypeInfo); + } + @Test public void testPartitionerLambda() { Partitioner> partitioner = (key, numPartitions) -> key.f1.length() % numPartitions; -- Gitee From d62eb7ed08d277b07f67c483987a30a51a7a2a59 Mon Sep 17 00:00:00 2001 From: shuai-xu Date: Mon, 2 Mar 2020 15:08:50 +0800 Subject: [PATCH 019/885] [hotfix][docs] Fix the mismatch between java and scala CEP examples This closes #11278 --- docs/dev/libs/cep.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/dev/libs/cep.md b/docs/dev/libs/cep.md index a6b92512c6..8e196b6cfd 100644 --- a/docs/dev/libs/cep.md +++ b/docs/dev/libs/cep.md @@ -206,6 +206,9 @@ start.timesOrMore(2); // expecting 2 or more occurrences and repeating as many as possible start.timesOrMore(2).greedy(); +// expecting 0, 2 or more occurrences +start.timesOrMore(2).optional() + // expecting 0, 2 or more occurrences and repeating as many as possible start.timesOrMore(2).optional().greedy(); {% endhighlight %} -- Gitee From 2806215c0c4d88c336ee57ab639d24685b485273 Mon Sep 17 00:00:00 2001 From: Paul Lin Date: Sun, 1 Mar 2020 18:36:13 +0800 Subject: [PATCH 020/885] [hotfix][table] Remove unused import in DataStreamConversions This closes #11271 --- .../org/apache/flink/table/planner/DataStreamConversions.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/DataStreamConversions.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/DataStreamConversions.scala index e4cd58bec6..199a32ee3e 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/DataStreamConversions.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/DataStreamConversions.scala @@ -20,7 +20,6 @@ package org.apache.flink.table.planner import java.lang.{Boolean => JBool} -import org.apache.calcite.rel.RelNode import org.apache.flink.api.common.functions.MapFunction import org.apache.flink.api.common.typeinfo.{SqlTimeTypeInfo, TypeInformation} import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2} -- Gitee From 195b0d03ad994a82baee8864c1fedf1ae68b0144 Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Mon, 2 Mar 2020 14:50:13 +0100 Subject: [PATCH 021/885] [FLINK-16380][AZP] Fix jdk11 switch --- tools/azure-pipelines/jobs-template.yml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tools/azure-pipelines/jobs-template.yml b/tools/azure-pipelines/jobs-template.yml index 2056f51965..4d8bf6504a 100644 --- a/tools/azure-pipelines/jobs-template.yml +++ b/tools/azure-pipelines/jobs-template.yml @@ -53,7 +53,7 @@ jobs: displayName: Cache Maven local repo - script: | echo "##vso[task.setvariable variable=JAVA_HOME]$JAVA_HOME_11_X64" - echo "##vso[task.setvariable variable=PATH]$JAVA_HOME_11_X64/bin;$PATH" + echo "##vso[task.setvariable variable=PATH]$JAVA_HOME_11_X64/bin:$PATH" displayName: "Set to jdk11" condition: eq('${{parameters.jdk}}', 'jdk11') # Compile @@ -116,7 +116,7 @@ jobs: displayName: Cache Maven local repo - script: | echo "##vso[task.setvariable variable=JAVA_HOME]$JAVA_HOME_11_X64" - echo "##vso[task.setvariable variable=PATH]$JAVA_HOME_11_X64/bin;$PATH" + echo "##vso[task.setvariable variable=PATH]$JAVA_HOME_11_X64/bin:$PATH" displayName: "Set to jdk11" condition: eq('${{parameters.jdk}}', 'jdk11') # Test @@ -147,7 +147,7 @@ jobs: continueOnError: true - script: | echo "##vso[task.setvariable variable=JAVA_HOME]$JAVA_HOME_11_X64" - echo "##vso[task.setvariable variable=PATH]$JAVA_HOME_11_X64/bin;$PATH" + echo "##vso[task.setvariable variable=PATH]$JAVA_HOME_11_X64/bin:$PATH" displayName: "Set to jdk11" condition: eq('${{parameters.jdk}}', 'jdk11') - script: ./tools/travis/setup_maven.sh -- Gitee From 3e10f0a5ca1179f8a95185c3d3c03ec9fe0dabb6 Mon Sep 17 00:00:00 2001 From: Leonard Xu Date: Tue, 3 Mar 2020 09:25:34 +0800 Subject: [PATCH 022/885] [FLINK-16281][jdbc] Parameter 'maxRetryTimes' can not work in AppendOnlyWriter (#11223) --- .../java/io/jdbc/JDBCUpsertOutputFormat.java | 8 +- .../java/io/jdbc/writer/AppendOnlyWriter.java | 18 ++- .../api/java/io/jdbc/writer/JDBCWriter.java | 2 +- .../api/java/io/jdbc/writer/UpsertWriter.java | 23 ++-- .../java/io/jdbc/JDBCAppenOnlyWriterTest.java | 105 ++++++++++++++++++ 5 files changed, 133 insertions(+), 23 deletions(-) create mode 100644 flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCAppenOnlyWriterTest.java diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertOutputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertOutputFormat.java index cd1ced3a6f..1736c28f73 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertOutputFormat.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertOutputFormat.java @@ -62,6 +62,7 @@ public class JDBCUpsertOutputFormat extends AbstractJDBCOutputFormat record = objectReuse ? new Tuple2<>(tuple2.f0, Row.copy(tuple2.f1)) : tuple2; + jdbcWriter.addRecord(record); batchCount++; if (batchCount >= flushMaxSize) { flush(); diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/writer/AppendOnlyWriter.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/writer/AppendOnlyWriter.java index 899a22f4f7..ff07874a18 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/writer/AppendOnlyWriter.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/writer/AppendOnlyWriter.java @@ -24,6 +24,8 @@ import org.apache.flink.types.Row; import java.sql.Connection; import java.sql.PreparedStatement; import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; import static org.apache.flink.api.java.io.jdbc.JDBCUtils.setRecordToStatement; import static org.apache.flink.util.Preconditions.checkArgument; @@ -38,6 +40,7 @@ public class AppendOnlyWriter implements JDBCWriter { private final String insertSQL; private final int[] fieldTypes; + private transient List cachedRows; private transient PreparedStatement statement; public AppendOnlyWriter(String insertSQL, int[] fieldTypes) { @@ -47,19 +50,26 @@ public class AppendOnlyWriter implements JDBCWriter { @Override public void open(Connection connection) throws SQLException { + this.cachedRows = new ArrayList<>(); this.statement = connection.prepareStatement(insertSQL); } @Override - public void addRecord(Tuple2 record) throws SQLException { + public void addRecord(Tuple2 record) { checkArgument(record.f0, "Append mode can not receive retract/delete message."); - setRecordToStatement(statement, fieldTypes, record.f1); - statement.addBatch(); + cachedRows.add(record.f1); } @Override public void executeBatch() throws SQLException { - statement.executeBatch(); + if (cachedRows.size() > 0) { + for (Row row : cachedRows) { + setRecordToStatement(statement, fieldTypes, row); + statement.addBatch(); + } + statement.executeBatch(); + cachedRows.clear(); + } } @Override diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/writer/JDBCWriter.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/writer/JDBCWriter.java index 8ce475971c..1a6797a61d 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/writer/JDBCWriter.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/writer/JDBCWriter.java @@ -38,7 +38,7 @@ public interface JDBCWriter extends Serializable { /** * Add record to writer, the writer may cache the data. */ - void addRecord(Tuple2 record) throws SQLException; + void addRecord(Tuple2 record); /** * Submits a batch of commands to the database for execution. diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/writer/UpsertWriter.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/writer/UpsertWriter.java index dcc807e55c..7d07b95d3d 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/writer/UpsertWriter.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/writer/UpsertWriter.java @@ -48,8 +48,7 @@ public abstract class UpsertWriter implements JDBCWriter { String tableName, String[] fieldNames, int[] fieldTypes, - String[] keyFields, - boolean objectReuse) { + String[] keyFields) { checkNotNull(keyFields); @@ -62,10 +61,10 @@ public abstract class UpsertWriter implements JDBCWriter { Optional upsertSQL = dialect.getUpsertStatement(tableName, fieldNames, keyFields); return upsertSQL.map((Function) sql -> new UpsertWriterUsingUpsertStatement( - fieldTypes, pkFields, pkTypes, objectReuse, deleteSQL, sql)) + fieldTypes, pkFields, pkTypes, deleteSQL, sql)) .orElseGet(() -> new UpsertWriterUsingInsertUpdateStatement( - fieldTypes, pkFields, pkTypes, objectReuse, deleteSQL, + fieldTypes, pkFields, pkTypes, deleteSQL, dialect.getRowExistsStatement(tableName, keyFields), dialect.getInsertIntoStatement(tableName, fieldNames), dialect.getUpdateStatement(tableName, fieldNames, keyFields))); @@ -75,17 +74,15 @@ public abstract class UpsertWriter implements JDBCWriter { final int[] pkTypes; private final int[] pkFields; private final String deleteSQL; - private final boolean objectReuse; private transient Map> keyToRows; private transient PreparedStatement deleteStatement; - private UpsertWriter(int[] fieldTypes, int[] pkFields, int[] pkTypes, String deleteSQL, boolean objectReuse) { + private UpsertWriter(int[] fieldTypes, int[] pkFields, int[] pkTypes, String deleteSQL) { this.fieldTypes = fieldTypes; this.pkFields = pkFields; this.pkTypes = pkTypes; this.deleteSQL = deleteSQL; - this.objectReuse = objectReuse; } @Override @@ -94,11 +91,9 @@ public abstract class UpsertWriter implements JDBCWriter { this.deleteStatement = connection.prepareStatement(deleteSQL); } - public void addRecord(Tuple2 record) throws SQLException { - // we don't need perform a deep copy, because jdbc field are immutable object. - Tuple2 tuple2 = objectReuse ? new Tuple2<>(record.f0, Row.copy(record.f1)) : record; + public void addRecord(Tuple2 record) { // add records to buffer - keyToRows.put(getPrimaryKey(tuple2.f1), tuple2); + keyToRows.put(getPrimaryKey(record.f1), record); } @Override @@ -153,10 +148,9 @@ public abstract class UpsertWriter implements JDBCWriter { int[] fieldTypes, int[] pkFields, int[] pkTypes, - boolean objectReuse, String deleteSQL, String upsertSQL) { - super(fieldTypes, pkFields, pkTypes, deleteSQL, objectReuse); + super(fieldTypes, pkFields, pkTypes, deleteSQL); this.upsertSQL = upsertSQL; } @@ -202,12 +196,11 @@ public abstract class UpsertWriter implements JDBCWriter { int[] fieldTypes, int[] pkFields, int[] pkTypes, - boolean objectReuse, String deleteSQL, String existSQL, String insertSQL, String updateSQL) { - super(fieldTypes, pkFields, pkTypes, deleteSQL, objectReuse); + super(fieldTypes, pkFields, pkTypes, deleteSQL); this.existSQL = existSQL; this.insertSQL = insertSQL; this.updateSQL = updateSQL; diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCAppenOnlyWriterTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCAppenOnlyWriterTest.java new file mode 100644 index 0000000000..36bae77b43 --- /dev/null +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCAppenOnlyWriterTest.java @@ -0,0 +1,105 @@ +/* + * 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.api.java.io.jdbc; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.java.io.jdbc.writer.AppendOnlyWriter; +import org.apache.flink.api.java.tuple.Tuple2; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; + +import java.sql.BatchUpdateException; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.Statement; + +import static org.apache.flink.api.java.io.jdbc.JDBCOutputFormatTest.toRow; +import static org.mockito.Mockito.doReturn; + +/** + * Test for the {@link AppendOnlyWriter}. + */ +public class JDBCAppenOnlyWriterTest extends JDBCTestBase { + + private JDBCUpsertOutputFormat format; + private String[] fieldNames; + + @Before + public void setup() { + fieldNames = new String[]{"id", "title", "author", "price", "qty"}; + } + + @Test(expected = BatchUpdateException.class) + public void testMaxRetry() throws Exception { + format = JDBCUpsertOutputFormat.builder() + .setOptions(JDBCOptions.builder() + .setDBUrl(DB_URL) + .setTableName(OUTPUT_TABLE) + .build()) + .setFieldNames(fieldNames) + .setKeyFields(null) + .build(); + RuntimeContext context = Mockito.mock(RuntimeContext.class); + ExecutionConfig config = Mockito.mock(ExecutionConfig.class); + doReturn(config).when(context).getExecutionConfig(); + doReturn(true).when(config).isObjectReuseEnabled(); + format.setRuntimeContext(context); + format.open(0, 1); + + // alter table schema to trigger retry logic after failure. + alterTable(); + for (TestEntry entry : TEST_DATA) { + format.writeRecord(Tuple2.of(true, toRow(entry))); + } + + // after retry default times, throws a BatchUpdateException. + format.flush(); + } + + private void alterTable() throws Exception { + Class.forName(DRIVER_CLASS); + try (Connection conn = DriverManager.getConnection(DB_URL); + Statement stat = conn.createStatement()) { + stat.execute("ALTER TABLE " + OUTPUT_TABLE + " DROP COLUMN " + fieldNames[1]); + } + } + + @After + public void clear() throws Exception { + if (format != null) { + try { + format.close(); + } catch (RuntimeException e) { + // ignore exception when close. + } + } + format = null; + Class.forName(DRIVER_CLASS); + try ( + Connection conn = DriverManager.getConnection(DB_URL); + Statement stat = conn.createStatement()) { + stat.execute("DELETE FROM " + OUTPUT_TABLE); + } + } + +} -- Gitee From 242efcdc3170341c5f8ccef5b29f3317cead3fa3 Mon Sep 17 00:00:00 2001 From: Kurt Young Date: Sun, 1 Mar 2020 20:47:07 +0800 Subject: [PATCH 023/885] [FLINK-12814][sql-client] Support a traditional and scrolling view of result (tableau format) This closes #11273 --- .../flink/table/client/cli/CliClient.java | 38 +- .../client/cli/CliTableauResultView.java | 404 ++++++++++++++++++ .../client/config/entries/ExecutionEntry.java | 8 + .../client/gateway/ResultDescriptor.java | 13 +- .../client/gateway/local/LocalExecutor.java | 3 +- .../client/gateway/local/ResultStore.java | 10 +- .../table/client/cli/CliResultViewTest.java | 7 +- .../client/cli/CliTableauResultViewTest.java | 373 ++++++++++++++++ .../table/client/cli/utils/TerminalUtils.java | 6 +- 9 files changed, 840 insertions(+), 22 deletions(-) create mode 100644 flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliTableauResultView.java create mode 100644 flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliTableauResultViewTest.java diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliClient.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliClient.java index 797bca6859..76039f8e3e 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliClient.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliClient.java @@ -512,21 +512,35 @@ public class CliClient { printExecutionException(e); return; } - final CliResultView view; - if (resultDesc.isMaterialized()) { - view = new CliTableResultView(this, resultDesc); + + if (resultDesc.isTableauMode()) { + try (CliTableauResultView tableauResultView = new CliTableauResultView( + terminal, executor, sessionId, resultDesc)) { + if (resultDesc.isMaterialized()) { + tableauResultView.displayBatchResults(); + } else { + tableauResultView.displayStreamResults(); + } + } catch (SqlExecutionException e) { + printExecutionException(e); + } } else { - view = new CliChangelogResultView(this, resultDesc); - } + final CliResultView view; + if (resultDesc.isMaterialized()) { + view = new CliTableResultView(this, resultDesc); + } else { + view = new CliChangelogResultView(this, resultDesc); + } - // enter view - try { - view.open(); + // enter view + try { + view.open(); - // view left - printInfo(CliStrings.MESSAGE_RESULT_QUIT); - } catch (SqlExecutionException e) { - printExecutionException(e); + // view left + printInfo(CliStrings.MESSAGE_RESULT_QUIT); + } catch (SqlExecutionException e) { + printExecutionException(e); + } } } diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliTableauResultView.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliTableauResultView.java new file mode 100644 index 0000000000..e82a026b1d --- /dev/null +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliTableauResultView.java @@ -0,0 +1,404 @@ +/* + * 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.table.client.cli; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.table.api.TableColumn; +import org.apache.flink.table.client.gateway.Executor; +import org.apache.flink.table.client.gateway.ResultDescriptor; +import org.apache.flink.table.client.gateway.SqlExecutionException; +import org.apache.flink.table.client.gateway.TypedResult; +import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LocalZonedTimestampType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.SmallIntType; +import org.apache.flink.table.types.logical.TimeType; +import org.apache.flink.table.types.logical.TimestampType; +import org.apache.flink.table.types.logical.TinyIntType; +import org.apache.flink.types.Row; + +import org.apache.commons.lang3.StringUtils; +import org.jline.terminal.Terminal; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CancellationException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Stream; + +import static org.apache.flink.table.client.cli.CliUtils.rowToString; + +/** + * Print result in tableau mode. + */ +public class CliTableauResultView implements AutoCloseable { + + private static final int NULL_COLUMN_WIDTH = CliStrings.NULL_COLUMN.length(); + private static final int MAX_COLUMN_WIDTH = 30; + private static final int DEFAULT_COLUMN_WIDTH = 20; + private static final String COLUMN_TRUNCATED_FLAG = "..."; + private static final String CHANGEFLAG_COLUMN_NAME = "+/-"; + + private final Terminal terminal; + private final Executor sqlExecutor; + private final String sessionId; + private final ResultDescriptor resultDescriptor; + private final ExecutorService displayResultExecutorService; + + public CliTableauResultView( + final Terminal terminal, + final Executor sqlExecutor, + final String sessionId, + final ResultDescriptor resultDescriptor) { + this.terminal = terminal; + this.sqlExecutor = sqlExecutor; + this.sessionId = sessionId; + this.resultDescriptor = resultDescriptor; + this.displayResultExecutorService = Executors.newSingleThreadExecutor(); + } + + public void displayStreamResults() throws SqlExecutionException { + final AtomicInteger receivedRowCount = new AtomicInteger(0); + Future resultFuture = displayResultExecutorService.submit(() -> { + printStreamResults(receivedRowCount); + }); + + // capture CTRL-C + terminal.handle(Terminal.Signal.INT, signal -> { + resultFuture.cancel(true); + }); + + boolean cleanUpQuery = true; + try { + resultFuture.get(); + cleanUpQuery = false; // job finished successfully + } catch (CancellationException e) { + terminal.writer().println("Query terminated, received a total of " + receivedRowCount.get() + " rows"); + terminal.flush(); + } catch (ExecutionException e) { + if (e.getCause() instanceof SqlExecutionException) { + throw (SqlExecutionException) e.getCause(); + } + throw new SqlExecutionException("unknown exception", e.getCause()); + } catch (InterruptedException e) { + throw new SqlExecutionException("Query interrupted", e); + } finally { + checkAndCleanUpQuery(cleanUpQuery); + } + } + + public void displayBatchResults() throws SqlExecutionException { + Future resultFuture = displayResultExecutorService.submit(() -> { + final List resultRows = waitBatchResults(); + printBatchResults(resultRows); + }); + + // capture CTRL-C + terminal.handle(Terminal.Signal.INT, signal -> { + resultFuture.cancel(true); + }); + + boolean cleanUpQuery = true; + try { + resultFuture.get(); + cleanUpQuery = false; // job finished successfully + } catch (CancellationException e) { + terminal.writer().println("Query terminated"); + terminal.flush(); + } catch (ExecutionException e) { + if (e.getCause() instanceof SqlExecutionException) { + throw (SqlExecutionException) e.getCause(); + } + throw new SqlExecutionException("unknown exception", e.getCause()); + } catch (InterruptedException e) { + throw new SqlExecutionException("Query interrupted", e); + } finally { + checkAndCleanUpQuery(cleanUpQuery); + } + } + + @Override + public void close() { + this.displayResultExecutorService.shutdown(); + } + + private void checkAndCleanUpQuery(boolean cleanUpQuery) { + if (cleanUpQuery) { + try { + sqlExecutor.cancelQuery(sessionId, resultDescriptor.getResultId()); + } catch (SqlExecutionException e) { + // ignore further exceptions + } + } + } + + private List waitBatchResults() { + List resultRows; + // take snapshot and make all results in one page + do { + try { + Thread.sleep(50); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + TypedResult result = sqlExecutor.snapshotResult( + sessionId, + resultDescriptor.getResultId(), + Integer.MAX_VALUE); + + if (result.getType() == TypedResult.ResultType.EOS) { + resultRows = Collections.emptyList(); + break; + } else if (result.getType() == TypedResult.ResultType.PAYLOAD) { + resultRows = sqlExecutor.retrieveResultPage(resultDescriptor.getResultId(), 1); + break; + } else { + // result not retrieved yet + } + } while (true); + + return resultRows; + } + + private void printStreamResults(AtomicInteger receivedRowCount) { + List columns = resultDescriptor.getResultSchema().getTableColumns(); + String[] fieldNames = + Stream.concat( + Stream.of("+/-"), + columns.stream().map(TableColumn::getName) + ).toArray(String[]::new); + + int[] colWidths = columnWidthsByType(columns, true); + String borderline = genBorderLine(colWidths); + + // print filed names + terminal.writer().println(borderline); + printSingleRow(colWidths, fieldNames); + terminal.writer().println(borderline); + terminal.flush(); + + while (true) { + final TypedResult>> result = + sqlExecutor.retrieveResultChanges(sessionId, resultDescriptor.getResultId()); + + switch (result.getType()) { + case EMPTY: + // do nothing + break; + case EOS: + if (receivedRowCount.get() > 0) { + terminal.writer().println(borderline); + } + terminal.writer().println("Received a total of " + receivedRowCount.get() + " rows"); + terminal.flush(); + return; + case PAYLOAD: + List> changes = result.getPayload(); + for (Tuple2 change : changes) { + final String[] cols = rowToString(change.f1); + String[] row = new String[cols.length + 1]; + row[0] = change.f0 ? "+" : "-"; + System.arraycopy(cols, 0, row, 1, cols.length); + printSingleRow(colWidths, row); + receivedRowCount.incrementAndGet(); + } + break; + default: + throw new SqlExecutionException("Unknown result type: " + result.getType()); + } + } + } + + private void printBatchResults(List resultRows) { + List rows = new ArrayList<>(resultRows.size() + 1); + + // fill field names first + List columns = resultDescriptor.getResultSchema().getTableColumns(); + rows.add(columns.stream().map(TableColumn::getName).toArray(String[]::new)); + resultRows.forEach(row -> rows.add(rowToString(row))); + + int[] colWidths = columnWidthsByContent(columns, rows); + String borderline = genBorderLine(colWidths); + + // print field names + terminal.writer().println(borderline); + printSingleRow(colWidths, rows.get(0)); + terminal.writer().println(borderline); + + // print content + rows.subList(1, rows.size()).forEach(row -> printSingleRow(colWidths, row)); + if (!resultRows.isEmpty()) { + terminal.writer().println(borderline); + } + + // print footer + terminal.writer().println(resultRows.size() + " row in set"); + terminal.flush(); + } + + private String genBorderLine(int[] colWidths) { + StringBuilder sb = new StringBuilder(); + sb.append("+"); + for (int width : colWidths) { + sb.append(StringUtils.repeat('-', width + 1)); + sb.append("-+"); + } + return sb.toString(); + } + + private void printSingleRow(int[] colWidths, String[] cols) { + StringBuilder sb = new StringBuilder(); + sb.append("|"); + int idx = 0; + for (String col : cols) { + sb.append(" "); + if (col.length() <= colWidths[idx]) { + sb.append(StringUtils.repeat(' ', colWidths[idx] - col.length())); + sb.append(col); + } else { + sb.append(col, 0, colWidths[idx] - COLUMN_TRUNCATED_FLAG.length()); + sb.append(COLUMN_TRUNCATED_FLAG); + } + sb.append(" |"); + idx++; + } + terminal.writer().println(sb.toString()); + terminal.flush(); + } + + /** + * Try to infer column width based on column types. In streaming case, we will have an + * endless result set, thus couldn't determine column widths based on column values. + */ + private int[] columnWidthsByType(List columns, boolean includeChangeflag) { + // fill width with field names first + int[] colWidths = columns.stream() + .mapToInt(col -> col.getName().length()) + .toArray(); + + // determine proper column width based on types + for (int i = 0; i < columns.size(); ++i) { + LogicalType type = columns.get(i).getType().getLogicalType(); + int len; + switch (type.getTypeRoot()) { + case TINYINT: + len = TinyIntType.PRECISION + 1; // extra for negative value + break; + case SMALLINT: + len = SmallIntType.PRECISION + 1; // extra for negative value + break; + case INTEGER: + len = IntType.PRECISION + 1; // extra for negative value + break; + case BIGINT: + len = BigIntType.PRECISION + 1; // extra for negative value + break; + case DECIMAL: + len = ((DecimalType) type).getPrecision() + 2; // extra for negative value and decimal point + break; + case BOOLEAN: + len = 5; // "true" or "false" + break; + case DATE: + len = 10; // e.g. 9999-12-31 + break; + case TIME_WITHOUT_TIME_ZONE: + int precision = ((TimeType) type).getPrecision(); + len = precision == 0 ? 8 : precision + 9; // 23:59:59[.999999999] + break; + case TIMESTAMP_WITHOUT_TIME_ZONE: + precision = ((TimestampType) type).getPrecision(); + len = timestampTypeColumnWidth(precision); + break; + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + precision = ((LocalZonedTimestampType) type).getPrecision(); + len = timestampTypeColumnWidth(precision); + break; + default: + len = DEFAULT_COLUMN_WIDTH; + } + + // adjust column width with potential null values + colWidths[i] = Math.max(colWidths[i], Math.max(len, NULL_COLUMN_WIDTH)); + } + + // add an extra column for change flag if necessary + if (includeChangeflag) { + int[] ret = new int[columns.size() + 1]; + ret[0] = CHANGEFLAG_COLUMN_NAME.length(); + System.arraycopy(colWidths, 0, ret, 1, columns.size()); + return ret; + } else { + return colWidths; + } + } + + /** + * Here we consider two popular class for timestamp: LocalDateTime and java.sql.Timestamp. + * + *

According to LocalDateTime's comment, the string output will be one of the following + * ISO-8601 formats: + *

  • {@code uuuu-MM-dd'T'HH:mm:ss}
  • + *
  • {@code uuuu-MM-dd'T'HH:mm:ss.SSS}
  • + *
  • {@code uuuu-MM-dd'T'HH:mm:ss.SSSSSS}
  • + *
  • {@code uuuu-MM-dd'T'HH:mm:ss.SSSSSSSSS}
  • + * + *

    And for java.sql.Timestamp, the number of digits after point will be precision except + * when precision is 0. In that case, the format would be 'uuuu-MM-dd HH:mm:ss.0' + */ + int timestampTypeColumnWidth(int precision) { + int base = 19; // length of uuuu-MM-dd HH:mm:ss + if (precision == 0) { + return base + 2; // consider java.sql.Timestamp + } else if (precision <= 3) { + return base + 4; + } else if (precision <= 6) { + return base + 7; + } else { + return base + 10; + } + } + + private int[] columnWidthsByContent(List columns, List rows) { + // fill width with field names first + int[] colWidths = columns.stream().mapToInt(col -> col.getName().length()).toArray(); + + // fill column width with real data + for (String[] row : rows) { + for (int i = 0; i < row.length; ++i) { + colWidths[i] = Math.max(colWidths[i], row[i].length()); + } + } + + // adjust column width with maximum length + for (int i = 0; i < colWidths.length; ++i) { + colWidths[i] = Math.min(colWidths[i], MAX_COLUMN_WIDTH); + } + + return colWidths; + } + +} diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/config/entries/ExecutionEntry.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/config/entries/ExecutionEntry.java index a65642bb89..6e60c9594a 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/config/entries/ExecutionEntry.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/config/entries/ExecutionEntry.java @@ -84,6 +84,8 @@ public class ExecutionEntry extends ConfigEntry { private static final String EXECUTION_RESULT_MODE_VALUE_TABLE = "table"; + private static final String EXECUTION_RESULT_MODE_VALUE_TABLEAU = "tableau"; + private static final String EXECUTION_MAX_TABLE_RESULT_ROWS = "max-table-result-rows"; private static final String EXECUTION_RESTART_STRATEGY_TYPE = "restart-strategy.type"; @@ -322,6 +324,12 @@ public class ExecutionEntry extends ConfigEntry { .orElse(false); } + public boolean isTableauMode() { + return properties.getOptionalString(EXECUTION_RESULT_MODE) + .map((v) -> v.equals(EXECUTION_RESULT_MODE_VALUE_TABLEAU)) + .orElse(false); + } + public Map asTopLevelMap() { return properties.asPrefixedMap(EXECUTION_ENTRY + '.'); } diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/ResultDescriptor.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/ResultDescriptor.java index 3cfaa6f882..265e5c4825 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/ResultDescriptor.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/ResultDescriptor.java @@ -31,10 +31,17 @@ public class ResultDescriptor { private final boolean isMaterialized; - public ResultDescriptor(String resultId, TableSchema resultSchema, boolean isMaterialized) { + private final boolean isTableauMode; + + public ResultDescriptor( + String resultId, + TableSchema resultSchema, + boolean isMaterialized, + boolean isTableauMode) { this.resultId = resultId; this.resultSchema = resultSchema; this.isMaterialized = isMaterialized; + this.isTableauMode = isTableauMode; } public String getResultId() { @@ -48,4 +55,8 @@ public class ResultDescriptor { public boolean isMaterialized() { return isMaterialized; } + + public boolean isTableauMode() { + return isTableauMode; + } } diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java index b1445cbd9d..3035d91acf 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java @@ -663,7 +663,8 @@ public class LocalExecutor implements Executor { return new ResultDescriptor( resultId, removeTimeAttributes(table.getSchema()), - result.isMaterialized()); + result.isMaterialized(), + context.getEnvironment().getExecution().isTableauMode()); } /** diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ResultStore.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ResultStore.java index 2cea4aa797..476296f5a3 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ResultStore.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ResultStore.java @@ -67,7 +67,7 @@ public class ResultStore { final InetAddress gatewayAddress = getGatewayAddress(env.getDeployment()); final int gatewayPort = getGatewayPort(env.getDeployment()); - if (env.getExecution().isChangelogMode()) { + if (env.getExecution().isChangelogMode() || env.getExecution().isTableauMode()) { return new ChangelogCollectStreamResult<>( schema, config, @@ -86,10 +86,12 @@ public class ResultStore { } else { // Batch Execution - if (!env.getExecution().isTableMode()) { - throw new SqlExecutionException("Results of batch queries can only be served in table mode."); + if (env.getExecution().isTableMode() || env.getExecution().isTableauMode()) { + return new MaterializedCollectBatchResult<>(schema, config, classLoader); + } else { + throw new SqlExecutionException( + "Results of batch queries can only be served in table or tableau mode."); } - return new MaterializedCollectBatchResult<>(schema, config, classLoader); } } diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliResultViewTest.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliResultViewTest.java index 29ac7a1bb0..7d5bca0dc9 100644 --- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliResultViewTest.java +++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliResultViewTest.java @@ -84,9 +84,10 @@ public class CliResultViewTest { final MockExecutor executor = new MockExecutor(typedResult, cancellationCounterLatch); String sessionId = executor.openSession(session); final ResultDescriptor descriptor = new ResultDescriptor( - "result-id", - TableSchema.builder().field("Null Field", Types.STRING()).build(), - false); + "result-id", + TableSchema.builder().field("Null Field", Types.STRING()).build(), + false, + false); Thread resultViewRunner = null; CliClient cli = null; diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliTableauResultViewTest.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliTableauResultViewTest.java new file mode 100644 index 0000000000..3ab8ae3bf9 --- /dev/null +++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliTableauResultViewTest.java @@ -0,0 +1,373 @@ +/* + * 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.table.client.cli; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.client.cli.utils.TerminalUtils; +import org.apache.flink.table.client.gateway.Executor; +import org.apache.flink.table.client.gateway.ResultDescriptor; +import org.apache.flink.table.client.gateway.SqlExecutionException; +import org.apache.flink.table.client.gateway.TypedResult; +import org.apache.flink.types.Row; + +import org.jline.terminal.Terminal; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.ByteArrayOutputStream; +import java.math.BigDecimal; +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.timeout; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** + * Tests for CliTableauResultView. + */ +public class CliTableauResultViewTest { + + private ByteArrayOutputStream terminalOutput; + private Terminal terminal; + private TableSchema schema; + private List data; + private List> streamingData; + + @Before + public void setUp() { + terminalOutput = new ByteArrayOutputStream(); + terminal = TerminalUtils.createDummyTerminal(terminalOutput); + + schema = TableSchema.builder() + .field("boolean", DataTypes.BOOLEAN()) + .field("int", DataTypes.INT()) + .field("bigint", DataTypes.BIGINT()) + .field("varchar", DataTypes.STRING()) + .field("decimal(10, 5)", DataTypes.DECIMAL(10, 5)) + .field("timestamp", DataTypes.TIMESTAMP(6)) + .build(); + + data = new ArrayList<>(); + data.add( + Row.of( + null, + 1, + 2, + "abc", + BigDecimal.valueOf(1.23), + Timestamp.valueOf("2020-03-01 18:39:14")) + ); + data.add( + Row.of( + false, + null, + 0, + "", + BigDecimal.valueOf(1), + Timestamp.valueOf("2020-03-01 18:39:14.1")) + ); + data.add( + Row.of( + true, + Integer.MAX_VALUE, + null, + "abcdefg", + BigDecimal.valueOf(1234567890), + Timestamp.valueOf("2020-03-01 18:39:14.12")) + ); + data.add( + Row.of( + false, + Integer.MIN_VALUE, + Long.MAX_VALUE, + null, + BigDecimal.valueOf(12345.06789), + Timestamp.valueOf("2020-03-01 18:39:14.123")) + ); + data.add( + Row.of( + true, + 100, + Long.MIN_VALUE, + "abcdefg111", + null, + Timestamp.valueOf("2020-03-01 18:39:14.123456")) + ); + data.add( + Row.of( + null, + -1, + -1, + "abcdefghijklmnopqrstuvwxyz", + BigDecimal.valueOf(-12345.06789), + null) + ); + + streamingData = new ArrayList<>(); + for (int i = 0; i < data.size(); ++i) { + streamingData.add(new Tuple2<>(i % 2 == 0, data.get(i))); + } + } + + @Test + public void testBatchResult() { + ResultDescriptor resultDescriptor = new ResultDescriptor("", schema, true, true); + Executor mockExecutor = mock(Executor.class); + CliTableauResultView view = new CliTableauResultView( + terminal, mockExecutor, "session", resultDescriptor); + + when(mockExecutor.snapshotResult(anyString(), anyString(), anyInt())) + .thenReturn(TypedResult.payload(1)) + .thenReturn(TypedResult.endOfStream()); + when(mockExecutor.retrieveResultPage(anyString(), anyInt())) + .thenReturn(data); + + view.displayBatchResults(); + view.close(); + + Assert.assertEquals( + "+---------+-------------+----------------------+----------------------------+----------------+----------------------------+\n" + + "| boolean | int | bigint | varchar | decimal(10, 5) | timestamp |\n" + + "+---------+-------------+----------------------+----------------------------+----------------+----------------------------+\n" + + "| (NULL) | 1 | 2 | abc | 1.23 | 2020-03-01 18:39:14.0 |\n" + + "| false | (NULL) | 0 | | 1 | 2020-03-01 18:39:14.1 |\n" + + "| true | 2147483647 | (NULL) | abcdefg | 1234567890 | 2020-03-01 18:39:14.12 |\n" + + "| false | -2147483648 | 9223372036854775807 | (NULL) | 12345.06789 | 2020-03-01 18:39:14.123 |\n" + + "| true | 100 | -9223372036854775808 | abcdefg111 | (NULL) | 2020-03-01 18:39:14.123456 |\n" + + "| (NULL) | -1 | -1 | abcdefghijklmnopqrstuvwxyz | -12345.06789 | (NULL) |\n" + + "+---------+-------------+----------------------+----------------------------+----------------+----------------------------+\n" + + "6 row in set\n", + terminalOutput.toString()); + verify(mockExecutor, times(0)).cancelQuery(anyString(), anyString()); + } + + @Test + public void testCancelBatchResult() throws InterruptedException, ExecutionException, TimeoutException { + ResultDescriptor resultDescriptor = new ResultDescriptor("", schema, true, true); + Executor mockExecutor = mock(Executor.class); + CliTableauResultView view = new CliTableauResultView( + terminal, mockExecutor, "session", resultDescriptor); + + when(mockExecutor.snapshotResult(anyString(), anyString(), anyInt())) + .thenReturn(TypedResult.empty()); + + // submit result display in another thread + ExecutorService executorService = Executors.newSingleThreadExecutor(); + Future furture = executorService.submit(view::displayBatchResults); + + // wait until we trying to get batch result + verify(mockExecutor, timeout(5000).atLeast(1)).snapshotResult(anyString(), anyString(), anyInt()); + + // send signal to cancel + terminal.raise(Terminal.Signal.INT); + furture.get(5, TimeUnit.SECONDS); + + Assert.assertEquals("Query terminated\n", terminalOutput.toString()); + // didn't have a chance to read page + verify(mockExecutor, times(0)).retrieveResultPage(anyString(), anyInt()); + // tried to cancel query + verify(mockExecutor, times(1)).cancelQuery(anyString(), anyString()); + + view.close(); + } + + @Test + public void testEmptyBatchResult() { + ResultDescriptor resultDescriptor = new ResultDescriptor("", schema, true, true); + Executor mockExecutor = mock(Executor.class); + CliTableauResultView view = new CliTableauResultView( + terminal, mockExecutor, "session", resultDescriptor); + + when(mockExecutor.snapshotResult(anyString(), anyString(), anyInt())) + .thenReturn(TypedResult.payload(1)) + .thenReturn(TypedResult.endOfStream()); + when(mockExecutor.retrieveResultPage(anyString(), anyInt())) + .thenReturn(Collections.emptyList()); + + view.displayBatchResults(); + view.close(); + + Assert.assertEquals( + "+---------+-----+--------+---------+----------------+-----------+\n" + + "| boolean | int | bigint | varchar | decimal(10, 5) | timestamp |\n" + + "+---------+-----+--------+---------+----------------+-----------+\n" + + "0 row in set\n", + terminalOutput.toString()); + verify(mockExecutor, times(0)).cancelQuery(anyString(), anyString()); + } + + @Test + public void testFailedBatchResult() { + ResultDescriptor resultDescriptor = new ResultDescriptor("", schema, true, true); + Executor mockExecutor = mock(Executor.class); + CliTableauResultView view = new CliTableauResultView( + terminal, mockExecutor, "session", resultDescriptor); + + when(mockExecutor.snapshotResult(anyString(), anyString(), anyInt())) + .thenReturn(TypedResult.payload(1)) + .thenReturn(TypedResult.endOfStream()); + when(mockExecutor.retrieveResultPage(anyString(), anyInt())) + .thenThrow(new SqlExecutionException("query failed")); + + try { + view.displayBatchResults(); + Assert.fail("Shouldn't get here"); + } catch (SqlExecutionException e) { + Assert.assertEquals("query failed", e.getMessage()); + } + view.close(); + + verify(mockExecutor, times(1)).cancelQuery(anyString(), anyString()); + } + + @Test + public void testStreamingResult() { + ResultDescriptor resultDescriptor = new ResultDescriptor("", schema, true, true); + Executor mockExecutor = mock(Executor.class); + CliTableauResultView view = new CliTableauResultView( + terminal, mockExecutor, "session", resultDescriptor); + + when(mockExecutor.retrieveResultChanges(anyString(), anyString())) + .thenReturn(TypedResult.payload(streamingData.subList(0, streamingData.size() / 2))) + .thenReturn(TypedResult.payload(streamingData.subList(streamingData.size() / 2, streamingData.size()))) + .thenReturn(TypedResult.endOfStream()); + + view.displayStreamResults(); + view.close(); + + Assert.assertEquals( + "+-----+---------+-------------+----------------------+----------------------+----------------+----------------------------+\n" + + "| +/- | boolean | int | bigint | varchar | decimal(10, 5) | timestamp |\n" + + "+-----+---------+-------------+----------------------+----------------------+----------------+----------------------------+\n" + + "| + | (NULL) | 1 | 2 | abc | 1.23 | 2020-03-01 18:39:14.0 |\n" + + "| - | false | (NULL) | 0 | | 1 | 2020-03-01 18:39:14.1 |\n" + + "| + | true | 2147483647 | (NULL) | abcdefg | 1234567890 | 2020-03-01 18:39:14.12 |\n" + + "| - | false | -2147483648 | 9223372036854775807 | (NULL) | 12345.06789 | 2020-03-01 18:39:14.123 |\n" + + "| + | true | 100 | -9223372036854775808 | abcdefg111 | (NULL) | 2020-03-01 18:39:14.123456 |\n" + + "| - | (NULL) | -1 | -1 | abcdefghijklmnopq... | -12345.06789 | (NULL) |\n" + + "+-----+---------+-------------+----------------------+----------------------+----------------+----------------------------+\n" + + "Received a total of 6 rows\n", + terminalOutput.toString()); + verify(mockExecutor, times(0)).cancelQuery(anyString(), anyString()); + } + + @Test + public void testEmptyStreamingResult() { + ResultDescriptor resultDescriptor = new ResultDescriptor("", schema, true, true); + Executor mockExecutor = mock(Executor.class); + CliTableauResultView view = new CliTableauResultView( + terminal, mockExecutor, "session", resultDescriptor); + + when(mockExecutor.retrieveResultChanges(anyString(), anyString())) + .thenReturn(TypedResult.endOfStream()); + + view.displayStreamResults(); + view.close(); + + Assert.assertEquals( + "+-----+---------+-------------+----------------------+----------------------+----------------+----------------------------+\n" + + "| +/- | boolean | int | bigint | varchar | decimal(10, 5) | timestamp |\n" + + "+-----+---------+-------------+----------------------+----------------------+----------------+----------------------------+\n" + + "Received a total of 0 rows\n", + terminalOutput.toString()); + verify(mockExecutor, times(0)).cancelQuery(anyString(), anyString()); + } + + @Test + public void testCancelStreamingResult() throws InterruptedException, ExecutionException, TimeoutException { + ResultDescriptor resultDescriptor = new ResultDescriptor("", schema, true, true); + Executor mockExecutor = mock(Executor.class); + CliTableauResultView view = new CliTableauResultView( + terminal, mockExecutor, "session", resultDescriptor); + + when(mockExecutor.retrieveResultChanges(anyString(), anyString())) + .thenReturn(TypedResult.payload(streamingData.subList(0, streamingData.size() / 2))) + .thenReturn(TypedResult.empty()); + + // submit result display in another thread + ExecutorService executorService = Executors.newSingleThreadExecutor(); + Future furture = executorService.submit(view::displayStreamResults); + + // wait until we processed first result + verify(mockExecutor, timeout(5000).atLeast(2)).retrieveResultChanges(anyString(), anyString()); + + // send signal to cancel + terminal.raise(Terminal.Signal.INT); + furture.get(5, TimeUnit.SECONDS); + view.close(); + + Assert.assertEquals( + "+-----+---------+-------------+----------------------+----------------------+----------------+----------------------------+\n" + + "| +/- | boolean | int | bigint | varchar | decimal(10, 5) | timestamp |\n" + + "+-----+---------+-------------+----------------------+----------------------+----------------+----------------------------+\n" + + "| + | (NULL) | 1 | 2 | abc | 1.23 | 2020-03-01 18:39:14.0 |\n" + + "| - | false | (NULL) | 0 | | 1 | 2020-03-01 18:39:14.1 |\n" + + "| + | true | 2147483647 | (NULL) | abcdefg | 1234567890 | 2020-03-01 18:39:14.12 |\n" + + "Query terminated, received a total of 3 rows\n", + terminalOutput.toString()); + + verify(mockExecutor, times(1)).cancelQuery(anyString(), anyString()); + } + + @Test + public void testFailedStreamingResult() { + ResultDescriptor resultDescriptor = new ResultDescriptor("", schema, true, true); + Executor mockExecutor = mock(Executor.class); + CliTableauResultView view = new CliTableauResultView( + terminal, mockExecutor, "session", resultDescriptor); + + when(mockExecutor.retrieveResultChanges(anyString(), anyString())) + .thenReturn(TypedResult.payload(streamingData.subList(0, streamingData.size() / 2))) + .thenThrow(new SqlExecutionException("query failed")); + + try { + view.displayStreamResults(); + Assert.fail("Shouldn't get here"); + } catch (SqlExecutionException e) { + Assert.assertEquals("query failed", e.getMessage()); + } + view.close(); + + Assert.assertEquals( + "+-----+---------+-------------+----------------------+----------------------+----------------+----------------------------+\n" + + "| +/- | boolean | int | bigint | varchar | decimal(10, 5) | timestamp |\n" + + "+-----+---------+-------------+----------------------+----------------------+----------------+----------------------------+\n" + + "| + | (NULL) | 1 | 2 | abc | 1.23 | 2020-03-01 18:39:14.0 |\n" + + "| - | false | (NULL) | 0 | | 1 | 2020-03-01 18:39:14.1 |\n" + + "| + | true | 2147483647 | (NULL) | abcdefg | 1234567890 | 2020-03-01 18:39:14.12 |\n", + terminalOutput.toString()); + verify(mockExecutor, times(1)).cancelQuery(anyString(), anyString()); + } +} diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/utils/TerminalUtils.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/utils/TerminalUtils.java index 772fb013e3..0ee65cc4aa 100644 --- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/utils/TerminalUtils.java +++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/utils/TerminalUtils.java @@ -35,8 +35,12 @@ public class TerminalUtils { } public static Terminal createDummyTerminal() { + return createDummyTerminal(new MockOutputStream()); + } + + public static Terminal createDummyTerminal(OutputStream out) { try { - return new DumbTerminal(new MockInputStream(), new MockOutputStream()); + return new DumbTerminal(new MockInputStream(), out); } catch (IOException e) { throw new RuntimeException("Unable to create dummy terminal."); } -- Gitee From 53079f7f04059dd353122deaa5889ad4733835e1 Mon Sep 17 00:00:00 2001 From: CrazyTomatoOo <1650459038@qq.com> Date: Fri, 10 Jan 2020 11:37:08 +0800 Subject: [PATCH 024/885] [FLINK-15081][docs-zh] Translate "Concepts & Common API" page of Table This closes #10822 --- docs/dev/table/common.zh.md | 349 +++++++++++++++++------------------- 1 file changed, 164 insertions(+), 185 deletions(-) diff --git a/docs/dev/table/common.zh.md b/docs/dev/table/common.zh.md index c2b579cf3e..ff6b95a19d 100644 --- a/docs/dev/table/common.zh.md +++ b/docs/dev/table/common.zh.md @@ -1,5 +1,5 @@ --- -title: "Concepts & Common API" +title: "概念与通用 API" nav-parent_id: tableapi nav-pos: 0 --- @@ -22,28 +22,28 @@ specific language governing permissions and limitations under the License. --> -The Table API and SQL are integrated in a joint API. The central concept of this API is a `Table` which serves as input and output of queries. This document shows the common structure of programs with Table API and SQL queries, how to register a `Table`, how to query a `Table`, and how to emit a `Table`. +Table API 和 SQL 集成在同一套 API 中。这套 API 的核心概念是`Table`,用作查询的输入和输出。本文介绍了 Table API 和 SQL 查询程序的通用结构、如何注册 `Table` 、如何查询 `Table` 以及如何输出 `Table` 。 * This will be replaced by the TOC {:toc} -Main Differences Between the Two Planners +两种计划器(Planner)的主要区别 ----------------------------------------- -1. Blink treats batch jobs as a special case of streaming. As such, the conversion between Table and DataSet is also not supported, and batch jobs will not be translated into `DateSet` programs but translated into `DataStream` programs, the same as the streaming jobs. -2. The Blink planner does not support `BatchTableSource`, use bounded `StreamTableSource` instead of it. -3. The Blink planner only support the brand new `Catalog` and does not support `ExternalCatalog` which is deprecated. -4. The implementations of `FilterableTableSource` for the old planner and the Blink planner are incompatible. The old planner will push down `PlannerExpression`s into `FilterableTableSource`, while the Blink planner will push down `Expression`s. -5. String based key-value config options (Please see the documentation about [Configuration]({{ site.baseurl }}/dev/table/config.html) for details) are only used for the Blink planner. -6. The implementation(`CalciteConfig`) of `PlannerConfig` in two planners is different. -7. The Blink planner will optimize multiple-sinks into one DAG (supported only on `TableEnvironment`, not on `StreamTableEnvironment`). The old planner will always optimize each sink into a new DAG, where all DAGs are independent of each other. -8. The old planner does not support catalog statistics now, while the Blink planner does. +1. Blink 将批处理作业视作流处理的一种特例。严格来说,`Table` 和 `DataSet` 之间不支持相互转换,并且批处理作业也不会转换成 `DataSet` 程序而是转换成 `DataStream` 程序,流处理作业也一样。 +2. Blink 计划器不支持 `BatchTableSource`,而是使用有界的 `StreamTableSource` 来替代。 +3. Blink 计划器仅支持全新的 `Catalog` 不支持被弃用的 `ExternalCatalog`。 +4. 旧计划器和 Blink 计划器中 `FilterableTableSource` 的实现是不兼容的。旧计划器会将 `PlannerExpression` 下推至 `FilterableTableSource`,而 Blink 计划器则是将 `Expression` 下推。 +5. 基于字符串的键值配置选项仅在 Blink 计划器中使用。(详情参见 [配置]({{ site.baseurl }}/zh/dev/table/config.html) ) +6. `PlannerConfig` 在两种计划器中的实现(`CalciteConfig`)是不同的。 +7. Blink 计划器会将多sink(multiple-sinks)优化成一张有向无环图(DAG)(仅支持 `TableEnvironment`,不支持 `StreamTableEnvironment`)。旧计划器总是将每个sink都优化成一个新的有向无环图,且所有图相互独立。 +8. 旧计划器目前不支持 catalog 统计数据,而 Blink 支持。 -Structure of Table API and SQL Programs +Table API 和 SQL 程序的结构 --------------------------------------- -All Table API and SQL programs for batch and streaming follow the same pattern. The following code example shows the common structure of Table API and SQL programs. +所有用于批处理和流处理的 Table API 和 SQL 程序都遵循相同的模式。下面的代码示例展示了 Table API 和 SQL 程序的通用结构。

    @@ -123,30 +123,30 @@ table_env.execute("python_job")
    -**Note:** Table API and SQL queries can be easily integrated with and embedded into DataStream or DataSet programs. Have a look at the [Integration with DataStream and DataSet API](#integration-with-datastream-and-dataset-api) section to learn how DataStreams and DataSets can be converted into Tables and vice versa. +**注意:** Table API 和 SQL 查询可以很容易地集成并嵌入到 DataStream 或 DataSet 程序中。 请参阅[与 DataStream 和 DataSet API 结合](#integration-with-datastream-and-dataset-api) 章节了解如何将 DataSet 和 DataStream 与表之间的相互转化。 {% top %} -Create a TableEnvironment +创建 TableEnvironment ------------------------- -The `TableEnvironment` is a central concept of the Table API and SQL integration. It is responsible for: +`TableEnvironment` 是 Table API 和 SQL 的核心概念。它负责: -* Registering a `Table` in the internal catalog -* Registering catalogs -* Loading pluggable modules -* Executing SQL queries -* Registering a user-defined (scalar, table, or aggregation) function -* Converting a `DataStream` or `DataSet` into a `Table` -* Holding a reference to an `ExecutionEnvironment` or `StreamExecutionEnvironment` +* 在内部的 catalog 中注册 `Table` +* 注册外部的 catalog +* 加载可插拔模块 +* 执行 SQL 查询 +* 注册自定义函数 (scalar、table 或 aggregation) +* 将 `DataStream` 或 `DataSet` 转换成 `Table` +* 持有对 `ExecutionEnvironment` 或 `StreamExecutionEnvironment` 的引用 -A `Table` is always bound to a specific `TableEnvironment`. It is not possible to combine tables of different TableEnvironments in the same query, e.g., to join or union them. +`Table` 总是与特定的 `TableEnvironment` 绑定。不能在同一条查询中使用不同 TableEnvironment 中的表,例如,对它们进行 join 或 union 操作。 -A `TableEnvironment` is created by calling the static `BatchTableEnvironment.create()` or `StreamTableEnvironment.create()` method with a `StreamExecutionEnvironment` or an `ExecutionEnvironment` and an optional `TableConfig`. The `TableConfig` can be used to configure the `TableEnvironment` or to customize the query optimization and translation process (see [Query Optimization](#query-optimization)). +`TableEnvironment` 可以通过静态方法 `BatchTableEnvironment.create()` 或者 `StreamTableEnvironment.create()` 在 `StreamExecutionEnvironment` 或者 `ExecutionEnvironment` 中创建,`TableConfig` 是可选项。`TableConfig`可用于配置`TableEnvironment`或定制的查询优化和转换过程(参见 [查询优化](#query-optimization))。 -Make sure to choose the specific planner `BatchTableEnvironment`/`StreamTableEnvironment` that matches your programming language. +请确保选择与你的编程语言匹配的特定的计划器`BatchTableEnvironment`/`StreamTableEnvironment`。 -If both planner jars are on the classpath (the default behavior), you should explicitly set which planner to use in the current program. +如果两种计划器的 jar 包都在 classpath 中(默认行为),你应该明确地设置要在当前程序中使用的计划器。
    @@ -288,53 +288,36 @@ b_b_t_env = BatchTableEnvironment.create(environment_settings=b_b_settings)
    -**Note:** If there is only one planner jar in `/lib` directory, you can use `useAnyPlanner` (`use_any_planner` for python) to create specific `EnvironmentSettings`. +**注意:** 如果`/lib`目录中只有一种计划器的 jar 包,则可以使用`useAnyPlanner`(python 使用 `use any_u_planner`)创建 `EnvironmentSettings`。 {% top %} -Create Tables in the Catalog +在 Catalog 中创建表 ------------------------------- -A `TableEnvironment` maintains a map of catalogs of tables which are created with an identifier. Each -identifier consists of 3 parts: catalog name, database name and object name. If a catalog or database is not -specified, the current default value will be used (see examples in the [Table identifier expanding]({{ site.baseurl }}/dev/table/common.html#table-identifier-expanding) section). +`TableEnvironment` 维护着一个由标识符(identifier)创建的表 catalog 的映射。标识符由三个部分组成:catalog 名称、数据库名称以及对象名称。如果 catalog 或者数据库没有指明,就会使用当前默认值(参见[表标识符扩展]({{ site.baseurl }}/zh/dev/table/common.html#table-identifier-expanding)章节中的例子)。 -Tables can be either virtual (`VIEWS`) or regular (`TABLES`). `VIEWS` can be created from an -existing `Table` object, usually the result of a Table API or SQL query. `TABLES` describe -external data, such as a file, database table, or message queue. +`Table` 可以是虚拟的(视图 `VIEWS`)也可以是常规的(表 `TABLES`)。视图 `VIEWS`可以从已经存在的`Table`中创建,一般是 Table API 或者 SQL 的查询结果。 表`TABLES`描述的是外部数据,例如文件、数据库表或者消息队列。 -### Temporary vs Permanent tables. +### 临时表(Temporary Table)和永久表(Permanent Table) -Tables may either be temporary, and tied to the lifecycle of a single Flink session, or permanent, -and visible across multiple Flink sessions and clusters. +表可以是临时的,并与单个 Flink 会话(session)的生命周期相关,也可以是永久的,并且在多个 Flink 会话和群集(cluster)中可见。 -Permanent tables require a [catalog]({{ site.baseurl }}/dev/table/catalogs.html) (such as Hive Metastore) -to maintain metadata about the table. Once a permanent table is created, it is visible to any Flink -session that is connected to the catalog and will continue to exist until the table is explicitly -dropped. +永久表需要 [catalog]({{ site.baseurl }}/zh/dev/table/catalogs.html)(例如 Hive Metastore)以维护表的元数据。一旦永久表被创建,它将对任何连接到 catalog 的 Flink 会话可见且持续存在,直至被明确删除。 -On the other hand, temporary tables are always stored in memory and only exist for the duration of -the Flink session they are created within. These tables are not visible to other sessions. They are -not bound to any catalog or database but can be created in the namespace of one. Temporary tables -are not dropped if their corresponding database is removed. +另一方面,临时表通常保存于内存中并且仅在创建它们的 Flink 会话持续期间存在。这些表对于其它会话是不可见的。它们不与任何 catalog 或者数据库绑定但可以在一个命名空间(namespace)中创建。即使它们对应的数据库被删除,临时表也不会被删除。 -#### Shadowing +#### 屏蔽(Shadowing) -It is possible to register a temporary table with the same identifier as an existing permanent -table. The temporary table shadows the permanent one and makes the permanent table inaccessible as -long as the temporary one exists. All queries with that identifier will be executed against the -temporary table. +可以使用与已存在的永久表相同的标识符去注册临时表。临时表会屏蔽永久表,并且只要临时表存在,永久表就无法访问。所有使用该标识符的查询都将作用于临时表。 -This might be useful for experimentation. It allows running exactly the same query first against a -temporary table that e.g. has just a subset of data, or the data is obfuscated. Once verified that -the query is correct it can be run against the real production table. +这可能对实验(experimentation)有用。它允许先对一个临时表进行完全相同的查询,例如只有一个子集的数据,或者数据是不确定的。一旦验证了查询的正确性,就可以对实际的生产表进行查询。 -### Create a Table +### 创建表 -#### Virtual Tables +#### 虚拟表 -A `Table` API object corresponds to a `VIEW` (virtual table) in a SQL terms. It encapsulates a logical -query plan. It can be created in a catalog as follows: +在 SQL 的术语中,Table API 的对象对应于`视图`(虚拟表)。它封装了一个逻辑查询计划。它可以通过以下方法在 catalog 中创建:
    @@ -377,18 +360,15 @@ table_env.register_table("projectedTable", proj_table)
    -**Note:** `Table` objects are similar to `VIEW`'s from relational database -systems, i.e., the query that defines the `Table` is not optimized but will be inlined when another -query references the registered `Table`. If multiple queries reference the same registered `Table`, -it will be inlined for each referencing query and executed multiple times, i.e., the result of the -registered `Table` will *not* be shared. +**注意:** 从传统数据库系统的角度来看,`Table` 对象与 `VIEW` 视图非常像。也就是,定义了 `Table` 的查询是没有被优化的, +而且会被内嵌到另一个引用了这个注册了的 `Table`的查询中。如果多个查询都引用了同一个注册了的`Table`,那么它会被内嵌每个查询中并被执行多次, +也就是说注册了的`Table`的结果**不会**被共享(注:Blink 计划器的`TableEnvironment`会优化成只执行一次)。 {% top %} #### Connector Tables -It is also possible to create a `TABLE` as known from relational databases from a [connector]({{ site.baseurl }}/dev/table/connect.html) declaration. -The connector describes the external system that stores the data of a table. Storage systems such as Apacha Kafka or a regular file system can be declared here. +另外一个方式去创建 `TABLE` 是通过 [connector]({{ site.baseurl }}/dev/table/connect.html) 声明。Connector 描述了存储表数据的外部系统。存储系统例如 Apache Kafka 或者常规的文件系统都可以通过这种方式来声明。
    @@ -431,17 +411,15 @@ tableEnvironment.sqlUpdate("CREATE [TEMPORARY] TABLE MyTable (...) WITH (...)")
    -### Expanding Table identifiers +### 扩展表标识符 + -Tables are always registered with a 3-part identifier consisting of catalog, database, and table name. +表总是通过三元标识符注册,包括 catalog 名、数据库名和表名。 -Users can set one catalog and one database inside it to be the “current catalog” and “current database”. -With them, the first two parts in the 3-parts identifier mentioned above can be optional - if they are not provided, -the current catalog and current database will be referred. Users can switch the current catalog and current database via -table API or SQL. +用户可以指定一个 catalog 和数据库作为 "当前catalog" 和"当前数据库"。有了这些,那么刚刚提到的三元标识符的前两个部分就可以被省略了。如果前两部分的标识符没有指定, +那么会使用当前的 catalog 和当前数据库。用户也可以通过 Table API 或 SQL 切换当前的 catalog 和当前的数据库。 -Identifiers follow SQL requirements which means that they can be escaped with a backtick character (`` ` ``). -Additionally all SQL reserved keywords must be escaped. +标识符遵循 SQL 标准,因此使用时需要用反引号(`` ` ``)进行转义。此外,所有 SQL 保留关键字都必须转义。
    @@ -508,18 +486,18 @@ tableEnv.createTemporaryView("other_catalog.other_database.exampleView", table)
    -Query a Table +查询表 ------------- ### Table API -The Table API is a language-integrated query API for Scala and Java. In contrast to SQL, queries are not specified as Strings but are composed step-by-step in the host language. +Table API 是关于 Scala 和 Java 的集成语言式查询 API。与 SQL 相反,Table API 的查询不是由字符串指定,而是在宿主语言中逐步构建。 -The API is based on the `Table` class which represents a table (streaming or batch) and offers methods to apply relational operations. These methods return a new `Table` object, which represents the result of applying the relational operation on the input `Table`. Some relational operations are composed of multiple method calls such as `table.groupBy(...).select()`, where `groupBy(...)` specifies a grouping of `table`, and `select(...)` the projection on the grouping of `table`. +Table API 是基于 `Table` 类的,该类表示一个表(流或批处理),并提供使用关系操作的方法。这些方法返回一个新的 Table 对象,该对象表示对输入 Table 进行关系操作的结果。 一些关系操作由多个方法调用组成,例如 `table.groupBy(...).select()`,其中 `groupBy(...)` 指定 `table` 的分组,而 `select(...)` 在 `table` 分组上的投影。 -The [Table API]({{ site.baseurl }}/dev/table/tableApi.html) document describes all Table API operations that are supported on streaming and batch tables. +文档 [Table API]({{ site.baseurl }}/zh/dev/table/tableApi.html) 说明了所有流处理和批处理表支持的 Table API 算子。 -The following example shows a simple Table API aggregation query: +以下示例展示了一个简单的 Table API 聚合查询:
    @@ -589,11 +567,11 @@ revenue = orders \ ### SQL -Flink's SQL integration is based on [Apache Calcite](https://calcite.apache.org), which implements the SQL standard. SQL queries are specified as regular Strings. +Flink SQL 是基于实现了SQL标准的 [Apache Calcite](https://calcite.apache.org) 的。SQL 查询由常规字符串指定。 -The [SQL]({{ site.baseurl }}/dev/table/sql/index.html) document describes Flink's SQL support for streaming and batch tables. +文档 [SQL]({{ site.baseurl }}/zh/dev/table/sql.html) 描述了Flink对流处理和批处理表的SQL支持。 -The following example shows how to specify a query and return the result as a `Table`. +下面的示例演示了如何指定查询并将结果作为 `Table` 对象返回。
    @@ -658,7 +636,7 @@ revenue = table_env.sql_query(
    -The following example shows how to specify an update query that inserts its result into a registered table. +如下的示例展示了如何指定一个更新查询,将查询的结果插入到已注册的表中。
    @@ -728,27 +706,27 @@ table_env.sql_update( {% top %} -### Mixing Table API and SQL +### 混用 Table API 和 SQL -Table API and SQL queries can be easily mixed because both return `Table` objects: +Table API 和 SQL 查询的混用非常简单因为它们都返回 `Table` 对象: -* A Table API query can be defined on the `Table` object returned by a SQL query. -* A SQL query can be defined on the result of a Table API query by [registering the resulting Table](#register-a-table) in the `TableEnvironment` and referencing it in the `FROM` clause of the SQL query. +* 可以在 SQL 查询返回的 `Table` 对象上定义 Table API 查询。 +* 在 `TableEnvironment` 中注册的[结果表](#register-a-table)可以在 SQL 查询的 `FROM` 子句中引用,通过这种方法就可以在 Table API 查询的结果上定义 SQL 查询。 {% top %} -Emit a Table +输出表 ------------ -A `Table` is emitted by writing it to a `TableSink`. A `TableSink` is a generic interface to support a wide variety of file formats (e.g. CSV, Apache Parquet, Apache Avro), storage systems (e.g., JDBC, Apache HBase, Apache Cassandra, Elasticsearch), or messaging systems (e.g., Apache Kafka, RabbitMQ). +`Table` 通过写入 `TableSink` 输出。`TableSink` 是一个通用接口,用于支持多种文件格式(如 CSV、Apache Parquet、Apache Avro)、存储系统(如 JDBC、Apache HBase、Apache Cassandra、Elasticsearch)或消息队列系统(如 Apache Kafka、RabbitMQ)。 -A batch `Table` can only be written to a `BatchTableSink`, while a streaming `Table` requires either an `AppendStreamTableSink`, a `RetractStreamTableSink`, or an `UpsertStreamTableSink`. +批处理 `Table` 只能写入 `BatchTableSink`,而流处理 `Table` 需要指定写入 `AppendStreamTableSink`,`RetractStreamTableSink` 或者 `UpsertStreamTableSink`。 -Please see the documentation about [Table Sources & Sinks]({{ site.baseurl }}/dev/table/sourceSinks.html) for details about available sinks and instructions for how to implement a custom `TableSink`. +请参考文档 [Table Sources & Sinks]({{ site.baseurl }}/zh/dev/table/sourceSinks.html) 以获取更多关于可用 Sink 的信息以及如何自定义 `TableSink`。 -The `Table.insertInto(String tableName)` method emits the `Table` to a registered `TableSink`. The method looks up the `TableSink` from the catalog by the name and validates that the schema of the `Table` is identical to the schema of the `TableSink`. +方法 `Table.insertInto(String tableName)` 将 `Table` 发送至已注册的 `TableSink`。该方法通过名称在 catalog 中查找 `TableSink` 并确认`Table` schema 和 `TableSink` schema 一致。 -The following examples shows how to emit a `Table`: +下面的示例演示如何输出 `Table`:
    @@ -832,70 +810,71 @@ result.insert_into("CsvSinkTable") {% top %} -Translate and Execute a Query +翻译与执行查询 ----------------------------- -The behavior of translating and executing a query is different for the two planners. +两种计划器翻译和执行查询的方式是不同的。
    -Table API and SQL queries are translated into [DataStream]({{ site.baseurl }}/dev/datastream_api.html) or [DataSet]({{ site.baseurl }}/dev/batch) programs depending on whether their input is a streaming or batch input. A query is internally represented as a logical query plan and is translated in two phases: +Table API 和 SQL 查询会被翻译成 [DataStream]({{ site.baseurl }}/zh/dev/datastream_api.html) 或者 [DataSet]({{ site.baseurl }}/zh/dev/batch) 程序, 这取决于它们的输入数据源是流式的还是批式的。查询在内部表示为逻辑查询计划,并被翻译成两个阶段: -1. Optimization of the logical plan -2. Translation into a DataStream or DataSet program +1. 优化逻辑执行计划 +2. 翻译成 DataStream 或 DataSet 程序 -A Table API or SQL query is translated when: +Table API 或者 SQL 查询在下列情况下会被翻译: -* a `Table` is emitted to a `TableSink`, i.e., when `Table.insertInto()` is called. -* a SQL update query is specified, i.e., when `TableEnvironment.sqlUpdate()` is called. -* a `Table` is converted into a `DataStream` or `DataSet` (see [Integration with DataStream and DataSet API](#integration-with-datastream-and-dataset-api)). +* `Table` 被输出给 `TableSink`,即当调用 `Table.insertInto()` 时。 +* SQL 更新语句执行时,即,当调用 `TableEnvironment.sqlUpdate()` 时。 +* `Table` 被转换成 `DataStream` 或者 `DataSet` 时(参阅[与 DataStream 和 DataSet API 结合](#integration-with-datastream-and-dataset-api))。 -Once translated, a Table API or SQL query is handled like a regular DataStream or DataSet program and is executed when `StreamExecutionEnvironment.execute()` or `ExecutionEnvironment.execute()` is called. +翻译完成后,Table API 或者 SQL 查询会被当做普通的 DataStream 或 DataSet 程序对待并且会在调用 `StreamExecutionEnvironment.execute()` 或 `ExecutionEnvironment.execute()` 的时候被执行。
    -Table API and SQL queries are translated into [DataStream]({{ site.baseurl }}/dev/datastream_api.html) programs whether their input is streaming or batch. A query is internally represented as a logical query plan and is translated in two phases: +不论输入数据源是流式的还是批式的,Table API 和 SQL 查询都会被转换成 [DataStream]({{ site.baseurl }}/zh/dev/datastream_api.html) 程序。查询在内部表示为逻辑查询计划,并被翻译成两个阶段: -1. Optimization of the logical plan, -2. Translation into a DataStream program. +1. 优化逻辑执行计划 +2. 翻译成 DataStream 程序 -The behavior of translating a query is different for `TableEnvironment` and `StreamTableEnvironment`. +TableEnvironment 和 StreamTableEnvironment 翻译查询的方式不同。 -For `TableEnvironment`, a Table API or SQL query is translated when `TableEnvironment.execute()` is called, because `TableEnvironment` will optimize multiple-sinks into one DAG. +对于 `TableEnvironment`,Table API 和 SQL 查询会在调用 `TableEnvironment.execute()` 时被翻译,因为 `TableEnvironment` 会将多 sink 优化成一张有向无环图。 -While for `StreamTableEnvironment`, a Table API or SQL query is translated when: +而对于 `StreamTableEnvironment`,当下列情况发生时,Table API 和 SQL 查询才会被翻译: -* a `Table` is emitted to a `TableSink`, i.e., when `Table.insertInto()` is called. -* a SQL update query is specified, i.e., when `TableEnvironment.sqlUpdate()` is called. -* a `Table` is converted into a `DataStream`. +* `Table 被发送至`TableSink`,即,当 `Table.insertInto()` 被调用时。 +* SQL 更新语句执行时,即,当调用 `TableEnvironment.sqlUpdate()` 时。 +* `Table` 被转换成 `DataStream` 时。 -Once translated, a Table API or SQL query is handled like a regular DataStream program and is executed when `TableEnvironment.execute()` or `StreamExecutionEnvironment.execute()` is called. +翻译完成后,Table API 或者 SQL 查询会被当做普通的 DataStream 程序对待并且会在调用 `TableEnvironment.execute()` 或者 `StreamExecutionEnvironment.execute()` 的时候被执行。
    {% top %} -Integration with DataStream and DataSet API +与 DataStream 和 DataSet API 结合 ------------------------------------------- -Both planners on stream can integrate with the `DataStream` API. Only old planner can integrate with the `DataSet API`, Blink planner on batch could not be combined with both. -**Note:** The `DataSet` API discussed below is only relevant for the old planner on batch. +在流处理方面两种计划器都可以与 `DataStream` API 结合。只有旧计划器可以与 `DataSet API` 结合。在批处理方面,Blink 计划器不能同两种计划器中的任何一个结合。 -Table API and SQL queries can be easily integrated with and embedded into [DataStream]({{ site.baseurl }}/dev/datastream_api.html) and [DataSet]({{ site.baseurl }}/dev/batch) programs. For instance, it is possible to query an external table (for example from a RDBMS), do some pre-processing, such as filtering, projecting, aggregating, or joining with meta data, and then further process the data with either the DataStream or DataSet API (and any of the libraries built on top of these APIs, such as CEP or Gelly). Inversely, a Table API or SQL query can also be applied on the result of a DataStream or DataSet program. +**注意:** 下文讨论的 `DataSet` API 只与旧计划起有关。 -This interaction can be achieved by converting a `DataStream` or `DataSet` into a `Table` and vice versa. In this section, we describe how these conversions are done. +Table API 和 SQL 可以被很容易地集成并嵌入到 [DataStream]({{ site.baseurl }}/zh/dev/datastream_api.html) 和 [DataSet]({{ site.baseurl }}/zh/dev/batch) 程序中。例如,可以查询外部表(例如从 RDBMS),进行一些预处理,例如过滤,投影,聚合或与元数据 join,然后使用 DataStream 或 DataSet API(以及在这些 API 之上构建的任何库,例如 CEP 或 Gelly)。相反,也可以将 Table API 或 SQL 查询应用于 DataStream 或 DataSet 程序的结果。 -### Implicit Conversion for Scala +这种交互可以通过 `DataStream` 或 `DataSet` 与 `Table` 的相互转化实现。本节我们会介绍这些转化是如何实现的。 -The Scala Table API features implicit conversions for the `DataSet`, `DataStream`, and `Table` classes. These conversions are enabled by importing the package `org.apache.flink.table.api.scala._` in addition to `org.apache.flink.api.scala._` for the Scala DataStream API. +### Scala 隐式转换 -### Create a View from a DataStream or DataSet +Scala Table API 含有对 `DataSet`、`DataStream` 和 `Table` 类的隐式转换。 通过为 Scala DataStream API 导入 `org.apache.flink.table.api.scala._` 包以及 `org.apache.flink.api.scala._` 包,可以启用这些转换。 -A `DataStream` or `DataSet` can be registered in a `TableEnvironment` as a View. The schema of the resulting view depends on the data type of the registered `DataStream` or `DataSet`. Please check the section about [mapping of data types to table schema](#mapping-of-data-types-to-table-schema) for details. +### 通过 DataSet 或 DataStream 创建`视图` -**Note:** Views created from a `DataStream` or `DataSet` can be registered as temporary views only. +在 `TableEnvironment` 中可以将 `DataStream` 或 `DataSet` 注册成视图。结果视图的 schema 取决于注册的 `DataStream` 或 `DataSet` 的数据类型。请参阅文档 [数据类型到 table schema 的映射](#mapping-of-data-types-to-table-schema)获取详细信息。 + +**注意:** 通过 `DataStream` 或 `DataSet` 创建的视图只能注册成临时视图。
    @@ -933,9 +912,9 @@ tableEnv.createTemporaryView("myTable2", stream, 'myLong, 'myString) {% top %} -### Convert a DataStream or DataSet into a Table +### 将 DataStream 或 DataSet 转换成表 -Instead of registering a `DataStream` or `DataSet` in a `TableEnvironment`, it can also be directly converted into a `Table`. This is convenient if you want to use the Table in a Table API query. +与在 `TableEnvironment` 注册 `DataStream` 或 `DataSet` 不同,DataStream 和 DataSet 还可以直接转换成 `Table`。如果你想在 Table API 的查询中使用表,这将非常便捷。
    @@ -973,26 +952,26 @@ val table2: Table = tableEnv.fromDataStream(stream, 'myLong, 'myString) {% top %} -### Convert a Table into a DataStream or DataSet +### 将表转换成 DataStream 或 DataSet -A `Table` can be converted into a `DataStream` or `DataSet`. In this way, custom DataStream or DataSet programs can be run on the result of a Table API or SQL query. +`Table` 可以被转换成 `DataStream` 或 `DataSet`。通过这种方式,定制的 DataSet 或 DataStream 程序就可以在 Table API 或者 SQL 的查询结果上运行了。 -When converting a `Table` into a `DataStream` or `DataSet`, you need to specify the data type of the resulting `DataStream` or `DataSet`, i.e., the data type into which the rows of the `Table` are to be converted. Often the most convenient conversion type is `Row`. The following list gives an overview of the features of the different options: +将 `Table` 转换为 `DataStream` 或者 `DataSet` 时,你需要指定生成的 `DataStream` 或者 `DataSet` 的数据类型,即,`Table` 的每行数据要转换成的数据类型。通常最方便的选择是转换成 `Row` 。以下列表概述了不同选项的功能: -- **Row**: fields are mapped by position, arbitrary number of fields, support for `null` values, no type-safe access. -- **POJO**: fields are mapped by name (POJO fields must be named as `Table` fields), arbitrary number of fields, support for `null` values, type-safe access. -- **Case Class**: fields are mapped by position, no support for `null` values, type-safe access. -- **Tuple**: fields are mapped by position, limitation to 22 (Scala) or 25 (Java) fields, no support for `null` values, type-safe access. -- **Atomic Type**: `Table` must have a single field, no support for `null` values, type-safe access. +- **Row**: 字段按位置映射,字段数量任意,支持 `null` 值,无类型安全(type-safe)检查。 +- **POJO**: 字段按名称映射(POJO 必须按`Table` 中字段名称命名),字段数量任意,支持 `null` 值,无类型安全检查。 +- **Case Class**: 字段按位置映射,不支持 `null` 值,有类型安全检查。 +- **Tuple**: 字段按位置映射,字段数量少于 22(Scala)或者 25(Java),不支持 `null` 值,无类型安全检查。 +- **Atomic Type**: `Table` 必须有一个字段,不支持 `null` 值,有类型安全检查。 -#### Convert a Table into a DataStream +#### 将表转换成 DataStream -A `Table` that is the result of a streaming query will be updated dynamically, i.e., it is changing as new records arrive on the query's input streams. Hence, the `DataStream` into which such a dynamic query is converted needs to encode the updates of the table. +流式查询(streaming query)的结果表会动态更新,即,当新纪录到达查询的输入流时,查询结果会改变。因此,像这样将动态查询结果转换成 DataStream 需要对表的更新方式进行编码。 -There are two modes to convert a `Table` into a `DataStream`: +将 `Table` 转换为 `DataStream` 有两种模式: -1. **Append Mode**: This mode can only be used if the dynamic `Table` is only modified by `INSERT` changes, i.e, it is append-only and previously emitted results are never updated. -2. **Retract Mode**: This mode can always be used. It encodes `INSERT` and `DELETE` changes with a `boolean` flag. +1. **Append Mode**: 仅当动态 `Table` 仅通过`INSERT`更改进行修改时,才可以使用此模式,即,它仅是追加操作,并且之前输出的结果永远不会更新。 +2. **Retract Mode**: 任何情形都可以使用此模式。它使用 boolean 值对 `INSERT` 和 `DELETE` 操作的数据进行标记。
    @@ -1049,11 +1028,11 @@ val retractStream: DataStream[(Boolean, Row)] = tableEnv.toRetractStream[Row](ta
    -**Note:** A detailed discussion about dynamic tables and their properties is given in the [Dynamic Tables](streaming/dynamic_tables.html) document. +**注意:** 文档[动态表](streaming/dynamic_tables.html)给出了有关动态表及其属性的详细讨论。 -#### Convert a Table into a DataSet +#### 将表转换成 DataSet -A `Table` is converted into a `DataSet` as follows: +将 `Table` 转换成 `DataSet` 的过程如下:
    @@ -1096,17 +1075,17 @@ val dsTuple: DataSet[(String, Int)] = tableEnv.toDataSet[(String, Int)](table) {% top %} -### Mapping of Data Types to Table Schema +### 数据类型到 Table Schema 的映射 -Flink's DataStream and DataSet APIs support very diverse types. Composite types such as Tuples (built-in Scala and Flink Java tuples), POJOs, Scala case classes, and Flink's Row type allow for nested data structures with multiple fields that can be accessed in table expressions. Other types are treated as atomic types. In the following, we describe how the Table API converts these types into an internal row representation and show examples of converting a `DataStream` into a `Table`. +Flink 的 DataStream 和 DataSet APIs 支持多样的数据类型。例如 Tuple(Scala 内置以及Flink Java tuple)、POJO 类型、Scala case class 类型以及 Flink 的 Row 类型等允许嵌套且有多个可在表的表达式中访问的字段的复合数据类型。其他类型被视为原子类型。下面,我们讨论 Table API 如何将这些数据类型类型转换为内部 row 表示形式,并提供将 `DataStream` 转换成 `Table` 的样例。 -The mapping of a data type to a table schema can happen in two ways: **based on the field positions** or **based on the field names**. +数据类型到 table schema 的映射有两种方式:**基于字段位置**或**基于字段名称**。 -**Position-based Mapping** +**基于位置映射** -Position-based mapping can be used to give fields a more meaningful name while keeping the field order. This mapping is available for composite data types *with a defined field order* as well as atomic types. Composite data types such as tuples, rows, and case classes have such a field order. However, fields of a POJO must be mapped based on the field names (see next section). Fields can be projected out but can't be renamed using an alias `as`. +基于位置的映射可在保持字段顺序的同时为字段提供更有意义的名称。这种映射方式可用于*具有特定的字段顺序*的复合数据类型以及原子类型。如 tuple、row 以及 case class 这些复合数据类型都有这样的字段顺序。然而,POJO 类型的字段则必须通过名称映射(参见下一章)。可以将字段投影出来,但不能使用`as`重命名。 -When defining a position-based mapping, the specified names must not exist in the input data type, otherwise the API will assume that the mapping should happen based on the field names. If no field names are specified, the default field names and field order of the composite type are used or `f0` for atomic types. +定义基于位置的映射时,输入数据类型中一定不能存在指定的名称,否则 API 会假定应该基于字段名称进行映射。如果未指定任何字段名称,则使用默认的字段名称和复合数据类型的字段顺序,或者使用 `f0` 表示原子类型。
    @@ -1146,11 +1125,11 @@ val table: Table = tableEnv.fromDataStream(stream, 'myLong, 'myInt)
    -**Name-based Mapping** +**基于名称的映射** -Name-based mapping can be used for any data type including POJOs. It is the most flexible way of defining a table schema mapping. All fields in the mapping are referenced by name and can be possibly renamed using an alias `as`. Fields can be reordered and projected out. +基于名称的映射适用于任何数据类型包括 POJO 类型。这是定义 table schema 映射最灵活的方式。映射中的所有字段均按名称引用,并且可以通过 `as` 重命名。字段可以被重新排序和映射。 -If no field names are specified, the default field names and field order of the composite type are used or `f0` for atomic types. +若果没有指定任何字段名称,则使用默认的字段名称和复合数据类型的字段顺序,或者使用 `f0` 表示原子类型。
    @@ -1196,9 +1175,9 @@ val table: Table = tableEnv.fromDataStream(stream, '_2 as 'myInt, '_1 as 'myLong
    -#### Atomic Types +#### 原子类型 -Flink treats primitives (`Integer`, `Double`, `String`) or generic types (types that cannot be analyzed and decomposed) as atomic types. A `DataStream` or `DataSet` of an atomic type is converted into a `Table` with a single attribute. The type of the attribute is inferred from the atomic type and the name of the attribute can be specified. +Flink 将基础数据类型(`Integer`、`Double`、`String`)或者通用数据类型(不可再拆分的数据类型)视为原子类型。原子类型的 `DataStream` 或者 `DataSet` 会被转换成只有一条属性的 `Table`。属性的数据类型可以由原子类型推断出,还可以重新命名属性。
    @@ -1232,9 +1211,9 @@ val table: Table = tableEnv.fromDataStream(stream, 'myLong)
    -#### Tuples (Scala and Java) and Case Classes (Scala only) +#### Tuple类型(Scala 和 Java)和 Case Class类型(仅 Scala) -Flink supports Scala's built-in tuples and provides its own tuple classes for Java. DataStreams and DataSets of both kinds of tuples can be converted into tables. Fields can be renamed by providing names for all fields (mapping based on position). If no field names are specified, the default field names are used. If the original field names (`f0`, `f1`, ... for Flink Tuples and `_1`, `_2`, ... for Scala Tuples) are referenced, the API assumes that the mapping is name-based instead of position-based. Name-based mapping allows for reordering fields and projection with alias (`as`). +Flink 支持 Scala 的内置 tuple 类型并给 Java 提供自己的 tuple 类型。两种 tuple 的 DataStream 和 DataSet 都能被转换成表。可以通过提供所有字段名称来重命名字段(基于位置映射)。如果没有指明任何字段名称,则会使用默认的字段名称。如果引用了原始字段名称(对于 Flink tuple 为`f0`、`f1` ... ...,对于 Scala tuple 为`_1`、`_2` ... ...),则 API 会假定映射是基于名称的而不是基于位置的。基于名称的映射可以通过 `as` 对字段和投影进行重新排序。
    @@ -1300,11 +1279,11 @@ val table: Table = tableEnv.fromDataStream(stream, 'age as 'myAge, 'name as 'myN
    -#### POJO (Java and Scala) +#### POJO 类型 (Java 和 Scala) -Flink supports POJOs as composite types. The rules for what determines a POJO are documented [here]({{ site.baseurl }}/dev/api_concepts.html#pojos). +Flink 支持 POJO 类型作为复合类型。确定 POJO 类型的规则记录在[这里]({{ site.baseurl }}/zh/dev/api_concepts.html#pojos). -When converting a POJO `DataStream` or `DataSet` into a `Table` without specifying field names, the names of the original POJO fields are used. The name mapping requires the original names and cannot be done by positions. Fields can be renamed using an alias (with the `as` keyword), reordered, and projected. +在不指定字段名称的情况下将 POJO 类型的 `DataStream` 或 `DataSet` 转换成 `Table` 时,将使用原始 POJO 类型字段的名称。名称映射需要原始名称,并且不能按位置进行。字段可以使用别名(带有 `as` 关键字)来重命名,重新排序和投影。
    @@ -1352,9 +1331,9 @@ val table: Table = tableEnv.fromDataStream(stream, 'name as 'myName)
    -#### Row +#### Row类型 -The `Row` data type supports an arbitrary number of fields and fields with `null` values. Field names can be specified via a `RowTypeInfo` or when converting a `Row` `DataStream` or `DataSet` into a `Table`. The row type supports mapping of fields by position and by name. Fields can be renamed by providing names for all fields (mapping based on position) or selected individually for projection/ordering/renaming (mapping based on name). +`Row` 类型支持任意数量的字段以及具有 `null` 值的字段。字段名称可以通过 `RowTypeInfo` 指定,也可以在将 `Row` 的 `DataStream` 或 `DataSet` 转换为 `Table` 时指定。Row 类型的字段映射支持基于名称和基于位置两种方式。字段可以通过提供所有字段的名称的方式重命名(基于位置映射)或者分别选择进行投影/排序/重命名(基于名称映射)。
    @@ -1411,54 +1390,54 @@ val table: Table = tableEnv.fromDataStream(stream, 'name as 'myName) {% top %} -Query Optimization +查询优化 ------------------
    -Apache Flink leverages Apache Calcite to optimize and translate queries. The optimization currently performed include projection and filter push-down, subquery decorrelation, and other kinds of query rewriting. Old planner does not yet optimize the order of joins, but executes them in the same order as defined in the query (order of Tables in the `FROM` clause and/or order of join predicates in the `WHERE` clause). +Apache Flink 利用 Apache Calcite 来优化和翻译查询。当前执行的优化包括投影和过滤器下推,子查询消除以及其他类型的查询重写。原版计划程序尚未优化 join 的顺序,而是按照查询中定义的顺序执行它们(FROM 子句中的表顺序和/或 WHERE 子句中的 join 谓词顺序)。 -It is possible to tweak the set of optimization rules which are applied in different phases by providing a `CalciteConfig` object. This can be created via a builder by calling `CalciteConfig.createBuilder())` and is provided to the TableEnvironment by calling `tableEnv.getConfig.setPlannerConfig(calciteConfig)`. +通过提供一个 `CalciteConfig` 对象,可以调整在不同阶段应用的优化规则集合。这个对象可以通过调用构造器 `CalciteConfig.createBuilder()` 创建,并通过调用 `tableEnv.getConfig.setPlannerConfig(calciteConfig)` 提供给 TableEnvironment。
    -Apache Flink leverages and extends Apache Calcite to perform sophisticated query optimization. -This includes a series of rule and cost-based optimizations such as: +Apache Flink 使用并扩展了 Apache Calcite 来执行复杂的查询优化。 +这包括一系列基于规则和成本的优化,例如: -* Subquery decorrelation based on Apache Calcite -* Project pruning -* Partition pruning -* Filter push-down -* Sub-plan deduplication to avoid duplicate computation -* Special subquery rewriting, including two parts: - * Converts IN and EXISTS into left semi-joins - * Converts NOT IN and NOT EXISTS into left anti-join -* Optional join reordering - * Enabled via `table.optimizer.join-reorder-enabled` +* 基于 Apache Calcite 的子查询解相关 +* 投影剪裁 +* 分区剪裁 +* 过滤器下推 +* 子计划消除重复数据以避免重复计算 +* 特殊子查询重写,包括两部分: + * 将 IN 和 EXISTS 转换为 left semi-joins + * 将 NOT IN 和 NOT EXISTS 转换为 left anti-join +* 可选 join 重新排序 + * 通过 `table.optimizer.join-reorder-enabled` 启用 -**Note:** IN/EXISTS/NOT IN/NOT EXISTS are currently only supported in conjunctive conditions in subquery rewriting. +**注意:** 当前仅在子查询重写的结合条件下支持 IN / EXISTS / NOT IN / NOT EXISTS。 -The optimizer makes intelligent decisions, based not only on the plan but also rich statistics available from the data sources and fine-grain costs for each operator such as io, cpu, network, and memory. +优化器不仅基于计划,而且还基于可从数据源获得的丰富统计信息以及每个算子(例如 io,cpu,网络和内存)的细粒度成本来做出明智的决策。 -Advanced users may provide custom optimizations via a `CalciteConfig` object that can be provided to the table environment by calling `TableEnvironment#getConfig#setPlannerConfig`. +高级用户可以通过 `CalciteConfig` 对象提供自定义优化,可以通过调用 `TableEnvironment#getConfig#setPlannerConfig` 将其提供给 TableEnvironment。
    -### Explaining a Table +### 解释表 -The Table API provides a mechanism to explain the logical and optimized query plans to compute a `Table`. -This is done through the `TableEnvironment.explain(table)` method or `TableEnvironment.explain()` method. `explain(table)` returns the plan of a given `Table`. `explain()` returns the result of a multiple sinks plan and is mainly used for the Blink planner. It returns a String describing three plans: +Table API 提供了一种机制来解释计算 `Table` 的逻辑和优化查询计划。 +这是通过 `TableEnvironment.explain(table)` 或者 `TableEnvironment.explain()` 完成的。`explain(table)` 返回给定 `Table` 的计划。 `explain()` 返回多 sink 计划的结果并且主要用于 Blink 计划器。它返回一个描述三种计划的字符串: -1. the Abstract Syntax Tree of the relational query, i.e., the unoptimized logical query plan, -2. the optimized logical query plan, and -3. the physical execution plan. +1. 关系查询的抽象语法树(the Abstract Syntax Tree),即未优化的逻辑查询计划, +2. 优化的逻辑查询计划,以及 +3. 物理执行计划。 -The following code shows an example and the corresponding output for given `Table` using `explain(table)`: +以下代码展示了一个示例以及对给定 `Table` 使用 `explain(table)` 的相应输出:
    @@ -1636,7 +1615,7 @@ Stage 4 : Data Source
    -The following code shows an example and the corresponding output for multiple-sinks plan using `explain()`: +以下代码展示了一个示例以及使用 `explain()` 的多 sink 计划的相应输出:
    @@ -1754,7 +1733,7 @@ print(explanation)
    -the result of multiple-sinks plan is +多 sink 计划的结果是:
    {% highlight text %} -- Gitee From dfd79bce78a16af9b5a534f1e9539c530979ba58 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Mon, 2 Mar 2020 14:19:08 +0100 Subject: [PATCH 025/885] [FLINK-16370][build] Only bundle javax as Java11-exclusive dependency --- flink-dist/pom.xml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-dist/pom.xml b/flink-dist/pom.xml index d5447e3916..a10b01d53e 100644 --- a/flink-dist/pom.xml +++ b/flink-dist/pom.xml @@ -635,7 +635,7 @@ under the License. true - ${project.build.directory}/temporary + ${project.build.directory}/temporary/java11_exclusive @@ -665,16 +665,16 @@ under the License. maven-antrun-plugin - unpack-javax-libraries + bundle-java11-exclusive-dependencies process-resources run - + - + -- Gitee From 4a509fc02b223b2f81b39f055213b42e7265e4cb Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Tue, 3 Mar 2020 19:39:32 +0800 Subject: [PATCH 026/885] [FLINK-16359][table-runtime] Introduce WritableVectors for abstract writing (#11275) In FLINK-11899 , we need write vectors from parquet input streams. We need abstract vector writing, in future, we can provide OffHeapVectors. --- .../vector/AbstractOrcNoHiveVector.java | 5 - .../orc/vector/AbstractOrcColumnVector.java | 5 - .../vectors/ArrowBigIntColumnVector.java | 5 - .../arrow/vectors/ArrowIntColumnVector.java | 5 - .../vectors/ArrowSmallIntColumnVector.java | 5 - .../vectors/ArrowTinyIntColumnVector.java | 5 - .../dataformat/vector/BytesColumnVector.java | 5 +- .../table/dataformat/vector/ColumnVector.java | 6 - .../table/dataformat/vector/Dictionary.java | 4 + .../vector/VectorizedColumnBatch.java | 10 - .../vector/heap/AbstractHeapVector.java | 43 +- .../vector/heap/HeapBooleanVector.java | 16 +- .../vector/heap/HeapByteVector.java | 16 +- .../vector/heap/HeapBytesVector.java | 41 +- .../vector/heap/HeapDoubleVector.java | 37 +- .../vector/heap/HeapFloatVector.java | 37 +- .../dataformat/vector/heap/HeapIntVector.java | 47 +- .../vector/heap/HeapLongVector.java | 35 +- .../vector/heap/HeapShortVector.java | 16 +- .../vector/heap/HeapTimestampVector.java | 64 +++ .../AbstractWritableVector.java} | 20 +- .../writable/WritableBooleanVector.java | 37 ++ .../vector/writable/WritableByteVector.java | 37 ++ .../vector/writable/WritableBytesVector.java | 38 ++ .../vector/writable/WritableColumnVector.java | 71 +++ .../vector/writable/WritableDoubleVector.java | 47 ++ .../vector/writable/WritableFloatVector.java | 47 ++ .../vector/writable/WritableIntVector.java | 57 +++ .../vector/writable/WritableLongVector.java | 47 ++ .../vector/writable/WritableShortVector.java | 37 ++ .../writable/WritableTimestampVector.java | 38 ++ .../dataformat/vector/ColumnVectorTest.java | 406 ++++++++++++++++++ .../vector/VectorizedColumnBatchTest.java | 60 +-- 33 files changed, 1191 insertions(+), 158 deletions(-) create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapTimestampVector.java rename flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/{AbstractColumnVector.java => writable/AbstractWritableVector.java} (69%) create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableBooleanVector.java create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableByteVector.java create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableBytesVector.java create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableColumnVector.java create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableDoubleVector.java create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableFloatVector.java create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableIntVector.java create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableLongVector.java create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableShortVector.java create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableTimestampVector.java create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/vector/ColumnVectorTest.java diff --git a/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/vector/AbstractOrcNoHiveVector.java b/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/vector/AbstractOrcNoHiveVector.java index 62fd12b7d3..f55ef5cd2c 100644 --- a/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/vector/AbstractOrcNoHiveVector.java +++ b/flink-formats/flink-orc-nohive/src/main/java/org/apache/flink/orc/nohive/vector/AbstractOrcNoHiveVector.java @@ -55,11 +55,6 @@ public abstract class AbstractOrcNoHiveVector implements return !orcVector.noNulls && orcVector.isNull[orcVector.isRepeating ? 0 : i]; } - @Override - public void reset() { - throw new UnsupportedOperationException(); - } - public static org.apache.flink.table.dataformat.vector.ColumnVector createFlinkVector( ColumnVector vector) { if (vector instanceof LongColumnVector) { diff --git a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/AbstractOrcColumnVector.java b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/AbstractOrcColumnVector.java index 7319187921..c9c64e2058 100644 --- a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/AbstractOrcColumnVector.java +++ b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/AbstractOrcColumnVector.java @@ -55,11 +55,6 @@ public abstract class AbstractOrcColumnVector implements return !vector.noNulls && vector.isNull[vector.isRepeating ? 0 : i]; } - @Override - public void reset() { - throw new UnsupportedOperationException(); - } - public static org.apache.flink.table.dataformat.vector.ColumnVector createFlinkVector( ColumnVector vector) { if (vector instanceof LongColumnVector) { diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowBigIntColumnVector.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowBigIntColumnVector.java index 3282717ba3..eab20b3890 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowBigIntColumnVector.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowBigIntColumnVector.java @@ -48,9 +48,4 @@ public final class ArrowBigIntColumnVector implements LongColumnVector { public boolean isNullAt(int i) { return bigIntVector.isNull(i); } - - @Override - public void reset() { - bigIntVector.reset(); - } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowIntColumnVector.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowIntColumnVector.java index 5673d9f1ab..6fbdd42b35 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowIntColumnVector.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowIntColumnVector.java @@ -45,9 +45,4 @@ public final class ArrowIntColumnVector implements IntColumnVector { public boolean isNullAt(int i) { return intVector.isNull(i); } - - @Override - public void reset() { - intVector.reset(); - } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowSmallIntColumnVector.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowSmallIntColumnVector.java index 83bcdc119a..991f6659a8 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowSmallIntColumnVector.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowSmallIntColumnVector.java @@ -45,9 +45,4 @@ public final class ArrowSmallIntColumnVector implements ShortColumnVector { public boolean isNullAt(int i) { return smallIntVector.isNull(i); } - - @Override - public void reset() { - smallIntVector.reset(); - } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowTinyIntColumnVector.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowTinyIntColumnVector.java index 11225968ca..f0363c4cd3 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowTinyIntColumnVector.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowTinyIntColumnVector.java @@ -45,9 +45,4 @@ public final class ArrowTinyIntColumnVector implements ByteColumnVector { public boolean isNullAt(int i) { return tinyIntVector.isNull(i); } - - @Override - public void reset() { - tinyIntVector.reset(); - } } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/BytesColumnVector.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/BytesColumnVector.java index b09020be77..2c63c26728 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/BytesColumnVector.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/BytesColumnVector.java @@ -28,7 +28,7 @@ public interface BytesColumnVector extends ColumnVector { /** * Bytes data. */ - class Bytes{ + class Bytes { public final byte[] data; public final int offset; public final int len; @@ -40,6 +40,9 @@ public interface BytesColumnVector extends ColumnVector { } public byte[] getBytes() { + if (offset == 0 && len == data.length) { + return data; + } byte[] res = new byte[len]; System.arraycopy(data, offset, res, 0, len); return res; diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/ColumnVector.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/ColumnVector.java index ae372a202e..a2963d626a 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/ColumnVector.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/ColumnVector.java @@ -22,11 +22,5 @@ package org.apache.flink.table.dataformat.vector; * Nullable column vector. Access data through specific subclasses. */ public interface ColumnVector { - boolean isNullAt(int i); - - /** - * Resets the column to default state. - */ - void reset(); } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/Dictionary.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/Dictionary.java index 0f7d2545fc..fb91e074d4 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/Dictionary.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/Dictionary.java @@ -17,6 +17,8 @@ package org.apache.flink.table.dataformat.vector; +import org.apache.flink.table.dataformat.SqlTimestamp; + /** * The interface for dictionary in AbstractColumnVector to decode dictionary encoded values. */ @@ -31,4 +33,6 @@ public interface Dictionary { double decodeToDouble(int id); byte[] decodeToBinary(int id); + + SqlTimestamp decodeToTimestamp(int id); } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/VectorizedColumnBatch.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/VectorizedColumnBatch.java index 7807ce38cc..a7f0341f37 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/VectorizedColumnBatch.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/VectorizedColumnBatch.java @@ -47,16 +47,6 @@ public class VectorizedColumnBatch implements Serializable { this.columns = vectors; } - /** - * Resets the batch for writing. - */ - public void reset() { - for (ColumnVector column : columns) { - column.reset(); - } - this.numRows = 0; - } - public void setNumRows(int numRows) { this.numRows = numRows; } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/AbstractHeapVector.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/AbstractHeapVector.java index af864dfa79..84c5182bc2 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/AbstractHeapVector.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/AbstractHeapVector.java @@ -18,14 +18,25 @@ package org.apache.flink.table.dataformat.vector.heap; -import org.apache.flink.table.dataformat.vector.AbstractColumnVector; +import org.apache.flink.core.memory.MemoryUtils; +import org.apache.flink.table.dataformat.vector.writable.AbstractWritableVector; +import java.nio.ByteOrder; import java.util.Arrays; /** * Heap vector that nullable shared structure. */ -public abstract class AbstractHeapVector extends AbstractColumnVector { +public abstract class AbstractHeapVector extends AbstractWritableVector { + + public static final boolean LITTLE_ENDIAN = ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN; + + public static final sun.misc.Unsafe UNSAFE = MemoryUtils.UNSAFE; + public static final int BYTE_ARRAY_OFFSET = UNSAFE.arrayBaseOffset(byte[].class); + public static final int INT_ARRAY_OFFSET = UNSAFE.arrayBaseOffset(int[].class); + public static final int LONG_ARRAY_OFFSET = UNSAFE.arrayBaseOffset(long[].class); + public static final int FLOAT_ARRAY_OFFSET = UNSAFE.arrayBaseOffset(float[].class); + public static final int DOUBLE_ARRAY_OFFSET = UNSAFE.arrayBaseOffset(double[].class); /* * If hasNulls is true, then this array contains true if the value @@ -56,11 +67,28 @@ public abstract class AbstractHeapVector extends AbstractColumnVector { noNulls = true; } + @Override public void setNullAt(int i) { isNull[i] = true; noNulls = false; } + @Override + public void setNulls(int i, int count) { + for (int j = 0; j < count; j++) { + isNull[i + j] = true; + } + if (count > 0) { + noNulls = false; + } + } + + @Override + public void fillWithNulls() { + this.noNulls = false; + Arrays.fill(isNull, true); + } + @Override public boolean isNullAt(int i) { return !noNulls && isNull[i]; @@ -71,7 +99,15 @@ public abstract class AbstractHeapVector extends AbstractColumnVector { if (dictionaryIds == null) { dictionaryIds = new HeapIntVector(capacity); } else { - dictionaryIds.reset(); + if (capacity > dictionaryIds.vector.length) { + int current = dictionaryIds.vector.length; + while (current < capacity) { + current <<= 1; + } + dictionaryIds = new HeapIntVector(current); + } else { + dictionaryIds.reset(); + } } return dictionaryIds; } @@ -79,6 +115,7 @@ public abstract class AbstractHeapVector extends AbstractColumnVector { /** * Returns the underlying integer column for ids of dictionary. */ + @Override public HeapIntVector getDictionaryIds() { return dictionaryIds; } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapBooleanVector.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapBooleanVector.java index 52c8f0d144..531010925b 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapBooleanVector.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapBooleanVector.java @@ -18,12 +18,14 @@ package org.apache.flink.table.dataformat.vector.heap; -import org.apache.flink.table.dataformat.vector.BooleanColumnVector; +import org.apache.flink.table.dataformat.vector.writable.WritableBooleanVector; + +import java.util.Arrays; /** * This class represents a nullable heap boolean column vector. */ -public class HeapBooleanVector extends AbstractHeapVector implements BooleanColumnVector { +public class HeapBooleanVector extends AbstractHeapVector implements WritableBooleanVector { private static final long serialVersionUID = 4131239076731313596L; @@ -48,4 +50,14 @@ public class HeapBooleanVector extends AbstractHeapVector implements BooleanColu public boolean getBoolean(int i) { return vector[i]; } + + @Override + public void setBoolean(int i, boolean value) { + vector[i] = value; + } + + @Override + public void fill(boolean value) { + Arrays.fill(vector, value); + } } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapByteVector.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapByteVector.java index e0845548cc..fbd78d1300 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapByteVector.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapByteVector.java @@ -18,12 +18,14 @@ package org.apache.flink.table.dataformat.vector.heap; -import org.apache.flink.table.dataformat.vector.ByteColumnVector; +import org.apache.flink.table.dataformat.vector.writable.WritableByteVector; + +import java.util.Arrays; /** * This class represents a nullable byte column vector. */ -public class HeapByteVector extends AbstractHeapVector implements ByteColumnVector { +public class HeapByteVector extends AbstractHeapVector implements WritableByteVector { private static final long serialVersionUID = 7216045902943789034L; @@ -47,4 +49,14 @@ public class HeapByteVector extends AbstractHeapVector implements ByteColumnVect return (byte) dictionary.decodeToInt(dictionaryIds.vector[i]); } } + + @Override + public void setByte(int i, byte value) { + vector[i] = value; + } + + @Override + public void fill(byte value) { + Arrays.fill(vector, value); + } } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapBytesVector.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapBytesVector.java index 7d3a992286..f5a77980d4 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapBytesVector.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapBytesVector.java @@ -18,7 +18,9 @@ package org.apache.flink.table.dataformat.vector.heap; -import org.apache.flink.table.dataformat.vector.BytesColumnVector; +import org.apache.flink.table.dataformat.vector.writable.WritableBytesVector; + +import java.util.Arrays; /** * This class supports string and binary data by value reference -- i.e. each field is @@ -35,7 +37,7 @@ import org.apache.flink.table.dataformat.vector.BytesColumnVector; * You can mix "by value" and "by reference" in the same column vector, * though that use is probably not typical. */ -public class HeapBytesVector extends AbstractHeapVector implements BytesColumnVector { +public class HeapBytesVector extends AbstractHeapVector implements WritableBytesVector { private static final long serialVersionUID = -8529155738773478597L; @@ -79,18 +81,8 @@ public class HeapBytesVector extends AbstractHeapVector implements BytesColumnVe elementsAppended = 0; } - /** - * Set a field by actually copying in to a local buffer. - * If you must actually copy data in to the array, use this method. - * DO NOT USE this method unless it's not practical to set data by reference with setRef(). - * Setting data by reference tends to run a lot faster than copying data in. - * - * @param elementNum index within column vector to set - * @param sourceBuf container of source data - * @param start start byte position within source - * @param length length of source byte sequence - */ - public void setVal(int elementNum, byte[] sourceBuf, int start, int length) { + @Override + public void appendBytes(int elementNum, byte[] sourceBuf, int start, int length) { reserve(elementsAppended + length); System.arraycopy(sourceBuf, start, buffer, elementsAppended, length); this.start[elementNum] = elementsAppended; @@ -98,17 +90,16 @@ public class HeapBytesVector extends AbstractHeapVector implements BytesColumnVe elementsAppended += length; } - /** - * Set a field by actually copying in to a local buffer. - * If you must actually copy data in to the array, use this method. - * DO NOT USE this method unless it's not practical to set data by reference with setRef(). - * Setting data by reference tends to run a lot faster than copying data in. - * - * @param elementNum index within column vector to set - * @param sourceBuf container of source data - */ - public void setVal(int elementNum, byte[] sourceBuf) { - setVal(elementNum, sourceBuf, 0, sourceBuf.length); + @Override + public void fill(byte[] value) { + reserve(start.length * value.length); + for (int i = 0; i < start.length; i++) { + System.arraycopy(value, 0, buffer, i * value.length, value.length); + } + for (int i = 0; i < start.length; i++) { + this.start[i] = i * value.length; + } + Arrays.fill(this.length, value.length); } private void reserve(int requiredCapacity) { diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapDoubleVector.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapDoubleVector.java index 9e83bf31ee..e422b77d0c 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapDoubleVector.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapDoubleVector.java @@ -18,14 +18,18 @@ package org.apache.flink.table.dataformat.vector.heap; -import org.apache.flink.table.dataformat.vector.DoubleColumnVector; +import org.apache.flink.table.dataformat.vector.writable.WritableDoubleVector; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Arrays; /** * This class represents a nullable double precision floating point column vector. * This class will be used for operations on all floating point double types * and as such will use a 64-bit double value to hold the biggest possible value. */ -public class HeapDoubleVector extends AbstractHeapVector implements DoubleColumnVector { +public class HeapDoubleVector extends AbstractHeapVector implements WritableDoubleVector { private static final long serialVersionUID = 6193940154117411328L; @@ -49,4 +53,33 @@ public class HeapDoubleVector extends AbstractHeapVector implements DoubleColumn return dictionary.decodeToDouble(dictionaryIds.vector[i]); } } + + @Override + public void setDouble(int i, double value) { + vector[i] = value; + } + + @Override + public void setDoublesFromBinary(int rowId, int count, byte[] src, int srcIndex) { + if (rowId + count > vector.length || srcIndex + count * 8L > src.length) { + throw new IndexOutOfBoundsException(String.format( + "Index out of bounds, row id is %s, count is %s, binary src index is %s, binary" + + " length is %s, double array src index is %s, double array length is %s.", + rowId, count, srcIndex, src.length, rowId, vector.length)); + } + if (LITTLE_ENDIAN) { + UNSAFE.copyMemory(src, BYTE_ARRAY_OFFSET + srcIndex, vector, + DOUBLE_ARRAY_OFFSET + rowId * 8L, count * 8L); + } else { + ByteBuffer bb = ByteBuffer.wrap(src).order(ByteOrder.BIG_ENDIAN); + for (int i = 0; i < count; ++i) { + vector[i + rowId] = bb.getDouble(srcIndex + (8 * i)); + } + } + } + + @Override + public void fill(double value) { + Arrays.fill(vector, value); + } } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapFloatVector.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapFloatVector.java index 116f59a67f..27aa231a1d 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapFloatVector.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapFloatVector.java @@ -18,13 +18,17 @@ package org.apache.flink.table.dataformat.vector.heap; -import org.apache.flink.table.dataformat.vector.FloatColumnVector; +import org.apache.flink.table.dataformat.vector.writable.WritableFloatVector; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Arrays; /** * This class represents a nullable double precision floating point column vector. * This class will be used for operations on all floating point float types. */ -public class HeapFloatVector extends AbstractHeapVector implements FloatColumnVector { +public class HeapFloatVector extends AbstractHeapVector implements WritableFloatVector { private static final long serialVersionUID = 8928878923550041110L; @@ -48,4 +52,33 @@ public class HeapFloatVector extends AbstractHeapVector implements FloatColumnVe return dictionary.decodeToFloat(dictionaryIds.vector[i]); } } + + @Override + public void setFloat(int i, float value) { + vector[i] = value; + } + + @Override + public void setFloatsFromBinary(int rowId, int count, byte[] src, int srcIndex) { + if (rowId + count > vector.length || srcIndex + count * 4L > src.length) { + throw new IndexOutOfBoundsException(String.format( + "Index out of bounds, row id is %s, count is %s, binary src index is %s, binary" + + " length is %s, float array src index is %s, float array length is %s.", + rowId, count, srcIndex, src.length, rowId, vector.length)); + } + if (LITTLE_ENDIAN) { + UNSAFE.copyMemory(src, BYTE_ARRAY_OFFSET + srcIndex, vector, + FLOAT_ARRAY_OFFSET + rowId * 4L, count * 4L); + } else { + ByteBuffer bb = ByteBuffer.wrap(src).order(ByteOrder.BIG_ENDIAN); + for (int i = 0; i < count; ++i) { + vector[i + rowId] = bb.getFloat(srcIndex + (4 * i)); + } + } + } + + @Override + public void fill(float value) { + Arrays.fill(vector, value); + } } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapIntVector.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapIntVector.java index dfb0aeb9fd..ced5203d94 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapIntVector.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapIntVector.java @@ -18,12 +18,14 @@ package org.apache.flink.table.dataformat.vector.heap; -import org.apache.flink.table.dataformat.vector.IntColumnVector; +import org.apache.flink.table.dataformat.vector.writable.WritableIntVector; + +import java.util.Arrays; /** * This class represents a nullable int column vector. */ -public class HeapIntVector extends AbstractHeapVector implements IntColumnVector { +public class HeapIntVector extends AbstractHeapVector implements WritableIntVector { private static final long serialVersionUID = -2749499358889718254L; @@ -47,4 +49,45 @@ public class HeapIntVector extends AbstractHeapVector implements IntColumnVector return dictionary.decodeToInt(dictionaryIds.vector[i]); } } + + @Override + public void setInt(int i, int value) { + vector[i] = value; + } + + @Override + public void setIntsFromBinary(int rowId, int count, byte[] src, int srcIndex) { + if (rowId + count > vector.length || srcIndex + count * 4L > src.length) { + throw new IndexOutOfBoundsException(String.format( + "Index out of bounds, row id is %s, count is %s, binary src index is %s, binary" + + " length is %s, int array src index is %s, int array length is %s.", + rowId, count, srcIndex, src.length, rowId, vector.length)); + } + if (LITTLE_ENDIAN) { + UNSAFE.copyMemory(src, BYTE_ARRAY_OFFSET + srcIndex, vector, + INT_ARRAY_OFFSET + rowId * 4L, count * 4L); + } else { + long srcOffset = srcIndex + BYTE_ARRAY_OFFSET; + for (int i = 0; i < count; ++i, srcOffset += 4) { + vector[i + rowId] = Integer.reverseBytes(UNSAFE.getInt(src, srcOffset)); + } + } + } + + @Override + public void setInts(int rowId, int count, int value) { + for (int i = 0; i < count; ++i) { + vector[i + rowId] = value; + } + } + + @Override + public void setInts(int rowId, int count, int[] src, int srcIndex) { + System.arraycopy(src, srcIndex, vector, rowId, count); + } + + @Override + public void fill(int value) { + Arrays.fill(vector, value); + } } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapLongVector.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapLongVector.java index 479a592429..cf401726ac 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapLongVector.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapLongVector.java @@ -18,12 +18,14 @@ package org.apache.flink.table.dataformat.vector.heap; -import org.apache.flink.table.dataformat.vector.LongColumnVector; +import org.apache.flink.table.dataformat.vector.writable.WritableLongVector; + +import java.util.Arrays; /** * This class represents a nullable long column vector. */ -public class HeapLongVector extends AbstractHeapVector implements LongColumnVector { +public class HeapLongVector extends AbstractHeapVector implements WritableLongVector { private static final long serialVersionUID = 8534925169458006397L; @@ -47,4 +49,33 @@ public class HeapLongVector extends AbstractHeapVector implements LongColumnVect return dictionary.decodeToLong(dictionaryIds.vector[i]); } } + + @Override + public void setLong(int i, long value) { + vector[i] = value; + } + + @Override + public void setLongsFromBinary(int rowId, int count, byte[] src, int srcIndex) { + if (rowId + count > vector.length || srcIndex + count * 8L > src.length) { + throw new IndexOutOfBoundsException(String.format( + "Index out of bounds, row id is %s, count is %s, binary src index is %s, binary" + + " length is %s, long array src index is %s, long array length is %s.", + rowId, count, srcIndex, src.length, rowId, vector.length)); + } + if (LITTLE_ENDIAN) { + UNSAFE.copyMemory(src, BYTE_ARRAY_OFFSET + srcIndex, vector, + LONG_ARRAY_OFFSET + rowId * 8L, count * 8L); + } else { + long srcOffset = srcIndex + BYTE_ARRAY_OFFSET; + for (int i = 0; i < count; ++i, srcOffset += 8) { + vector[i + rowId] = Long.reverseBytes(UNSAFE.getLong(src, srcOffset)); + } + } + } + + @Override + public void fill(long value) { + Arrays.fill(vector, value); + } } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapShortVector.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapShortVector.java index c4dbf834e3..ee2d2a1c47 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapShortVector.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapShortVector.java @@ -18,12 +18,14 @@ package org.apache.flink.table.dataformat.vector.heap; -import org.apache.flink.table.dataformat.vector.ShortColumnVector; +import org.apache.flink.table.dataformat.vector.writable.WritableShortVector; + +import java.util.Arrays; /** * This class represents a nullable short column vector. */ -public class HeapShortVector extends AbstractHeapVector implements ShortColumnVector { +public class HeapShortVector extends AbstractHeapVector implements WritableShortVector { private static final long serialVersionUID = -8278486456144676292L; @@ -47,4 +49,14 @@ public class HeapShortVector extends AbstractHeapVector implements ShortColumnVe return (short) dictionary.decodeToInt(dictionaryIds.vector[i]); } } + + @Override + public void setShort(int i, short value) { + vector[i] = value; + } + + @Override + public void fill(short value) { + Arrays.fill(vector, value); + } } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapTimestampVector.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapTimestampVector.java new file mode 100644 index 0000000000..3fa9947641 --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/heap/HeapTimestampVector.java @@ -0,0 +1,64 @@ +/* + * 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.table.dataformat.vector.heap; + +import org.apache.flink.table.dataformat.SqlTimestamp; +import org.apache.flink.table.dataformat.vector.writable.WritableTimestampVector; + +import java.util.Arrays; + +import static org.apache.flink.table.dataformat.SqlTimestamp.fromEpochMillis; + +/** + * This class represents a nullable byte column vector. + */ +public class HeapTimestampVector extends AbstractHeapVector implements WritableTimestampVector { + + private static final long serialVersionUID = 1L; + + private final long[] milliseconds; + private final int[] nanoOfMilliseconds; + + public HeapTimestampVector(int len) { + super(len); + this.milliseconds = new long[len]; + this.nanoOfMilliseconds = new int[len]; + } + + @Override + public SqlTimestamp getTimestamp(int i, int precision) { + if (dictionary == null) { + return fromEpochMillis(milliseconds[i], nanoOfMilliseconds[i]); + } else { + return dictionary.decodeToTimestamp(dictionaryIds.vector[i]); + } + } + + @Override + public void setTimestamp(int i, SqlTimestamp timestamp) { + milliseconds[i] = timestamp.getMillisecond(); + nanoOfMilliseconds[i] = timestamp.getNanoOfMillisecond(); + } + + @Override + public void fill(SqlTimestamp value) { + Arrays.fill(milliseconds, value.getMillisecond()); + Arrays.fill(nanoOfMilliseconds, value.getNanoOfMillisecond()); + } +} diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/AbstractColumnVector.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/AbstractWritableVector.java similarity index 69% rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/AbstractColumnVector.java rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/AbstractWritableVector.java index 10928afb31..20f33513f9 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/AbstractColumnVector.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/AbstractWritableVector.java @@ -16,7 +16,10 @@ * limitations under the License. */ -package org.apache.flink.table.dataformat.vector; +package org.apache.flink.table.dataformat.vector.writable; + +import org.apache.flink.table.dataformat.vector.ColumnVector; +import org.apache.flink.table.dataformat.vector.Dictionary; import java.io.Serializable; @@ -24,9 +27,9 @@ import java.io.Serializable; * Contains the shared structure for {@link ColumnVector}s, including NULL information and dictionary. * NOTE: if there are some nulls, must set {@link #noNulls} to false. */ -public abstract class AbstractColumnVector implements ColumnVector, Serializable { +public abstract class AbstractWritableVector implements WritableColumnVector, Serializable { - private static final long serialVersionUID = 5340018531388047747L; + private static final long serialVersionUID = 1L; // If the whole column vector has no nulls, this is true, otherwise false. protected boolean noNulls = true; @@ -40,22 +43,15 @@ public abstract class AbstractColumnVector implements ColumnVector, Serializable /** * Update the dictionary. */ + @Override public void setDictionary(Dictionary dictionary) { this.dictionary = dictionary; } - /** - * Reserve a integer column for ids of dictionary. - * DictionaryIds maybe inconsistent with {@link #setDictionary}. Suppose a ColumnVector's data - * comes from two pages. Perhaps one page uses a dictionary and the other page does not use a - * dictionary. The first page that uses a field will have dictionaryIds, which requires - * decoding the first page (Out batch does not support a mix of dictionary). - */ - public abstract IntColumnVector reserveDictionaryIds(int capacity); - /** * Returns true if this column has a dictionary. */ + @Override public boolean hasDictionary() { return this.dictionary != null; } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableBooleanVector.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableBooleanVector.java new file mode 100644 index 0000000000..38aeea795f --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableBooleanVector.java @@ -0,0 +1,37 @@ +/* + * 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.table.dataformat.vector.writable; + +import org.apache.flink.table.dataformat.vector.BooleanColumnVector; + +/** + * Writable {@link BooleanColumnVector}. + */ +public interface WritableBooleanVector extends WritableColumnVector, BooleanColumnVector { + + /** + * Set boolean at rowId with the provided value. + */ + void setBoolean(int rowId, boolean value); + + /** + * Fill the column vector with the provided value. + */ + void fill(boolean value); +} diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableByteVector.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableByteVector.java new file mode 100644 index 0000000000..97ddb17967 --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableByteVector.java @@ -0,0 +1,37 @@ +/* + * 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.table.dataformat.vector.writable; + +import org.apache.flink.table.dataformat.vector.ByteColumnVector; + +/** + * Writable {@link ByteColumnVector}. + */ +public interface WritableByteVector extends WritableColumnVector, ByteColumnVector { + + /** + * Set byte at rowId with the provided value. + */ + void setByte(int rowId, byte value); + + /** + * Fill the column vector with the provided value. + */ + void fill(byte value); +} diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableBytesVector.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableBytesVector.java new file mode 100644 index 0000000000..b9ce7ee699 --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableBytesVector.java @@ -0,0 +1,38 @@ +/* + * 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.table.dataformat.vector.writable; + +import org.apache.flink.table.dataformat.vector.BytesColumnVector; + +/** + * Writable {@link BytesColumnVector}. + */ +public interface WritableBytesVector extends WritableColumnVector, BytesColumnVector { + + /** + * Append byte[] at rowId with the provided value. + * Note: Must append values according to the order of rowId, can not random append. + */ + void appendBytes(int rowId, byte[] value, int offset, int length); + + /** + * Fill the column vector with the provided value. + */ + void fill(byte[] value); +} diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableColumnVector.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableColumnVector.java new file mode 100644 index 0000000000..b366204a8d --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableColumnVector.java @@ -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.flink.table.dataformat.vector.writable; + +import org.apache.flink.table.dataformat.vector.ColumnVector; +import org.apache.flink.table.dataformat.vector.Dictionary; + +/** + * Writable {@link ColumnVector}. + */ +public interface WritableColumnVector extends ColumnVector { + + /** + * Resets the column to default state. + */ + void reset(); + + /** + * Set null at rowId. + */ + void setNullAt(int rowId); + + /** + * Set nulls from rowId to rowId + count (exclude). + */ + void setNulls(int rowId, int count); + + /** + * Fill the column vector with nulls. + */ + void fillWithNulls(); + + /** + * Set the dictionary, it should work with dictionary ids. + */ + void setDictionary(Dictionary dictionary); + + /** + * Check if there's a dictionary. + */ + boolean hasDictionary(); + + /** + * Reserve a integer column for ids of dictionary. The size of return {@link WritableIntVector} + * should be equal to or bigger than capacity. + * DictionaryIds must inconsistent with {@link #setDictionary}. We don't support a mix of + * dictionary. + */ + WritableIntVector reserveDictionaryIds(int capacity); + + /** + * Get reserved dictionary ids. + */ + WritableIntVector getDictionaryIds(); +} diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableDoubleVector.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableDoubleVector.java new file mode 100644 index 0000000000..55aec3fee9 --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableDoubleVector.java @@ -0,0 +1,47 @@ +/* + * 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.table.dataformat.vector.writable; + +import org.apache.flink.table.dataformat.vector.DoubleColumnVector; + +/** + * Writable {@link DoubleColumnVector}. + */ +public interface WritableDoubleVector extends WritableColumnVector, DoubleColumnVector { + + /** + * Set double at rowId with the provided value. + */ + void setDouble(int rowId, double value); + + /** + * Set doubles from binary, need use UNSAFE to copy. + * + * @param rowId set start rowId. + * @param count count for double, so the bytes size is count * 8. + * @param src source binary. + * @param srcIndex source binary index, it is the index for byte index. + */ + void setDoublesFromBinary(int rowId, int count, byte[] src, int srcIndex); + + /** + * Fill the column vector with the provided value. + */ + void fill(double value); +} diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableFloatVector.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableFloatVector.java new file mode 100644 index 0000000000..4e74d420d5 --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableFloatVector.java @@ -0,0 +1,47 @@ +/* + * 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.table.dataformat.vector.writable; + +import org.apache.flink.table.dataformat.vector.FloatColumnVector; + +/** + * Writable {@link FloatColumnVector}. + */ +public interface WritableFloatVector extends WritableColumnVector, FloatColumnVector { + + /** + * Set float at rowId with the provided value. + */ + void setFloat(int rowId, float value); + + /** + * Set floats from binary, need use UNSAFE to copy. + * + * @param rowId set start rowId. + * @param count count for float, so the bytes size is count * 4. + * @param src source binary. + * @param srcIndex source binary index, it is the index for byte index. + */ + void setFloatsFromBinary(int rowId, int count, byte[] src, int srcIndex); + + /** + * Fill the column vector with the provided value. + */ + void fill(float value); +} diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableIntVector.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableIntVector.java new file mode 100644 index 0000000000..3232ad4ae3 --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableIntVector.java @@ -0,0 +1,57 @@ +/* + * 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.table.dataformat.vector.writable; + +import org.apache.flink.table.dataformat.vector.IntColumnVector; + +/** + * Writable {@link IntColumnVector}. + */ +public interface WritableIntVector extends WritableColumnVector, IntColumnVector { + + /** + * Set int at rowId with the provided value. + */ + void setInt(int rowId, int value); + + /** + * Set ints from binary, need use UNSAFE to copy. + * + * @param rowId set start rowId. + * @param count count for int, so the bytes size is count * 4. + * @param src source binary. + * @param srcIndex source binary index, it is the index for byte index. + */ + void setIntsFromBinary(int rowId, int count, byte[] src, int srcIndex); + + /** + * Sets value to [rowId, rowId + count) by the value, this is data that repeats continuously. + */ + void setInts(int rowId, int count, int value); + + /** + * Sets values from [src[srcIndex], src[srcIndex + count]) to [rowId, rowId + count). + */ + void setInts(int rowId, int count, int[] src, int srcIndex); + + /** + * Fill the column vector with the provided value. + */ + void fill(int value); +} diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableLongVector.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableLongVector.java new file mode 100644 index 0000000000..e29d9c2a4b --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableLongVector.java @@ -0,0 +1,47 @@ +/* + * 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.table.dataformat.vector.writable; + +import org.apache.flink.table.dataformat.vector.LongColumnVector; + +/** + * Writable {@link LongColumnVector}. + */ +public interface WritableLongVector extends WritableColumnVector, LongColumnVector { + + /** + * Set long at rowId with the provided value. + */ + void setLong(int rowId, long value); + + /** + * Set longs from binary, need use UNSAFE to copy. + * + * @param rowId set start rowId. + * @param count count for long, so the bytes size is count * 8. + * @param src source binary. + * @param srcIndex source binary index, it is the index for byte index. + */ + void setLongsFromBinary(int rowId, int count, byte[] src, int srcIndex); + + /** + * Fill the column vector with the provided value. + */ + void fill(long value); +} diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableShortVector.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableShortVector.java new file mode 100644 index 0000000000..2560e16ee8 --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableShortVector.java @@ -0,0 +1,37 @@ +/* + * 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.table.dataformat.vector.writable; + +import org.apache.flink.table.dataformat.vector.ShortColumnVector; + +/** + * Writable {@link ShortColumnVector}. + */ +public interface WritableShortVector extends WritableColumnVector, ShortColumnVector { + + /** + * Set short at rowId with the provided value. + */ + void setShort(int rowId, short value); + + /** + * Fill the column vector with the provided value. + */ + void fill(short value); +} diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableTimestampVector.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableTimestampVector.java new file mode 100644 index 0000000000..1b03a5640c --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/writable/WritableTimestampVector.java @@ -0,0 +1,38 @@ +/* + * 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.table.dataformat.vector.writable; + +import org.apache.flink.table.dataformat.SqlTimestamp; +import org.apache.flink.table.dataformat.vector.TimestampColumnVector; + +/** + * Writable {@link TimestampColumnVector}. + */ +public interface WritableTimestampVector extends WritableColumnVector, TimestampColumnVector { + + /** + * Set {@link SqlTimestamp} at rowId with the provided value. + */ + void setTimestamp(int rowId, SqlTimestamp timestamp); + + /** + * Fill the column vector with the provided value. + */ + void fill(SqlTimestamp value); +} diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/vector/ColumnVectorTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/vector/ColumnVectorTest.java new file mode 100644 index 0000000000..094653a0a9 --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/vector/ColumnVectorTest.java @@ -0,0 +1,406 @@ +/* + * 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.table.dataformat.vector; + +import org.apache.flink.table.dataformat.SqlTimestamp; +import org.apache.flink.table.dataformat.vector.heap.HeapBooleanVector; +import org.apache.flink.table.dataformat.vector.heap.HeapByteVector; +import org.apache.flink.table.dataformat.vector.heap.HeapBytesVector; +import org.apache.flink.table.dataformat.vector.heap.HeapDoubleVector; +import org.apache.flink.table.dataformat.vector.heap.HeapFloatVector; +import org.apache.flink.table.dataformat.vector.heap.HeapIntVector; +import org.apache.flink.table.dataformat.vector.heap.HeapLongVector; +import org.apache.flink.table.dataformat.vector.heap.HeapShortVector; +import org.apache.flink.table.dataformat.vector.heap.HeapTimestampVector; +import org.apache.flink.table.dataformat.vector.writable.WritableColumnVector; + +import org.junit.Test; + +import java.nio.charset.StandardCharsets; +import java.util.stream.IntStream; +import java.util.stream.LongStream; + +import static org.apache.flink.table.dataformat.vector.heap.AbstractHeapVector.BYTE_ARRAY_OFFSET; +import static org.apache.flink.table.dataformat.vector.heap.AbstractHeapVector.DOUBLE_ARRAY_OFFSET; +import static org.apache.flink.table.dataformat.vector.heap.AbstractHeapVector.FLOAT_ARRAY_OFFSET; +import static org.apache.flink.table.dataformat.vector.heap.AbstractHeapVector.INT_ARRAY_OFFSET; +import static org.apache.flink.table.dataformat.vector.heap.AbstractHeapVector.LONG_ARRAY_OFFSET; +import static org.apache.flink.table.dataformat.vector.heap.AbstractHeapVector.UNSAFE; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +/** + * Test {@link ColumnVector}. + */ +public class ColumnVectorTest { + + private static final int SIZE = 10; + + @Test + public void testNulls() { + HeapBooleanVector vector = new HeapBooleanVector(SIZE); + + for (int i = 0; i < SIZE; i++) { + if (i % 2 == 0) { + vector.setNullAt(i); + } + } + for (int i = 0; i < SIZE; i++) { + if (i % 2 == 0) { + assertTrue(vector.isNullAt(i)); + } else { + assertFalse(vector.isNullAt(i)); + } + } + + vector.fillWithNulls(); + for (int i = 0; i < SIZE; i++) { + assertTrue(vector.isNullAt(i)); + } + + vector.reset(); + for (int i = 0; i < SIZE; i++) { + assertFalse(vector.isNullAt(i)); + } + + vector.setNulls(0, SIZE / 2); + for (int i = 0; i < SIZE / 2; i++) { + assertTrue(vector.isNullAt(i)); + } + } + + @Test + public void testBoolean() { + HeapBooleanVector vector = new HeapBooleanVector(SIZE); + + for (int i = 0; i < SIZE; i++) { + vector.setBoolean(i, i % 2 == 0); + } + for (int i = 0; i < SIZE; i++) { + assertEquals(i % 2 == 0, vector.getBoolean(i)); + } + + vector.fill(true); + for (int i = 0; i < SIZE; i++) { + assertTrue(vector.getBoolean(i)); + } + } + + @Test + public void testByte() { + HeapByteVector vector = new HeapByteVector(SIZE); + + for (int i = 0; i < SIZE; i++) { + vector.setByte(i, (byte) i); + } + for (int i = 0; i < SIZE; i++) { + assertEquals(i, vector.getByte(i)); + } + + vector.fill((byte) 22); + for (int i = 0; i < SIZE; i++) { + assertEquals(22, vector.getByte(i)); + } + + vector.setDictionary(new TestDictionary(IntStream.range(0, SIZE).boxed().toArray())); + setRangeDictIds(vector); + for (int i = 0; i < SIZE; i++) { + assertEquals(i, vector.getByte(i)); + } + } + + @Test + public void testShort() { + HeapShortVector vector = new HeapShortVector(SIZE); + + for (int i = 0; i < SIZE; i++) { + vector.setShort(i, (short) i); + } + for (int i = 0; i < SIZE; i++) { + assertEquals(i, vector.getShort(i)); + } + + vector.fill((short) 22); + for (int i = 0; i < SIZE; i++) { + assertEquals(22, vector.getShort(i)); + } + + vector.setDictionary(new TestDictionary(IntStream.range(0, SIZE).boxed().toArray())); + setRangeDictIds(vector); + for (int i = 0; i < SIZE; i++) { + assertEquals(i, vector.getShort(i)); + } + } + + @Test + public void testInt() { + HeapIntVector vector = new HeapIntVector(SIZE); + + for (int i = 0; i < SIZE; i++) { + vector.setInt(i, i); + } + for (int i = 0; i < SIZE; i++) { + assertEquals(i, vector.getInt(i)); + } + + vector.fill(22); + for (int i = 0; i < SIZE; i++) { + assertEquals(22, vector.getInt(i)); + } + + vector = new HeapIntVector(SIZE); + vector.setInts(0, SIZE, 22); + for (int i = 0; i < SIZE; i++) { + assertEquals(22, vector.getInt(i)); + } + + vector.setDictionary(new TestDictionary(IntStream.range(0, SIZE).boxed().toArray())); + setRangeDictIds(vector); + for (int i = 0; i < SIZE; i++) { + assertEquals(i, vector.getInt(i)); + } + + int[] ints = IntStream.range(0, SIZE).toArray(); + byte[] binary = new byte[SIZE * 8]; + UNSAFE.copyMemory(ints, INT_ARRAY_OFFSET, binary, BYTE_ARRAY_OFFSET, binary.length); + vector = new HeapIntVector(SIZE); + vector.setIntsFromBinary(0, SIZE, binary, 0); + for (int i = 0; i < SIZE; i++) { + assertEquals(i, vector.getInt(i)); + } + } + + @Test + public void testLong() { + HeapLongVector vector = new HeapLongVector(SIZE); + + for (int i = 0; i < SIZE; i++) { + vector.setLong(i, i); + } + for (int i = 0; i < SIZE; i++) { + assertEquals(i, vector.getLong(i)); + } + + vector.fill(22); + for (int i = 0; i < SIZE; i++) { + assertEquals(22, vector.getLong(i)); + } + + vector.setDictionary(new TestDictionary(LongStream.range(0, SIZE).boxed().toArray())); + setRangeDictIds(vector); + for (int i = 0; i < SIZE; i++) { + assertEquals(i, vector.getLong(i)); + } + + long[] longs = LongStream.range(0, SIZE).toArray(); + byte[] binary = new byte[SIZE * 8]; + UNSAFE.copyMemory(longs, LONG_ARRAY_OFFSET, binary, BYTE_ARRAY_OFFSET, binary.length); + vector = new HeapLongVector(SIZE); + vector.setLongsFromBinary(0, SIZE, binary, 0); + for (int i = 0; i < SIZE; i++) { + assertEquals(i, vector.getLong(i)); + } + } + + @Test + public void testFloat() { + HeapFloatVector vector = new HeapFloatVector(SIZE); + + for (int i = 0; i < SIZE; i++) { + vector.setFloat(i, i); + } + for (int i = 0; i < SIZE; i++) { + assertEquals(i, vector.getFloat(i), 0); + } + + vector.fill(22); + for (int i = 0; i < SIZE; i++) { + assertEquals(22, vector.getFloat(i), 0); + } + + vector.setDictionary(new TestDictionary(LongStream.range(0, SIZE).boxed() + .map(Number::floatValue).toArray())); + setRangeDictIds(vector); + for (int i = 0; i < SIZE; i++) { + assertEquals(i, vector.getFloat(i), 0); + } + + float[] floats = new float[SIZE]; + for (int i = 0; i < SIZE; i++) { + floats[i] = i; + } + byte[] binary = new byte[SIZE * 4]; + UNSAFE.copyMemory(floats, FLOAT_ARRAY_OFFSET, binary, BYTE_ARRAY_OFFSET, binary.length); + vector = new HeapFloatVector(SIZE); + vector.setFloatsFromBinary(0, SIZE, binary, 0); + for (int i = 0; i < SIZE; i++) { + assertEquals(i, vector.getFloat(i), 0); + } + } + + @Test + public void testDouble() { + HeapDoubleVector vector = new HeapDoubleVector(SIZE); + + for (int i = 0; i < SIZE; i++) { + vector.setDouble(i, i); + } + for (int i = 0; i < SIZE; i++) { + assertEquals(i, vector.getDouble(i), 0); + } + + vector.fill(22); + for (int i = 0; i < SIZE; i++) { + assertEquals(22, vector.getDouble(i), 0); + } + + vector.setDictionary(new TestDictionary(LongStream.range(0, SIZE).boxed() + .map(Number::doubleValue).toArray())); + setRangeDictIds(vector); + for (int i = 0; i < SIZE; i++) { + assertEquals(i, vector.getDouble(i), 0); + } + + double[] doubles = LongStream.range(0, SIZE).boxed().mapToDouble(Number::doubleValue).toArray(); + byte[] binary = new byte[SIZE * 8]; + UNSAFE.copyMemory(doubles, DOUBLE_ARRAY_OFFSET, binary, BYTE_ARRAY_OFFSET, binary.length); + vector = new HeapDoubleVector(SIZE); + vector.setDoublesFromBinary(0, SIZE, binary, 0); + for (int i = 0; i < SIZE; i++) { + assertEquals(i, vector.getDouble(i), 0); + } + } + + private byte[] produceBytes(int i) { + return (i + "").getBytes(StandardCharsets.UTF_8); + } + + @Test + public void testBytes() { + HeapBytesVector vector = new HeapBytesVector(SIZE); + + for (int i = 0; i < SIZE; i++) { + byte[] bytes = produceBytes(i); + vector.appendBytes(i, bytes, 0, bytes.length); + } + for (int i = 0; i < SIZE; i++) { + assertArrayEquals(produceBytes(i), vector.getBytes(i).getBytes()); + } + vector.reset(); + for (int i = 0; i < SIZE; i++) { + byte[] bytes = produceBytes(i); + vector.appendBytes(i, bytes, 0, bytes.length); + } + for (int i = 0; i < SIZE; i++) { + assertArrayEquals(produceBytes(i), vector.getBytes(i).getBytes()); + } + + vector.fill(produceBytes(22)); + for (int i = 0; i < SIZE; i++) { + assertArrayEquals(produceBytes(22), vector.getBytes(i).getBytes()); + } + + vector.setDictionary(new TestDictionary(IntStream.range(0, SIZE) + .mapToObj(this::produceBytes).toArray())); + setRangeDictIds(vector); + for (int i = 0; i < SIZE; i++) { + assertArrayEquals(produceBytes(i), vector.getBytes(i).getBytes()); + } + } + + @Test + public void testTimestamp() { + HeapTimestampVector vector = new HeapTimestampVector(SIZE); + + for (int i = 0; i < SIZE; i++) { + vector.setTimestamp(i, SqlTimestamp.fromEpochMillis(i, i)); + } + for (int i = 0; i < SIZE; i++) { + assertEquals(SqlTimestamp.fromEpochMillis(i, i), vector.getTimestamp(i, 9)); + } + + vector.fill(SqlTimestamp.fromEpochMillis(22, 22)); + for (int i = 0; i < SIZE; i++) { + assertEquals(SqlTimestamp.fromEpochMillis(22, 22), vector.getTimestamp(i, 9)); + } + + vector.setDictionary(new TestDictionary(IntStream.range(0, SIZE) + .mapToObj(i -> SqlTimestamp.fromEpochMillis(i, i)).toArray())); + setRangeDictIds(vector); + for (int i = 0; i < SIZE; i++) { + assertEquals(SqlTimestamp.fromEpochMillis(i, i), vector.getTimestamp(i, 9)); + } + } + + @Test + public void testReserveDictIds() { + HeapIntVector vector = new HeapIntVector(SIZE); + assertTrue(vector.reserveDictionaryIds(2).vector.length >= 2); + assertTrue(vector.reserveDictionaryIds(5).vector.length >= 5); + assertTrue(vector.reserveDictionaryIds(2).vector.length >= 2); + } + + private void setRangeDictIds(WritableColumnVector vector) { + vector.reserveDictionaryIds(SIZE) + .setInts(0, SIZE, IntStream.range(0, SIZE).toArray(), 0); + } + + /** + * Test Dictionary. Just return Object value. + */ + static final class TestDictionary implements Dictionary { + private Object[] intDictionary; + + TestDictionary(Object[] dictionary) { + this.intDictionary = dictionary; + } + + @Override + public int decodeToInt(int id) { + return (int) intDictionary[id]; + } + + @Override + public long decodeToLong(int id) { + return (long) intDictionary[id]; + } + + @Override + public float decodeToFloat(int id) { + return (float) intDictionary[id]; + } + + @Override + public double decodeToDouble(int id) { + return (double) intDictionary[id]; + } + + @Override + public byte[] decodeToBinary(int id) { + return (byte[]) intDictionary[id]; + } + + @Override + public SqlTimestamp decodeToTimestamp(int id) { + return (SqlTimestamp) intDictionary[id]; + } + } +} diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/vector/VectorizedColumnBatchTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/vector/VectorizedColumnBatchTest.java index d93d51ef8a..d88a958301 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/vector/VectorizedColumnBatchTest.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/vector/VectorizedColumnBatchTest.java @@ -57,7 +57,7 @@ public class VectorizedColumnBatchTest { HeapBytesVector col1 = new HeapBytesVector(VECTOR_SIZE); for (int i = 0; i < VECTOR_SIZE; i++) { byte[] bytes = String.valueOf(i).getBytes(StandardCharsets.UTF_8); - col1.setVal(i, bytes, 0, bytes.length); + col1.appendBytes(i, bytes, 0, bytes.length); } HeapByteVector col2 = new HeapByteVector(VECTOR_SIZE); @@ -106,11 +106,6 @@ public class VectorizedColumnBatchTest { return false; } - @Override - public void reset() { - - } - @Override public SqlTimestamp getTimestamp(int i, int precision) { return SqlTimestamp.fromEpochMillis(vector8[i]); @@ -133,11 +128,6 @@ public class VectorizedColumnBatchTest { public boolean isNullAt(int i) { return false; } - - @Override - public void reset() { - - } }; HeapBytesVector vector10 = new HeapBytesVector(VECTOR_SIZE); @@ -191,11 +181,6 @@ public class VectorizedColumnBatchTest { public boolean isNullAt(int i) { return false; } - - @Override - public void reset() { - - } }; long[] vector11 = new long[VECTOR_SIZE]; @@ -206,10 +191,6 @@ public class VectorizedColumnBatchTest { return false; } - @Override - public void reset() { - } - @Override public Decimal getDecimal(int i, int precision, int scale) { return Decimal.fromLong(vector11[i], precision, scale); @@ -252,8 +233,6 @@ public class VectorizedColumnBatchTest { } assertEquals(VECTOR_SIZE, batch.getNumRows()); - batch.reset(); - assertEquals(0, batch.getNumRows()); } @Test @@ -291,10 +270,10 @@ public class VectorizedColumnBatchTest { public void testDictionary() { // all null HeapIntVector col = new HeapIntVector(VECTOR_SIZE); - int[] dict = new int[2]; + Integer[] dict = new Integer[2]; dict[0] = 1998; dict[1] = 9998; - col.setDictionary(new TestDictionary(dict)); + col.setDictionary(new ColumnVectorTest.TestDictionary(dict)); HeapIntVector heapIntVector = col.reserveDictionaryIds(VECTOR_SIZE); for (int i = 0; i < VECTOR_SIZE; i++) { heapIntVector.vector[i] = i % 2 == 0 ? 0 : 1; @@ -311,37 +290,4 @@ public class VectorizedColumnBatchTest { } } } - - private final class TestDictionary implements Dictionary { - private int[] intDictionary; - - public TestDictionary(int[] dictionary) { - this.intDictionary = dictionary; - } - - @Override - public int decodeToInt(int id) { - return intDictionary[id]; - } - - @Override - public long decodeToLong(int id) { - throw new UnsupportedOperationException("Dictionary encoding does not support float"); - } - - @Override - public float decodeToFloat(int id) { - throw new UnsupportedOperationException("Dictionary encoding does not support float"); - } - - @Override - public double decodeToDouble(int id) { - throw new UnsupportedOperationException("Dictionary encoding does not support double"); - } - - @Override - public byte[] decodeToBinary(int id) { - throw new UnsupportedOperationException("Dictionary encoding does not support String"); - } - } } -- Gitee From eb814e861c89488b63572350cc1db880ab40213a Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 20 Feb 2020 15:41:24 +0100 Subject: [PATCH 027/885] [FLINK-16190][e2e] Migrate tests to FlinkResource --- .../util/kafka/SQLClientKafkaITCase.java | 7 +- .../util/kafka/StreamingKafkaITCase.java | 15 ++-- .../flink/tests/util/FlinkDistribution.java | 36 +++++--- .../flink/tests/util/flink/FlinkResource.java | 36 +++++--- .../util/flink/FlinkResourceFactory.java | 3 +- .../tests/util/flink/FlinkResourceSetup.java | 82 +++++++++++++++++++ .../flink/tests/util/flink/JarLocation.java | 26 ++++++ .../flink/tests/util/flink/JarMove.java | 46 +++++++++++ .../flink/LocalStandaloneFlinkResource.java | 25 ++++-- .../LocalStandaloneFlinkResourceFactory.java | 4 +- .../tests/MetricsAvailabilityITCase.java | 23 ++++-- .../PrometheusReporterEndToEndITCase.java | 68 ++++++++------- 12 files changed, 296 insertions(+), 75 deletions(-) create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkResourceSetup.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/JarLocation.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/JarMove.java diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientKafkaITCase.java b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientKafkaITCase.java index 728c65d8da..aa8832a9d6 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientKafkaITCase.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientKafkaITCase.java @@ -23,7 +23,8 @@ import org.apache.flink.tests.util.categories.Hadoop; import org.apache.flink.tests.util.categories.TravisGroup1; import org.apache.flink.tests.util.flink.ClusterController; import org.apache.flink.tests.util.flink.FlinkResource; -import org.apache.flink.tests.util.flink.LocalStandaloneFlinkResource; +import org.apache.flink.tests.util.flink.FlinkResourceSetup; +import org.apache.flink.tests.util.flink.LocalStandaloneFlinkResourceFactory; import org.apache.flink.tests.util.flink.SQLJobSubmission; import org.apache.flink.testutils.junit.FailsOnJava11; import org.apache.flink.util.FileUtils; @@ -76,7 +77,9 @@ public class SQLClientKafkaITCase extends TestLogger { } @Rule - public final FlinkResource flink = new LocalStandaloneFlinkResource(); + public final FlinkResource flink = new LocalStandaloneFlinkResourceFactory() + .create(FlinkResourceSetup.builder().build()) + .get(); @Rule public final KafkaResource kafka; diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/StreamingKafkaITCase.java b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/StreamingKafkaITCase.java index e684336392..a7ffb14f09 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/StreamingKafkaITCase.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/StreamingKafkaITCase.java @@ -25,6 +25,7 @@ import org.apache.flink.tests.util.categories.PreCommit; import org.apache.flink.tests.util.categories.TravisGroup1; import org.apache.flink.tests.util.flink.ClusterController; import org.apache.flink.tests.util.flink.FlinkResource; +import org.apache.flink.tests.util.flink.FlinkResourceSetup; import org.apache.flink.tests.util.flink.JobSubmission; import org.apache.flink.testutils.junit.FailsOnJava11; import org.apache.flink.util.TestLogger; @@ -68,7 +69,14 @@ public class StreamingKafkaITCase extends TestLogger { public final KafkaResource kafka; @Rule - public final FlinkResource flink = FlinkResource.get(); + public final FlinkResource flink = FlinkResource.get(FlinkResourceSetup.builder().addConfiguration(getConfiguration()).build()); + + private static Configuration getConfiguration() { + // modify configuration to have enough slots + final Configuration flinkConfig = new Configuration(); + flinkConfig.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, 3); + return flinkConfig; + } public StreamingKafkaITCase(final String kafkaExampleJarPattern, final String kafkaVersion) { this.kafkaExampleJar = TestUtils.getResourceJar(kafkaExampleJarPattern); @@ -77,11 +85,6 @@ public class StreamingKafkaITCase extends TestLogger { @Test public void testKafka() throws Exception { - // modify configuration to have enough slots - final Configuration flinkConfig = new Configuration(); - flinkConfig.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, 3); - flink.addConfiguration(flinkConfig); - try (final ClusterController clusterController = flink.startCluster(1)) { final String inputTopic = "test-input"; diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java index 3a5ff133aa..b57bec8e8c 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java @@ -22,6 +22,8 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.configuration.UnmodifiableConfiguration; +import org.apache.flink.tests.util.flink.JarLocation; +import org.apache.flink.tests.util.flink.JarMove; import org.apache.flink.tests.util.flink.JobSubmission; import org.apache.flink.tests.util.flink.SQLJobSubmission; import org.apache.flink.util.ExceptionUtils; @@ -261,19 +263,33 @@ public final class FlinkDistribution implements ExternalResource { AutoClosableProcess.runBlocking(commands.toArray(new String[0])); } - public void copyOptJarsToLib(String jarNamePrefix) throws FileNotFoundException, IOException { - final Optional reporterJarOptional; - try (Stream logFiles = Files.walk(opt)) { - reporterJarOptional = logFiles - .filter(path -> path.getFileName().toString().startsWith(jarNamePrefix)) + public void moveJar(JarMove move) throws IOException { + final Path source = mapJarLocationToPath(move.getSource()); + final Path target = mapJarLocationToPath(move.getTarget()); + + final Optional jarOptional; + try (Stream files = Files.walk(source)) { + jarOptional = files + .filter(path -> path.getFileName().toString().startsWith(move.getJarNamePrefix())) .findFirst(); } - if (reporterJarOptional.isPresent()) { - final Path optReporterJar = reporterJarOptional.get(); - final Path libReporterJar = lib.resolve(optReporterJar.getFileName()); - Files.copy(optReporterJar, libReporterJar); + if (jarOptional.isPresent()) { + final Path sourceJar = jarOptional.get(); + final Path targetJar = target.resolve(sourceJar.getFileName()); + Files.move(sourceJar, targetJar); } else { - throw new FileNotFoundException("No jar could be found matching the pattern " + jarNamePrefix + "."); + throw new FileNotFoundException("No jar could be found matching the pattern " + move.getJarNamePrefix() + "."); + } + } + + private Path mapJarLocationToPath(JarLocation location) { + switch (location) { + case LIB: + return lib; + case OPT: + return opt; + default: + throw new IllegalStateException(); } } diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkResource.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkResource.java index 0d4f317f2e..0e285f2ba0 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkResource.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkResource.java @@ -18,26 +18,20 @@ package org.apache.flink.tests.util.flink; -import org.apache.flink.configuration.Configuration; import org.apache.flink.tests.util.util.FactoryUtils; import org.apache.flink.util.ExternalResource; import java.io.IOException; +import java.util.function.Function; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Stream; /** * Generic interface for interacting with Flink. */ public interface FlinkResource extends ExternalResource { - /** - * Adds the given configuration to the existing configuration of this resource. Entries in the existing configuration - * will be overwritten. - * - * @param config config to add - * @throws IOException - */ - void addConfiguration(Configuration config) throws IOException; - /** * Starts a cluster. * @@ -52,15 +46,35 @@ public interface FlinkResource extends ExternalResource { */ ClusterController startCluster(int numTaskManagers) throws IOException; + /** + * Searches the logs of all processes for the given pattern, and applies the given processor for every line for + * which {@link Matcher#matches()} returned true. + * + * @param pattern pattern to search for + * @param matchProcessor match processor + * @return stream of matched strings + */ + Stream searchAllLogs(Pattern pattern, Function matchProcessor) throws IOException; + /** * Returns the configured FlinkResource implementation, or a {@link LocalStandaloneFlinkResource} if none is configured. * * @return configured FlinkResource, or {@link LocalStandaloneFlinkResource} is none is configured */ static FlinkResource get() { + return get(FlinkResourceSetup.builder().build()); + } + + /** + * Returns the configured FlinkResource implementation, or a {@link LocalStandaloneFlinkResource} if none is configured. + * + * @param setup setup instructions for the FlinkResource + * @return configured FlinkResource, or {@link LocalStandaloneFlinkResource} is none is configured + */ + static FlinkResource get(FlinkResourceSetup setup) { return FactoryUtils.loadAndInvokeFactory( FlinkResourceFactory.class, - FlinkResourceFactory::create, + factory -> factory.create(setup), LocalStandaloneFlinkResourceFactory::new); } } diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkResourceFactory.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkResourceFactory.java index ce89be7104..385282be8c 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkResourceFactory.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkResourceFactory.java @@ -30,7 +30,8 @@ public interface FlinkResourceFactory { * Returns a {@link FlinkResource} instance. If the instance could not be instantiated (for example, because a * mandatory parameter was missing), then an empty {@link Optional} should be returned. * + * @param setup setup instructions for the FlinkResource * @return FlinkResource instance, or an empty Optional if the instance could not be instantiated */ - Optional create(); + Optional create(FlinkResourceSetup setup); } diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkResourceSetup.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkResourceSetup.java new file mode 100644 index 0000000000..7eebe41b01 --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkResourceSetup.java @@ -0,0 +1,82 @@ +/* + * 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.tests.util.flink; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.util.Preconditions; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Optional; + +/** + * Setup instructions for a {@link FlinkResource}. + */ +public class FlinkResourceSetup { + + @Nullable + private final Configuration config; + private final Collection jarMoveOperations; + + private FlinkResourceSetup(@Nullable Configuration config, Collection jarMoveOperations) { + this.config = config; + this.jarMoveOperations = Preconditions.checkNotNull(jarMoveOperations); + } + + public Optional getConfig() { + return Optional.ofNullable(config); + } + + public Collection getJarMoveOperations() { + return jarMoveOperations; + } + + public static FlinkResourceSetupBuilder builder() { + return new FlinkResourceSetupBuilder(); + } + + /** + * Builder for {@link FlinkResourceSetup}. + */ + public static class FlinkResourceSetupBuilder { + + private Configuration config; + private final Collection jarMoveOperations = new ArrayList<>(); + + private FlinkResourceSetupBuilder() { + } + + public FlinkResourceSetupBuilder addConfiguration(Configuration config) { + this.config = config; + return this; + } + + public FlinkResourceSetupBuilder moveJar(String jarNamePrefix, JarLocation source, JarLocation target) { + this.jarMoveOperations.add(new JarMove(jarNamePrefix, source, target)); + return this; + } + + public FlinkResourceSetup build() { + return new FlinkResourceSetup(config, Collections.unmodifiableCollection(jarMoveOperations)); + } + } + +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/JarLocation.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/JarLocation.java new file mode 100644 index 0000000000..5af964a8eb --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/JarLocation.java @@ -0,0 +1,26 @@ +/* + * 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.tests.util.flink; + +/** + * Enum for specifying jar locations. + */ +public enum JarLocation { + LIB, + OPT, +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/JarMove.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/JarMove.java new file mode 100644 index 0000000000..00eb418b76 --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/JarMove.java @@ -0,0 +1,46 @@ +/* + * 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.tests.util.flink; + +/** + * Represents a move operation for a jar. + */ +public class JarMove { + + private final String jarNamePrefix; + private final JarLocation source; + private final JarLocation target; + + JarMove(String jarNamePrefix, JarLocation source, JarLocation target) { + this.jarNamePrefix = jarNamePrefix; + this.source = source; + this.target = target; + } + + public String getJarNamePrefix() { + return jarNamePrefix; + } + + public JarLocation getSource() { + return source; + } + + public JarLocation getTarget() { + return target; + } +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResource.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResource.java index db4400ac4f..442085b636 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResource.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResource.java @@ -40,6 +40,10 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.function.Function; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Stream; /** * Flink resource that start local standalone clusters. @@ -49,10 +53,21 @@ public class LocalStandaloneFlinkResource implements FlinkResource { private static final Logger LOG = LoggerFactory.getLogger(LocalStandaloneFlinkResource.class); private final FlinkDistribution distribution = new FlinkDistribution(); + private final FlinkResourceSetup setup; + + LocalStandaloneFlinkResource(FlinkResourceSetup setup) { + this.setup = setup; + } @Override public void before() throws Exception { distribution.before(); + for (JarMove jarMove : setup.getJarMoveOperations()) { + distribution.moveJar(jarMove); + } + if (setup.getConfig().isPresent()) { + distribution.appendConfiguration(setup.getConfig().get()); + } } @Override @@ -65,11 +80,6 @@ public class LocalStandaloneFlinkResource implements FlinkResource { distribution.afterTestFailure(); } - @Override - public void addConfiguration(final Configuration config) throws IOException { - distribution.appendConfiguration(config); - } - @Override public ClusterController startCluster(int numTaskManagers) throws IOException { distribution.setTaskExecutorHosts(Collections.nCopies(numTaskManagers, "localhost")); @@ -116,6 +126,11 @@ public class LocalStandaloneFlinkResource implements FlinkResource { throw new RuntimeException("Cluster did not start in expected time-frame."); } + @Override + public Stream searchAllLogs(Pattern pattern, Function matchProcessor) throws IOException { + return distribution.searchAllLogs(pattern, matchProcessor); + } + private static class StandaloneClusterController implements ClusterController { private final FlinkDistribution distribution; diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResourceFactory.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResourceFactory.java index af998121af..bbfaef6fd2 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResourceFactory.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResourceFactory.java @@ -30,8 +30,8 @@ public final class LocalStandaloneFlinkResourceFactory implements FlinkResourceF private static final Logger LOG = LoggerFactory.getLogger(LocalStandaloneFlinkResourceFactory.class); @Override - public Optional create() { + public Optional create(FlinkResourceSetup setup) { LOG.info("Created {}.", LocalStandaloneFlinkResource.class.getSimpleName()); - return Optional.of(new LocalStandaloneFlinkResource()); + return Optional.of(new LocalStandaloneFlinkResource(setup)); } } diff --git a/flink-end-to-end-tests/flink-metrics-availability-test/src/test/java/org/pache/flink/metrics/tests/MetricsAvailabilityITCase.java b/flink-end-to-end-tests/flink-metrics-availability-test/src/test/java/org/pache/flink/metrics/tests/MetricsAvailabilityITCase.java index 49d151e9f1..68c94981f2 100644 --- a/flink-end-to-end-tests/flink-metrics-availability-test/src/test/java/org/pache/flink/metrics/tests/MetricsAvailabilityITCase.java +++ b/flink-end-to-end-tests/flink-metrics-availability-test/src/test/java/org/pache/flink/metrics/tests/MetricsAvailabilityITCase.java @@ -36,8 +36,11 @@ import org.apache.flink.runtime.rest.messages.job.metrics.TaskManagerMetricsMess import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerInfo; import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagersHeaders; import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagersInfo; -import org.apache.flink.tests.util.FlinkDistribution; import org.apache.flink.tests.util.categories.TravisGroup1; +import org.apache.flink.tests.util.flink.ClusterController; +import org.apache.flink.tests.util.flink.FlinkResource; +import org.apache.flink.tests.util.flink.FlinkResourceSetup; +import org.apache.flink.tests.util.flink.LocalStandaloneFlinkResourceFactory; import org.apache.flink.util.TestLogger; import org.apache.flink.util.function.SupplierWithException; @@ -72,7 +75,9 @@ public class MetricsAvailabilityITCase extends TestLogger { private static final int PORT = 8081; @Rule - public final FlinkDistribution dist = new FlinkDistribution(); + public final FlinkResource dist = new LocalStandaloneFlinkResourceFactory() + .create(FlinkResourceSetup.builder().build()) + .get(); @Nullable private static ScheduledExecutorService scheduledExecutorService = null; @@ -91,16 +96,16 @@ public class MetricsAvailabilityITCase extends TestLogger { @Test public void testReporter() throws Exception { - dist.startFlinkCluster(); + try (ClusterController ignored = dist.startCluster(1)) { + final RestClient restClient = new RestClient(RestClientConfiguration.fromConfiguration(new Configuration()), scheduledExecutorService); - final RestClient restClient = new RestClient(RestClientConfiguration.fromConfiguration(new Configuration()), scheduledExecutorService); + checkJobManagerMetricAvailability(restClient); - checkJobManagerMetricAvailability(restClient); + final Collection taskManagerIds = getTaskManagerIds(restClient); - final Collection taskManagerIds = getTaskManagerIds(restClient); - - for (final ResourceID taskManagerId : taskManagerIds) { - checkTaskManagerMetricAvailability(restClient, taskManagerId); + for (final ResourceID taskManagerId : taskManagerIds) { + checkTaskManagerMetricAvailability(restClient, taskManagerId); + } } } diff --git a/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java index fe8fe10ae3..663744061e 100644 --- a/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java +++ b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java @@ -23,9 +23,13 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.prometheus.PrometheusReporter; import org.apache.flink.tests.util.AutoClosableProcess; import org.apache.flink.tests.util.CommandLineWrapper; -import org.apache.flink.tests.util.FlinkDistribution; import org.apache.flink.tests.util.cache.DownloadCache; import org.apache.flink.tests.util.categories.TravisGroup1; +import org.apache.flink.tests.util.flink.ClusterController; +import org.apache.flink.tests.util.flink.FlinkResource; +import org.apache.flink.tests.util.flink.FlinkResourceSetup; +import org.apache.flink.tests.util.flink.JarLocation; +import org.apache.flink.tests.util.flink.LocalStandaloneFlinkResourceFactory; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.OperatingSystem; import org.apache.flink.util.ProcessorArchitecture; @@ -111,7 +115,12 @@ public class PrometheusReporterEndToEndITCase extends TestLogger { } @Rule - public final FlinkDistribution dist = new FlinkDistribution(); + public final FlinkResource dist = new LocalStandaloneFlinkResourceFactory() + .create(FlinkResourceSetup.builder() + .moveJar("flink-metrics-prometheus", JarLocation.OPT, JarLocation.LIB) + .addConfiguration(getFlinkConfig()) + .build()) + .get(); @Rule public final TemporaryFolder tmp = new TemporaryFolder(); @@ -119,16 +128,16 @@ public class PrometheusReporterEndToEndITCase extends TestLogger { @Rule public final DownloadCache downloadCache = DownloadCache.get(); - @Test - public void testReporter() throws Exception { - dist.copyOptJarsToLib("flink-metrics-prometheus"); - + private static Configuration getFlinkConfig() { final Configuration config = new Configuration(); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, PrometheusReporter.class.getCanonicalName()); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom.port", "9000-9100"); - dist.appendConfiguration(config); + return config; + } + @Test + public void testReporter() throws Exception { final Path tmpPrometheusDir = tmp.newFolder().toPath().resolve("prometheus"); final Path prometheusArchive = tmpPrometheusDir.resolve(PROMETHEUS_FILE_NAME + ".tar.gz"); final Path prometheusBinDir = tmpPrometheusDir.resolve(PROMETHEUS_FILE_NAME); @@ -157,34 +166,35 @@ public class PrometheusReporterEndToEndITCase extends TestLogger { .inPlace() .build()); - dist.startFlinkCluster(); + try (ClusterController ignored = dist.startCluster(1)) { - final List ports = dist - .searchAllLogs(LOG_REPORTER_PORT_PATTERN, matcher -> matcher.group(1)) - .map(Integer::valueOf) - .collect(Collectors.toList()); + final List ports = dist + .searchAllLogs(LOG_REPORTER_PORT_PATTERN, matcher -> matcher.group(1)) + .map(Integer::valueOf) + .collect(Collectors.toList()); - final String scrapeTargets = ports.stream() - .map(port -> "'localhost:" + port + "'") - .collect(Collectors.joining(", ")); + final String scrapeTargets = ports.stream() + .map(port -> "'localhost:" + port + "'") + .collect(Collectors.joining(", ")); - LOG.info("Setting Prometheus scrape targets to {}.", scrapeTargets); - runBlocking( - CommandLineWrapper - .sed("s/\\(targets:\\).*/\\1 [" + scrapeTargets + "]/", prometheusConfig) - .inPlace() - .build()); + LOG.info("Setting Prometheus scrape targets to {}.", scrapeTargets); + runBlocking( + CommandLineWrapper + .sed("s/\\(targets:\\).*/\\1 [" + scrapeTargets + "]/", prometheusConfig) + .inPlace() + .build()); - LOG.info("Starting Prometheus server."); - try (AutoClosableProcess prometheus = runNonBlocking( - prometheusBinary.toAbsolutePath().toString(), - "--config.file=" + prometheusConfig.toAbsolutePath(), - "--storage.tsdb.path=" + prometheusBinDir.resolve("data").toAbsolutePath())) { + LOG.info("Starting Prometheus server."); + try (AutoClosableProcess prometheus = runNonBlocking( + prometheusBinary.toAbsolutePath().toString(), + "--config.file=" + prometheusConfig.toAbsolutePath(), + "--storage.tsdb.path=" + prometheusBinDir.resolve("data").toAbsolutePath())) { - final OkHttpClient client = new OkHttpClient(); + final OkHttpClient client = new OkHttpClient(); - checkMetricAvailability(client, "flink_jobmanager_numRegisteredTaskManagers"); - checkMetricAvailability(client, "flink_taskmanager_Status_Network_TotalMemorySegments"); + checkMetricAvailability(client, "flink_jobmanager_numRegisteredTaskManagers"); + checkMetricAvailability(client, "flink_taskmanager_Status_Network_TotalMemorySegments"); + } } } -- Gitee From e3307b7dae60709acd90e388a64a35be5ecb9876 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 20 Feb 2020 15:46:45 +0100 Subject: [PATCH 028/885] [FLINK-16190][e2e] Move FlinkDistribution into flink package --- .../tests/util/{ => flink}/FlinkDistribution.java | 12 +++++------- .../org/apache/flink/tests/util/flink/JarMove.java | 2 +- .../util/flink/LocalStandaloneFlinkResource.java | 1 - 3 files changed, 6 insertions(+), 9 deletions(-) rename flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/{ => flink}/FlinkDistribution.java (96%) diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkDistribution.java similarity index 96% rename from flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java rename to flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkDistribution.java index b57bec8e8c..c0cebe0c82 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkDistribution.java @@ -16,16 +16,14 @@ * limitations under the License. */ -package org.apache.flink.tests.util; +package org.apache.flink.tests.util.flink; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.configuration.UnmodifiableConfiguration; -import org.apache.flink.tests.util.flink.JarLocation; -import org.apache.flink.tests.util.flink.JarMove; -import org.apache.flink.tests.util.flink.JobSubmission; -import org.apache.flink.tests.util.flink.SQLJobSubmission; +import org.apache.flink.tests.util.AutoClosableProcess; +import org.apache.flink.tests.util.TestUtils; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.ExternalResource; @@ -67,7 +65,7 @@ import java.util.stream.Stream; /** * A wrapper around a Flink distribution. */ -public final class FlinkDistribution implements ExternalResource { +final class FlinkDistribution implements ExternalResource { private static final Logger LOG = LoggerFactory.getLogger(FlinkDistribution.class); @@ -86,7 +84,7 @@ public final class FlinkDistribution implements ExternalResource { private Configuration defaultConfig; - public FlinkDistribution() { + FlinkDistribution() { final String distDirProperty = System.getProperty("distDir"); if (distDirProperty == null) { Assert.fail("The distDir property was not set. You can set it when running maven via -DdistDir= ."); diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/JarMove.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/JarMove.java index 00eb418b76..d9c407692f 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/JarMove.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/JarMove.java @@ -20,7 +20,7 @@ package org.apache.flink.tests.util.flink; /** * Represents a move operation for a jar. */ -public class JarMove { +class JarMove { private final String jarNamePrefix; private final JarLocation source; diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResource.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResource.java index 442085b636..1a0fb13f7e 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResource.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResource.java @@ -28,7 +28,6 @@ import org.apache.flink.runtime.rest.messages.EmptyMessageParameters; import org.apache.flink.runtime.rest.messages.EmptyRequestBody; import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagersHeaders; import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagersInfo; -import org.apache.flink.tests.util.FlinkDistribution; import org.apache.flink.util.ConfigurationException; import org.slf4j.Logger; -- Gitee From a9d47c75468dccf4785027b01d90bfd46c2e700e Mon Sep 17 00:00:00 2001 From: Kevin Bohinski Date: Tue, 3 Mar 2020 14:06:21 +0000 Subject: [PATCH 029/885] [FLINK-16348][WebUI] Add commas to numeric accumulators --- .../job-overview-drawer-accumulators.component.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime-web/web-dashboard/src/app/pages/job/overview/accumulators/job-overview-drawer-accumulators.component.html b/flink-runtime-web/web-dashboard/src/app/pages/job/overview/accumulators/job-overview-drawer-accumulators.component.html index c750094b67..fe1585ec82 100644 --- a/flink-runtime-web/web-dashboard/src/app/pages/job/overview/accumulators/job-overview-drawer-accumulators.component.html +++ b/flink-runtime-web/web-dashboard/src/app/pages/job/overview/accumulators/job-overview-drawer-accumulators.component.html @@ -37,7 +37,7 @@ {{ accumulator.name }} {{ accumulator.type }} - {{ accumulator.value }} + {{ (accumulator.value | number:'1.0-3' ) || accumulator.value }} -- Gitee From 68b2b56c7dd5c47a627562c80a257618129a7264 Mon Sep 17 00:00:00 2001 From: Paul Lin Date: Tue, 3 Mar 2020 22:09:55 +0800 Subject: [PATCH 030/885] [hotfix][dataset][javadoc] Fix typo typo in InputFormat. --- .../main/java/org/apache/flink/api/common/io/InputFormat.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/InputFormat.java b/flink-core/src/main/java/org/apache/flink/api/common/io/InputFormat.java index 1439f0ae4f..9d1339c38f 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/io/InputFormat.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/io/InputFormat.java @@ -80,7 +80,7 @@ public interface InputFormat extends InputSplitSource< * This method optionally gets a cached version of the statistics. The input format may examine them and decide * whether it directly returns them without spending effort to re-gather the statistics. *

    - * When this method is called, the input format it guaranteed to be configured. + * When this method is called, the input format is guaranteed to be configured. * * @param cachedStatistics The statistics that were cached. May be null. * @return The base statistics for the input, or null, if not available. -- Gitee From 69bdb727eb66c391b5c308cb0c0617f7faaff3c4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Beno=C3=AEt=20Paris?= Date: Tue, 28 Jan 2020 20:22:06 +0100 Subject: [PATCH 031/885] [hotfix][table][docs] Fix typo --- docs/dev/table/streaming/joins.md | 2 +- docs/dev/table/streaming/joins.zh.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/dev/table/streaming/joins.md b/docs/dev/table/streaming/joins.md index deb31995de..e621a2057d 100644 --- a/docs/dev/table/streaming/joins.md +++ b/docs/dev/table/streaming/joins.md @@ -63,7 +63,7 @@ WHERE o.id = s.orderId AND o.ordertime BETWEEN s.shiptime - INTERVAL '4' HOUR AND s.shiptime {% endhighlight %} -Compared to a regular join operation, this kind of join only supports append-only tables with time attributes. Since time attributes are quasi-monontic increasing, Flink can remove old values from its state without affecting the correctness of the result. +Compared to a regular join operation, this kind of join only supports append-only tables with time attributes. Since time attributes are quasi-monotonic increasing, Flink can remove old values from its state without affecting the correctness of the result. Join with a Temporal Table Function -------------------------- diff --git a/docs/dev/table/streaming/joins.zh.md b/docs/dev/table/streaming/joins.zh.md index deb31995de..e621a2057d 100644 --- a/docs/dev/table/streaming/joins.zh.md +++ b/docs/dev/table/streaming/joins.zh.md @@ -63,7 +63,7 @@ WHERE o.id = s.orderId AND o.ordertime BETWEEN s.shiptime - INTERVAL '4' HOUR AND s.shiptime {% endhighlight %} -Compared to a regular join operation, this kind of join only supports append-only tables with time attributes. Since time attributes are quasi-monontic increasing, Flink can remove old values from its state without affecting the correctness of the result. +Compared to a regular join operation, this kind of join only supports append-only tables with time attributes. Since time attributes are quasi-monotonic increasing, Flink can remove old values from its state without affecting the correctness of the result. Join with a Temporal Table Function -------------------------- -- Gitee From 197991aaccf1388b9ef86af1d0bba72a8c286a84 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 3 Mar 2020 15:25:23 +0100 Subject: [PATCH 032/885] [hotfix][kinesis][build] Document why guava must not be relocated --- flink-connectors/flink-connector-kinesis/pom.xml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-kinesis/pom.xml b/flink-connectors/flink-connector-kinesis/pom.xml index 388a978c28..edddad5e44 100644 --- a/flink-connectors/flink-connector-kinesis/pom.xml +++ b/flink-connectors/flink-connector-kinesis/pom.xml @@ -200,7 +200,8 @@ under the License. - + com.google.protobuf org.apache.flink.kinesis.shaded.com.google.protobuf -- Gitee From f17912748d4632bf6b585277eff499af5739cb1a Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Mon, 2 Mar 2020 15:12:54 +0100 Subject: [PATCH 033/885] [FLINK-10885][e2e] Stabilize confluent schema registry test Potential problem: - the error message "No supported Kafka endpoints are configured. Either kafkastore.bootstrap.servers must have at least one endpoint matching kafkastore.security.protocol or broker endpoints loaded from ZooKeeper must have at least one endpoint matching" indicates that the one of the mentioned config parameters has to be set, or the brokers need to be listed in Zookeeper. - The source code of the "KafkaStore" in the schema registry supports this hypothesis: https://github.com/confluentinc/schema-registry/blob/3.2.0-post/core/src/main/java/io/confluent/kafka/schemaregistry/storage/KafkaStore.java#L121 - after starting the Kafka cluster, we wait until "Node does not exist" is not returned by the ZK Cli. - However, if ZK is not (yet) running, we'll also have outputs such as "WARN Session 0x0 for server null, unexpected error, closing socket connection and attempting reconnect (org.apache.zookeeper.ClientCnxn)", stopping the wait loop. Solution: Wait till we have a ZK Cli output indicating that the broker is registered. --- flink-end-to-end-tests/test-scripts/kafka-common.sh | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/flink-end-to-end-tests/test-scripts/kafka-common.sh b/flink-end-to-end-tests/test-scripts/kafka-common.sh index e0b7380488..0ca5336d56 100644 --- a/flink-end-to-end-tests/test-scripts/kafka-common.sh +++ b/flink-end-to-end-tests/test-scripts/kafka-common.sh @@ -69,8 +69,11 @@ function start_kafka_cluster { $KAFKA_DIR/bin/kafka-server-start.sh -daemon $KAFKA_DIR/config/server.properties start_time=$(date +%s) - # zookeeper outputs the "Node does not exist" bit to stderr - while [[ $($KAFKA_DIR/bin/zookeeper-shell.sh localhost:2181 get /brokers/ids/0 2>&1) =~ .*Node\ does\ not\ exist.* ]]; do + # + # Wait for the broker info to appear in ZK. We assume propery registration once an entry + # similar to this is in ZK: {"listener_security_protocol_map":{"PLAINTEXT":"PLAINTEXT"},"endpoints":["PLAINTEXT://my-host:9092"],"jmx_port":-1,"host":"honorary-pig","timestamp":"1583157804932","port":9092,"version":4} + # + while ! [[ $($KAFKA_DIR/bin/zookeeper-shell.sh localhost:2181 get /brokers/ids/0 2>&1) =~ .*listener_security_protocol_map.* ]]; do current_time=$(date +%s) time_diff=$((current_time - start_time)) -- Gitee From bc73338152152da7c7cdc901bfd951c4d15ffae0 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Tue, 25 Feb 2020 20:55:25 +0800 Subject: [PATCH 034/885] [FLINK-15582][tests] Enable batch scheduling tests for both LegacyScheduler and DefaultScheduler --- .../scheduler/BatchSchedulingTestBase.java | 209 ++++++++++++++++++ .../DefaultSchedulerBatchSchedulingTest.java | 38 ++++ .../LegacySchedulerBatchSchedulingTest.java | 194 +--------------- 3 files changed, 257 insertions(+), 184 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/BatchSchedulingTestBase.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerBatchSchedulingTest.java diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/BatchSchedulingTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/BatchSchedulingTestBase.java new file mode 100644 index 0000000000..710ac50c00 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/BatchSchedulingTestBase.java @@ -0,0 +1,209 @@ +/* + * 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.runtime.scheduler; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.executiongraph.JobStatusListener; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobgraph.ScheduleMode; +import org.apache.flink.runtime.jobmaster.JobMasterId; +import org.apache.flink.runtime.jobmaster.RpcTaskManagerGateway; +import org.apache.flink.runtime.jobmaster.slotpool.LocationPreferenceSlotSelectionStrategy; +import org.apache.flink.runtime.jobmaster.slotpool.SchedulerImpl; +import org.apache.flink.runtime.jobmaster.slotpool.SlotPool; +import org.apache.flink.runtime.jobmaster.slotpool.SlotPoolBuilder; +import org.apache.flink.runtime.jobmaster.slotpool.SlotPoolImpl; +import org.apache.flink.runtime.jobmaster.slotpool.SlotPoolUtils; +import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGateway; +import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder; +import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.util.TestLogger; +import org.apache.flink.util.function.CheckedSupplier; + +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnull; + +import java.util.Collections; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.fail; + +/** + * Tests base for the scheduling of batch jobs. + */ +public abstract class BatchSchedulingTestBase extends TestLogger { + + protected final Logger log = LoggerFactory.getLogger(getClass()); + + private static final JobID jobId = new JobID(); + + private static ScheduledExecutorService singleThreadScheduledExecutorService; + private static ComponentMainThreadExecutor mainThreadExecutor; + + @BeforeClass + public static void setupClass() { + singleThreadScheduledExecutorService = Executors.newSingleThreadScheduledExecutor(); + mainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forSingleThreadExecutor(singleThreadScheduledExecutorService); + } + + @AfterClass + public static void teardownClass() { + if (singleThreadScheduledExecutorService != null) { + singleThreadScheduledExecutorService.shutdownNow(); + } + } + + /** + * Tests that a batch job can be executed with fewer slots than its parallelism. + * See FLINK-13187 for more information. + */ + @Test + public void testSchedulingOfJobWithFewerSlotsThanParallelism() throws Exception { + final int parallelism = 5; + final Time batchSlotTimeout = Time.milliseconds(5L); + final JobGraph jobGraph = createJobGraph(parallelism); + jobGraph.setScheduleMode(ScheduleMode.LAZY_FROM_SOURCES_WITH_BATCH_SLOT_REQUEST); + + try (final SlotPoolImpl slotPool = createSlotPool(mainThreadExecutor, batchSlotTimeout)) { + final ArrayBlockingQueue submittedTasksQueue = new ArrayBlockingQueue<>(parallelism); + TestingTaskExecutorGateway testingTaskExecutorGateway = new TestingTaskExecutorGatewayBuilder() + .setSubmitTaskConsumer( + (tdd, ignored) -> { + submittedTasksQueue.offer(tdd.getExecutionAttemptId()); + return CompletableFuture.completedFuture(Acknowledge.get()); + }) + .createTestingTaskExecutorGateway(); + + // register a single slot at the slot pool + SlotPoolUtils.offerSlots( + slotPool, + mainThreadExecutor, + Collections.singletonList(ResourceProfile.ANY), + new RpcTaskManagerGateway(testingTaskExecutorGateway, JobMasterId.generate())); + + final SlotProvider slotProvider = createSlotProvider(slotPool, mainThreadExecutor); + final SchedulerNG scheduler = createScheduler(jobGraph, slotProvider, batchSlotTimeout); + + final GloballyTerminalJobStatusListener jobStatusListener = new GloballyTerminalJobStatusListener(); + scheduler.registerJobStatusListener(jobStatusListener); + startScheduling(scheduler, mainThreadExecutor); + + // wait until the batch slot timeout has been reached + Thread.sleep(batchSlotTimeout.toMilliseconds()); + + final CompletableFuture terminationFuture = jobStatusListener.getTerminationFuture(); + + for (int i = 0; i < parallelism; i++) { + final CompletableFuture submittedTaskFuture = CompletableFuture.supplyAsync(CheckedSupplier.unchecked(submittedTasksQueue::take)); + + // wait until one of them is completed + CompletableFuture.anyOf(submittedTaskFuture, terminationFuture).join(); + + if (submittedTaskFuture.isDone()) { + finishExecution(submittedTaskFuture.get(), scheduler, mainThreadExecutor); + } else { + fail(String.format("Job reached a globally terminal state %s before all executions were finished.", terminationFuture.get())); + } + } + + assertThat(terminationFuture.get(), is(JobStatus.FINISHED)); + } + } + + private void finishExecution( + ExecutionAttemptID executionAttemptId, + SchedulerNG scheduler, + ComponentMainThreadExecutor mainThreadExecutor) { + CompletableFuture.runAsync( + () -> { + scheduler.updateTaskExecutionState(new TaskExecutionState(jobId, executionAttemptId, ExecutionState.RUNNING)); + scheduler.updateTaskExecutionState(new TaskExecutionState(jobId, executionAttemptId, ExecutionState.FINISHED)); + }, + mainThreadExecutor + ).join(); + } + + @Nonnull + private SlotProvider createSlotProvider(SlotPool slotPool, ComponentMainThreadExecutor mainThreadExecutor) { + final SchedulerImpl scheduler = new SchedulerImpl(LocationPreferenceSlotSelectionStrategy.createDefault(), slotPool); + scheduler.start(mainThreadExecutor); + + return scheduler; + } + + private void startScheduling(SchedulerNG scheduler, ComponentMainThreadExecutor mainThreadExecutor) { + scheduler.setMainThreadExecutor(mainThreadExecutor); + CompletableFuture.runAsync( + scheduler::startScheduling, + mainThreadExecutor) + .join(); + } + + private SlotPoolImpl createSlotPool(ComponentMainThreadExecutor mainThreadExecutor, Time batchSlotTimeout) throws Exception { + return new SlotPoolBuilder(mainThreadExecutor) + .setBatchSlotTimeout(batchSlotTimeout) + .build(); + } + + private JobGraph createJobGraph(int parallelism) { + final JobVertex jobVertex = new JobVertex("testing task"); + jobVertex.setParallelism(parallelism); + jobVertex.setInvokableClass(NoOpInvokable.class); + return new JobGraph(jobId, "test job", jobVertex); + } + + private static class GloballyTerminalJobStatusListener implements JobStatusListener { + + private final CompletableFuture globallyTerminalJobStatusFuture = new CompletableFuture<>(); + + @Override + public void jobStatusChanges(JobID jobId, JobStatus newJobStatus, long timestamp, Throwable error) { + if (newJobStatus.isGloballyTerminalState()) { + globallyTerminalJobStatusFuture.complete(newJobStatus); + } + } + + public CompletableFuture getTerminationFuture() { + return globallyTerminalJobStatusFuture; + } + } + + protected abstract SchedulerNG createScheduler(JobGraph jobGraph, SlotProvider slotProvider, Time slotRequestTimeout) throws Exception; +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerBatchSchedulingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerBatchSchedulingTest.java new file mode 100644 index 0000000000..c8bc57376c --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerBatchSchedulingTest.java @@ -0,0 +1,38 @@ +/* + * 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.runtime.scheduler; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; + +/** + * Tests for the scheduling of batch jobs with {@link DefaultScheduler}. + */ +public class DefaultSchedulerBatchSchedulingTest extends BatchSchedulingTestBase { + + @Override + protected DefaultScheduler createScheduler( + final JobGraph jobGraph, + final SlotProvider slotProvider, + final Time slotRequestTimeout) throws Exception { + + return SchedulerTestingUtils.createScheduler(jobGraph, slotProvider, slotRequestTimeout); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/LegacySchedulerBatchSchedulingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/LegacySchedulerBatchSchedulingTest.java index df34c34a4b..c5b6b42122 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/LegacySchedulerBatchSchedulingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/LegacySchedulerBatchSchedulingTest.java @@ -18,186 +18,37 @@ package org.apache.flink.runtime.scheduler; -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.VoidBlobWriter; import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; -import org.apache.flink.runtime.clusterframework.types.ResourceProfile; -import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor; -import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; -import org.apache.flink.runtime.execution.ExecutionState; -import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; -import org.apache.flink.runtime.executiongraph.JobStatusListener; import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; import org.apache.flink.runtime.io.network.partition.NoOpJobMasterPartitionTracker; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobgraph.JobVertex; -import org.apache.flink.runtime.jobgraph.ScheduleMode; -import org.apache.flink.runtime.jobmaster.JobMasterId; -import org.apache.flink.runtime.jobmaster.RpcTaskManagerGateway; -import org.apache.flink.runtime.jobmaster.slotpool.LocationPreferenceSlotSelectionStrategy; -import org.apache.flink.runtime.jobmaster.slotpool.Scheduler; -import org.apache.flink.runtime.jobmaster.slotpool.SchedulerImpl; -import org.apache.flink.runtime.jobmaster.slotpool.SlotPool; -import org.apache.flink.runtime.jobmaster.slotpool.SlotPoolBuilder; -import org.apache.flink.runtime.jobmaster.slotpool.SlotPoolImpl; -import org.apache.flink.runtime.jobmaster.slotpool.SlotPoolUtils; -import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.rest.handler.legacy.backpressure.VoidBackPressureStatsTracker; import org.apache.flink.runtime.shuffle.NettyShuffleMaster; -import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGateway; -import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder; -import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.runtime.testingUtils.TestingUtils; -import org.apache.flink.runtime.testtasks.NoOpInvokable; -import org.apache.flink.util.TestLogger; -import org.apache.flink.util.function.CheckedSupplier; - -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.annotation.Nonnull; - -import java.util.Collections; -import java.util.concurrent.ArrayBlockingQueue; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.is; -import static org.junit.Assert.fail; /** - * Tests for the scheduling of batch jobs. + * Tests for the scheduling of batch jobs with {@link LegacyScheduler}. */ -public class LegacySchedulerBatchSchedulingTest extends TestLogger { - - private static final Logger LOG = LoggerFactory.getLogger(LegacySchedulerBatchSchedulingTest.class); - - private static final JobID jobId = new JobID(); - - private static ScheduledExecutorService singleThreadScheduledExecutorService; - private static ComponentMainThreadExecutor mainThreadExecutor; - - @BeforeClass - public static void setupClass() { - singleThreadScheduledExecutorService = Executors.newSingleThreadScheduledExecutor(); - mainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forSingleThreadExecutor(singleThreadScheduledExecutorService); - } - - @AfterClass - public static void teardownClass() { - if (singleThreadScheduledExecutorService != null) { - singleThreadScheduledExecutorService.shutdownNow(); - } - } - - /** - * Tests that a batch job can be executed with fewer slots than its parallelism. - * See FLINK-13187 for more information. - */ - @Test - public void testSchedulingOfJobWithFewerSlotsThanParallelism() throws Exception { - final int parallelism = 5; - final Time batchSlotTimeout = Time.milliseconds(5L); - final JobGraph jobGraph = createJobGraph(parallelism); - jobGraph.setScheduleMode(ScheduleMode.LAZY_FROM_SOURCES_WITH_BATCH_SLOT_REQUEST); - - try (final SlotPoolImpl slotPool = createSlotPool(mainThreadExecutor, batchSlotTimeout)) { - final ArrayBlockingQueue submittedTasksQueue = new ArrayBlockingQueue<>(parallelism); - TestingTaskExecutorGateway testingTaskExecutorGateway = new TestingTaskExecutorGatewayBuilder() - .setSubmitTaskConsumer( - (tdd, ignored) -> { - submittedTasksQueue.offer(tdd.getExecutionAttemptId()); - return CompletableFuture.completedFuture(Acknowledge.get()); - }) - .createTestingTaskExecutorGateway(); - - // register a single slot at the slot pool - SlotPoolUtils.offerSlots( - slotPool, - mainThreadExecutor, - Collections.singletonList(ResourceProfile.ANY), - new RpcTaskManagerGateway(testingTaskExecutorGateway, JobMasterId.generate())); - - final LegacyScheduler legacyScheduler = createLegacyScheduler(jobGraph, slotPool, mainThreadExecutor, batchSlotTimeout); - - final GloballyTerminalJobStatusListener jobStatusListener = new GloballyTerminalJobStatusListener(); - legacyScheduler.registerJobStatusListener(jobStatusListener); - startScheduling(legacyScheduler, mainThreadExecutor); - - // wait until the batch slot timeout has been reached - Thread.sleep(batchSlotTimeout.toMilliseconds()); - - final CompletableFuture terminationFuture = jobStatusListener.getTerminationFuture(); - - for (int i = 0; i < parallelism; i++) { - final CompletableFuture submittedTaskFuture = CompletableFuture.supplyAsync(CheckedSupplier.unchecked(submittedTasksQueue::take)); - - // wait until one of them is completed - CompletableFuture.anyOf(submittedTaskFuture, terminationFuture).join(); +public class LegacySchedulerBatchSchedulingTest extends BatchSchedulingTestBase { - if (submittedTaskFuture.isDone()) { - finishExecution(submittedTaskFuture.get(), legacyScheduler, mainThreadExecutor); - } else { - fail(String.format("Job reached a globally terminal state %s before all executions were finished.", terminationFuture.get())); - } - } + @Override + protected LegacyScheduler createScheduler( + final JobGraph jobGraph, + final SlotProvider slotProvider, + final Time slotRequestTimeout) throws Exception { - assertThat(terminationFuture.get(), is(JobStatus.FINISHED)); - } - } - - private void finishExecution( - ExecutionAttemptID executionAttemptId, - LegacyScheduler legacyScheduler, - ComponentMainThreadExecutor mainThreadExecutor) { - CompletableFuture.runAsync( - () -> { - legacyScheduler.updateTaskExecutionState(new TaskExecutionState(jobId, executionAttemptId, ExecutionState.RUNNING)); - legacyScheduler.updateTaskExecutionState(new TaskExecutionState(jobId, executionAttemptId, ExecutionState.FINISHED)); - }, - mainThreadExecutor - ).join(); - } - - @Nonnull - private SchedulerImpl createScheduler(SlotPool slotPool, ComponentMainThreadExecutor mainThreadExecutor) { - final SchedulerImpl scheduler = new SchedulerImpl(LocationPreferenceSlotSelectionStrategy.createDefault(), slotPool); - scheduler.start(mainThreadExecutor); - - return scheduler; - } - - private void startScheduling(LegacyScheduler legacyScheduler, ComponentMainThreadExecutor mainThreadExecutor) { - CompletableFuture.runAsync( - legacyScheduler::startScheduling, - mainThreadExecutor) - .join(); - } - - private SlotPoolImpl createSlotPool(ComponentMainThreadExecutor mainThreadExecutor, Time batchSlotTimeout) throws Exception { - return new SlotPoolBuilder(mainThreadExecutor) - .setBatchSlotTimeout(batchSlotTimeout) - .build(); - } - - private LegacyScheduler createLegacyScheduler(JobGraph jobGraph, SlotPool slotPool, ComponentMainThreadExecutor mainThreadExecutor, Time slotRequestTimeout) throws Exception { - final Scheduler scheduler = createScheduler(slotPool, mainThreadExecutor); final LegacyScheduler legacyScheduler = new LegacyScheduler( - LOG, + log, jobGraph, VoidBackPressureStatsTracker.INSTANCE, TestingUtils.defaultExecutor(), new Configuration(), - scheduler, + slotProvider, TestingUtils.defaultExecutor(), getClass().getClassLoader(), new StandaloneCheckpointRecoveryFactory(), @@ -209,31 +60,6 @@ public class LegacySchedulerBatchSchedulingTest extends TestLogger { NettyShuffleMaster.INSTANCE, NoOpJobMasterPartitionTracker.INSTANCE); - legacyScheduler.setMainThreadExecutor(mainThreadExecutor); - return legacyScheduler; } - - private JobGraph createJobGraph(int parallelism) { - final JobVertex jobVertex = new JobVertex("testing task"); - jobVertex.setParallelism(parallelism); - jobVertex.setInvokableClass(NoOpInvokable.class); - return new JobGraph(jobId, "test job", jobVertex); - } - - private static class GloballyTerminalJobStatusListener implements JobStatusListener { - - private final CompletableFuture globallyTerminalJobStatusFuture = new CompletableFuture<>(); - - @Override - public void jobStatusChanges(JobID jobId, JobStatus newJobStatus, long timestamp, Throwable error) { - if (newJobStatus.isGloballyTerminalState()) { - globallyTerminalJobStatusFuture.complete(newJobStatus); - } - } - - public CompletableFuture getTerminationFuture() { - return globallyTerminalJobStatusFuture; - } - } } -- Gitee From b3f8e33121b83628e2d20e12ebe84e269a05e7fe Mon Sep 17 00:00:00 2001 From: "shuai.xu" Date: Wed, 4 Mar 2020 15:50:25 +0800 Subject: [PATCH 035/885] [FLINK-16140] [docs-zh] Translate Event Processing (CEP) page into Chinese (#11168) --- docs/dev/libs/cep.zh.md | 800 +++++++++++++++++++--------------------- 1 file changed, 374 insertions(+), 426 deletions(-) diff --git a/docs/dev/libs/cep.zh.md b/docs/dev/libs/cep.zh.md index 9dcd7e6c6d..3805282d54 100644 --- a/docs/dev/libs/cep.zh.md +++ b/docs/dev/libs/cep.zh.md @@ -1,5 +1,5 @@ --- -title: "FlinkCEP - Flink 的复杂事件处理" +title: "FlinkCEP - Flink的复杂事件处理" nav-title: 事件处理 (CEP) nav-parent_id: libs nav-pos: 1 @@ -23,23 +23,20 @@ specific language governing permissions and limitations under the License. --> -FlinkCEP is the Complex Event Processing (CEP) library implemented on top of Flink. -It allows you to detect event patterns in an endless stream of events, giving you the opportunity to get hold of what's important in your -data. +FlinkCEP是在Flink上层实现的复杂事件处理库。 +它可以让你在无限事件流中检测出特定的事件模型,有机会掌握数据中重要的那部分。 -This page describes the API calls available in Flink CEP. We start by presenting the [Pattern API](#the-pattern-api), -which allows you to specify the patterns that you want to detect in your stream, before presenting how you can -[detect and act upon matching event sequences](#detecting-patterns). We then present the assumptions the CEP -library makes when [dealing with lateness](#handling-lateness-in-event-time) in event time and how you can -[migrate your job](#migrating-from-an-older-flink-versionpre-13) from an older Flink version to Flink-1.3. +本页讲述了Flink CEP中可用的API,我们首先讲述[模式API](#模式api),它可以让你指定想在数据流中检测的模式,然后讲述如何[检测匹配的事件序列并进行处理](#检测模式)。 +再然后我们讲述Flink在按照事件时间[处理迟到事件](#按照事件时间处理迟到事件)时的假设, +以及如何从旧版本的Flink向1.3之后的版本[迁移作业](#从旧版本迁移13之前)。 * This will be replaced by the TOC {:toc} -## Getting Started +## 开始 -If you want to jump right in, [set up a Flink program]({{ site.baseurl }}/dev/projectsetup/dependencies.html) and -add the FlinkCEP dependency to the `pom.xml` of your project. +如果你想现在开始尝试,[创建一个Flink程序]({{ site.baseurl }}/zh/dev/projectsetup/dependencies.html), +添加FlinkCEP的依赖到项目的`pom.xml`文件中。

    @@ -63,13 +60,12 @@ add the FlinkCEP dependency to the `pom.xml` of your project.
    -{% info %} FlinkCEP is not part of the binary distribution. See how to link with it for cluster execution [here]({{site.baseurl}}/dev/projectsetup/dependencies.html). +{% info 提示 %} FlinkCEP不是二进制发布包的一部分。在集群上执行如何链接它可以看[这里]({{site.baseurl}}/zh/dev/projectsetup/dependencies.html)。 -Now you can start writing your first CEP program using the Pattern API. +现在可以开始使用Pattern API写你的第一个CEP程序了。 -{% warn Attention %} The events in the `DataStream` to which -you want to apply pattern matching must implement proper `equals()` and `hashCode()` methods -because FlinkCEP uses them for comparing and matching events. +{% warn 注意 %} `DataStream`中的事件,如果你想在上面进行模式匹配的话,必须实现合适的 `equals()`和`hashCode()`方法, +因为FlinkCEP使用它们来比较和匹配事件。
    @@ -136,140 +132,143 @@ val result: DataStream[Alert] = patternStream.process(
    -## The Pattern API +## 模式API -The pattern API allows you to define complex pattern sequences that you want to extract from your input stream. +模式API可以让你定义想从输入流中抽取的复杂模式序列。 -Each complex pattern sequence consists of multiple simple patterns, i.e. patterns looking for individual events with the same properties. From now on, we will call these simple patterns **patterns**, and the final complex pattern sequence we are searching for in the stream, the **pattern sequence**. You can see a pattern sequence as a graph of such patterns, where transitions from one pattern to the next occur based on user-specified -*conditions*, e.g. `event.getName().equals("end")`. A **match** is a sequence of input events which visits all -patterns of the complex pattern graph, through a sequence of valid pattern transitions. +每个复杂的模式序列包括多个简单的模式,比如,寻找拥有相同属性事件序列的模式。从现在开始,我们把这些简单的模式称作**模式**, +把我们在数据流中最终寻找的复杂模式序列称作**模式序列**,你可以把模式序列看作是这样的模式构成的图, +这些模式基于用户指定的**条件**从一个转换到另外一个,比如 `event.getName().equals("end")`。 +一个**匹配**是输入事件的一个序列,这些事件通过一系列有效的模式转换,能够访问到复杂模式图中的所有模式。 -{% warn Attention %} Each pattern must have a unique name, which you use later to identify the matched events. +{% warn 注意 %} 每个模式必须有一个独一无二的名字,你可以在后面使用它来识别匹配到的事件。 -{% warn Attention %} Pattern names **CANNOT** contain the character `":"`. +{% warn 注意 %} 模式的名字不能包含字符`":"`. -In the rest of this section we will first describe how to define [Individual Patterns](#individual-patterns), and then how you can combine individual patterns into [Complex Patterns](#combining-patterns). +这一节的剩余部分我们会先讲述如何定义[单个模式](#单个模式),然后讲如何将单个模式组合成[复杂模式](#组合模式)。 -### Individual Patterns +### 单个模式 -A **Pattern** can be either a *singleton* or a *looping* pattern. Singleton patterns accept a single -event, while looping patterns can accept more than one. In pattern matching symbols, the pattern `"a b+ c? d"` (or `"a"`, followed by *one or more* `"b"`'s, optionally followed by a `"c"`, followed by a `"d"`), `a`, `c?`, and `d` are -singleton patterns, while `b+` is a looping one. By default, a pattern is a singleton pattern and you can transform -it to a looping one by using [Quantifiers](#quantifiers). Each pattern can have one or more -[Conditions](#conditions) based on which it accepts events. +一个**模式**可以是一个**单例**或者**循环**模式。单例模式只接受一个事件,循环模式可以接受多个事件。 +在模式匹配表达式中,模式`"a b+ c? d"`(或者`"a"`,后面跟着一个或者多个`"b"`,再往后可选择的跟着一个`"c"`,最后跟着一个`"d"`), +`a`,`c?`,和 `d`都是单例模式,`b+`是一个循环模式。默认情况下,模式都是单例的,你可以通过使用[量词](#量词)把它们转换成循环模式。 +每个模式可以有一个或者多个[条件](#条件)来决定它接受哪些事件。 -#### Quantifiers +#### 量词 -In FlinkCEP, you can specify looping patterns using these methods: `pattern.oneOrMore()`, for patterns that expect one or more occurrences of a given event (e.g. the `b+` mentioned before); and `pattern.times(#ofTimes)`, for patterns that -expect a specific number of occurrences of a given type of event, e.g. 4 `a`'s; and `pattern.times(#fromTimes, #toTimes)`, for patterns that expect a specific minimum number of occurrences and a maximum number of occurrences of a given type of event, e.g. 2-4 `a`s. +在FlinkCEP中,你可以通过这些方法指定循环模式:`pattern.oneOrMore()`,指定期望一个给定事件出现一次或者多次的模式(例如前面提到的`b+`模式); +`pattern.times(#ofTimes)`,指定期望一个给定事件出现特定次数的模式,例如出现4次`a`; +`pattern.times(#fromTimes, #toTimes)`,指定期望一个给定事件出现次数在一个最小值和最大值中间的模式,比如出现2-4次`a`。 -You can make looping patterns greedy using the `pattern.greedy()` method, but you cannot yet make group patterns greedy. You can make all patterns, looping or not, optional using the `pattern.optional()` method. +你可以使用`pattern.greedy()`方法让循环模式变成贪心的,但现在还不能让模式组贪心。 +你可以使用`pattern.optional()`方法让所有的模式变成可选的,不管是否是循环模式。 -For a pattern named `start`, the following are valid quantifiers: +对一个命名为`start`的模式,以下量词是有效的:
    {% highlight java %} -// expecting 4 occurrences +// 期望出现4次 start.times(4); -// expecting 0 or 4 occurrences +// 期望出现0或者4次 start.times(4).optional(); -// expecting 2, 3 or 4 occurrences +// 期望出现2、3或者4次 start.times(2, 4); -// expecting 2, 3 or 4 occurrences and repeating as many as possible +// 期望出现2、3或者4次,并且尽可能的重复次数多 start.times(2, 4).greedy(); -// expecting 0, 2, 3 or 4 occurrences +// 期望出现0、2、3或者4次 start.times(2, 4).optional(); -// expecting 0, 2, 3 or 4 occurrences and repeating as many as possible +// 期望出现0、2、3或者4次,并且尽可能的重复次数多 start.times(2, 4).optional().greedy(); -// expecting 1 or more occurrences +// 期望出现1到多次 start.oneOrMore(); -// expecting 1 or more occurrences and repeating as many as possible +// 期望出现1到多次,并且尽可能的重复次数多 start.oneOrMore().greedy(); -// expecting 0 or more occurrences +// 期望出现0到多次 start.oneOrMore().optional(); -// expecting 0 or more occurrences and repeating as many as possible +// 期望出现0到多次,并且尽可能的重复次数多 start.oneOrMore().optional().greedy(); -// expecting 2 or more occurrences +// 期望出现2到多次 start.timesOrMore(2); -// expecting 2 or more occurrences and repeating as many as possible +// 期望出现2到多次,并且尽可能的重复次数多 start.timesOrMore(2).greedy(); -// expecting 0, 2 or more occurrences and repeating as many as possible +// 期望出现0、2或多次 +start.timesOrMore(2).optional(); + +// 期望出现0、2或多次,并且尽可能的重复次数多 start.timesOrMore(2).optional().greedy(); {% endhighlight %}
    {% highlight scala %} -// expecting 4 occurrences +// 期望出现4次 start.times(4) -// expecting 0 or 4 occurrences +// 期望出现0或者4次 start.times(4).optional() -// expecting 2, 3 or 4 occurrences +// 期望出现2、3或者4次 start.times(2, 4) -// expecting 2, 3 or 4 occurrences and repeating as many as possible +// 期望出现2、3或者4次,并且尽可能的重复次数多 start.times(2, 4).greedy() -// expecting 0, 2, 3 or 4 occurrences +// 期望出现0、2、3或者4次 start.times(2, 4).optional() -// expecting 0, 2, 3 or 4 occurrences and repeating as many as possible +// 期望出现0、2、3或者4次,并且尽可能的重复次数多 start.times(2, 4).optional().greedy() -// expecting 1 or more occurrences +// 期望出现1到多次 start.oneOrMore() -// expecting 1 or more occurrences and repeating as many as possible +// 期望出现1到多次,并且尽可能的重复次数多 start.oneOrMore().greedy() -// expecting 0 or more occurrences +// 期望出现0到多次 start.oneOrMore().optional() -// expecting 0 or more occurrences and repeating as many as possible +// 期望出现0到多次,并且尽可能的重复次数多 start.oneOrMore().optional().greedy() -// expecting 2 or more occurrences +// 期望出现2到多次 start.timesOrMore(2) -// expecting 2 or more occurrences and repeating as many as possible +// 期望出现2到多次,并且尽可能的重复次数多 start.timesOrMore(2).greedy() -// expecting 0, 2 or more occurrences +// 期望出现0、2或多次 start.timesOrMore(2).optional() -// expecting 0, 2 or more occurrences and repeating as many as possible +// 期望出现0、2或多次,并且尽可能的重复次数多 start.timesOrMore(2).optional().greedy() {% endhighlight %}
    -#### Conditions +#### 条件 -For every pattern you can specify a condition that an incoming event has to meet in order to be "accepted" into the pattern e.g. its value should be larger than 5, -or larger than the average value of the previously accepted events. -You can specify conditions on the event properties via the `pattern.where()`, `pattern.or()` or `pattern.until()` methods. -These can be either `IterativeCondition`s or `SimpleCondition`s. +对每个模式你可以指定一个条件来决定一个进来的事件是否被接受进入这个模式,例如,它的value字段应该大于5,或者大于前面接受的事件的平均值。 +指定判断事件属性的条件可以通过`pattern.where()`、`pattern.or()`或者`pattern.until()`方法。 +这些可以是`IterativeCondition`或者`SimpleCondition`。 -**Iterative Conditions:** This is the most general type of condition. This is how you can specify a condition that -accepts subsequent events based on properties of the previously accepted events or a statistic over a subset of them. +**迭代条件:** 这是最普遍的条件类型。使用它可以指定一个基于前面已经被接受的事件的属性或者它们的一个子集的统计数据来决定是否接受时间序列的条件。 -Below is the code for an iterative condition that accepts the next event for a pattern named "middle" if its name starts -with "foo", and if the sum of the prices of the previously accepted events for that pattern plus the price of the current event do not exceed the value of 5.0. Iterative conditions can be powerful, especially in combination with looping patterns, e.g. `oneOrMore()`. +下面是一个迭代条件的代码,它接受"middle"模式下一个事件的名称开头是"foo", 并且前面已经匹配到的事件加上这个事件的价格小于5.0。 +迭代条件非常强大,尤其是跟循环模式结合使用时。
    @@ -307,14 +306,12 @@ middle.oneOrMore()
    -{% warn Attention %} The call to `ctx.getEventsForPattern(...)` finds all the -previously accepted events for a given potential match. The cost of this operation can vary, so when implementing -your condition, try to minimize its use. +{% warn 注意 %} 调用`ctx.getEventsForPattern(...)`可以获得所有前面已经接受作为可能匹配的事件。 +调用这个操作的代价可能很小也可能很大,所以在实现你的条件时,尽量少使用它。 -Described context gives one access to event time characteristics as well. For more info see [Time context](#time-context). +描述的上下文提供了获取事件时间属性的方法。更多细节可以看[时间上下文](#时间上下文)。 -**Simple Conditions:** This type of condition extends the aforementioned `IterativeCondition` class and decides -whether to accept an event or not, based *only* on properties of the event itself. +**简单条件:** 这种类型的条件扩展了前面提到的`IterativeCondition`类,它决定是否接受一个事件只取决于事件自身的属性。
    @@ -335,8 +332,7 @@ start.where(event => event.getName.startsWith("foo"))
    -Finally, you can also restrict the type of the accepted event to a subtype of the initial event type (here `Event`) -via the `pattern.subtype(subClass)` method. +最后,你可以通过`pattern.subtype(subClass)`方法限制接受的事件类型是初始事件的子类型。
    @@ -344,7 +340,7 @@ via the `pattern.subtype(subClass)` method. start.subtype(SubEvent.class).where(new SimpleCondition() { @Override public boolean filter(SubEvent value) { - return ... // some condition + return ... // 一些判断条件 } }); {% endhighlight %} @@ -352,12 +348,13 @@ start.subtype(SubEvent.class).where(new SimpleCondition() {
    {% highlight scala %} -start.subtype(classOf[SubEvent]).where(subEvent => ... /* some condition */) +start.subtype(classOf[SubEvent]).where(subEvent => ... /* 一些判断条件 */) {% endhighlight %}
    -**Combining Conditions:** As shown above, you can combine the `subtype` condition with additional conditions. This holds for every condition. You can arbitrarily combine conditions by sequentially calling `where()`. The final result will be the logical **AND** of the results of the individual conditions. To combine conditions using **OR**, you can use the `or()` method, as shown below. +**组合条件:** 如上所示,你可以把`subtype`条件和其他的条件结合起来使用。这适用于任何条件,你可以通过依次调用`where()`来组合条件。 +最终的结果是每个单一条件的结果的逻辑**AND**。如果想使用**OR**来组合条件,你可以像下面这样使用`or()`方法。
    @@ -365,12 +362,12 @@ start.subtype(classOf[SubEvent]).where(subEvent => ... /* some condition */) pattern.where(new SimpleCondition() { @Override public boolean filter(Event value) { - return ... // some condition + return ... // 一些判断条件 } }).or(new SimpleCondition() { @Override public boolean filter(Event value) { - return ... // or condition + return ... // 一些判断条件 } }); {% endhighlight %} @@ -378,45 +375,44 @@ pattern.where(new SimpleCondition() {
    {% highlight scala %} -pattern.where(event => ... /* some condition */).or(event => ... /* or condition */) +pattern.where(event => ... /* 一些判断条件 */).or(event => ... /* 一些判断条件 */) {% endhighlight %}
    -**Stop condition:** In case of looping patterns (`oneOrMore()` and `oneOrMore().optional()`) you can -also specify a stop condition, e.g. accept events with value larger than 5 until the sum of values is smaller than 50. +**停止条件:** 如果使用循环模式(`oneOrMore()`和`oneOrMore().optional()`),你可以指定一个停止条件,例如,接受事件的值大于5直到值的和小于50。 -To better understand it, have a look at the following example. Given +为了更好的理解它,看下面的例子。给定 -* pattern like `"(a+ until b)"` (one or more `"a"` until `"b"`) +* 模式如`"(a+ until b)"` (一个或者更多的`"a"`直到`"b"`) -* a sequence of incoming events `"a1" "c" "a2" "b" "a3"` +* 到来的事件序列`"a1" "c" "a2" "b" "a3"` -* the library will output results: `{a1 a2} {a1} {a2} {a3}`. +* 输出结果会是: `{a1 a2} {a1} {a2} {a3}`. -As you can see `{a1 a2 a3}` or `{a2 a3}` are not returned due to the stop condition. +你可以看到`{a1 a2 a3}`和`{a2 a3}`由于停止条件没有被输出。
    - - + + - + - +
    Pattern OperationDescription模式操作描述
    where(condition) -

    Defines a condition for the current pattern. To match the pattern, an event must satisfy the condition. - Multiple consecutive where() clauses lead to their conditions being ANDed:

    +

    为当前模式定义一个条件。为了匹配这个模式,一个事件必须满足某些条件。 + 多个连续的where()语句取与组成判断条件:

    {% highlight java %} pattern.where(new IterativeCondition() { @Override public boolean filter(Event value, Context ctx) throws Exception { - return ... // some condition + return ... // 一些判断条件 } }); {% endhighlight %} @@ -425,18 +421,17 @@ pattern.where(new IterativeCondition() {
    or(condition) -

    Adds a new condition which is ORed with an existing one. An event can match the pattern only if it - passes at least one of the conditions:

    +

    增加一个新的判断,和当前的判断取或。一个事件只要满足至少一个判断条件就匹配到模式:

    {% highlight java %} pattern.where(new IterativeCondition() { @Override public boolean filter(Event value, Context ctx) throws Exception { - return ... // some condition + return ... // 一些判断条件 } }).or(new IterativeCondition() { @Override public boolean filter(Event value, Context ctx) throws Exception { - return ... // alternative condition + return ... // 替代条件 } }); {% endhighlight %} @@ -445,15 +440,14 @@ pattern.where(new IterativeCondition() {
    until(condition) -

    Specifies a stop condition for a looping pattern. Meaning if event matching the given condition occurs, no more - events will be accepted into the pattern.

    -

    Applicable only in conjunction with oneOrMore()

    -

    NOTE: It allows for cleaning state for corresponding pattern on event-based condition.

    +

    为循环模式指定一个停止条件。意思是满足了给定的条件的事件出现后,就不会再有事件被接受进入模式了。

    +

    只适用于和oneOrMore()同时使用。

    +

    NOTE: 在基于事件的条件中,它可用于清理对应模式的状态。

    {% highlight java %} pattern.oneOrMore().until(new IterativeCondition() { @Override public boolean filter(Event value, Context ctx) throws Exception { - return ... // alternative condition + return ... // 替代条件 } }); {% endhighlight %} @@ -462,8 +456,7 @@ pattern.oneOrMore().until(new IterativeCondition() {
    subtype(subClass) -

    Defines a subtype condition for the current pattern. An event can only match the pattern if it is - of this subtype:

    +

    为当前模式定义一个子类型条件。一个事件只有是这个子类型的时候才能匹配到模式:

    {% highlight java %} pattern.subtype(SubEvent.class); {% endhighlight %} @@ -472,10 +465,10 @@ pattern.subtype(SubEvent.class);
    oneOrMore() -

    Specifies that this pattern expects at least one occurrence of a matching event.

    -

    By default a relaxed internal contiguity (between subsequent events) is used. For more info on - internal contiguity see consecutive.

    -

    NOTE: It is advised to use either until() or within() to enable state clearing

    +

    指定模式期望匹配到的事件至少出现一次。.

    +

    默认(在子事件间)使用松散的内部连续性。 + 关于内部连续性的更多信息可以参考连续性

    +

    NOTE: 推荐使用until()或者within()来清理状态。

    {% highlight java %} pattern.oneOrMore(); {% endhighlight %} @@ -484,10 +477,9 @@ pattern.oneOrMore();
    timesOrMore(#times) -

    Specifies that this pattern expects at least #times occurrences - of a matching event.

    -

    By default a relaxed internal contiguity (between subsequent events) is used. For more info on - internal contiguity see consecutive.

    +

    指定模式期望匹配到的事件至少出现#times次。.

    +

    默认(在子事件间)使用松散的内部连续性。 + 关于内部连续性的更多信息可以参考连续性

    {% highlight java %} pattern.timesOrMore(2); {% endhighlight %} @@ -496,9 +488,9 @@ pattern.timesOrMore(2);
    times(#ofTimes) -

    Specifies that this pattern expects an exact number of occurrences of a matching event.

    -

    By default a relaxed internal contiguity (between subsequent events) is used. For more info on - internal contiguity see consecutive.

    +

    指定模式期望匹配到的事件正好出现的次数。

    +

    默认(在子事件间)使用松散的内部连续性。 + 关于内部连续性的更多信息可以参考连续性

    {% highlight java %} pattern.times(2); {% endhighlight %} @@ -507,10 +499,9 @@ pattern.times(2);
    times(#fromTimes, #toTimes) -

    Specifies that this pattern expects occurrences between #fromTimes - and #toTimes of a matching event.

    -

    By default a relaxed internal contiguity (between subsequent events) is used. For more info on - internal contiguity see consecutive.

    +

    指定模式期望匹配到的事件出现次数在#fromTimes#toTimes之间。

    +

    默认(在子事件间)使用松散的内部连续性。 + 关于内部连续性的更多信息可以参考连续性

    {% highlight java %} pattern.times(2, 4); {% endhighlight %} @@ -519,8 +510,7 @@ pattern.times(2, 4);
    optional() -

    Specifies that this pattern is optional, i.e. it may not occur at all. This is applicable to all - aforementioned quantifiers.

    +

    指定这个模式是可选的,也就是说,它可能根本不出现。这对所有之前提到的量词都适用。

    {% highlight java %} pattern.oneOrMore().optional(); {% endhighlight %} @@ -529,8 +519,7 @@ pattern.oneOrMore().optional();
    greedy() -

    Specifies that this pattern is greedy, i.e. it will repeat as many as possible. This is only applicable - to quantifiers and it does not support group pattern currently.

    +

    指定这个模式是贪心的,也就是说,它会重复尽可能多的次数。这只对量词适用,现在还不支持模式组。

    {% highlight java %} pattern.oneOrMore().greedy(); {% endhighlight %} @@ -544,8 +533,8 @@ pattern.oneOrMore().greedy(); - - + + @@ -553,41 +542,38 @@ pattern.oneOrMore().greedy(); - - + + + + + + + diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java index 0ddb84801d..e91f0632d9 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java @@ -147,6 +147,13 @@ public class KubernetesConfigOptions { .defaultValue("/opt/flink/log") .withDescription("The directory that logs of jobmanager and taskmanager be saved in the pod."); + public static final ConfigOption HADOOP_CONF_CONFIG_MAP = + key("kubernetes.hadoop.conf.config-map.name") + .stringType() + .noDefaultValue() + .withDescription("Specify the name of an existing ConfigMap that contains custom Hadoop configuration " + + "to be mounted on the JobManager(s) and TaskManagers."); + /** * The flink rest service exposed type. */ diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/FlinkConfMountDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/FlinkConfMountDecorator.java index 2f5a329e02..07cf85e5a3 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/FlinkConfMountDecorator.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/FlinkConfMountDecorator.java @@ -183,7 +183,7 @@ public class FlinkConfMountDecorator extends AbstractKubernetesStepDecorator { } @VisibleForTesting - String getFlinkConfConfigMapName(String clusterId) { + public static String getFlinkConfConfigMapName(String clusterId) { return CONFIG_MAP_PREFIX + clusterId; } } diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/HadoopConfMountDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/HadoopConfMountDecorator.java new file mode 100644 index 0000000000..896fd5c554 --- /dev/null +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/HadoopConfMountDecorator.java @@ -0,0 +1,186 @@ +/* + * 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.kubernetes.kubeclient.decorators; + +import org.apache.flink.kubernetes.kubeclient.FlinkPod; +import org.apache.flink.kubernetes.kubeclient.parameters.AbstractKubernetesParameters; +import org.apache.flink.kubernetes.utils.Constants; +import org.apache.flink.util.FileUtils; + +import io.fabric8.kubernetes.api.model.ConfigMap; +import io.fabric8.kubernetes.api.model.ConfigMapBuilder; +import io.fabric8.kubernetes.api.model.Container; +import io.fabric8.kubernetes.api.model.ContainerBuilder; +import io.fabric8.kubernetes.api.model.HasMetadata; +import io.fabric8.kubernetes.api.model.KeyToPath; +import io.fabric8.kubernetes.api.model.KeyToPathBuilder; +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.PodBuilder; +import io.fabric8.kubernetes.api.model.Volume; +import io.fabric8.kubernetes.api.model.VolumeBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Mount the custom Hadoop Configuration to the JobManager(s)/TaskManagers. We provide two options: + * 1. Mount an existing ConfigMap containing custom Hadoop Configuration. + * 2. Create and mount a dedicated ConfigMap containing the custom Hadoop configuration from a local directory + * specified via the HADOOP_CONF_DIR or HADOOP_HOME environment variable. + */ +public class HadoopConfMountDecorator extends AbstractKubernetesStepDecorator { + + private static final Logger LOG = LoggerFactory.getLogger(HadoopConfMountDecorator.class); + + private final AbstractKubernetesParameters kubernetesParameters; + + public HadoopConfMountDecorator(AbstractKubernetesParameters kubernetesParameters) { + this.kubernetesParameters = checkNotNull(kubernetesParameters); + } + + @Override + public FlinkPod decorateFlinkPod(FlinkPod flinkPod) { + Volume hadoopConfVolume; + + final Optional existingConfigMap = kubernetesParameters.getExistingHadoopConfigurationConfigMap(); + if (existingConfigMap.isPresent()) { + hadoopConfVolume = new VolumeBuilder() + .withName(Constants.HADOOP_CONF_VOLUME) + .withNewConfigMap() + .withName(existingConfigMap.get()) + .endConfigMap() + .build(); + } else { + final Optional localHadoopConfigurationDirectory = kubernetesParameters.getLocalHadoopConfigurationDirectory(); + if (!localHadoopConfigurationDirectory.isPresent()) { + return flinkPod; + } + + final List hadoopConfigurationFileItems = getHadoopConfigurationFileItems(localHadoopConfigurationDirectory.get()); + if (hadoopConfigurationFileItems.isEmpty()) { + LOG.warn("Found 0 files in directory {}, skip to mount the Hadoop Configuration ConfigMap.", + localHadoopConfigurationDirectory.get()); + return flinkPod; + } + + final List keyToPaths = hadoopConfigurationFileItems.stream() + .map(file -> new KeyToPathBuilder() + .withKey(file.getName()) + .withPath(file.getName()) + .build()) + .collect(Collectors.toList()); + + hadoopConfVolume = new VolumeBuilder() + .withName(Constants.HADOOP_CONF_VOLUME) + .withNewConfigMap() + .withName(getHadoopConfConfigMapName(kubernetesParameters.getClusterId())) + .withItems(keyToPaths) + .endConfigMap() + .build(); + } + + final Pod podWithHadoopConf = new PodBuilder(flinkPod.getPod()) + .editOrNewSpec() + .addNewVolumeLike(hadoopConfVolume) + .endVolume() + .endSpec() + .build(); + + final Container containerWithHadoopConf = new ContainerBuilder(flinkPod.getMainContainer()) + .addNewVolumeMount() + .withName(Constants.HADOOP_CONF_VOLUME) + .withMountPath(Constants.HADOOP_CONF_DIR_IN_POD) + .endVolumeMount() + .addNewEnv() + .withName(Constants.ENV_HADOOP_CONF_DIR) + .withValue(Constants.HADOOP_CONF_DIR_IN_POD) + .endEnv() + .build(); + + return new FlinkPod.Builder(flinkPod) + .withPod(podWithHadoopConf) + .withMainContainer(containerWithHadoopConf) + .build(); + } + + @Override + public List buildAccompanyingKubernetesResources() throws IOException { + if (kubernetesParameters.getExistingHadoopConfigurationConfigMap().isPresent()) { + return Collections.emptyList(); + } + + final Optional localHadoopConfigurationDirectory = kubernetesParameters.getLocalHadoopConfigurationDirectory(); + if (!localHadoopConfigurationDirectory.isPresent()) { + return Collections.emptyList(); + } + + final List hadoopConfigurationFileItems = getHadoopConfigurationFileItems(localHadoopConfigurationDirectory.get()); + if (hadoopConfigurationFileItems.isEmpty()) { + LOG.warn("Found 0 files in directory {}, skip to create the Hadoop Configuration ConfigMap.", localHadoopConfigurationDirectory.get()); + return Collections.emptyList(); + } + + final Map data = new HashMap<>(); + for (File file: hadoopConfigurationFileItems) { + data.put(file.getName(), FileUtils.readFileUtf8(file)); + } + + final ConfigMap hadoopConfigMap = new ConfigMapBuilder() + .withApiVersion(Constants.API_VERSION) + .withNewMetadata() + .withName(getHadoopConfConfigMapName(kubernetesParameters.getClusterId())) + .withLabels(kubernetesParameters.getCommonLabels()) + .endMetadata() + .addToData(data) + .build(); + + return Collections.singletonList(hadoopConfigMap); + } + + private List getHadoopConfigurationFileItems(String localHadoopConfigurationDirectory) { + final List expectedFileNames = new ArrayList<>(); + expectedFileNames.add("core-site.xml"); + expectedFileNames.add("hdfs-site.xml"); + + final File directory = new File(localHadoopConfigurationDirectory); + if (directory.exists() && directory.isDirectory()) { + return Arrays.stream(directory.listFiles()) + .filter(file -> file.isFile() && expectedFileNames.stream().anyMatch(name -> file.getName().equals(name))) + .collect(Collectors.toList()); + } else { + return Collections.emptyList(); + } + } + + public static String getHadoopConfConfigMapName(String clusterId) { + return Constants.HADOOP_CONF_CONFIG_MAP_PREFIX + clusterId; + } +} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactory.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactory.java index f1961915f9..32977a2f5b 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactory.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactory.java @@ -22,6 +22,7 @@ import org.apache.flink.kubernetes.kubeclient.FlinkPod; import org.apache.flink.kubernetes.kubeclient.KubernetesJobManagerSpecification; import org.apache.flink.kubernetes.kubeclient.decorators.ExternalServiceDecorator; import org.apache.flink.kubernetes.kubeclient.decorators.FlinkConfMountDecorator; +import org.apache.flink.kubernetes.kubeclient.decorators.HadoopConfMountDecorator; import org.apache.flink.kubernetes.kubeclient.decorators.InitJobManagerDecorator; import org.apache.flink.kubernetes.kubeclient.decorators.InternalServiceDecorator; import org.apache.flink.kubernetes.kubeclient.decorators.JavaCmdJobManagerDecorator; @@ -58,6 +59,7 @@ public class KubernetesJobManagerFactory { new JavaCmdJobManagerDecorator(kubernetesJobManagerParameters), new InternalServiceDecorator(kubernetesJobManagerParameters), new ExternalServiceDecorator(kubernetesJobManagerParameters), + new HadoopConfMountDecorator(kubernetesJobManagerParameters), new FlinkConfMountDecorator(kubernetesJobManagerParameters)}; for (KubernetesStepDecorator stepDecorator: stepDecorators) { diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesTaskManagerFactory.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesTaskManagerFactory.java index 60c2d8846a..fdfc0fa3d2 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesTaskManagerFactory.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesTaskManagerFactory.java @@ -20,6 +20,7 @@ package org.apache.flink.kubernetes.kubeclient.factory; import org.apache.flink.kubernetes.kubeclient.FlinkPod; import org.apache.flink.kubernetes.kubeclient.decorators.FlinkConfMountDecorator; +import org.apache.flink.kubernetes.kubeclient.decorators.HadoopConfMountDecorator; import org.apache.flink.kubernetes.kubeclient.decorators.InitTaskManagerDecorator; import org.apache.flink.kubernetes.kubeclient.decorators.JavaCmdTaskManagerDecorator; import org.apache.flink.kubernetes.kubeclient.decorators.KubernetesStepDecorator; @@ -40,6 +41,7 @@ public class KubernetesTaskManagerFactory { final KubernetesStepDecorator[] stepDecorators = new KubernetesStepDecorator[] { new InitTaskManagerDecorator(kubernetesTaskManagerParameters), new JavaCmdTaskManagerDecorator(kubernetesTaskManagerParameters), + new HadoopConfMountDecorator(kubernetesTaskManagerParameters), new FlinkConfMountDecorator(kubernetesTaskManagerParameters)}; for (KubernetesStepDecorator stepDecorator: stepDecorators) { diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/AbstractKubernetesParameters.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/AbstractKubernetesParameters.java index 5f94a55872..fc4ff48a95 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/AbstractKubernetesParameters.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/AbstractKubernetesParameters.java @@ -25,11 +25,13 @@ import org.apache.flink.kubernetes.utils.Constants; import org.apache.flink.runtime.clusterframework.BootstrapTools; import io.fabric8.kubernetes.api.model.LocalObjectReference; +import org.apache.commons.lang3.StringUtils; import java.io.File; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import static org.apache.flink.kubernetes.configuration.KubernetesConfigOptions.CONTAINER_IMAGE_PULL_SECRETS; import static org.apache.flink.kubernetes.utils.Constants.CONFIG_FILE_LOG4J_NAME; @@ -134,6 +136,33 @@ public abstract class AbstractKubernetesParameters implements KubernetesParamete return log4jFile.exists(); } + @Override + public Optional getExistingHadoopConfigurationConfigMap() { + final String existingHadoopConfigMap = flinkConfig.getString(KubernetesConfigOptions.HADOOP_CONF_CONFIG_MAP); + if (StringUtils.isBlank(existingHadoopConfigMap)) { + return Optional.empty(); + } else { + return Optional.of(existingHadoopConfigMap.trim()); + } + } + + @Override + public Optional getLocalHadoopConfigurationDirectory() { + final String[] possibleHadoopConfPaths = new String[] { + System.getenv(Constants.ENV_HADOOP_CONF_DIR), + System.getenv(Constants.ENV_HADOOP_HOME) + "/etc/hadoop", // hadoop 2.2 + System.getenv(Constants.ENV_HADOOP_HOME) + "/conf" + }; + + for (String hadoopConfPath: possibleHadoopConfPaths) { + if (StringUtils.isNotBlank(hadoopConfPath)) { + return Optional.of(hadoopConfPath); + } + } + + return Optional.empty(); + } + /** * Extract container customized environment variable properties with a given name prefix. * @param envPrefix the given property name prefix diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesParameters.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesParameters.java index 00163412f4..f18f067733 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesParameters.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesParameters.java @@ -23,6 +23,7 @@ import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; import io.fabric8.kubernetes.api.model.LocalObjectReference; import java.util.Map; +import java.util.Optional; /** * A common collection of parameters that is used to construct the JobManager/TaskManager Pods, @@ -80,4 +81,14 @@ public interface KubernetesParameters { * Whether the log4j.properties is located. */ boolean hasLog4j(); + + /** + * The existing ConfigMap containing custom Hadoop configuration. + */ + Optional getExistingHadoopConfigurationConfigMap(); + + /** + * The local directory to locate the custom Hadoop configuration. + */ + Optional getLocalHadoopConfigurationDirectory(); } diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/Constants.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/Constants.java index 5338431f0f..65a078f7d2 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/Constants.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/Constants.java @@ -28,13 +28,17 @@ public class Constants { public static final String APPS_API_VERSION = "apps/v1"; public static final String CONFIG_FILE_LOGBACK_NAME = "logback.xml"; - public static final String CONFIG_FILE_LOG4J_NAME = "log4j.properties"; public static final String FLINK_CONF_VOLUME = "flink-config-volume"; - public static final String CONFIG_MAP_PREFIX = "flink-config-"; + public static final String HADOOP_CONF_VOLUME = "hadoop-config-volume"; + public static final String HADOOP_CONF_CONFIG_MAP_PREFIX = "hadoop-config-"; + public static final String HADOOP_CONF_DIR_IN_POD = "/opt/hadoop/conf"; + public static final String ENV_HADOOP_CONF_DIR = "HADOOP_CONF_DIR"; + public static final String ENV_HADOOP_HOME = "HADOOP_HOME"; + public static final String FLINK_REST_SERVICE_SUFFIX = "-rest"; public static final String NAME_SEPARATOR = "-"; diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesTestBase.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesTestBase.java index d53330bc3a..3c3441253b 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesTestBase.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesTestBase.java @@ -20,6 +20,7 @@ package org.apache.flink.kubernetes; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.testutils.CommonTestUtils; import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; import org.apache.flink.kubernetes.kubeclient.Fabric8FlinkKubeClient; import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient; @@ -57,6 +58,8 @@ public class KubernetesTestBase extends TestLogger { protected File flinkConfDir; + protected File hadoopConfDir; + protected final Configuration flinkConfig = new Configuration(); protected KubernetesClient kubeClient; @@ -71,6 +74,8 @@ public class KubernetesTestBase extends TestLogger { flinkConfig.set(KubernetesConfigOptions.CONTAINER_IMAGE_PULL_POLICY, CONTAINER_IMAGE_PULL_POLICY); flinkConfDir = temporaryFolder.newFolder().getAbsoluteFile(); + hadoopConfDir = temporaryFolder.newFolder().getAbsoluteFile(); + writeFlinkConfiguration(); Map map = new HashMap<>(); @@ -91,4 +96,15 @@ public class KubernetesTestBase extends TestLogger { labels.put(Constants.LABEL_APP_KEY, CLUSTER_ID); return labels; } + + protected void setHadoopConfDirEnv() { + Map map = new HashMap<>(); + map.put(Constants.ENV_HADOOP_CONF_DIR, hadoopConfDir.toString()); + CommonTestUtils.setEnv(map, false); + } + + protected void generateHadoopConfFileItems() throws IOException { + KubernetesTestUtils.createTemporyFile("some data", hadoopConfDir, "core-site.xml"); + KubernetesTestUtils.createTemporyFile("some data", hadoopConfDir, "hdfs-site.xml"); + } } diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/HadoopConfMountDecoratorTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/HadoopConfMountDecoratorTest.java new file mode 100644 index 0000000000..8c7517908e --- /dev/null +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/HadoopConfMountDecoratorTest.java @@ -0,0 +1,178 @@ +/* + * 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.kubernetes.kubeclient.decorators; + +import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; +import org.apache.flink.kubernetes.kubeclient.FlinkPod; +import org.apache.flink.kubernetes.kubeclient.KubernetesJobManagerTestBase; +import org.apache.flink.kubernetes.utils.Constants; + +import io.fabric8.kubernetes.api.model.ConfigMap; +import io.fabric8.kubernetes.api.model.ConfigMapVolumeSource; +import io.fabric8.kubernetes.api.model.EnvVar; +import io.fabric8.kubernetes.api.model.HasMetadata; +import io.fabric8.kubernetes.api.model.KeyToPath; +import io.fabric8.kubernetes.api.model.Volume; +import io.fabric8.kubernetes.api.model.VolumeMount; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +/** + * General tests for the {@link HadoopConfMountDecorator}. + */ +public class HadoopConfMountDecoratorTest extends KubernetesJobManagerTestBase { + + private static final String EXISTING_HADOOP_CONF_CONFIG_MAP = "hadoop-conf"; + + private HadoopConfMountDecorator hadoopConfMountDecorator; + + @Before + public void setup() throws Exception { + super.setup(); + this.hadoopConfMountDecorator = new HadoopConfMountDecorator(kubernetesJobManagerParameters); + } + + @Test + public void testExistingHadoopConfigMap() throws IOException { + flinkConfig.set(KubernetesConfigOptions.HADOOP_CONF_CONFIG_MAP, EXISTING_HADOOP_CONF_CONFIG_MAP); + assertEquals(0, hadoopConfMountDecorator.buildAccompanyingKubernetesResources().size()); + + final FlinkPod resultFlinkPod = hadoopConfMountDecorator.decorateFlinkPod(baseFlinkPod); + final List volumes = resultFlinkPod.getPod().getSpec().getVolumes(); + assertTrue(volumes.stream().anyMatch(volume -> volume.getConfigMap().getName().equals(EXISTING_HADOOP_CONF_CONFIG_MAP))); + } + + @Test + public void testExistingConfigMapPrecedeOverHadoopConfEnv() throws IOException { + // set existing ConfigMap + flinkConfig.set(KubernetesConfigOptions.HADOOP_CONF_CONFIG_MAP, EXISTING_HADOOP_CONF_CONFIG_MAP); + + // set HADOOP_CONF_DIR + setHadoopConfDirEnv(); + generateHadoopConfFileItems(); + + assertEquals(0, hadoopConfMountDecorator.buildAccompanyingKubernetesResources().size()); + + final FlinkPod resultFlinkPod = hadoopConfMountDecorator.decorateFlinkPod(baseFlinkPod); + final List volumes = resultFlinkPod.getPod().getSpec().getVolumes(); + assertTrue(volumes.stream().anyMatch(volume -> + volume.getConfigMap().getName().equals(EXISTING_HADOOP_CONF_CONFIG_MAP))); + assertFalse(volumes.stream().anyMatch(volume -> + volume.getConfigMap().getName().equals(HadoopConfMountDecorator.getHadoopConfConfigMapName(CLUSTER_ID)))); + } + + @Test + public void testHadoopConfDirectoryUnset() throws IOException { + assertEquals(0, hadoopConfMountDecorator.buildAccompanyingKubernetesResources().size()); + + final FlinkPod resultFlinkPod = hadoopConfMountDecorator.decorateFlinkPod(baseFlinkPod); + assertEquals(baseFlinkPod.getPod(), resultFlinkPod.getPod()); + assertEquals(baseFlinkPod.getMainContainer(), resultFlinkPod.getMainContainer()); + } + + @Test + public void testEmptyHadoopConfDirectory() throws IOException { + setHadoopConfDirEnv(); + + assertEquals(0, hadoopConfMountDecorator.buildAccompanyingKubernetesResources().size()); + + final FlinkPod resultFlinkPod = hadoopConfMountDecorator.decorateFlinkPod(baseFlinkPod); + assertEquals(baseFlinkPod.getPod(), resultFlinkPod.getPod()); + assertEquals(baseFlinkPod.getMainContainer(), resultFlinkPod.getMainContainer()); + } + + @Test + public void testHadoopConfConfigMap() throws IOException { + setHadoopConfDirEnv(); + generateHadoopConfFileItems(); + + final List additionalResources = hadoopConfMountDecorator.buildAccompanyingKubernetesResources(); + assertEquals(1, additionalResources.size()); + + final ConfigMap resultConfigMap = (ConfigMap) additionalResources.get(0); + + assertEquals(Constants.API_VERSION, resultConfigMap.getApiVersion()); + assertEquals(HadoopConfMountDecorator.getHadoopConfConfigMapName(CLUSTER_ID), + resultConfigMap.getMetadata().getName()); + assertEquals(getCommonLabels(), resultConfigMap.getMetadata().getLabels()); + + Map resultDatas = resultConfigMap.getData(); + assertEquals("some data", resultDatas.get("core-site.xml")); + assertEquals("some data", resultDatas.get("hdfs-site.xml")); + } + + @Test + public void testPodWithHadoopConfVolume() throws IOException { + setHadoopConfDirEnv(); + generateHadoopConfFileItems(); + final FlinkPod resultFlinkPod = hadoopConfMountDecorator.decorateFlinkPod(baseFlinkPod); + + final List resultVolumes = resultFlinkPod.getPod().getSpec().getVolumes(); + assertEquals(1, resultVolumes.size()); + + final Volume resultVolume = resultVolumes.get(0); + assertEquals(Constants.HADOOP_CONF_VOLUME, resultVolume.getName()); + + final ConfigMapVolumeSource resultVolumeConfigMap = resultVolume.getConfigMap(); + assertEquals(HadoopConfMountDecorator.getHadoopConfConfigMapName(CLUSTER_ID), + resultVolumeConfigMap.getName()); + + final Map expectedKeyToPaths = new HashMap() { + { + put("hdfs-site.xml", "hdfs-site.xml"); + put("core-site.xml", "core-site.xml"); + } + }; + final Map resultKeyToPaths = resultVolumeConfigMap.getItems().stream() + .collect(Collectors.toMap(KeyToPath::getKey, KeyToPath::getPath)); + assertEquals(expectedKeyToPaths, resultKeyToPaths); + } + + @Test + public void testMainContainerWithHadoopConfVolumeMount() throws IOException { + setHadoopConfDirEnv(); + generateHadoopConfFileItems(); + final FlinkPod resultFlinkPod = hadoopConfMountDecorator.decorateFlinkPod(baseFlinkPod); + + final List resultVolumeMounts = resultFlinkPod.getMainContainer().getVolumeMounts(); + assertEquals(1, resultVolumeMounts.size()); + final VolumeMount resultVolumeMount = resultVolumeMounts.get(0); + assertEquals(Constants.HADOOP_CONF_VOLUME, resultVolumeMount.getName()); + assertEquals(Constants.HADOOP_CONF_DIR_IN_POD, resultVolumeMount.getMountPath()); + + final Map expectedEnvs = new HashMap() { + { + put(Constants.ENV_HADOOP_CONF_DIR, Constants.HADOOP_CONF_DIR_IN_POD); + } + }; + final Map resultEnvs = resultFlinkPod.getMainContainer().getEnv() + .stream().collect(Collectors.toMap(EnvVar::getName, EnvVar::getValue)); + assertEquals(expectedEnvs, resultEnvs); + } +} diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactoryTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactoryTest.java index 7b17c9fd44..b179ddc61a 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactoryTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactoryTest.java @@ -24,6 +24,8 @@ import org.apache.flink.kubernetes.configuration.KubernetesConfigOptionsInternal import org.apache.flink.kubernetes.entrypoint.KubernetesSessionClusterEntrypoint; import org.apache.flink.kubernetes.kubeclient.KubernetesJobManagerSpecification; import org.apache.flink.kubernetes.kubeclient.KubernetesJobManagerTestBase; +import org.apache.flink.kubernetes.kubeclient.decorators.FlinkConfMountDecorator; +import org.apache.flink.kubernetes.kubeclient.decorators.HadoopConfMountDecorator; import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters; import org.apache.flink.kubernetes.utils.Constants; import org.apache.flink.kubernetes.utils.KubernetesUtils; @@ -39,6 +41,7 @@ import io.fabric8.kubernetes.api.model.apps.DeploymentSpec; import org.junit.Before; import org.junit.Test; +import java.io.IOException; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -46,6 +49,7 @@ import java.util.stream.Collectors; import static org.apache.flink.configuration.GlobalConfiguration.FLINK_CONF_FILENAME; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; @@ -57,7 +61,9 @@ public class KubernetesJobManagerFactoryTest extends KubernetesJobManagerTestBas private static final String SERVICE_ACCOUNT_NAME = "service-test"; private static final String ENTRY_POINT_CLASS = KubernetesSessionClusterEntrypoint.class.getCanonicalName(); - private KubernetesJobManagerSpecification kubernetesJobManagerSpecification; + private static final String EXISTING_HADOOP_CONF_CONFIG_MAP = "hadoop-conf"; + + protected KubernetesJobManagerSpecification kubernetesJobManagerSpecification; @Before public void setup() throws Exception { @@ -187,7 +193,8 @@ public class KubernetesJobManagerFactoryTest extends KubernetesJobManagerTestBas public void testFlinkConfConfigMap() { final ConfigMap resultConfigMap = (ConfigMap) this.kubernetesJobManagerSpecification.getAccompanyingResources() .stream() - .filter(x -> x instanceof ConfigMap) + .filter(x -> x instanceof ConfigMap && + x.getMetadata().getName().equals(FlinkConfMountDecorator.getFlinkConfConfigMapName(CLUSTER_ID))) .collect(Collectors.toList()) .get(0); @@ -200,4 +207,46 @@ public class KubernetesJobManagerFactoryTest extends KubernetesJobManagerTestBas assertTrue(resultDatas.get(FLINK_CONF_FILENAME) .contains(KubernetesConfigOptionsInternal.ENTRY_POINT_CLASS.key() + ": " + ENTRY_POINT_CLASS)); } + + @Test + public void testExistingHadoopConfigMap() throws IOException { + flinkConfig.set(KubernetesConfigOptions.HADOOP_CONF_CONFIG_MAP, EXISTING_HADOOP_CONF_CONFIG_MAP); + kubernetesJobManagerSpecification = KubernetesJobManagerFactory.createJobManagerComponent(kubernetesJobManagerParameters); + + assertFalse(kubernetesJobManagerSpecification.getAccompanyingResources().stream() + .anyMatch(resource -> resource.getMetadata().getName().equals(HadoopConfMountDecorator.getHadoopConfConfigMapName(CLUSTER_ID)))); + + final PodSpec podSpec = kubernetesJobManagerSpecification.getDeployment().getSpec().getTemplate().getSpec(); + assertTrue(podSpec.getVolumes().stream().anyMatch(volume -> volume.getConfigMap().getName().equals(EXISTING_HADOOP_CONF_CONFIG_MAP))); + } + + @Test + public void testHadoopConfConfigMap() throws IOException { + setHadoopConfDirEnv(); + generateHadoopConfFileItems(); + kubernetesJobManagerSpecification = KubernetesJobManagerFactory.createJobManagerComponent(kubernetesJobManagerParameters); + + final ConfigMap resultConfigMap = (ConfigMap) kubernetesJobManagerSpecification.getAccompanyingResources() + .stream() + .filter(x -> x instanceof ConfigMap && + x.getMetadata().getName().equals(HadoopConfMountDecorator.getHadoopConfConfigMapName(CLUSTER_ID))) + .collect(Collectors.toList()) + .get(0); + + assertEquals(2, resultConfigMap.getMetadata().getLabels().size()); + + final Map resultDatas = resultConfigMap.getData(); + assertEquals(2, resultDatas.size()); + assertEquals("some data", resultDatas.get("core-site.xml")); + assertEquals("some data", resultDatas.get("hdfs-site.xml")); + } + + @Test + public void testEmptyHadoopConfDirectory() throws IOException { + setHadoopConfDirEnv(); + kubernetesJobManagerSpecification = KubernetesJobManagerFactory.createJobManagerComponent(kubernetesJobManagerParameters); + + assertFalse(kubernetesJobManagerSpecification.getAccompanyingResources().stream() + .anyMatch(resource -> resource.getMetadata().getName().equals(HadoopConfMountDecorator.getHadoopConfConfigMapName(CLUSTER_ID)))); + } } diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesTaskManagerFactoryTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesTaskManagerFactoryTest.java index 14f6c48eb0..642b641100 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesTaskManagerFactoryTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesTaskManagerFactoryTest.java @@ -30,6 +30,7 @@ import org.junit.Test; import java.util.List; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; /** * General tests for the {@link KubernetesTaskManagerFactory}. @@ -45,6 +46,9 @@ public class KubernetesTaskManagerFactoryTest extends KubernetesTaskManagerTestB KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "logback.xml"); KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "log4j.properties"); + setHadoopConfDirEnv(); + generateHadoopConfFileItems(); + this.resultPod = KubernetesTaskManagerFactory.createTaskManagerComponent(kubernetesTaskManagerParameters).getInternalResource(); } @@ -53,7 +57,7 @@ public class KubernetesTaskManagerFactoryTest extends KubernetesTaskManagerTestB public void testPod() { assertEquals(POD_NAME, this.resultPod.getMetadata().getName()); assertEquals(3, this.resultPod.getMetadata().getLabels().size()); - assertEquals(1, this.resultPod.getSpec().getVolumes().size()); + assertEquals(2, this.resultPod.getSpec().getVolumes().size()); } @Test @@ -67,9 +71,15 @@ public class KubernetesTaskManagerFactoryTest extends KubernetesTaskManagerTestB resultMainContainer.getName()); assertEquals(CONTAINER_IMAGE, resultMainContainer.getImage()); assertEquals(CONTAINER_IMAGE_PULL_POLICY.name(), resultMainContainer.getImagePullPolicy()); + + assertEquals(4, resultMainContainer.getEnv().size()); + assertTrue(resultMainContainer.getEnv() + .stream() + .anyMatch(envVar -> envVar.getName().equals("key1"))); + assertEquals(1, resultMainContainer.getPorts().size()); assertEquals(1, resultMainContainer.getCommand().size()); assertEquals(3, resultMainContainer.getArgs().size()); - assertEquals(1, resultMainContainer.getVolumeMounts().size()); + assertEquals(2, resultMainContainer.getVolumeMounts().size()); } } -- Gitee From 6049b8307482a1d2d67d9642675345fc5093c197 Mon Sep 17 00:00:00 2001 From: felixzheng Date: Sat, 21 Mar 2020 20:25:04 +0800 Subject: [PATCH 239/885] [FLINK-16547][yarn] Respect the config option of FileJobGraphRetriever#JOB_GRAPH_FILE_PATH This closes #11472. --- .../flink/yarn/YarnClusterDescriptor.java | 72 +++++++++---------- 1 file changed, 36 insertions(+), 36 deletions(-) diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java index f3c6f3c7fe..4d6f69140d 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java @@ -694,7 +694,7 @@ public class YarnClusterDescriptor implements ClusterDescriptor { : jobGraph.getUserJars().stream().map(f -> f.toUri()).map(File::new).collect(Collectors.toSet()); int yarnFileReplication = yarnConfiguration.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, DFSConfigKeys.DFS_REPLICATION_DEFAULT); - int fileReplication = flinkConfiguration.getInteger(YarnConfigOptions.FILE_REPLICATION); + int fileReplication = configuration.getInteger(YarnConfigOptions.FILE_REPLICATION); fileReplication = fileReplication > 0 ? fileReplication : yarnFileReplication; // only for per job mode @@ -789,38 +789,6 @@ public class YarnClusterDescriptor implements ClusterDescriptor { paths.add(remotePathJar); classPathBuilder.append(flinkJarPath.getName()).append(File.pathSeparator); - // Upload the flink configuration - // write out configuration file - File tmpConfigurationFile = null; - try { - tmpConfigurationFile = File.createTempFile(appId + "-flink-conf.yaml", null); - BootstrapTools.writeConfiguration(configuration, tmpConfigurationFile); - - String flinkConfigKey = "flink-conf.yaml"; - Path remotePathConf = setupSingleLocalResource( - flinkConfigKey, - fs, - appId, - new Path(tmpConfigurationFile.getAbsolutePath()), - localResources, - homeDir, - "", - fileReplication); - envShipFileList.append(flinkConfigKey).append("=").append(remotePathConf).append(","); - paths.add(remotePathConf); - classPathBuilder.append("flink-conf.yaml").append(File.pathSeparator); - } finally { - if (tmpConfigurationFile != null && !tmpConfigurationFile.delete()) { - LOG.warn("Fail to delete temporary file {}.", tmpConfigurationFile.toPath()); - } - } - - if (userJarInclusion == YarnConfigOptions.UserJarInclusion.LAST) { - for (String userClassPath : userClassPaths) { - classPathBuilder.append(userClassPath).append(File.pathSeparator); - } - } - // write job graph to tmp file and add it to local resource // TODO: server use user main method to generate job graph if (jobGraph != null) { @@ -828,12 +796,12 @@ public class YarnClusterDescriptor implements ClusterDescriptor { try { tmpJobGraphFile = File.createTempFile(appId.toString(), null); try (FileOutputStream output = new FileOutputStream(tmpJobGraphFile); - ObjectOutputStream obOutput = new ObjectOutputStream(output);){ + ObjectOutputStream obOutput = new ObjectOutputStream(output)) { obOutput.writeObject(jobGraph); } final String jobGraphFilename = "job.graph"; - flinkConfiguration.setString(JOB_GRAPH_FILE_PATH, jobGraphFilename); + configuration.setString(JOB_GRAPH_FILE_PATH, jobGraphFilename); Path pathFromYarnURL = setupSingleLocalResource( jobGraphFilename, @@ -847,7 +815,7 @@ public class YarnClusterDescriptor implements ClusterDescriptor { paths.add(pathFromYarnURL); classPathBuilder.append(jobGraphFilename).append(File.pathSeparator); } catch (Exception e) { - LOG.warn("Add job graph to local resource fail"); + LOG.warn("Add job graph to local resource fail."); throw e; } finally { if (tmpJobGraphFile != null && !tmpJobGraphFile.delete()) { @@ -856,6 +824,38 @@ public class YarnClusterDescriptor implements ClusterDescriptor { } } + // Upload the flink configuration + // write out configuration file + File tmpConfigurationFile = null; + try { + tmpConfigurationFile = File.createTempFile(appId + "-flink-conf.yaml", null); + BootstrapTools.writeConfiguration(configuration, tmpConfigurationFile); + + String flinkConfigKey = "flink-conf.yaml"; + Path remotePathConf = setupSingleLocalResource( + flinkConfigKey, + fs, + appId, + new Path(tmpConfigurationFile.getAbsolutePath()), + localResources, + homeDir, + "", + fileReplication); + envShipFileList.append(flinkConfigKey).append("=").append(remotePathConf).append(","); + paths.add(remotePathConf); + classPathBuilder.append("flink-conf.yaml").append(File.pathSeparator); + } finally { + if (tmpConfigurationFile != null && !tmpConfigurationFile.delete()) { + LOG.warn("Fail to delete temporary file {}.", tmpConfigurationFile.toPath()); + } + } + + if (userJarInclusion == YarnConfigOptions.UserJarInclusion.LAST) { + for (String userClassPath : userClassPaths) { + classPathBuilder.append(userClassPath).append(File.pathSeparator); + } + } + final Path yarnFilesDir = getYarnFilesDir(appId); FsPermission permission = new FsPermission(FsAction.ALL, FsAction.NONE, FsAction.NONE); fs.setPermission(yarnFilesDir, permission); // set permission for path. -- Gitee From 50ee6554dffee784f6dbfeaba7b18a18bdba5659 Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Thu, 19 Mar 2020 10:55:21 +0100 Subject: [PATCH 240/885] [FLINK-16718][tests] Fix ByteBuf leak in KvStateServerHandlerTest This closes #11453. --- .../network/KvStateServerHandlerTest.java | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerHandlerTest.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerHandlerTest.java index d38cca8dd0..f6f396eeea 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerHandlerTest.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerHandlerTest.java @@ -170,6 +170,7 @@ public class KvStateServerHandlerTest extends TestLogger { assertEquals(MessageType.REQUEST_RESULT, MessageSerializer.deserializeHeader(buf)); long deserRequestId = MessageSerializer.getRequestId(buf); KvStateResponse response = serializer.deserializeResponse(buf); + buf.release(); assertEquals(requestId, deserRequestId); @@ -217,6 +218,7 @@ public class KvStateServerHandlerTest extends TestLogger { // Verify the response assertEquals(MessageType.REQUEST_FAILURE, MessageSerializer.deserializeHeader(buf)); RequestFailure response = MessageSerializer.deserializeRequestFailure(buf); + buf.release(); assertEquals(requestId, response.getRequestId()); @@ -278,6 +280,7 @@ public class KvStateServerHandlerTest extends TestLogger { // Verify the response assertEquals(MessageType.REQUEST_FAILURE, MessageSerializer.deserializeHeader(buf)); RequestFailure response = MessageSerializer.deserializeRequestFailure(buf); + buf.release(); assertEquals(requestId, response.getRequestId()); @@ -363,6 +366,7 @@ public class KvStateServerHandlerTest extends TestLogger { // Verify the response assertEquals(MessageType.REQUEST_FAILURE, MessageSerializer.deserializeHeader(buf)); RequestFailure response = MessageSerializer.deserializeRequestFailure(buf); + buf.release(); assertTrue(response.getCause().getMessage().contains("Expected test Exception")); @@ -392,6 +396,7 @@ public class KvStateServerHandlerTest extends TestLogger { // Verify the response assertEquals(MessageType.SERVER_FAILURE, MessageSerializer.deserializeHeader(buf)); Throwable response = MessageSerializer.deserializeServerFailure(buf); + buf.release(); assertTrue(response.getMessage().contains("Expected test Exception")); @@ -454,6 +459,7 @@ public class KvStateServerHandlerTest extends TestLogger { // Verify the response assertEquals(MessageType.REQUEST_FAILURE, MessageSerializer.deserializeHeader(buf)); RequestFailure response = MessageSerializer.deserializeRequestFailure(buf); + buf.release(); assertTrue(response.getCause().getMessage().contains("RejectedExecutionException")); @@ -490,6 +496,7 @@ public class KvStateServerHandlerTest extends TestLogger { // Verify the response assertEquals(MessageType.SERVER_FAILURE, MessageSerializer.deserializeHeader(buf)); Throwable response = MessageSerializer.deserializeServerFailure(buf); + buf.release(); assertEquals(0L, stats.getNumRequests()); assertEquals(0L, stats.getNumFailed()); @@ -505,6 +512,7 @@ public class KvStateServerHandlerTest extends TestLogger { // Verify the response assertEquals(MessageType.SERVER_FAILURE, MessageSerializer.deserializeHeader(buf)); response = MessageSerializer.deserializeServerFailure(buf); + buf.release(); assertTrue("Unexpected failure cause " + response.getClass().getName(), response instanceof IllegalArgumentException); @@ -544,6 +552,7 @@ public class KvStateServerHandlerTest extends TestLogger { channel.writeInbound(unexpected); assertEquals("Buffer not recycled", 0L, unexpected.refCnt()); + channel.finishAndReleaseAll(); } /** @@ -610,6 +619,7 @@ public class KvStateServerHandlerTest extends TestLogger { // Verify the response assertEquals(MessageType.REQUEST_FAILURE, MessageSerializer.deserializeHeader(buf)); RequestFailure response = MessageSerializer.deserializeRequestFailure(buf); + buf.release(); assertEquals(182828L, response.getRequestId()); assertTrue(response.getCause().getMessage().contains("IOException")); @@ -626,6 +636,7 @@ public class KvStateServerHandlerTest extends TestLogger { // Verify the response assertEquals(MessageType.REQUEST_FAILURE, MessageSerializer.deserializeHeader(buf)); response = MessageSerializer.deserializeRequestFailure(buf); + buf.release(); assertEquals(182829L, response.getRequestId()); assertTrue(response.getCause().getMessage().contains("IOException")); @@ -696,6 +707,7 @@ public class KvStateServerHandlerTest extends TestLogger { Object msg = readInboundBlocking(channel); assertTrue("Not ChunkedByteBuf", msg instanceof ChunkedByteBuf); + ((ChunkedByteBuf) msg).close(); } // ------------------------------------------------------------------------ -- Gitee From a2b669faabf372da7d5c8c0f100dac808f9b0821 Mon Sep 17 00:00:00 2001 From: Jacob Sevart Date: Sat, 21 Mar 2020 13:03:49 -0700 Subject: [PATCH 241/885] [hotfix][doc] Document union list metadata gotcha This closes #11475. --- docs/dev/stream/state/state.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/dev/stream/state/state.md b/docs/dev/stream/state/state.md index 028ad8fe17..e9869e738a 100644 --- a/docs/dev/stream/state/state.md +++ b/docs/dev/stream/state/state.md @@ -567,7 +567,9 @@ the following redistribution schemes are defined: while `element2` will go to operator instance 1. - **Union redistribution:** Each operator returns a List of state elements. The whole state is logically a concatenation of - all lists. On restore/redistribution, each operator gets the complete list of state elements. + all lists. On restore/redistribution, each operator gets the complete list of state elements. Do not use this feature if + your list may have high cardinality. Checkpoint metadata will store an offset to each list entry, which could lead to RPC + framesize or out-of-memory errors. Below is an example of a stateful `SinkFunction` that uses `CheckpointedFunction` to buffer elements before sending them to the outside world. It demonstrates -- Gitee From c125c04185febca0a2bbf4fa04dee207208f1a5b Mon Sep 17 00:00:00 2001 From: Lining Jing Date: Thu, 27 Feb 2020 17:27:08 +0800 Subject: [PATCH 242/885] [FLINK-16302][rest] Enable retrieval of custom TaskManager log files Add log list handler which lists all files in TaskManager log directory. Enable retrieval of custom TaskManager log files by name. This closes #11250. --- .../generated/rest_v1_dispatcher.html | 66 ++++++++++ .../src/test/resources/rest_api_v1.snapshot | 37 ++++++ .../resourcemanager/ResourceManager.java | 34 ++++- .../ResourceManagerGateway.java | 23 +++- .../AbstractTaskManagerFileHandler.java | 29 +++-- .../AbstractTaskManagerHandler.java | 14 ++- .../TaskManagerCustomLogHandler.java | 67 ++++++++++ .../TaskManagerLogFileHandler.java | 5 +- .../TaskManagerLogListHandler.java | 88 +++++++++++++ .../TaskManagerStdoutFileHandler.java | 5 +- .../taskmanager/LogFileNamePathParameter.java | 49 ++++++++ .../rest/messages/taskmanager/LogInfo.java | 68 ++++++++++ .../messages/taskmanager/LogListInfo.java | 73 +++++++++++ .../TaskManagerCustomLogHeaders.java | 58 +++++++++ .../TaskManagerFileMessageParameters.java | 42 +++++++ .../taskmanager/TaskManagerLogsHeaders.java | 77 ++++++++++++ .../runtime/taskexecutor/TaskExecutor.java | 82 +++++++++---- .../taskexecutor/TaskExecutorGateway.java | 20 ++- .../TaskManagerConfiguration.java | 15 +++ .../webmonitor/WebMonitorEndpoint.java | 23 ++++ .../utils/TestingResourceManagerGateway.java | 45 ++++++- .../AbstractTaskManagerFileHandlerTest.java | 8 +- .../TaskManagerLogListHandlerTest.java | 116 ++++++++++++++++++ .../taskexecutor/TaskExecutorTest.java | 2 +- .../TestingTaskExecutorGateway.java | 14 ++- 25 files changed, 1000 insertions(+), 60 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerCustomLogHandler.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerLogListHandler.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/LogFileNamePathParameter.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/LogInfo.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/LogListInfo.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerCustomLogHeaders.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerFileMessageParameters.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerLogsHeaders.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerLogListHandlerTest.java diff --git a/docs/_includes/generated/rest_v1_dispatcher.html b/docs/_includes/generated/rest_v1_dispatcher.html index 113c37e480..79c1ba41a9 100644 --- a/docs/_includes/generated/rest_v1_dispatcher.html +++ b/docs/_includes/generated/rest_v1_dispatcher.html @@ -4070,6 +4070,72 @@ Using 'curl' you can upload a jar via 'curl -X POST -H "Expect:" -F "jarfile=@pa
    Pattern OperationDescription模式操作描述
    where(condition) -

    Defines a condition for the current pattern. To match the pattern, an event must satisfy the condition. - Multiple consecutive where() clauses lead to their conditions being ANDed:

    +

    为当前模式定义一个条件。为了匹配这个模式,一个事件必须满足某些条件。 + 多个连续的where()语句取与组成判断条件:

    {% highlight scala %} -pattern.where(event => ... /* some condition */) +pattern.where(event => ... /* 一些判断条件 */) {% endhighlight %}
    or(condition) -

    Adds a new condition which is ORed with an existing one. An event can match the pattern only if it - passes at least one of the conditions:

    +

    增加一个新的判断,和当前的判断取或。一个事件只要满足至少一个判断条件就匹配到模式:

    {% highlight scala %} -pattern.where(event => ... /* some condition */) - .or(event => ... /* alternative condition */) +pattern.where(event => ... /* 一些判断条件 */) + .or(event => ... /* 替代条件 */) {% endhighlight %}
    until(condition) -

    Specifies a stop condition for looping pattern. Meaning if event matching the given condition occurs, no more - events will be accepted into the pattern.

    -

    Applicable only in conjunction with oneOrMore()

    -

    NOTE: It allows for cleaning state for corresponding pattern on event-based condition.

    +

    为循环模式指定一个停止条件。意思是满足了给定的条件的事件出现后,就不会再有事件被接受进入模式了。

    +

    只适用于和oneOrMore()同时使用。

    +

    提示: 在基于事件的条件中,它可用于清理对应模式的状态。

    {% highlight scala %} -pattern.oneOrMore().until(event => ... /* some condition */) +pattern.oneOrMore().until(event => ... /* 替代条件 */) {% endhighlight %}
    subtype(subClass) -

    Defines a subtype condition for the current pattern. An event can only match the pattern if it is - of this subtype:

    +

    为当前模式定义一个子类型条件。一个事件只有是这个子类型的时候才能匹配到模式:

    {% highlight scala %} pattern.subtype(classOf[SubEvent]) {% endhighlight %} @@ -596,10 +582,10 @@ pattern.subtype(classOf[SubEvent])
    oneOrMore() -

    Specifies that this pattern expects at least one occurrence of a matching event.

    -

    By default a relaxed internal contiguity (between subsequent events) is used. For more info on - internal contiguity see consecutive.

    -

    NOTE: It is advised to use either until() or within() to enable state clearing

    +

    指定模式期望匹配到的事件至少出现一次。.

    +

    默认(在子事件间)使用松散的内部连续性。 + 关于内部连续性的更多信息可以参考连续性

    +

    提示: 推荐使用until()或者within()来清理状态。

    {% highlight scala %} pattern.oneOrMore() {% endhighlight %} @@ -608,21 +594,20 @@ pattern.oneOrMore()
    timesOrMore(#times) -

    Specifies that this pattern expects at least #times occurrences - of a matching event.

    -

    By default a relaxed internal contiguity (between subsequent events) is used. For more info on - internal contiguity see consecutive.

    +

    指定模式期望匹配到的事件至少出现#times次。.

    +

    默认(在子事件间)使用松散的内部连续性。 + 关于内部连续性的更多信息可以参考连续性

    {% highlight scala %} pattern.timesOrMore(2) {% endhighlight %}
    times(#ofTimes) -

    Specifies that this pattern expects an exact number of occurrences of a matching event.

    -

    By default a relaxed internal contiguity (between subsequent events) is used. - For more info on internal contiguity see consecutive.

    +
    times(#ofTimes) +

    指定模式期望匹配到的事件正好出现的次数。

    +

    默认(在子事件间)使用松散的内部连续性。 + 关于内部连续性的更多信息可以参考连续性

    {% highlight scala %} pattern.times(2) {% endhighlight %} @@ -631,10 +616,9 @@ pattern.times(2)
    times(#fromTimes, #toTimes) -

    Specifies that this pattern expects occurrences between #fromTimes - and #toTimes of a matching event.

    -

    By default a relaxed internal contiguity (between subsequent events) is used. For more info on - internal contiguity see consecutive.

    +

    指定模式期望匹配到的事件出现次数在#fromTimes#toTimes之间。

    +

    默认(在子事件间)使用松散的内部连续性。 + 关于内部连续性的更多信息可以参考连续性

    {% highlight scala %} pattern.times(2, 4) {% endhighlight %} @@ -643,8 +627,7 @@ pattern.times(2, 4)
    optional() -

    Specifies that this pattern is optional, i.e. it may not occur at all. This is applicable to all - aforementioned quantifiers.

    +

    指定这个模式是可选的,也就是说,它可能根本不出现。这对所有之前提到的量词都适用。

    {% highlight scala %} pattern.oneOrMore().optional() {% endhighlight %} @@ -653,8 +636,7 @@ pattern.oneOrMore().optional()
    greedy() -

    Specifies that this pattern is greedy, i.e. it will repeat as many as possible. This is only applicable - to quantifiers and it does not support group pattern currently.

    +

    指定这个模式是贪心的,也就是说,它会重复尽可能多的次数。这只对量词适用,现在还不支持模式组。

    {% highlight scala %} pattern.oneOrMore().greedy() {% endhighlight %} @@ -665,12 +647,11 @@ pattern.oneOrMore().greedy() -### Combining Patterns +### 组合模式 -Now that you've seen what an individual pattern can look like, it is time to see how to combine them -into a full pattern sequence. +现在你已经看到单个的模式是什么样的了,该去看看如何把它们连接起来组成一个完整的模式序列。 -A pattern sequence has to start with an initial pattern, as shown below: +模式序列由一个初始模式作为开头,如下所示:
    @@ -686,48 +667,46 @@ val start : Pattern[Event, _] = Pattern.begin("start")
    -Next, you can append more patterns to your pattern sequence by specifying the desired *contiguity conditions* between -them. FlinkCEP supports the following forms of contiguity between events: +接下来,你可以增加更多的模式到模式序列中并指定它们之间所需的*连续条件*。FlinkCEP支持事件之间如下形式的连续策略: - 1. **Strict Contiguity**: Expects all matching events to appear strictly one after the other, without any non-matching events in-between. + 1. **严格连续**: 期望所有匹配的事件严格的一个接一个出现,中间没有任何不匹配的事件。 - 2. **Relaxed Contiguity**: Ignores non-matching events appearing in-between the matching ones. + 2. **松散连续**: 忽略匹配的事件之间的不匹配的事件。 - 3. **Non-Deterministic Relaxed Contiguity**: Further relaxes contiguity, allowing additional matches - that ignore some matching events. + 3. **不确定的松散连续**: 更进一步的松散连续,允许忽略掉一些匹配事件的附加匹配。 -To apply them between consecutive patterns, you can use: +可以使用下面的方法来指定模式之间的连续策略: -1. `next()`, for *strict*, -2. `followedBy()`, for *relaxed*, and -3. `followedByAny()`, for *non-deterministic relaxed* contiguity. +1. `next()`,指定*严格连续*, +2. `followedBy()`,指定*松散连续*, +3. `followedByAny()`,指定*不确定的松散*连续。 -or +或者 -1. `notNext()`, if you do not want an event type to directly follow another -2. `notFollowedBy()`, if you do not want an event type to be anywhere between two other event types. +1. `notNext()`,如果不想后面直接连着一个特定事件 +2. `notFollowedBy()`,如果不想一个特定事件发生在两个事件之间的任何地方。 -{% warn Attention %} A pattern sequence cannot end in `notFollowedBy()`. +{% warn 注意 %} 模式序列不能以`notFollowedBy()`结尾。 -{% warn Attention %} A `NOT` pattern cannot be preceded by an optional one. +{% warn 注意 %} 一个`NOT`模式前面不能是可选的模式。
    {% highlight java %} -// strict contiguity +// 严格连续 Pattern strict = start.next("middle").where(...); -// relaxed contiguity +// 松散连续 Pattern relaxed = start.followedBy("middle").where(...); -// non-deterministic relaxed contiguity +// 不确定的松散连续 Pattern nonDetermin = start.followedByAny("middle").where(...); -// NOT pattern with strict contiguity +// 严格连续的NOT模式 Pattern strictNot = start.notNext("not").where(...); -// NOT pattern with relaxed contiguity +// 松散连续的NOT模式 Pattern relaxedNot = start.notFollowedBy("not").where(...); {% endhighlight %} @@ -736,41 +715,39 @@ Pattern relaxedNot = start.notFollowedBy("not").where(...);
    {% highlight scala %} -// strict contiguity +// 严格连续 val strict: Pattern[Event, _] = start.next("middle").where(...) -// relaxed contiguity +// 松散连续 val relaxed: Pattern[Event, _] = start.followedBy("middle").where(...) -// non-deterministic relaxed contiguity +// 不确定的松散连续 val nonDetermin: Pattern[Event, _] = start.followedByAny("middle").where(...) -// NOT pattern with strict contiguity +// 严格连续的NOT模式 val strictNot: Pattern[Event, _] = start.notNext("not").where(...) -// NOT pattern with relaxed contiguity +// 松散连续的NOT模式 val relaxedNot: Pattern[Event, _] = start.notFollowedBy("not").where(...) {% endhighlight %}
    -Relaxed contiguity means that only the first succeeding matching event will be matched, while -with non-deterministic relaxed contiguity, multiple matches will be emitted for the same beginning. As an example, -a pattern `"a b"`, given the event sequence `"a", "c", "b1", "b2"`, will give the following results: +松散连续意味着跟着的事件中,只有第一个可匹配的事件会被匹配上,而不确定的松散连接情况下,有着同样起始的多个匹配会被输出。 +举例来说,模式`"a b"`,给定事件序列`"a","c","b1","b2"`,会产生如下的结果: -1. Strict Contiguity between `"a"` and `"b"`: `{}` (no match), the `"c"` after `"a"` causes `"a"` to be discarded. +1. `"a"`和`"b"`之间严格连续: `{}` (没有匹配),`"a"`之后的`"c"`导致`"a"`被丢弃。 -2. Relaxed Contiguity between `"a"` and `"b"`: `{a b1}`, as relaxed continuity is viewed as "skip non-matching events -till the next matching one". +2. `"a"`和`"b"`之间松散连续: `{a b1}`,松散连续会"跳过不匹配的事件直到匹配上的事件"。 -3. Non-Deterministic Relaxed Contiguity between `"a"` and `"b"`: `{a b1}`, `{a b2}`, as this is the most general form. +3. `"a"`和`"b"`之间不确定的松散连续: `{a b1}`, `{a b2}`,这是最常见的情况。 -It's also possible to define a temporal constraint for the pattern to be valid. -For example, you can define that a pattern should occur within 10 seconds via the `pattern.within()` method. -Temporal patterns are supported for both [processing and event time]({{site.baseurl}}/dev/event_time.html). +也可以为模式定义一个有效时间约束。 +例如,你可以通过`pattern.within()`方法指定一个模式应该在10秒内发生。 +这种时间模式支持[处理时间和事件时间]({{site.baseurl}}/zh/dev/event_time.html). -{% warn Attention %} A pattern sequence can only have one temporal constraint. If multiple such constraints are defined on different individual patterns, then the smallest is applied. +{% warn 注意 %} 一个模式序列只能有一个时间限制。如果限制了多个时间在不同的单个模式上,会使用最小的那个时间限制。
    @@ -786,42 +763,41 @@ next.within(Time.seconds(10))
    -#### Contiguity within looping patterns +#### 循环模式中的连续性 -You can apply the same contiguity condition as discussed in the previous [section](#combining-patterns) within a looping pattern. -The contiguity will be applied between elements accepted into such a pattern. -To illustrate the above with an example, a pattern sequence `"a b+ c"` (`"a"` followed by any(non-deterministic relaxed) sequence of one or more `"b"`'s followed by a `"c"`) with -input `"a", "b1", "d1", "b2", "d2", "b3" "c"` will have the following results: +你可以在循环模式中使用和前面[章节](#组合模式)讲过的同样的连续性。 +连续性会被运用在被接受进入模式的事件之间。 +用这个例子来说明上面所说的连续性,一个模式序列`"a b+ c"`(`"a"`后面跟着一个或者多个(不确定连续的)`"b"`,然后跟着一个`"c"`) +输入为`"a","b1","d1","b2","d2","b3","c"`,输出结果如下: - 1. **Strict Contiguity**: `{a b3 c}` -- the `"d1"` after `"b1"` causes `"b1"` to be discarded, the same happens for `"b2"` because of `"d2"`. + 1. **严格连续**: `{a b3 c}` -- `"b1"`之后的`"d1"`导致`"b1"`被丢弃,同样`"b2"`因为`"d2"`被丢弃。 - 2. **Relaxed Contiguity**: `{a b1 c}`, `{a b1 b2 c}`, `{a b1 b2 b3 c}`, `{a b2 c}`, `{a b2 b3 c}`, `{a b3 c}` - `"d"`'s are ignored. + 2. **松散连续**: `{a b1 c}`,`{a b1 b2 c}`,`{a b1 b2 b3 c}`,`{a b2 c}`,`{a b2 b3 c}`,`{a b3 c}` - `"d"`都被忽略了。 - 3. **Non-Deterministic Relaxed Contiguity**: `{a b1 c}`, `{a b1 b2 c}`, `{a b1 b3 c}`, `{a b1 b2 b3 c}`, `{a b2 c}`, `{a b2 b3 c}`, `{a b3 c}` - - notice the `{a b1 b3 c}`, which is the result of relaxing contiguity between `"b"`'s. + 3. **不确定松散连续**: `{a b1 c}`,`{a b1 b2 c}`,`{a b1 b3 c}`,`{a b1 b2 b3 c}`,`{a b2 c}`,`{a b2 b3 c}`,`{a b3 c}` - + 注意`{a b1 b3 c}`,这是因为`"b"`之间是不确定松散连续产生的。 -For looping patterns (e.g. `oneOrMore()` and `times()`) the default is *relaxed contiguity*. If you want -strict contiguity, you have to explicitly specify it by using the `consecutive()` call, and if you want -*non-deterministic relaxed contiguity* you can use the `allowCombinations()` call. +对于循环模式(例如`oneOrMore()`和`times()`)),默认是*松散连续*。如果想使用*严格连续*,你需要使用`consecutive()`方法明确指定, +如果想使用*不确定松散连续*,你可以使用`allowCombinations()`方法。
    - - + + @@ -890,19 +866,19 @@ Pattern.begin("start").where(new SimpleCondition() {
    Pattern OperationDescription模式操作描述
    consecutive() -

    Works in conjunction with oneOrMore() and times() and imposes strict contiguity between the matching - events, i.e. any non-matching element breaks the match (as in next()).

    -

    If not applied a relaxed contiguity (as in followedBy()) is used.

    +

    oneOrMore()times()一起使用, 在匹配的事件之间施加严格的连续性, + 也就是说,任何不匹配的事件都会终止匹配(和next()一样)。

    +

    如果不使用它,那么就是松散连续(和followedBy()一样)。

    -

    E.g. a pattern like:

    +

    例如,一个如下的模式:

    {% highlight java %} Pattern.begin("start").where(new SimpleCondition() { @Override @@ -842,20 +818,20 @@ Pattern.begin("start").where(new SimpleCondition() { } }); {% endhighlight %} -

    Will generate the following matches for an input sequence: C D A1 A2 A3 D A4 B

    +

    输入:C D A1 A2 A3 D A4 B,会产生下面的输出:

    -

    with consecutive applied: {C A1 B}, {C A1 A2 B}, {C A1 A2 A3 B}

    -

    without consecutive applied: {C A1 B}, {C A1 A2 B}, {C A1 A2 A3 B}, {C A1 A2 A3 A4 B}

    +

    如果施加严格连续性: {C A1 B},{C A1 A2 B},{C A1 A2 A3 B}

    +

    不施加严格连续性: {C A1 B},{C A1 A2 B},{C A1 A2 A3 B},{C A1 A2 A3 A4 B}

    allowCombinations() -

    Works in conjunction with oneOrMore() and times() and imposes non-deterministic relaxed contiguity - between the matching events (as in followedByAny()).

    -

    If not applied a relaxed contiguity (as in followedBy()) is used.

    +

    oneOrMore()times()一起使用, + 在匹配的事件中间施加不确定松散连续性(和followedByAny()一样)。

    +

    如果不使用,就是松散连续(和followedBy()一样)。

    -

    E.g. a pattern like:

    +

    例如,一个如下的模式:

    {% highlight java %} Pattern.begin("start").where(new SimpleCondition() { @Override @@ -876,10 +852,10 @@ Pattern.begin("start").where(new SimpleCondition() { } }); {% endhighlight %} -

    Will generate the following matches for an input sequence: C D A1 A2 A3 D A4 B

    +

    输入:C D A1 A2 A3 D A4 B,会产生如下的输出:

    -

    with combinations enabled: {C A1 B}, {C A1 A2 B}, {C A1 A3 B}, {C A1 A4 B}, {C A1 A2 A3 B}, {C A1 A2 A4 B}, {C A1 A3 A4 B}, {C A1 A2 A3 A4 B}

    -

    without combinations enabled: {C A1 B}, {C A1 A2 B}, {C A1 A2 A3 B}, {C A1 A2 A3 A4 B}

    +

    如果使用不确定松散连续: {C A1 B},{C A1 A2 B},{C A1 A3 B},{C A1 A4 B},{C A1 A2 A3 B},{C A1 A2 A4 B},{C A1 A3 A4 B},{C A1 A2 A3 A4 B}

    +

    如果不使用:{C A1 B},{C A1 A2 B},{C A1 A2 A3 B},{C A1 A2 A3 A4 B}

    - - + + @@ -942,12 +918,11 @@ Pattern.begin("start").where(_.getName().equals("c")) -### Groups of patterns +### 模式组 -It's also possible to define a pattern sequence as the condition for `begin`, `followedBy`, `followedByAny` and -`next`. The pattern sequence will be considered as the matching condition logically and a `GroupPattern` will be -returned and it is possible to apply `oneOrMore()`, `times(#ofTimes)`, `times(#fromTimes, #toTimes)`, `optional()`, -`consecutive()`, `allowCombinations()` to the `GroupPattern`. +也可以定义一个模式序列作为`begin`,`followedBy`,`followedByAny`和`next`的条件。这个模式序列在逻辑上会被当作匹配的条件, +并且返回一个`GroupPattern`,可以在`GroupPattern`上使用`oneOrMore()`,`times(#ofTimes)`, +`times(#fromTimes, #toTimes)`,`optional()`,`consecutive()`,`allowCombinations()`。
    @@ -957,17 +932,17 @@ Pattern start = Pattern.begin( Pattern.begin("start").where(...).followedBy("start_middle").where(...) ); -// strict contiguity +// 严格连续 Pattern strict = start.next( Pattern.begin("next_start").where(...).followedBy("next_middle").where(...) ).times(3); -// relaxed contiguity +// 松散连续 Pattern relaxed = start.followedBy( Pattern.begin("followedby_start").where(...).followedBy("followedby_middle").where(...) ).oneOrMore(); -// non-deterministic relaxed contiguity +// 不确定松散连续 Pattern nonDetermin = start.followedByAny( Pattern.begin("followedbyany_start").where(...).followedBy("followedbyany_middle").where(...) ).optional(); @@ -982,17 +957,17 @@ val start: Pattern[Event, _] = Pattern.begin( Pattern.begin[Event]("start").where(...).followedBy("start_middle").where(...) ) -// strict contiguity +// 严格连续 val strict: Pattern[Event, _] = start.next( Pattern.begin[Event]("next_start").where(...).followedBy("next_middle").where(...) ).times(3) -// relaxed contiguity +// 松散连续 val relaxed: Pattern[Event, _] = start.followedBy( Pattern.begin[Event]("followedby_start").where(...).followedBy("followedby_middle").where(...) ).oneOrMore() -// non-deterministic relaxed contiguity +// 不确定松散连续 val nonDetermin: Pattern[Event, _] = start.followedByAny( Pattern.begin[Event]("followedbyany_start").where(...).followedBy("followedbyany_middle").where(...) ).optional() @@ -1008,15 +983,15 @@ val nonDetermin: Pattern[Event, _] = start.followedByAny(
    Pattern OperationDescription模式操作描述
    consecutive() -

    Works in conjunction with oneOrMore() and times() and imposes strict contiguity between the matching - events, i.e. any non-matching element breaks the match (as in next()).

    -

    If not applied a relaxed contiguity (as in followedBy()) is used.

    +

    oneOrMore()times()一起使用, 在匹配的事件之间施加严格的连续性, + 也就是说,任何不匹配的事件都会终止匹配(和next()一样)。

    +

    如果不使用它,那么就是松散连续(和followedBy()一样)。

    -

    E.g. a pattern like:

    +

    例如,一个如下的模式:

    {% highlight scala %} Pattern.begin("start").where(_.getName().equals("c")) .followedBy("middle").where(_.getName().equals("a")) @@ -910,20 +886,20 @@ Pattern.begin("start").where(_.getName().equals("c")) .followedBy("end1").where(_.getName().equals("b")) {% endhighlight %} -

    Will generate the following matches for an input sequence: C D A1 A2 A3 D A4 B

    +

    输入:C D A1 A2 A3 D A4 B,会产生下面的输出:

    -

    with consecutive applied: {C A1 B}, {C A1 A2 B}, {C A1 A2 A3 B}

    -

    without consecutive applied: {C A1 B}, {C A1 A2 B}, {C A1 A2 A3 B}, {C A1 A2 A3 A4 B}

    +

    如果施加严格连续性: {C A1 B},{C A1 A2 B},{C A1 A2 A3 B}

    +

    不施加严格连续性: {C A1 B},{C A1 A2 B},{C A1 A2 A3 B},{C A1 A2 A3 A4 B}

    allowCombinations() -

    Works in conjunction with oneOrMore() and times() and imposes non-deterministic relaxed contiguity - between the matching events (as in followedByAny()).

    -

    If not applied a relaxed contiguity (as in followedBy()) is used.

    +

    oneOrMore()times()一起使用, + 在匹配的事件中间施加不确定松散连续性(和followedByAny()一样)。

    +

    如果不使用,就是松散连续(和followedBy()一样)。

    -

    E.g. a pattern like:

    +

    例如,一个如下的模式:

    {% highlight scala %} Pattern.begin("start").where(_.getName().equals("c")) .followedBy("middle").where(_.getName().equals("a")) @@ -931,10 +907,10 @@ Pattern.begin("start").where(_.getName().equals("c")) .followedBy("end1").where(_.getName().equals("b")) {% endhighlight %} -

    Will generate the following matches for an input sequence: C D A1 A2 A3 D A4 B

    +

    输入:C D A1 A2 A3 D A4 B,会产生如下的输出:

    -

    with combinations enabled: {C A1 B}, {C A1 A2 B}, {C A1 A3 B}, {C A1 A4 B}, {C A1 A2 A3 B}, {C A1 A2 A4 B}, {C A1 A3 A4 B}, {C A1 A2 A3 A4 B}

    -

    without combinations enabled: {C A1 B}, {C A1 A2 B}, {C A1 A2 A3 B}, {C A1 A2 A3 A4 B}

    +

    如果使用不确定松散连续: {C A1 B},{C A1 A2 B},{C A1 A3 B},{C A1 A4 B},{C A1 A2 A3 B},{C A1 A2 A4 B},{C A1 A3 A4 B},{C A1 A2 A3 A4 B}

    +

    如果不使用:{C A1 B},{C A1 A2 B},{C A1 A2 A3 B},{C A1 A2 A3 A4 B}

    - - + + + + + + + + + + + + + + -- Gitee From c83bc655fc28cd10c77d0b6991ef916e80429d18 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 13 Mar 2020 14:36:32 +0100 Subject: [PATCH 165/885] [FLINK-11088] Add unit test for Kerberos parsing in YarnEntrypointUtilsTest --- .../entrypoint/YarnEntrypointUtilsTest.java | 40 +++++++++++++++++-- 1 file changed, 37 insertions(+), 3 deletions(-) diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtilsTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtilsTest.java index ab5278f34e..e80a712e8c 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtilsTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtilsTest.java @@ -20,8 +20,10 @@ package org.apache.flink.yarn.entrypoint; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.RestOptions; +import org.apache.flink.configuration.SecurityOptions; import org.apache.flink.runtime.clusterframework.BootstrapTools; import org.apache.flink.util.TestLogger; +import org.apache.flink.yarn.YarnConfigKeys; import org.apache.hadoop.yarn.api.ApplicationConstants; import org.junit.ClassRule; @@ -37,6 +39,7 @@ import java.util.Map; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.nullValue; import static org.junit.Assert.assertThat; /** @@ -93,14 +96,45 @@ public class YarnEntrypointUtilsTest extends TestLogger { assertThat(configuration.getString(RestOptions.BIND_PORT), is(equalTo(bindingPortRange))); } + @Test + public void testParsingValidKerberosEnv() throws IOException { + final Configuration initialConfiguration = new Configuration(); + Map env = new HashMap<>(); + File keytabFile = TEMPORARY_FOLDER.newFile(); + env.put(YarnConfigKeys.LOCAL_KEYTAB_PATH, keytabFile.getAbsolutePath()); + env.put(YarnConfigKeys.KEYTAB_PRINCIPAL, "starlord"); + + Configuration configuration = loadConfiguration(initialConfiguration, env); + + assertThat(configuration.get(SecurityOptions.KERBEROS_LOGIN_KEYTAB), is(keytabFile.getAbsolutePath())); + assertThat(configuration.get(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL), is("starlord")); + } + + @Test + public void testParsingKerberosEnvWithMissingKeytab() throws IOException { + final Configuration initialConfiguration = new Configuration(); + Map env = new HashMap<>(); + env.put(YarnConfigKeys.LOCAL_KEYTAB_PATH, "/hopefully/doesnt/exist"); + env.put(YarnConfigKeys.KEYTAB_PRINCIPAL, "starlord"); + + Configuration configuration = loadConfiguration(initialConfiguration, env); + + // both keytab and principal should be null + assertThat(configuration.get(SecurityOptions.KERBEROS_LOGIN_KEYTAB), nullValue()); + assertThat(configuration.get(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL), nullValue()); + } + @Nonnull private static Configuration loadConfiguration(Configuration initialConfiguration) throws IOException { + final Map env = new HashMap<>(); + return loadConfiguration(initialConfiguration, env); + } + + @Nonnull + private static Configuration loadConfiguration(Configuration initialConfiguration, Map env) throws IOException { final File workingDirectory = TEMPORARY_FOLDER.newFolder(); - final Map env = new HashMap<>(4); env.put(ApplicationConstants.Environment.NM_HOST.key(), "foobar"); - BootstrapTools.writeConfiguration(initialConfiguration, new File(workingDirectory, "flink-conf.yaml")); return YarnEntrypointUtils.loadConfiguration(workingDirectory.getAbsolutePath(), env); } - } -- Gitee From 0477368813fba33f0f9aa5ff82304516c499ed41 Mon Sep 17 00:00:00 2001 From: Niels Basjes Date: Fri, 6 Mar 2020 16:42:30 +0100 Subject: [PATCH 166/885] [FLINK-14121] Update commons-compress because of CVE-2019-12402 This closes #11333. --- flink-dist/src/main/resources/META-INF/NOTICE | 2 +- .../flink-oss-fs-hadoop/src/main/resources/META-INF/NOTICE | 2 +- .../flink-swift-fs-hadoop/src/main/resources/META-INF/NOTICE | 2 +- pom.xml | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-dist/src/main/resources/META-INF/NOTICE b/flink-dist/src/main/resources/META-INF/NOTICE index dc978fe286..2c9da733a0 100644 --- a/flink-dist/src/main/resources/META-INF/NOTICE +++ b/flink-dist/src/main/resources/META-INF/NOTICE @@ -19,7 +19,7 @@ This project bundles the following dependencies under the Apache Software Licens - commons-cli:commons-cli:1.3.1 - commons-collections:commons-collections:3.2.2 - commons-io:commons-io:2.4 -- org.apache.commons:commons-compress:1.18 +- org.apache.commons:commons-compress:1.20 - org.apache.commons:commons-lang3:3.3.2 - org.apache.commons:commons-math3:3.5 - org.javassist:javassist:3.24.0-GA diff --git a/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/META-INF/NOTICE b/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/META-INF/NOTICE index 06a0dbd414..b4ecba50eb 100644 --- a/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/META-INF/NOTICE +++ b/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/META-INF/NOTICE @@ -30,7 +30,7 @@ This project bundles the following dependencies under the Apache Software Licens - net.minidev:accessors-smart:1.2 - net.minidev:json-smart:2.3 - org.apache.avro:avro:1.8.2 -- org.apache.commons:commons-compress:1.18 +- org.apache.commons:commons-compress:1.20 - org.apache.commons:commons-configuration2:2.1.1 - org.apache.commons:commons-lang3:3.3.2 - org.apache.commons:commons-math3:3.5 diff --git a/flink-filesystems/flink-swift-fs-hadoop/src/main/resources/META-INF/NOTICE b/flink-filesystems/flink-swift-fs-hadoop/src/main/resources/META-INF/NOTICE index 1cc5b08cc3..4343aafbb2 100644 --- a/flink-filesystems/flink-swift-fs-hadoop/src/main/resources/META-INF/NOTICE +++ b/flink-filesystems/flink-swift-fs-hadoop/src/main/resources/META-INF/NOTICE @@ -16,7 +16,7 @@ This project bundles the following dependencies under the Apache Software Licens - org.apache.htrace:htrace-core4:4.0.1-incubating - org.apache.httpcomponents:httpclient:4.5.3 - org.apache.httpcomponents:httpcore:4.4.6 -- org.apache.commons:commons-compress:1.18 +- org.apache.commons:commons-compress:1.20 - org.apache.commons:commons-math3:3.5 - commons-beanutils:commons-beanutils:1.8.3 - commons-cli:commons-cli:1.3.1 diff --git a/pom.xml b/pom.xml index 689114656f..71f0265037 100644 --- a/pom.xml +++ b/pom.xml @@ -538,7 +538,7 @@ under the License. org.apache.commons commons-compress - 1.18 + 1.20 -- Gitee From 51576916b2c2fbbc2d0e742b23f096c847158627 Mon Sep 17 00:00:00 2001 From: Terry Wang Date: Mon, 16 Mar 2020 09:55:38 +0800 Subject: [PATCH 167/885] [hotfix][docs] fix mistakes in Modules document This closes #11399 --- docs/dev/table/modules.md | 2 +- docs/dev/table/modules.zh.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/dev/table/modules.md b/docs/dev/table/modules.md index e38e2438ed..245dbe43bd 100644 --- a/docs/dev/table/modules.md +++ b/docs/dev/table/modules.md @@ -109,7 +109,7 @@ modules: -### List Available Catalogs +### List Available Modules
    diff --git a/docs/dev/table/modules.zh.md b/docs/dev/table/modules.zh.md index 10eb32b0b1..9cc84c6888 100644 --- a/docs/dev/table/modules.zh.md +++ b/docs/dev/table/modules.zh.md @@ -109,7 +109,7 @@ modules:
    -### List Available Catalogs +### List Available Modules
    -- Gitee From 3d119079288f3e9da7a19a32d68446ae5228b626 Mon Sep 17 00:00:00 2001 From: Canbin Zheng Date: Mon, 16 Mar 2020 10:35:23 +0800 Subject: [PATCH 168/885] [FLINK-16508][k8s] Name the ports exposed by the main Container in Pod This closes #11360 . --- .../kubeclient/decorators/AbstractServiceDecorator.java | 7 +------ .../kubeclient/decorators/InitJobManagerDecorator.java | 8 ++++++-- .../kubeclient/decorators/InitTaskManagerDecorator.java | 5 +++-- .../kubeclient/decorators/InternalServiceDecorator.java | 6 +++--- .../java/org/apache/flink/kubernetes/utils/Constants.java | 5 +++++ .../decorators/ExternalServiceDecoratorTest.java | 2 +- .../decorators/InitJobManagerDecoratorTest.java | 3 +++ .../decorators/InitTaskManagerDecoratorTest.java | 1 + .../decorators/InternalServiceDecoratorTest.java | 6 +++--- 9 files changed, 26 insertions(+), 17 deletions(-) diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/AbstractServiceDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/AbstractServiceDecorator.java index 7a54599452..444de3aa39 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/AbstractServiceDecorator.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/AbstractServiceDecorator.java @@ -18,7 +18,6 @@ package org.apache.flink.kubernetes.kubeclient.decorators; -import org.apache.flink.configuration.RestOptions; import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters; import org.apache.flink.kubernetes.utils.Constants; @@ -73,7 +72,7 @@ public abstract class AbstractServiceDecorator extends AbstractKubernetesStepDec final List servicePorts = new ArrayList<>(); servicePorts.add(getServicePort( - getPortName(RestOptions.PORT.key()), + Constants.REST_PORT_NAME, kubernetesJobManagerParameters.getRestPort())); return servicePorts; @@ -85,8 +84,4 @@ public abstract class AbstractServiceDecorator extends AbstractKubernetesStepDec .withPort(port) .build(); } - - protected static String getPortName(String portName){ - return portName.replace('.', '-'); - } } diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecorator.java index bef6b24b1b..18e7c405a0 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecorator.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecorator.java @@ -20,6 +20,7 @@ package org.apache.flink.kubernetes.kubeclient.decorators; import org.apache.flink.kubernetes.kubeclient.FlinkPod; import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters; +import org.apache.flink.kubernetes.utils.Constants; import org.apache.flink.kubernetes.utils.KubernetesUtils; import io.fabric8.kubernetes.api.model.Container; @@ -98,13 +99,16 @@ public class InitJobManagerDecorator extends AbstractKubernetesStepDecorator { private List getContainerPorts() { return Arrays.asList( new ContainerPortBuilder() + .withName(Constants.REST_PORT_NAME) .withContainerPort(kubernetesJobManagerParameters.getRestPort()) .build(), new ContainerPortBuilder() + .withName(Constants.JOB_MANAGER_RPC_PORT_NAME) .withContainerPort(kubernetesJobManagerParameters.getRPCPort()) .build(), - new ContainerPortBuilder(). - withContainerPort(kubernetesJobManagerParameters.getBlobServerPort()) + new ContainerPortBuilder() + .withName(Constants.BLOB_SERVER_PORT_NAME) + .withContainerPort(kubernetesJobManagerParameters.getBlobServerPort()) .build()); } diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecorator.java index 660e005e3c..4d521cf72b 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecorator.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecorator.java @@ -80,8 +80,9 @@ public class InitTaskManagerDecorator extends AbstractKubernetesStepDecorator { .withImagePullPolicy(kubernetesTaskManagerParameters.getImagePullPolicy()) .withResources(resourceRequirements) .withPorts(new ContainerPortBuilder() - .withContainerPort(kubernetesTaskManagerParameters.getRPCPort()) - .build()) + .withName(Constants.TASK_MANAGER_RPC_PORT_NAME) + .withContainerPort(kubernetesTaskManagerParameters.getRPCPort()) + .build()) .withEnv(getCustomizedEnvs()) .addNewEnv() .withName(ENV_FLINK_POD_NAME) diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InternalServiceDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InternalServiceDecorator.java index 0258b0faa3..8ca4b12537 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InternalServiceDecorator.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InternalServiceDecorator.java @@ -18,10 +18,10 @@ package org.apache.flink.kubernetes.kubeclient.decorators; -import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters; +import org.apache.flink.kubernetes.utils.Constants; import org.apache.flink.kubernetes.utils.KubernetesUtils; import io.fabric8.kubernetes.api.model.HasMetadata; @@ -59,10 +59,10 @@ public class InternalServiceDecorator extends AbstractServiceDecorator { final List servicePorts = super.getServicePorts(); servicePorts.add(getServicePort( - getPortName(JobManagerOptions.PORT.key()), + Constants.JOB_MANAGER_RPC_PORT_NAME, kubernetesJobManagerParameters.getRPCPort())); servicePorts.add(getServicePort( - getPortName(BlobServerOptions.PORT.key()), + Constants.BLOB_SERVER_PORT_NAME, kubernetesJobManagerParameters.getBlobServerPort())); return servicePorts; diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/Constants.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/Constants.java index 51db5ace58..5338431f0f 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/Constants.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/Constants.java @@ -51,6 +51,11 @@ public class Constants { public static final int BLOB_SERVER_PORT = 6124; public static final int TASK_MANAGER_RPC_PORT = 6122; + public static final String JOB_MANAGER_RPC_PORT_NAME = "jobmanager-rpc"; + public static final String BLOB_SERVER_PORT_NAME = "blobserver"; + public static final String REST_PORT_NAME = "rest"; + public static final String TASK_MANAGER_RPC_PORT_NAME = "taskmanager-rpc"; + public static final String RESOURCE_NAME_MEMORY = "memory"; public static final String RESOURCE_NAME_CPU = "cpu"; diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecoratorTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecoratorTest.java index 2d0e1b030a..d34cf9bb91 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecoratorTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecoratorTest.java @@ -69,7 +69,7 @@ public class ExternalServiceDecoratorTest extends KubernetesJobManagerTestBase { List expectedServicePorts = Collections.singletonList( new ServicePortBuilder() - .withName("rest-port") + .withName(Constants.REST_PORT_NAME) .withPort(REST_PORT) .build()); assertEquals(expectedServicePorts, restService.getSpec().getPorts()); diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecoratorTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecoratorTest.java index 57a09c2f2f..ba10a7234a 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecoratorTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecoratorTest.java @@ -102,12 +102,15 @@ public class InitJobManagerDecoratorTest extends KubernetesJobManagerTestBase { public void testMainContainerPorts() { final List expectedContainerPorts = Arrays.asList( new ContainerPortBuilder() + .withName(Constants.REST_PORT_NAME) .withContainerPort(REST_PORT) .build(), new ContainerPortBuilder() + .withName(Constants.JOB_MANAGER_RPC_PORT_NAME) .withContainerPort(RPC_PORT) .build(), new ContainerPortBuilder() + .withName(Constants.BLOB_SERVER_PORT_NAME) .withContainerPort(BLOB_SERVER_PORT) .build()); diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecoratorTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecoratorTest.java index 2c1cd145f2..59bb7cf8b6 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecoratorTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecoratorTest.java @@ -105,6 +105,7 @@ public class InitTaskManagerDecoratorTest extends KubernetesTaskManagerTestBase public void testMainContainerPorts() { final List expectedContainerPorts = Collections.singletonList( new ContainerPortBuilder() + .withName(Constants.TASK_MANAGER_RPC_PORT_NAME) .withContainerPort(RPC_PORT) .build()); diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InternalServiceDecoratorTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InternalServiceDecoratorTest.java index 857ae4eada..12f91ea824 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InternalServiceDecoratorTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InternalServiceDecoratorTest.java @@ -72,15 +72,15 @@ public class InternalServiceDecoratorTest extends KubernetesJobManagerTestBase { List expectedServicePorts = Arrays.asList( new ServicePortBuilder() - .withName("rest-port") + .withName(Constants.REST_PORT_NAME) .withPort(REST_PORT) .build(), new ServicePortBuilder() - .withName("jobmanager-rpc-port") + .withName(Constants.JOB_MANAGER_RPC_PORT_NAME) .withPort(RPC_PORT) .build(), new ServicePortBuilder() - .withName("blob-server-port") + .withName(Constants.BLOB_SERVER_PORT_NAME) .withPort(BLOB_SERVER_PORT) .build()); assertEquals(expectedServicePorts, internalService.getSpec().getPorts()); -- Gitee From 328193db25a089b321b80730e21bb122763565eb Mon Sep 17 00:00:00 2001 From: TsReaper Date: Mon, 16 Mar 2020 10:37:18 +0800 Subject: [PATCH 169/885] [FLINK-16541][doc] Fix document of table.exec.shuffle-mode This closed #11386 --- docs/_includes/generated/execution_config_configuration.html | 4 ++-- .../apache/flink/table/api/config/ExecutionConfigOptions.java | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/_includes/generated/execution_config_configuration.html b/docs/_includes/generated/execution_config_configuration.html index 80043227ac..6dcc484e3c 100644 --- a/docs/_includes/generated/execution_config_configuration.html +++ b/docs/_includes/generated/execution_config_configuration.html @@ -56,9 +56,9 @@ By default no operator is disabled.
    - +pipelined: the job will run in streaming mode, but it may cause resource deadlock that receiver waits for resource to start when the sender holds resource to wait to send data to the receiver. diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/ExecutionConfigOptions.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/ExecutionConfigOptions.java index e9f595fe79..054b81deb2 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/ExecutionConfigOptions.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/ExecutionConfigOptions.java @@ -217,8 +217,8 @@ public class ExecutionConfigOptions { public static final ConfigOption TABLE_EXEC_SHUFFLE_MODE = key("table.exec.shuffle-mode") .defaultValue("batch") - .withDescription("Sets exec shuffle mode. Only batch or pipeline can be set.\n" + + .withDescription("Sets exec shuffle mode. Only batch or pipelined can be set.\n" + "batch: the job will run stage by stage. \n" + - "pipeline: the job will run in streaming mode, but it may cause resource deadlock that receiver waits for resource to start when " + + "pipelined: the job will run in streaming mode, but it may cause resource deadlock that receiver waits for resource to start when " + "the sender holds resource to wait to send data to the receiver."); } -- Gitee From 608e9e78b51dae60814601b1bf5e7779a7cfeb67 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Fri, 28 Feb 2020 14:16:03 +0100 Subject: [PATCH 170/885] [FLINK-16336][table] Add new type inference for temporal table functions This closes #11253. --- .../functions/TemporalTableFunctionImpl.java | 15 ++++++++++++++- .../logical/FlinkLogicalTableFunctionScan.scala | 13 +++++++------ ...elateToJoinFromTemporalTableFunctionRule.scala | 13 +++++++------ .../plan/stream/sql/join/TemporalJoinTest.xml | 2 +- .../plan/stream/sql/join/TemporalJoinTest.scala | 2 +- .../runtime/stream/sql/TemporalJoinITCase.scala | 4 ++-- .../flink/table/planner/utils/TableTestBase.scala | 7 +++++++ 7 files changed, 39 insertions(+), 17 deletions(-) diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/functions/TemporalTableFunctionImpl.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/functions/TemporalTableFunctionImpl.java index 03967f491c..f3f31bf9e4 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/functions/TemporalTableFunctionImpl.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/functions/TemporalTableFunctionImpl.java @@ -20,8 +20,13 @@ package org.apache.flink.table.functions; import org.apache.flink.annotation.Internal; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.catalog.DataTypeFactory; import org.apache.flink.table.expressions.Expression; import org.apache.flink.table.operations.QueryOperation; +import org.apache.flink.table.types.inference.InputTypeStrategies; +import org.apache.flink.table.types.inference.TypeInference; +import org.apache.flink.table.types.inference.TypeStrategies; import java.sql.Timestamp; @@ -52,7 +57,7 @@ public final class TemporalTableFunctionImpl extends TemporalTableFunction { this.resultType = resultType; } - public void eval(Timestamp row) { + public void eval(Timestamp t) { throw new IllegalStateException("This should never be called"); } @@ -64,6 +69,14 @@ public final class TemporalTableFunctionImpl extends TemporalTableFunction { return primaryKey; } + @Override + public TypeInference getTypeInference(DataTypeFactory typeFactory) { + return TypeInference.newBuilder() + .inputTypeStrategy(InputTypeStrategies.explicitSequence(DataTypes.TIMESTAMP(3))) + .outputTypeStrategy(TypeStrategies.explicit(underlyingHistoryTable.getTableSchema().toRowDataType())) + .build(); + } + @Override public RowTypeInfo getResultType() { return resultType; diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalTableFunctionScan.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalTableFunctionScan.scala index 38f4b0392f..699b252ae3 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalTableFunctionScan.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalTableFunctionScan.scala @@ -21,7 +21,6 @@ package org.apache.flink.table.planner.plan.nodes.logical import org.apache.flink.table.functions.TemporalTableFunction import org.apache.flink.table.planner.functions.utils.TableSqlFunction import org.apache.flink.table.planner.plan.nodes.FlinkConventions - import org.apache.calcite.plan.{Convention, RelOptCluster, RelOptRuleCall, RelTraitSet} import org.apache.calcite.rel.RelNode import org.apache.calcite.rel.`type`.RelDataType @@ -30,10 +29,11 @@ import org.apache.calcite.rel.core.TableFunctionScan import org.apache.calcite.rel.logical.LogicalTableFunctionScan import org.apache.calcite.rel.metadata.RelColumnMapping import org.apache.calcite.rex.{RexCall, RexNode} - import java.lang.reflect.Type import java.util +import org.apache.flink.table.planner.functions.bridging.BridgingSqlFunction + /** * Sub-class of [[TableFunctionScan]] that is a relational expression * which calls a table-valued function in Flink. @@ -96,11 +96,12 @@ class FlinkLogicalTableFunctionScanConverter return false } val rexCall = logicalTableFunction.getCall.asInstanceOf[RexCall] - if (!rexCall.getOperator.isInstanceOf[TableSqlFunction]) { - return false + val functionDefinition = rexCall.getOperator match { + case tsf: TableSqlFunction => tsf.udtf + case bsf: BridgingSqlFunction => bsf.getDefinition + case _ => return false } - val tableFunction = rexCall.getOperator.asInstanceOf[TableSqlFunction] - tableFunction.udtf.isInstanceOf[TemporalTableFunction] + functionDefinition.isInstanceOf[TemporalTableFunction] } def convert(rel: RelNode): RelNode = { diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalCorrelateToJoinFromTemporalTableFunctionRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalCorrelateToJoinFromTemporalTableFunctionRule.scala index b40a7dd3cf..4ed2c0a17b 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalCorrelateToJoinFromTemporalTableFunctionRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalCorrelateToJoinFromTemporalTableFunctionRule.scala @@ -30,7 +30,6 @@ import org.apache.flink.table.planner.plan.utils.{ExpandTableScanShuttle, RexDef import org.apache.flink.table.types.logical.LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE import org.apache.flink.table.types.logical.utils.LogicalTypeChecks.{hasRoot, isProctimeAttribute} import org.apache.flink.util.Preconditions.checkState - import org.apache.calcite.plan.RelOptRule.{any, none, operand, some} import org.apache.calcite.plan.hep.HepRelVertex import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelOptSchema} @@ -38,6 +37,7 @@ import org.apache.calcite.rel.{BiRel, RelNode, SingleRel} import org.apache.calcite.rel.core.{JoinRelType, TableFunctionScan} import org.apache.calcite.rel.logical.LogicalCorrelate import org.apache.calcite.rex._ +import org.apache.flink.table.planner.functions.bridging.BridgingSqlFunction /** * The initial temporal TableFunction join (LATERAL TemporalTableFunction(o.proctime)) is @@ -188,16 +188,17 @@ class GetTemporalTableFunctionCall( } override def visitCall(rexCall: RexCall): TemporalTableFunctionCall = { - if (!rexCall.getOperator.isInstanceOf[TableSqlFunction]) { - return null + val functionDefinition = rexCall.getOperator match { + case tsf: TableSqlFunction => tsf.udtf + case bsf: BridgingSqlFunction => bsf.getDefinition + case _ => return null } - val tableFunction = rexCall.getOperator.asInstanceOf[TableSqlFunction] - if (!tableFunction.udtf.isInstanceOf[TemporalTableFunction]) { + if (!functionDefinition.isInstanceOf[TemporalTableFunction]) { return null } val temporalTableFunction = - tableFunction.udtf.asInstanceOf[TemporalTableFunctionImpl] + functionDefinition.asInstanceOf[TemporalTableFunctionImpl] checkState( rexCall.getOperands.size().equals(1), diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/TemporalJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/TemporalJoinTest.xml index 31b93bf758..a9789a440c 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/TemporalJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/TemporalJoinTest.xml @@ -53,7 +53,7 @@ LogicalProject(rate=[$0], secondary_key=[$1], t3_comment=[$2], t3_secondary_key= : +- LogicalFilter(condition=[OR(=($7, $3), =($9, $4))]) : +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{0}]) : :- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - : +- LogicalTableFunctionScan(invocation=[Rates($cor0.o_rowtime)], rowType=[RecordType(TIME ATTRIBUTE(ROWTIME) rowtime, VARCHAR(2147483647) comment, VARCHAR(2147483647) currency, INTEGER rate, INTEGER secondary_key)], elementType=[class [Ljava.lang.Object;]) + : +- LogicalTableFunctionScan(invocation=[Rates($cor0.o_rowtime)], rowType=[RecordType:peek_no_expand(TIME ATTRIBUTE(ROWTIME) rowtime, VARCHAR(2147483647) comment, VARCHAR(2147483647) currency, INTEGER rate, INTEGER secondary_key)]) +- LogicalTableScan(table=[[default_catalog, default_database, Table3]]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/TemporalJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/TemporalJoinTest.scala index fa2d0e16a7..58f43d9024 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/TemporalJoinTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/TemporalJoinTest.scala @@ -104,7 +104,7 @@ class TemporalJoinTest extends TableTestBase { val rates = util.tableEnv .sqlQuery("SELECT * FROM RatesHistory WHERE rate > 110") .createTemporalTableFunction("rowtime", "currency") - util.addFunction("Rates", rates) + util.addTemporarySystemFunction("Rates", rates) val sqlQuery = "SELECT * FROM " + diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TemporalJoinITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TemporalJoinITCase.scala index 6c57d818e9..e83f944649 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TemporalJoinITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TemporalJoinITCase.scala @@ -142,7 +142,7 @@ class TemporalJoinITCase(state: StateBackendMode) tEnv.registerTable("RatesHistory", ratesHistory) tEnv.registerTable("FilteredRatesHistory", tEnv.sqlQuery("SELECT * FROM RatesHistory WHERE rate > 110")) - tEnv.registerFunction( + tEnv.createTemporarySystemFunction( "Rates", tEnv .scan("FilteredRatesHistory") @@ -213,7 +213,7 @@ class TemporalJoinITCase(state: StateBackendMode) tEnv.createTemporaryView("Orders", orders) tEnv.createTemporaryView("RatesHistory", ratesHistory) - tEnv.registerFunction( + tEnv.createTemporarySystemFunction( "Rates", ratesHistory.createTemporalTableFunction("rowtime", "currency")) tEnv.registerFunction( diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala index 5e45312fc1..a93b5a2505 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala @@ -562,6 +562,13 @@ abstract class TableTestUtil( testingTableEnv.scan(name) } + /** + * Registers a [[UserDefinedFunction]] according to FLIP-65. + */ + def addTemporarySystemFunction(name: String, function: UserDefinedFunction): Unit = { + testingTableEnv.createTemporarySystemFunction(name, function) + } + /** * Registers a [[TableFunction]] under given name into the TableEnvironment's catalog. */ -- Gitee From 4a4baf0c81c23ed3125078500605a7db11f3cbba Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Mon, 16 Mar 2020 11:42:49 +0100 Subject: [PATCH 171/885] [FLINK-16613] make build trigger on azp more reliable --- azure-pipelines.yml | 4 ++++ tools/azure-pipelines/build-apache-repo.yml | 5 +++++ 2 files changed, 9 insertions(+) diff --git a/azure-pipelines.yml b/azure-pipelines.yml index d486643a5d..1257a49e62 100644 --- a/azure-pipelines.yml +++ b/azure-pipelines.yml @@ -29,6 +29,10 @@ # of the Azure Pipelines web ui. # +trigger: + branches: + include: + - '*' # must quote since "*" is a YAML reserved character; we want a string resources: containers: diff --git a/tools/azure-pipelines/build-apache-repo.yml b/tools/azure-pipelines/build-apache-repo.yml index 5940f4d1d5..05ee7e9210 100644 --- a/tools/azure-pipelines/build-apache-repo.yml +++ b/tools/azure-pipelines/build-apache-repo.yml @@ -29,6 +29,11 @@ schedules: - master always: true # run even if there were no changes to the mentioned branches +trigger: + branches: + include: + - '*' # must quote since "*" is a YAML reserved character; we want a string + resources: containers: # Container with Maven 3.2.5, SSL to have the same environment everywhere. -- Gitee From 4da44329e19f0b9bbcd7096b7632bc7b6a663627 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Thu, 12 Mar 2020 13:23:01 +0100 Subject: [PATCH 172/885] [FLINK-16573] Ensure Kinesis RecordFetcher threads shutdown on cancel The threads may not shut down correctly because they do not check for the running flag in the inner loops. The threads also do not get interrupted because they are not connected to the main task thread. These threads keep lingering around after the job has shut down: ``` Thread 23168: (state = BLOCKED) - java.lang.Object.wait(long) @bci=0 (Compiled frame; information may be imprecise) - org.apache.flink.streaming.connectors.kinesis.util.RecordEmitter.emitRecords() @bci=140, line=209 (Compiled frame) - org.apache.flink.streaming.connectors.kinesis.util.RecordEmitter.run() @bci=18, line=177 (Interpreted frame) - java.lang.Thread.run() @bci=11, line=748 (Compiled frame) ``` --- .../streaming/connectors/kinesis/util/RecordEmitter.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/RecordEmitter.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/RecordEmitter.java index dfa33883a2..6ce0c7c89c 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/RecordEmitter.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/RecordEmitter.java @@ -205,6 +205,10 @@ public abstract class RecordEmitter implements Runna } } } + if (!running) { + // Make sure we can exit this loop so the thread can shut down + break runLoop; + } } // wait until ready to emit min or another queue receives elements @@ -223,6 +227,10 @@ public abstract class RecordEmitter implements Runna continue runLoop; } } + if (!running) { + // Make sure we can exit this loop so the thread can shut down + break runLoop; + } } // emit up to queue capacity records -- Gitee From f062b95cea915e8440e17e44c6f78f41c59c6dd4 Mon Sep 17 00:00:00 2001 From: zhule Date: Thu, 12 Mar 2020 14:48:24 +0800 Subject: [PATCH 173/885] [FLINK-16096][docs-zh] Translate dev/table/catalog.zh.md into Chinese This closes #11390 --- docs/dev/table/catalogs.zh.md | 82 +++++++++++++++++------------------ 1 file changed, 41 insertions(+), 41 deletions(-) diff --git a/docs/dev/table/catalogs.zh.md b/docs/dev/table/catalogs.zh.md index 1814f5c02c..c569785f40 100644 --- a/docs/dev/table/catalogs.zh.md +++ b/docs/dev/table/catalogs.zh.md @@ -22,44 +22,44 @@ specific language governing permissions and limitations under the License. --> -Catalogs provide metadata, such as databases, tables, partitions, views, and functions and information needed to access data stored in a database or other external systems. +Catalog 提供了元数据信息,例如数据库、表、分区、视图以及数据库或其他外部系统中存储的函数和信息。 -One of the most crucial aspects of data processing is managing metadata. -It may be transient metadata like temporary tables, or UDFs registered against the table environment. -Or permanent metadata, like that in a Hive Metastore. Catalogs provide a unified API for managing metadata and making it accessible from the Table API and SQL Queries. +数据处理最关键的方面之一是管理元数据。 +元数据可以是临时的,例如临时表、或者通过 TableEnvironment 注册的 UDF。 +元数据也可以是持久化的,例如 Hive Metastore 中的元数据。Catalog 提供了一个统一的API,用于管理元数据,并使其可以从 Table API 和 SQL 查询语句中来访问。 * This will be replaced by the TOC {:toc} -## Catalog Types +## Catalog 类型 ### GenericInMemoryCatalog -The `GenericInMemoryCatalog` is an in-memory implementation of a catalog. All objects will be available only for the lifetime of the session. +`GenericInMemoryCatalog` 是基于内存实现的 Catalog,所有元数据只在 session 的生命周期内可用。 ### HiveCatalog -The `HiveCatalog` serves two purposes; as persistent storage for pure Flink metadata, and as an interface for reading and writing existing Hive metadata. -Flink's [Hive documentation]({{ site.baseurl }}/dev/table/hive/index.html) provides full details on setting up the catalog and interfacing with an existing Hive installation. +`HiveCatalog` 有两个用途:作为原生 Flink 元数据的持久化存储,以及作为读写现有 Hive 元数据的接口。 +Flink 的 [Hive 文档]({{ site.baseurl }}/zh/dev/table/hive/index.html) 提供了有关设置 `HiveCatalog` 以及访问现有 Hive 元数据的详细信息。 -{% warn %} The Hive Metastore stores all meta-object names in lower case. This is unlike `GenericInMemoryCatalog` which is case-sensitive +警告 Hive Metastore 以小写形式存储所有元数据对象名称。而 `GenericInMemoryCatalog` 区分大小写。 -### User-Defined Catalog +### 用户自定义 Catalog -Catalogs are pluggable and users can develop custom catalogs by implementing the `Catalog` interface. -To use custom catalogs in SQL CLI, users should develop both a catalog and its corresponding catalog factory by implementing the `CatalogFactory` interface. +Catalog 是可扩展的,用户可以通过实现 `Catalog` 接口来开发自定义 Catalog。 +想要在 SQL CLI 中使用自定义 Catalog,用户除了需要实现自定义的 Catalog 之外,还需要为这个 Catalog 实现对应的 `CatalogFactory` 接口。 -The catalog factory defines a set of properties for configuring the catalog when the SQL CLI bootstraps. -The set of properties will be passed to a discovery service where the service tries to match the properties to a `CatalogFactory` and initiate a corresponding catalog instance. +`CatalogFactory` 定义了一组属性,用于 SQL CLI 启动时配置 Catalog。 +这组属性集将传递给发现服务,在该服务中,服务会尝试将属性关联到 `CatalogFactory` 并初始化相应的 Catalog 实例。 -## How to Create and Register Flink Tables to Catalog +## 如何创建 Flink 表并将其注册到 Catalog -### Using SQL DDL +### 使用 SQL DDL -Users can use SQL DDL to create tables in catalogs in both Table API and SQL. +用户可以使用 DDL 通过 Table API 或者 SQL Client 在 Catalog 中创建表。 -For Table API: +使用 Table API:
    @@ -84,7 +84,7 @@ tableEnv.listTables(); // should return the tables in current catalog and databa
    -For SQL Client: +使用 SQL Client: {% highlight sql %} // the catalog should have been registered via yaml file @@ -96,11 +96,11 @@ Flink SQL> SHOW TABLES; mytable {% endhighlight %} -For detailed information, please check out [Flink SQL CREATE DDL]({{ site.baseurl }}/dev/table/sql/create.html). +更多详细信息,请参考[Flink SQL CREATE DDL]({{ site.baseurl }}/zh/dev/table/sql/create.html)。 -### Using Java/Scala/Python API +### 使用 Java/Scala/Python API -Users can use Java, Scala, or Python API to create catalog tables programmatically. +用户可以用编程的方式使用Java、Scala 或者 Python API 来创建 Catalog 表。
    @@ -142,11 +142,11 @@ List tables = catalog.listTables("mydb"); // tables should contain "myta ## Catalog API -Note: only catalog program APIs are listed here. Users can achieve many of the same funtionalities with SQL DDL. -For detailed DDL information, please refer to [SQL CREATE DDL]({{ site.baseurl }}/dev/table/sql/create.html). +注意:这里只列出了编程方式的 Catalog API,用户可以使用 SQL DDL 实现许多相同的功能。 +关于 DDL 的详细信息请参考 [SQL CREATE DDL]({{ site.baseurl }}/zh/dev/table/sql/create.html)。 -### Database operations +### 数据库操作
    @@ -172,7 +172,7 @@ catalog.listDatabases("mycatalog");
    -### Table operations +### 表操作
    @@ -201,7 +201,7 @@ catalog.listTables("mydb");
    -### View operations +### 视图操作
    @@ -231,7 +231,7 @@ catalog.listViews("mydb");
    -### Partition operations +### 分区操作
    @@ -272,7 +272,7 @@ catalog.listPartitions(new ObjectPath("mydb", "mytable"), Arrays.asList(epr1, ..
    -### Function operations +### 函数操作
    @@ -299,12 +299,12 @@ catalog.listFunctions("mydb");
    -## Table API and SQL for Catalog +## 通过 Table API 和 SQL Client 操作 Catalog -### Registering a Catalog +### 注册 Catalog -Users have access to a default in-memory catalog named `default_catalog`, that is always created by default. This catalog by default has a single database called `default_database`. -Users can also register additional catalogs into an existing Flink session. +用户可以访问默认创建的内存 Catalog `default_catalog`,这个 Catalog 默认拥有一个默认数据库 `default_database`。 +用户也可以注册其他的 Catalog 到现有的 Flink 会话中。
    @@ -314,8 +314,8 @@ tableEnv.registerCatalog(new CustomCatalog("myCatalog"));
    -All catalogs defined using YAML must provide a `type` property that specifies the type of catalog. -The following types are supported out of the box. +使用 YAML 定义的 Catalog 必须提供 `type` 属性,以表示指定的 Catalog 类型。 +以下几种类型可以直接使用。
    Pattern OperationDescription模式操作描述
    begin(#name) -

    Defines a starting pattern:

    +

    定义一个开始的模式:

    {% highlight java %} Pattern start = Pattern.begin("start"); {% endhighlight %} @@ -1025,7 +1000,7 @@ Pattern start = Pattern.begin("start");
    begin(#pattern_sequence) -

    Defines a starting pattern:

    +

    定义一个开始的模式:

    {% highlight java %} Pattern start = Pattern.begin( Pattern.begin("start").where(...).followedBy("middle").where(...) @@ -1036,8 +1011,7 @@ Pattern start = Pattern.begin(
    next(#name) -

    Appends a new pattern. A matching event has to directly succeed the previous matching event - (strict contiguity):

    +

    增加一个新的模式。匹配的事件必须是直接跟在前面匹配到的事件后面(严格连续):

    {% highlight java %} Pattern next = start.next("middle"); {% endhighlight %} @@ -1046,8 +1020,7 @@ Pattern next = start.next("middle");
    next(#pattern_sequence) -

    Appends a new pattern. A sequence of matching events have to directly succeed the previous matching event - (strict contiguity):

    +

    增加一个新的模式。匹配的事件序列必须是直接跟在前面匹配到的事件后面(严格连续):

    {% highlight java %} Pattern next = start.next( Pattern.begin("start").where(...).followedBy("middle").where(...) @@ -1058,8 +1031,7 @@ Pattern next = start.next(
    followedBy(#name) -

    Appends a new pattern. Other events can occur between a matching event and the previous - matching event (relaxed contiguity):

    +

    增加一个新的模式。可以有其他事件出现在匹配的事件和之前匹配到的事件中间(松散连续):

    {% highlight java %} Pattern followedBy = start.followedBy("middle"); {% endhighlight %} @@ -1068,8 +1040,7 @@ Pattern followedBy = start.followedBy("middle");
    followedBy(#pattern_sequence) -

    Appends a new pattern. Other events can occur between a sequence of matching events and the previous - matching event (relaxed contiguity):

    +

    增加一个新的模式。可以有其他事件出现在匹配的事件序列和之前匹配到的事件中间(松散连续):

    {% highlight java %} Pattern followedBy = start.followedBy( Pattern.begin("start").where(...).followedBy("middle").where(...) @@ -1080,9 +1051,8 @@ Pattern followedBy = start.followedBy(
    followedByAny(#name) -

    Appends a new pattern. Other events can occur between a matching event and the previous - matching event, and alternative matches will be presented for every alternative matching event - (non-deterministic relaxed contiguity):

    +

    增加一个新的模式。可以有其他事件出现在匹配的事件和之前匹配到的事件中间, + 每个可选的匹配事件都会作为可选的匹配结果输出(不确定的松散连续):

    {% highlight java %} Pattern followedByAny = start.followedByAny("middle"); {% endhighlight %} @@ -1091,9 +1061,8 @@ Pattern followedByAny = start.followedByAny("middle");
    followedByAny(#pattern_sequence) -

    Appends a new pattern. Other events can occur between a sequence of matching events and the previous - matching event, and alternative matches will be presented for every alternative sequence of matching events - (non-deterministic relaxed contiguity):

    +

    增加一个新的模式。可以有其他事件出现在匹配的事件序列和之前匹配到的事件中间, + 每个可选的匹配事件序列都会作为可选的匹配结果输出(不确定的松散连续):

    {% highlight java %} Pattern followedByAny = start.followedByAny( Pattern.begin("start").where(...).followedBy("middle").where(...) @@ -1104,8 +1073,7 @@ Pattern followedByAny = start.followedByAny(
    notNext() -

    Appends a new negative pattern. A matching (negative) event has to directly succeed the - previous matching event (strict contiguity) for the partial match to be discarded:

    +

    增加一个新的否定模式。匹配的(否定)事件必须直接跟在前面匹配到的事件之后(严格连续)来丢弃这些部分匹配:

    {% highlight java %} Pattern notNext = start.notNext("not"); {% endhighlight %} @@ -1114,9 +1082,8 @@ Pattern notNext = start.notNext("not");
    notFollowedBy() -

    Appends a new negative pattern. A partial matching event sequence will be discarded even - if other events occur between the matching (negative) event and the previous matching event - (relaxed contiguity):

    +

    增加一个新的否定模式。即使有其他事件在匹配的(否定)事件和之前匹配的事件之间发生, + 部分匹配的事件序列也会被丢弃(松散连续):

    {% highlight java %} Pattern notFollowedBy = start.notFollowedBy("not"); {% endhighlight %} @@ -1125,8 +1092,7 @@ Pattern notFollowedBy = start.notFollowedBy("not");
    within(time) -

    Defines the maximum time interval for an event sequence to match the pattern. If a non-completed event - sequence exceeds this time, it is discarded:

    +

    定义匹配模式的事件序列出现的最大时间间隔。如果未完成的事件序列超过了这个事件,就会被丢弃:

    {% highlight java %} pattern.within(Time.seconds(10)); {% endhighlight %} @@ -1140,15 +1106,15 @@ pattern.within(Time.seconds(10)); - - + + diff --git a/docs/dev/table/sql/queries.zh.md b/docs/dev/table/sql/queries.zh.md index 0d68ae4671..711f15caf3 100644 --- a/docs/dev/table/sql/queries.zh.md +++ b/docs/dev/table/sql/queries.zh.md @@ -406,7 +406,7 @@ SELECT PRETTY_PRINT(user) FROM Orders @@ -545,7 +547,7 @@ FROM Orders INNER JOIN Product ON Orders.productId = Product.id - - - + + + diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java index 2e44b05725..fff5c13bd8 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java @@ -39,12 +39,12 @@ public class KubernetesConfigOptions { "for interacting with the cluster. This could be helpful if one has multiple contexts configured and " + "wants to administrate different Flink clusters on different Kubernetes clusters/contexts."); - public static final ConfigOption REST_SERVICE_EXPOSED_TYPE = + public static final ConfigOption REST_SERVICE_EXPOSED_TYPE = key("kubernetes.rest-service.exposed.type") - .stringType() - .defaultValue(ServiceExposedType.LoadBalancer.toString()) - .withDescription("It could be ClusterIP/NodePort/LoadBalancer(default). When set to ClusterIP, the rest service" + - "will not be created."); + .enumType(ServiceExposedType.class) + .defaultValue(ServiceExposedType.LoadBalancer) + .withDescription("The type of the rest service (ClusterIP or NodePort or LoadBalancer). " + + "When set to ClusterIP, the rest service will not be created."); public static final ConfigOption JOB_MANAGER_SERVICE_ACCOUNT = key("kubernetes.jobmanager.service-account") diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java index e347e870b6..df0af2a3a4 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java @@ -127,10 +127,11 @@ public class Fabric8FlinkKubeClient implements FlinkKubeClient { @Nullable public Endpoint getRestEndpoint(String clusterId) { int restPort = this.flinkConfig.getInteger(RestOptions.PORT); - String serviceExposedType = flinkConfig.getString(KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE); + final KubernetesConfigOptions.ServiceExposedType serviceExposedType = + flinkConfig.get(KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE); // Return the service.namespace directly when use ClusterIP. - if (serviceExposedType.equals(KubernetesConfigOptions.ServiceExposedType.ClusterIP.toString())) { + if (serviceExposedType == KubernetesConfigOptions.ServiceExposedType.ClusterIP) { return new Endpoint(KubernetesUtils.getInternalServiceName(clusterId) + "." + nameSpace, restPort); } diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/AbstractServiceDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/AbstractServiceDecorator.java index 77a584bce2..7a54599452 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/AbstractServiceDecorator.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/AbstractServiceDecorator.java @@ -19,6 +19,7 @@ package org.apache.flink.kubernetes.kubeclient.decorators; import org.apache.flink.configuration.RestOptions; +import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters; import org.apache.flink.kubernetes.utils.Constants; @@ -55,7 +56,7 @@ public abstract class AbstractServiceDecorator extends AbstractKubernetesStepDec .withLabels(kubernetesJobManagerParameters.getCommonLabels()) .endMetadata() .withNewSpec() - .withType(getServiceType()) + .withType(getServiceType().name()) .withPorts(getServicePorts()) .withSelector(kubernetesJobManagerParameters.getLabels()) .endSpec() @@ -64,7 +65,7 @@ public abstract class AbstractServiceDecorator extends AbstractKubernetesStepDec return Collections.singletonList(service); } - protected abstract String getServiceType(); + protected abstract KubernetesConfigOptions.ServiceExposedType getServiceType(); protected abstract String getServiceName(); diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecorator.java index 21ee943f2f..3c05915e30 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecorator.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecorator.java @@ -39,8 +39,7 @@ public class ExternalServiceDecorator extends AbstractServiceDecorator { @Override public List buildAccompanyingKubernetesResources() throws IOException { - if (kubernetesJobManagerParameters.getRestServiceExposedType().equals( - KubernetesConfigOptions.ServiceExposedType.ClusterIP.name())) { + if (kubernetesJobManagerParameters.getRestServiceExposedType() == KubernetesConfigOptions.ServiceExposedType.ClusterIP) { return Collections.emptyList(); } @@ -48,7 +47,7 @@ public class ExternalServiceDecorator extends AbstractServiceDecorator { } @Override - protected String getServiceType() { + protected KubernetesConfigOptions.ServiceExposedType getServiceType() { return kubernetesJobManagerParameters.getRestServiceExposedType(); } diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InternalServiceDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InternalServiceDecorator.java index 971ee94c82..0258b0faa3 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InternalServiceDecorator.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InternalServiceDecorator.java @@ -69,8 +69,8 @@ public class InternalServiceDecorator extends AbstractServiceDecorator { } @Override - protected String getServiceType() { - return KubernetesConfigOptions.ServiceExposedType.ClusterIP.name(); + protected KubernetesConfigOptions.ServiceExposedType getServiceType() { + return KubernetesConfigOptions.ServiceExposedType.ClusterIP; } @Override diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java index bc44444266..3af42d55fe 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java @@ -100,8 +100,7 @@ public class KubernetesJobManagerParameters extends AbstractKubernetesParameters return entrypointClass; } - public String getRestServiceExposedType() { - final String exposedType = flinkConfig.getString(KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE); - return KubernetesConfigOptions.ServiceExposedType.valueOf(exposedType).name(); + public KubernetesConfigOptions.ServiceExposedType getRestServiceExposedType() { + return flinkConfig.get(KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE); } } diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecoratorTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecoratorTest.java index c368d4132d..2d0e1b030a 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecoratorTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecoratorTest.java @@ -80,11 +80,14 @@ public class ExternalServiceDecoratorTest extends KubernetesJobManagerTestBase { @Test public void testSetServiceExposedType() throws IOException { - this.flinkConfig.set(KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE, "NodePort"); + this.flinkConfig.set(KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE, + KubernetesConfigOptions.ServiceExposedType.NodePort); List resources = this.externalServiceDecorator.buildAccompanyingKubernetesResources(); - assertEquals("NodePort", ((Service) resources.get(0)).getSpec().getType()); + assertEquals(KubernetesConfigOptions.ServiceExposedType.NodePort.name(), + ((Service) resources.get(0)).getSpec().getType()); - this.flinkConfig.set(KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE, "ClusterIP"); + this.flinkConfig.set(KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE, + KubernetesConfigOptions.ServiceExposedType.ClusterIP); assertTrue(this.externalServiceDecorator.buildAccompanyingKubernetesResources().isEmpty()); } } diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParametersTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParametersTest.java index 46e714d47f..2870c14ea9 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParametersTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParametersTest.java @@ -146,8 +146,8 @@ public class KubernetesJobManagerParametersTest { @Test public void testGetRestServiceExposedType() { flinkConfig.set(KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE, - KubernetesConfigOptions.ServiceExposedType.NodePort.name()); - assertEquals(KubernetesConfigOptions.ServiceExposedType.NodePort.name(), + KubernetesConfigOptions.ServiceExposedType.NodePort); + assertEquals(KubernetesConfigOptions.ServiceExposedType.NodePort, kubernetesJobManagerParameters.getRestServiceExposedType()); } } -- Gitee From bb8e3876ba37405cf0e703b51956fa0792649b5a Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Tue, 3 Mar 2020 16:46:31 +0100 Subject: [PATCH 141/885] [FLINK-16406] Increase default value for JVM Metaspace to minimise its OutOfMemoryError This closes #11300. --- docs/_includes/generated/common_memory_section.html | 2 +- docs/_includes/generated/task_manager_memory_configuration.html | 2 +- .../java/org/apache/flink/configuration/TaskManagerOptions.java | 2 +- flink-dist/src/main/resources/flink-conf.yaml | 2 +- .../test-scripts/test_rocksdb_state_memory_control.sh | 2 +- .../kubernetes/kubeclient/KubernetesTaskManagerTestBase.java | 2 +- .../kubeclient/decorators/JavaCmdTaskManagerDecoratorTest.java | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/_includes/generated/common_memory_section.html b/docs/_includes/generated/common_memory_section.html index 713edff3b2..329916fb92 100644 --- a/docs/_includes/generated/common_memory_section.html +++ b/docs/_includes/generated/common_memory_section.html @@ -28,7 +28,7 @@ - + diff --git a/docs/_includes/generated/task_manager_memory_configuration.html b/docs/_includes/generated/task_manager_memory_configuration.html index a7777e4753..3a25b14bbd 100644 --- a/docs/_includes/generated/task_manager_memory_configuration.html +++ b/docs/_includes/generated/task_manager_memory_configuration.html @@ -28,7 +28,7 @@ - + diff --git a/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java index a5014c817a..e2477fc472 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java @@ -415,7 +415,7 @@ public class TaskManagerOptions { public static final ConfigOption JVM_METASPACE = key("taskmanager.memory.jvm-metaspace.size") .memoryType() - .defaultValue(MemorySize.parse("96m")) + .defaultValue(MemorySize.parse("256m")) .withDescription("JVM Metaspace Size for the TaskExecutors."); /** diff --git a/flink-dist/src/main/resources/flink-conf.yaml b/flink-dist/src/main/resources/flink-conf.yaml index 084f24a3ea..f58c04bc85 100644 --- a/flink-dist/src/main/resources/flink-conf.yaml +++ b/flink-dist/src/main/resources/flink-conf.yaml @@ -46,7 +46,7 @@ jobmanager.heap.size: 1024m # # Note this accounts for all memory usage within the TaskManager process, including JVM metaspace and other overhead. -taskmanager.memory.process.size: 1568m +taskmanager.memory.process.size: 1728m # To exclude JVM metaspace and overhead, please, use total Flink memory size instead of 'taskmanager.memory.process.size'. # It is not recommended to set both 'taskmanager.memory.process.size' and Flink memory. diff --git a/flink-end-to-end-tests/test-scripts/test_rocksdb_state_memory_control.sh b/flink-end-to-end-tests/test-scripts/test_rocksdb_state_memory_control.sh index 0674f72da8..5d15d0e022 100755 --- a/flink-end-to-end-tests/test-scripts/test_rocksdb_state_memory_control.sh +++ b/flink-end-to-end-tests/test-scripts/test_rocksdb_state_memory_control.sh @@ -41,7 +41,7 @@ CHECKPOINT_DIR_URI="file://$CHECKPOINT_DIR" EXPECTED_MAX_MEMORY_USAGE=200000000 set_config_key "taskmanager.numberOfTaskSlots" "$PARALLELISM" -set_config_key "taskmanager.memory.process.size" "1024m" +set_config_key "taskmanager.memory.process.size" "1184m" set_config_key "taskmanager.memory.managed.size" "300m" set_config_key "state.backend.rocksdb.memory.managed" "true" set_config_key "state.backend.rocksdb.memory.write-buffer-ratio" "0.8" diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/KubernetesTaskManagerTestBase.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/KubernetesTaskManagerTestBase.java index a1799a073a..c4308a92f2 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/KubernetesTaskManagerTestBase.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/KubernetesTaskManagerTestBase.java @@ -42,7 +42,7 @@ public class KubernetesTaskManagerTestBase extends KubernetesTestBase { protected static final String POD_NAME = "taskmanager-pod-1"; private static final String DYNAMIC_PROPERTIES = ""; - protected static final int TOTAL_PROCESS_MEMORY = 1024; + protected static final int TOTAL_PROCESS_MEMORY = 1184; protected static final double TASK_MANAGER_CPU = 2.0; protected final Map customizedEnvs = new HashMap() { diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/JavaCmdTaskManagerDecoratorTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/JavaCmdTaskManagerDecoratorTest.java index f3653a8e4b..711c552f93 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/JavaCmdTaskManagerDecoratorTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/JavaCmdTaskManagerDecoratorTest.java @@ -52,7 +52,7 @@ public class JavaCmdTaskManagerDecoratorTest extends KubernetesTaskManagerTestBa private static final String jvmOpts = "-Djvm"; private static final String tmJvmMem = - "-Xmx251658235 -Xms251658235 -XX:MaxDirectMemorySize=211392922 -XX:MaxMetaspaceSize=100663296"; + "-Xmx251658235 -Xms251658235 -XX:MaxDirectMemorySize=211392922 -XX:MaxMetaspaceSize=268435456"; private static final String mainClass = KubernetesTaskExecutorRunner.class.getCanonicalName(); private String mainClassArgs; -- Gitee From 42f49482a3217287abba5e8903cf50791acf287a Mon Sep 17 00:00:00 2001 From: Wei Zhong Date: Thu, 12 Mar 2020 18:26:56 +0800 Subject: [PATCH 142/885] [FLINK-16304][python] Remove python packages bundled in the flink-python jar. (#11238) --- flink-dist/src/main/assemblies/bin.xml | 3 - flink-dist/src/main/assemblies/opt.xml | 7 ++ flink-python/MANIFEST.in | 1 + flink-python/bin/pyflink-shell.sh | 17 ++--- .../licenses => lib}/LICENSE.cloudpickle | 0 .../lib/{py4j-LICENSE.txt => LICENSE.py4j} | 0 flink-python/lib/cloudpickle-LICENSE.txt | 32 --------- flink-python/pom.xml | 17 ----- .../tests/test_process_mode_boot.py | 3 +- flink-python/pyflink/shell.py | 21 ------ .../client/python/PythonDriverEnvUtils.java | 50 ++++++++++---- .../python/PythonResourceExtractor.java | 48 ------------- .../python/AbstractPythonFunctionRunner.java | 18 ----- .../env/ProcessPythonEnvironmentManager.java | 16 +---- .../flink/python/util/ResourceUtil.java | 68 ++++++------------- .../src/main/resources/META-INF/NOTICE | 1 - .../main/resources}/pyflink-udf-runner.sh | 0 .../ProcessPythonEnvironmentManagerTest.java | 12 +--- .../flink/python/util/ResourceUtilTest.java | 30 ++------ tools/releasing/NOTICE-binary_PREAMBLE.txt | 5 ++ tools/releasing/collect_license_files.sh | 5 ++ 21 files changed, 92 insertions(+), 262 deletions(-) rename flink-python/{src/main/resources/META-INF/licenses => lib}/LICENSE.cloudpickle (100%) rename flink-python/lib/{py4j-LICENSE.txt => LICENSE.py4j} (100%) delete mode 100644 flink-python/lib/cloudpickle-LICENSE.txt delete mode 100644 flink-python/src/main/java/org/apache/flink/client/python/PythonResourceExtractor.java rename flink-python/{bin => src/main/resources}/pyflink-udf-runner.sh (100%) diff --git a/flink-dist/src/main/assemblies/bin.xml b/flink-dist/src/main/assemblies/bin.xml index bcb069ff58..08c8fa7e35 100644 --- a/flink-dist/src/main/assemblies/bin.xml +++ b/flink-dist/src/main/assemblies/bin.xml @@ -125,9 +125,6 @@ under the License. ../flink-python/bin/ bin 0755 - - pyflink-udf-runner.sh - diff --git a/flink-dist/src/main/assemblies/opt.xml b/flink-dist/src/main/assemblies/opt.xml index 7f900f8faa..d515d587b0 100644 --- a/flink-dist/src/main/assemblies/opt.xml +++ b/flink-dist/src/main/assemblies/opt.xml @@ -184,6 +184,13 @@ 0644 + + + ../flink-python/lib/ + opt/python/ + 0644 + + false diff --git a/flink-python/MANIFEST.in b/flink-python/MANIFEST.in index a6e3bf5200..a8e2d619bf 100644 --- a/flink-python/MANIFEST.in +++ b/flink-python/MANIFEST.in @@ -29,3 +29,4 @@ include README.md include pyflink/LICENSE include pyflink/NOTICE include pyflink/README.txt +recursive-exclude deps/opt/python * diff --git a/flink-python/bin/pyflink-shell.sh b/flink-python/bin/pyflink-shell.sh index f4a83b7479..b54c4b1551 100755 --- a/flink-python/bin/pyflink-shell.sh +++ b/flink-python/bin/pyflink-shell.sh @@ -34,6 +34,12 @@ PYFLINK_PYTHON="${PYFLINK_PYTHON:-"python"}" export FLINK_BIN_DIR=$FLINK_BIN_DIR export FLINK_HOME +# Add pyflink & py4j & cloudpickle to PYTHONPATH +export PYTHONPATH="$FLINK_OPT_DIR/python/pyflink.zip:$PYTHONPATH" +PY4J_ZIP=`echo "$FLINK_OPT_DIR"/python/py4j-*-src.zip` +CLOUDPICKLE_ZIP=`echo "$FLINK_OPT_DIR"/python/cloudpickle-*-src.zip` +export PYTHONPATH="$PY4J_ZIP:$CLOUDPICKLE_ZIP:$PYTHONPATH" + PARSER="org.apache.flink.client.python.PythonShellParser" function parse_options() { ${JAVA_RUN} ${JVM_ARGS} -cp ${FLINK_CLASSPATH}:${PYTHON_JAR_PATH} ${PARSER} "$@" @@ -73,17 +79,6 @@ OPTIONS=("${OPTIONS[@]:0:$LAST}") export SUBMIT_ARGS=${OPTIONS[@]} -EXTRACTOR="org.apache.flink.client.python.PythonResourceExtractor" -PYFLINK_INTERNAL_LIB=`${JAVA_RUN} ${JVM_ARGS} -cp ${PYTHON_JAR_PATH} ${EXTRACTOR}` -export PYTHONPATH="$PYFLINK_INTERNAL_LIB:$PYTHONPATH" -export PYFLINK_INTERNAL_LIB - # -i: interactive # -m: execute shell.py in the zip package ${PYFLINK_PYTHON} -i -m pyflink.shell -IFS=':' read -ra LIBS_TO_DELETE <<< "$PYFLINK_INTERNAL_LIB" -for lib_to_delete in "${LIBS_TO_DELETE[@]}"; do - if [[ -f ${lib_to_delete} ]]; then - rm ${lib_to_delete} - fi -done diff --git a/flink-python/src/main/resources/META-INF/licenses/LICENSE.cloudpickle b/flink-python/lib/LICENSE.cloudpickle similarity index 100% rename from flink-python/src/main/resources/META-INF/licenses/LICENSE.cloudpickle rename to flink-python/lib/LICENSE.cloudpickle diff --git a/flink-python/lib/py4j-LICENSE.txt b/flink-python/lib/LICENSE.py4j similarity index 100% rename from flink-python/lib/py4j-LICENSE.txt rename to flink-python/lib/LICENSE.py4j diff --git a/flink-python/lib/cloudpickle-LICENSE.txt b/flink-python/lib/cloudpickle-LICENSE.txt deleted file mode 100644 index d112c4806a..0000000000 --- a/flink-python/lib/cloudpickle-LICENSE.txt +++ /dev/null @@ -1,32 +0,0 @@ -This module was extracted from the `cloud` package, developed by -PiCloud, Inc. - -Copyright (c) 2015, Cloudpickle contributors. -Copyright (c) 2012, Regents of the University of California. -Copyright (c) 2009 PiCloud, Inc. http://www.picloud.com. -All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions -are met: - * Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above copyright - notice, this list of conditions and the following disclaimer in the - documentation and/or other materials provided with the distribution. - * Neither the name of the University of California, Berkeley nor the - names of its contributors may be used to endorse or promote - products derived from this software without specific prior written - permission. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED -TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR -PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF -LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING -NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS -SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/flink-python/pom.xml b/flink-python/pom.xml index f8c279e4d0..7f2938391b 100644 --- a/flink-python/pom.xml +++ b/flink-python/pom.xml @@ -200,18 +200,6 @@ under the License. - - lib - - *.zip - - - - bin - - pyflink-udf-runner.sh - - src/main/resources @@ -315,11 +303,6 @@ under the License. py4j org.apache.flink.api.python.shaded.py4j - - - py4j/** - net.razorvine diff --git a/flink-python/pyflink/fn_execution/tests/test_process_mode_boot.py b/flink-python/pyflink/fn_execution/tests/test_process_mode_boot.py index 05dbf29e88..f5d73523af 100644 --- a/flink-python/pyflink/fn_execution/tests/test_process_mode_boot.py +++ b/flink-python/pyflink/fn_execution/tests/test_process_mode_boot.py @@ -103,7 +103,8 @@ class PythonBootTests(PyFlinkTestCase): # assume that this file is in flink-python source code directory. flink_python_source_root = os.path.dirname( os.path.dirname(os.path.dirname(os.path.dirname(os.path.abspath(__file__))))) - self.runner_path = os.path.join(flink_python_source_root, "bin", "pyflink-udf-runner.sh") + self.runner_path = os.path.join( + flink_python_source_root, "src", "main", "resources", "pyflink-udf-runner.sh") def run_boot_py(self): args = [self.runner_path, "--id", "1", diff --git a/flink-python/pyflink/shell.py b/flink-python/pyflink/shell.py index 4a2b3739bd..604779f8a5 100644 --- a/flink-python/pyflink/shell.py +++ b/flink-python/pyflink/shell.py @@ -16,11 +16,7 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ -import atexit -import codecs -import os import platform -import signal import sys from pyflink.common import * @@ -31,23 +27,6 @@ from pyflink.table.catalog import * from pyflink.table.descriptors import * from pyflink.table.window import * - -def _register_exit_handler(): - def clean(*args, **kwargs): - try: - if "PYFLINK_INTERNAL_LIB" in os.environ: - files = os.environ["PYFLINK_INTERNAL_LIB"].split(os.pathsep) - for file in files: - if os.path.exists(file): - os.remove(file) - finally: - sys.exit() - atexit.register(clean) - # we already ignore the SIGINT so only process the SIGTERM - signal.signal(signal.SIGTERM, clean) - - -_register_exit_handler() utf8_out = open(sys.stdout.fileno(), mode='w', encoding='utf8', buffering=1) print("Using Python version %s (%s, %s)" % ( diff --git a/flink-python/src/main/java/org/apache/flink/client/python/PythonDriverEnvUtils.java b/flink-python/src/main/java/org/apache/flink/client/python/PythonDriverEnvUtils.java index 7637c5aac9..54ca9e697c 100644 --- a/flink-python/src/main/java/org/apache/flink/client/python/PythonDriverEnvUtils.java +++ b/flink-python/src/main/java/org/apache/flink/client/python/PythonDriverEnvUtils.java @@ -20,6 +20,7 @@ package org.apache.flink.client.python; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.Path; import org.apache.flink.util.FileUtils; @@ -29,8 +30,12 @@ import org.slf4j.LoggerFactory; import java.io.File; import java.io.IOException; +import java.nio.file.FileSystems; +import java.nio.file.FileVisitResult; import java.nio.file.Files; import java.nio.file.Paths; +import java.nio.file.SimpleFileVisitor; +import java.nio.file.attribute.BasicFileAttributes; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -39,14 +44,16 @@ import java.util.List; import java.util.Map; import java.util.UUID; -import static org.apache.flink.python.util.ResourceUtil.extractBuiltInDependencies; - /** * The util class help to prepare Python env and run the python process. */ public final class PythonDriverEnvUtils { private static final Logger LOG = LoggerFactory.getLogger(PythonDriverEnvUtils.class); + private static final String FLINK_OPT_DIR = System.getenv(ConfigConstants.ENV_FLINK_OPT_DIR); + + private static final String FLINK_OPT_DIR_PYTHON = FLINK_OPT_DIR + File.separator + "python"; + @VisibleForTesting public static final String PYFLINK_PY_FILES = "PYFLINK_PY_FILES"; @@ -115,18 +122,12 @@ public final class PythonDriverEnvUtils { fs.mkdirs(tmpDirPath); env.tempDirectory = tmpDir; - - // 2. append the internal lib files to PYTHONPATH. List pythonPathList = new ArrayList<>(); - List internalLibs = extractBuiltInDependencies( - tmpDir, - UUID.randomUUID().toString(), - true); - - for (File file: internalLibs) { - pythonPathList.add(file.getAbsolutePath()); - file.deleteOnExit(); + // 2. append the internal lib files to PYTHONPATH. + List pythonLibs = getLibFiles(FLINK_OPT_DIR_PYTHON); + for (java.nio.file.Path lib: pythonLibs) { + pythonPathList.add(lib.toFile().getAbsolutePath()); } // 3. copy relevant python files to tmp dir and set them in PYTHONPATH. @@ -200,6 +201,31 @@ public final class PythonDriverEnvUtils { } } + /** + * Gets pyflink dependent libs in specified directory. + * + * @param libDir The lib directory + */ + public static List getLibFiles(String libDir) { + final List libFiles = new ArrayList<>(); + SimpleFileVisitor finder = new SimpleFileVisitor() { + @Override + public FileVisitResult visitFile(java.nio.file.Path file, BasicFileAttributes attrs) throws IOException { + // only include zip file + if (file.toString().endsWith(".zip")) { + libFiles.add(file); + } + return FileVisitResult.CONTINUE; + } + }; + try { + Files.walkFileTree(FileSystems.getDefault().getPath(libDir), finder); + } catch (IOException e) { + LOG.error("Gets pyflink dependent libs failed.", e); + } + return libFiles; + } + /** * Starts python process. * diff --git a/flink-python/src/main/java/org/apache/flink/client/python/PythonResourceExtractor.java b/flink-python/src/main/java/org/apache/flink/client/python/PythonResourceExtractor.java deleted file mode 100644 index e56399b371..0000000000 --- a/flink-python/src/main/java/org/apache/flink/client/python/PythonResourceExtractor.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * 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.client.python; - -import java.io.File; -import java.io.IOException; -import java.util.List; -import java.util.UUID; -import java.util.stream.Collectors; - -import static org.apache.flink.python.util.ResourceUtil.extractBuiltInDependencies; - -/** - * The program that extracts the internal python libraries and join their absolute paths to append to PYTHONPATH. it can - * accept one argument as the temp directory. - */ -public class PythonResourceExtractor { - - public static void main(String[] args) throws IOException, InterruptedException { - String tmpdir = System.getProperty("java.io.tmpdir"); - - List files = extractBuiltInDependencies( - tmpdir, - UUID.randomUUID().toString(), - true); - - System.out.print( - files.stream() - .map(File::getAbsolutePath) - .collect(Collectors.joining(File.pathSeparator))); - } -} diff --git a/flink-python/src/main/java/org/apache/flink/python/AbstractPythonFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/python/AbstractPythonFunctionRunner.java index 99f78b3ed8..597057e091 100644 --- a/flink-python/src/main/java/org/apache/flink/python/AbstractPythonFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/python/AbstractPythonFunctionRunner.java @@ -42,9 +42,6 @@ import org.apache.beam.sdk.options.PortablePipelineOptions; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Struct; -import java.io.File; -import java.util.List; - /** * An base class for {@link PythonFunctionRunner}. * @@ -111,12 +108,6 @@ public abstract class AbstractPythonFunctionRunner implements PythonFunction */ protected transient DataOutputViewStreamWrapper baosWrapper; - /** - * Python libraries and shell script extracted from resource of flink-python jar. - * They are used to support running python udf worker in process mode. - */ - private transient List pythonInternalLibs; - public AbstractPythonFunctionRunner( String taskName, FnDataReceiver resultReceiver, @@ -160,15 +151,6 @@ public abstract class AbstractPythonFunctionRunner implements PythonFunction @Override public void close() throws Exception { - - try { - if (pythonInternalLibs != null) { - pythonInternalLibs.forEach(File::delete); - } - } finally { - pythonInternalLibs = null; - } - try { if (jobBundleFactory != null) { jobBundleFactory.close(); diff --git a/flink-python/src/main/java/org/apache/flink/python/env/ProcessPythonEnvironmentManager.java b/flink-python/src/main/java/org/apache/flink/python/env/ProcessPythonEnvironmentManager.java index 10f2bd2057..ff0e930ad7 100644 --- a/flink-python/src/main/java/org/apache/flink/python/env/ProcessPythonEnvironmentManager.java +++ b/flink-python/src/main/java/org/apache/flink/python/env/ProcessPythonEnvironmentManager.java @@ -49,7 +49,6 @@ import java.util.Map; import java.util.Objects; import java.util.Random; import java.util.UUID; -import java.util.stream.Collectors; /** * The ProcessPythonEnvironmentManager is used to prepare the working dir of python UDF worker and create @@ -138,6 +137,7 @@ public final class ProcessPythonEnvironmentManager implements PythonEnvironmentM @Override public RunnerApi.Environment createEnvironment() throws IOException, InterruptedException { Map env = constructEnvironmentVariables(); + ResourceUtil.extractUdfRunner(baseDirectory); String pythonWorkerCommand = String.join(File.separator, baseDirectory, "pyflink-udf-runner.sh"); return Environments.createProcessEnvironment( @@ -181,8 +181,6 @@ public final class ProcessPythonEnvironmentManager implements PythonEnvironmentM throws IOException, IllegalArgumentException, InterruptedException { Map env = new HashMap<>(this.systemEnv); - constructBuiltInDependencies(env); - constructFilesDirectory(env); constructArchivesDirectory(env); @@ -200,18 +198,6 @@ public final class ProcessPythonEnvironmentManager implements PythonEnvironmentM return env; } - private void constructBuiltInDependencies(Map env) throws IOException, InterruptedException { - // Extract built-in python dependencies and udf runner script. - ResourceUtil.extractBuiltInDependencies(baseDirectory, "", false); - - // add the built-in python dependencies to PYTHONPATH - List builtInDependencies = Arrays.stream(ResourceUtil.BUILT_IN_PYTHON_DEPENDENCIES) - .filter(file -> file.endsWith(".zip")) - .map(file -> String.join(File.separator, baseDirectory, file)) - .collect(Collectors.toList()); - appendToPythonPath(env, builtInDependencies); - } - private void constructFilesDirectory(Map env) throws IOException { // link or copy python files to filesDirectory and add them to PYTHONPATH List pythonFilePaths = new ArrayList<>(); diff --git a/flink-python/src/main/java/org/apache/flink/python/util/ResourceUtil.java b/flink-python/src/main/java/org/apache/flink/python/util/ResourceUtil.java index 39ff706905..2d64f040e0 100644 --- a/flink-python/src/main/java/org/apache/flink/python/util/ResourceUtil.java +++ b/flink-python/src/main/java/org/apache/flink/python/util/ResourceUtil.java @@ -22,8 +22,6 @@ import java.io.File; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Paths; -import java.util.ArrayList; -import java.util.List; /** * Utils for building the most basic environment for running python udf workers. The basic environment does not include @@ -31,62 +29,38 @@ import java.util.List; */ public class ResourceUtil { - public static final String[] BUILT_IN_PYTHON_DEPENDENCIES = { - "pyflink.zip", - "py4j-0.10.8.1-src.zip", - "cloudpickle-1.2.2-src.zip", - "pyflink-udf-runner.sh" - }; + public static final String PYFLINK_UDF_RUNNER = "pyflink-udf-runner.sh"; - public static List extractBuiltInDependencies( - String tmpdir, - String prefix, - boolean skipShellScript) throws IOException, InterruptedException { - List extractedFiles = new ArrayList<>(); - for (String fileName : BUILT_IN_PYTHON_DEPENDENCIES) { - if (skipShellScript && fileName.endsWith(".sh")) { - continue; - } - - File file = new File(tmpdir, prefix + fileName); - if (fileName.endsWith(".sh")) { - // TODO: This is a hacky solution to prevent subprocesses to hold the file descriptor of shell scripts, - // which will cause the execution of shell scripts failed with the exception "test file is busy" - // randomly. It's a bug of JDK, see https://bugs.openjdk.java.net/browse/JDK-8068370. After moving flink - // python jar to lib directory, we can solve this problem elegantly by extracting these files only once. - String javaExecutable = String.join(File.separator, System.getProperty("java.home"), "bin", "java"); - String classPath = new File( - ResourceUtil.class.getProtectionDomain().getCodeSource().getLocation().getPath()).getAbsolutePath(); - new ProcessBuilder( - javaExecutable, - "-cp", - classPath, - ResourceUtil.class.getName(), - tmpdir, - prefix, - fileName).inheritIO().start().waitFor(); - } else { - Files.copy( - ResourceUtil.class.getClassLoader().getResourceAsStream(fileName), - Paths.get(file.getAbsolutePath())); - } - extractedFiles.add(file); - } - return extractedFiles; + public static File extractUdfRunner(String tmpdir) throws IOException, InterruptedException { + File file = new File(tmpdir, PYFLINK_UDF_RUNNER); + // TODO: This is a hacky solution to prevent subprocesses to hold the file descriptor of shell scripts, + // which will cause the execution of shell scripts failed with the exception "test file is busy" + // randomly. It's a bug of JDK, see https://bugs.openjdk.java.net/browse/JDK-8068370. After moving flink + // python jar to lib directory, we can solve this problem elegantly by extracting these files only once. + String javaExecutable = String.join(File.separator, System.getProperty("java.home"), "bin", "java"); + String classPath = new File( + ResourceUtil.class.getProtectionDomain().getCodeSource().getLocation().getPath()).getAbsolutePath(); + new ProcessBuilder( + javaExecutable, + "-cp", + classPath, + ResourceUtil.class.getName(), + tmpdir, + PYFLINK_UDF_RUNNER).inheritIO().start().waitFor(); + return file; } /** * This main method is used to create the shell script in a subprocess, see the "TODO" hints in method - * {@link ResourceUtil#extractBuiltInDependencies}. + * {@link ResourceUtil#extractUdfRunner}. * @param args First argument is the directory where shell script will be created. Second argument is the prefix of * the shell script. Third argument is the fileName of the shell script. * @throws IOException */ public static void main(String[] args) throws IOException { String tmpdir = args[0]; - String prefix = args[1]; - String fileName = args[2]; - File file = new File(tmpdir, prefix + fileName); + String fileName = args[1]; + File file = new File(tmpdir, fileName); Files.copy( ResourceUtil.class.getClassLoader().getResourceAsStream(fileName), Paths.get(file.getAbsolutePath())); diff --git a/flink-python/src/main/resources/META-INF/NOTICE b/flink-python/src/main/resources/META-INF/NOTICE index d32850b7d4..a8c4a364d8 100644 --- a/flink-python/src/main/resources/META-INF/NOTICE +++ b/flink-python/src/main/resources/META-INF/NOTICE @@ -59,7 +59,6 @@ See bundled license files for details - com.google.protobuf:protobuf-java:3.7.1 - com.google.protobuf:protobuf-java-util:3.7.1 - com.google.auth:google-auth-library-credentials:0.13.0 -- cloudpickle:1.2.2 This project bundles the following dependencies under the MIT license. (https://opensource.org/licenses/MIT) See bundled license files for details. diff --git a/flink-python/bin/pyflink-udf-runner.sh b/flink-python/src/main/resources/pyflink-udf-runner.sh similarity index 100% rename from flink-python/bin/pyflink-udf-runner.sh rename to flink-python/src/main/resources/pyflink-udf-runner.sh diff --git a/flink-python/src/test/java/org/apache/flink/python/env/ProcessPythonEnvironmentManagerTest.java b/flink-python/src/test/java/org/apache/flink/python/env/ProcessPythonEnvironmentManagerTest.java index fa9b0d86a7..b26c989db8 100644 --- a/flink-python/src/test/java/org/apache/flink/python/env/ProcessPythonEnvironmentManagerTest.java +++ b/flink-python/src/test/java/org/apache/flink/python/env/ProcessPythonEnvironmentManagerTest.java @@ -169,10 +169,7 @@ public class ProcessPythonEnvironmentManagerTest { String.join(File.separator, baseDir, PYTHON_FILES_DIR, "file2", "test_file3.egg"), String.join(File.separator, baseDir, PYTHON_FILES_DIR, "dir0", "test_dir") }; - String expectedPythonPath = String.join( - File.pathSeparator, - String.join(File.pathSeparator, expectedUserPythonPaths), - getBasicExpectedEnv(environmentManager).get("PYTHONPATH")); + String expectedPythonPath = String.join(File.pathSeparator, expectedUserPythonPaths); assertEquals( expectedPythonPath, @@ -380,13 +377,6 @@ public class ProcessPythonEnvironmentManagerTest { private static Map getBasicExpectedEnv(ProcessPythonEnvironmentManager environmentManager) { Map map = new HashMap<>(); String tmpBase = environmentManager.getBaseDirectory(); - map.put( - "PYTHONPATH", - String.join( - File.pathSeparator, - String.join(File.separator, tmpBase, "pyflink.zip"), - String.join(File.separator, tmpBase, "py4j-0.10.8.1-src.zip"), - String.join(File.separator, tmpBase, "cloudpickle-1.2.2-src.zip"))); map.put("BOOT_LOG_DIR", tmpBase); return map; } diff --git a/flink-python/src/test/java/org/apache/flink/python/util/ResourceUtilTest.java b/flink-python/src/test/java/org/apache/flink/python/util/ResourceUtilTest.java index 31866e4a06..273ea13dff 100644 --- a/flink-python/src/test/java/org/apache/flink/python/util/ResourceUtilTest.java +++ b/flink-python/src/test/java/org/apache/flink/python/util/ResourceUtilTest.java @@ -24,10 +24,9 @@ import org.junit.Test; import java.io.File; import java.io.IOException; -import java.util.List; import java.util.UUID; -import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; /** @@ -36,7 +35,7 @@ import static org.junit.Assert.assertTrue; public class ResourceUtilTest { @Test - public void testExtractBasicDependenciesFromResource() throws IOException, InterruptedException { + public void testExtractUdfRunnerFromResource() throws IOException, InterruptedException { File tmpdir = File.createTempFile(UUID.randomUUID().toString(), null); tmpdir.delete(); tmpdir.mkdirs(); @@ -49,28 +48,9 @@ public class ResourceUtilTest { }); Runtime.getRuntime().addShutdownHook(hook); try { - String prefix = "tmp_"; - List files = ResourceUtil.extractBuiltInDependencies( - tmpdir.getAbsolutePath(), - prefix, - true); - files.forEach(file -> assertTrue(file.exists())); - assertArrayEquals(new File[] { - new File(tmpdir, "tmp_pyflink.zip"), - new File(tmpdir, "tmp_py4j-0.10.8.1-src.zip"), - new File(tmpdir, "tmp_cloudpickle-1.2.2-src.zip")}, files.toArray()); - files.forEach(File::delete); - files = ResourceUtil.extractBuiltInDependencies( - tmpdir.getAbsolutePath(), - prefix, - false); - files.forEach(file -> assertTrue(file.exists())); - assertArrayEquals(new File[] { - new File(tmpdir, "tmp_pyflink.zip"), - new File(tmpdir, "tmp_py4j-0.10.8.1-src.zip"), - new File(tmpdir, "tmp_cloudpickle-1.2.2-src.zip"), - new File(tmpdir, "tmp_pyflink-udf-runner.sh")}, files.toArray()); - assertTrue(new File(tmpdir, "tmp_pyflink-udf-runner.sh").canExecute()); + File file = ResourceUtil.extractUdfRunner(tmpdir.getAbsolutePath()); + assertEquals(file, new File(tmpdir, "pyflink-udf-runner.sh")); + assertTrue(file.canExecute()); } finally { hook.run(); Runtime.getRuntime().removeShutdownHook(hook); diff --git a/tools/releasing/NOTICE-binary_PREAMBLE.txt b/tools/releasing/NOTICE-binary_PREAMBLE.txt index c8f1d6d176..278e89e2e6 100644 --- a/tools/releasing/NOTICE-binary_PREAMBLE.txt +++ b/tools/releasing/NOTICE-binary_PREAMBLE.txt @@ -12,3 +12,8 @@ This project bundles the following dependencies under the Apache Software Licens - org.apache.logging.log4j:log4j-core:2.12.1 - org.apache.logging.log4j:log4j-slf4j-impl:2.12.1 - org.apache.logging.log4j:log4j-1.2-api:2.12.1 + +This project bundles the following dependencies under the BSD license. +See bundled license files for details. + +- cloudpickle:1.2.2 diff --git a/tools/releasing/collect_license_files.sh b/tools/releasing/collect_license_files.sh index 44b6dcbf72..c84a59ec24 100755 --- a/tools/releasing/collect_license_files.sh +++ b/tools/releasing/collect_license_files.sh @@ -57,4 +57,9 @@ LICENSES="${DST}/licenses" [ -f "${LICENSES}" ] && rm -r "${LICENSES}" find "${TMP}" -name "licenses" -type d -exec cp -r -- "{}" "${DST}" \; +# Search the binary distribution directory and collect those license files that +# not bundled in any jars. +find "${SRC}" -name "LICENSE.*" -type f \ +! -path "${DST}/licenses/*" ! -path "${TMP}/licenses/*" -exec cp -- "{}" "${DST}/licenses" \; + rm -r "${TMP}" -- Gitee From ab37867c474a9e5754e79045fa06c9ac145a3787 Mon Sep 17 00:00:00 2001 From: Yangze Guo Date: Thu, 12 Mar 2020 20:16:04 +0800 Subject: [PATCH 143/885] [FLINK-15727][scripts] Return dynamic configs and JVM parameters in one call --- flink-dist/src/main/flink-bin/bin/config.sh | 3 +- .../src/main/flink-bin/bin/taskmanager.sh | 16 ++++----- .../src/test/bin/runBashJavaUtilsCmd.sh | 5 +-- .../flink/dist/BashJavaUtilsITCase.java | 36 +++++-------------- .../flink/runtime/util/BashJavaUtils.java | 29 ++++++--------- 5 files changed, 30 insertions(+), 59 deletions(-) diff --git a/flink-dist/src/main/flink-bin/bin/config.sh b/flink-dist/src/main/flink-bin/bin/config.sh index b2f3d00dcd..406976a115 100755 --- a/flink-dist/src/main/flink-bin/bin/config.sh +++ b/flink-dist/src/main/flink-bin/bin/config.sh @@ -633,10 +633,9 @@ runBashJavaUtilsCmd() { } extractExecutionParams() { - local output=$1 + local execution_config=$1 local EXECUTION_PREFIX="BASH_JAVA_UTILS_EXEC_RESULT:" - local execution_config=`echo "$output" | tail -n 1` if ! [[ $execution_config =~ ^${EXECUTION_PREFIX}.* ]]; then echo "[ERROR] Unexpected result: $execution_config" 1>&2 echo "[ERROR] The last line of the BashJavaUtils outputs is expected to be the execution result, following the prefix '${EXECUTION_PREFIX}'" 1>&2 diff --git a/flink-dist/src/main/flink-bin/bin/taskmanager.sh b/flink-dist/src/main/flink-bin/bin/taskmanager.sh index 176326a533..befeb99760 100755 --- a/flink-dist/src/main/flink-bin/bin/taskmanager.sh +++ b/flink-dist/src/main/flink-bin/bin/taskmanager.sh @@ -48,8 +48,9 @@ if [[ $STARTSTOP == "start" ]] || [[ $STARTSTOP == "start-foreground" ]]; then # Startup parameters - jvm_params_output=`runBashJavaUtilsCmd GET_TM_RESOURCE_JVM_PARAMS ${FLINK_CONF_DIR}` - jvm_params=`extractExecutionParams "$jvm_params_output"` + params_output=$(runBashJavaUtilsCmd GET_TM_RESOURCE_PARAMS ${FLINK_CONF_DIR} | tail -n 2) + + jvm_params=$(extractExecutionParams "$(echo "$params_output" | head -n 1)") if [[ $? -ne 0 ]]; then echo "[ERROR] Could not get JVM parameters properly." exit 1 @@ -58,8 +59,7 @@ if [[ $STARTSTOP == "start" ]] || [[ $STARTSTOP == "start-foreground" ]]; then IFS=$" " - dynamic_configs_output=`runBashJavaUtilsCmd GET_TM_RESOURCE_DYNAMIC_CONFIGS ${FLINK_CONF_DIR}` - dynamic_configs=`extractExecutionParams "$dynamic_configs_output"` + dynamic_configs=$(extractExecutionParams "$(echo "$params_output" | tail -n 1)") if [[ $? -ne 0 ]]; then echo "[ERROR] Could not get dynamic configurations properly." exit 1 @@ -69,11 +69,9 @@ if [[ $STARTSTOP == "start" ]] || [[ $STARTSTOP == "start-foreground" ]]; then export FLINK_INHERITED_LOGS=" $FLINK_INHERITED_LOGS -TM_RESOURCES_JVM_PARAMS extraction logs: -$jvm_params_output - -TM_RESOURCES_DYNAMIC_CONFIGS extraction logs: -$dynamic_configs_output +TM_RESOURCE_PARAMS extraction logs: +jvm_params: $jvm_params +dynamic_configs: $dynamic_configs " fi diff --git a/flink-dist/src/test/bin/runBashJavaUtilsCmd.sh b/flink-dist/src/test/bin/runBashJavaUtilsCmd.sh index 745d129657..2a7f883c1d 100755 --- a/flink-dist/src/test/bin/runBashJavaUtilsCmd.sh +++ b/flink-dist/src/test/bin/runBashJavaUtilsCmd.sh @@ -36,5 +36,6 @@ FLINK_DIST_JAR=`find $FLINK_TARGET_DIR -name 'flink-dist*.jar'` . ${bin}/../../main/flink-bin/bin/config.sh > /dev/null -output=`runBashJavaUtilsCmd ${COMMAND} ${FLINK_CONF_DIR} "$FLINK_TARGET_DIR/bash-java-utils.jar:$FLINK_DIST_JAR}"` -extractExecutionParams "$output" +output=$(runBashJavaUtilsCmd GET_TM_RESOURCE_PARAMS ${FLINK_CONF_DIR} "$FLINK_TARGET_DIR/bash-java-utils.jar:$FLINK_DIST_JAR}" | tail -n 2) +extractExecutionParams "$(echo "$output" | head -n 1)" +extractExecutionParams "$(echo "$output" | tail -n 1)" diff --git a/flink-dist/src/test/java/org/apache/flink/dist/BashJavaUtilsITCase.java b/flink-dist/src/test/java/org/apache/flink/dist/BashJavaUtilsITCase.java index 5be92a1dd6..daf6f1b7a7 100644 --- a/flink-dist/src/test/java/org/apache/flink/dist/BashJavaUtilsITCase.java +++ b/flink-dist/src/test/java/org/apache/flink/dist/BashJavaUtilsITCase.java @@ -23,9 +23,10 @@ import org.apache.flink.runtime.util.BashJavaUtils; import org.junit.Test; -import java.io.IOException; +import java.util.Arrays; +import java.util.List; -import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertEquals; /** * Tests for BashJavaUtils. @@ -36,32 +37,13 @@ public class BashJavaUtilsITCase extends JavaBashTestBase { private static final String RUN_BASH_JAVA_UTILS_CMD_SCRIPT = "src/test/bin/runBashJavaUtilsCmd.sh"; - /** - * Executes the given shell script wrapper and returns the last line. - */ - private String executeScriptAndFetchLastLine(final String command) throws IOException { - String[] commands = {RUN_BASH_JAVA_UTILS_CMD_SCRIPT, command}; - String[] lines = executeScript(commands).split(System.lineSeparator()); - if (lines.length == 0) { - return ""; - } else { - return lines[lines.length - 1]; - } - } - - @Test - public void testGetTmResourceDynamicConfigs() throws Exception { - String result = executeScriptAndFetchLastLine(BashJavaUtils.Command.GET_TM_RESOURCE_DYNAMIC_CONFIGS.toString()); - - assertNotNull(result); - ConfigurationUtils.parseTmResourceDynamicConfigs(result); - } - @Test - public void testGetTmResourceJvmParams() throws Exception { - String result = executeScriptAndFetchLastLine(BashJavaUtils.Command.GET_TM_RESOURCE_JVM_PARAMS.toString()); + public void testGetTmResourceParamsConfigs() throws Exception { + String[] commands = {RUN_BASH_JAVA_UTILS_CMD_SCRIPT, BashJavaUtils.Command.GET_TM_RESOURCE_PARAMS.toString()}; + List lines = Arrays.asList(executeScript(commands).split(System.lineSeparator())); - assertNotNull(result); - ConfigurationUtils.parseTmResourceJvmParams(result); + assertEquals(2, lines.size()); + ConfigurationUtils.parseTmResourceJvmParams(lines.get(lines.size() - 2)); + ConfigurationUtils.parseTmResourceDynamicConfigs(lines.get(lines.size() - 1)); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/BashJavaUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/BashJavaUtils.java index 21702ca37a..a0db33d7d1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/BashJavaUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/BashJavaUtils.java @@ -39,11 +39,8 @@ public class BashJavaUtils { checkArgument(args.length > 0, "Command not specified."); switch (Command.valueOf(args[0])) { - case GET_TM_RESOURCE_DYNAMIC_CONFIGS: - getTmResourceDynamicConfigs(args); - break; - case GET_TM_RESOURCE_JVM_PARAMS: - getTmResourceJvmParams(args); + case GET_TM_RESOURCE_PARAMS: + getTmResourceParams(args); break; default: // unexpected, Command#valueOf should fail if a unknown command is passed in @@ -51,16 +48,15 @@ public class BashJavaUtils { } } - private static void getTmResourceDynamicConfigs(String[] args) throws Exception { - Configuration configuration = getConfigurationForStandaloneTaskManagers(args); - TaskExecutorProcessSpec taskExecutorProcessSpec = TaskExecutorProcessUtils.processSpecFromConfig(configuration); - System.out.println(EXECUTION_PREFIX + TaskExecutorProcessUtils.generateDynamicConfigsStr(taskExecutorProcessSpec)); - } - - private static void getTmResourceJvmParams(String[] args) throws Exception { + /** + * Generate and print JVM parameters and dynamic configs of task executor resources. The last two lines of + * the output should be JVM parameters and dynamic configs respectively. + */ + private static void getTmResourceParams(String[] args) throws Exception { Configuration configuration = getConfigurationForStandaloneTaskManagers(args); TaskExecutorProcessSpec taskExecutorProcessSpec = TaskExecutorProcessUtils.processSpecFromConfig(configuration); System.out.println(EXECUTION_PREFIX + TaskExecutorProcessUtils.generateJvmParametersStr(taskExecutorProcessSpec)); + System.out.println(EXECUTION_PREFIX + TaskExecutorProcessUtils.generateDynamicConfigsStr(taskExecutorProcessSpec)); } private static Configuration getConfigurationForStandaloneTaskManagers(String[] args) throws Exception { @@ -74,13 +70,8 @@ public class BashJavaUtils { */ public enum Command { /** - * Get dynamic configs of task executor resources. - */ - GET_TM_RESOURCE_DYNAMIC_CONFIGS, - - /** - * Get JVM parameters of task executor resources. + * Get JVM parameters and dynamic configs of task executor resources. */ - GET_TM_RESOURCE_JVM_PARAMS + GET_TM_RESOURCE_PARAMS } } -- Gitee From 62abfa97a497f03d8701f4acdbdd9217f0d33a06 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Wed, 11 Mar 2020 22:37:42 +0100 Subject: [PATCH 144/885] [FLINK-16550][s3] Force namespace parsing in XmlResponsesSaxParser. This commit explicitly enables Sax' NAMESPACES_FEATURE (default true) in JAXP (default false), such that multi part uploads to S3 actually work. --- .../services/s3/model/transform/XmlResponsesSaxParser.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/flink-filesystems/flink-s3-fs-base/src/main/java/com/amazonaws/services/s3/model/transform/XmlResponsesSaxParser.java b/flink-filesystems/flink-s3-fs-base/src/main/java/com/amazonaws/services/s3/model/transform/XmlResponsesSaxParser.java index bc87b234df..d6a322553c 100644 --- a/flink-filesystems/flink-s3-fs-base/src/main/java/com/amazonaws/services/s3/model/transform/XmlResponsesSaxParser.java +++ b/flink-filesystems/flink-s3-fs-base/src/main/java/com/amazonaws/services/s3/model/transform/XmlResponsesSaxParser.java @@ -128,6 +128,8 @@ public class XmlResponsesSaxParser { public XmlResponsesSaxParser() throws SdkClientException { // Ensure we can load the XML Reader. try { + // Need to explicitly enable Sax' NAMESPACES_FEATURE (default true) in JAXP (default false)! + SAX_PARSER_FACTORY.setNamespaceAware(true); xr = SAX_PARSER_FACTORY.newSAXParser().getXMLReader(); disableExternalResourceFetching(xr); } catch (SAXException | ParserConfigurationException e) { -- Gitee From fac04a7ec8f762c49d3db8c74e12de17d081cd20 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Thu, 12 Mar 2020 20:58:13 +0800 Subject: [PATCH 145/885] [FLINK-16343][table-planner-blink] Improve exception message when reading an unbounded source in batch mode This closes #11387 --- .../plan/schema/CatalogSourceTable.scala | 21 +++-- .../planner/catalog/CatalogTableITCase.scala | 82 +------------------ .../plan/batch/sql/TableScanTest.scala | 23 +++++- .../batch/sql/PartitionableSinkITCase.scala | 30 ++----- 4 files changed, 42 insertions(+), 114 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/CatalogSourceTable.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/CatalogSourceTable.scala index 9a82a8d297..12801cca7f 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/CatalogSourceTable.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/CatalogSourceTable.scala @@ -19,14 +19,13 @@ package org.apache.flink.table.planner.plan.schema import org.apache.flink.configuration.ReadableConfig -import org.apache.flink.table.api.TableException +import org.apache.flink.table.api.{TableException, ValidationException} import org.apache.flink.table.catalog.CatalogTable import org.apache.flink.table.factories.{TableFactoryUtil, TableSourceFactory, TableSourceFactoryContextImpl} import org.apache.flink.table.planner.calcite.{FlinkContext, FlinkRelBuilder} import org.apache.flink.table.planner.catalog.CatalogSchemaTable import org.apache.flink.table.sources.{StreamTableSource, TableSource, TableSourceValidation} import org.apache.flink.table.utils.TableConnectorUtils.generateRuntimeName - import org.apache.calcite.plan.{RelOptSchema, RelOptTable} import org.apache.calcite.rel.RelNode import org.apache.calcite.rel.`type`.RelDataType @@ -158,16 +157,26 @@ class CatalogSourceTable[T]( tableFactoryOpt.get() match { case tableSourceFactory: TableSourceFactory[_] => tableSourceFactory.createTableSource(context) - case _ => throw new TableException("Cannot query a sink-only table. " + case _ => throw new ValidationException("Cannot query a sink-only table. " + "TableFactory provided by catalog must implement TableSourceFactory") } } else { TableFactoryUtil.findAndCreateTableSource(context) } - if (!tableSource.isInstanceOf[StreamTableSource[_]]) { - throw new TableException("Catalog tables support only " - + "StreamTableSource and InputFormatTableSource") + + // validation + val tableName = schemaTable.getTableIdentifier.asSummaryString(); + tableSource match { + case ts: StreamTableSource[_] => + if (!schemaTable.isStreamingMode && !ts.isBounded) { + throw new ValidationException("Cannot query on an unbounded source in batch mode, " + + s"but '$tableName' is unbounded.") + } + case _ => + throw new ValidationException("Catalog tables only support " + + "StreamTableSource and InputFormatTableSource") } + tableSource.asInstanceOf[TableSource[T]] } } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala index 9e2d628af0..5d64efee07 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala @@ -34,7 +34,7 @@ import org.junit.Assert.{assertEquals, fail} import org.junit.rules.ExpectedException import org.junit.runner.RunWith import org.junit.runners.Parameterized -import org.junit.{Before, Ignore, Rule, Test} +import org.junit.{Before, Rule, Test} import java.io.File import java.util @@ -646,86 +646,6 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase { assertEquals(expected.sorted, TestCollectionTableFactory.RESULT.sorted) } - @Test @Ignore("FLINK-14320") // need to implement - def testStreamSourceTableWithRowtime(): Unit = { - val sourceData = List( - toRow(1, 1000), - toRow(2, 2000), - toRow(3, 3000) - ) - TestCollectionTableFactory.initData(sourceData, emitInterval = 1000L) - val sourceDDL = - """ - |create table t1( - | a timestamp(3), - | b bigint, - | WATERMARK FOR a AS a - interval '1' SECOND - |) with ( - | 'connector' = 'COLLECTION' - |) - """.stripMargin - val sinkDDL = - """ - |create table t2( - | a timestamp(3), - | b bigint - |) with ( - | 'connector' = 'COLLECTION' - |) - """.stripMargin - val query = - """ - |insert into t2 - |select a, sum(b) from t1 group by TUMBLE(a, INTERVAL '1' SECOND) - """.stripMargin - - tableEnv.sqlUpdate(sourceDDL) - tableEnv.sqlUpdate(sinkDDL) - tableEnv.sqlUpdate(query) - execJob("testJob") - assertEquals(TestCollectionTableFactory.RESULT.sorted, sourceData.sorted) - } - - @Test @Ignore("FLINK-14320") // need to implement - def testBatchTableWithRowtime(): Unit = { - val sourceData = List( - toRow(1, 1000), - toRow(2, 2000), - toRow(3, 3000) - ) - TestCollectionTableFactory.initData(sourceData, emitInterval = 1000L) - val sourceDDL = - """ - |create table t1( - | a timestamp(3), - | b bigint, - | WATERMARK FOR a AS a - interval '1' SECOND - |) with ( - | 'connector' = 'COLLECTION' - |) - """.stripMargin - val sinkDDL = - """ - |create table t2( - | a timestamp(3), - | b bigint - |) with ( - | 'connector' = 'COLLECTION' - |) - """.stripMargin - val query = - """ - |insert into t2 - |select a, sum(b) from t1 group by TUMBLE(a, INTERVAL '1' SECOND) - """.stripMargin - - tableEnv.sqlUpdate(sourceDDL) - tableEnv.sqlUpdate(sinkDDL) - tableEnv.sqlUpdate(query) - execJob("testJob") - assertEquals(TestCollectionTableFactory.RESULT.sorted, sourceData.sorted) - } - @Test def testDropTableWithFullPath(): Unit = { val ddl1 = diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableScanTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableScanTest.scala index 6a8039cc35..48c5d850df 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableScanTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableScanTest.scala @@ -19,12 +19,11 @@ package org.apache.flink.table.planner.plan.batch.sql import org.apache.flink.api.scala._ -import org.apache.flink.table.api.DataTypes +import org.apache.flink.table.api.{DataTypes, ValidationException} import org.apache.flink.table.api.scala._ import org.apache.flink.table.descriptors.{FileSystem, OldCsv, Schema} import org.apache.flink.table.planner.expressions.utils.Func0 import org.apache.flink.table.planner.utils.TableTestBase - import org.junit.{Before, Test} class TableScanTest extends TableTestBase { @@ -88,6 +87,26 @@ class TableScanTest extends TableTestBase { util.verifyPlan("SELECT * FROM src WHERE a > 1") } + @Test + def testScanOnUnboundedSource(): Unit = { + util.addTable( + """ + |CREATE TABLE src ( + | ts TIMESTAMP(3), + | a INT, + | b DOUBLE, + | WATERMARK FOR ts AS ts - INTERVAL '0.001' SECOND + |) WITH ( + | 'connector' = 'COLLECTION', + | 'is-bounded' = 'false' + |) + """.stripMargin) + thrown.expect(classOf[ValidationException]) + thrown.expectMessage("Cannot query on an unbounded source in batch mode, " + + "but 'default_catalog.default_database.src' is unbounded") + util.verifyPlan("SELECT * FROM src WHERE a > 1") + } + @Test def testDDLWithComputedColumn(): Unit = { util.verifyPlan("SELECT * FROM computed_column_t") diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala index add6f2ac67..1cb559786d 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala @@ -23,21 +23,19 @@ import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.datastream.{DataStream, DataStreamSink} -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment import org.apache.flink.streaming.api.functions.sink.RichSinkFunction import org.apache.flink.table.api.config.ExecutionConfigOptions -import org.apache.flink.table.api.{SqlDialect, TableEnvironment, TableException, TableSchema, ValidationException} +import org.apache.flink.table.api.{TableEnvironment, TableException, TableSchema, ValidationException} import org.apache.flink.table.catalog.{CatalogTableImpl, ObjectPath} import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE import org.apache.flink.table.descriptors.DescriptorProperties import org.apache.flink.table.descriptors.Schema.SCHEMA -import org.apache.flink.table.factories.{TableSinkFactory, TableSourceFactory} +import org.apache.flink.table.factories.TableSinkFactory import org.apache.flink.table.planner.runtime.batch.sql.PartitionableSinkITCase._ import org.apache.flink.table.planner.runtime.utils.BatchTestBase import org.apache.flink.table.planner.runtime.utils.BatchTestBase.row import org.apache.flink.table.planner.runtime.utils.TestData._ import org.apache.flink.table.sinks.{PartitionableTableSink, StreamTableSink, TableSink} -import org.apache.flink.table.sources.{StreamTableSource, TableSource} import org.apache.flink.table.types.logical.{BigIntType, IntType, VarCharType} import org.apache.flink.types.Row @@ -168,7 +166,7 @@ class PartitionableSinkITCase extends BatchTestBase { expectedEx.expect(classOf[ValidationException]) registerTableSink(tableName = "sinkTable2", rowType = type4, partitionColumns = Array("a", "b")) - tEnv.sqlUpdate("insert into sinkTable2 partition(c=1) select a, b from sinkTable2") + tEnv.sqlUpdate("insert into sinkTable2 partition(c=1) select a, b from sortTable") tEnv.execute("testJob") } @@ -176,7 +174,7 @@ class PartitionableSinkITCase extends BatchTestBase { def testInsertStaticPartitionOnNonPartitionedSink(): Unit = { expectedEx.expect(classOf[TableException]) registerTableSink(tableName = "sinkTable2", rowType = type4, partitionColumns = Array()) - tEnv.sqlUpdate("insert into sinkTable2 partition(c=1) select a, b from sinkTable2") + tEnv.sqlUpdate("insert into sinkTable2 partition(c=1) select a, b from sortTable") tEnv.execute("testJob") } @@ -316,7 +314,7 @@ private class TestSink( } } -class TestPartitionableSinkFactory extends TableSinkFactory[Row] with TableSourceFactory[Row] { +class TestPartitionableSinkFactory extends TableSinkFactory[Row] { override def requiredContext(): util.Map[String, String] = { val context = new util.HashMap[String, String]() @@ -344,23 +342,5 @@ class TestPartitionableSinkFactory extends TableSinkFactory[Row] with TableSourc supportsGrouping, partitionColumns.asScala.toArray[String]) } - - /** - * Remove it after FLINK-14387. - */ - override def createTableSource(properties: JMap[String, String]): TableSource[Row] = { - val dp = new DescriptorProperties() - dp.putProperties(properties) - - new StreamTableSource[Row] { - override def getTableSchema: TableSchema = { - dp.getTableSchema(SCHEMA) - } - - override def getDataStream(execEnv: StreamExecutionEnvironment): DataStream[Row] = { - throw new RuntimeException - } - } - } } -- Gitee From 108f52804cfe5541c922f4aa461d6b9dc52a806d Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 12 Mar 2020 14:26:22 +0100 Subject: [PATCH 146/885] [FLINK-16350][e2e] Run half of streaming HA tests against ZK 3.5 --- flink-end-to-end-tests/run-nightly-tests.sh | 18 +++++++++--------- flink-end-to-end-tests/test-scripts/common.sh | 16 ++++++++++++++++ .../test-scripts/test_ha_datastream.sh | 5 ++++- tools/travis/splits/split_ha.sh | 18 +++++++++--------- 4 files changed, 38 insertions(+), 19 deletions(-) diff --git a/flink-end-to-end-tests/run-nightly-tests.sh b/flink-end-to-end-tests/run-nightly-tests.sh index a7b68517eb..cee17d54b2 100755 --- a/flink-end-to-end-tests/run-nightly-tests.sh +++ b/flink-end-to-end-tests/run-nightly-tests.sh @@ -127,15 +127,15 @@ fi run_test "Running HA dataset end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_dataset.sh" "skip_check_exceptions" -run_test "Running HA (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh file true false" "skip_check_exceptions" -run_test "Running HA (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh file false false" "skip_check_exceptions" -run_test "Running HA (rocks, non-incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh rocks true false" "skip_check_exceptions" -run_test "Running HA (rocks, incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh rocks true true" "skip_check_exceptions" - -run_test "Running HA per-job cluster (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh file true false" "skip_check_exceptions" -run_test "Running HA per-job cluster (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh file false false" "skip_check_exceptions" -run_test "Running HA per-job cluster (rocks, non-incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh rocks true false" "skip_check_exceptions" -run_test "Running HA per-job cluster (rocks, incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh rocks true true" "skip_check_exceptions" +run_test "Running HA (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh file true false 3.4" "skip_check_exceptions" +run_test "Running HA (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh file false false 3.5" "skip_check_exceptions" +run_test "Running HA (rocks, non-incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh rocks true false 3.4" "skip_check_exceptions" +run_test "Running HA (rocks, incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh rocks true true 3.5" "skip_check_exceptions" + +run_test "Running HA per-job cluster (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh file true false 3.4" "skip_check_exceptions" +run_test "Running HA per-job cluster (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh file false false 3.5" "skip_check_exceptions" +run_test "Running HA per-job cluster (rocks, non-incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh rocks true false 3.4" "skip_check_exceptions" +run_test "Running HA per-job cluster (rocks, incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh rocks true true 3.5" "skip_check_exceptions" ################################################################################ # Miscellaneous diff --git a/flink-end-to-end-tests/test-scripts/common.sh b/flink-end-to-end-tests/test-scripts/common.sh index ed56b3bce2..dc4fc7bf18 100644 --- a/flink-end-to-end-tests/test-scripts/common.sh +++ b/flink-end-to-end-tests/test-scripts/common.sh @@ -101,6 +101,22 @@ function revert_flink_dir() { CURL_SSL_ARGS="" } +function setup_flink_shaded_zookeeper() { + local version=$1 + # if it is already in lib we don't have to do anything + if ! [ -e "${FLINK_DIR}"/lib/flink-shaded-zookeeper-${version}* ]; then + if ! [ -e "${FLINK_DIR}"/opt/flink-shaded-zookeeper-${version}* ]; then + echo "Could not find ZK ${version} in opt or lib." + exit 1 + else + # contents of 'opt' must not be changed since it is not backed up in common.sh#backup_flink_dir + # it is fine to delete jars from 'lib' since it is backed up and will be restored after the test + rm "${FLINK_DIR}"/lib/flink-shaded-zookeeper-* + cp "${FLINK_DIR}"/opt/flink-shaded-zookeeper-${version}* "${FLINK_DIR}/lib" + fi + fi +} + function add_optional_lib() { local lib_name=$1 cp "$FLINK_DIR/opt/flink-${lib_name}"*".jar" "$FLINK_DIR/lib" diff --git a/flink-end-to-end-tests/test-scripts/test_ha_datastream.sh b/flink-end-to-end-tests/test-scripts/test_ha_datastream.sh index 5d6f6db48f..10cccc5e48 100755 --- a/flink-end-to-end-tests/test-scripts/test_ha_datastream.sh +++ b/flink-end-to-end-tests/test-scripts/test_ha_datastream.sh @@ -34,6 +34,7 @@ function run_ha_test() { local BACKEND=$2 local ASYNC=$3 local INCREM=$4 + local ZOOKEEPER_VERSION=$5 local JM_KILLS=3 local CHECKPOINT_DIR="${TEST_DATA_DIR}/checkpoints/" @@ -46,6 +47,7 @@ function run_ha_test() { # jm killing loop set_config_key "env.pid.dir" "${TEST_DATA_DIR}" set_config_key "env.java.opts" "-ea" + setup_flink_shaded_zookeeper ${ZOOKEEPER_VERSION} start_local_zk start_cluster @@ -96,5 +98,6 @@ function run_ha_test() { STATE_BACKEND_TYPE=${1:-file} STATE_BACKEND_FILE_ASYNC=${2:-true} STATE_BACKEND_ROCKS_INCREMENTAL=${3:-false} +ZOOKEEPER_VERSION=${4:-3.4} -run_ha_test 4 ${STATE_BACKEND_TYPE} ${STATE_BACKEND_FILE_ASYNC} ${STATE_BACKEND_ROCKS_INCREMENTAL} +run_ha_test 4 ${STATE_BACKEND_TYPE} ${STATE_BACKEND_FILE_ASYNC} ${STATE_BACKEND_ROCKS_INCREMENTAL} ${ZOOKEEPER_VERSION} diff --git a/tools/travis/splits/split_ha.sh b/tools/travis/splits/split_ha.sh index 628b716d7c..c73cd6540a 100755 --- a/tools/travis/splits/split_ha.sh +++ b/tools/travis/splits/split_ha.sh @@ -45,15 +45,15 @@ echo "Flink distribution directory: $FLINK_DIR" run_test "Running HA dataset end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_dataset.sh" "skip_check_exceptions" -run_test "Running HA (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh file true false" "skip_check_exceptions" -run_test "Running HA (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh file false false" "skip_check_exceptions" -run_test "Running HA (rocks, non-incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh rocks true false" "skip_check_exceptions" -run_test "Running HA (rocks, incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh rocks true true" "skip_check_exceptions" - -run_test "Running HA per-job cluster (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh file true false" "skip_check_exceptions" -run_test "Running HA per-job cluster (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh file false false" "skip_check_exceptions" -run_test "Running HA per-job cluster (rocks, non-incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh rocks true false" "skip_check_exceptions" -run_test "Running HA per-job cluster (rocks, incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh rocks true true" "skip_check_exceptions" +run_test "Running HA (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh file true false 3.4" "skip_check_exceptions" +run_test "Running HA (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh file false false 3.5" "skip_check_exceptions" +run_test "Running HA (rocks, non-incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh rocks true false 3.4" "skip_check_exceptions" +run_test "Running HA (rocks, incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh rocks true true 3.5" "skip_check_exceptions" + +run_test "Running HA per-job cluster (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh file true false 3.4" "skip_check_exceptions" +run_test "Running HA per-job cluster (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh file false false 3.5" "skip_check_exceptions" +run_test "Running HA per-job cluster (rocks, non-incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh rocks true false 3.4" "skip_check_exceptions" +run_test "Running HA per-job cluster (rocks, incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh rocks true true 3.5" "skip_check_exceptions" printf "\n[PASS] All tests passed\n" exit 0 -- Gitee From 625da7b51004bbda8cd97c46208766ba98c63288 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E5=86=9B?= <25563794+zhangjun888@users.noreply.github.com> Date: Thu, 12 Mar 2020 21:45:45 +0800 Subject: [PATCH 147/885] [FLINK-16539][sql-client] Trim the value when calling set command in sql cli This closes #11392 --- .../main/java/org/apache/flink/table/client/cli/CliClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliClient.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliClient.java index 76039f8e3e..a028c9c681 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliClient.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliClient.java @@ -367,7 +367,7 @@ public class CliClient { } // set a property else { - executor.setSessionProperty(sessionId, cmdCall.operands[0], cmdCall.operands[1]); + executor.setSessionProperty(sessionId, cmdCall.operands[0], cmdCall.operands[1].trim()); terminal.writer().println(CliStrings.messageInfo(CliStrings.MESSAGE_SET).toAnsi()); } terminal.flush(); -- Gitee From 88d7d95b2bcc689a9c2aa864902e7bc635b40972 Mon Sep 17 00:00:00 2001 From: tison Date: Sat, 7 Mar 2020 03:26:31 +0800 Subject: [PATCH 148/885] [FLINK-15090][tests] Explicitly depend on flink-clients --- flink-connectors/flink-connector-kinesis/pom.xml | 7 +++++++ .../flink-connector-gcp-pubsub-emulator-tests/pom.xml | 6 ++++++ flink-ml-parent/flink-ml-lib/pom.xml | 6 ++++++ 3 files changed, 19 insertions(+) diff --git a/flink-connectors/flink-connector-kinesis/pom.xml b/flink-connectors/flink-connector-kinesis/pom.xml index edddad5e44..2b2626cae6 100644 --- a/flink-connectors/flink-connector-kinesis/pom.xml +++ b/flink-connectors/flink-connector-kinesis/pom.xml @@ -50,6 +50,13 @@ under the License. provided + + org.apache.flink + flink-clients_${scala.binary.version} + ${project.version} + provided + + com.google.guava guava diff --git a/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/pom.xml b/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/pom.xml index f34c01fa68..52b11463cb 100644 --- a/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/pom.xml +++ b/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/pom.xml @@ -45,6 +45,12 @@ under the License. ${project.version} test + + org.apache.flink + flink-clients_${scala.binary.version} + ${project.version} + test + org.apache.flink diff --git a/flink-ml-parent/flink-ml-lib/pom.xml b/flink-ml-parent/flink-ml-lib/pom.xml index 8207200ef7..df2930f4a7 100644 --- a/flink-ml-parent/flink-ml-lib/pom.xml +++ b/flink-ml-parent/flink-ml-lib/pom.xml @@ -54,6 +54,12 @@ under the License. ${project.version} test + + org.apache.flink + flink-clients_${scala.binary.version} + ${project.version} + provided + com.github.fommil.netlib core -- Gitee From de4c116451d8faafcb43f2463c163bbcb7e7085c Mon Sep 17 00:00:00 2001 From: tison Date: Sat, 7 Mar 2020 03:27:29 +0800 Subject: [PATCH 149/885] [FLINK-15090][tests] Move streaming env tests to flink-tests + GetOperatorUniqueIDTest + RemoteStreamEnvironmentTest + ReinterpretDataStreamAsKeyedStreamITCase + LocalStreamEnvironmentITCase --- .../streaming/api/datastream}/GetOperatorUniqueIDTest.java | 3 ++- .../ReinterpretDataStreamAsKeyedStreamITCase.java | 4 +++- .../api/environment/LocalStreamEnvironmentITCase.java | 4 +++- .../api/environment/RemoteStreamEnvironmentTest.java | 6 ++++-- 4 files changed, 12 insertions(+), 5 deletions(-) rename {flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators => flink-tests/src/test/java/org/apache/flink/test/streaming/api/datastream}/GetOperatorUniqueIDTest.java (95%) rename {flink-streaming-java/src/test/java/org/apache/flink => flink-tests/src/test/java/org/apache/flink/test}/streaming/api/datastream/ReinterpretDataStreamAsKeyedStreamITCase.java (98%) rename {flink-streaming-java/src/test/java/org/apache/flink => flink-tests/src/test/java/org/apache/flink/test}/streaming/api/environment/LocalStreamEnvironmentITCase.java (91%) rename flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/RemoteStreamExecutionEnvironmentTest.java => flink-tests/src/test/java/org/apache/flink/test/streaming/api/environment/RemoteStreamEnvironmentTest.java (96%) diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/GetOperatorUniqueIDTest.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/datastream/GetOperatorUniqueIDTest.java similarity index 95% rename from flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/GetOperatorUniqueIDTest.java rename to flink-tests/src/test/java/org/apache/flink/test/streaming/api/datastream/GetOperatorUniqueIDTest.java index 9693e42c6f..7ee49bcec3 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/GetOperatorUniqueIDTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/datastream/GetOperatorUniqueIDTest.java @@ -16,12 +16,13 @@ * limitations under the License. */ -package org.apache.flink.streaming.api.operators; +package org.apache.flink.test.streaming.api.datastream; import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.util.TestLogger; import org.junit.Test; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/ReinterpretDataStreamAsKeyedStreamITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/datastream/ReinterpretDataStreamAsKeyedStreamITCase.java similarity index 98% rename from flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/ReinterpretDataStreamAsKeyedStreamITCase.java rename to flink-tests/src/test/java/org/apache/flink/test/streaming/api/datastream/ReinterpretDataStreamAsKeyedStreamITCase.java index 3bf1573d6e..db03220d32 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/datastream/ReinterpretDataStreamAsKeyedStreamITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/datastream/ReinterpretDataStreamAsKeyedStreamITCase.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.api.datastream; +package org.apache.flink.test.streaming.api.datastream; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.restartstrategy.RestartStrategies; @@ -30,6 +30,8 @@ import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.FunctionSnapshotContext; import org.apache.flink.streaming.api.TimeCharacteristic; import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamUtils; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironmentITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/environment/LocalStreamEnvironmentITCase.java similarity index 91% rename from flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironmentITCase.java rename to flink-tests/src/test/java/org/apache/flink/test/streaming/api/environment/LocalStreamEnvironmentITCase.java index 84a1ce94f0..3bda757a1a 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironmentITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/environment/LocalStreamEnvironmentITCase.java @@ -16,9 +16,11 @@ * limitations under the License. */ -package org.apache.flink.streaming.api.environment; +package org.apache.flink.test.streaming.api.environment; import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.LocalStreamEnvironment; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.util.TestLogger; import org.junit.Test; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/RemoteStreamExecutionEnvironmentTest.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/environment/RemoteStreamEnvironmentTest.java similarity index 96% rename from flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/RemoteStreamExecutionEnvironmentTest.java rename to flink-tests/src/test/java/org/apache/flink/test/streaming/api/environment/RemoteStreamEnvironmentTest.java index 93e62485a4..00bc2b9126 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/RemoteStreamExecutionEnvironmentTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/environment/RemoteStreamEnvironmentTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.api.environment; +package org.apache.flink.test.streaming.api.environment; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.JobID; @@ -35,6 +35,8 @@ import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.jobmaster.JobResult.Builder; import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.streaming.api.environment.RemoteStreamEnvironment; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.graph.StreamGraph; import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; @@ -57,7 +59,7 @@ import static org.junit.Assert.assertTrue; /** * Tests for the {@link RemoteStreamEnvironment}. */ -public class RemoteStreamExecutionEnvironmentTest extends TestLogger { +public class RemoteStreamEnvironmentTest extends TestLogger { /** * Verifies that the port passed to the RemoteStreamEnvironment is used for connecting to the -- Gitee From 9504252c12186bb82526095bdcc62f182dfad6ca Mon Sep 17 00:00:00 2001 From: tison Date: Fri, 6 Mar 2020 16:17:13 +0800 Subject: [PATCH 150/885] [FLINK-15090][compiler] Extract JobGraphGenerator#addUserArtifactEntries to a flink-runtime utils --- .../plantranslate/JobGraphGenerator.java | 37 +--------- .../plantranslate/JobGraphGeneratorTest.java | 3 +- .../flink/runtime/jobgraph/JobGraphUtils.java | 71 +++++++++++++++++++ .../api/graph/StreamingJobGraphGenerator.java | 4 +- 4 files changed, 77 insertions(+), 38 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraphUtils.java diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java index fae214d01c..b950e7f354 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java @@ -35,8 +35,6 @@ import org.apache.flink.configuration.AlgorithmOptions; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; -import org.apache.flink.core.fs.FileSystem; -import org.apache.flink.core.fs.Path; import org.apache.flink.optimizer.CompilerException; import org.apache.flink.optimizer.dag.TempMode; import org.apache.flink.optimizer.plan.BulkIterationPlanNode; @@ -68,6 +66,7 @@ import org.apache.flink.runtime.jobgraph.InputOutputFormatVertex; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.JobEdge; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobGraphUtils; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; @@ -86,8 +85,6 @@ import org.apache.flink.runtime.operators.chaining.ChainedDriver; import org.apache.flink.runtime.operators.shipping.ShipStrategyType; import org.apache.flink.runtime.operators.util.LocalStrategy; import org.apache.flink.runtime.operators.util.TaskConfig; -import org.apache.flink.util.FileUtils; -import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.StringUtils; import org.apache.flink.util.Visitor; @@ -97,7 +94,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; -import java.nio.file.Files; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -264,7 +260,7 @@ public class JobGraphGenerator implements Visitor { program.getOriginalPlan().getCachedFiles().stream() .map(entry -> Tuple2.of(entry.getKey(), entry.getValue())) .collect(Collectors.toList()); - addUserArtifactEntries(userArtifacts, graph); + JobGraphUtils.addUserArtifactEntries(userArtifacts, graph); // release all references again this.vertices = null; @@ -278,35 +274,6 @@ public class JobGraphGenerator implements Visitor { return graph; } - public static void addUserArtifactEntries(Collection> userArtifacts, JobGraph jobGraph) { - if (userArtifacts != null && !userArtifacts.isEmpty()) { - try { - java.nio.file.Path tmpDir = Files.createTempDirectory("flink-distributed-cache-" + jobGraph.getJobID()); - for (Tuple2 originalEntry : userArtifacts) { - Path filePath = new Path(originalEntry.f1.filePath); - boolean isLocalDir = false; - try { - FileSystem sourceFs = filePath.getFileSystem(); - isLocalDir = !sourceFs.isDistributedFS() && sourceFs.getFileStatus(filePath).isDir(); - } catch (IOException ioe) { - LOG.warn("Could not determine whether {} denotes a local path.", filePath, ioe); - } - // zip local directories because we only support file uploads - DistributedCache.DistributedCacheEntry entry; - if (isLocalDir) { - Path zip = FileUtils.compressDirectory(filePath, new Path(tmpDir.toString(), filePath.getName() + ".zip")); - entry = new DistributedCache.DistributedCacheEntry(zip.toString(), originalEntry.f1.isExecutable, true); - } else { - entry = new DistributedCache.DistributedCacheEntry(filePath.toString(), originalEntry.f1.isExecutable, false); - } - jobGraph.addUserArtifact(originalEntry.f0, entry); - } - } catch (IOException ioe) { - throw new FlinkRuntimeException("Could not compress distributed-cache artifacts.", ioe); - } - } - } - /** * This methods implements the pre-visiting during a depth-first traversal. It create the job vertex and * sets local strategy. diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/plantranslate/JobGraphGeneratorTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/plantranslate/JobGraphGeneratorTest.java index bb483c37f0..bc21158133 100644 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/plantranslate/JobGraphGeneratorTest.java +++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/plantranslate/JobGraphGeneratorTest.java @@ -40,6 +40,7 @@ import org.apache.flink.optimizer.testfunctions.IdentityMapper; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobGraphUtils; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.util.AbstractID; @@ -245,7 +246,7 @@ public class JobGraphGeneratorTest { Tuple2.of(nonExecutableDirName, new DistributedCache.DistributedCacheEntry(directory2.toString(), false)) ); - JobGraphGenerator.addUserArtifactEntries(originalArtifacts, jb); + JobGraphUtils.addUserArtifactEntries(originalArtifacts, jb); Map submittedArtifacts = jb.getUserArtifacts(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraphUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraphUtils.java new file mode 100644 index 0000000000..ff786dca4c --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraphUtils.java @@ -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.flink.runtime.jobgraph; + +import org.apache.flink.api.common.cache.DistributedCache; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.util.FileUtils; +import org.apache.flink.util.FlinkRuntimeException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.file.Files; +import java.util.Collection; + +/** + * Utilities for generating {@link JobGraph}. + */ +public enum JobGraphUtils { + ; + + private static final Logger LOG = LoggerFactory.getLogger(JobGraphUtils.class); + + public static void addUserArtifactEntries(Collection> userArtifacts, JobGraph jobGraph) { + if (userArtifacts != null && !userArtifacts.isEmpty()) { + try { + java.nio.file.Path tmpDir = Files.createTempDirectory("flink-distributed-cache-" + jobGraph.getJobID()); + for (Tuple2 originalEntry : userArtifacts) { + Path filePath = new Path(originalEntry.f1.filePath); + boolean isLocalDir = false; + try { + FileSystem sourceFs = filePath.getFileSystem(); + isLocalDir = !sourceFs.isDistributedFS() && sourceFs.getFileStatus(filePath).isDir(); + } catch (IOException ioe) { + LOG.warn("Could not determine whether {} denotes a local path.", filePath, ioe); + } + // zip local directories because we only support file uploads + DistributedCache.DistributedCacheEntry entry; + if (isLocalDir) { + Path zip = FileUtils.compressDirectory(filePath, new Path(tmpDir.toString(), filePath.getName() + ".zip")); + entry = new DistributedCache.DistributedCacheEntry(zip.toString(), originalEntry.f1.isExecutable, true); + } else { + entry = new DistributedCache.DistributedCacheEntry(filePath.toString(), originalEntry.f1.isExecutable, false); + } + jobGraph.addUserArtifact(originalEntry.f0, entry); + } + } catch (IOException ioe) { + throw new FlinkRuntimeException("Could not compress distributed-cache artifacts.", ioe); + } + } + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java index ef11df043e..277e450ca1 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java @@ -25,7 +25,6 @@ import org.apache.flink.api.common.operators.ResourceSpec; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.IllegalConfigurationException; -import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; import org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy; import org.apache.flink.runtime.checkpoint.MasterTriggerRestoreHook; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; @@ -34,6 +33,7 @@ import org.apache.flink.runtime.jobgraph.InputOutputFormatContainer; import org.apache.flink.runtime.jobgraph.InputOutputFormatVertex; import org.apache.flink.runtime.jobgraph.JobEdge; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobGraphUtils; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.OperatorID; @@ -182,7 +182,7 @@ public class StreamingJobGraphGenerator { jobGraph.setSavepointRestoreSettings(streamGraph.getSavepointRestoreSettings()); - JobGraphGenerator.addUserArtifactEntries(streamGraph.getUserArtifacts(), jobGraph); + JobGraphUtils.addUserArtifactEntries(streamGraph.getUserArtifacts(), jobGraph); // set the ExecutionConfig last when it has been finalized try { -- Gitee From 0523ef6451a93da450c6bdf5dd4757c3702f3962 Mon Sep 17 00:00:00 2001 From: tison Date: Fri, 6 Mar 2020 16:56:22 +0800 Subject: [PATCH 151/885] [FLINK-15090][api] Reverse the dependency from flink-streaming-java to flink-clients This closes #10526 . --- flink-clients/pom.xml | 6 + .../org/apache/flink/client/ClientUtils.java | 19 ++- .../client/FlinkPipelineTranslationUtil.java | 42 ++---- .../flink/client}/StreamGraphTranslator.java | 4 +- .../client/program/ContextEnvironment.java | 14 +- .../program/OptimizerPlanEnvironment.java | 134 ++---------------- .../client/program/PackagedProgramUtils.java | 106 ++++++++++++-- ...actory.java => ProgramAbortException.java} | 38 +---- .../program}/StreamContextEnvironment.java | 49 ++++--- .../program}/StreamPlanEnvironment.java | 39 ++--- .../program/OptimizerPlanEnvironmentTest.java | 4 +- flink-python/pom.xml | 6 + .../flink/client/python/PythonDriver.java | 4 +- flink-streaming-java/pom.xml | 2 +- .../StreamExecutionEnvironment.java | 24 +--- ...treamingJobGraphGeneratorNodeHashTest.java | 4 +- 16 files changed, 220 insertions(+), 275 deletions(-) rename {flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph => flink-clients/src/main/java/org/apache/flink/client}/StreamGraphTranslator.java (95%) rename flink-clients/src/main/java/org/apache/flink/client/program/{ContextEnvironmentFactory.java => ProgramAbortException.java} (39%) rename {flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment => flink-clients/src/main/java/org/apache/flink/client/program}/StreamContextEnvironment.java (69%) rename {flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment => flink-clients/src/main/java/org/apache/flink/client/program}/StreamPlanEnvironment.java (60%) diff --git a/flink-clients/pom.xml b/flink-clients/pom.xml index fa2c5e5266..8781d7b7eb 100644 --- a/flink-clients/pom.xml +++ b/flink-clients/pom.xml @@ -82,6 +82,12 @@ under the License. test test-jar + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${project.version} + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + + + diff --git a/flink-ml-parent/flink-ml-lib/src/test/java/org/apache/flink/ml/pipeline/UserDefinedPipelineStages.java b/flink-ml-parent/flink-ml-lib/src/test/java/org/apache/flink/ml/pipeline/UserDefinedPipelineStages.java new file mode 100644 index 0000000000..7c5fd983be --- /dev/null +++ b/flink-ml-parent/flink-ml-lib/src/test/java/org/apache/flink/ml/pipeline/UserDefinedPipelineStages.java @@ -0,0 +1,54 @@ +/* + * 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.ml.pipeline; + +import org.apache.flink.ml.api.core.Transformer; +import org.apache.flink.ml.api.misc.param.Params; +import org.apache.flink.ml.params.shared.colname.HasSelectedCols; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; + +/** + * Util class for testing {@link org.apache.flink.ml.api.core.PipelineStage}. + */ +public class UserDefinedPipelineStages { + + /** + * A {@link Transformer} which is used to perform column selection. + */ + public static class SelectColumnTransformer implements + Transformer, HasSelectedCols { + + private Params params; + + public SelectColumnTransformer() { + this.params = new Params(); + } + + @Override + public Table transform(TableEnvironment tEnv, Table input) { + return input.select(String.join(", ", this.getSelectedCols())); + } + + @Override + public Params getParams() { + return params; + } + } +} diff --git a/flink-python/pyflink/ml/api/__init__.py b/flink-python/pyflink/ml/api/__init__.py index faca34a226..90b3eb8050 100644 --- a/flink-python/pyflink/ml/api/__init__.py +++ b/flink-python/pyflink/ml/api/__init__.py @@ -18,7 +18,11 @@ from pyflink.ml.api.ml_environment import MLEnvironment from pyflink.ml.api.ml_environment_factory import MLEnvironmentFactory +from pyflink.ml.api.base import Transformer, Estimator, Model, Pipeline, \ + PipelineStage, JavaTransformer, JavaEstimator, JavaModel + __all__ = [ - "MLEnvironment", "MLEnvironmentFactory" + "MLEnvironment", "MLEnvironmentFactory", "Transformer", "Estimator", "Model", + "Pipeline", "PipelineStage", "JavaTransformer", "JavaEstimator", "JavaModel" ] diff --git a/flink-python/pyflink/ml/api/base.py b/flink-python/pyflink/ml/api/base.py new file mode 100644 index 0000000000..8888df3a83 --- /dev/null +++ b/flink-python/pyflink/ml/api/base.py @@ -0,0 +1,275 @@ +################################################################################ +# 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. +################################################################################ + +import re + +from abc import ABCMeta, abstractmethod + +from pyflink.table.table_environment import TableEnvironment +from pyflink.table.table import Table +from pyflink.ml.api.param import WithParams, Params +from py4j.java_gateway import get_field + + +class PipelineStage(WithParams): + """ + Base class for a stage in a pipeline. The interface is only a concept, and does not have any + actual functionality. Its subclasses must be either Estimator or Transformer. No other classes + should inherit this interface directly. + + Each pipeline stage is with parameters, and requires a public empty constructor for + restoration in Pipeline. + """ + + def __init__(self, params=None): + if params is None: + self._params = Params() + else: + self._params = params + + def get_params(self) -> Params: + return self._params + + def _convert_params_to_java(self, j_pipeline_stage): + for param in self._params._param_map: + java_param = self._make_java_param(j_pipeline_stage, param) + java_value = self._make_java_value(self._params._param_map[param]) + j_pipeline_stage.set(java_param, java_value) + + @staticmethod + def _make_java_param(j_pipeline_stage, param): + # camel case to snake case + name = re.sub(r'(? str: + return self.get_params().to_json() + + def load_json(self, json: str) -> None: + self.get_params().load_json(json) + + +class Transformer(PipelineStage): + """ + A transformer is a PipelineStage that transforms an input Table to a result Table. + """ + + __metaclass__ = ABCMeta + + @abstractmethod + def transform(self, table_env: TableEnvironment, table: Table) -> Table: + """ + Applies the transformer on the input table, and returns the result table. + + :param table_env: the table environment to which the input table is bound. + :param table: the table to be transformed + :returns: the transformed table + """ + raise NotImplementedError() + + +class JavaTransformer(Transformer): + """ + Base class for Transformer that wrap Java implementations. Subclasses should + ensure they have the transformer Java object available as j_obj. + """ + + def __init__(self, j_obj): + super().__init__() + self._j_obj = j_obj + + def transform(self, table_env: TableEnvironment, table: Table) -> Table: + """ + Applies the transformer on the input table, and returns the result table. + + :param table_env: the table environment to which the input table is bound. + :param table: the table to be transformed + :returns: the transformed table + """ + self._convert_params_to_java(self._j_obj) + return Table(self._j_obj.transform(table_env._j_tenv, table._j_table)) + + +class Model(Transformer): + """ + Abstract class for models that are fitted by estimators. + + A model is an ordinary Transformer except how it is created. While ordinary transformers + are defined by specifying the parameters directly, a model is usually generated by an Estimator + when Estimator.fit(table_env, table) is invoked. + """ + + __metaclass__ = ABCMeta + + +class JavaModel(JavaTransformer, Model): + """ + Base class for JavaTransformer that wrap Java implementations. + Subclasses should ensure they have the model Java object available as j_obj. + """ + + +class Estimator(PipelineStage): + """ + Estimators are PipelineStages responsible for training and generating machine learning models. + + The implementations are expected to take an input table as training samples and generate a + Model which fits these samples. + """ + + __metaclass__ = ABCMeta + + def fit(self, table_env: TableEnvironment, table: Table) -> Model: + """ + Train and produce a Model which fits the records in the given Table. + + :param table_env: the table environment to which the input table is bound. + :param table: the table with records to train the Model. + :returns: a model trained to fit on the given Table. + """ + raise NotImplementedError() + + +class JavaEstimator(Estimator): + """ + Base class for Estimator that wrap Java implementations. + Subclasses should ensure they have the estimator Java object available as j_obj. + """ + + def __init__(self, j_obj): + super().__init__() + self._j_obj = j_obj + + def fit(self, table_env: TableEnvironment, table: Table) -> JavaModel: + """ + Train and produce a Model which fits the records in the given Table. + + :param table_env: the table environment to which the input table is bound. + :param table: the table with records to train the Model. + :returns: a model trained to fit on the given Table. + """ + self._convert_params_to_java(self._j_obj) + return JavaModel(self._j_obj.fit(table_env._j_tenv, table._j_table)) + + +class Pipeline(Estimator, Model, Transformer): + """ + A pipeline is a linear workflow which chains Estimators and Transformers to + execute an algorithm. + + A pipeline itself can either act as an Estimator or a Transformer, depending on the stages it + includes. More specifically: + + + If a Pipeline has an Estimator, one needs to call `Pipeline.fit(TableEnvironment, Table)` + before use the pipeline as a Transformer. In this case the Pipeline is an Estimator and + can produce a Pipeline as a `Model`. + + If a Pipeline has noEstimator, it is a Transformer and can be applied to a Table directly. + In this case, `Pipeline#fit(TableEnvironment, Table)` will simply return the pipeline itself. + + + In addition, a pipeline can also be used as a PipelineStage in another pipeline, just like an + ordinaryEstimator or Transformer as describe above. + """ + + def __init__(self, stages=None, pipeline_json=None): + super().__init__() + self.stages = [] + self.last_estimator_index = -1 + if stages is not None: + for stage in stages: + self.append_stage(stage) + if pipeline_json is not None: + self.load_json(pipeline_json) + + def need_fit(self): + return self.last_estimator_index >= 0 + + @staticmethod + def _is_stage_need_fit(stage): + return (isinstance(stage, Pipeline) and stage.need_fit()) or \ + ((not isinstance(stage, Pipeline)) and isinstance(stage, Estimator)) + + def get_stages(self) -> tuple: + # make it immutable by changing to tuple + return tuple(self.stages) + + def append_stage(self, stage: PipelineStage) -> 'Pipeline': + if self._is_stage_need_fit(stage): + self.last_estimator_index = len(self.stages) + elif not isinstance(stage, Transformer): + raise RuntimeError("All PipelineStages should be Estimator or Transformer!") + self.stages.append(stage) + return self + + def fit(self, t_env: TableEnvironment, input: Table) -> 'Pipeline': + """ + Train the pipeline to fit on the records in the given Table. + + :param t_env: the table environment to which the input table is bound. + :param input: the table with records to train the Pipeline. + :returns: a pipeline with same stages as this Pipeline except all Estimators \ + replaced with their corresponding Models. + """ + transform_stages = [] + for i in range(0, len(self.stages)): + s = self.stages[i] + if i <= self.last_estimator_index: + need_fit = self._is_stage_need_fit(s) + if need_fit: + t = s.fit(t_env, input) + else: + t = s + transform_stages.append(t) + input = t.transform(t_env, input) + else: + transform_stages.append(s) + return Pipeline(transform_stages) + + def transform(self, t_env: TableEnvironment, input: Table) -> Table: + """ + Generate a result table by applying all the stages in this pipeline to + the input table in order. + + :param t_env: the table environment to which the input table is bound. + :param input: the table to be transformed. + :returns: a result table with all the stages applied to the input tables in order. + """ + if self.need_fit(): + raise RuntimeError("Pipeline contains Estimator, need to fit first.") + for s in self.stages: + input = s.transform(t_env, input) + return input + + def to_json(self) -> str: + import jsonpickle + return str(jsonpickle.encode(self, keys=True)) + + def load_json(self, json: str) -> None: + import jsonpickle + pipeline = jsonpickle.decode(json, keys=True) + for stage in pipeline.get_stages(): + self.append_stage(stage) diff --git a/flink-python/pyflink/ml/api/param/base.py b/flink-python/pyflink/ml/api/param/base.py index b784bc766d..5188f2caf8 100644 --- a/flink-python/pyflink/ml/api/param/base.py +++ b/flink-python/pyflink/ml/api/param/base.py @@ -164,17 +164,16 @@ class Params(Generic[V]): import jsonpickle return str(jsonpickle.encode(self._param_map, keys=True)) - def load_json(self, json: str) -> 'Params': + def load_json(self, json: str) -> None: """ Restores the parameters from the given json. The parameters should be exactly the same with the one who was serialized to the input json after the restoration. :param json: the json String to restore from. - :return: the Params. + :return: None. """ import jsonpickle self._param_map.update(jsonpickle.decode(json, keys=True)) - return self @staticmethod def from_json(json) -> 'Params': @@ -184,7 +183,9 @@ class Params(Generic[V]): :param json: the json string to load. :return: the `Params` loaded from the json string. """ - return Params().load_json(json) + ret = Params() + ret.load_json(json) + return ret def merge(self, other_params: 'Params') -> 'Params': """ diff --git a/flink-python/pyflink/ml/lib/param/colname.py b/flink-python/pyflink/ml/lib/param/colname.py index 034cd4c58d..581551bdd0 100644 --- a/flink-python/pyflink/ml/lib/param/colname.py +++ b/flink-python/pyflink/ml/lib/param/colname.py @@ -53,3 +53,20 @@ class HasOutputCol(WithParams): def get_output_col(self) -> str: return super().get(self.output_col) + + +class HasPredictionCol(WithParams): + """ + An interface for classes with a parameter specifying the column name of the prediction. + """ + prediction_col = ParamInfo( + "predictionCol", + "Column name of prediction.", + is_optional=False, + type_converter=TypeConverters.to_string) + + def set_prediction_col(self, v: str) -> 'HasPredictionCol': + return super().set(self.prediction_col, v) + + def get_prediction_col(self) -> str: + return super().get(self.prediction_col) diff --git a/flink-python/pyflink/ml/tests/test_pipeline.py b/flink-python/pyflink/ml/tests/test_pipeline.py new file mode 100644 index 0000000000..31c3068741 --- /dev/null +++ b/flink-python/pyflink/ml/tests/test_pipeline.py @@ -0,0 +1,150 @@ +################################################################################ +# 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. +################################################################################ + +import unittest +from pyflink.ml.api import JavaTransformer, Transformer, Estimator, Model, \ + Pipeline, JavaEstimator, JavaModel +from pyflink.ml.api.param.base import WithParams, ParamInfo, TypeConverters +from pyflink import keyword + + +class PipelineTest(unittest.TestCase): + + @staticmethod + def describe_pipeline(pipeline): + res = [stage.get_desc() for stage in pipeline.get_stages()] + return "_".join(res) + + def test_construct_pipeline(self): + pipeline1 = Pipeline() + pipeline1.append_stage(MockTransformer(self_desc="a1")) + pipeline1.append_stage(MockJavaTransformer(self_desc="ja1")) + + pipeline2 = Pipeline() + pipeline2.append_stage(MockTransformer(self_desc="a2")) + pipeline2.append_stage(MockJavaTransformer(self_desc="ja2")) + + pipeline3 = Pipeline(pipeline1.get_stages(), pipeline2.to_json()) + self.assertEqual("a1_ja1_a2_ja2", PipelineTest.describe_pipeline(pipeline3)) + + def test_pipeline_behavior(self): + pipeline = Pipeline() + pipeline.append_stage(MockTransformer(self_desc="a")) + pipeline.append_stage(MockJavaTransformer(self_desc="ja")) + pipeline.append_stage(MockEstimator(self_desc="b")) + pipeline.append_stage(MockJavaEstimator(self_desc="jb")) + pipeline.append_stage(MockEstimator(self_desc="c")) + pipeline.append_stage(MockTransformer(self_desc="d")) + self.assertEqual("a_ja_b_jb_c_d", PipelineTest.describe_pipeline(pipeline)) + + pipeline_model = pipeline.fit(None, None) + self.assertEqual("a_ja_mb_mjb_mc_d", PipelineTest.describe_pipeline(pipeline_model)) + + def test_pipeline_restore(self): + pipeline = Pipeline() + pipeline.append_stage(MockTransformer(self_desc="a")) + pipeline.append_stage(MockJavaTransformer(self_desc="ja")) + pipeline.append_stage(MockEstimator(self_desc="b")) + pipeline.append_stage(MockJavaEstimator(self_desc="jb")) + pipeline.append_stage(MockEstimator(self_desc="c")) + pipeline.append_stage(MockTransformer(self_desc="d")) + + pipeline_new = Pipeline() + pipeline_new.load_json(pipeline.to_json()) + self.assertEqual("a_ja_b_jb_c_d", PipelineTest.describe_pipeline(pipeline_new)) + + pipeline_model = pipeline_new.fit(None, None) + self.assertEqual("a_ja_mb_mjb_mc_d", PipelineTest.describe_pipeline(pipeline_model)) + + +class SelfDescribe(WithParams): + self_desc = ParamInfo("selfDesc", "selfDesc", type_converter=TypeConverters.to_string) + + def set_desc(self, v): + return super().set(self.self_desc, v) + + def get_desc(self): + return super().get(self.self_desc) + + +class MockTransformer(Transformer, SelfDescribe): + @keyword + def __init__(self, *, self_desc=None): + super().__init__() + kwargs = self._input_kwargs + self._set(**kwargs) + + def transform(self, table_env, table): + return table + + +class MockEstimator(Estimator, SelfDescribe): + @keyword + def __init__(self, *, self_desc=None): + super().__init__() + self._self_desc = self_desc + kwargs = self._input_kwargs + self._set(**kwargs) + + def fit(self, table_env, table): + return MockModel(self_desc="m" + self._self_desc) + + +class MockModel(Model, SelfDescribe): + @keyword + def __init__(self, *, self_desc=None): + super().__init__() + kwargs = self._input_kwargs + self._set(**kwargs) + + def transform(self, table_env, table): + return table + + +class MockJavaTransformer(JavaTransformer, SelfDescribe): + @keyword + def __init__(self, *, self_desc=None): + super().__init__(None) + kwargs = self._input_kwargs + self._set(**kwargs) + + def transform(self, table_env, table): + return table + + +class MockJavaEstimator(JavaEstimator, SelfDescribe): + @keyword + def __init__(self, *, self_desc=None): + super().__init__(None) + self._self_desc = self_desc + kwargs = self._input_kwargs + self._set(**kwargs) + + def fit(self, table_env, table): + return MockJavaModel(self_desc="m" + self._self_desc) + + +class MockJavaModel(JavaModel, SelfDescribe): + @keyword + def __init__(self, *, self_desc=None): + super().__init__(None) + kwargs = self._input_kwargs + self._set(**kwargs) + + def transform(self, table_env, table): + return table diff --git a/flink-python/pyflink/ml/tests/test_pipeline_it_case.py b/flink-python/pyflink/ml/tests/test_pipeline_it_case.py new file mode 100644 index 0000000000..655d20c4df --- /dev/null +++ b/flink-python/pyflink/ml/tests/test_pipeline_it_case.py @@ -0,0 +1,171 @@ +################################################################################ +# 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. +################################################################################ + +from pyflink.table.types import DataTypes +from pyflink.testing.test_case_utils import MLTestCase + +from pyflink.ml.api import JavaTransformer, Transformer, Estimator, Model, \ + MLEnvironmentFactory, Pipeline +from pyflink.ml.api.param import WithParams, ParamInfo, TypeConverters +from pyflink.ml.lib.param.colname import HasSelectedCols,\ + HasPredictionCol, HasOutputCol +from pyflink import keyword +from pyflink.testing import source_sink_utils +from pyflink.java_gateway import get_gateway + + +class HasVectorCol(WithParams): + """ + Trait for parameter vectorColName. + """ + vector_col = ParamInfo( + "vectorCol", + "Name of a vector column", + is_optional=False, + type_converter=TypeConverters.to_string) + + def set_vector_col(self, v: str) -> 'HasVectorCol': + return super().set(self.vector_col, v) + + def get_vector_col(self) -> str: + return super().get(self.vector_col) + + +class WrapperTransformer(JavaTransformer, HasSelectedCols): + """ + A Transformer wrappers Java Transformer. + """ + @keyword + def __init__(self, *, selected_cols=None): + _j_obj = get_gateway().jvm.org.apache.flink.ml.pipeline.\ + UserDefinedPipelineStages.SelectColumnTransformer() + super().__init__(_j_obj) + kwargs = self._input_kwargs + self._set(**kwargs) + + +class PythonAddTransformer(Transformer, HasSelectedCols, HasOutputCol): + """ + A Transformer which is implemented with Python. Output a column + contains the sum of all columns. + """ + @keyword + def __init__(self, *, selected_cols=None, output_col=None): + super().__init__() + kwargs = self._input_kwargs + self._set(**kwargs) + + def transform(self, table_env, table): + input_columns = self.get_selected_cols() + expr = "+".join(input_columns) + expr = expr + " as " + self.get_output_col() + return table.add_columns(expr) + + +class PythonEstimator(Estimator, HasVectorCol, HasPredictionCol): + + def __init__(self): + super().__init__() + + def fit(self, table_env, table): + return PythonModel( + table_env, + table.select("max(features) as max_sum"), + self.get_prediction_col()) + + +class PythonModel(Model): + + def __init__(self, table_env, model_data_table, output_col_name): + self._model_data_table = model_data_table + self._output_col_name = output_col_name + self.max_sum = 0 + self.load_model(table_env) + + def load_model(self, table_env): + """ + Train the model to get the max_sum value which is used to predict data. + """ + table_sink = source_sink_utils.TestRetractSink(["max_sum"], [DataTypes.BIGINT()]) + table_env.register_table_sink("Model_Results", table_sink) + self._model_data_table.insert_into("Model_Results") + table_env.execute("load model") + actual = source_sink_utils.results() + self.max_sum = actual.apply(0) + + def transform(self, table_env, table): + """ + Use max_sum to predict input. Return turn if input value is bigger than max_sum + """ + return table\ + .add_columns("features > {} as {}".format(self.max_sum, self._output_col_name))\ + .select("{}".format(self._output_col_name)) + + +class PythonPipelineTest(MLTestCase): + + def test_java_transformer(self): + t_env = MLEnvironmentFactory().get_default().get_stream_table_environment() + + table_sink = source_sink_utils.TestAppendSink( + ['a', 'b'], [DataTypes.BIGINT(), DataTypes.BIGINT()]) + t_env.register_table_sink("TransformerResults", table_sink) + + source_table = t_env.from_elements([(1, 2, 3, 4), (4, 3, 2, 1)], ['a', 'b', 'c', 'd']) + transformer = WrapperTransformer(selected_cols=["a", "b"]) + transformer\ + .transform(t_env, source_table)\ + .insert_into("TransformerResults") + + # execute + t_env.execute('JavaPipelineITCase') + actual = source_sink_utils.results() + self.assert_equals(actual, ["1,2", "4,3"]) + + def test_pipeline(self): + t_env = MLEnvironmentFactory().get_default().get_stream_table_environment() + train_table = t_env.from_elements( + [(1, 2), (1, 4), (1, 0), (10, 2), (10, 4), (10, 0)], ['a', 'b']) + serving_table = t_env.from_elements([(0, 0), (12, 3)], ['a', 'b']) + + table_sink = source_sink_utils.TestAppendSink( + ['predict_result'], + [DataTypes.BOOLEAN()]) + t_env.register_table_sink("PredictResults", table_sink) + + # transformer, output features column which is the sum of a and b. + transformer = PythonAddTransformer(selected_cols=["a", "b"], output_col="features") + + # estimator + estimator = PythonEstimator()\ + .set_vector_col("features")\ + .set_prediction_col("predict_result") + + # pipeline + pipeline = Pipeline().append_stage(transformer).append_stage(estimator) + pipeline\ + .fit(t_env, train_table)\ + .transform(t_env, serving_table)\ + .insert_into('PredictResults') + # execute + t_env.execute('PipelineITCase') + + actual = source_sink_utils.results() + # the first input is false since 0 + 0 is smaller than the max_sum 14. + # the second input is true since 12 + 3 is bigger than the max_sum 14. + self.assert_equals(actual, ["false", "true"]) diff --git a/flink-python/pyflink/ml/tests/test_pipeline_stage.py b/flink-python/pyflink/ml/tests/test_pipeline_stage.py new file mode 100644 index 0000000000..cce53f2516 --- /dev/null +++ b/flink-python/pyflink/ml/tests/test_pipeline_stage.py @@ -0,0 +1,89 @@ +################################################################################ +# 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. +################################################################################ + +from abc import ABC, abstractmethod +from pyflink.ml.api import Transformer, Estimator, MLEnvironmentFactory +import unittest + + +class PipelineStageTestBase(ABC): + """ + The base class for testing the base implementation of pipeline stages, i.e. Estimators + and Transformers. This class is package private because we do not expect extension outside + of the package. + """ + + @abstractmethod + def create_pipeline_stage(self): + pass + + +class TransformerBaseTest(PipelineStageTestBase, unittest.TestCase): + """ + Test for TransformerBase. + """ + def test_fit_table(self): + id = MLEnvironmentFactory.get_new_ml_environment_id() + env = MLEnvironmentFactory.get(id) + table = env.get_stream_table_environment().from_elements([(1, 2, 3)]) + transformer = self.create_pipeline_stage() + transformer.transform(env.get_stream_table_environment(), table) + self.assertTrue(transformer.transformed) + + def create_pipeline_stage(self): + return self.FakeTransFormer() + + class FakeTransFormer(Transformer): + """ + This fake transformer simply record which transform method is invoked. + """ + + def __init__(self): + self.transformed = False + + def transform(self, table_env, table): + self.transformed = True + return table + + +class EstimatorBaseTest(PipelineStageTestBase, unittest.TestCase): + """ + Test for EstimatorBase. + """ + def test_fit_table(self): + id = MLEnvironmentFactory.get_new_ml_environment_id() + env = MLEnvironmentFactory.get(id) + table = env.get_stream_table_environment().from_elements([(1, 2, 3)]) + estimator = self.create_pipeline_stage() + estimator.fit(env.get_stream_table_environment(), table) + self.assertTrue(estimator.fitted) + + def create_pipeline_stage(self): + return self.FakeEstimator() + + class FakeEstimator(Estimator): + """ + This fake estimator simply record which fit method is invoked. + """ + + def __init__(self): + self.fitted = False + + def fit(self, table_env, table): + self.fitted = True + return None -- Gitee From 73387194e5cfc84f45eb0b9fb6e66525a85be848 Mon Sep 17 00:00:00 2001 From: Gao Yun Date: Thu, 13 Feb 2020 01:15:22 +0800 Subject: [PATCH 154/885] [hotfix][network] Fix the comments error in NettyMessage --- .../apache/flink/runtime/io/network/netty/NettyMessage.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java index ea41b42ddb..9dfd05329f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java @@ -84,7 +84,7 @@ public abstract class NettyMessage { * {@link NettyMessage} subclass ID * * @return a newly allocated direct buffer with header data written for {@link - * NettyMessageDecoder} + * NettyMessageEncoder} */ private static ByteBuf allocateBuffer(ByteBufAllocator allocator, byte id) { return allocateBuffer(allocator, id, -1); @@ -105,7 +105,7 @@ public abstract class NettyMessage { * content length (or -1 if unknown) * * @return a newly allocated direct buffer with header data written for {@link - * NettyMessageDecoder} + * NettyMessageEncoder} */ private static ByteBuf allocateBuffer(ByteBufAllocator allocator, byte id, int contentLength) { return allocateBuffer(allocator, id, 0, contentLength, true); @@ -131,7 +131,7 @@ public abstract class NettyMessage { * only return a buffer with the header information (false) * * @return a newly allocated direct buffer with header data written for {@link - * NettyMessageDecoder} + * NettyMessageEncoder} */ private static ByteBuf allocateBuffer( ByteBufAllocator allocator, -- Gitee From db5d753da7dbac3cc274bba683081503be94c99d Mon Sep 17 00:00:00 2001 From: Gao Yun Date: Mon, 24 Jun 2019 00:51:41 +0800 Subject: [PATCH 155/885] [FLINK-10742][network] Let Netty use Flink's buffers directly on receiver side For credit-based flow control, we always have buffers available for data that is sent to use. We could thus use them directly and not copy the network stream into Netty buffers first and then into our buffers. This improvement can reduce the total Netty memory overhead to aovid potential direct OutOfMemoryError in large scale jobs. --- .../pom.xml | 71 ++++ .../NettyShuffleMemoryControlTestProgram.java | 140 +++++++ flink-end-to-end-tests/pom.xml | 1 + .../test_netty_shuffle_memory_control.sh | 49 +++ .../io/network/NetworkClientHandler.java | 5 + .../network/netty/BufferResponseDecoder.java | 119 ++++++ .../io/network/netty/ByteBufUtils.java | 64 ++++ ...ditBasedPartitionRequestClientHandler.java | 56 +-- .../io/network/netty/NettyMessage.java | 99 +++-- .../NettyMessageClientDecoderDelegate.java | 153 ++++++++ .../io/network/netty/NettyMessageDecoder.java | 92 +++++ .../io/network/netty/NettyProtocol.java | 9 +- .../network/netty/NetworkBufferAllocator.java | 77 ++++ .../netty/NonBufferResponseDecoder.java | 85 +++++ .../io/network/netty/ByteBufUtilsTest.java | 129 +++++++ ...asedPartitionRequestClientHandlerTest.java | 163 ++++++-- ...NettyMessageClientDecoderDelegateTest.java | 355 ++++++++++++++++++ ...ttyMessageClientSideSerializationTest.java | 212 +++++++++++ .../netty/NettyMessageSerializationTest.java | 239 ------------ ...ttyMessageServerSideSerializationTest.java | 115 ++++++ .../io/network/netty/NettyTestUtil.java | 40 ++ tools/travis/splits/split_misc.sh | 2 + 22 files changed, 1932 insertions(+), 343 deletions(-) create mode 100644 flink-end-to-end-tests/flink-netty-shuffle-memory-control-test/pom.xml create mode 100644 flink-end-to-end-tests/flink-netty-shuffle-memory-control-test/src/main/java/org/apache/flink/streaming/tests/NettyShuffleMemoryControlTestProgram.java create mode 100755 flink-end-to-end-tests/test-scripts/test_netty_shuffle_memory_control.sh create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/BufferResponseDecoder.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/ByteBufUtils.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientDecoderDelegate.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessageDecoder.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NetworkBufferAllocator.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NonBufferResponseDecoder.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ByteBufUtilsTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientDecoderDelegateTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientSideSerializationTest.java delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageSerializationTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageServerSideSerializationTest.java diff --git a/flink-end-to-end-tests/flink-netty-shuffle-memory-control-test/pom.xml b/flink-end-to-end-tests/flink-netty-shuffle-memory-control-test/pom.xml new file mode 100644 index 0000000000..b094bf65de --- /dev/null +++ b/flink-end-to-end-tests/flink-netty-shuffle-memory-control-test/pom.xml @@ -0,0 +1,71 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-end-to-end-tests + 1.11-SNAPSHOT + .. + + + flink-netty-shuffle-memory-control-test + flink-netty-shuffle-memory-control-test + jar + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${project.version} + provided + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + NettyShuffleMemoryControlTestProgram + package + + shade + + + NettyShuffleMemoryControlTestProgram + + + org.apache.flink.streaming.tests.NettyShuffleMemoryControlTestProgram + + + + + + + + + + diff --git a/flink-end-to-end-tests/flink-netty-shuffle-memory-control-test/src/main/java/org/apache/flink/streaming/tests/NettyShuffleMemoryControlTestProgram.java b/flink-end-to-end-tests/flink-netty-shuffle-memory-control-test/src/main/java/org/apache/flink/streaming/tests/NettyShuffleMemoryControlTestProgram.java new file mode 100644 index 0000000000..e5a75bef97 --- /dev/null +++ b/flink-end-to-end-tests/flink-netty-shuffle-memory-control-test/src/main/java/org/apache/flink/streaming/tests/NettyShuffleMemoryControlTestProgram.java @@ -0,0 +1,140 @@ +/* + * 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.tests; + +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; + +import org.apache.flink.shaded.netty4.io.netty.util.internal.OutOfDirectMemoryError; + +import sun.misc.Unsafe; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * Test program to verify the direct memory consumption of Netty. Without zero-copy Netty + * may create more than one chunk, thus we may encounter {@link OutOfDirectMemoryError} if + * we limit the total direct memory to be less than two chunks. Instead, with zero-copy + * introduced in (https://issues.apache.org/jira/browse/FLINK-10742) one chunk will be + * enough and the exception will not occur. + * + *

    Since Netty uses low level API of {@link Unsafe} to allocate direct buffer when using + * JDK8 and these memory will not be counted in direct memory, the test is only effective + * when JDK11 is used. + */ +public class NettyShuffleMemoryControlTestProgram { + private static final int RECORD_LENGTH = 2048; + + private static final ConfigOption RUNNING_TIME_IN_SECONDS = ConfigOptions + .key("test.running_time_in_seconds") + .defaultValue(120) + .withDescription("The time to run."); + + private static final ConfigOption MAP_PARALLELISM = ConfigOptions + .key("test.map_parallelism") + .defaultValue(1) + .withDescription("The number of map tasks."); + + private static final ConfigOption REDUCE_PARALLELISM = ConfigOptions + .key("test.reduce_parallelism") + .defaultValue(1) + .withDescription("The number of reduce tasks."); + + public static void main(String[] args) throws Exception { + // parse the parameters + final ParameterTool params = ParameterTool.fromArgs(args); + + final int runningTimeInSeconds = params.getInt(RUNNING_TIME_IN_SECONDS.key(), RUNNING_TIME_IN_SECONDS.defaultValue()); + final int mapParallelism = params.getInt(MAP_PARALLELISM.key(), MAP_PARALLELISM.defaultValue()); + final int reduceParallelism = params.getInt(REDUCE_PARALLELISM.key(), REDUCE_PARALLELISM.defaultValue()); + + checkArgument(runningTimeInSeconds > 0, + "The running time in seconds should be positive, but it is {}", + runningTimeInSeconds); + checkArgument(mapParallelism > 0, + "The number of map tasks should be positive, but it is {}", + mapParallelism); + checkArgument(reduceParallelism > 0, + "The number of reduce tasks should be positve, but it is {}", + reduceParallelism); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.addSource(new StringSourceFunction(runningTimeInSeconds)) + .setParallelism(mapParallelism) + .slotSharingGroup("a") + .shuffle() + .addSink(new DummySink()) + .setParallelism(reduceParallelism) + .slotSharingGroup("b"); + + // execute program + env.execute("Netty Shuffle Memory Control Test"); + } + + private static class StringSourceFunction extends RichParallelSourceFunction { + private static final long serialVersionUID = 1L; + + private volatile boolean isRunning; + + private final long runningTimeInSeconds; + + private transient long stopTime; + + public StringSourceFunction(long runningTimeInSeconds) { + this.runningTimeInSeconds = runningTimeInSeconds; + } + + @Override + public void open(Configuration parameters) { + isRunning = true; + stopTime = System.nanoTime() + runningTimeInSeconds * 1_000_000_000L; + } + + @Override + public void run(SourceContext ctx) { + byte[] bytes = new byte[RECORD_LENGTH]; + for (int i = 0; i < RECORD_LENGTH; ++i) { + bytes[i] = 'a'; + } + String str = new String(bytes); + + while (isRunning && (System.nanoTime() < stopTime)) { + ctx.collect(str); + } + } + + @Override + public void cancel() { + isRunning = false; + } + } + + private static class DummySink extends RichSinkFunction { + + @Override + public void invoke(String value, Context context) throws Exception { + // Do nothing. + } + } +} diff --git a/flink-end-to-end-tests/pom.xml b/flink-end-to-end-tests/pom.xml index 6ea395eec4..8689fae199 100644 --- a/flink-end-to-end-tests/pom.xml +++ b/flink-end-to-end-tests/pom.xml @@ -88,6 +88,7 @@ under the License. flink-elasticsearch7-test flink-end-to-end-tests-common-kafka flink-tpcds-test + flink-netty-shuffle-memory-control-test diff --git a/flink-end-to-end-tests/test-scripts/test_netty_shuffle_memory_control.sh b/flink-end-to-end-tests/test-scripts/test_netty_shuffle_memory_control.sh new file mode 100755 index 0000000000..cefc4cd447 --- /dev/null +++ b/flink-end-to-end-tests/test-scripts/test_netty_shuffle_memory_control.sh @@ -0,0 +1,49 @@ +#!/usr/bin/env bash +################################################################################ +# 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. +################################################################################ + +source "$(dirname "$0")"/common.sh + +TEST=flink-netty-shuffle-memory-control-test +TEST_PROGRAM_NAME=NettyShuffleMemoryControlTestProgram +TEST_PROGRAM_JAR=${END_TO_END_DIR}/$TEST/target/$TEST_PROGRAM_NAME.jar + +set_config_key "taskmanager.memory.flink.size" "512m" +set_config_key "taskmanager.memory.network.min" "128m" +set_config_key "taskmanager.memory.network.max" "128m" + +# 20 slots per task manager. +set_config_key "taskmanager.numberOfTaskSlots" "20" + +# Sets only one arena per TM for boosting the netty internal memory overhead. +set_config_key "taskmanager.network.netty.num-arenas" "1" + +# Limits the direct memory to be one chunk (16M) plus some margins. +set_config_key "taskmanager.memory.framework.off-heap.size" "20m" + +# Starts the cluster which includes one TaskManager. +start_cluster + +# Starts 4 more TaskManagers. Then we will have 5 TaskManagers and 100 slots in total. +start_taskmanagers 4 + +# Starts a job with 80 map tasks and 20 reduce tasks. +$FLINK_DIR/bin/flink run ${TEST_PROGRAM_JAR} \ +--test.map_parallelism 80 \ +--test.reduce_parallelism 20 \ +--test.running_time_in_seconds 120 diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkClientHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkClientHandler.java index 7f631b4ea3..e36ef8be3c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkClientHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkClientHandler.java @@ -23,6 +23,8 @@ import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler; +import javax.annotation.Nullable; + import java.io.IOException; /** @@ -34,6 +36,9 @@ public interface NetworkClientHandler extends ChannelHandler { void removeInputChannel(RemoteInputChannel inputChannel); + @Nullable + RemoteInputChannel getInputChannel(InputChannelID inputChannelId); + void cancelRequestFor(InputChannelID inputChannelId); /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/BufferResponseDecoder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/BufferResponseDecoder.java new file mode 100644 index 0000000000..508f469e96 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/BufferResponseDecoder.java @@ -0,0 +1,119 @@ +/* + * 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.runtime.io.network.netty; + +import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext; + +import javax.annotation.Nullable; + +import static org.apache.flink.runtime.io.network.netty.NettyMessage.BufferResponse; +import static org.apache.flink.runtime.io.network.netty.NettyMessage.BufferResponse.MESSAGE_HEADER_LENGTH; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * The decoder for {@link BufferResponse}. + */ +class BufferResponseDecoder extends NettyMessageDecoder { + + /** The Buffer allocator. */ + private final NetworkBufferAllocator allocator; + + /** The accumulation buffer of message header. */ + private ByteBuf messageHeaderBuffer; + + /** + * The BufferResponse message that has its message header decoded, but still + * not received all the bytes of the buffer part. + */ + @Nullable + private BufferResponse bufferResponse; + + /** How many bytes have been received or discarded for the data buffer part. */ + private int decodedDataBufferSize; + + BufferResponseDecoder(NetworkBufferAllocator allocator) { + this.allocator = checkNotNull(allocator); + } + + @Override + public void onChannelActive(ChannelHandlerContext ctx) { + messageHeaderBuffer = ctx.alloc().directBuffer(MESSAGE_HEADER_LENGTH); + } + + @Override + public DecodingResult onChannelRead(ByteBuf data) throws Exception { + if (bufferResponse == null) { + decodeMessageHeader(data); + } + + if (bufferResponse != null) { + int remainingBufferSize = bufferResponse.bufferSize - decodedDataBufferSize; + int actualBytesToDecode = Math.min(data.readableBytes(), remainingBufferSize); + + // For the case of data buffer really exists in BufferResponse now. + if (actualBytesToDecode > 0) { + // For the case of released input channel, the respective data buffer part would be + // discarded from the received buffer. + if (bufferResponse.getBuffer() == null) { + data.readerIndex(data.readerIndex() + actualBytesToDecode); + } else { + bufferResponse.getBuffer().asByteBuf().writeBytes(data, actualBytesToDecode); + } + + decodedDataBufferSize += actualBytesToDecode; + } + + if (decodedDataBufferSize == bufferResponse.bufferSize) { + BufferResponse result = bufferResponse; + clearState(); + return DecodingResult.fullMessage(result); + } + } + + return DecodingResult.NOT_FINISHED; + } + + private void decodeMessageHeader(ByteBuf data) { + ByteBuf fullFrameHeaderBuf = ByteBufUtils.accumulate( + messageHeaderBuffer, + data, + MESSAGE_HEADER_LENGTH, + messageHeaderBuffer.readableBytes()); + if (fullFrameHeaderBuf != null) { + bufferResponse = BufferResponse.readFrom(fullFrameHeaderBuf, allocator); + } + } + + private void clearState() { + bufferResponse = null; + decodedDataBufferSize = 0; + + messageHeaderBuffer.clear(); + } + + @Override + public void close() { + if (bufferResponse != null) { + bufferResponse.releaseBuffer(); + } + + messageHeaderBuffer.release(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/ByteBufUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/ByteBufUtils.java new file mode 100644 index 0000000000..0f10e1e7b4 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/ByteBufUtils.java @@ -0,0 +1,64 @@ +/* + * 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.runtime.io.network.netty; + +import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; + +import javax.annotation.Nullable; + +/** + * Utility routines to process {@link ByteBuf}. + */ +public class ByteBufUtils { + + /** + * Accumulates data from source to target. If no data has been + * accumulated yet and source has enough data, source will be + * returned directly. Otherwise, data will be copied into target. If the + * size of data copied after this operation has reached targetAccumulationSize, + * target will be returned, otherwise null will be returned to indicate + * more data is required. + * + * @param target The target buffer. + * @param source The source buffer. + * @param targetAccumulationSize The target size of data to accumulate. + * @param accumulatedSize The size of data accumulated so far. + * + * @return The ByteBuf containing accumulated data. If not enough data has been accumulated, + * null will be returned. + */ + @Nullable + public static ByteBuf accumulate(ByteBuf target, ByteBuf source, int targetAccumulationSize, int accumulatedSize) { + if (accumulatedSize == 0 && source.readableBytes() >= targetAccumulationSize) { + return source; + } + + int copyLength = Math.min(source.readableBytes(), targetAccumulationSize - accumulatedSize); + if (copyLength > 0) { + target.writeBytes(source, copyLength); + } + + if (accumulatedSize + copyLength == targetAccumulationSize) { + return target; + } + + return null; + } + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandler.java index e261ff7853..2cb7faa19c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandler.java @@ -18,12 +18,7 @@ package org.apache.flink.runtime.io.network.netty; -import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.core.memory.MemorySegmentFactory; import org.apache.flink.runtime.io.network.NetworkClientHandler; -import org.apache.flink.runtime.io.network.buffer.Buffer; -import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; -import org.apache.flink.runtime.io.network.buffer.NetworkBuffer; import org.apache.flink.runtime.io.network.netty.exception.LocalTransportException; import org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException; import org.apache.flink.runtime.io.network.netty.exception.TransportException; @@ -32,7 +27,6 @@ import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException; import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID; import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel; -import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; import org.apache.flink.shaded.netty4.io.netty.channel.Channel; import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture; import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFutureListener; @@ -97,6 +91,11 @@ class CreditBasedPartitionRequestClientHandler extends ChannelInboundHandlerAdap inputChannels.remove(listener.getInputChannelId()); } + @Override + public RemoteInputChannel getInputChannel(InputChannelID inputChannelId) { + return inputChannels.get(inputChannelId); + } + @Override public void cancelRequestFor(InputChannelID inputChannelId) { if (inputChannelId == null || ctx == null) { @@ -249,7 +248,7 @@ class CreditBasedPartitionRequestClientHandler extends ChannelInboundHandlerAdap NettyMessage.BufferResponse bufferOrEvent = (NettyMessage.BufferResponse) msg; RemoteInputChannel inputChannel = inputChannels.get(bufferOrEvent.receiverId); - if (inputChannel == null) { + if (inputChannel == null || inputChannel.isReleased()) { bufferOrEvent.releaseBuffer(); cancelRequestFor(bufferOrEvent.receiverId); @@ -290,43 +289,12 @@ class CreditBasedPartitionRequestClientHandler extends ChannelInboundHandlerAdap } private void decodeBufferOrEvent(RemoteInputChannel inputChannel, NettyMessage.BufferResponse bufferOrEvent) throws Throwable { - try { - ByteBuf nettyBuffer = bufferOrEvent.getNettyBuffer(); - final int receivedSize = nettyBuffer.readableBytes(); - if (bufferOrEvent.isBuffer()) { - // ---- Buffer ------------------------------------------------ - - // Early return for empty buffers. Otherwise Netty's readBytes() throws an - // IndexOutOfBoundsException. - if (receivedSize == 0) { - inputChannel.onEmptyBuffer(bufferOrEvent.sequenceNumber, bufferOrEvent.backlog); - return; - } - - Buffer buffer = inputChannel.requestBuffer(); - if (buffer != null) { - nettyBuffer.readBytes(buffer.asByteBuf(), receivedSize); - buffer.setCompressed(bufferOrEvent.isCompressed); - - inputChannel.onBuffer(buffer, bufferOrEvent.sequenceNumber, bufferOrEvent.backlog); - } else if (inputChannel.isReleased()) { - cancelRequestFor(bufferOrEvent.receiverId); - } else { - throw new IllegalStateException("No buffer available in credit-based input channel."); - } - } else { - // ---- Event ------------------------------------------------- - // TODO We can just keep the serialized data in the Netty buffer and release it later at the reader - byte[] byteArray = new byte[receivedSize]; - nettyBuffer.readBytes(byteArray); - - MemorySegment memSeg = MemorySegmentFactory.wrap(byteArray); - Buffer buffer = new NetworkBuffer(memSeg, FreeingBufferRecycler.INSTANCE, false, receivedSize); - - inputChannel.onBuffer(buffer, bufferOrEvent.sequenceNumber, bufferOrEvent.backlog); - } - } finally { - bufferOrEvent.releaseBuffer(); + if (bufferOrEvent.isBuffer() && bufferOrEvent.bufferSize == 0) { + inputChannel.onEmptyBuffer(bufferOrEvent.sequenceNumber, bufferOrEvent.backlog); + } else if (bufferOrEvent.getBuffer() != null) { + inputChannel.onBuffer(bufferOrEvent.getBuffer(), bufferOrEvent.sequenceNumber, bufferOrEvent.backlog); + } else { + throw new IllegalStateException("The read buffer is null in credit-based input channel."); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java index 9dfd05329f..2ed875db2a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java @@ -228,18 +228,12 @@ public abstract class NettyMessage { final NettyMessage decodedMsg; switch (msgId) { - case BufferResponse.ID: - decodedMsg = BufferResponse.readFrom(msg); - break; case PartitionRequest.ID: decodedMsg = PartitionRequest.readFrom(msg); break; case TaskEventRequest.ID: decodedMsg = TaskEventRequest.readFrom(msg, getClass().getClassLoader()); break; - case ErrorResponse.ID: - decodedMsg = ErrorResponse.readFrom(msg); - break; case CancelPartitionRequest.ID: decodedMsg = CancelPartitionRequest.readFrom(msg); break; @@ -269,9 +263,12 @@ public abstract class NettyMessage { static class BufferResponse extends NettyMessage { - private static final byte ID = 0; + static final byte ID = 0; - final ByteBuf buffer; + // receiver ID (16), sequence number (4), backlog (4), isBuffer (1), isCompressed (1), buffer size (4) + static final int MESSAGE_HEADER_LENGTH = 16 + 4 + 4 + 1 + 1 + 4; + + final Buffer buffer; final InputChannelID receiverId; @@ -283,19 +280,23 @@ public abstract class NettyMessage { final boolean isCompressed; + final int bufferSize; + private BufferResponse( - ByteBuf buffer, + @Nullable Buffer buffer, boolean isBuffer, boolean isCompressed, int sequenceNumber, InputChannelID receiverId, - int backlog) { - this.buffer = checkNotNull(buffer); + int backlog, + int bufferSize) { + this.buffer = buffer; this.isBuffer = isBuffer; this.isCompressed = isCompressed; this.sequenceNumber = sequenceNumber; this.receiverId = checkNotNull(receiverId); this.backlog = backlog; + this.bufferSize = bufferSize; } BufferResponse( @@ -303,24 +304,28 @@ public abstract class NettyMessage { int sequenceNumber, InputChannelID receiverId, int backlog) { - this.buffer = checkNotNull(buffer).asByteBuf(); + this.buffer = checkNotNull(buffer); this.isBuffer = buffer.isBuffer(); this.isCompressed = buffer.isCompressed(); this.sequenceNumber = sequenceNumber; this.receiverId = checkNotNull(receiverId); this.backlog = backlog; + this.bufferSize = buffer.getSize(); } boolean isBuffer() { return isBuffer; } - ByteBuf getNettyBuffer() { + @Nullable + public Buffer getBuffer() { return buffer; } void releaseBuffer() { - buffer.release(); + if (buffer != null) { + buffer.recycleBuffer(); + } } // -------------------------------------------------------------------- @@ -329,18 +334,13 @@ public abstract class NettyMessage { @Override ByteBuf write(ByteBufAllocator allocator) throws IOException { - // receiver ID (16), sequence number (4), backlog (4), isBuffer (1), isCompressed (1), buffer size (4) - final int messageHeaderLength = 16 + 4 + 4 + 1 + 1 + 4; - ByteBuf headerBuf = null; try { - if (buffer instanceof Buffer) { - // in order to forward the buffer to netty, it needs an allocator set - ((Buffer) buffer).setAllocator(allocator); - } + // in order to forward the buffer to netty, it needs an allocator set + buffer.setAllocator(allocator); // only allocate header buffer - we will combine it with the data buffer below - headerBuf = allocateBuffer(allocator, ID, messageHeaderLength, buffer.readableBytes(), false); + headerBuf = allocateBuffer(allocator, ID, MESSAGE_HEADER_LENGTH, bufferSize, false); receiverId.writeTo(headerBuf); headerBuf.writeInt(sequenceNumber); @@ -351,38 +351,67 @@ public abstract class NettyMessage { CompositeByteBuf composityBuf = allocator.compositeDirectBuffer(); composityBuf.addComponent(headerBuf); - composityBuf.addComponent(buffer); + composityBuf.addComponent(buffer.asByteBuf()); // update writer index since we have data written to the components: - composityBuf.writerIndex(headerBuf.writerIndex() + buffer.writerIndex()); + composityBuf.writerIndex(headerBuf.writerIndex() + buffer.asByteBuf().writerIndex()); return composityBuf; } catch (Throwable t) { if (headerBuf != null) { headerBuf.release(); } - buffer.release(); + buffer.recycleBuffer(); ExceptionUtils.rethrowIOException(t); return null; // silence the compiler } } - static BufferResponse readFrom(ByteBuf buffer) { - InputChannelID receiverId = InputChannelID.fromByteBuf(buffer); - int sequenceNumber = buffer.readInt(); - int backlog = buffer.readInt(); - boolean isBuffer = buffer.readBoolean(); - boolean isCompressed = buffer.readBoolean(); - int size = buffer.readInt(); + /** + * Parses the message header part and composes a new BufferResponse with an empty data buffer. The + * data buffer will be filled in later. + * + * @param messageHeader the serialized message header. + * @param bufferAllocator the allocator for network buffer. + * @return a BufferResponse object with the header parsed and the data buffer to fill in later. The + * data buffer will be null if the target channel has been released or the buffer size is 0. + */ + static BufferResponse readFrom(ByteBuf messageHeader, NetworkBufferAllocator bufferAllocator) { + InputChannelID receiverId = InputChannelID.fromByteBuf(messageHeader); + int sequenceNumber = messageHeader.readInt(); + int backlog = messageHeader.readInt(); + boolean isBuffer = messageHeader.readBoolean(); + boolean isCompressed = messageHeader.readBoolean(); + int size = messageHeader.readInt(); + + Buffer dataBuffer = null; + + if (size != 0) { + if (isBuffer) { + dataBuffer = bufferAllocator.allocatePooledNetworkBuffer(receiverId); + } else { + dataBuffer = bufferAllocator.allocateUnPooledNetworkBuffer(size); + } + } + + if (dataBuffer != null) { + dataBuffer.setCompressed(isCompressed); + } - ByteBuf retainedSlice = buffer.readSlice(size).retain(); - return new BufferResponse(retainedSlice, isBuffer, isCompressed, sequenceNumber, receiverId, backlog); + return new BufferResponse( + dataBuffer, + isBuffer, + isCompressed, + sequenceNumber, + receiverId, + backlog, + size); } } static class ErrorResponse extends NettyMessage { - private static final byte ID = 1; + static final byte ID = 1; final Throwable cause; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientDecoderDelegate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientDecoderDelegate.java new file mode 100644 index 0000000000..d3fe208c87 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientDecoderDelegate.java @@ -0,0 +1,153 @@ +/* + * 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.runtime.io.network.netty; + +import org.apache.flink.runtime.io.network.NetworkClientHandler; +import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext; +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter; +import org.apache.flink.util.IOUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.flink.runtime.io.network.netty.NettyMessage.FRAME_HEADER_LENGTH; +import static org.apache.flink.runtime.io.network.netty.NettyMessage.MAGIC_NUMBER; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * Decodes messages from the received netty buffers. This decoder assumes the + * messages have the following format: + * +-----------------------------------+--------------------------------+ + * | FRAME_HEADER || MESSAGE_HEADER | DATA BUFFER (Optional) | + * +-----------------------------------+--------------------------------+ + * + *

    This decoder decodes the frame header and delegates the following work to the + * corresponding message decoders according to the message type. During this process + * The frame header and message header are only accumulated if they span received + * multiple netty buffers, and the data buffer is copied directly to the buffer + * of corresponding input channel to avoid more copying. + * + *

    The format of the frame header is + * +------------------+------------------+--------+ + * | FRAME LENGTH (4) | MAGIC NUMBER (4) | ID (1) | + * +------------------+------------------+--------+ + */ +public class NettyMessageClientDecoderDelegate extends ChannelInboundHandlerAdapter { + private final Logger LOG = LoggerFactory.getLogger(NettyMessageClientDecoderDelegate.class); + + /** The decoder for BufferResponse. */ + private final NettyMessageDecoder bufferResponseDecoder; + + /** The decoder for messages other than BufferResponse. */ + private final NettyMessageDecoder nonBufferResponseDecoder; + + /** The accumulation buffer for the frame header. */ + private ByteBuf frameHeaderBuffer; + + /** The decoder for the current message. It is null if we are decoding the frame header. */ + private NettyMessageDecoder currentDecoder; + + NettyMessageClientDecoderDelegate(NetworkClientHandler networkClientHandler) { + this.bufferResponseDecoder = new BufferResponseDecoder( + new NetworkBufferAllocator( + checkNotNull(networkClientHandler))); + this.nonBufferResponseDecoder = new NonBufferResponseDecoder(); + } + + @Override + public void channelActive(ChannelHandlerContext ctx) throws Exception { + bufferResponseDecoder.onChannelActive(ctx); + nonBufferResponseDecoder.onChannelActive(ctx); + + frameHeaderBuffer = ctx.alloc().directBuffer(FRAME_HEADER_LENGTH); + + super.channelActive(ctx); + } + + /** + * Releases resources when the channel is closed. When exceptions are thrown during + * processing received netty buffers, {@link CreditBasedPartitionRequestClientHandler} + * is expected to catch the exception and close the channel and trigger this notification. + * + * @param ctx The context of the channel close notification. + */ + @Override + public void channelInactive(ChannelHandlerContext ctx) throws Exception { + IOUtils.cleanup(LOG, bufferResponseDecoder, nonBufferResponseDecoder); + frameHeaderBuffer.release(); + + super.channelInactive(ctx); + } + + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + if (!(msg instanceof ByteBuf)) { + ctx.fireChannelRead(msg); + return; + } + + ByteBuf data = (ByteBuf) msg; + try { + while (data.isReadable()) { + if (currentDecoder != null) { + NettyMessageDecoder.DecodingResult result = currentDecoder.onChannelRead(data); + if (!result.isFinished()) { + break; + } + ctx.fireChannelRead(result.getMessage()); + + currentDecoder = null; + frameHeaderBuffer.clear(); + } + + decodeFrameHeader(data); + } + checkState(!data.isReadable(), "Not all data of the received buffer consumed."); + } finally { + data.release(); + } + } + + private void decodeFrameHeader(ByteBuf data) { + ByteBuf fullFrameHeaderBuf = ByteBufUtils.accumulate( + frameHeaderBuffer, + data, + FRAME_HEADER_LENGTH, + frameHeaderBuffer.readableBytes()); + + if (fullFrameHeaderBuf != null) { + int messageAndFrameLength = fullFrameHeaderBuf.readInt(); + checkState(messageAndFrameLength >= 0, "The length field of current message must be non-negative"); + + int magicNumber = fullFrameHeaderBuf.readInt(); + checkState(magicNumber == MAGIC_NUMBER, "Network stream corrupted: received incorrect magic number."); + + int msgId = fullFrameHeaderBuf.readByte(); + if (msgId == NettyMessage.BufferResponse.ID) { + currentDecoder = bufferResponseDecoder; + } else { + currentDecoder = nonBufferResponseDecoder; + } + + currentDecoder.onNewMessageReceived(msgId, messageAndFrameLength - FRAME_HEADER_LENGTH); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessageDecoder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessageDecoder.java new file mode 100644 index 0000000000..b514eb569f --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessageDecoder.java @@ -0,0 +1,92 @@ +/* + * 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.runtime.io.network.netty; + +import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext; + +import javax.annotation.Nullable; + +/** + * Base class of decoders for specified netty messages. + */ +abstract class NettyMessageDecoder implements AutoCloseable { + + /** ID of the message under decoding. */ + protected int msgId; + + /** Length of the message under decoding. */ + protected int messageLength; + + /** + * The result of decoding one {@link ByteBuf}. + */ + static class DecodingResult { + final static DecodingResult NOT_FINISHED = new DecodingResult(false, null); + + private final boolean finished; + + @Nullable + private final NettyMessage message; + + private DecodingResult(boolean finished, @Nullable NettyMessage message) { + this.finished = finished; + this.message = message; + } + + public boolean isFinished() { + return finished; + } + + @Nullable + public NettyMessage getMessage() { + return message; + } + + static DecodingResult fullMessage(NettyMessage message) { + return new DecodingResult(true, message); + } + } + + /** + * Notifies that the underlying channel becomes active. + * + * @param ctx The context for the callback. + */ + abstract void onChannelActive(ChannelHandlerContext ctx); + + /** + * Notifies that a new message is to be decoded. + * + * @param msgId The type of the message to be decoded. + * @param messageLength The length of the message to be decoded. + */ + void onNewMessageReceived(int msgId, int messageLength) { + this.msgId = msgId; + this.messageLength = messageLength; + } + + /** + * Notifies that more data is received to continue decoding. + * + * @param data The data received. + * @return The result of decoding received data. + */ + abstract DecodingResult onChannelRead(ByteBuf data) throws Exception; +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyProtocol.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyProtocol.java index 4cf98e0351..497d40d19d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyProtocol.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyProtocol.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.io.network.netty; +import org.apache.flink.runtime.io.network.NetworkClientHandler; import org.apache.flink.runtime.io.network.TaskEventPublisher; import org.apache.flink.runtime.io.network.partition.ResultPartitionProvider; @@ -120,10 +121,12 @@ public class NettyProtocol { * @return channel handlers */ public ChannelHandler[] getClientChannelHandlers() { - return new ChannelHandler[] { + NetworkClientHandler networkClientHandler = new CreditBasedPartitionRequestClientHandler(); + + return new ChannelHandler[]{ messageEncoder, - new NettyMessage.NettyMessageDecoder(), - new CreditBasedPartitionRequestClientHandler()}; + new NettyMessageClientDecoderDelegate(networkClientHandler), + networkClientHandler}; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NetworkBufferAllocator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NetworkBufferAllocator.java new file mode 100644 index 0000000000..46e6fe9336 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NetworkBufferAllocator.java @@ -0,0 +1,77 @@ +/* + * 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.runtime.io.network.netty; + +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.core.memory.MemorySegmentFactory; +import org.apache.flink.runtime.io.network.NetworkClientHandler; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; +import org.apache.flink.runtime.io.network.buffer.NetworkBuffer; +import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID; +import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel; + +import javax.annotation.Nullable; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * An allocator used for requesting buffers in the client side netty handlers. + */ +class NetworkBufferAllocator { + private final NetworkClientHandler networkClientHandler; + + NetworkBufferAllocator(NetworkClientHandler networkClientHandler) { + this.networkClientHandler = checkNotNull(networkClientHandler); + } + + /** + * Allocates a pooled network buffer for the specific input channel. + * + * @param receiverId The id of the requested input channel. + * @return The pooled network buffer. + */ + @Nullable + Buffer allocatePooledNetworkBuffer(InputChannelID receiverId) { + Buffer buffer = null; + + RemoteInputChannel inputChannel = networkClientHandler.getInputChannel(receiverId); + + // If the input channel has been released, we cannot allocate buffer and the received message + // will be discarded. + if (inputChannel != null) { + buffer = inputChannel.requestBuffer(); + } + + return buffer; + } + + /** + * Allocates an un-pooled network buffer with the specific size. + * + * @param size The requested buffer size. + * @return The un-pooled network buffer. + */ + Buffer allocateUnPooledNetworkBuffer(int size) { + byte[] byteArray = new byte[size]; + MemorySegment memSeg = MemorySegmentFactory.wrap(byteArray); + + return new NetworkBuffer(memSeg, FreeingBufferRecycler.INSTANCE, false); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NonBufferResponseDecoder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NonBufferResponseDecoder.java new file mode 100644 index 0000000000..7380e92bba --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NonBufferResponseDecoder.java @@ -0,0 +1,85 @@ +/* + * 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.runtime.io.network.netty; + +import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext; + +import java.net.ProtocolException; + +import static org.apache.flink.runtime.io.network.netty.NettyMessage.BufferResponse; +import static org.apache.flink.runtime.io.network.netty.NettyMessage.ErrorResponse; + +/** + * The decoder for messages other than {@link BufferResponse}. + */ +class NonBufferResponseDecoder extends NettyMessageDecoder { + + /** The initial size of the message header accumulation buffer. */ + private static final int INITIAL_MESSAGE_HEADER_BUFFER_LENGTH = 128; + + /** The accumulation buffer of the message. */ + private ByteBuf messageBuffer; + + @Override + public void onChannelActive(ChannelHandlerContext ctx) { + messageBuffer = ctx.alloc().directBuffer(INITIAL_MESSAGE_HEADER_BUFFER_LENGTH); + } + + @Override + void onNewMessageReceived(int msgId, int messageLength) { + super.onNewMessageReceived(msgId, messageLength); + messageBuffer.clear(); + ensureBufferCapacity(); + } + + @Override + public DecodingResult onChannelRead(ByteBuf data) throws Exception { + ByteBuf fullFrameHeaderBuf = ByteBufUtils.accumulate( + messageBuffer, + data, + messageLength, + messageBuffer.readableBytes()); + if (fullFrameHeaderBuf == null) { + return DecodingResult.NOT_FINISHED; + } + + switch (msgId) { + case ErrorResponse.ID: + return DecodingResult.fullMessage(ErrorResponse.readFrom(fullFrameHeaderBuf)); + default: + throw new ProtocolException("Received unknown message from producer: " + msgId); + } + } + + /** + * Ensures the message header accumulation buffer has enough capacity for + * the current message. + */ + private void ensureBufferCapacity() { + if (messageBuffer.capacity() < messageLength) { + messageBuffer.capacity(messageLength); + } + } + + @Override + public void close() { + messageBuffer.release(); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ByteBufUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ByteBufUtilsTest.java new file mode 100644 index 0000000000..6886276ca0 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ByteBufUtilsTest.java @@ -0,0 +1,129 @@ +/* + * 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.runtime.io.network.netty; + +import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; +import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled; +import org.apache.flink.util.TestLogger; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertSame; + +/** + * Tests the methods in {@link ByteBufUtils}. + */ +public class ByteBufUtilsTest extends TestLogger { + private static final byte ACCUMULATION_BYTE = 0x7d; + private static final byte NON_ACCUMULATION_BYTE = 0x23; + + @Test + public void testAccumulateWithoutCopy() { + int sourceLength = 128; + int sourceReaderIndex = 32; + int expectedAccumulationSize = 16; + + ByteBuf src = createSourceBuffer(sourceLength, sourceReaderIndex, expectedAccumulationSize); + ByteBuf target = Unpooled.buffer(expectedAccumulationSize); + + // If src has enough data and no data has been copied yet, src will be returned without modification. + ByteBuf accumulated = ByteBufUtils.accumulate(target, src, expectedAccumulationSize, target.readableBytes()); + + assertSame(src, accumulated); + assertEquals(sourceReaderIndex, src.readerIndex()); + verifyBufferContent(src, sourceReaderIndex, expectedAccumulationSize); + } + + @Test + public void testAccumulateWithCopy() { + int sourceLength = 128; + int firstSourceReaderIndex = 32; + int secondSourceReaderIndex = 0; + int expectedAccumulationSize = 128; + + int firstAccumulationSize = sourceLength - firstSourceReaderIndex; + int secondAccumulationSize = expectedAccumulationSize - firstAccumulationSize; + + ByteBuf firstSource = createSourceBuffer(sourceLength, firstSourceReaderIndex, firstAccumulationSize); + ByteBuf secondSource = createSourceBuffer(sourceLength, secondSourceReaderIndex, secondAccumulationSize); + + ByteBuf target = Unpooled.buffer(expectedAccumulationSize); + + // If src does not have enough data, src will be copied into target and null will be returned. + ByteBuf accumulated = ByteBufUtils.accumulate( + target, + firstSource, + expectedAccumulationSize, + target.readableBytes()); + assertNull(accumulated); + assertEquals(sourceLength, firstSource.readerIndex()); + assertEquals(firstAccumulationSize, target.readableBytes()); + + // The remaining data will be copied from the second buffer, and the target buffer will be returned + // after all data is accumulated. + accumulated = ByteBufUtils.accumulate( + target, + secondSource, + expectedAccumulationSize, + target.readableBytes()); + assertSame(target, accumulated); + assertEquals(secondSourceReaderIndex + secondAccumulationSize, secondSource.readerIndex()); + assertEquals(expectedAccumulationSize, target.readableBytes()); + + verifyBufferContent(accumulated, 0, expectedAccumulationSize); + } + + /** + * Create a source buffer whose length is size. The content between readerIndex and + * readerIndex + accumulationSize is ACCUMULATION_BYTE and the remaining is + * NON_ACCUMULATION_BYTE. + * + * @param size The size of the source buffer. + * @param readerIndex The reader index of the source buffer. + * @param accumulationSize The size of bytes that will be read for accumulating. + * + * @return The required source buffer. + */ + private ByteBuf createSourceBuffer(int size, int readerIndex, int accumulationSize) { + ByteBuf buf = Unpooled.buffer(size); + + for (int i = 0; i < readerIndex; i++) { + buf.writeByte(NON_ACCUMULATION_BYTE); + } + + for (int i = readerIndex; i < readerIndex + accumulationSize; i++) { + buf.writeByte(ACCUMULATION_BYTE); + } + + for (int i = readerIndex + accumulationSize; i < size; i++) { + buf.writeByte(NON_ACCUMULATION_BYTE); + } + + buf.readerIndex(readerIndex); + return buf; + } + + private void verifyBufferContent(ByteBuf buf, int start, int length) { + for (int i = 0; i < length; ++i) { + byte b = buf.getByte(start + i); + assertEquals(String.format("The byte at position %d is not right.", start + i), ACCUMULATION_BYTE, b); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandlerTest.java index 0f313a4679..fb68252220 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandlerTest.java @@ -96,12 +96,16 @@ public class CreditBasedPartitionRequestClientHandlerTest { when(inputChannel.getInputChannelId()).thenReturn(new InputChannelID()); when(inputChannel.getBufferProvider()).thenReturn(bufferProvider); - final BufferResponse receivedBuffer = createBufferResponse( - TestBufferFactory.createBuffer(TestBufferFactory.BUFFER_SIZE), 0, inputChannel.getInputChannelId(), 2); - final CreditBasedPartitionRequestClientHandler client = new CreditBasedPartitionRequestClientHandler(); client.addInputChannel(inputChannel); + final BufferResponse receivedBuffer = createBufferResponse( + TestBufferFactory.createBuffer(TestBufferFactory.BUFFER_SIZE), + 0, + inputChannel.getInputChannelId(), + 2, + new NetworkBufferAllocator(client)); + client.channelRead(mock(ChannelHandlerContext.class), receivedBuffer); } @@ -123,13 +127,17 @@ public class CreditBasedPartitionRequestClientHandlerTest { // An empty buffer of size 0 final Buffer emptyBuffer = TestBufferFactory.createBuffer(0); - final int backlog = 2; - final BufferResponse receivedBuffer = createBufferResponse( - emptyBuffer, 0, inputChannel.getInputChannelId(), backlog); - final CreditBasedPartitionRequestClientHandler client = new CreditBasedPartitionRequestClientHandler(); client.addInputChannel(inputChannel); + final int backlog = 2; + final BufferResponse receivedBuffer = createBufferResponse( + emptyBuffer, + 0, + inputChannel.getInputChannelId(), + backlog, + new NetworkBufferAllocator(client)); + // Read the empty buffer client.channelRead(mock(ChannelHandlerContext.class), receivedBuffer); @@ -159,7 +167,11 @@ public class CreditBasedPartitionRequestClientHandlerTest { final int backlog = 2; final BufferResponse bufferResponse = createBufferResponse( - TestBufferFactory.createBuffer(32), 0, inputChannel.getInputChannelId(), backlog); + TestBufferFactory.createBuffer(32), + 0, + inputChannel.getInputChannelId(), + backlog, + new NetworkBufferAllocator(handler)); handler.channelRead(mock(ChannelHandlerContext.class), bufferResponse); assertEquals(1, inputChannel.getNumberOfQueuedBuffers()); @@ -191,7 +203,12 @@ public class CreditBasedPartitionRequestClientHandlerTest { handler.addInputChannel(inputChannel); Buffer buffer = compressor.compressToOriginalBuffer(TestBufferFactory.createBuffer(bufferSize)); - BufferResponse bufferResponse = createBufferResponse(buffer, 0, inputChannel.getInputChannelId(), 2); + BufferResponse bufferResponse = createBufferResponse( + buffer, + 0, + inputChannel.getInputChannelId(), + 2, + new NetworkBufferAllocator(handler)); assertTrue(bufferResponse.isCompressed); handler.channelRead(null, bufferResponse); @@ -220,9 +237,14 @@ public class CreditBasedPartitionRequestClientHandlerTest { 0, inputChannel.getNumberOfAvailableBuffers()); final BufferResponse bufferResponse = createBufferResponse( - TestBufferFactory.createBuffer(TestBufferFactory.BUFFER_SIZE), 0, inputChannel.getInputChannelId(), 2); - handler.channelRead(mock(ChannelHandlerContext.class), bufferResponse); + TestBufferFactory.createBuffer(TestBufferFactory.BUFFER_SIZE), + 0, + inputChannel.getInputChannelId(), + 2, + new NetworkBufferAllocator(handler)); + assertNull(bufferResponse.getBuffer()); + handler.channelRead(mock(ChannelHandlerContext.class), bufferResponse); verify(inputChannel, times(1)).onError(any(IllegalStateException.class)); } @@ -281,6 +303,7 @@ public class CreditBasedPartitionRequestClientHandlerTest { @Test public void testNotifyCreditAvailable() throws Exception { final CreditBasedPartitionRequestClientHandler handler = new CreditBasedPartitionRequestClientHandler(); + final NetworkBufferAllocator allocator = new NetworkBufferAllocator(handler); final EmbeddedChannel channel = new EmbeddedChannel(handler); final PartitionRequestClient client = new NettyPartitionRequestClient( channel, handler, mock(ConnectionID.class), mock(PartitionRequestClientFactory.class)); @@ -312,9 +335,17 @@ public class CreditBasedPartitionRequestClientHandlerTest { // The buffer response will take one available buffer from input channel, and it will trigger // requesting (backlog + numExclusiveBuffers - numAvailableBuffers) floating buffers final BufferResponse bufferResponse1 = createBufferResponse( - TestBufferFactory.createBuffer(32), 0, inputChannel1.getInputChannelId(), 1); + TestBufferFactory.createBuffer(32), + 0, + inputChannel1.getInputChannelId(), + 1, + allocator); final BufferResponse bufferResponse2 = createBufferResponse( - TestBufferFactory.createBuffer(32), 0, inputChannel2.getInputChannelId(), 1); + TestBufferFactory.createBuffer(32), + 0, + inputChannel2.getInputChannelId(), + 1, + allocator); handler.channelRead(mock(ChannelHandlerContext.class), bufferResponse1); handler.channelRead(mock(ChannelHandlerContext.class), bufferResponse2); @@ -339,7 +370,11 @@ public class CreditBasedPartitionRequestClientHandlerTest { // Trigger notify credits availability via buffer response on the condition of an un-writable channel final BufferResponse bufferResponse3 = createBufferResponse( - TestBufferFactory.createBuffer(32), 1, inputChannel1.getInputChannelId(), 1); + TestBufferFactory.createBuffer(32), + 1, + inputChannel1.getInputChannelId(), + 1, + allocator); handler.channelRead(mock(ChannelHandlerContext.class), bufferResponse3); assertEquals(1, inputChannel1.getUnannouncedCredit()); @@ -367,6 +402,7 @@ public class CreditBasedPartitionRequestClientHandlerTest { assertNull(channel.readOutbound()); } finally { releaseResource(inputGate, networkBufferPool); + channel.close(); } } @@ -398,7 +434,11 @@ public class CreditBasedPartitionRequestClientHandlerTest { // Trigger request floating buffers via buffer response to notify credits available final BufferResponse bufferResponse = createBufferResponse( - TestBufferFactory.createBuffer(32), 0, inputChannel.getInputChannelId(), 1); + TestBufferFactory.createBuffer(32), + 0, + inputChannel.getInputChannelId(), + 1, + new NetworkBufferAllocator(handler)); handler.channelRead(mock(ChannelHandlerContext.class), bufferResponse); assertEquals(2, inputChannel.getUnannouncedCredit()); @@ -416,6 +456,88 @@ public class CreditBasedPartitionRequestClientHandlerTest { assertNull(channel.readOutbound()); } finally { releaseResource(inputGate, networkBufferPool); + channel.close(); + } + } + + @Test + public void testReadBufferResponseBeforeReleasingChannel() throws Exception { + testReadBufferResponseWithReleasingOrRemovingChannel(false, true); + } + + @Test + public void testReadBufferResponseBeforeRemovingChannel() throws Exception { + testReadBufferResponseWithReleasingOrRemovingChannel(true, true); + } + + @Test + public void testReadBufferResponseAfterReleasingChannel() throws Exception { + testReadBufferResponseWithReleasingOrRemovingChannel(false, false); + } + + @Test + public void testReadBufferResponseAfterRemovingChannel() throws Exception { + testReadBufferResponseWithReleasingOrRemovingChannel(true, false); + } + + private void testReadBufferResponseWithReleasingOrRemovingChannel( + boolean isRemoved, + boolean readBeforeReleasingOrRemoving) throws Exception { + + int bufferSize = 1024; + + NetworkBufferPool networkBufferPool = new NetworkBufferPool(10, bufferSize, 2); + SingleInputGate inputGate = createSingleInputGate(1); + RemoteInputChannel inputChannel = new InputChannelBuilder() + .setMemorySegmentProvider(networkBufferPool) + .buildRemoteAndSetToGate(inputGate); + inputGate.assignExclusiveSegments(); + + CreditBasedPartitionRequestClientHandler handler = new CreditBasedPartitionRequestClientHandler(); + EmbeddedChannel embeddedChannel = new EmbeddedChannel(handler); + handler.addInputChannel(inputChannel); + + try { + if (!readBeforeReleasingOrRemoving) { + // Release the channel. + inputGate.close(); + if (isRemoved) { + handler.removeInputChannel(inputChannel); + } + } + + BufferResponse bufferResponse = createBufferResponse( + TestBufferFactory.createBuffer(bufferSize), + 0, + inputChannel.getInputChannelId(), + 1, + new NetworkBufferAllocator(handler)); + + if (readBeforeReleasingOrRemoving) { + // Release the channel. + inputGate.close(); + if (isRemoved) { + handler.removeInputChannel(inputChannel); + } + } + + handler.channelRead(null, bufferResponse); + + assertEquals(0, inputChannel.getNumberOfQueuedBuffers()); + if (!readBeforeReleasingOrRemoving) { + assertNull(bufferResponse.getBuffer()); + } else { + assertNotNull(bufferResponse.getBuffer()); + assertTrue(bufferResponse.getBuffer().isRecycled()); + } + + embeddedChannel.runScheduledPendingTasks(); + NettyMessage.CancelPartitionRequest cancelPartitionRequest = embeddedChannel.readOutbound(); + assertNotNull(cancelPartitionRequest); + assertEquals(inputChannel.getInputChannelId(), cancelPartitionRequest.receiverId); + } finally { + releaseResource(inputGate, networkBufferPool); + embeddedChannel.close(); } } @@ -434,7 +556,8 @@ public class CreditBasedPartitionRequestClientHandlerTest { Buffer buffer, int sequenceNumber, InputChannelID receivingChannelId, - int backlog) throws IOException { + int backlog, + NetworkBufferAllocator allocator) throws IOException { // Mock buffer to serialize BufferResponse resp = new BufferResponse(buffer, sequenceNumber, receivingChannelId, backlog); @@ -443,11 +566,7 @@ public class CreditBasedPartitionRequestClientHandlerTest { // Skip general header bytes serialized.readBytes(NettyMessage.FRAME_HEADER_LENGTH); - // Deserialize the bytes again. We have to go this way, because we only partly deserialize - // the header of the response and wait for a buffer from the buffer pool to copy the payload - // data into. - BufferResponse deserialized = BufferResponse.readFrom(serialized); - - return deserialized; + // Deserialize the bytes to construct the BufferResponse. + return BufferResponse.readFrom(serialized, allocator); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientDecoderDelegateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientDecoderDelegateTest.java new file mode 100644 index 0000000000..d02cfc739e --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientDecoderDelegateTest.java @@ -0,0 +1,355 @@ +/* + * 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.runtime.io.network.netty; + +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.core.memory.MemorySegmentFactory; +import org.apache.flink.runtime.io.network.TestingPartitionRequestClient; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; +import org.apache.flink.runtime.io.network.buffer.NetworkBuffer; +import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; +import org.apache.flink.runtime.io.network.partition.consumer.InputChannelBuilder; +import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID; +import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel; +import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate; +import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; +import org.apache.flink.shaded.netty4.io.netty.channel.embedded.EmbeddedChannel; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.TestLogger; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import static junit.framework.TestCase.assertEquals; +import static junit.framework.TestCase.assertTrue; +import static org.apache.flink.runtime.io.network.netty.NettyMessage.BufferResponse; +import static org.apache.flink.runtime.io.network.netty.NettyMessage.ErrorResponse; +import static org.apache.flink.runtime.io.network.netty.NettyTestUtil.verifyBufferResponseHeader; +import static org.apache.flink.runtime.io.network.netty.NettyTestUtil.verifyErrorResponse; +import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createRemoteInputChannel; +import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createSingleInputGate; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.fail; + +/** + * Tests the client side message decoder. + */ +public class NettyMessageClientDecoderDelegateTest extends TestLogger { + + private static final int BUFFER_SIZE = 1024; + + private static final int NUMBER_OF_BUFFER_RESPONSES = 5; + + private static final NettyBufferPool ALLOCATOR = new NettyBufferPool(1); + + private EmbeddedChannel channel; + + private NetworkBufferPool networkBufferPool; + + private SingleInputGate inputGate; + + private InputChannelID inputChannelId; + + private InputChannelID releasedInputChannelId; + + @Before + public void setup() throws IOException, InterruptedException { + CreditBasedPartitionRequestClientHandler handler = new CreditBasedPartitionRequestClientHandler(); + networkBufferPool = new NetworkBufferPool( + NUMBER_OF_BUFFER_RESPONSES, + BUFFER_SIZE, + NUMBER_OF_BUFFER_RESPONSES); + channel = new EmbeddedChannel(new NettyMessageClientDecoderDelegate(handler)); + + inputGate = createSingleInputGate(1); + RemoteInputChannel inputChannel = createRemoteInputChannel( + inputGate, + new TestingPartitionRequestClient(), + networkBufferPool); + inputGate.assignExclusiveSegments(); + inputChannel.requestSubpartition(0); + handler.addInputChannel(inputChannel); + inputChannelId = inputChannel.getInputChannelId(); + + SingleInputGate releasedInputGate = createSingleInputGate(1); + RemoteInputChannel releasedInputChannel = new InputChannelBuilder() + .setMemorySegmentProvider(networkBufferPool) + .buildRemoteAndSetToGate(inputGate); + releasedInputGate.close(); + handler.addInputChannel(releasedInputChannel); + releasedInputChannelId = releasedInputChannel.getInputChannelId(); + } + + @After + public void tearDown() throws IOException { + if (inputGate != null) { + inputGate.close(); + } + + if (networkBufferPool != null) { + networkBufferPool.destroyAllBufferPools(); + networkBufferPool.destroy(); + } + + if (channel != null) { + channel.close(); + } + } + + /** + * Verifies that the client side decoder works well for unreleased input channels. + */ + @Test + public void testClientMessageDecode() throws Exception { + testNettyMessageClientDecoding(false, false, false); + } + + /** + * Verifies that the client side decoder works well for empty buffers. Empty buffers should not + * consume data buffers of the input channels. + */ + @Test + public void testClientMessageDecodeWithEmptyBuffers() throws Exception { + testNettyMessageClientDecoding(true, false, false); + } + + /** + * Verifies that the client side decoder works well with buffers sent to a released input channel. + * The data buffer part should be discarded before reading the next message. + */ + @Test + public void testClientMessageDecodeWithReleasedInputChannel() throws Exception { + testNettyMessageClientDecoding(false, true, false); + } + + /** + * Verifies that the client side decoder works well with buffers sent to a removed input channel. + * The data buffer part should be discarded before reading the next message. + */ + @Test + public void testClientMessageDecodeWithRemovedInputChannel() throws Exception { + testNettyMessageClientDecoding(false, false, true); + } + + //------------------------------------------------------------------------------------------------------------------ + + private void testNettyMessageClientDecoding( + boolean hasEmptyBuffer, + boolean hasBufferForReleasedChannel, + boolean hasBufferForRemovedChannel) throws Exception { + + ByteBuf[] encodedMessages = null; + List decodedMessages = null; + try { + List messages = createMessageList( + hasEmptyBuffer, + hasBufferForReleasedChannel, + hasBufferForRemovedChannel); + + encodedMessages = encodeMessages(messages); + + List partitionedBuffers = repartitionMessages(encodedMessages); + decodedMessages = decodeMessages(channel, partitionedBuffers); + verifyDecodedMessages(messages, decodedMessages); + } finally { + releaseBuffers(encodedMessages); + + if (decodedMessages != null) { + for (NettyMessage nettyMessage : decodedMessages) { + if (nettyMessage instanceof BufferResponse) { + ((BufferResponse) nettyMessage).releaseBuffer(); + } + } + } + } + } + + private List createMessageList( + boolean hasEmptyBuffer, + boolean hasBufferForRemovedChannel, + boolean hasBufferForReleasedChannel) { + + int seqNumber = 1; + List messages = new ArrayList<>(); + + for (int i = 0; i < NUMBER_OF_BUFFER_RESPONSES - 1; i++) { + addBufferResponse(messages, inputChannelId, true, BUFFER_SIZE, seqNumber++); + } + + if (hasEmptyBuffer) { + addBufferResponse(messages, inputChannelId, true, 0, seqNumber++); + } + if (hasBufferForReleasedChannel) { + addBufferResponse(messages, releasedInputChannelId, true, BUFFER_SIZE, seqNumber++); + } + if (hasBufferForRemovedChannel) { + addBufferResponse(messages, new InputChannelID(), true, BUFFER_SIZE, seqNumber++); + } + + addBufferResponse(messages, inputChannelId, false, 32, seqNumber++); + addBufferResponse(messages, inputChannelId, true, BUFFER_SIZE, seqNumber); + messages.add(new NettyMessage.ErrorResponse(new RuntimeException("test"), inputChannelId)); + + return messages; + } + + private void addBufferResponse( + List messages, + InputChannelID inputChannelId, + boolean isBuffer, + int bufferSize, + int seqNumber) { + + Buffer buffer = createDataBuffer(bufferSize); + if (!isBuffer) { + buffer.tagAsEvent(); + } + messages.add(new BufferResponse(buffer, seqNumber, inputChannelId, 1)); + } + + private Buffer createDataBuffer(int size) { + MemorySegment segment = MemorySegmentFactory.allocateUnpooledSegment(size); + NetworkBuffer buffer = new NetworkBuffer(segment, FreeingBufferRecycler.INSTANCE); + for (int i = 0; i < size / 4; ++i) { + buffer.writeInt(i); + } + + return buffer; + } + + private ByteBuf[] encodeMessages(List messages) throws Exception { + ByteBuf[] encodedMessages = new ByteBuf[messages.size()]; + for (int i = 0; i < messages.size(); ++i) { + encodedMessages[i] = messages.get(i).write(ALLOCATOR); + } + + return encodedMessages; + } + + private List repartitionMessages(ByteBuf[] encodedMessages) { + List result = new ArrayList<>(); + + ByteBuf mergedBuffer1 = null; + ByteBuf mergedBuffer2 = null; + + try { + mergedBuffer1 = mergeBuffers(encodedMessages, 0, encodedMessages.length / 2); + mergedBuffer2 = mergeBuffers( + encodedMessages, + encodedMessages.length / 2, + encodedMessages.length); + + result.addAll(partitionBuffer(mergedBuffer1, BUFFER_SIZE * 2)); + result.addAll(partitionBuffer(mergedBuffer2, BUFFER_SIZE / 4)); + } catch (Throwable t) { + releaseBuffers(result.toArray(new ByteBuf[0])); + ExceptionUtils.rethrow(t); + } finally { + releaseBuffers(mergedBuffer1, mergedBuffer2); + } + + return result; + } + + private ByteBuf mergeBuffers(ByteBuf[] buffers, int start, int end) { + ByteBuf mergedBuffer = ALLOCATOR.buffer(); + for (int i = start; i < end; ++i) { + mergedBuffer.writeBytes(buffers[i]); + } + + return mergedBuffer; + } + + private List partitionBuffer(ByteBuf buffer, int partitionSize) { + List result = new ArrayList<>(); + + try { + int bufferSize = buffer.readableBytes(); + for (int position = 0; position < bufferSize; position += partitionSize) { + int endPosition = Math.min(position + partitionSize, bufferSize); + ByteBuf partitionedBuffer = ALLOCATOR.buffer(endPosition - position); + partitionedBuffer.writeBytes(buffer, position, endPosition - position); + result.add(partitionedBuffer); + } + } catch (Throwable t) { + releaseBuffers(result.toArray(new ByteBuf[0])); + ExceptionUtils.rethrow(t); + } + + return result; + } + + private List decodeMessages(EmbeddedChannel channel, List inputBuffers) { + for (ByteBuf buffer : inputBuffers) { + channel.writeInbound(buffer); + } + + channel.runPendingTasks(); + + List decodedMessages = new ArrayList<>(); + Object input; + while ((input = channel.readInbound()) != null) { + assertTrue(input instanceof NettyMessage); + decodedMessages.add((NettyMessage) input); + } + + return decodedMessages; + } + + private void verifyDecodedMessages(List expectedMessages, List decodedMessages) { + assertEquals(expectedMessages.size(), decodedMessages.size()); + for (int i = 0; i < expectedMessages.size(); ++i) { + assertEquals(expectedMessages.get(i).getClass(), decodedMessages.get(i).getClass()); + + if (expectedMessages.get(i) instanceof NettyMessage.BufferResponse) { + BufferResponse expected = (BufferResponse) expectedMessages.get(i); + BufferResponse actual = (BufferResponse) decodedMessages.get(i); + + verifyBufferResponseHeader(expected, actual); + if (expected.bufferSize == 0 || !expected.receiverId.equals(inputChannelId)) { + assertNull(actual.getBuffer()); + } else { + assertEquals(expected.getBuffer(), actual.getBuffer()); + } + + } else if (expectedMessages.get(i) instanceof NettyMessage.ErrorResponse) { + verifyErrorResponse((ErrorResponse) expectedMessages.get(i), (ErrorResponse) decodedMessages.get(i)); + } else { + fail("Unsupported message type"); + } + } + } + + private void releaseBuffers(@Nullable ByteBuf... buffers) { + if (buffers != null) { + for (ByteBuf buffer : buffers) { + if (buffer != null) { + buffer.release(); + } + } + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientSideSerializationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientSideSerializationTest.java new file mode 100644 index 0000000000..6fd59a8f9f --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientSideSerializationTest.java @@ -0,0 +1,212 @@ +/* + * 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.runtime.io.network.netty; + +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.core.memory.MemorySegmentFactory; +import org.apache.flink.runtime.io.network.TestingPartitionRequestClient; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferCompressor; +import org.apache.flink.runtime.io.network.buffer.BufferDecompressor; +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; +import org.apache.flink.runtime.io.network.buffer.NetworkBuffer; +import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; +import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID; +import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel; +import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate; +import org.apache.flink.shaded.netty4.io.netty.channel.embedded.EmbeddedChannel; +import org.apache.flink.util.TestLogger; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.Random; + +import static org.apache.flink.runtime.io.network.netty.NettyMessage.BufferResponse; +import static org.apache.flink.runtime.io.network.netty.NettyMessage.ErrorResponse; +import static org.apache.flink.runtime.io.network.netty.NettyMessage.NettyMessageEncoder; +import static org.apache.flink.runtime.io.network.netty.NettyTestUtil.encodeAndDecode; +import static org.apache.flink.runtime.io.network.netty.NettyTestUtil.verifyBufferResponseHeader; +import static org.apache.flink.runtime.io.network.netty.NettyTestUtil.verifyErrorResponse; +import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createRemoteInputChannel; +import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createSingleInputGate; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +/** + * Tests for the serialization and deserialization of the various {@link NettyMessage} sub-classes + * sent from server side to client side. + */ +public class NettyMessageClientSideSerializationTest extends TestLogger { + + private static final int BUFFER_SIZE = 1024; + + private static final BufferCompressor COMPRESSOR = new BufferCompressor(BUFFER_SIZE, "LZ4"); + + private static final BufferDecompressor DECOMPRESSOR = new BufferDecompressor(BUFFER_SIZE, "LZ4"); + + private final Random random = new Random(); + + private EmbeddedChannel channel; + + private NetworkBufferPool networkBufferPool; + + private SingleInputGate inputGate; + + private InputChannelID inputChannelId; + + @Before + public void setup() throws IOException, InterruptedException { + networkBufferPool = new NetworkBufferPool(8, BUFFER_SIZE, 8); + inputGate = createSingleInputGate(1); + RemoteInputChannel inputChannel = createRemoteInputChannel( + inputGate, + new TestingPartitionRequestClient(), + networkBufferPool); + inputChannel.requestSubpartition(0); + inputGate.assignExclusiveSegments(); + + CreditBasedPartitionRequestClientHandler handler = new CreditBasedPartitionRequestClientHandler(); + handler.addInputChannel(inputChannel); + + channel = new EmbeddedChannel( + new NettyMessageEncoder(), // For outbound messages + new NettyMessageClientDecoderDelegate(handler)); // For inbound messages + + inputChannelId = inputChannel.getInputChannelId(); + } + + @After + public void tearDown() throws IOException { + if (inputGate != null) { + inputGate.close(); + } + + if (networkBufferPool != null) { + networkBufferPool.destroyAllBufferPools(); + networkBufferPool.destroy(); + } + + if (channel != null) { + channel.close(); + } + } + + @Test + public void testErrorResponseWithoutErrorMessage() { + testErrorResponse(new ErrorResponse(new IllegalStateException(), inputChannelId)); + } + + @Test + public void testErrorResponseWithErrorMessage() { + testErrorResponse(new ErrorResponse( + new IllegalStateException("Illegal illegal illegal"), + inputChannelId)); + } + + @Test + public void testErrorResponseWithFatalError() { + testErrorResponse(new ErrorResponse(new IllegalStateException("Illegal illegal illegal"))); + } + + @Test + public void testOrdinaryBufferResponse() { + testBufferResponse(false, false); + } + + @Test + public void testBufferResponseWithReadOnlySlice() { + testBufferResponse(true, false); + } + + @Test + public void testCompressedBufferResponse() { + testBufferResponse(false, true); + } + + private void testErrorResponse(ErrorResponse expect) { + ErrorResponse actual = encodeAndDecode(expect, channel); + verifyErrorResponse(expect, actual); + } + + private void testBufferResponse(boolean testReadOnlyBuffer, boolean testCompressedBuffer) { + checkArgument( + !(testReadOnlyBuffer & testCompressedBuffer), + "There are no cases with both readonly slice and compression."); + + NetworkBuffer buffer = new NetworkBuffer( + MemorySegmentFactory.allocateUnpooledSegment(BUFFER_SIZE), + FreeingBufferRecycler.INSTANCE); + for (int i = 0; i < BUFFER_SIZE; i += 8) { + buffer.writeLong(i); + } + + Buffer testBuffer = buffer; + if (testReadOnlyBuffer) { + testBuffer = buffer.readOnlySlice(); + } else if (testCompressedBuffer) { + testBuffer = COMPRESSOR.compressToOriginalBuffer(buffer); + } + + BufferResponse expected = new BufferResponse( + testBuffer, + random.nextInt(), + inputChannelId, + random.nextInt()); + BufferResponse actual = encodeAndDecode(expected, channel); + + assertTrue(buffer.isRecycled()); + assertTrue(testBuffer.isRecycled()); + assertNotNull( + "The request input channel should always have available buffers in this test.", + actual.getBuffer()); + + Buffer decodedBuffer = actual.getBuffer(); + if (testCompressedBuffer) { + assertTrue(actual.isCompressed); + decodedBuffer = decompress(decodedBuffer); + } + + verifyBufferResponseHeader(expected, actual); + assertEquals(BUFFER_SIZE, decodedBuffer.readableBytes()); + for (int i = 0; i < BUFFER_SIZE; i += 8) { + assertEquals(i, decodedBuffer.asByteBuf().readLong()); + } + + // Release the received message. + actual.releaseBuffer(); + if (testCompressedBuffer) { + decodedBuffer.recycleBuffer(); + } + + assertTrue(actual.getBuffer().isRecycled()); + } + + private Buffer decompress(Buffer buffer) { + MemorySegment segment = MemorySegmentFactory.allocateUnpooledSegment(BUFFER_SIZE); + Buffer compressedBuffer = new NetworkBuffer(segment, FreeingBufferRecycler.INSTANCE); + buffer.asByteBuf().readBytes(compressedBuffer.asByteBuf(), buffer.readableBytes()); + compressedBuffer.setCompressed(true); + return DECOMPRESSOR.decompressToOriginalBuffer(compressedBuffer); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageSerializationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageSerializationTest.java deleted file mode 100644 index 79deb8d5d7..0000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageSerializationTest.java +++ /dev/null @@ -1,239 +0,0 @@ -/* - * 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.runtime.io.network.netty; - -import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.core.memory.MemorySegmentFactory; -import org.apache.flink.runtime.event.task.IntegerTaskEvent; -import org.apache.flink.runtime.io.network.buffer.Buffer; -import org.apache.flink.runtime.io.network.buffer.BufferCompressor; -import org.apache.flink.runtime.io.network.buffer.BufferDecompressor; -import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; -import org.apache.flink.runtime.io.network.buffer.NetworkBuffer; -import org.apache.flink.runtime.io.network.partition.ResultPartitionID; -import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID; - -import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; -import org.apache.flink.shaded.netty4.io.netty.channel.embedded.EmbeddedChannel; - -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.util.Arrays; -import java.util.Collection; -import java.util.Random; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; - -/** - * Tests for the serialization and deserialization of the various {@link NettyMessage} sub-classes. - */ -@RunWith(Parameterized.class) -public class NettyMessageSerializationTest { - - private static final int BUFFER_SIZE = 1024; - - private static final BufferCompressor COMPRESSOR = new BufferCompressor(BUFFER_SIZE, "LZ4"); - - private static final BufferDecompressor DECOMPRESSOR = new BufferDecompressor(BUFFER_SIZE, "LZ4"); - - private final EmbeddedChannel channel = new EmbeddedChannel( - new NettyMessage.NettyMessageEncoder(), // outbound messages - new NettyMessage.NettyMessageDecoder()); // inbound messages - - private final Random random = new Random(); - - // ------------------------------------------------------------------------ - // parameters - // ------------------------------------------------------------------------ - - private final boolean testReadOnlyBuffer; - - private final boolean testCompressedBuffer; - - @Parameterized.Parameters(name = "testReadOnlyBuffer = {0}, testCompressedBuffer = {1}") - public static Collection testReadOnlyBuffer() { - return Arrays.asList(new Object[][] { - {false, false}, - {true, false}, - {false, true}, - {true, true} - }); - } - - public NettyMessageSerializationTest(boolean testReadOnlyBuffer, boolean testCompressedBuffer) { - this.testReadOnlyBuffer = testReadOnlyBuffer; - this.testCompressedBuffer = testCompressedBuffer; - } - - @Test - public void testEncodeDecode() { - testEncodeDecodeBuffer(testReadOnlyBuffer, testCompressedBuffer); - - { - { - IllegalStateException expectedError = new IllegalStateException(); - InputChannelID receiverId = new InputChannelID(); - - NettyMessage.ErrorResponse expected = new NettyMessage.ErrorResponse(expectedError, receiverId); - NettyMessage.ErrorResponse actual = encodeAndDecode(expected); - - assertEquals(expected.cause.getClass(), actual.cause.getClass()); - assertEquals(expected.cause.getMessage(), actual.cause.getMessage()); - assertEquals(receiverId, actual.receiverId); - } - - { - IllegalStateException expectedError = new IllegalStateException("Illegal illegal illegal"); - InputChannelID receiverId = new InputChannelID(); - - NettyMessage.ErrorResponse expected = new NettyMessage.ErrorResponse(expectedError, receiverId); - NettyMessage.ErrorResponse actual = encodeAndDecode(expected); - - assertEquals(expected.cause.getClass(), actual.cause.getClass()); - assertEquals(expected.cause.getMessage(), actual.cause.getMessage()); - assertEquals(receiverId, actual.receiverId); - } - - { - IllegalStateException expectedError = new IllegalStateException("Illegal illegal illegal"); - - NettyMessage.ErrorResponse expected = new NettyMessage.ErrorResponse(expectedError); - NettyMessage.ErrorResponse actual = encodeAndDecode(expected); - - assertEquals(expected.cause.getClass(), actual.cause.getClass()); - assertEquals(expected.cause.getMessage(), actual.cause.getMessage()); - assertNull(actual.receiverId); - assertTrue(actual.isFatalError()); - } - } - - { - NettyMessage.PartitionRequest expected = new NettyMessage.PartitionRequest(new ResultPartitionID(), random.nextInt(), new InputChannelID(), random.nextInt()); - NettyMessage.PartitionRequest actual = encodeAndDecode(expected); - - assertEquals(expected.partitionId, actual.partitionId); - assertEquals(expected.queueIndex, actual.queueIndex); - assertEquals(expected.receiverId, actual.receiverId); - assertEquals(expected.credit, actual.credit); - } - - { - NettyMessage.TaskEventRequest expected = new NettyMessage.TaskEventRequest(new IntegerTaskEvent(random.nextInt()), new ResultPartitionID(), new InputChannelID()); - NettyMessage.TaskEventRequest actual = encodeAndDecode(expected); - - assertEquals(expected.event, actual.event); - assertEquals(expected.partitionId, actual.partitionId); - assertEquals(expected.receiverId, actual.receiverId); - } - - { - NettyMessage.CancelPartitionRequest expected = new NettyMessage.CancelPartitionRequest(new InputChannelID()); - NettyMessage.CancelPartitionRequest actual = encodeAndDecode(expected); - - assertEquals(expected.receiverId, actual.receiverId); - } - - { - NettyMessage.CloseRequest expected = new NettyMessage.CloseRequest(); - NettyMessage.CloseRequest actual = encodeAndDecode(expected); - - assertEquals(expected.getClass(), actual.getClass()); - } - - { - NettyMessage.AddCredit expected = new NettyMessage.AddCredit(random.nextInt(Integer.MAX_VALUE) + 1, new InputChannelID()); - NettyMessage.AddCredit actual = encodeAndDecode(expected); - - assertEquals(expected.credit, actual.credit); - assertEquals(expected.receiverId, actual.receiverId); - } - } - - private void testEncodeDecodeBuffer(boolean testReadOnlyBuffer, boolean testCompressedBuffer) { - NetworkBuffer buffer = new NetworkBuffer(MemorySegmentFactory.allocateUnpooledSegment(BUFFER_SIZE), FreeingBufferRecycler.INSTANCE); - - for (int i = 0; i < BUFFER_SIZE; i += 8) { - buffer.writeLong(i); - } - - Buffer testBuffer = testReadOnlyBuffer ? buffer.readOnlySlice() : buffer; - if (testCompressedBuffer) { - testBuffer = COMPRESSOR.compressToOriginalBuffer(buffer); - } - - NettyMessage.BufferResponse expected = new NettyMessage.BufferResponse( - testBuffer, random.nextInt(), new InputChannelID(), random.nextInt()); - NettyMessage.BufferResponse actual = encodeAndDecode(expected); - - // Netty 4.1 is not copying the messages, but retaining slices of them. BufferResponse actual is in this case - // holding a reference to the buffer. Buffer will be recycled only once "actual" will be released. - assertFalse(buffer.isRecycled()); - assertFalse(testBuffer.isRecycled()); - - ByteBuf retainedSlice = actual.getNettyBuffer(); - if (testCompressedBuffer) { - assertTrue(actual.isCompressed); - retainedSlice = decompress(retainedSlice); - } - - // Ensure not recycled and same size as original buffer - assertEquals(1, retainedSlice.refCnt()); - assertEquals(BUFFER_SIZE, retainedSlice.readableBytes()); - - for (int i = 0; i < BUFFER_SIZE; i += 8) { - assertEquals(i, retainedSlice.readLong()); - } - - // Release the retained slice - actual.releaseBuffer(); - if (testCompressedBuffer) { - retainedSlice.release(); - } - assertEquals(0, retainedSlice.refCnt()); - assertTrue(buffer.isRecycled()); - assertTrue(testBuffer.isRecycled()); - - assertEquals(expected.sequenceNumber, actual.sequenceNumber); - assertEquals(expected.receiverId, actual.receiverId); - assertEquals(expected.backlog, actual.backlog); - } - - private ByteBuf decompress(ByteBuf buffer) { - MemorySegment segment = MemorySegmentFactory.allocateUnpooledSegment(BUFFER_SIZE); - Buffer compressedBuffer = new NetworkBuffer(segment, FreeingBufferRecycler.INSTANCE); - buffer.readBytes(compressedBuffer.asByteBuf(), buffer.readableBytes()); - compressedBuffer.setCompressed(true); - return DECOMPRESSOR.decompressToOriginalBuffer(compressedBuffer).asByteBuf(); - } - - @SuppressWarnings("unchecked") - private T encodeAndDecode(T msg) { - channel.writeOutbound(msg); - ByteBuf encoded = (ByteBuf) channel.readOutbound(); - - assertTrue(channel.writeInbound(encoded)); - - return (T) channel.readInbound(); - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageServerSideSerializationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageServerSideSerializationTest.java new file mode 100644 index 0000000000..3818767abc --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageServerSideSerializationTest.java @@ -0,0 +1,115 @@ +/* + * 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.runtime.io.network.netty; + +import org.apache.flink.runtime.event.task.IntegerTaskEvent; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID; +import org.apache.flink.shaded.netty4.io.netty.channel.embedded.EmbeddedChannel; +import org.apache.flink.util.TestLogger; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.util.Random; + +import static org.apache.flink.runtime.io.network.netty.NettyTestUtil.encodeAndDecode; +import static org.junit.Assert.assertEquals; + +/** + * Tests for the serialization and deserialization of the various {@link NettyMessage} sub-classes + * sent from client side to server side. + */ +public class NettyMessageServerSideSerializationTest extends TestLogger { + + private final Random random = new Random(); + + private EmbeddedChannel channel; + + @Before + public void setup() { + channel = new EmbeddedChannel( + new NettyMessage.NettyMessageEncoder(), // For outbound messages + new NettyMessage.NettyMessageDecoder()); // For inbound messages + } + + @After + public void tearDown() { + if (channel != null) { + channel.close(); + } + } + + @Test + public void testPartitionRequest() { + NettyMessage.PartitionRequest expected = new NettyMessage.PartitionRequest( + new ResultPartitionID(), + random.nextInt(), + new InputChannelID(), + random.nextInt()); + + NettyMessage.PartitionRequest actual = encodeAndDecode(expected, channel); + + assertEquals(expected.partitionId, actual.partitionId); + assertEquals(expected.queueIndex, actual.queueIndex); + assertEquals(expected.receiverId, actual.receiverId); + assertEquals(expected.credit, actual.credit); + } + + @Test + public void testTaskEventRequest() { + NettyMessage.TaskEventRequest expected = new NettyMessage.TaskEventRequest( + new IntegerTaskEvent(random.nextInt()), + new ResultPartitionID(), + new InputChannelID()); + NettyMessage.TaskEventRequest actual = encodeAndDecode(expected, channel); + + assertEquals(expected.event, actual.event); + assertEquals(expected.partitionId, actual.partitionId); + assertEquals(expected.receiverId, actual.receiverId); + } + + @Test + public void testCancelPartitionRequest() { + NettyMessage.CancelPartitionRequest expected = new NettyMessage.CancelPartitionRequest( + new InputChannelID()); + NettyMessage.CancelPartitionRequest actual = encodeAndDecode(expected, channel); + + assertEquals(expected.receiverId, actual.receiverId); + } + + @Test + public void testCloseRequest() { + NettyMessage.CloseRequest expected = new NettyMessage.CloseRequest(); + NettyMessage.CloseRequest actual = encodeAndDecode(expected, channel); + + assertEquals(expected.getClass(), actual.getClass()); + } + + @Test + public void testAddCredit() { + NettyMessage.AddCredit expected = new NettyMessage.AddCredit( + random.nextInt(Integer.MAX_VALUE) + 1, + new InputChannelID()); + NettyMessage.AddCredit actual = encodeAndDecode(expected, channel); + + assertEquals(expected.credit, actual.credit); + assertEquals(expected.receiverId, actual.receiverId); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyTestUtil.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyTestUtil.java index 7d0138b050..0aa93b8392 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyTestUtil.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyTestUtil.java @@ -19,6 +19,8 @@ package org.apache.flink.runtime.io.network.netty; import org.apache.flink.configuration.Configuration; +import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; +import org.apache.flink.shaded.netty4.io.netty.channel.embedded.EmbeddedChannel; import org.apache.flink.util.NetUtils; import org.apache.flink.shaded.netty4.io.netty.channel.Channel; @@ -28,8 +30,12 @@ import java.net.InetSocketAddress; import java.util.concurrent.TimeUnit; import java.util.function.Function; +import static junit.framework.TestCase.assertEquals; +import static org.apache.flink.runtime.io.network.netty.NettyMessage.ErrorResponse; +import static org.apache.flink.runtime.io.network.netty.NettyMessage.BufferResponse; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.junit.Assert.assertTrue; /** * Test utility for Netty server and client setup. @@ -162,6 +168,40 @@ public class NettyTestUtil { config); } + // --------------------------------------------------------------------------------------------- + // Encoding & Decoding + // --------------------------------------------------------------------------------------------- + + static T encodeAndDecode(T msg, EmbeddedChannel channel) { + channel.writeOutbound(msg); + ByteBuf encoded = channel.readOutbound(); + + assertTrue(channel.writeInbound(encoded)); + + return channel.readInbound(); + } + + // --------------------------------------------------------------------------------------------- + // Message Verification + // --------------------------------------------------------------------------------------------- + + static void verifyErrorResponse(ErrorResponse expected, ErrorResponse actual) { + assertEquals(expected.receiverId, actual.receiverId); + assertEquals(expected.cause.getClass(), actual.cause.getClass()); + assertEquals(expected.cause.getMessage(), actual.cause.getMessage()); + + if (expected.receiverId == null) { + assertTrue(actual.isFatalError()); + } + } + + static void verifyBufferResponseHeader(BufferResponse expected, BufferResponse actual) { + assertEquals(expected.backlog, actual.backlog); + assertEquals(expected.sequenceNumber, actual.sequenceNumber); + assertEquals(expected.bufferSize, actual.bufferSize); + assertEquals(expected.receiverId, actual.receiverId); + } + // ------------------------------------------------------------------------ static final class NettyServerAndClient { diff --git a/tools/travis/splits/split_misc.sh b/tools/travis/splits/split_misc.sh index 30f1d92736..dfc2494965 100755 --- a/tools/travis/splits/split_misc.sh +++ b/tools/travis/splits/split_misc.sh @@ -57,6 +57,8 @@ run_test "Streaming File Sink end-to-end test" "$END_TO_END_DIR/test-scripts/tes run_test "Streaming File Sink s3 end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_file_sink.sh s3" "skip_check_exceptions" run_test "Stateful stream job upgrade end-to-end test" "$END_TO_END_DIR/test-scripts/test_stateful_stream_job_upgrade.sh 2 4" +run_test "Netty shuffle direct memory consumption end-to-end test" "$END_TO_END_DIR/test-scripts/test_netty_shuffle_memory_control.sh" + run_test "Elasticsearch (v5.3.3) sink end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 5 https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-5.3.3.tar.gz" run_test "Elasticsearch (v6.3.1) sink end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 6 https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-6.3.1.tar.gz" -- Gitee From ae1d1b2481b4c585b7ee53f53ec44b0d9bcda3b5 Mon Sep 17 00:00:00 2001 From: HuangXingBo Date: Fri, 13 Mar 2020 11:07:15 +0800 Subject: [PATCH 156/885] [FLINK-16008][python][table-planner][table-planner-blink] Add rules to transpose the join condition as a Calc on top of the Python Correlate node (#11299) Since currently we don't support joining a Python UDTF with conditions, add a rule to transpose the condition as a Calc on top of the Python Correlate node. --- .../CalcPythonCorrelateTransposeRule.java | 110 +++++++++++++++++ .../stream/StreamExecPythonCorrelate.scala | 5 + .../plan/rules/FlinkStreamRuleSets.scala | 2 + ...plitPythonConditionFromCorrelateRule.scala | 8 +- .../CalcPythonCorrelateTransposeRuleTest.xml | 43 +++++++ ...CalcPythonCorrelateTransposeRuleTest.scala | 67 +++++++++++ .../CalcPythonCorrelateTransposeRule.java | 113 ++++++++++++++++++ .../DataStreamPythonCorrelate.scala | 6 +- .../table/plan/rules/FlinkRuleSets.scala | 2 + ...plitPythonConditionFromCorrelateRule.scala | 5 +- ...CalcPythonCorrelateTransposeRuleTest.scala | 63 ++++++++++ 11 files changed, 420 insertions(+), 4 deletions(-) create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/CalcPythonCorrelateTransposeRule.java create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/CalcPythonCorrelateTransposeRuleTest.xml create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/CalcPythonCorrelateTransposeRuleTest.scala create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/plan/rules/logical/CalcPythonCorrelateTransposeRule.java create mode 100644 flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/CalcPythonCorrelateTransposeRuleTest.scala diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/CalcPythonCorrelateTransposeRule.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/CalcPythonCorrelateTransposeRule.java new file mode 100644 index 0000000000..5501949ef7 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/CalcPythonCorrelateTransposeRule.java @@ -0,0 +1,110 @@ +/* + * 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.table.planner.plan.rules.logical; + +import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalCalc; +import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalCorrelate; +import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalRel; +import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalTableFunctionScan; +import org.apache.flink.table.planner.plan.rules.physical.stream.StreamExecCorrelateRule; +import org.apache.flink.table.planner.plan.utils.PythonUtil; + +import org.apache.calcite.plan.RelOptRule; +import org.apache.calcite.plan.RelOptRuleCall; +import org.apache.calcite.plan.RelOptUtil; +import org.apache.calcite.rel.core.JoinRelType; +import org.apache.calcite.rex.RexBuilder; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexProgram; +import org.apache.calcite.rex.RexProgramBuilder; +import org.apache.calcite.rex.RexUtil; + +import java.util.List; +import java.util.stream.Collectors; + +/** + * Rule will transpose the conditions after the Python correlate node if the join type is inner join. + */ +public class CalcPythonCorrelateTransposeRule extends RelOptRule { + + public static final CalcPythonCorrelateTransposeRule INSTANCE = + new CalcPythonCorrelateTransposeRule(); + + private CalcPythonCorrelateTransposeRule() { + super(operand(FlinkLogicalCorrelate.class, + operand(FlinkLogicalRel.class, any()), + operand(FlinkLogicalCalc.class, any())), + "CalcPythonCorrelateTransposeRule"); + } + + @Override + public boolean matches(RelOptRuleCall call) { + FlinkLogicalCorrelate correlate = call.rel(0); + FlinkLogicalCalc right = call.rel(2); + JoinRelType joinType = correlate.getJoinType(); + FlinkLogicalCalc mergedCalc = StreamExecCorrelateRule.getMergedCalc(right); + FlinkLogicalTableFunctionScan scan = StreamExecCorrelateRule.getTableScan(mergedCalc); + return joinType == JoinRelType.INNER && + PythonUtil.isPythonCall(scan.getCall(), null) && + mergedCalc.getProgram().getCondition() != null; + } + + @Override + public void onMatch(RelOptRuleCall call) { + FlinkLogicalCorrelate correlate = call.rel(0); + FlinkLogicalCalc right = call.rel(2); + RexBuilder rexBuilder = call.builder().getRexBuilder(); + FlinkLogicalCalc mergedCalc = StreamExecCorrelateRule.getMergedCalc(right); + FlinkLogicalTableFunctionScan tableScan = StreamExecCorrelateRule.getTableScan(mergedCalc); + RexProgram mergedCalcProgram = mergedCalc.getProgram(); + + InputRefRewriter inputRefRewriter = new InputRefRewriter( + correlate.getRowType().getFieldCount() - mergedCalc.getRowType().getFieldCount()); + List correlateFilters = RelOptUtil + .conjunctions(mergedCalcProgram.expandLocalRef(mergedCalcProgram.getCondition())) + .stream() + .map(x -> x.accept(inputRefRewriter)) + .collect(Collectors.toList()); + + FlinkLogicalCorrelate newCorrelate = new FlinkLogicalCorrelate( + correlate.getCluster(), + correlate.getTraitSet(), + correlate.getLeft(), + tableScan, + correlate.getCorrelationId(), + correlate.getRequiredColumns(), + correlate.getJoinType()); + + RexNode topCalcCondition = RexUtil.composeConjunction(rexBuilder, correlateFilters); + RexProgram rexProgram = new RexProgramBuilder( + newCorrelate.getRowType(), rexBuilder).getProgram(); + FlinkLogicalCalc newTopCalc = new FlinkLogicalCalc( + newCorrelate.getCluster(), + newCorrelate.getTraitSet(), + newCorrelate, + RexProgram.create( + newCorrelate.getRowType(), + rexProgram.getExprList(), + topCalcCondition, + newCorrelate.getRowType(), + rexBuilder)); + + call.transformTo(newTopCalc); + } +} diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecPythonCorrelate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecPythonCorrelate.scala index ca847fef5a..4ece477540 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecPythonCorrelate.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecPythonCorrelate.scala @@ -26,6 +26,7 @@ import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.JoinRelType import org.apache.calcite.rel.RelNode import org.apache.calcite.rex.{RexNode, RexProgram} +import org.apache.flink.table.api.TableException import org.apache.flink.table.planner.plan.nodes.common.CommonPythonCorrelate /** @@ -51,6 +52,10 @@ class StreamExecPythonCorrelate( joinType) with CommonPythonCorrelate { + if (condition.isDefined) { + throw new TableException("Currently Python correlate does not support conditions in left join.") + } + def copy( traitSet: RelTraitSet, newChild: RelNode, diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala index 44760f6f69..41fe922622 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala @@ -342,6 +342,8 @@ object FlinkStreamRuleSets { // Rule that splits python ScalarFunctions from // java/scala ScalarFunctions in correlate conditions SplitPythonConditionFromCorrelateRule.INSTANCE, + // Rule that transpose the conditions after the Python correlate node. + CalcPythonCorrelateTransposeRule.INSTANCE, // Rule that splits java calls from python TableFunction PythonCorrelateSplitRule.INSTANCE, // merge calc after calc transpose diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromCorrelateRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromCorrelateRule.scala index a5de6b7f77..4806d40cbd 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromCorrelateRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromCorrelateRule.scala @@ -22,9 +22,9 @@ import org.apache.calcite.plan.RelOptRule.{any, operand} import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelOptUtil} import org.apache.calcite.rel.core.JoinRelType import org.apache.calcite.rex._ -import org.apache.flink.table.planner.plan.nodes.logical.{FlinkLogicalCalc, FlinkLogicalCorrelate, FlinkLogicalRel} +import org.apache.flink.table.planner.plan.nodes.logical.{FlinkLogicalCalc, FlinkLogicalCorrelate, FlinkLogicalRel, FlinkLogicalTableFunctionScan} import org.apache.flink.table.planner.plan.rules.physical.stream.StreamExecCorrelateRule -import org.apache.flink.table.planner.plan.utils.PythonUtil.containsPythonCall +import org.apache.flink.table.planner.plan.utils.PythonUtil.{containsPythonCall, isNonPythonCall} import org.apache.flink.table.planner.plan.utils.RexDefaultVisitor import scala.collection.JavaConversions._ @@ -52,7 +52,11 @@ class SplitPythonConditionFromCorrelateRule val right: FlinkLogicalCalc = call.rel(2).asInstanceOf[FlinkLogicalCalc] val joinType: JoinRelType = correlate.getJoinType val mergedCalc = StreamExecCorrelateRule.getMergedCalc(right) + val tableScan = StreamExecCorrelateRule + .getTableScan(mergedCalc) + .asInstanceOf[FlinkLogicalTableFunctionScan] joinType == JoinRelType.INNER && + isNonPythonCall(tableScan.getCall) && Option(mergedCalc.getProgram.getCondition) .map(mergedCalc.getProgram.expandLocalRef) .exists(containsPythonCall(_)) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/CalcPythonCorrelateTransposeRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/CalcPythonCorrelateTransposeRuleTest.xml new file mode 100644 index 0000000000..b1dda00c72 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/CalcPythonCorrelateTransposeRuleTest.xml @@ -0,0 +1,43 @@ + + + + + + + + + + + + + + + diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/CalcPythonCorrelateTransposeRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/CalcPythonCorrelateTransposeRuleTest.scala new file mode 100644 index 0000000000..5f97cfaca4 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/CalcPythonCorrelateTransposeRuleTest.scala @@ -0,0 +1,67 @@ +/* + * 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.table.planner.plan.rules.logical + +import org.apache.calcite.plan.hep.HepMatchOrder +import org.apache.flink.api.scala._ +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.planner.plan.nodes.FlinkConventions +import org.apache.flink.table.planner.plan.optimize.program._ +import org.apache.flink.table.planner.plan.rules.FlinkStreamRuleSets +import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.PythonScalarFunction +import org.apache.flink.table.planner.utils.{MockPythonTableFunction, TableTestBase} +import org.junit.{Before, Test} + +class CalcPythonCorrelateTransposeRuleTest extends TableTestBase { + + private val util = streamTestUtil() + + @Before + def setup(): Unit = { + val programs = new FlinkChainedProgram[StreamOptimizeContext]() + // query decorrelation + programs.addLast("decorrelate", new FlinkDecorrelateProgram) + programs.addLast( + "logical", + FlinkVolcanoProgramBuilder.newBuilder + .add(FlinkStreamRuleSets.LOGICAL_OPT_RULES) + .setRequiredOutputTraits(Array(FlinkConventions.LOGICAL)) + .build()) + programs.addLast( + "logical_rewrite", + FlinkHepRuleSetProgramBuilder.newBuilder + .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_SEQUENCE) + .setHepMatchOrder(HepMatchOrder.BOTTOM_UP) + .add(FlinkStreamRuleSets.LOGICAL_REWRITE) + .build()) + util.replaceStreamProgram(programs) + + util.addTableSource[(Int, Int, Int)]("MyTable", 'a, 'b, 'c) + util.addFunction("func", new MockPythonTableFunction) + util.addFunction("pyFunc", new PythonScalarFunction("pyFunc")) + } + + @Test + def testPythonFunctionInCorrelateCondition(): Unit = { + val sqlQuery = "SELECT a, b, c, x, y FROM MyTable, LATERAL TABLE(func(a * a, b)) AS T(x, y) " + + "WHERE x = a and pyFunc(x, x) = 2 and y + 1 = y * y" + util.verifyPlan(sqlQuery) + } + +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/plan/rules/logical/CalcPythonCorrelateTransposeRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/plan/rules/logical/CalcPythonCorrelateTransposeRule.java new file mode 100644 index 0000000000..41975b7814 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/plan/rules/logical/CalcPythonCorrelateTransposeRule.java @@ -0,0 +1,113 @@ +/* + * 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.table.plan.rules.logical; + +import org.apache.flink.table.plan.nodes.logical.FlinkLogicalCalc; +import org.apache.flink.table.plan.nodes.logical.FlinkLogicalCorrelate; +import org.apache.flink.table.plan.nodes.logical.FlinkLogicalRel; +import org.apache.flink.table.plan.nodes.logical.FlinkLogicalTableFunctionScan; +import org.apache.flink.table.plan.util.CorrelateUtil; +import org.apache.flink.table.plan.util.PythonUtil; + +import org.apache.calcite.plan.RelOptRule; +import org.apache.calcite.plan.RelOptRuleCall; +import org.apache.calcite.plan.RelOptUtil; +import org.apache.calcite.rel.core.JoinRelType; +import org.apache.calcite.rex.RexBuilder; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexProgram; +import org.apache.calcite.rex.RexProgramBuilder; +import org.apache.calcite.rex.RexUtil; + +import java.util.List; +import java.util.stream.Collectors; + +import scala.Option; + +/** + * Rule will transpose the conditions after the Python correlate node if the join type is inner join. + */ +public class CalcPythonCorrelateTransposeRule extends RelOptRule { + + public static final CalcPythonCorrelateTransposeRule INSTANCE = + new CalcPythonCorrelateTransposeRule(); + + private CalcPythonCorrelateTransposeRule() { + super(operand(FlinkLogicalCorrelate.class, + operand(FlinkLogicalRel.class, any()), + operand(FlinkLogicalCalc.class, any())), + "CalcPythonCorrelateTransposeRule"); + } + + @Override + public boolean matches(RelOptRuleCall call) { + FlinkLogicalCorrelate correlate = call.rel(0); + FlinkLogicalCalc right = call.rel(2); + JoinRelType joinType = correlate.getJoinType(); + FlinkLogicalCalc mergedCalc = CorrelateUtil.getMergedCalc(right); + Option scan = CorrelateUtil.getTableFunctionScan(mergedCalc); + return joinType == JoinRelType.INNER && + scan.isDefined() && + PythonUtil.isPythonCall(scan.get().getCall(), null) && + mergedCalc.getProgram().getCondition() != null; + } + + @Override + public void onMatch(RelOptRuleCall call) { + FlinkLogicalCorrelate correlate = call.rel(0); + FlinkLogicalCalc right = call.rel(2); + RexBuilder rexBuilder = call.builder().getRexBuilder(); + FlinkLogicalCalc mergedCalc = CorrelateUtil.getMergedCalc(right); + FlinkLogicalTableFunctionScan tableScan = CorrelateUtil.getTableFunctionScan(mergedCalc).get(); + RexProgram mergedCalcProgram = mergedCalc.getProgram(); + + InputRefRewriter inputRefRewriter = new InputRefRewriter( + correlate.getRowType().getFieldCount() - mergedCalc.getRowType().getFieldCount()); + List correlateFilters = RelOptUtil + .conjunctions(mergedCalcProgram.expandLocalRef(mergedCalcProgram.getCondition())) + .stream() + .map(x -> x.accept(inputRefRewriter)) + .collect(Collectors.toList()); + + FlinkLogicalCorrelate newCorrelate = new FlinkLogicalCorrelate( + correlate.getCluster(), + correlate.getTraitSet(), + correlate.getLeft(), + tableScan, + correlate.getCorrelationId(), + correlate.getRequiredColumns(), + correlate.getJoinType()); + + RexNode topCalcCondition = RexUtil.composeConjunction(rexBuilder, correlateFilters); + RexProgram rexProgram = new RexProgramBuilder( + newCorrelate.getRowType(), rexBuilder).getProgram(); + FlinkLogicalCalc newTopCalc = new FlinkLogicalCalc( + newCorrelate.getCluster(), + newCorrelate.getTraitSet(), + newCorrelate, + RexProgram.create( + newCorrelate.getRowType(), + rexProgram.getExprList(), + topCalcCondition, + newCorrelate.getRowType(), + rexBuilder)); + + call.transformTo(newTopCalc); + } +} diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamPythonCorrelate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamPythonCorrelate.scala index 0021baccff..1e46c3571b 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamPythonCorrelate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamPythonCorrelate.scala @@ -22,7 +22,7 @@ import org.apache.calcite.rel.RelNode import org.apache.calcite.rel.core.JoinRelType import org.apache.calcite.rex.{RexCall, RexNode} import org.apache.flink.streaming.api.datastream.DataStream -import org.apache.flink.table.api.StreamQueryConfig +import org.apache.flink.table.api.{StreamQueryConfig, TableException} import org.apache.flink.table.functions.utils.TableSqlFunction import org.apache.flink.table.plan.nodes.CommonPythonCorrelate import org.apache.flink.table.plan.nodes.logical.FlinkLogicalTableFunctionScan @@ -57,6 +57,10 @@ class DataStreamPythonCorrelate( joinType) with CommonPythonCorrelate { + if (condition.isDefined) { + throw new TableException("Currently Python correlate does not support conditions in left join.") + } + override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = { new DataStreamPythonCorrelate( cluster, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala index 7bbfa6a864..534abe4929 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala @@ -152,6 +152,8 @@ object FlinkRuleSets { // Rule that splits python ScalarFunctions from // java/scala ScalarFunctions in correlate conditions SplitPythonConditionFromCorrelateRule.INSTANCE, + // Rule that transpose the conditions after the Python correlate node. + CalcPythonCorrelateTransposeRule.INSTANCE, // Rule that splits java calls from python TableFunction PythonCorrelateSplitRule.INSTANCE, CalcMergeRule.INSTANCE, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/SplitPythonConditionFromCorrelateRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/SplitPythonConditionFromCorrelateRule.scala index 09a90a68e1..cee224a1a8 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/SplitPythonConditionFromCorrelateRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/SplitPythonConditionFromCorrelateRule.scala @@ -23,7 +23,7 @@ import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelOptUtil} import org.apache.calcite.rel.core.JoinRelType import org.apache.calcite.rex._ import org.apache.flink.table.plan.nodes.logical.{FlinkLogicalCalc, FlinkLogicalCorrelate, FlinkLogicalRel} -import org.apache.flink.table.plan.util.PythonUtil.containsPythonCall +import org.apache.flink.table.plan.util.PythonUtil.{containsPythonCall, isNonPythonCall} import org.apache.flink.table.plan.util.{CorrelateUtil, RexDefaultVisitor} import scala.collection.JavaConversions._ @@ -51,7 +51,10 @@ class SplitPythonConditionFromCorrelateRule val right: FlinkLogicalCalc = call.rel(2).asInstanceOf[FlinkLogicalCalc] val joinType: JoinRelType = correlate.getJoinType val mergedCalc = CorrelateUtil.getMergedCalc(right) + val tableScan = CorrelateUtil.getTableFunctionScan(mergedCalc) joinType == JoinRelType.INNER && + tableScan.isDefined && + isNonPythonCall(tableScan.get.getCall) && Option(mergedCalc.getProgram.getCondition) .map(mergedCalc.getProgram.expandLocalRef) .exists(containsPythonCall(_)) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/CalcPythonCorrelateTransposeRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/CalcPythonCorrelateTransposeRuleTest.scala new file mode 100644 index 0000000000..a5bd851be5 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/CalcPythonCorrelateTransposeRuleTest.scala @@ -0,0 +1,63 @@ +/* + * 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.table.plan + +import org.apache.flink.api.scala._ +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.runtime.utils.JavaUserDefinedScalarFunctions.PythonScalarFunction +import org.apache.flink.table.utils.{MockPythonTableFunction, TableTestBase} +import org.apache.flink.table.utils.TableTestUtil.{streamTableNode, term, unaryNode} +import org.junit.Test + +class CalcPythonCorrelateTransposeRuleTest extends TableTestBase { + @Test + def testPythonTableFunctionWithCondition(): Unit = { + val util = streamTestUtil() + val table = util.addTable[(Int, Int, Int)]("MyTable", 'a, 'b, 'c) + val scalarFunc = new PythonScalarFunction("pyFunc") + val tableFunc = new MockPythonTableFunction() + + val resultTable = table.joinLateral( + tableFunc('a * 'a, 'b) as('x, 'y), + 'x === 'a && scalarFunc('x, 'x) === 2 && 'y + 1 === 'y * 'y) + val expected = unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamPythonCalc", + unaryNode( + "DataStreamPythonCorrelate", + unaryNode( + "DataStreamCalc", + streamTableNode(table), + term("select", "a, b, c, *(a, a) AS f0")), + term("invocation", s"${tableFunc.functionIdentifier}" + + s"($$3, $$1)"), + term("correlate", s"table(${tableFunc.getClass.getSimpleName}" + + s"(f0, b))"), + term("select", "a, b, c, f0, x, y"), + term("rowType", + "RecordType(INTEGER a, INTEGER b, INTEGER c, INTEGER f0, INTEGER x, INTEGER y)"), + term("joinType", "INNER")), + term("select", "a, b, c, x, y, pyFunc(x, x) AS f0")), + term("select", "a, b, c, x, y"), + term("where", "AND(AND(=(f0, 2), =(+(y, 1), *(y, y))), =(x, a))")) + util.verifyTable(resultTable, expected) + } + +} -- Gitee From 242430199e7fa58ac8d528475d09be65cb2dc7cf Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Fri, 13 Mar 2020 14:32:57 +0800 Subject: [PATCH 157/885] [FLINK-16538][python][docs] Restructure Python Table API documentation (#11375) Adds an item "Python Table API" under "Table API & SQL" and move the documentation about Python Table API under it. --- docs/dev/table/config.md | 4 - docs/dev/table/config.zh.md | 4 - docs/dev/table/functions/udfs.md | 149 +----------------- docs/dev/table/functions/udfs.zh.md | 149 +----------------- .../dev/table/python/dependency_management.md | 94 +++++++++++ .../table/python/dependency_management.zh.md | 94 +++++++++++ docs/dev/table/python/index.md | 35 ++++ docs/dev/table/python/index.zh.md | 35 ++++ docs/dev/table/python/installation.md | 44 ++++++ docs/dev/table/python/installation.zh.md | 44 ++++++ docs/dev/table/python/python_config.md | 32 ++++ docs/dev/table/python/python_config.zh.md | 32 ++++ docs/dev/table/python/python_udfs.md | 125 +++++++++++++++ docs/dev/table/python/python_udfs.zh.md | 125 +++++++++++++++ .../walkthroughs/python_table_api.md | 16 +- .../walkthroughs/python_table_api.zh.md | 12 +- docs/ops/python_shell.md | 7 +- docs/ops/python_shell.zh.md | 7 +- 18 files changed, 682 insertions(+), 326 deletions(-) create mode 100644 docs/dev/table/python/dependency_management.md create mode 100644 docs/dev/table/python/dependency_management.zh.md create mode 100644 docs/dev/table/python/index.md create mode 100644 docs/dev/table/python/index.zh.md create mode 100644 docs/dev/table/python/installation.md create mode 100644 docs/dev/table/python/installation.zh.md create mode 100644 docs/dev/table/python/python_config.md create mode 100644 docs/dev/table/python/python_config.zh.md create mode 100644 docs/dev/table/python/python_udfs.md create mode 100644 docs/dev/table/python/python_udfs.zh.md diff --git a/docs/dev/table/config.md b/docs/dev/table/config.md index a9e9b7119c..dd312d6fc5 100644 --- a/docs/dev/table/config.md +++ b/docs/dev/table/config.md @@ -104,7 +104,3 @@ The following options can be used to tune the performance of the query execution The following options can be used to adjust the behavior of the query optimizer to get a better execution plan. {% include generated/optimizer_config_configuration.html %} - -### Python Options - -{% include generated/python_configuration.html %} diff --git a/docs/dev/table/config.zh.md b/docs/dev/table/config.zh.md index 0ef01a1a62..1947ab8b54 100644 --- a/docs/dev/table/config.zh.md +++ b/docs/dev/table/config.zh.md @@ -104,7 +104,3 @@ The following options can be used to tune the performance of the query execution The following options can be used to adjust the behavior of the query optimizer to get a better execution plan. {% include generated/optimizer_config_configuration.html %} - -### Python Options - -{% include generated/python_configuration.html %} diff --git a/docs/dev/table/functions/udfs.md b/docs/dev/table/functions/udfs.md index 724f1ae39e..37cfd04973 100644 --- a/docs/dev/table/functions/udfs.md +++ b/docs/dev/table/functions/udfs.md @@ -134,35 +134,12 @@ object TimestampModifier extends ScalarFunction {

    -Note Python 3.5+ and apache-beam==2.19.0 are required to run the Python scalar function. +In order to define a Python scalar function, one can extend the base class `ScalarFunction` in `pyflink.table.udf` and implement an evaluation method. The behavior of a Python scalar function is determined by the evaluation method which is named `eval`. -Note By default PyFlink uses the command “python” to run the python udf workers. Before starting cluster, run the following command to confirm that it meets the requirements: - -{% highlight bash %} -$ python --version -# the version printed here must be 3.5+ -$ python -m pip install apache-beam==2.19.0 -{% endhighlight %} - -It supports to use both Java/Scala scalar functions and Python scalar functions in Python Table API and SQL. In order to define a Python scalar function, one can extend the base class `ScalarFunction` in `pyflink.table.udf` and implement an evaluation method. The behavior of a Python scalar function is determined by the evaluation method. An evaluation method must be named `eval`. Evaluation method can also support variable arguments, such as `eval(*args)`. - -The following example shows how to define your own Java and Python hash code functions, register them in the TableEnvironment, and call them in a query. Note that you can configure your scalar function via a constructor before it is registered: +The following example shows how to define your own Python hash code function, register it in the TableEnvironment, and call it in a query. Note that you can configure your scalar function via a constructor before it is registered: {% highlight python %} -''' -Java code: - -// The Java class must have a public no-argument constructor and can be founded in current Java classloader. -public class HashCode extends ScalarFunction { - private int factor = 12; - - public int eval(String s) { - return s.hashCode() * factor; - } -} -''' - -class PyHashCode(ScalarFunction): +class HashCode(ScalarFunction): def __init__(self): self.factor = 12 @@ -171,128 +148,18 @@ class PyHashCode(ScalarFunction): table_env = BatchTableEnvironment.create(env) -# register the Java function -table_env.register_java_function("hashCode", "my.java.function.HashCode") - # register the Python function -table_env.register_function("py_hash_code", udf(PyHashCode(), DataTypes.BIGINT(), DataTypes.BIGINT())) +table_env.register_function("hash_code", udf(HashCode(), DataTypes.BIGINT(), DataTypes.BIGINT())) # use the function in Python Table API -my_table.select("string, bigint, string.hashCode(), hashCode(string), bigint.py_hash_code(), py_hash_code(bigint)") +my_table.select("string, bigint, string.hash_code(), hash_code(string)") # use the function in SQL API -table_env.sql_query("SELECT string, bigint, hashCode(string), py_hash_code(bigint) FROM MyTable") -{% endhighlight %} - -There are many ways to define a Python scalar function besides extending the base class `ScalarFunction`. The following example shows the different ways to define a Python scalar function which takes two columns of bigint as input parameters and returns the sum of them as the result. - -{% highlight python %} -# option 1: extending the base class `ScalarFunction` -class Add(ScalarFunction): - def eval(self, i, j): - return i + j - -add = udf(Add(), [DataTypes.BIGINT(), DataTypes.BIGINT()], DataTypes.BIGINT()) - -# option 2: Python function -@udf(input_types=[DataTypes.BIGINT(), DataTypes.BIGINT()], result_type=DataTypes.BIGINT()) -def add(i, j): - return i + j - -# option 3: lambda function -add = udf(lambda i, j: i + j, [DataTypes.BIGINT(), DataTypes.BIGINT()], DataTypes.BIGINT()) - -# option 4: callable function -class CallableAdd(object): - def __call__(self, i, j): - return i + j - -add = udf(CallableAdd(), [DataTypes.BIGINT(), DataTypes.BIGINT()], DataTypes.BIGINT()) - -# option 5: partial function -def partial_add(i, j, k): - return i + j + k - -add = udf(functools.partial(partial_add, k=1), [DataTypes.BIGINT(), DataTypes.BIGINT()], - DataTypes.BIGINT()) - -# register the Python function -table_env.register_function("add", add) -# use the function in Python Table API -my_table.select("add(a, b)") -{% endhighlight %} - -If the python scalar function depends on third-party dependencies, you can specify the dependencies with the following table APIs or through command line directly when submitting the job. - -
    Pattern OperationDescription模式操作描述
    begin(#name) -

    Defines a starting pattern:

    +

    定一个开始模式:

    {% highlight scala %} val start = Pattern.begin[Event]("start") {% endhighlight %} @@ -1157,7 +1123,7 @@ val start = Pattern.begin[Event]("start")
    begin(#pattern_sequence) -

    Defines a starting pattern:

    +

    定一个开始模式:

    {% highlight scala %} val start = Pattern.begin( Pattern.begin[Event]("start").where(...).followedBy("middle").where(...) @@ -1168,8 +1134,7 @@ val start = Pattern.begin(
    next(#name) -

    Appends a new pattern. A matching event has to directly succeed the previous matching event - (strict contiguity):

    +

    增加一个新的模式,匹配的事件必须是直接跟在前面匹配到的事件后面(严格连续):

    {% highlight scala %} val next = start.next("middle") {% endhighlight %} @@ -1178,8 +1143,7 @@ val next = start.next("middle")
    next(#pattern_sequence) -

    Appends a new pattern. A sequence of matching events have to directly succeed the previous matching event - (strict contiguity):

    +

    增加一个新的模式。匹配的事件序列必须是直接跟在前面匹配到的事件后面(严格连续):

    {% highlight scala %} val next = start.next( Pattern.begin[Event]("start").where(...).followedBy("middle").where(...) @@ -1190,8 +1154,7 @@ val next = start.next(
    followedBy(#name) -

    Appends a new pattern. Other events can occur between a matching event and the previous - matching event (relaxed contiguity) :

    +

    增加一个新的模式。可以有其他事件出现在匹配的事件和之前匹配到的事件中间(松散连续):

    {% highlight scala %} val followedBy = start.followedBy("middle") {% endhighlight %} @@ -1200,8 +1163,7 @@ val followedBy = start.followedBy("middle")
    followedBy(#pattern_sequence) -

    Appends a new pattern. Other events can occur between a sequence of matching events and the previous - matching event (relaxed contiguity) :

    +

    增加一个新的模式。可以有其他事件出现在匹配的事件和之前匹配到的事件中间(松散连续):

    {% highlight scala %} val followedBy = start.followedBy( Pattern.begin[Event]("start").where(...).followedBy("middle").where(...) @@ -1212,9 +1174,8 @@ val followedBy = start.followedBy(
    followedByAny(#name) -

    Appends a new pattern. Other events can occur between a matching event and the previous - matching event, and alternative matches will be presented for every alternative matching event - (non-deterministic relaxed contiguity):

    +

    增加一个新的模式。可以有其他事件出现在匹配的事件和之前匹配到的事件中间, + 每个可选的匹配事件都会作为可选的匹配结果输出(不确定的松散连续):

    {% highlight scala %} val followedByAny = start.followedByAny("middle") {% endhighlight %} @@ -1223,9 +1184,8 @@ val followedByAny = start.followedByAny("middle")
    followedByAny(#pattern_sequence) -

    Appends a new pattern. Other events can occur between a sequence of matching events and the previous - matching event, and alternative matches will be presented for every alternative sequence of matching events - (non-deterministic relaxed contiguity):

    +

    增加一个新的模式。可以有其他事件出现在匹配的事件序列和之前匹配到的事件中间, + 每个可选的匹配事件序列都会作为可选的匹配结果输出(不确定的松散连续):

    {% highlight scala %} val followedByAny = start.followedByAny( Pattern.begin[Event]("start").where(...).followedBy("middle").where(...) @@ -1237,8 +1197,8 @@ val followedByAny = start.followedByAny(
    notNext() -

    Appends a new negative pattern. A matching (negative) event has to directly succeed the - previous matching event (strict contiguity) for the partial match to be discarded:

    +

    增加一个新的否定模式。匹配的(否定)事件必须直接跟在前面匹配到的事件之后 + (严格连续)来丢弃这些部分匹配:

    {% highlight scala %} val notNext = start.notNext("not") {% endhighlight %} @@ -1247,9 +1207,8 @@ val notNext = start.notNext("not")
    notFollowedBy() -

    Appends a new negative pattern. A partial matching event sequence will be discarded even - if other events occur between the matching (negative) event and the previous matching event - (relaxed contiguity):

    +

    增加一个新的否定模式。即使有其他事件在匹配的(否定)事件和之前匹配的事件之间发生, + 部分匹配的事件序列也会被丢弃(松散连续):

    {% highlight scala %} val notFollowedBy = start.notFollowedBy("not") {% endhighlight %} @@ -1259,8 +1218,7 @@ val notFollowedBy = start.notFollowedBy("not")
    within(time) -

    Defines the maximum time interval for an event sequence to match the pattern. If a non-completed event - sequence exceeds this time, it is discarded:

    +

    定义匹配模式的事件序列出现的最大时间间隔。如果未完成的事件序列超过了这个事件,就会被丢弃:

    {% highlight scala %} pattern.within(Time.seconds(10)) {% endhighlight %} @@ -1272,25 +1230,26 @@ pattern.within(Time.seconds(10)) -### After Match Skip Strategy +### 匹配后跳过策略 -For a given pattern, the same event may be assigned to multiple successful matches. To control to how many matches an event will be assigned, you need to specify the skip strategy called `AfterMatchSkipStrategy`. There are five types of skip strategies, listed as follows: +对于一个给定的模式,同一个事件可能会分配到多个成功的匹配上。为了控制一个事件会分配到多少个匹配上,你需要指定跳过策略`AfterMatchSkipStrategy`。 +有五种跳过策略,如下: -* *NO_SKIP*: Every possible match will be emitted. -* *SKIP_TO_NEXT*: Discards every partial match that started with the same event, emitted match was started. -* *SKIP_PAST_LAST_EVENT*: Discards every partial match that started after the match started but before it ended. -* *SKIP_TO_FIRST*: Discards every partial match that started after the match started but before the first event of *PatternName* occurred. -* *SKIP_TO_LAST*: Discards every partial match that started after the match started but before the last event of *PatternName* occurred. +* *NO_SKIP*: 每个成功的匹配都会被输出。 +* *SKIP_TO_NEXT*: 丢弃以相同事件开始的所有部分匹配。 +* *SKIP_PAST_LAST_EVENT*: 丢弃起始在这个匹配的开始和结束之间的所有部分匹配。 +* *SKIP_TO_FIRST*: 丢弃起始在这个匹配的开始和第一个出现的名称为*PatternName*事件之间的所有部分匹配。 +* *SKIP_TO_LAST*: 丢弃起始在这个匹配的开始和最后一个出现的名称为*PatternName*事件之间的所有部分匹配。 -Notice that when using *SKIP_TO_FIRST* and *SKIP_TO_LAST* skip strategy, a valid *PatternName* should also be specified. +注意当使用*SKIP_TO_FIRST*和*SKIP_TO_LAST*策略时,需要指定一个合法的*PatternName*. -For example, for a given pattern `b+ c` and a data stream `b1 b2 b3 c`, the differences between these four skip strategies are as follows: +例如,给定一个模式`b+ c`和一个数据流`b1 b2 b3 c`,不同跳过策略之间的不同如下: - - - + + + @@ -1299,7 +1258,7 @@ For example, for a given pattern `b+ c` and a data stream `b1 b2 b3 c`, the diff b2 b3 c
    b3 c
    - + @@ -1308,14 +1267,14 @@ For example, for a given pattern `b+ c` and a data stream `b1 b2 b3 c`, the diff b2 b3 c
    b3 c
    - + - + @@ -1324,7 +1283,7 @@ For example, for a given pattern `b+ c` and a data stream `b1 b2 b3 c`, the diff b2 b3 c
    b3 c
    - + @@ -1332,19 +1291,19 @@ For example, for a given pattern `b+ c` and a data stream `b1 b2 b3 c`, the diff b1 b2 b3 c
    b3 c
    - +
    Skip StrategyResult Description跳过策略结果 描述
    NO_SKIP After found matching b1 b2 b3 c, the match process will not discard any result.找到匹配b1 b2 b3 c之后,不会丢弃任何结果。
    SKIP_TO_NEXT After found matching b1 b2 b3 c, the match process will not discard any result, because no other match could start at b1.找到匹配b1 b2 b3 c之后,不会丢弃任何结果,因为没有以b1开始的其他匹配。
    SKIP_PAST_LAST_EVENT b1 b2 b3 c
    After found matching b1 b2 b3 c, the match process will discard all started partial matches.找到匹配b1 b2 b3 c之后,会丢弃其他所有的部分匹配。
    SKIP_TO_FIRST[b] After found matching b1 b2 b3 c, the match process will try to discard all partial matches started before b1, but there are no such matches. Therefore nothing will be discarded.找到匹配b1 b2 b3 c之后,会尝试丢弃所有在b1之前开始的部分匹配,但没有这样的匹配,所以没有任何匹配被丢弃。
    SKIP_TO_LAST[b] After found matching b1 b2 b3 c, the match process will try to discard all partial matches started before b3. There is one such match b2 b3 c找到匹配b1 b2 b3 c之后,会尝试丢弃所有在b3之前开始的部分匹配,有一个这样的b2 b3 c被丢弃。
    -Have a look also at another example to better see the difference between NO_SKIP and SKIP_TO_FIRST: -Pattern: `(a | b | c) (b | c) c+.greedy d` and sequence: `a b c1 c2 c3 d` Then the results will be: +在看另外一个例子来说明NO_SKIP和SKIP_TO_FIRST之间的差别: +模式: `(a | b | c) (b | c) c+.greedy d`,输入:`a b c1 c2 c3 d`,结果将会是: - - - + + + @@ -1353,7 +1312,7 @@ Pattern: `(a | b | c) (b | c) c+.greedy d` and sequence: `a b c1 c2 c3 d` Then t b c1 c2 c3 d
    c1 c2 c3 d
    - + @@ -1361,19 +1320,19 @@ Pattern: `(a | b | c) (b | c) c+.greedy d` and sequence: `a b c1 c2 c3 d` Then t a b c1 c2 c3 d
    c1 c2 c3 d
    - +
    Skip StrategyResult Description跳过策略结果 描述
    NO_SKIP After found matching a b c1 c2 c3 d, the match process will not discard any result.找到匹配a b c1 c2 c3 d之后,不会丢弃任何结果。
    SKIP_TO_FIRST[c*] After found matching a b c1 c2 c3 d, the match process will discard all partial matches started before c1. There is one such match b c1 c2 c3 d.找到匹配a b c1 c2 c3 d之后,会丢弃所有在c1之前开始的部分匹配,有一个这样的b c1 c2 c3 d被丢弃。
    -To better understand the difference between NO_SKIP and SKIP_TO_NEXT take a look at following example: -Pattern: `a b+` and sequence: `a b1 b2 b3` Then the results will be: +为了更好的理解NO_SKIP和SKIP_TO_NEXT之间的差别,看下面的例子: +模式:`a b+`,输入:`a b1 b2 b3`,结果将会是: - - - + + + @@ -1382,46 +1341,46 @@ Pattern: `a b+` and sequence: `a b1 b2 b3` Then the results will be: a b1 b2
    a b1 b2 b3
    - + - +
    Skip StrategyResult Description跳过策略结果 描述
    NO_SKIP After found matching a b1, the match process will not discard any result.找到匹配a b1之后,不会丢弃任何结果。
    SKIP_TO_NEXT a b1
    After found matching a b1, the match process will discard all partial matches started at a. This means neither a b1 b2 nor a b1 b2 b3 could be generated.找到匹配a b1之后,会丢弃所有以a开始的部分匹配。这意味着不会产生a b1 b2a b1 b2 b3了。
    -To specify which skip strategy to use, just create an `AfterMatchSkipStrategy` by calling: +想指定要使用的跳过策略,只需要调用下面的方法创建`AfterMatchSkipStrategy`: - - + + - + - + - + - + - +
    FunctionDescription方法描述
    AfterMatchSkipStrategy.noSkip()Create a NO_SKIP skip strategy 创建NO_SKIP策略
    AfterMatchSkipStrategy.skipToNext()Create a SKIP_TO_NEXT skip strategy 创建SKIP_TO_NEXT策略
    AfterMatchSkipStrategy.skipPastLastEvent()Create a SKIP_PAST_LAST_EVENT skip strategy 创建SKIP_PAST_LAST_EVENT策略
    AfterMatchSkipStrategy.skipToFirst(patternName)Create a SKIP_TO_FIRST skip strategy with the referenced pattern name patternName创建引用模式名称为patternNameSKIP_TO_FIRST策略
    AfterMatchSkipStrategy.skipToLast(patternName)Create a SKIP_TO_LAST skip strategy with the referenced pattern name patternName创建引用模式名称为patternNameSKIP_TO_LAST策略
    -Then apply the skip strategy to a pattern by calling: +可以通过调用下面方法将跳过策略应用到模式上:
    @@ -1438,9 +1397,9 @@ Pattern.begin("patternName", skipStrategy)
    -{% warn Attention %} For SKIP_TO_FIRST/LAST there are two options how to handle cases when there are no elements mapped to -the specified variable. By default a NO_SKIP strategy will be used in this case. The other option is to throw exception in such situation. -One can enable this option by: +{% warn 注意 %} 使用SKIP_TO_FIRST/LAST时,有两个选项可以用来处理没有事件可以映射到对应的变量名上的情况。 +默认情况下会使用NO_SKIP策略,另外一个选项是抛出异常。 +可以使用如下的选项:
    @@ -1455,18 +1414,18 @@ AfterMatchSkipStrategy.skipToFirst(patternName).throwExceptionOnMiss()
    -## Detecting Patterns +## 检测模式 -After specifying the pattern sequence you are looking for, it is time to apply it to your input stream to detect -potential matches. To run a stream of events against your pattern sequence, you have to create a `PatternStream`. -Given an input stream `input`, a pattern `pattern` and an optional comparator `comparator` used to sort events with the same timestamp in case of EventTime or that arrived at the same moment, you create the `PatternStream` by calling: +在指定了要寻找的模式后,该把它们应用到输入流上来发现可能的匹配了。为了在事件流上运行你的模式,需要创建一个`PatternStream`。 +给定一个输入流`input`,一个模式`pattern`和一个可选的用来对使用事件时间时有同样时间戳或者同时到达的事件进行排序的比较器`comparator`, +你可以通过调用如下方法来创建`PatternStream`:
    {% highlight java %} DataStream input = ... Pattern pattern = ... -EventComparator comparator = ... // optional +EventComparator comparator = ... // 可选的 PatternStream patternStream = CEP.pattern(input, pattern, comparator); {% endhighlight %} @@ -1476,27 +1435,25 @@ PatternStream patternStream = CEP.pattern(input, pattern, comparator); {% highlight scala %} val input : DataStream[Event] = ... val pattern : Pattern[Event, _] = ... -var comparator : EventComparator[Event] = ... // optional +var comparator : EventComparator[Event] = ... // 可选的 val patternStream: PatternStream[Event] = CEP.pattern(input, pattern, comparator) {% endhighlight %}
    -The input stream can be *keyed* or *non-keyed* depending on your use-case. +输入流根据你的使用场景可以是*keyed*或者*non-keyed*。 -{% warn Attention %} Applying your pattern on a non-keyed stream will result in a job with parallelism equal to 1. +{% warn 注意 %} 在*non-keyed*流上使用模式将会使你的作业并发度被设为1。 -### Selecting from Patterns +### 从模式中选取 -Once you have obtained a `PatternStream` you can apply transformation to detected event sequences. The suggested way of doing that -is by `PatternProcessFunction`. +在获得到一个`PatternStream`之后,你可以应用各种转换来发现事件序列。推荐使用`PatternProcessFunction`。 -A `PatternProcessFunction` has a `processMatch` method which is called for each matching event sequence. -It receives a match in the form of `Map>` where the key is the name of each pattern in your pattern -sequence and the value is a list of all accepted events for that pattern (`IN` is the type of your input elements). -The events for a given pattern are ordered by timestamp. The reason for returning a list of accepted events for each -pattern is that when using looping patterns (e.g. `oneToMany()` and `times()`), more than one event may be accepted for a given pattern. +`PatternProcessFunction`有一个`processMatch`的方法在每找到一个匹配的事件序列时都会被调用。 +它按照`Map>`的格式接收一个匹配,映射的键是你的模式序列中的每个模式的名称,值是被接受的事件列表(`IN`是输入事件的类型)。 +模式的输入事件按照时间戳进行排序。为每个模式返回一个接受的事件列表的原因是当使用循环模式(比如`oneToMany()`和`times()`)时, +对一个模式会有不止一个事件被接受。 {% highlight java %} class MyPatternProcessFunction extends PatternProcessFunction { @@ -1509,18 +1466,17 @@ class MyPatternProcessFunction extends PatternProcessFunction } {% endhighlight %} -The `PatternProcessFunction` gives access to a `Context` object. Thanks to it, one can access time related -characteristics such as `currentProcessingTime` or `timestamp` of current match (which is the timestamp of the last element assigned to the match). -For more info see [Time context](#time-context). -Through this context one can also emit results to a [side-output]({{ site.baseurl }}/dev/stream/side_output.html). +`PatternProcessFunction`可以访问`Context`对象。有了它之后,你可以访问时间属性,比如`currentProcessingTime`或者当前匹配的`timestamp` +(最新分配到匹配上的事件的时间戳)。 +更多信息可以看[时间上下文](#时间上下文)。 +通过这个上下文也可以将结果输出到[侧输出]({{ site.baseurl }}/zh/dev/stream/side_output.html). -#### Handling Timed Out Partial Patterns +#### 处理超时的部分匹配 -Whenever a pattern has a window length attached via the `within` keyword, it is possible that partial event sequences -are discarded because they exceed the window length. To act upon a timed out partial match one can use `TimedOutPartialMatchHandler` interface. -The interface is supposed to be used in a mixin style. This mean you can additionally implement this interface with your `PatternProcessFunction`. -The `TimedOutPartialMatchHandler` provides the additional `processTimedOutMatch` method which will be called for every timed out partial match. +当一个模式上通过`within`加上窗口长度后,部分匹配的事件序列就可能因为超过窗口长度而被丢弃。可以使用`TimedOutPartialMatchHandler`接口 +来处理超时的部分匹配。这个接口可以和其它的混合使用。也就是说你可以在自己的`PatternProcessFunction`里另外实现这个接口。 +`TimedOutPartialMatchHandler`提供了另外的`processTimedOutMatch`方法,这个方法对每个超时的部分匹配都会调用。 {% highlight java %} class MyPatternProcessFunction extends PatternProcessFunction implements TimedOutPartialMatchHandler { @@ -1537,14 +1493,14 @@ class MyPatternProcessFunction extends PatternProcessFunction } {% endhighlight %} -Note The `processTimedOutMatch` does not give one access to the main output. You can still emit results -through [side-outputs]({{ site.baseurl }}/dev/stream/side_output.html) though, through the `Context` object. +Note `processTimedOutMatch`不能访问主输出。 +但你可以通过`Context`对象把结果输出到[侧输出]({{ site.baseurl }}/zh/dev/stream/side_output.html)。 -#### Convenience API +#### 便捷的API -The aforementioned `PatternProcessFunction` was introduced in Flink 1.8 and since then it is the recommended way to interact with matches. -One can still use the old style API like `select`/`flatSelect`, which internally will be translated into a `PatternProcessFunction`. +前面提到的`PatternProcessFunction`是在Flink 1.8之后引入的,从那之后推荐使用这个接口来处理匹配到的结果。 +用户仍然可以使用像`select`/`flatSelect`这样旧格式的API,它们会在内部被转换为`PatternProcessFunction`。
    @@ -1598,17 +1554,17 @@ val timeoutResult: DataStream[TimeoutEvent] = result.getSideOutput(outputTag)
    -## Time in CEP library +## CEP库中的时间 -### Handling Lateness in Event Time +### 按照事件时间处理迟到事件 -In `CEP` the order in which elements are processed matters. To guarantee that elements are processed in the correct order when working in event time, an incoming element is initially put in a buffer where elements are *sorted in ascending order based on their timestamp*, and when a watermark arrives, all the elements in this buffer with timestamps smaller than that of the watermark are processed. This implies that elements between watermarks are processed in event-time order. +在`CEP`中,事件的处理顺序很重要。在使用事件时间时,为了保证事件按照正确的顺序被处理,一个事件到来后会先被放到一个缓冲区中, +在缓冲区里事件都按照时间戳从小到大排序,当水位线到达后,缓冲区中所有小于水位线的事件被处理。这意味着水位线之间的数据都按照时间戳被顺序处理。 -{% warn Attention %} The library assumes correctness of the watermark when working in event time. +{% warn 注意 %} 这个库假定按照事件时间时水位线一定是正确的。 -To guarantee that elements across watermarks are processed in event-time order, Flink's CEP library assumes -*correctness of the watermark*, and considers as *late* elements whose timestamp is smaller than that of the last -seen watermark. Late elements are not further processed. Also, you can specify a sideOutput tag to collect the late elements come after the last seen watermark, you can use it like this. +为了保证跨水位线的事件按照事件时间处理,Flink CEP库假定*水位线一定是正确的*,并且把时间戳小于最新水位线的事件看作是*晚到*的。 +晚到的事件不会被处理。你也可以指定一个侧输出标志来收集比最新水位线晚到的事件,你可以这样做:
    @@ -1651,44 +1607,42 @@ val lateData: DataStream[String] = result.getSideOutput(lateDataOutputTag)
    -### Time context +### 时间上下文 -In [PatternProcessFunction](#selecting-from-patterns) as well as in [IterativeCondition](#conditions) user has access to a context -that implements `TimeContext` as follows: +在[PatternProcessFunction](#从模式中选取)中,用户可以和[IterativeCondition](#条件)中 +一样按照下面的方法使用实现了`TimeContext`的上下文: {% highlight java %} /** - * Enables access to time related characteristics such as current processing time or timestamp of - * currently processed element. Used in {@link PatternProcessFunction} and - * {@link org.apache.flink.cep.pattern.conditions.IterativeCondition} + * 支持获取事件属性比如当前处理事件或当前正处理的事件的时间。 + * 用在{@link PatternProcessFunction}和{@link org.apache.flink.cep.pattern.conditions.IterativeCondition}中 */ @PublicEvolving public interface TimeContext { /** - * Timestamp of the element currently being processed. + * 当前正处理的事件的时间戳。 * - *

    In case of {@link org.apache.flink.streaming.api.TimeCharacteristic#ProcessingTime} this - * will be set to the time when event entered the cep operator. + *

    如果是{@link org.apache.flink.streaming.api.TimeCharacteristic#ProcessingTime},这个值会被设置为事件进入CEP算子的时间。 */ long timestamp(); - /** Returns the current processing time. */ + /** 返回当前的处理时间。 */ long currentProcessingTime(); } {% endhighlight %} -This context gives user access to time characteristics of processed events (incoming records in case of `IterativeCondition` and matches in case of `PatternProcessFunction`). -Call to `TimeContext#currentProcessingTime` always gives you the value of current processing time and this call should be preferred to e.g. calling `System.currentTimeMillis()`. +这个上下文让用户可以获取处理的事件(在`IterativeCondition`时候是进来的记录,在`PatternProcessFunction`是匹配的结果)的时间属性。 +调用`TimeContext#currentProcessingTime`总是返回当前的处理时间,而且尽量去调用这个函数而不是调用其它的比如说`System.currentTimeMillis()`。 -In case of `TimeContext#timestamp()` the returned value is equal to assigned timestamp in case of `EventTime`. In `ProcessingTime` this will equal to the point of time when said event entered -cep operator (or when the match was generated in case of `PatternProcessFunction`). This means that the value will be consistent across multiple calls to that method. +使用`EventTime`时,`TimeContext#timestamp()`返回的值等于分配的时间戳。 +使用`ProcessingTime`时,这个值等于事件进入CEP算子的时间点(在`PatternProcessFunction`中是匹配产生的时间)。 +这意味着多次调用这个方法得到的值是一致的。 -## Examples +## 例子 -The following example detects the pattern `start, middle(name = "error") -> end(name = "critical")` on a keyed data -stream of `Events`. The events are keyed by their `id`s and a valid pattern has to occur within 10 seconds. -The whole processing is done with event time. +下面的例子在一个分片的`Events`流上检测模式`start, middle(name = "error") -> end(name = "critical")`。 +事件按照`id`分片,一个有效的模式需要发生在10秒内。

    @@ -1750,31 +1704,25 @@ val alerts = patternStream.select(createAlert(_))
    -## Migrating from an older Flink version(pre 1.3) +## 从旧版本迁移(1.3之前) -### Migrating to 1.4+ +### 迁移到1.4+ -In Flink-1.4 the backward compatibility of CEP library with <= Flink 1.2 was dropped. Unfortunately -it is not possible to restore a CEP job that was once run with 1.2.x +在Flink-1.4放弃了和<= Flink 1.2版本的兼容性。很不幸,不能再恢复用1.2.x运行过的CEP作业。 -### Migrating to 1.3.x +### 迁移到1.3.x -The CEP library in Flink-1.3 ships with a number of new features which have led to some changes in the API. Here we -describe the changes that you need to make to your old CEP jobs, in order to be able to run them with Flink-1.3. After -making these changes and recompiling your job, you will be able to resume its execution from a savepoint taken with the -old version of your job, *i.e.* without having to re-process your past data. +CEP库在Flink-1.3发布的一系列的新特性引入了一些API上的修改。这里我们描述你需要对旧的CEP作业所做的修改,以能够用Flink-1.3来运行它们。 +在做完这些修改并重新编译你的作业之后,可以从旧版本作业的保存点之后继续运行,*也就是说*不需要再重新处理旧的数据。 -The changes required are: +需要的修改是: -1. Change your conditions (the ones in the `where(...)` clause) to extend the `SimpleCondition` class instead of -implementing the `FilterFunction` interface. +1. 修改你的条件(在`where(...)`语句中的)来继承`SimpleCondition`类而不是实现`FilterFunction`接口。 -2. Change your functions provided as arguments to the `select(...)` and `flatSelect(...)` methods to expect a list of -events associated with each pattern (`List` in `Java`, `Iterable` in `Scala`). This is because with the addition of -the looping patterns, multiple input events can match a single (looping) pattern. +2. 修改你作为`select(...)`和`flatSelect(...)`方法的参数的函数为期望每个模式关联一个事件列表(`Java`中`List`,`Scala`中`Iterable`)。 +这是因为增加了循环模式后,多个事件可能匹配一个单一的(循环)模式。 -3. The `followedBy()` in Flink 1.1 and 1.2 implied `non-deterministic relaxed contiguity` (see -[here](#conditions-on-contiguity)). In Flink 1.3 this has changed and `followedBy()` implies `relaxed contiguity`, -while `followedByAny()` should be used if `non-deterministic relaxed contiguity` is required. +3. 在Flink 1.1和1.2中,`followedBy()`隐含了`不确定的松散连续` (参见[这里](#组合模式))。 +在Flink 1.3中,这里发生了变化, `followedBy()`隐含了`松散连续`,如果需要`不确定的松散连续`,应该使用`followedByAny()`。 {% top %} -- Gitee From 2b13a4155fd4284f6092decba867e71eea058043 Mon Sep 17 00:00:00 2001 From: godfrey he Date: Wed, 4 Mar 2020 15:52:37 +0800 Subject: [PATCH 036/885] [FLINK-16362][table] Remove deprecated `emitDataStream` method in StreamTableSink This closes #11279 --- docs/dev/table/sourceSinks.md | 12 ++++++------ docs/dev/table/sourceSinks.zh.md | 12 ++++++------ .../cassandra/CassandraAppendTableSink.java | 4 ---- .../ElasticsearchUpsertTableSinkBase.java | 5 ----- ...lasticsearch6UpsertTableSinkFactoryTest.java | 2 +- ...lasticsearch7UpsertTableSinkFactoryTest.java | 2 +- .../connectors/kafka/KafkaTableSinkBase.java | 5 ----- .../KafkaTableSourceSinkFactoryTestBase.java | 4 ++-- .../addons/hbase/HBaseUpsertTableSink.java | 5 ----- .../api/java/io/jdbc/JDBCAppendTableSink.java | 5 ----- .../api/java/io/jdbc/JDBCUpsertTableSink.java | 5 ----- .../java/io/jdbc/JDBCAppendTableSinkTest.java | 2 +- .../gateway/local/CollectStreamTableSink.java | 5 ----- .../gateway/utils/TestTableSinkFactoryBase.java | 5 +++-- .../apache/flink/table/sinks/CsvTableSink.java | 5 ----- .../table/sinks/OutputFormatTableSink.java | 5 ----- .../flink/table/sinks/StreamTableSink.java | 17 +---------------- .../org/apache/flink/table/api/TableUtils.java | 5 ----- .../table/planner/sinks/CollectTableSink.scala | 4 ---- .../utils/TestCollectionTableFactory.scala | 4 ---- .../batch/sql/PartitionableSinkITCase.scala | 5 ----- .../planner/runtime/utils/StreamTestSink.scala | 12 ------------ .../utils/MemoryTableSourceSinkUtil.scala | 5 ----- .../utils/TestCollectionTableFactory.scala | 4 ++-- .../runtime/stream/table/TableSinkITCase.scala | 15 ++++++--------- .../table/utils/MemoryTableSourceSinkUtil.scala | 4 ++-- .../common/table/SpendReportTableSink.java | 6 ++++-- 27 files changed, 35 insertions(+), 129 deletions(-) diff --git a/docs/dev/table/sourceSinks.md b/docs/dev/table/sourceSinks.md index dcd6829d5b..7b3647f067 100644 --- a/docs/dev/table/sourceSinks.md +++ b/docs/dev/table/sourceSinks.md @@ -452,7 +452,7 @@ The interface looks as follows: {% highlight java %} AppendStreamTableSink implements TableSink { - public void emitDataStream(DataStream dataStream); + public DataStreamSink consumeDataStream(DataStream dataStream); } {% endhighlight %} @@ -461,7 +461,7 @@ AppendStreamTableSink implements TableSink { {% highlight scala %} AppendStreamTableSink[T] extends TableSink[T] { - def emitDataStream(dataStream: DataStream[T]): Unit + def consumeDataStream(dataStream: DataStream[T]): DataStreamSink[_] } {% endhighlight %} @@ -484,7 +484,7 @@ RetractStreamTableSink implements TableSink> { public TypeInformation getRecordType(); - public void emitDataStream(DataStream> dataStream); + public DataStreamSink consumeDataStream(DataStream> dataStream); } {% endhighlight %} @@ -495,7 +495,7 @@ RetractStreamTableSink[T] extends TableSink[Tuple2[Boolean, T]] { def getRecordType: TypeInformation[T] - def emitDataStream(dataStream: DataStream[Tuple2[Boolean, T]]): Unit + def consumeDataStream(dataStream: DataStream[Tuple2[Boolean, T]]): DataStreamSink[_] } {% endhighlight %} @@ -522,7 +522,7 @@ UpsertStreamTableSink implements TableSink> { public TypeInformation getRecordType(); - public void emitDataStream(DataStream> dataStream); + public DataStreamSink consumeDataStream(DataStream> dataStream); } {% endhighlight %} @@ -537,7 +537,7 @@ UpsertStreamTableSink[T] extends TableSink[Tuple2[Boolean, T]] { def getRecordType: TypeInformation[T] - def emitDataStream(dataStream: DataStream[Tuple2[Boolean, T]]): Unit + def consumeDataStream(dataStream: DataStream[Tuple2[Boolean, T]]): DataStreamSink[_] } {% endhighlight %} diff --git a/docs/dev/table/sourceSinks.zh.md b/docs/dev/table/sourceSinks.zh.md index fab6930dbe..e9f2c77595 100644 --- a/docs/dev/table/sourceSinks.zh.md +++ b/docs/dev/table/sourceSinks.zh.md @@ -452,7 +452,7 @@ The interface looks as follows: {% highlight java %} AppendStreamTableSink implements TableSink { - public void emitDataStream(DataStream dataStream); + public DataStreamSink consumeDataStream(DataStream dataStream); } {% endhighlight %} @@ -461,7 +461,7 @@ AppendStreamTableSink implements TableSink { {% highlight scala %} AppendStreamTableSink[T] extends TableSink[T] { - def emitDataStream(dataStream: DataStream[T]): Unit + def consumeDataStream(dataStream: DataStream[T]): DataStreamSink[_] } {% endhighlight %} @@ -484,7 +484,7 @@ RetractStreamTableSink implements TableSink> { public TypeInformation getRecordType(); - public void emitDataStream(DataStream> dataStream); + public DataStreamSink consumeDataStream(DataStream> dataStream); } {% endhighlight %} @@ -495,7 +495,7 @@ RetractStreamTableSink[T] extends TableSink[Tuple2[Boolean, T]] { def getRecordType: TypeInformation[T] - def emitDataStream(dataStream: DataStream[Tuple2[Boolean, T]]): Unit + def consumeDataStream(dataStream: DataStream[Tuple2[Boolean, T]]): DataStreamSink[_] } {% endhighlight %} @@ -522,7 +522,7 @@ UpsertStreamTableSink implements TableSink> { public TypeInformation getRecordType(); - public void emitDataStream(DataStream> dataStream); + public DataStreamSink consumeDataStream(DataStream> dataStream); } {% endhighlight %} @@ -537,7 +537,7 @@ UpsertStreamTableSink[T] extends TableSink[Tuple2[Boolean, T]] { def getRecordType: TypeInformation[T] - def emitDataStream(dataStream: DataStream[Tuple2[Boolean, T]]): Unit + def consumeDataStream(dataStream: DataStream[Tuple2[Boolean, T]]): DataStreamSink[_] } {% endhighlight %} diff --git a/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraAppendTableSink.java b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraAppendTableSink.java index 72afc3bc49..af152accca 100644 --- a/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraAppendTableSink.java +++ b/flink-connectors/flink-connector-cassandra/src/main/java/org/apache/flink/streaming/connectors/cassandra/CassandraAppendTableSink.java @@ -97,8 +97,4 @@ public class CassandraAppendTableSink implements AppendStreamTableSink { } - @Override - public void emitDataStream(DataStream dataStream) { - consumeDataStream(dataStream); - } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java index 3c1346f720..7ee5d92401 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java @@ -191,11 +191,6 @@ public abstract class ElasticsearchUpsertTableSinkBase implements UpsertStreamTa .name(TableConnectorUtils.generateRuntimeName(this.getClass(), getFieldNames())); } - @Override - public void emitDataStream(DataStream> dataStream) { - consumeDataStream(dataStream); - } - @Override public TypeInformation> getOutputType() { return Types.TUPLE(Types.BOOLEAN, getRecordType()); diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java index 257634d173..021bbdd08a 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java @@ -79,7 +79,7 @@ public class Elasticsearch6UpsertTableSinkFactoryTest extends ElasticsearchUpser new StreamExecutionEnvironmentMock(), Types.TUPLE(Types.BOOLEAN, schema.toRowType())); - testSink.emitDataStream(dataStreamMock); + testSink.consumeDataStream(dataStreamMock); final ElasticsearchSink.Builder> expectedBuilder = new ElasticsearchSink.Builder<>( Collections.singletonList(new HttpHost(HOSTNAME, PORT, SCHEMA)), diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java index 5837733b85..f3493b34e1 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java @@ -78,7 +78,7 @@ public class Elasticsearch7UpsertTableSinkFactoryTest extends ElasticsearchUpser new StreamExecutionEnvironmentMock(), Types.TUPLE(Types.BOOLEAN, schema.toRowType())); - testSink.emitDataStream(dataStreamMock); + testSink.consumeDataStream(dataStreamMock); final ElasticsearchSink.Builder> expectedBuilder = new ElasticsearchSink.Builder<>( Collections.singletonList(new HttpHost(ElasticsearchUpsertTableSinkFactoryTestBase.HOSTNAME, ElasticsearchUpsertTableSinkFactoryTestBase.PORT, ElasticsearchUpsertTableSinkFactoryTestBase.SCHEMA)), diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkBase.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkBase.java index 3cf2fe0713..bf3471b783 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkBase.java @@ -103,11 +103,6 @@ public abstract class KafkaTableSinkBase implements AppendStreamTableSink { .name(TableConnectorUtils.generateRuntimeName(this.getClass(), getFieldNames())); } - @Override - public void emitDataStream(DataStream dataStream) { - consumeDataStream(dataStream); - } - @Override public TypeInformation getOutputType() { return schema.toRowType(); diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceSinkFactoryTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceSinkFactoryTestBase.java index a313956986..0875c28d4e 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceSinkFactoryTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceSinkFactoryTestBase.java @@ -304,7 +304,7 @@ public abstract class KafkaTableSourceSinkFactoryTestBase extends TestLogger { // test Kafka producer final KafkaTableSinkBase actualKafkaSink = (KafkaTableSinkBase) actualSink; final DataStreamMock streamMock = new DataStreamMock(new StreamExecutionEnvironmentMock(), schema.toRowType()); - actualKafkaSink.emitDataStream(streamMock); + actualKafkaSink.consumeDataStream(streamMock); assertTrue(getExpectedFlinkKafkaProducer().isAssignableFrom(streamMock.sinkFunction.getClass())); } @@ -357,7 +357,7 @@ public abstract class KafkaTableSourceSinkFactoryTestBase extends TestLogger { // test Kafka producer final KafkaTableSinkBase actualKafkaSink = (KafkaTableSinkBase) actualSink; final DataStreamMock streamMock = new DataStreamMock(new StreamExecutionEnvironmentMock(), schema.toRowType()); - actualKafkaSink.emitDataStream(streamMock); + actualKafkaSink.consumeDataStream(streamMock); assertTrue(getExpectedFlinkKafkaProducer().isAssignableFrom(streamMock.sinkFunction.getClass())); } diff --git a/flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseUpsertTableSink.java b/flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseUpsertTableSink.java index 1dc7f97f2c..bbdf93336c 100644 --- a/flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseUpsertTableSink.java +++ b/flink-connectors/flink-hbase/src/main/java/org/apache/flink/addons/hbase/HBaseUpsertTableSink.java @@ -100,11 +100,6 @@ public class HBaseUpsertTableSink implements UpsertStreamTableSink { .name(TableConnectorUtils.generateRuntimeName(this.getClass(), tableSchema.getFieldNames())); } - @Override - public void emitDataStream(DataStream> dataStream) { - consumeDataStream(dataStream); - } - @Override public TableSink> configure(String[] fieldNames, TypeInformation[] fieldTypes) { if (!Arrays.equals(getFieldNames(), fieldNames) || !Arrays.equals(getFieldTypes(), fieldTypes)) { diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSink.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSink.java index 7c7e7ba56e..b5bc34cc47 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSink.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSink.java @@ -68,11 +68,6 @@ public class JDBCAppendTableSink implements AppendStreamTableSink, BatchTab .name(TableConnectorUtils.generateRuntimeName(this.getClass(), fieldNames)); } - @Override - public void emitDataStream(DataStream dataStream) { - consumeDataStream(dataStream); - } - @Override public void emitDataSet(DataSet dataSet) { dataSet.output(outputFormat); diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSink.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSink.java index 42e4a384f1..77dac7b400 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSink.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSink.java @@ -97,11 +97,6 @@ public class JDBCUpsertTableSink implements UpsertStreamTableSink { .name(TableConnectorUtils.generateRuntimeName(this.getClass(), schema.getFieldNames())); } - @Override - public void emitDataStream(DataStream> dataStream) { - consumeDataStream(dataStream); - } - @Override public void setKeyFields(String[] keys) { this.keyFields = keys; diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSinkTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSinkTest.java index c35dd395e1..b3f2b249f5 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSinkTest.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSinkTest.java @@ -59,7 +59,7 @@ public class JDBCAppendTableSinkTest { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataStream ds = env.fromCollection(Collections.singleton(Row.of("foo")), ROW_TYPE); - sink.emitDataStream(ds); + sink.consumeDataStream(ds); Collection sinkIds = env .getStreamGraph(StreamExecutionEnvironment.DEFAULT_JOB_NAME, false) diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/CollectStreamTableSink.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/CollectStreamTableSink.java index 00f0cbed9f..62cd39873e 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/CollectStreamTableSink.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/CollectStreamTableSink.java @@ -76,11 +76,6 @@ public class CollectStreamTableSink implements RetractStreamTableSink { return getTableSchema().toRowType(); } - @Override - public void emitDataStream(DataStream> stream) { - consumeDataStream(stream); - } - @Override public DataStreamSink consumeDataStream(DataStream> stream) { // add sink diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSinkFactoryBase.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSinkFactoryBase.java index e934959e40..bc201c13aa 100644 --- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSinkFactoryBase.java +++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSinkFactoryBase.java @@ -20,6 +20,7 @@ package org.apache.flink.table.client.gateway.utils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.Types; import org.apache.flink.table.factories.StreamTableSinkFactory; @@ -138,8 +139,8 @@ public abstract class TestTableSinkFactoryBase implements StreamTableSinkFactory } @Override - public void emitDataStream(DataStream dataStream) { - // do nothing + public DataStreamSink consumeDataStream(DataStream dataStream) { + return null; } } } diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSink.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSink.java index 61b7ad32f7..dfe5e8763f 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSink.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSink.java @@ -157,11 +157,6 @@ public class CsvTableSink implements BatchTableSink, AppendStreamTableSink< return sink; } - @Override - public void emitDataStream(DataStream dataStream) { - consumeDataStream(dataStream); - } - @Override public TableSink configure(String[] fieldNames, TypeInformation[] fieldTypes) { if (this.fieldNames != null || this.fieldTypes != null) { diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/OutputFormatTableSink.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/OutputFormatTableSink.java index 692082fe9e..e650859883 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/OutputFormatTableSink.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/OutputFormatTableSink.java @@ -37,11 +37,6 @@ public abstract class OutputFormatTableSink implements StreamTableSink { */ public abstract OutputFormat getOutputFormat(); - @Override - public final void emitDataStream(DataStream dataStream) { - consumeDataStream(dataStream); - } - @Override public final DataStreamSink consumeDataStream(DataStream dataStream) { return dataStream diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/StreamTableSink.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/StreamTableSink.java index 825d5bbbab..09b72f189b 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/StreamTableSink.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/StreamTableSink.java @@ -28,24 +28,9 @@ import org.apache.flink.streaming.api.datastream.DataStreamSink; */ public interface StreamTableSink extends TableSink { - /** - * Emits the DataStream. - * - * @deprecated This method will be removed in future versions as it returns nothing. - * It is recommended to use {@link #consumeDataStream(DataStream)} instead which - * returns the {@link DataStreamSink}. The returned {@link DataStreamSink} will be - * used to set resources for the sink operator. If the {@link #consumeDataStream(DataStream)} - * is implemented, this method can be empty implementation. - */ - @Deprecated - void emitDataStream(DataStream dataStream); - /** * Consumes the DataStream and return the sink transformation {@link DataStreamSink}. * The returned {@link DataStreamSink} will be used to set resources for the sink operator. */ - default DataStreamSink consumeDataStream(DataStream dataStream) { - emitDataStream(dataStream); - return null; - } + DataStreamSink consumeDataStream(DataStream dataStream); } diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/api/TableUtils.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/api/TableUtils.java index a06cb199b5..8aaf5c519c 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/api/TableUtils.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/api/TableUtils.java @@ -153,11 +153,6 @@ public class TableUtils { "This sink is configured by passing a static schema when initiating"); } - @Override - public void emitDataStream(DataStream dataStream) { - throw new UnsupportedOperationException("Deprecated method, use consumeDataStream instead"); - } - @Override public DataStreamSink consumeDataStream(DataStream dataStream) { return dataStream.writeUsingOutputFormat(outputFormat).setParallelism(1).name("tableResult"); diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sinks/CollectTableSink.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sinks/CollectTableSink.scala index 7535baac0e..a4c92280cc 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sinks/CollectTableSink.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sinks/CollectTableSink.scala @@ -42,10 +42,6 @@ class CollectTableSink[T](produceOutputType: (Array[TypeInformation[_]] => TypeI .name("collect") } - override def emitDataStream(dataStream: DataStream[T]): Unit = { - consumeDataStream(dataStream) - } - override protected def copy: TableSinkBase[T] = { new CollectTableSink(produceOutputType) } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/factories/utils/TestCollectionTableFactory.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/factories/utils/TestCollectionTableFactory.scala index 170f1188f6..228ea3c5f6 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/factories/utils/TestCollectionTableFactory.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/factories/utils/TestCollectionTableFactory.scala @@ -173,10 +173,6 @@ object TestCollectionTableFactory { override def getTableSchema: TableSchema = schema - override def emitDataStream(dataStream: DataStream[Row]): Unit = { - dataStream.addSink(new UnsafeMemorySinkFunction(schema.toRowType)).setParallelism(1) - } - override def consumeDataStream(dataStream: DataStream[Row]): DataStreamSink[_] = { dataStream.addSink(new UnsafeMemorySinkFunction(schema.toRowType)).setParallelism(1) } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala index 12579dc8e0..add6f2ac67 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala @@ -306,11 +306,6 @@ private class TestSink( override def getOutputType: RowTypeInfo = rowType - override def emitDataStream(dataStream: DataStream[Row]): Unit = { - dataStream.addSink(new UnsafeMemorySinkFunction(rowType)) - .setParallelism(dataStream.getParallelism) - } - override def consumeDataStream(dataStream: DataStream[Row]): DataStreamSink[_] = { dataStream.addSink(new UnsafeMemorySinkFunction(rowType)) .setParallelism(dataStream.getParallelism) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamTestSink.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamTestSink.scala index ac54ecab71..24fb71b85e 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamTestSink.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamTestSink.scala @@ -310,10 +310,6 @@ final class TestingUpsertTableSink(val keys: Array[Int], val tz: TimeZone) .setParallelism(dataStream.getParallelism) } - override def emitDataStream(dataStream: DataStream[JTuple2[JBoolean, BaseRow]]): Unit = { - consumeDataStream(dataStream) - } - override def configure( fieldNames: Array[String], fieldTypes: Array[TypeInformation[_]]): TestingUpsertTableSink = { @@ -345,10 +341,6 @@ final class TestingAppendTableSink(tz: TimeZone) extends AppendStreamTableSink[R .setParallelism(dataStream.getParallelism) } - override def emitDataStream(dataStream: DataStream[Row]): Unit = { - consumeDataStream(dataStream) - } - override def getOutputType: TypeInformation[Row] = new RowTypeInfo(fTypes, fNames) override def configure( @@ -511,10 +503,6 @@ final class TestingRetractTableSink(tz: TimeZone) extends RetractStreamTableSink .setParallelism(dataStream.getParallelism) } - override def emitDataStream(dataStream: DataStream[JTuple2[JBoolean, Row]]): Unit = { - consumeDataStream(dataStream) - } - override def getRecordType: TypeInformation[Row] = new RowTypeInfo(fTypes, fNames) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/MemoryTableSourceSinkUtil.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/MemoryTableSourceSinkUtil.scala index 0d263f6f9c..062a7dab81 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/MemoryTableSourceSinkUtil.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/MemoryTableSourceSinkUtil.scala @@ -110,10 +110,6 @@ object MemoryTableSourceSinkUtil { .setParallelism(dataStream.getParallelism) .name(TableConnectorUtil.generateRuntimeName(this.getClass, getFieldNames)) } - - override def emitDataStream(dataStream: DataStream[Row]): Unit = { - consumeDataStream(dataStream) - } } final class UnsafeMemoryOutputFormatTableSink extends OutputFormatTableSink[Row] { @@ -192,6 +188,5 @@ object MemoryTableSourceSinkUtil { dataStream.writeUsingOutputFormat(new MemoryCollectionOutputFormat) } - override def emitDataStream(dataStream: DataStream[Row]): Unit = ??? } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/factories/utils/TestCollectionTableFactory.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/factories/utils/TestCollectionTableFactory.scala index 3b17986202..bd31d46419 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/factories/utils/TestCollectionTableFactory.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/factories/utils/TestCollectionTableFactory.scala @@ -25,7 +25,7 @@ import org.apache.flink.api.java.io.{CollectionInputFormat, LocalCollectionOutpu import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.api.java.{DataSet, ExecutionEnvironment} import org.apache.flink.configuration.Configuration -import org.apache.flink.streaming.api.datastream.{DataStream, DataStreamSource} +import org.apache.flink.streaming.api.datastream.{DataStream, DataStreamSink, DataStreamSource} import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment import org.apache.flink.streaming.api.functions.sink.RichSinkFunction import org.apache.flink.table.api.TableSchema @@ -188,7 +188,7 @@ object TestCollectionTableFactory { outputType.getFieldTypes } - override def emitDataStream(dataStream: DataStream[Row]): Unit = { + override def consumeDataStream(dataStream: DataStream[Row]): DataStreamSink[_] = { dataStream.addSink(new UnsafeMemorySinkFunction(outputType)).setParallelism(1) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSinkITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSinkITCase.scala index e6f671d9f8..3e6b6375f5 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSinkITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSinkITCase.scala @@ -18,9 +18,6 @@ package org.apache.flink.table.runtime.stream.table -import java.io.File -import java.lang.{Boolean => JBool} - import org.apache.flink.api.common.functions.MapFunction import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2} @@ -39,9 +36,13 @@ import org.apache.flink.table.utils.MemoryTableSourceSinkUtil import org.apache.flink.test.util.{AbstractTestBase, TestBaseUtils} import org.apache.flink.types.Row import org.apache.flink.util.Collector + import org.junit.Assert._ import org.junit.Test +import java.io.File +import java.lang.{Boolean => JBool} + import scala.collection.JavaConverters._ import scala.collection.mutable @@ -623,10 +624,6 @@ private[flink] class TestAppendSink extends AppendStreamTableSink[Row] { var fNames: Array[String] = _ var fTypes: Array[TypeInformation[_]] = _ - override def emitDataStream(s: DataStream[Row]): Unit = { - consumeDataStream(s) - } - override def consumeDataStream(dataStream: DataStream[Row]): DataStreamSink[_] = { dataStream.map( new MapFunction[Row, JTuple2[JBool, Row]] { @@ -656,7 +653,7 @@ private[flink] class TestRetractSink extends RetractStreamTableSink[Row] { var fNames: Array[String] = _ var fTypes: Array[TypeInformation[_]] = _ - override def emitDataStream(s: DataStream[JTuple2[JBool, Row]]): Unit = { + override def consumeDataStream(s: DataStream[JTuple2[JBool, Row]]): DataStreamSink[_] = { s.addSink(new RowSink) } @@ -703,7 +700,7 @@ private[flink] class TestUpsertSink( override def getRecordType: TypeInformation[Row] = new RowTypeInfo(fTypes, fNames) - override def emitDataStream(s: DataStream[JTuple2[JBool, Row]]): Unit = { + override def consumeDataStream(s: DataStream[JTuple2[JBool, Row]]): DataStreamSink[_] = { s.addSink(new RowSink) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MemoryTableSourceSinkUtil.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MemoryTableSourceSinkUtil.scala index 5559c629ef..5b2e9d2e36 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MemoryTableSourceSinkUtil.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MemoryTableSourceSinkUtil.scala @@ -24,7 +24,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.api.java.{DataSet, ExecutionEnvironment} import org.apache.flink.configuration.Configuration -import org.apache.flink.streaming.api.datastream.DataStream +import org.apache.flink.streaming.api.datastream.{DataStream, DataStreamSink} import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment import org.apache.flink.streaming.api.functions.sink.RichSinkFunction import org.apache.flink.streaming.api.functions.source.SourceFunction @@ -116,7 +116,7 @@ object MemoryTableSourceSinkUtil { .name(TableConnectorUtils.generateRuntimeName(this.getClass, getTableSchema.getFieldNames)) } - override def emitDataStream(dataStream: DataStream[Row]): Unit = { + override def consumeDataStream(dataStream: DataStream[Row]): DataStreamSink[_] = { val inputParallelism = dataStream.getParallelism dataStream .addSink(new MemoryAppendSink) diff --git a/flink-walkthroughs/flink-walkthrough-common/src/main/java/org/apache/flink/walkthrough/common/table/SpendReportTableSink.java b/flink-walkthroughs/flink-walkthrough-common/src/main/java/org/apache/flink/walkthrough/common/table/SpendReportTableSink.java index ae167fd6c2..a143306d07 100644 --- a/flink-walkthroughs/flink-walkthrough-common/src/main/java/org/apache/flink/walkthrough/common/table/SpendReportTableSink.java +++ b/flink-walkthroughs/flink-walkthrough-common/src/main/java/org/apache/flink/walkthrough/common/table/SpendReportTableSink.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.DataSet; import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.sinks.AppendStreamTableSink; import org.apache.flink.table.sinks.BatchTableSink; @@ -57,8 +58,8 @@ public class SpendReportTableSink implements AppendStreamTableSink, BatchTa } @Override - public void emitDataStream(DataStream dataStream) { - dataStream + public DataStreamSink consumeDataStream(DataStream dataStream) { + return dataStream .map(SpendReportTableSink::format) .writeUsingOutputFormat(new LoggerOutputFormat()) .setParallelism(dataStream.getParallelism()); @@ -93,4 +94,5 @@ public class SpendReportTableSink implements AppendStreamTableSink, BatchTa //noinspection MalformedFormatString return String.format("%s, %s, $%.2f", row.getField(0), row.getField(1), row.getField(2)); } + } -- Gitee From 3cf8fe5177e21443f15ad030bd66e71d74496452 Mon Sep 17 00:00:00 2001 From: Aitozi <1059789585@qq.com> Date: Wed, 4 Sep 2019 22:59:09 +0800 Subject: [PATCH 037/885] [FLINK-11193][State] Fix Rockdb timer service factory configuration option to be settable per job This closes #8479. --- .../streaming/state/RocksDBStateBackend.java | 32 +++++++++++++----- .../state/RocksDBStateBackendConfigTest.java | 33 +++++++++++++++++++ 2 files changed, 57 insertions(+), 8 deletions(-) diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java index f0711048de..873f57d697 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java @@ -153,7 +153,8 @@ public class RocksDBStateBackend extends AbstractStateBackend implements Configu private final RocksDBMemoryConfiguration memoryConfiguration; /** This determines the type of priority queue state. */ - private final PriorityQueueStateType priorityQueueStateType; + @Nullable + private PriorityQueueStateType priorityQueueStateType; /** The default rocksdb metrics options. */ private final RocksDBNativeMetricOptions defaultMetricOptions; @@ -274,8 +275,6 @@ public class RocksDBStateBackend extends AbstractStateBackend implements Configu this.checkpointStreamBackend = checkNotNull(checkpointStreamBackend); this.enableIncrementalCheckpointing = enableIncrementalCheckpointing; this.numberOfTransferThreads = UNDEFINED_NUMBER_OF_TRANSFER_THREADS; - // use RocksDB-based implementation as default from FLINK-15637 - this.priorityQueueStateType = PriorityQueueStateType.ROCKSDB; this.defaultMetricOptions = new RocksDBNativeMetricOptions(); this.enableTtlCompactionFilter = TernaryBoolean.UNDEFINED; this.memoryConfiguration = new RocksDBMemoryConfiguration(); @@ -333,10 +332,11 @@ public class RocksDBStateBackend extends AbstractStateBackend implements Configu this.memoryConfiguration = RocksDBMemoryConfiguration.fromOtherAndConfiguration(original.memoryConfiguration, config); this.memoryConfiguration.validate(); - final String priorityQueueTypeString = config.getString(TIMER_SERVICE_FACTORY); - - this.priorityQueueStateType = priorityQueueTypeString.length() > 0 ? - PriorityQueueStateType.valueOf(priorityQueueTypeString.toUpperCase()) : original.priorityQueueStateType; + if (null == original.priorityQueueStateType) { + this.priorityQueueStateType = config.getEnum(PriorityQueueStateType.class, TIMER_SERVICE_FACTORY); + } else { + this.priorityQueueStateType = original.priorityQueueStateType; + } // configure local directories if (original.localRocksDbDirectories != null) { @@ -533,7 +533,7 @@ public class RocksDBStateBackend extends AbstractStateBackend implements Configu keyGroupRange, executionConfig, localRecoveryConfig, - priorityQueueStateType, + getPriorityQueueStateType(), ttlTimeProvider, metricGroup, stateHandles, @@ -770,6 +770,22 @@ public class RocksDBStateBackend extends AbstractStateBackend implements Configu enableTtlCompactionFilter = TernaryBoolean.FALSE; } + /** + * Gets the type of the priority queue state. It will fallback to the default value, if it is not explicitly set. + * @return The type of the priority queue state. + */ + public PriorityQueueStateType getPriorityQueueStateType() { + return priorityQueueStateType == null ? + PriorityQueueStateType.valueOf(TIMER_SERVICE_FACTORY.defaultValue()) : priorityQueueStateType; + } + + /** + * Sets the type of the priority queue state. It will fallback to the default value, if it is not explicitly set. + */ + public void setPriorityQueueStateType(PriorityQueueStateType priorityQueueStateType) { + this.priorityQueueStateType = checkNotNull(priorityQueueStateType); + } + // ------------------------------------------------------------------------ // Parametrize with RocksDB Options // ------------------------------------------------------------------------ diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java index e85e96b5db..2167a000d1 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java @@ -31,6 +31,7 @@ import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.operators.testutils.MockEnvironment; +import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; import org.apache.flink.runtime.query.KvStateRegistry; import org.apache.flink.runtime.state.AbstractKeyedStateBackend; import org.apache.flink.runtime.state.KeyGroupRange; @@ -59,6 +60,7 @@ import java.util.Collection; import java.util.Collections; import static org.hamcrest.CoreMatchers.anyOf; +import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.startsWith; import static org.junit.Assert.assertArrayEquals; @@ -174,6 +176,37 @@ public class RocksDBStateBackendConfigTest { env.close(); } + /** + * Validates that user custom configuration from code should override the flink-conf.yaml. + */ + @Test + public void testConfigureTimerServiceLoadingFromApplication() throws Exception { + final MockEnvironment env = new MockEnvironmentBuilder().build(); + + // priorityQueueStateType of the job backend + final RocksDBStateBackend backend = new RocksDBStateBackend(tempFolder.newFolder().toURI().toString()); + backend.setPriorityQueueStateType(RocksDBStateBackend.PriorityQueueStateType.HEAP); + + // priorityQueueStateType in the cluster config + final Configuration configFromConfFile = new Configuration(); + configFromConfFile.setString( + RocksDBOptions.TIMER_SERVICE_FACTORY, + RocksDBStateBackend.PriorityQueueStateType.ROCKSDB.toString()); + + // configure final backend from job and cluster config + final RocksDBStateBackend configuredRocksDBStateBackend = backend.configure( + configFromConfFile, + Thread.currentThread().getContextClassLoader()); + final RocksDBKeyedStateBackend keyedBackend = createKeyedStateBackend(configuredRocksDBStateBackend, env); + + // priorityQueueStateType of the job backend should be preserved + assertThat(keyedBackend.getPriorityQueueFactory(), instanceOf(HeapPriorityQueueSetFactory.class)); + + keyedBackend.close(); + keyedBackend.dispose(); + env.close(); + } + @Test public void testStoragePathWithFilePrefix() throws Exception { final File folder = tempFolder.newFolder(); -- Gitee From 095da703bb91b82ca281d61896c396614ceebcfd Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 4 Mar 2020 11:14:33 +0100 Subject: [PATCH 038/885] [hotfix] Fix chinese doc links to make master documentation build again --- docs/concepts/index.zh.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/concepts/index.zh.md b/docs/concepts/index.zh.md index 2498a434ab..cbed26ee3d 100644 --- a/docs/concepts/index.zh.md +++ b/docs/concepts/index.zh.md @@ -33,17 +33,17 @@ Flink offers different levels of abstraction for developing streaming/batch appl - The lowest level abstraction simply offers **stateful streaming**. It is embedded into the [DataStream API]({{ site.baseurl}}{% link - dev/datastream_api.md %}) via the [Process Function]({{ site.baseurl }}{% - link dev/stream/operators/process_function.md %}). It allows users freely + dev/datastream_api.zh.md %}) via the [Process Function]({{ site.baseurl }}{% + link dev/stream/operators/process_function.zh.md %}). It allows users freely process events from one or more streams, and use consistent fault tolerant *state*. In addition, users can register event time and processing time callbacks, allowing programs to realize sophisticated computations. - In practice, most applications would not need the above described low level abstraction, but would instead program against the **Core APIs** like the - [DataStream API]({{ site.baseurl }}{% link dev/datastream_api.md %}) + [DataStream API]({{ site.baseurl }}{% link dev/datastream_api.zh.md %}) (bounded/unbounded streams) and the [DataSet API]({{ site.baseurl }}{% link - dev/batch/index.md %}) (bounded data sets). These fluent APIs offer the + dev/batch/index.zh.md %}) (bounded data sets). These fluent APIs offer the common building blocks for data processing, like various forms of user-specified transformations, joins, aggregations, windows, state, etc. Data types processed in these APIs are represented as classes in the @@ -56,7 +56,7 @@ Flink offers different levels of abstraction for developing streaming/batch appl - The **Table API** is a declarative DSL centered around *tables*, which may be dynamically changing tables (when representing streams). The [Table - API]({{ site.baseurl }}{% link dev/table/index.md %}) follows the + API]({{ site.baseurl }}{% link dev/table/index.zh.md %}) follows the (extended) relational model: Tables have a schema attached (similar to tables in relational databases) and the API offers comparable operations, such as select, project, join, group-by, aggregate, etc. Table API @@ -74,7 +74,7 @@ Flink offers different levels of abstraction for developing streaming/batch appl - The highest level abstraction offered by Flink is **SQL**. This abstraction is similar to the *Table API* both in semantics and expressiveness, but represents programs as SQL query expressions. The [SQL]({{ site.baseurl - }}{% link dev/table/index.md %}#sql) abstraction closely interacts with the + }}{% link dev/table/index.zh.md %}#sql) abstraction closely interacts with the Table API, and SQL queries can be executed over tables defined in the *Table API*. -- Gitee From cfe1f9e2bd2fb9f315c4d93480340052791fa16d Mon Sep 17 00:00:00 2001 From: Paul Lam Date: Sun, 1 Mar 2020 18:42:28 +0800 Subject: [PATCH 039/885] [hotfix][scaladoc] Add missing parameter in DataStreamConversions's scaladoc This closes #11272. --- .../org/apache/flink/table/planner/DataStreamConversions.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/DataStreamConversions.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/DataStreamConversions.scala index 199a32ee3e..fe91dfb6e8 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/DataStreamConversions.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/DataStreamConversions.scala @@ -42,7 +42,8 @@ object DataStreamConversions { * field naming might be lost during optimization. * @param withChangeFlag Set to true to emit records with change flags. * @param requestedOutputType The [[TypeInformation]] of the resulting [[DataStream]]. - * @tparam A The type of the resulting [[DataStream]]. + * @param config The [[TableConfig]] of the current [[TableEnvironment]]. + * @tparam A The type of the resulting [[DataStream]]. * @return The [[DataStream]] of requested type. */ def convert[A]( -- Gitee From 78d3da7f2db04efad6b3412b8d418bb46dec92c2 Mon Sep 17 00:00:00 2001 From: Qingsheng Ren Date: Sat, 22 Feb 2020 16:19:23 +0800 Subject: [PATCH 040/885] [FLINK-16131] [docs] Translate /ops/filesystems/s3.zh.md This closes #11207. --- docs/ops/filesystems/s3.zh.md | 90 ++++++++++++++++------------------- 1 file changed, 40 insertions(+), 50 deletions(-) diff --git a/docs/ops/filesystems/s3.zh.md b/docs/ops/filesystems/s3.zh.md index 4e371c3d3f..5bfd39d01d 100644 --- a/docs/ops/filesystems/s3.zh.md +++ b/docs/ops/filesystems/s3.zh.md @@ -23,114 +23,104 @@ specific language governing permissions and limitations under the License. --> -[Amazon Simple Storage Service](http://aws.amazon.com/s3/) (Amazon S3) provides cloud object storage for a variety of use cases. You can use S3 with Flink for **reading** and **writing data** as well in conjunction with the [streaming **state backends**]({{ site.baseurl}}/ops/state/state_backends.html). +[Amazon Simple Storage Service](http://aws.amazon.com/s3/) (Amazon S3) 提供用于多种场景的云对象存储。S3 可与 Flink 一起使用以读取、写入数据,并可与 [流的 **State backends**]({{ site.baseurl}}/ops/state/state_backends.html) 相结合使用。 * This will be replaced by the TOC {:toc} -You can use S3 objects like regular files by specifying paths in the following format: +通过以下格式指定路径,S3 对象可类似于普通文件使用: {% highlight plain %} s3:/// {% endhighlight %} -The endpoint can either be a single file or a directory, for example: +Endpoint 可以是一个文件或目录,例如: {% highlight java %} -// Read from S3 bucket +// 读取 S3 bucket env.readTextFile("s3:///"); -// Write to S3 bucket +// 写入 S3 bucket stream.writeAsText("s3:///"); -// Use S3 as FsStatebackend +// 使用 S3 作为 FsStatebackend env.setStateBackend(new FsStateBackend("s3:///")); {% endhighlight %} -Note that these examples are *not* exhaustive and you can use S3 in other places as well, including your [high availability setup](../jobmanager_high_availability.html) or the [RocksDBStateBackend]({{ site.baseurl }}/ops/state/state_backends.html#the-rocksdbstatebackend); everywhere that Flink expects a FileSystem URI. +注意这些例子并*不详尽*,S3 同样可以用在其他场景,包括 [JobManager 高可用配置](../jobmanager_high_availability.html) 或 [RocksDBStateBackend]({{ site.baseurl }}/zh/ops/state/state_backends.html#the-rocksdbstatebackend),以及所有 Flink 需要使用文件系统 URI 的位置。 -For most use cases, you may use one of our `flink-s3-fs-hadoop` and `flink-s3-fs-presto` S3 filesystem plugins which are self-contained and easy to set up. -For some cases, however, e.g., for using S3 as YARN's resource storage dir, it may be necessary to set up a specific Hadoop S3 filesystem implementation. +在大部分使用场景下,可使用 `flink-s3-fs-hadoop` 或 `flink-s3-fs-presto` 两个独立且易于设置的 S3 文件系统插件。然而在某些情况下,例如使用 S3 作为 YARN 的资源存储目录时,可能需要配置 Hadoop S3 文件系统。 -### Hadoop/Presto S3 File Systems plugins +### Hadoop/Presto S3 文件系统插件 -{% panel **Note:** You don't have to configure this manually if you are running [Flink on EMR](https://docs.aws.amazon.com/emr/latest/ReleaseGuide/emr-flink.html). %} +{% panel **注意:** 如果您在使用 [Flink on EMR](https://docs.aws.amazon.com/emr/latest/ReleaseGuide/emr-flink.html),您无需手动对此进行配置。 %} -Flink provides two file systems to talk to Amazon S3, `flink-s3-fs-presto` and `flink-s3-fs-hadoop`. -Both implementations are self-contained with no dependency footprint, so there is no need to add Hadoop to the classpath to use them. +Flink 提供两种文件系统用来与 S3 交互:`flink-s3-fs-presto` 和 `flink-s3-fs-hadoop`。两种实现都是独立的且没有依赖项,因此使用时无需将 Hadoop 添加至 classpath。 - - `flink-s3-fs-presto`, registered under the scheme *s3://* and *s3p://*, is based on code from the [Presto project](https://prestodb.io/). - You can configure it the same way you can [configure the Presto file system](https://prestodb.io/docs/0.187/connector/hive.html#amazon-s3-configuration) by placing adding the configurations to your `flink-conf.yaml`. Presto is the recommended file system for checkpointing to S3. + - `flink-s3-fs-presto`,通过 *s3://* 和 *s3p://* 两种 scheme 使用,基于 [Presto project](https://prestodb.io/)。 + 可以使用与[配置 Presto 文件系统](https://prestodb.io/docs/0.187/connector/hive.html#amazon-s3-configuration)相同的方法进行配置,即将配置添加到 `flink-conf.yaml` 文件中。推荐使用 Presto 文件系统来在 S3 中建立 checkpoint。 - - `flink-s3-fs-hadoop`, registered under *s3://* and *s3a://*, based on code from the [Hadoop Project](https://hadoop.apache.org/). - The file system can be [configured exactly like Hadoop's s3a](https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/index.html#S3A) by placing adding the configurations to your `flink-conf.yaml`. It is the only S3 file system with support for the [StreamingFileSink]({{ site.baseurl}}/dev/connectors/streamfile_sink.html). + - `flink-s3-fs-hadoop`,通过 *s3://* 和 *s3a://* 两种 scheme 使用, 基于 [Hadoop Project](https://hadoop.apache.org/)。 + 文件系统可以使用与 [Hadoop S3A 完全相同的配置方法](https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/index.html#S3A)进行配置,即将配置添加到 `flink-conf.yaml` 文件中。它是唯一一个支持 [StreamingFileSink]({{ site.baseurl}}/zh/dev/connectors/streamfile_sink.html) 的文件系统。 -Both `flink-s3-fs-hadoop` and `flink-s3-fs-presto` register default FileSystem -wrappers for URIs with the *s3://* scheme, `flink-s3-fs-hadoop` also registers -for *s3a://* and `flink-s3-fs-presto` also registers for *s3p://*, so you can -use this to use both at the same time. -For example, the job uses the [StreamingFileSink]({{ site.baseurl}}/dev/connectors/streamfile_sink.html) which only supports Hadoop, but uses Presto for checkpointing. -In this case, it is advised to explicitly use *s3a://* as a scheme for the sink (Hadoop) and *s3p://* for checkpointing (Presto). +`flink-s3-fs-hadoop` 和 `flink-s3-fs-presto` 都为 *s3://* scheme 注册了默认的文件系统包装器,`flink-s3-fs-hadoop` 另外注册了 *s3a://*,`flink-s3-fs-presto` 注册了 *s3p://*,因此二者可以同时使用。 +例如某作业使用了 [StreamingFileSink]({{ site.baseurl}}/zh/dev/connectors/streamfile_sink.html),它仅支持 Hadoop,但建立 checkpoint 使用 Presto。在这种情况下,建议明确地使用 *s3a://* 作为 sink (Hadoop) 的 scheme,checkpoint (Presto) 使用 *s3p://*。 -To use `flink-s3-fs-hadoop` or `flink-s3-fs-presto`, copy the respective JAR file from the `opt` directory to the `plugins` directory of your Flink distribution before starting Flink, e.g. +在启动 Flink 之前,将对应的 JAR 文件从 `opt` 复制到 Flink 发行版的 `plugins` 目录下,以使用 `flink-s3-fs-hadoop` 或 `flink-s3-fs-presto`。 {% highlight bash %} mkdir ./plugins/s3-fs-presto cp ./opt/flink-s3-fs-presto-{{ site.version }}.jar ./plugins/s3-fs-presto/ {% endhighlight %} -#### Configure Access Credentials +#### 配置访问凭据 -After setting up the S3 FileSystem wrapper, you need to make sure that Flink is allowed to access your S3 buckets. +在设置好 S3 文件系统包装器后,您需要确认 Flink 具有访问 S3 Bucket 的权限。 -##### Identity and Access Management (IAM) (Recommended) +##### Identity and Access Management (IAM)(推荐使用) -The recommended way of setting up credentials on AWS is via [Identity and Access Management (IAM)](http://docs.aws.amazon.com/IAM/latest/UserGuide/introduction.html). You can use IAM features to securely give Flink instances the credentials that they need to access S3 buckets. Details about how to do this are beyond the scope of this documentation. Please refer to the AWS user guide. What you are looking for are [IAM Roles](http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/iam-roles-for-amazon-ec2.html). +建议通过 [Identity and Access Management (IAM)](http://docs.aws.amazon.com/IAM/latest/UserGuide/introduction.html) 来配置 AWS 凭据。可使用 IAM 功能为 Flink 实例安全地提供访问 S3 Bucket 所需的凭据。关于配置的细节超出了本文档的范围,请参考 AWS 用户手册中的 [IAM Roles](http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/iam-roles-for-amazon-ec2.html) 部分。 -If you set this up correctly, you can manage access to S3 within AWS and don't need to distribute any access keys to Flink. +如果配置正确,则可在 AWS 中管理对 S3 的访问,而无需为 Flink 分发任何访问密钥(Access Key)。 -##### Access Keys (Discouraged) +##### 访问密钥(Access Key)(不推荐) -Access to S3 can be granted via your **access and secret key pair**. Please note that this is discouraged since the [introduction of IAM roles](https://blogs.aws.amazon.com/security/post/Tx1XG3FX6VMU6O5/A-safer-way-to-distribute-AWS-credentials-to-EC2). +可以通过**访问密钥对(access and secret key)**授予 S3 访问权限。请注意,根据 [Introduction of IAM roles](https://blogs.aws.amazon.com/security/post/Tx1XG3FX6VMU6O5/A-safer-way-to-distribute-AWS-credentials-to-EC2),不推荐使用该方法。 -You need to configure both `s3.access-key` and `s3.secret-key` in Flink's `flink-conf.yaml`: + `s3.access-key` 和 `s3.secret-key` 均需要在 Flink 的 `flink-conf.yaml` 中进行配置: {% highlight yaml %} s3.access-key: your-access-key s3.secret-key: your-secret-key {% endhighlight %} -## Configure Non-S3 Endpoint +## 配置非 S3 访问点 -The S3 Filesystems also support using S3 compliant object stores such as [IBM's Cloud Object Storage](https://www.ibm.com/cloud/object-storage) and [Minio](https://min.io/). -To do so, configure your endpoint in `flink-conf.yaml`. +S3 文件系统还支持兼容 S3 的对象存储服务,如 [IBM's Cloud Object Storage](https://www.ibm.com/cloud/object-storage) 和 [Minio](https://min.io/)。可在 `flink-conf.yaml` 中配置使用的访问点: {% highlight yaml %} s3.endpoint: your-endpoint-hostname {% endhighlight %} -## Configure Path Style Access +## 配置路径样式的访问 -Some of the S3 compliant object stores might not have virtual host style addressing enabled by default. In such cases, you will have to provide the property to enable path style access in in `flink-conf.yaml`. +某些兼容 S3 的对象存储服务可能没有默认启用虚拟主机样式的寻址。这种情况下需要在 `flink-conf.yaml` 中添加配置以启用路径样式的访问: {% highlight yaml %} s3.path.style.access: true {% endhighlight %} -## Entropy injection for S3 file systems +## S3 文件系统的熵注入 -The bundled S3 file systems (`flink-s3-fs-presto` and `flink-s3-fs-hadoop`) support entropy injection. Entropy injection is -a technique to improve the scalability of AWS S3 buckets through adding some random characters near the beginning of the key. +内置的 S3 文件系统 (`flink-s3-fs-presto` and `flink-s3-fs-hadoop`) 支持熵注入。熵注入是通过在关键字开头附近添加随机字符,以提高 AWS S3 bucket 可扩展性的技术。 -If entropy injection is activated, a configured substring in the path is replaced with random characters. For example, path -`s3://my-bucket/checkpoints/_entropy_/dashboard-job/` would be replaced by something like `s3://my-bucket/checkpoints/gf36ikvg/dashboard-job/`. -**This only happens when the file creation passes the option to inject entropy!** -Otherwise, the file path removes the entropy key substring entirely. See [FileSystem.create(Path, WriteOption)](https://ci.apache.org/projects/flink/flink-docs-release-1.6/api/java/org/apache/flink/core/fs/FileSystem.html#create-org.apache.flink.core.fs.Path-org.apache.flink.core.fs.FileSystem.WriteOptions-) -for details. +如果熵注入被启用,路径中配置好的字串将会被随机字符所替换。例如路径 `s3://my-bucket/checkpoints/_entropy_/dashboard-job/` 将会被替换成类似于 `s3://my-bucket/checkpoints/gf36ikvg/dashboard-job/` 的路径。 +**这仅在使用熵注入选项创建文件时启用!** +否则将完全删除文件路径中的 entropy key。更多细节请参见 [FileSystem.create(Path, WriteOption)](https://ci.apache.org/projects/flink/flink-docs-release-1.6/api/java/org/apache/flink/core/fs/FileSystem.html#create-org.apache.flink.core.fs.Path-org.apache.flink.core.fs.FileSystem.WriteOptions-)。 -{% panel **Note:** The Flink runtime currently passes the option to inject entropy only to checkpoint data files. All other files, including checkpoint metadata and external URI, do not inject entropy to keep checkpoint URIs predictable. %} +{% panel **注意:** 目前 Flink 运行时仅对 checkpoint 数据文件使用熵注入选项。所有其他文件包括 chekcpoint 元数据与外部 URI 都不使用熵注入,以保证 checkpoint URI 的可预测性。 %} -To enable entropy injection, configure the *entropy key* and the *entropy length* parameters. +配置 *entropy key* 与 *entropy length* 参数以启用熵注入: ``` s3.entropy.key: _entropy_ @@ -138,8 +128,8 @@ s3.entropy.length: 4 (default) ``` -The `s3.entropy.key` defines the string in paths that is replaced by the random characters. Paths that do not contain the entropy key are left unchanged. -If a file system operation does not pass the *"inject entropy"* write option, the entropy key substring is simply removed. -The `s3.entropy.length` defines the number of random alphanumeric characters used for entropy. +`s3.entropy.key` 定义了路径中被随机字符替换掉的字符串。不包含 entropy key 路径将保持不变。 +如果文件系统操作没有经过 *"熵注入"* 写入,entropy key 字串将被直接移除。 +`s3.entropy.length` 定义了用于熵注入的随机字母/数字字符的数量。 {% top %} -- Gitee From 9dba72af1be502c429e51894e4a781dd931775b8 Mon Sep 17 00:00:00 2001 From: Qingsheng Ren Date: Sat, 22 Feb 2020 16:20:45 +0800 Subject: [PATCH 041/885] [hotfix] [docs] Fix typo in /ops/filesystems/s3.md --- docs/ops/filesystems/s3.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ops/filesystems/s3.md b/docs/ops/filesystems/s3.md index 4e371c3d3f..7c47f1f67e 100644 --- a/docs/ops/filesystems/s3.md +++ b/docs/ops/filesystems/s3.md @@ -111,7 +111,7 @@ s3.endpoint: your-endpoint-hostname ## Configure Path Style Access -Some of the S3 compliant object stores might not have virtual host style addressing enabled by default. In such cases, you will have to provide the property to enable path style access in in `flink-conf.yaml`. +Some of the S3 compliant object stores might not have virtual host style addressing enabled by default. In such cases, you will have to provide the property to enable path style access in `flink-conf.yaml`. {% highlight yaml %} s3.path.style.access: true -- Gitee From 8b9af0a2c3205f48e35b320396ed9839e68057c0 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 3 Mar 2020 14:28:01 +0100 Subject: [PATCH 042/885] [FLINK-10917][build] Move&rename drowpizard property This property is only relevant for metrics; hence there's no reason for this property to be defined in the root pom. The name was changed since it was inconsistent with current naming conventions (.version). --- flink-metrics/flink-metrics-dropwizard/pom.xml | 2 +- flink-metrics/flink-metrics-graphite/pom.xml | 4 ++-- flink-metrics/pom.xml | 4 ++++ pom.xml | 1 - 4 files changed, 7 insertions(+), 4 deletions(-) diff --git a/flink-metrics/flink-metrics-dropwizard/pom.xml b/flink-metrics/flink-metrics-dropwizard/pom.xml index 04e48bb8b1..7d031c81e3 100644 --- a/flink-metrics/flink-metrics-dropwizard/pom.xml +++ b/flink-metrics/flink-metrics-dropwizard/pom.xml @@ -50,7 +50,7 @@ under the License. io.dropwizard.metrics metrics-core - ${metrics.version} + ${dropwizard.version} diff --git a/flink-metrics/flink-metrics-graphite/pom.xml b/flink-metrics/flink-metrics-graphite/pom.xml index b4f511d7e6..e530c6fa8e 100644 --- a/flink-metrics/flink-metrics-graphite/pom.xml +++ b/flink-metrics/flink-metrics-graphite/pom.xml @@ -56,13 +56,13 @@ under the License. io.dropwizard.metrics metrics-core - ${metrics.version} + ${dropwizard.version} io.dropwizard.metrics metrics-graphite - ${metrics.version} + ${dropwizard.version} diff --git a/flink-metrics/pom.xml b/flink-metrics/pom.xml index 392174f09d..9b71fa7393 100644 --- a/flink-metrics/pom.xml +++ b/flink-metrics/pom.xml @@ -33,6 +33,10 @@ under the License. flink-metrics pom + + 3.1.5 + + flink-metrics-core flink-metrics-dropwizard diff --git a/pom.xml b/pom.xml index f8e8315d57..3088d6364f 100644 --- a/pom.xml +++ b/pom.xml @@ -123,7 +123,6 @@ under the License. 2.12.0 2.10.1 - 3.1.5 0.3.0 1.8.2 1.2.0 -- Gitee From 4921e3c2a317a526f82a66b2624c0ad5ff14bdcc Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 3 Mar 2020 14:48:16 +0100 Subject: [PATCH 043/885] [FLINK-10917][metrics] Bump dropwizard to 3.2.6 --- .../flink-metrics-graphite/src/main/resources/META-INF/NOTICE | 4 ++-- flink-metrics/pom.xml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-metrics/flink-metrics-graphite/src/main/resources/META-INF/NOTICE b/flink-metrics/flink-metrics-graphite/src/main/resources/META-INF/NOTICE index 0409dc8389..6c225e1965 100644 --- a/flink-metrics/flink-metrics-graphite/src/main/resources/META-INF/NOTICE +++ b/flink-metrics/flink-metrics-graphite/src/main/resources/META-INF/NOTICE @@ -6,5 +6,5 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) -- io.dropwizard.metrics:metrics-core:3.1.5 -- io.dropwizard.metrics:metrics-graphite:3.1.5 +- io.dropwizard.metrics:metrics-core:3.2.6 +- io.dropwizard.metrics:metrics-graphite:3.2.6 diff --git a/flink-metrics/pom.xml b/flink-metrics/pom.xml index 9b71fa7393..d4c3c368b9 100644 --- a/flink-metrics/pom.xml +++ b/flink-metrics/pom.xml @@ -34,7 +34,7 @@ under the License. pom - 3.1.5 + 3.2.6 -- Gitee From 4580fd6d89880f454963e2acf6e5a5ce30512ba0 Mon Sep 17 00:00:00 2001 From: sev7e0 Date: Tue, 25 Feb 2020 23:17:04 +0800 Subject: [PATCH 044/885] [FLINK-16280][doc] Fix sample code errors in the documentation about elasticsearch connector. This closes #11216. --- docs/dev/connectors/elasticsearch.md | 14 +++++++------- docs/dev/connectors/elasticsearch.zh.md | 14 +++++++------- 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/docs/dev/connectors/elasticsearch.md b/docs/dev/connectors/elasticsearch.md index 7774d92a1d..5bc1404e3a 100644 --- a/docs/dev/connectors/elasticsearch.md +++ b/docs/dev/connectors/elasticsearch.md @@ -268,14 +268,14 @@ val esSinkBuilder = new ElasticsearchSink.Builder[String]( esSinkBuilder.setBulkFlushMaxActions(1) // provide a RestClientFactory for custom configuration on the internally created REST client -esSinkBuilder.setRestClientFactory( - restClientBuilder -> { - restClientBuilder.setDefaultHeaders(...) - restClientBuilder.setMaxRetryTimeoutMillis(...) - restClientBuilder.setPathPrefix(...) - restClientBuilder.setHttpClientConfigCallback(...) +esSinkBuilder.setRestClientFactory(new RestClientFactory { + override def configureRestClientBuilder(restClientBuilder: RestClientBuilder): Unit = { + restClientBuilder.setDefaultHeaders(...) + restClientBuilder.setMaxRetryTimeoutMillis(...) + restClientBuilder.setPathPrefix(...) + restClientBuilder.setHttpClientConfigCallback(...) } -) +}) // finally, build and add the sink to the job's pipeline input.addSink(esSinkBuilder.build) diff --git a/docs/dev/connectors/elasticsearch.zh.md b/docs/dev/connectors/elasticsearch.zh.md index 97c6a28add..59219543d8 100644 --- a/docs/dev/connectors/elasticsearch.zh.md +++ b/docs/dev/connectors/elasticsearch.zh.md @@ -268,14 +268,14 @@ val esSinkBuilder = new ElasticsearchSink.Builder[String]( esSinkBuilder.setBulkFlushMaxActions(1) // provide a RestClientFactory for custom configuration on the internally created REST client -esSinkBuilder.setRestClientFactory( - restClientBuilder -> { - restClientBuilder.setDefaultHeaders(...) - restClientBuilder.setMaxRetryTimeoutMillis(...) - restClientBuilder.setPathPrefix(...) - restClientBuilder.setHttpClientConfigCallback(...) +esSinkBuilder.setRestClientFactory(new RestClientFactory { + override def configureRestClientBuilder(restClientBuilder: RestClientBuilder): Unit = { + restClientBuilder.setDefaultHeaders(...) + restClientBuilder.setMaxRetryTimeoutMillis(...) + restClientBuilder.setPathPrefix(...) + restClientBuilder.setHttpClientConfigCallback(...) } -) +}) // finally, build and add the sink to the job's pipeline input.addSink(esSinkBuilder.build) -- Gitee From b098ce505176720ba37da8f6d6c23096b1d3a260 Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Tue, 4 Feb 2020 00:56:55 +0530 Subject: [PATCH 045/885] [FLINK-15838] Dangling CountDownLatch.await(timeout) This closes #11005. --- .../flink/core/fs/AbstractCloseableRegistryTest.java | 2 +- .../runtime/webmonitor/history/HistoryServerTest.java | 8 +++++--- .../runtime/io/network/buffer/AbstractByteBufTest.java | 6 +++--- .../runtime/jobmaster/JobMasterStopWithSavepointIT.java | 8 +++++--- .../jobmaster/JobMasterTriggerSavepointITCase.java | 3 ++- .../apache/flink/test/checkpointing/RescalingITCase.java | 5 +++-- .../apache/flink/test/checkpointing/SavepointITCase.java | 6 +++--- 7 files changed, 22 insertions(+), 16 deletions(-) diff --git a/flink-core/src/test/java/org/apache/flink/core/fs/AbstractCloseableRegistryTest.java b/flink-core/src/test/java/org/apache/flink/core/fs/AbstractCloseableRegistryTest.java index d8d639e5f1..ab14f77ae0 100644 --- a/flink-core/src/test/java/org/apache/flink/core/fs/AbstractCloseableRegistryTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/fs/AbstractCloseableRegistryTest.java @@ -240,7 +240,7 @@ public abstract class AbstractCloseableRegistryTest { * Causes the current thread to wait until {@link #close()} is called. */ public void awaitClose(final long timeout, final TimeUnit timeUnit) throws InterruptedException { - closeCalledLatch.await(timeout, timeUnit); + assertTrue(closeCalledLatch.await(timeout, timeUnit)); } } diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/history/HistoryServerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/history/HistoryServerTest.java index bb486a6a2f..e5362defd7 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/history/HistoryServerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/history/HistoryServerTest.java @@ -67,6 +67,8 @@ import java.util.Collections; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import static org.junit.Assert.assertTrue; + /** * Tests for the HistoryServer. */ @@ -143,7 +145,7 @@ public class HistoryServerTest extends TestLogger { try { hs.start(); String baseUrl = "http://localhost:" + hs.getWebPort(); - numExpectedArchivedJobs.await(10L, TimeUnit.SECONDS); + assertTrue(numExpectedArchivedJobs.await(10L, TimeUnit.SECONDS)); Assert.assertEquals(numJobs + numLegacyJobs, getJobsOverview(baseUrl).getJobs().size()); @@ -194,7 +196,7 @@ public class HistoryServerTest extends TestLogger { try { hs.start(); String baseUrl = "http://localhost:" + hs.getWebPort(); - numExpectedArchivedJobs.await(10L, TimeUnit.SECONDS); + assertTrue(numExpectedArchivedJobs.await(10L, TimeUnit.SECONDS)); Collection jobs = getJobsOverview(baseUrl).getJobs(); Assert.assertEquals(numJobs, jobs.size()); @@ -208,7 +210,7 @@ public class HistoryServerTest extends TestLogger { // delete one archive from jm Files.deleteIfExists(jmDirectory.toPath().resolve(jobIdToDelete)); - numExpectedExpiredJobs.await(10L, TimeUnit.SECONDS); + assertTrue(numExpectedExpiredJobs.await(10L, TimeUnit.SECONDS)); // check that archive is present in hs Collection jobsAfterDeletion = getJobsOverview(baseUrl).getJobs(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/AbstractByteBufTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/AbstractByteBufTest.java index 4cca27900f..19fd597a5b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/AbstractByteBufTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/AbstractByteBufTest.java @@ -2402,7 +2402,7 @@ public abstract class AbstractByteBufTest extends TestLogger { } }).start(); } - latch.await(10, TimeUnit.SECONDS); + assertTrue(latch.await(10, TimeUnit.SECONDS)); barrier.await(5, TimeUnit.SECONDS); buffer.release(); } @@ -2457,7 +2457,7 @@ public abstract class AbstractByteBufTest extends TestLogger { } }).start(); } - latch.await(10, TimeUnit.SECONDS); + assertTrue(latch.await(10, TimeUnit.SECONDS)); barrier.await(5, TimeUnit.SECONDS); buffer.release(); } @@ -2512,7 +2512,7 @@ public abstract class AbstractByteBufTest extends TestLogger { } }).start(); } - latch.await(10, TimeUnit.SECONDS); + assertTrue(latch.await(10, TimeUnit.SECONDS)); barrier.await(5, TimeUnit.SECONDS); assertNull(cause.get()); buffer.release(); diff --git a/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointIT.java b/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointIT.java index c95cc61b41..7821afce4e 100644 --- a/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointIT.java +++ b/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointIT.java @@ -182,8 +182,10 @@ public class JobMasterStopWithSavepointIT extends AbstractTestBase { } // wait until we restart at least 2 times and until we see at least 10 checkpoints. - numberOfRestarts.await(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); - checkpointsToWaitFor.await(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); + assertTrue(numberOfRestarts.await(deadline.timeLeft().toMillis(), + TimeUnit.MILLISECONDS)); + assertTrue(checkpointsToWaitFor.await(deadline.timeLeft().toMillis(), + TimeUnit.MILLISECONDS)); // verifying that we actually received a synchronous checkpoint assertTrue(syncSavepointId.get() > 0); @@ -287,7 +289,7 @@ public class JobMasterStopWithSavepointIT extends AbstractTestBase { null)); ClientUtils.submitJob(clusterClient, jobGraph); - invokeLatch.await(60, TimeUnit.SECONDS); + assertTrue(invokeLatch.await(60, TimeUnit.SECONDS)); waitForJob(); } diff --git a/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTriggerSavepointITCase.java b/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTriggerSavepointITCase.java index 2fb3a93cd2..85cacb8ef4 100644 --- a/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTriggerSavepointITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTriggerSavepointITCase.java @@ -62,6 +62,7 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.isOneOf; +import static org.junit.Assert.assertTrue; /** * Tests for {@link org.apache.flink.runtime.jobmaster.JobMaster#triggerSavepoint(String, boolean, Time)}. @@ -114,7 +115,7 @@ public class JobMasterTriggerSavepointITCase extends AbstractTestBase { null)); ClientUtils.submitJob(clusterClient, jobGraph); - invokeLatch.await(60, TimeUnit.SECONDS); + assertTrue(invokeLatch.await(60, TimeUnit.SECONDS)); waitForJob(); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java index 735f79564c..80d0cb57bf 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java @@ -82,6 +82,7 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; /** * Test savepoint rescaling. @@ -193,7 +194,7 @@ public class RescalingITCase extends TestLogger { ClientUtils.submitJob(client, jobGraph); // wait til the sources have emitted numberElements for each key and completed a checkpoint - SubtaskIndexFlatMapper.workCompletedLatch.await(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); + assertTrue(SubtaskIndexFlatMapper.workCompletedLatch.await(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); // verify the current state @@ -337,7 +338,7 @@ public class RescalingITCase extends TestLogger { ClientUtils.submitJob(client, jobGraph); // wait til the sources have emitted numberElements for each key and completed a checkpoint - SubtaskIndexFlatMapper.workCompletedLatch.await(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); + assertTrue(SubtaskIndexFlatMapper.workCompletedLatch.await(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); // verify the current state diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java index c89c09598f..16a76fecde 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java @@ -436,7 +436,7 @@ public class SavepointITCase extends TestLogger { JobID jobID = submissionResult.getJobID(); // wait for the Tasks to be ready - StatefulCounter.getProgressLatch().await(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); + assertTrue(StatefulCounter.getProgressLatch().await(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); savepointPath = client.triggerSavepoint(jobID, null).get(); LOG.info("Retrieved savepoint: " + savepointPath + "."); @@ -484,10 +484,10 @@ public class SavepointITCase extends TestLogger { // Submit the job ClientUtils.submitJob(client, modifiedJobGraph); // Await state is restored - StatefulCounter.getRestoreLatch().await(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); + assertTrue(StatefulCounter.getRestoreLatch().await(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); // Await some progress after restore - StatefulCounter.getProgressLatch().await(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); + assertTrue(StatefulCounter.getProgressLatch().await(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); } finally { cluster.after(); } -- Gitee From de5f8fa5bbf8bebb429be76a8b0226666e81f8de Mon Sep 17 00:00:00 2001 From: Thomas Weise Date: Mon, 2 Mar 2020 16:12:33 -0800 Subject: [PATCH 046/885] [FLINK-16393][kinesis] Skip record emitter thread creation w/o source sync --- .../kinesis/internals/KinesisDataFetcher.java | 36 ++++++++++--------- 1 file changed, 19 insertions(+), 17 deletions(-) diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java index 52d6293998..054a532438 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java @@ -490,29 +490,31 @@ public class KinesisDataFetcher { getConsumerConfiguration().getProperty(ConsumerConfigConstants.WATERMARK_LOOKAHEAD_MILLIS, Long.toString(0))); recordEmitter.setMaxLookaheadMillis(Math.max(lookaheadMillis, watermarkSyncMillis * 3)); + + // record emitter depends on periodic watermark + // it runs in a separate thread since main thread is used for discovery + Runnable recordEmitterRunnable = new Runnable() { + @Override + public void run() { + try { + recordEmitter.run(); + } catch (Throwable error) { + // report the error that terminated the emitter loop to source thread + stopWithError(error); + } + } + }; + + Thread thread = new Thread(recordEmitterRunnable); + thread.setName("recordEmitter-" + runtimeContext.getTaskNameWithSubtasks()); + thread.setDaemon(true); + thread.start(); } } this.shardIdleIntervalMillis = Long.parseLong( getConsumerConfiguration().getProperty(ConsumerConfigConstants.SHARD_IDLE_INTERVAL_MILLIS, Long.toString(ConsumerConfigConstants.DEFAULT_SHARD_IDLE_INTERVAL_MILLIS))); - // run record emitter in separate thread since main thread is used for discovery - Runnable recordEmitterRunnable = new Runnable() { - @Override - public void run() { - try { - recordEmitter.run(); - } catch (Throwable error) { - // report the error that terminated the emitter loop to source thread - stopWithError(error); - } - } - }; - - Thread thread = new Thread(recordEmitterRunnable); - thread.setName("recordEmitter-" + runtimeContext.getTaskNameWithSubtasks()); - thread.setDaemon(true); - thread.start(); } // ------------------------------------------------------------------------ -- Gitee From 9b065dcf820f4c0ba2c3c900f62c47310606e696 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Thu, 27 Feb 2020 15:43:02 +0800 Subject: [PATCH 047/885] [hotfix] Fix checkstyle violations in ExecutionGraphTestUtils --- .../executiongraph/ExecutionGraphTestUtils.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java index e7e4bfe1ae..52a297e33c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java @@ -241,7 +241,7 @@ public class ExecutionGraphTestUtils { /** * Checks that all execution are in state DEPLOYING and then switches them - * to state RUNNING + * to state RUNNING. */ public static void switchToRunning(ExecutionGraph eg) { // check that all execution are in state DEPLOYING @@ -261,11 +261,11 @@ public class ExecutionGraphTestUtils { // ------------------------------------------------------------------------ // state modifications // ------------------------------------------------------------------------ - + public static void setVertexState(ExecutionVertex vertex, ExecutionState state) { try { Execution exec = vertex.getCurrentExecutionAttempt(); - + Field f = Execution.class.getDeclaredField("state"); f.setAccessible(true); f.set(exec, state); @@ -274,11 +274,11 @@ public class ExecutionGraphTestUtils { throw new RuntimeException("Modifying the state failed", e); } } - + public static void setVertexResource(ExecutionVertex vertex, LogicalSlot slot) { Execution exec = vertex.getCurrentExecutionAttempt(); - if(!exec.tryAssignResource(slot)) { + if (!exec.tryAssignResource(slot)) { throw new RuntimeException("Could not assign resource."); } } @@ -426,7 +426,7 @@ public class ExecutionGraphTestUtils { return new ExecutionJobVertex(graph, ajv, 1, AkkaUtils.getDefaultTimeout()); } - + public static ExecutionJobVertex getExecutionVertex(JobVertexID id) throws Exception { return getExecutionVertex(id, new DirectScheduledExecutorService()); } @@ -436,7 +436,7 @@ public class ExecutionGraphTestUtils { // ------------------------------------------------------------------------ /** - * Verifies the generated {@link ExecutionJobVertex} for a given {@link JobVertex} in a {@link ExecutionGraph} + * Verifies the generated {@link ExecutionJobVertex} for a given {@link JobVertex} in a {@link ExecutionGraph}. * * @param executionGraph the generated execution graph * @param originJobVertex the vertex to verify for -- Gitee From b2bcbd7a3f5e9a26054216007b5e01da83f41162 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Tue, 25 Feb 2020 13:58:18 +0800 Subject: [PATCH 048/885] =?UTF-8?q?[hotfix][tests]=20Change=20ExecutionGra?= =?UTF-8?q?phTestUtils#getExecutionVertex(=E2=80=A6)=20to=20ExecutionGraph?= =?UTF-8?q?TestUtils#getExecutionJobVertex(=E2=80=A6)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit These methods actually returns ExecutionJobVertex. --- .../ExecutionGraphTestUtils.java | 10 +++++----- .../ExecutionVertexCancelTest.java | 18 +++++++++--------- .../ExecutionVertexDeploymentTest.java | 16 ++++++++-------- .../ExecutionVertexSchedulingTest.java | 8 ++++---- .../IntermediateResultPartitionTest.java | 4 ++-- 5 files changed, 28 insertions(+), 28 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java index 52a297e33c..8575d6390e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java @@ -398,13 +398,13 @@ public class ExecutionGraphTestUtils { return groupVertex; } - public static ExecutionJobVertex getExecutionVertex( + public static ExecutionJobVertex getExecutionJobVertex( JobVertexID id, ScheduledExecutorService executor) throws Exception { - return getExecutionVertex(id, executor, ScheduleMode.LAZY_FROM_SOURCES); + return getExecutionJobVertex(id, executor, ScheduleMode.LAZY_FROM_SOURCES); } - public static ExecutionJobVertex getExecutionVertex( + public static ExecutionJobVertex getExecutionJobVertex( JobVertexID id, ScheduledExecutorService executor, ScheduleMode scheduleMode) throws Exception { @@ -427,8 +427,8 @@ public class ExecutionGraphTestUtils { return new ExecutionJobVertex(graph, ajv, 1, AkkaUtils.getDefaultTimeout()); } - public static ExecutionJobVertex getExecutionVertex(JobVertexID id) throws Exception { - return getExecutionVertex(id, new DirectScheduledExecutorService()); + public static ExecutionJobVertex getExecutionJobVertex(JobVertexID id) throws Exception { + return getExecutionJobVertex(id, new DirectScheduledExecutorService()); } // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java index 77b7474d36..665e97204e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java @@ -38,7 +38,7 @@ import java.io.IOException; import java.util.Collections; import java.util.concurrent.CompletableFuture; -import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getExecutionVertex; +import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getExecutionJobVertex; import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.setVertexResource; import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.setVertexState; import static org.junit.Assert.assertEquals; @@ -60,7 +60,7 @@ public class ExecutionVertexCancelTest extends TestLogger { public void testCancelFromCreated() { try { final JobVertexID jid = new JobVertexID(); - final ExecutionJobVertex ejv = getExecutionVertex(jid); + final ExecutionJobVertex ejv = getExecutionJobVertex(jid); final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0], AkkaUtils.getDefaultTimeout()); @@ -87,7 +87,7 @@ public class ExecutionVertexCancelTest extends TestLogger { public void testCancelFromScheduled() { try { final JobVertexID jid = new JobVertexID(); - final ExecutionJobVertex ejv = getExecutionVertex(jid); + final ExecutionJobVertex ejv = getExecutionJobVertex(jid); final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0], AkkaUtils.getDefaultTimeout()); @@ -115,7 +115,7 @@ public class ExecutionVertexCancelTest extends TestLogger { public void testCancelFromRunning() { try { final JobVertexID jid = new JobVertexID(); - final ExecutionJobVertex ejv = getExecutionVertex(jid, new DirectScheduledExecutorService()); + final ExecutionJobVertex ejv = ExecutionGraphTestUtils.getExecutionJobVertex(jid, new DirectScheduledExecutorService()); final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0], AkkaUtils.getDefaultTimeout()); @@ -150,7 +150,7 @@ public class ExecutionVertexCancelTest extends TestLogger { try { final JobVertexID jid = new JobVertexID(); - final ExecutionJobVertex ejv = getExecutionVertex(jid, new DirectScheduledExecutorService()); + final ExecutionJobVertex ejv = ExecutionGraphTestUtils.getExecutionJobVertex(jid, new DirectScheduledExecutorService()); final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0], AkkaUtils.getDefaultTimeout()); @@ -193,7 +193,7 @@ public class ExecutionVertexCancelTest extends TestLogger { // this may happen when the task finished or failed while the call was in progress try { final JobVertexID jid = new JobVertexID(); - final ExecutionJobVertex ejv = getExecutionVertex(jid, new DirectScheduledExecutorService()); + final ExecutionJobVertex ejv = ExecutionGraphTestUtils.getExecutionJobVertex(jid, new DirectScheduledExecutorService()); final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0], AkkaUtils.getDefaultTimeout()); @@ -223,7 +223,7 @@ public class ExecutionVertexCancelTest extends TestLogger { public void testCancelCallFails() { try { final JobVertexID jid = new JobVertexID(); - final ExecutionJobVertex ejv = getExecutionVertex(jid, new DirectScheduledExecutorService()); + final ExecutionJobVertex ejv = ExecutionGraphTestUtils.getExecutionJobVertex(jid, new DirectScheduledExecutorService()); final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0], AkkaUtils.getDefaultTimeout()); @@ -280,7 +280,7 @@ public class ExecutionVertexCancelTest extends TestLogger { public void testScheduleOrDeployAfterCancel() { try { final JobVertexID jid = new JobVertexID(); - final ExecutionJobVertex ejv = getExecutionVertex(jid); + final ExecutionJobVertex ejv = getExecutionJobVertex(jid); final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0], AkkaUtils.getDefaultTimeout()); @@ -325,7 +325,7 @@ public class ExecutionVertexCancelTest extends TestLogger { try { final JobVertexID jid = new JobVertexID(); - final ExecutionJobVertex ejv = getExecutionVertex(jid); + final ExecutionJobVertex ejv = getExecutionJobVertex(jid); // scheduling while canceling is an illegal state transition try { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java index f0a4524e4a..e51e61bcf2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java @@ -45,7 +45,7 @@ import java.net.InetAddress; import java.util.Collection; import java.util.concurrent.CompletableFuture; -import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getExecutionVertex; +import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getExecutionJobVertex; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; @@ -63,7 +63,7 @@ public class ExecutionVertexDeploymentTest extends TestLogger { try { final JobVertexID jid = new JobVertexID(); - final ExecutionJobVertex ejv = getExecutionVertex(jid); + final ExecutionJobVertex ejv = getExecutionJobVertex(jid); final LogicalSlot slot = new TestingLogicalSlotBuilder().createTestingLogicalSlot(); @@ -97,7 +97,7 @@ public class ExecutionVertexDeploymentTest extends TestLogger { try { final JobVertexID jid = new JobVertexID(); - final ExecutionJobVertex ejv = getExecutionVertex(jid, new DirectScheduledExecutorService()); + final ExecutionJobVertex ejv = ExecutionGraphTestUtils.getExecutionJobVertex(jid, new DirectScheduledExecutorService()); final LogicalSlot slot = new TestingLogicalSlotBuilder().createTestingLogicalSlot(); @@ -133,7 +133,7 @@ public class ExecutionVertexDeploymentTest extends TestLogger { public void testDeployWithAsynchronousAnswer() { try { final JobVertexID jid = new JobVertexID(); - final ExecutionJobVertex ejv = getExecutionVertex(jid); + final ExecutionJobVertex ejv = getExecutionJobVertex(jid); final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0], AkkaUtils.getDefaultTimeout()); @@ -175,7 +175,7 @@ public class ExecutionVertexDeploymentTest extends TestLogger { public void testDeployFailedSynchronous() { try { final JobVertexID jid = new JobVertexID(); - final ExecutionJobVertex ejv = getExecutionVertex(jid, new DirectScheduledExecutorService()); + final ExecutionJobVertex ejv = ExecutionGraphTestUtils.getExecutionJobVertex(jid, new DirectScheduledExecutorService()); final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0], AkkaUtils.getDefaultTimeout()); @@ -203,7 +203,7 @@ public class ExecutionVertexDeploymentTest extends TestLogger { public void testDeployFailedAsynchronously() { try { final JobVertexID jid = new JobVertexID(); - final ExecutionJobVertex ejv = getExecutionVertex(jid); + final ExecutionJobVertex ejv = getExecutionJobVertex(jid); final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0], AkkaUtils.getDefaultTimeout()); @@ -240,7 +240,7 @@ public class ExecutionVertexDeploymentTest extends TestLogger { try { final JobVertexID jid = new JobVertexID(); - final ExecutionJobVertex ejv = getExecutionVertex(jid, new DirectScheduledExecutorService()); + final ExecutionJobVertex ejv = ExecutionGraphTestUtils.getExecutionJobVertex(jid, new DirectScheduledExecutorService()); final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0], AkkaUtils.getDefaultTimeout()); @@ -288,7 +288,7 @@ public class ExecutionVertexDeploymentTest extends TestLogger { @Test public void testTddProducedPartitionsLazyScheduling() throws Exception { for (ScheduleMode scheduleMode: ScheduleMode.values()) { - ExecutionJobVertex jobVertex = getExecutionVertex( + ExecutionJobVertex jobVertex = ExecutionGraphTestUtils.getExecutionJobVertex( new JobVertexID(), new DirectScheduledExecutorService(), scheduleMode); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java index babb755bd2..45e920a0d6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java @@ -31,7 +31,7 @@ import org.junit.Test; import java.util.Collections; import java.util.concurrent.CompletableFuture; -import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getExecutionVertex; +import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getExecutionJobVertex; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.fail; @@ -41,7 +41,7 @@ public class ExecutionVertexSchedulingTest extends TestLogger { @Test public void testSlotReleasedWhenScheduledImmediately() { try { - final ExecutionJobVertex ejv = getExecutionVertex(new JobVertexID()); + final ExecutionJobVertex ejv = getExecutionJobVertex(new JobVertexID()); final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0], AkkaUtils.getDefaultTimeout()); @@ -73,7 +73,7 @@ public class ExecutionVertexSchedulingTest extends TestLogger { @Test public void testSlotReleasedWhenScheduledQueued() { try { - final ExecutionJobVertex ejv = getExecutionVertex(new JobVertexID()); + final ExecutionJobVertex ejv = getExecutionJobVertex(new JobVertexID()); final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0], AkkaUtils.getDefaultTimeout()); @@ -109,7 +109,7 @@ public class ExecutionVertexSchedulingTest extends TestLogger { @Test public void testScheduleToDeploying() { try { - final ExecutionJobVertex ejv = getExecutionVertex(new JobVertexID()); + final ExecutionJobVertex ejv = getExecutionJobVertex(new JobVertexID()); final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0], AkkaUtils.getDefaultTimeout()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartitionTest.java index 50f3d9dfbf..918f98b225 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartitionTest.java @@ -26,7 +26,7 @@ import org.apache.flink.runtime.testutils.DirectScheduledExecutorService; import org.apache.flink.util.TestLogger; import org.junit.Test; -import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getExecutionVertex; +import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getExecutionJobVertex; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -131,7 +131,7 @@ public class IntermediateResultPartitionTest extends TestLogger { ResultPartitionType resultPartitionType, int producerCount) throws Exception { - ExecutionJobVertex jobVertex = getExecutionVertex(new JobVertexID(), new DirectScheduledExecutorService()); + ExecutionJobVertex jobVertex = getExecutionJobVertex(new JobVertexID(), new DirectScheduledExecutorService()); IntermediateResult result = new IntermediateResult(new IntermediateDataSetID(), jobVertex, producerCount, resultPartitionType); for (int i = 0; i < producerCount; i++) { -- Gitee From 007b8755462e2237ac39a3218ccb9d1377b76180 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Wed, 4 Mar 2020 22:03:39 +0800 Subject: [PATCH 049/885] =?UTF-8?q?[FLINK-16300][tests]=20Introduce=20Exec?= =?UTF-8?q?utionGraphTestUtils#getExecutions(=E2=80=A6)=20to=20replace=20S?= =?UTF-8?q?chedulerTestUtils#getTestVertex(=E2=80=A6)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../ExecutionGraphTestUtils.java | 84 +++++++++ .../runtime/executiongraph/ExecutionTest.java | 10 +- .../executiongraph/TestExecutionVertex.java | 55 ++++++ .../ScheduleWithCoLocationHintTest.java | 123 +++++++------ .../scheduler/SchedulerIsolatedTasksTest.java | 43 +++-- .../scheduler/SchedulerTestUtils.java | 171 ------------------ .../slotpool/SlotPoolInteractionsTest.java | 8 +- 7 files changed, 235 insertions(+), 259 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestExecutionVertex.java delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java index 8575d6390e..2da1cd664f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java @@ -34,9 +34,11 @@ import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.ScheduleMode; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway; import org.apache.flink.runtime.jobmaster.LogicalSlot; import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.testtasks.NoOpInvokable; import org.apache.flink.runtime.testutils.DirectScheduledExecutorService; @@ -45,7 +47,10 @@ import javax.annotation.Nullable; import java.lang.reflect.Field; import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; import java.util.List; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeoutException; import java.util.function.Predicate; @@ -60,6 +65,8 @@ import static org.junit.Assert.assertNotNull; */ public class ExecutionGraphTestUtils { + private static final Time DEFAULT_TIMEOUT = AkkaUtils.getDefaultTimeout(); + // ------------------------------------------------------------------------ // reaching states // ------------------------------------------------------------------------ @@ -409,8 +416,22 @@ public class ExecutionGraphTestUtils { ScheduledExecutorService executor, ScheduleMode scheduleMode) throws Exception { + return getExecutionJobVertex(id, 1, null, executor, scheduleMode); + } + + public static ExecutionJobVertex getExecutionJobVertex( + JobVertexID id, + int parallelism, + @Nullable SlotSharingGroup slotSharingGroup, + ScheduledExecutorService executor, + ScheduleMode scheduleMode) throws Exception { + JobVertex ajv = new JobVertex("TestVertex", id); ajv.setInvokableClass(AbstractInvokable.class); + ajv.setParallelism(parallelism); + if (slotSharingGroup != null) { + ajv.setSlotSharingGroup(slotSharingGroup); + } JobGraph jobGraph = new JobGraph(ajv); jobGraph.setScheduleMode(scheduleMode); @@ -431,6 +452,69 @@ public class ExecutionGraphTestUtils { return getExecutionJobVertex(id, new DirectScheduledExecutorService()); } + public static Execution getExecution() throws Exception { + final ExecutionJobVertex ejv = getExecutionJobVertex(new JobVertexID()); + return ejv.getTaskVertices()[0].getCurrentExecutionAttempt(); + } + + public static Execution getExecution(final TaskManagerLocation... preferredLocations) throws Exception { + return getExecution(mapToPreferredLocationFutures(preferredLocations)); + } + + private static Collection> mapToPreferredLocationFutures( + final TaskManagerLocation... preferredLocations) { + + final Collection> preferredLocationFutures = new ArrayList<>(); + for (TaskManagerLocation preferredLocation : preferredLocations) { + preferredLocationFutures.add(CompletableFuture.completedFuture(preferredLocation)); + } + return preferredLocationFutures; + } + + public static Execution getExecution( + final Collection> preferredLocationFutures) throws Exception { + + final ExecutionJobVertex ejv = getExecutionJobVertex(new JobVertexID()); + final TestExecutionVertex ev = new TestExecutionVertex(ejv, 0, new IntermediateResult[0], DEFAULT_TIMEOUT); + ev.setPreferredLocationFutures(preferredLocationFutures); + return ev.getCurrentExecutionAttempt(); + } + + public static Execution getExecution( + final JobVertexID jid, + final int subtaskIndex, + final int numTasks, + final SlotSharingGroup slotSharingGroup) throws Exception { + + return getExecution(jid, subtaskIndex, numTasks, slotSharingGroup, null); + } + + public static Execution getExecution( + final JobVertexID jid, + final int subtaskIndex, + final int numTasks, + final SlotSharingGroup slotSharingGroup, + @Nullable final TaskManagerLocation... locations) throws Exception { + + final ExecutionJobVertex ejv = getExecutionJobVertex( + jid, + numTasks, + slotSharingGroup, + new DirectScheduledExecutorService(), + ScheduleMode.LAZY_FROM_SOURCES); + final TestExecutionVertex ev = new TestExecutionVertex( + ejv, + subtaskIndex, + new IntermediateResult[0], + DEFAULT_TIMEOUT); + + if (locations != null) { + ev.setPreferredLocationFutures(mapToPreferredLocationFutures(locations)); + } + + return ev.getCurrentExecutionAttempt(); + } + // ------------------------------------------------------------------------ // graph vertex verifications // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionTest.java index 42f0f4818f..d939429dbe 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionTest.java @@ -30,7 +30,6 @@ import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobmanager.scheduler.LocationPreferenceConstraint; -import org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils; import org.apache.flink.runtime.jobmaster.LogicalSlot; import org.apache.flink.runtime.jobmaster.SlotOwner; import org.apache.flink.runtime.jobmaster.SlotRequestId; @@ -61,6 +60,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; +import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getExecution; import static org.apache.flink.runtime.io.network.partition.ResultPartitionType.PIPELINED; import static org.apache.flink.runtime.jobgraph.DistributionPattern.POINTWISE; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -291,7 +291,7 @@ public class ExecutionTest extends TestLogger { * Tests that all preferred locations are calculated. */ @Test - public void testAllPreferredLocationCalculation() throws ExecutionException, InterruptedException { + public void testAllPreferredLocationCalculation() throws Exception { final TaskManagerLocation taskManagerLocation1 = new LocalTaskManagerLocation(); final TaskManagerLocation taskManagerLocation2 = new LocalTaskManagerLocation(); final TaskManagerLocation taskManagerLocation3 = new LocalTaskManagerLocation(); @@ -300,7 +300,7 @@ public class ExecutionTest extends TestLogger { final CompletableFuture locationFuture2 = new CompletableFuture<>(); final CompletableFuture locationFuture3 = new CompletableFuture<>(); - final Execution execution = SchedulerTestUtils.getTestVertex(Arrays.asList(locationFuture1, locationFuture2, locationFuture3)); + final Execution execution = getExecution(Arrays.asList(locationFuture1, locationFuture2, locationFuture3)); CompletableFuture> preferredLocationsFuture = execution.calculatePreferredLocations(LocationPreferenceConstraint.ALL); @@ -323,7 +323,7 @@ public class ExecutionTest extends TestLogger { * Tests that any preferred locations are calculated. */ @Test - public void testAnyPreferredLocationCalculation() throws ExecutionException, InterruptedException { + public void testAnyPreferredLocationCalculation() throws Exception { final TaskManagerLocation taskManagerLocation1 = new LocalTaskManagerLocation(); final TaskManagerLocation taskManagerLocation3 = new LocalTaskManagerLocation(); @@ -331,7 +331,7 @@ public class ExecutionTest extends TestLogger { final CompletableFuture locationFuture2 = new CompletableFuture<>(); final CompletableFuture locationFuture3 = CompletableFuture.completedFuture(taskManagerLocation3); - final Execution execution = SchedulerTestUtils.getTestVertex(Arrays.asList(locationFuture1, locationFuture2, locationFuture3)); + final Execution execution = getExecution(Arrays.asList(locationFuture1, locationFuture2, locationFuture3)); CompletableFuture> preferredLocationsFuture = execution.calculatePreferredLocations(LocationPreferenceConstraint.ANY); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestExecutionVertex.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestExecutionVertex.java new file mode 100644 index 0000000000..cb6f3387b0 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestExecutionVertex.java @@ -0,0 +1,55 @@ +/* + * 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.runtime.executiongraph; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; + +import java.util.Collection; +import java.util.concurrent.CompletableFuture; + +/** + * An {@link ExecutionVertex} implementation for testing. + */ +public class TestExecutionVertex extends ExecutionVertex { + + private Collection> preferredLocationFutures; + + TestExecutionVertex( + ExecutionJobVertex jobVertex, + int subTaskIndex, + IntermediateResult[] producedDataSets, + Time timeout) { + + super( + jobVertex, + subTaskIndex, + producedDataSets, + timeout); + } + + public void setPreferredLocationFutures(final Collection> preferredLocationFutures) { + this.preferredLocationFutures = preferredLocationFutures; + } + + @Override + public Collection> getPreferredLocations() { + return preferredLocationFutures != null ? preferredLocationFutures : super.getPreferredLocations(); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java index b672328041..79a6aba8e9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java @@ -34,8 +34,7 @@ import org.junit.Test; import java.util.Collections; import java.util.concurrent.ExecutionException; -import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getTestVertex; -import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getTestVertexWithLocation; +import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getExecution; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; @@ -73,18 +72,18 @@ public class ScheduleWithCoLocationHintTest extends SchedulerTestBase { CoLocationConstraint c6 = new CoLocationConstraint(ccg); // schedule 4 tasks from the first vertex group - LogicalSlot s1 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c1), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - LogicalSlot s2 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c2), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - LogicalSlot s3 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c3), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - LogicalSlot s4 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c4), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - LogicalSlot s5 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c1), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - LogicalSlot s6 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c2), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - LogicalSlot s7 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 2, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c3), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - LogicalSlot s8 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 4, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c5), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - LogicalSlot s9 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 5, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c6), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - LogicalSlot s10 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 3, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c4), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - LogicalSlot s11 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 4, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c5), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - LogicalSlot s12 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 5, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c6), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + LogicalSlot s1 = testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution(jid1, 0, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c1), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + LogicalSlot s2 = testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution(jid1, 1, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c2), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + LogicalSlot s3 = testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution(jid1, 2, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c3), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + LogicalSlot s4 = testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution(jid1, 3, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c4), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + LogicalSlot s5 = testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution(jid2, 0, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c1), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + LogicalSlot s6 = testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution(jid2, 1, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c2), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + LogicalSlot s7 = testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution(jid2, 2, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c3), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + LogicalSlot s8 = testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution(jid1, 4, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c5), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + LogicalSlot s9 = testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution(jid1, 5, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c6), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + LogicalSlot s10 = testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution(jid2, 3, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c4), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + LogicalSlot s11 = testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution(jid2, 4, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c5), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + LogicalSlot s12 = testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution(jid2, 5, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c6), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); assertNotNull(s1); assertNotNull(s2); @@ -134,7 +133,7 @@ public class ScheduleWithCoLocationHintTest extends SchedulerTestBase { assertTrue(testingSlotProvider.getNumberOfAvailableSlots() >= 1); LogicalSlot single = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertex(new JobVertexID(), 0, 1, null)), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(new JobVertexID(), 0, 1, null)), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); assertNotNull(single); s1.releaseSlot(); @@ -171,11 +170,11 @@ public class ScheduleWithCoLocationHintTest extends SchedulerTestBase { CoLocationConstraint c1 = new CoLocationConstraint(new CoLocationGroup()); LogicalSlot s1 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertex(jid1, 0, 1, sharingGroup), sharingGroup.getSlotSharingGroupId(), c1), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid1, 0, 1, sharingGroup), sharingGroup.getSlotSharingGroupId(), c1), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); LogicalSlot s2 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertex(jid2, 0, 1, sharingGroup), sharingGroup.getSlotSharingGroupId(), c1), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid2, 0, 1, sharingGroup), sharingGroup.getSlotSharingGroupId(), c1), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - LogicalSlot sSolo = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 0, 1, null)), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + LogicalSlot sSolo = testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution(jid4, 0, 1, null)), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); ResourceID taskManager = s1.getTaskManagerLocation().getResourceID(); @@ -184,7 +183,7 @@ public class ScheduleWithCoLocationHintTest extends SchedulerTestBase { sSolo.releaseSlot(); LogicalSlot sNew = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertex(jid3, 0, 1, sharingGroup), sharingGroup.getSlotSharingGroupId(), c1), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid3, 0, 1, sharingGroup), sharingGroup.getSlotSharingGroupId(), c1), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); assertEquals(taskManager, sNew.getTaskManagerLocation().getResourceID()); assertEquals(2, testingSlotProvider.getNumberOfLocalizedAssignments()); @@ -207,14 +206,14 @@ public class ScheduleWithCoLocationHintTest extends SchedulerTestBase { CoLocationConstraint c1 = new CoLocationConstraint(new CoLocationGroup()); LogicalSlot s1 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertex(jid1, 0, 1, sharingGroup), sharingGroup.getSlotSharingGroupId(), c1), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid1, 0, 1, sharingGroup), sharingGroup.getSlotSharingGroupId(), c1), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); s1.releaseSlot(); - testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 1, null)), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 2, null)), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution(jid2, 0, 1, null)), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution(jid2, 1, 2, null)), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); try { - testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 0, 1, sharingGroup), sharingGroup.getSlotSharingGroupId(), c1), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution(jid3, 0, 1, sharingGroup), sharingGroup.getSlotSharingGroupId(), c1), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); fail("Scheduled even though no resource was available."); } catch (ExecutionException e) { assertTrue(e.getCause() instanceof NoResourceAvailableException); @@ -248,35 +247,35 @@ public class ScheduleWithCoLocationHintTest extends SchedulerTestBase { SlotSharingGroup shareGroup = new SlotSharingGroup(); // first wave - testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 4, shareGroup), shareGroup.getSlotSharingGroupId()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 4, shareGroup), shareGroup.getSlotSharingGroupId()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 4, shareGroup), shareGroup.getSlotSharingGroupId()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 4, shareGroup), shareGroup.getSlotSharingGroupId()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution(jid1, 0, 4, shareGroup), shareGroup.getSlotSharingGroupId()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution(jid1, 2, 4, shareGroup), shareGroup.getSlotSharingGroupId()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution(jid1, 1, 4, shareGroup), shareGroup.getSlotSharingGroupId()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution(jid1, 3, 4, shareGroup), shareGroup.getSlotSharingGroupId()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); // second wave LogicalSlot s21 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertex(jid2, 0, 4, shareGroup), shareGroup.getSlotSharingGroupId(), clc1), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid2, 0, 4, shareGroup), shareGroup.getSlotSharingGroupId(), clc1), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); LogicalSlot s22 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertex(jid2, 2, 4, shareGroup), shareGroup.getSlotSharingGroupId(), clc2), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid2, 2, 4, shareGroup), shareGroup.getSlotSharingGroupId(), clc2), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); LogicalSlot s23 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertex(jid2, 1, 4, shareGroup), shareGroup.getSlotSharingGroupId(), clc3), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid2, 1, 4, shareGroup), shareGroup.getSlotSharingGroupId(), clc3), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); LogicalSlot s24 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertex(jid2, 3, 4, shareGroup), shareGroup.getSlotSharingGroupId(), clc4), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid2, 3, 4, shareGroup), shareGroup.getSlotSharingGroupId(), clc4), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); // third wave LogicalSlot s31 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertex(jid3, 1, 4, shareGroup), shareGroup.getSlotSharingGroupId(), clc2), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid3, 1, 4, shareGroup), shareGroup.getSlotSharingGroupId(), clc2), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); LogicalSlot s32 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertex(jid3, 2, 4, shareGroup), shareGroup.getSlotSharingGroupId(), clc3), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid3, 2, 4, shareGroup), shareGroup.getSlotSharingGroupId(), clc3), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); LogicalSlot s33 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertex(jid3, 3, 4, shareGroup), shareGroup.getSlotSharingGroupId(), clc4), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid3, 3, 4, shareGroup), shareGroup.getSlotSharingGroupId(), clc4), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); LogicalSlot s34 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertex(jid3, 0, 4, shareGroup), shareGroup.getSlotSharingGroupId(), clc1), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid3, 0, 4, shareGroup), shareGroup.getSlotSharingGroupId(), clc1), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 0, 4, shareGroup), shareGroup.getSlotSharingGroupId()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 1, 4, shareGroup), shareGroup.getSlotSharingGroupId()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 2, 4, shareGroup), shareGroup.getSlotSharingGroupId()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 3, 4, shareGroup), shareGroup.getSlotSharingGroupId()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution(jid4, 0, 4, shareGroup), shareGroup.getSlotSharingGroupId()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution(jid4, 1, 4, shareGroup), shareGroup.getSlotSharingGroupId()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution(jid4, 2, 4, shareGroup), shareGroup.getSlotSharingGroupId()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution(jid4, 3, 4, shareGroup), shareGroup.getSlotSharingGroupId()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); assertEquals(s21.getTaskManagerLocation(), s34.getTaskManagerLocation()); assertEquals(s22.getTaskManagerLocation(), s31.getTaskManagerLocation()); @@ -307,25 +306,25 @@ public class ScheduleWithCoLocationHintTest extends SchedulerTestBase { // schedule something into the shared group so that both instances are in the sharing group LogicalSlot s1 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId()), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid1, 0, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId()), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); LogicalSlot s2 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, sharingGroup, loc2), sharingGroup.getSlotSharingGroupId()), slotProfileForLocation(loc2), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid1, 1, 2, sharingGroup, loc2), sharingGroup.getSlotSharingGroupId()), slotProfileForLocation(loc2), TestingUtils.infiniteTime()).get(); // schedule one locally to instance 1 LogicalSlot s3 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc1), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid2, 0, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc1), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); // schedule with co location constraint (yet unassigned) and a preference for // instance 1, but it can only get instance 2 LogicalSlot s4 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc2), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid2, 1, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc2), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); // schedule something into the assigned co-location constraints and check that they override the // other preferences LogicalSlot s5 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid3, 0, 2, sharingGroup, loc2), sharingGroup.getSlotSharingGroupId(), cc1), slotProfileForLocation(loc2), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid3, 0, 2, sharingGroup, loc2), sharingGroup.getSlotSharingGroupId(), cc1), slotProfileForLocation(loc2), TestingUtils.infiniteTime()).get(); LogicalSlot s6 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid3, 1, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc2), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid3, 1, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc2), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); // check that each slot got three assertEquals(s1.getTaskManagerLocation(), s3.getTaskManagerLocation()); @@ -367,9 +366,9 @@ public class ScheduleWithCoLocationHintTest extends SchedulerTestBase { CoLocationConstraint cc2 = new CoLocationConstraint(ccg); LogicalSlot s1 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc1), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid1, 0, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc1), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); LogicalSlot s2 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, sharingGroup, loc2), sharingGroup.getSlotSharingGroupId(), cc2), slotProfileForLocation(loc2), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid1, 1, 2, sharingGroup, loc2), sharingGroup.getSlotSharingGroupId(), cc2), slotProfileForLocation(loc2), TestingUtils.infiniteTime()).get(); s1.releaseSlot(); s2.releaseSlot(); @@ -377,9 +376,9 @@ public class ScheduleWithCoLocationHintTest extends SchedulerTestBase { assertEquals(2, testingSlotProvider.getNumberOfAvailableSlots()); LogicalSlot s3 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, sharingGroup, loc2), sharingGroup.getSlotSharingGroupId(), cc1), slotProfileForLocation(loc2), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid2, 0, 2, sharingGroup, loc2), sharingGroup.getSlotSharingGroupId(), cc1), slotProfileForLocation(loc2), TestingUtils.infiniteTime()).get(); LogicalSlot s4 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc2), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid2, 1, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc2), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); // still preserves the previous instance mapping) assertEquals(loc1, s3.getTaskManagerLocation()); @@ -413,9 +412,9 @@ public class ScheduleWithCoLocationHintTest extends SchedulerTestBase { CoLocationConstraint cc2 = new CoLocationConstraint(ccg); LogicalSlot s1 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc1), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid1, 0, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc1), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); LogicalSlot s2 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, sharingGroup, loc2), sharingGroup.getSlotSharingGroupId(), cc2), slotProfileForLocation(loc2), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid1, 1, 2, sharingGroup, loc2), sharingGroup.getSlotSharingGroupId(), cc2), slotProfileForLocation(loc2), TestingUtils.infiniteTime()).get(); s1.releaseSlot(); s2.releaseSlot(); @@ -423,13 +422,13 @@ public class ScheduleWithCoLocationHintTest extends SchedulerTestBase { assertEquals(2, testingSlotProvider.getNumberOfAvailableSlots()); LogicalSlot sa = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jidx, 0, 2, null)), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jidx, 0, 2, null)), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); LogicalSlot sb = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jidx, 1, 2, null)), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jidx, 1, 2, null)), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); try { testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, sharingGroup, loc2), sharingGroup.getSlotSharingGroupId(), cc1), slotProfileForLocation(loc2), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid2, 0, 2, sharingGroup, loc2), sharingGroup.getSlotSharingGroupId(), cc1), slotProfileForLocation(loc2), TestingUtils.infiniteTime()).get(); fail("should not be able to find a resource"); } catch (ExecutionException e) { @@ -469,14 +468,14 @@ public class ScheduleWithCoLocationHintTest extends SchedulerTestBase { // and give locality preferences that hint at using the same shared slot for both // co location constraints (which we seek to prevent) LogicalSlot s1 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc1), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid1, 0, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc1), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); LogicalSlot s2 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc2), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid2, 0, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc2), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); LogicalSlot s3 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc1), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid2, 1, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc1), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); LogicalSlot s4 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc2), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid1, 1, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc2), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); // check that each slot got three assertEquals(s1.getTaskManagerLocation(), s3.getTaskManagerLocation()); @@ -514,14 +513,14 @@ public class ScheduleWithCoLocationHintTest extends SchedulerTestBase { CoLocationConstraint cc2 = new CoLocationConstraint(ccg); LogicalSlot s1 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc1), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid1, 0, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc1), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); LogicalSlot s2 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, sharingGroup, loc2), sharingGroup.getSlotSharingGroupId(), cc2), slotProfileForLocation(loc2), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid1, 1, 2, sharingGroup, loc2), sharingGroup.getSlotSharingGroupId(), cc2), slotProfileForLocation(loc2), TestingUtils.infiniteTime()).get(); LogicalSlot s3 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId()), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid2, 0, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId()), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); LogicalSlot s4 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId()), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getExecution(jid2, 1, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId()), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); // check that each slot got two assertEquals(s1.getTaskManagerLocation(), s3.getTaskManagerLocation()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java index e305426384..b78f4176fb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java @@ -48,9 +48,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; -import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.areAllDistinct; -import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getDummyTask; -import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getTestVertex; +import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getExecution; import static org.hamcrest.Matchers.instanceOf; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -93,7 +91,7 @@ public class SchedulerIsolatedTasksTest extends SchedulerTestBase { final AtomicBoolean errored = new AtomicBoolean(false); for (int i = 0; i < numTasksToSchedule; i++) { - CompletableFuture future = testingSlotProvider.allocateSlot(new ScheduledUnit(getDummyTask()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()); + CompletableFuture future = testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()); future.thenAcceptAsync( (LogicalSlot slot) -> { synchronized (toRelease) { @@ -146,11 +144,11 @@ public class SchedulerIsolatedTasksTest extends SchedulerTestBase { final TaskManagerLocation taskManagerLocation3 = testingSlotProvider.addTaskManager(1); List slots = new ArrayList<>(); - slots.add(testingSlotProvider.allocateSlot(new ScheduledUnit(getDummyTask()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get()); - slots.add(testingSlotProvider.allocateSlot(new ScheduledUnit(getDummyTask()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get()); - slots.add(testingSlotProvider.allocateSlot(new ScheduledUnit(getDummyTask()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get()); - slots.add(testingSlotProvider.allocateSlot(new ScheduledUnit(getDummyTask()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get()); - slots.add(testingSlotProvider.allocateSlot(new ScheduledUnit(getDummyTask()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get()); + slots.add(testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get()); + slots.add(testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get()); + slots.add(testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get()); + slots.add(testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get()); + slots.add(testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get()); testingSlotProvider.releaseTaskManager(taskManagerLocation2.getResourceID()); @@ -171,7 +169,7 @@ public class SchedulerIsolatedTasksTest extends SchedulerTestBase { // cannot get another slot, since all instances are dead try { - testingSlotProvider.allocateSlot(new ScheduledUnit(getDummyTask()), SlotProfile.noRequirements(), Time.milliseconds(10L)).get(); + testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution()), SlotProfile.noRequirements(), Time.milliseconds(10L)).get(); fail("Scheduler served a slot from a dead instance"); } catch (ExecutionException e) { @@ -193,7 +191,7 @@ public class SchedulerIsolatedTasksTest extends SchedulerTestBase { final TaskManagerLocation taskManagerLocation3 = testingSlotProvider.addTaskManager(2); // schedule something on an arbitrary instance - LogicalSlot s1 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(new LocalTaskManagerLocation())), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + LogicalSlot s1 = testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution(new LocalTaskManagerLocation())), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); // figure out how we use the location hints ResourceID firstResourceId = s1.getTaskManagerLocation().getResourceID(); @@ -215,28 +213,28 @@ public class SchedulerIsolatedTasksTest extends SchedulerTestBase { TaskManagerLocation third = taskManagerLocations.get((index + 2) % taskManagerLocations.size()); // something that needs to go to the first instance again - LogicalSlot s2 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(s1.getTaskManagerLocation())), slotProfileForLocation(s1.getTaskManagerLocation()), TestingUtils.infiniteTime()).get(); + LogicalSlot s2 = testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution(s1.getTaskManagerLocation())), slotProfileForLocation(s1.getTaskManagerLocation()), TestingUtils.infiniteTime()).get(); assertEquals(first.getResourceID(), s2.getTaskManagerLocation().getResourceID()); // first or second --> second, because first is full - LogicalSlot s3 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(first, second)), slotProfileForLocation(first, second), TestingUtils.infiniteTime()).get(); + LogicalSlot s3 = testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution(first, second)), slotProfileForLocation(first, second), TestingUtils.infiniteTime()).get(); assertEquals(second.getResourceID(), s3.getTaskManagerLocation().getResourceID()); // first or third --> third (because first is full) - LogicalSlot s4 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(first, third)), slotProfileForLocation(first, third), TestingUtils.infiniteTime()).get(); - LogicalSlot s5 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(first, third)), slotProfileForLocation(first, third), TestingUtils.infiniteTime()).get(); + LogicalSlot s4 = testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution(first, third)), slotProfileForLocation(first, third), TestingUtils.infiniteTime()).get(); + LogicalSlot s5 = testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution(first, third)), slotProfileForLocation(first, third), TestingUtils.infiniteTime()).get(); assertEquals(third.getResourceID(), s4.getTaskManagerLocation().getResourceID()); assertEquals(third.getResourceID(), s5.getTaskManagerLocation().getResourceID()); // first or third --> second, because all others are full - LogicalSlot s6 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(first, third)), slotProfileForLocation(first, third), TestingUtils.infiniteTime()).get(); + LogicalSlot s6 = testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution(first, third)), slotProfileForLocation(first, third), TestingUtils.infiniteTime()).get(); assertEquals(second.getResourceID(), s6.getTaskManagerLocation().getResourceID()); // release something on the first and second instance runInMainThreadExecutor(s2::releaseSlot); runInMainThreadExecutor(s6::releaseSlot); - LogicalSlot s7 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(first, third)), slotProfileForLocation(first, third), TestingUtils.infiniteTime()).get(); + LogicalSlot s7 = testingSlotProvider.allocateSlot(new ScheduledUnit(getExecution(first, third)), slotProfileForLocation(first, third), TestingUtils.infiniteTime()).get(); assertEquals(first.getResourceID(), s7.getTaskManagerLocation().getResourceID()); assertEquals(1, testingSlotProvider.getNumberOfUnconstrainedAssignments()); @@ -266,4 +264,15 @@ public class SchedulerIsolatedTasksTest extends SchedulerTestBase { private static SlotProfile slotProfileForLocation(TaskManagerLocation... location) { return SlotProfile.preferredLocality(ResourceProfile.UNKNOWN, Arrays.asList(location)); } + + private static boolean areAllDistinct(Object... obj) { + if (obj == null) { + return true; + } + + final HashSet set = new HashSet(); + Collections.addAll(set, obj); + + return set.size() == obj.length; + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java deleted file mode 100644 index 1ca999725d..0000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java +++ /dev/null @@ -1,171 +0,0 @@ -/* - * 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.runtime.jobmanager.scheduler; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.runtime.executiongraph.Execution; -import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; -import org.apache.flink.runtime.executiongraph.ExecutionVertex; -import org.apache.flink.runtime.jobgraph.JobVertexID; -import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; -import org.apache.flink.runtime.taskmanager.TaskManagerLocation; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.atomic.AtomicInteger; - -import static org.mockito.Matchers.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - - -public class SchedulerTestUtils { - - private static final AtomicInteger port = new AtomicInteger(10000); - - // -------------------------------------------------------------------------------------------- - - public static Execution getDummyTask() { - ExecutionJobVertex executionJobVertex = mock(ExecutionJobVertex.class); - - ExecutionVertex vertex = mock(ExecutionVertex.class); - - final JobVertexID jobVertexID = new JobVertexID(); - when(vertex.getID()).thenReturn(new ExecutionVertexID(jobVertexID, 0)); - when(vertex.getJobId()).thenReturn(new JobID()); - when(vertex.toString()).thenReturn("TEST-VERTEX"); - when(vertex.getJobVertex()).thenReturn(executionJobVertex); - when(vertex.getJobvertexId()).thenReturn(jobVertexID); - - Execution execution = mock(Execution.class); - when(execution.getVertex()).thenReturn(vertex); - - return execution; - } - - public static Execution getTestVertex(TaskManagerLocation... preferredLocations) { - return getTestVertex(Arrays.asList(preferredLocations)); - } - - - public static Execution getTestVertex(Iterable preferredLocations) { - Collection> preferredLocationFutures = new ArrayList<>(4); - - for (TaskManagerLocation preferredLocation : preferredLocations) { - preferredLocationFutures.add(CompletableFuture.completedFuture(preferredLocation)); - } - - return getTestVertex(preferredLocationFutures); - } - - public static Execution getTestVertex(Collection> preferredLocationFutures) { - ExecutionJobVertex executionJobVertex = mock(ExecutionJobVertex.class); - ExecutionVertex vertex = mock(ExecutionVertex.class); - - final JobVertexID jobVertexID = new JobVertexID(); - when(vertex.getID()).thenReturn(new ExecutionVertexID(jobVertexID, 0)); - when(vertex.getPreferredLocationsBasedOnInputs()).thenReturn(preferredLocationFutures); - when(vertex.getPreferredLocations()).thenReturn(preferredLocationFutures); - when(vertex.getJobId()).thenReturn(new JobID()); - when(vertex.toString()).thenReturn("TEST-VERTEX"); - when(vertex.getJobVertex()).thenReturn(executionJobVertex); - when(vertex.getJobvertexId()).thenReturn(jobVertexID); - - Execution execution = mock(Execution.class); - when(execution.getVertex()).thenReturn(vertex); - when(execution.calculatePreferredLocations(any(LocationPreferenceConstraint.class))).thenCallRealMethod(); - - return execution; - } - - public static Execution getTestVertex(JobVertexID jid, int taskIndex, int numTasks, SlotSharingGroup slotSharingGroup) { - ExecutionJobVertex executionJobVertex = mock(ExecutionJobVertex.class); - ExecutionVertex vertex = mock(ExecutionVertex.class); - - when(executionJobVertex.getSlotSharingGroup()).thenReturn(slotSharingGroup); - - when(vertex.getID()).thenReturn(new ExecutionVertexID(jid, taskIndex)); - when(vertex.getPreferredLocationsBasedOnInputs()).thenReturn(Collections.emptyList()); - when(vertex.getJobId()).thenReturn(new JobID()); - when(vertex.getJobvertexId()).thenReturn(jid); - when(vertex.getParallelSubtaskIndex()).thenReturn(taskIndex); - when(vertex.getTotalNumberOfParallelSubtasks()).thenReturn(numTasks); - when(vertex.getMaxParallelism()).thenReturn(numTasks); - when(vertex.toString()).thenReturn("TEST-VERTEX"); - when(vertex.getTaskNameWithSubtaskIndex()).thenReturn("TEST-VERTEX"); - when(vertex.getJobVertex()).thenReturn(executionJobVertex); - - Execution execution = mock(Execution.class); - when(execution.getVertex()).thenReturn(vertex); - - return execution; - } - - public static Execution getTestVertexWithLocation( - JobVertexID jid, - int taskIndex, - int numTasks, - SlotSharingGroup slotSharingGroup, - TaskManagerLocation... locations) { - - ExecutionJobVertex executionJobVertex = mock(ExecutionJobVertex.class); - - when(executionJobVertex.getSlotSharingGroup()).thenReturn(slotSharingGroup); - - ExecutionVertex vertex = mock(ExecutionVertex.class); - - Collection> preferredLocationFutures = new ArrayList<>(locations.length); - - for (TaskManagerLocation location : locations) { - preferredLocationFutures.add(CompletableFuture.completedFuture(location)); - } - - when(vertex.getID()).thenReturn(new ExecutionVertexID(jid, taskIndex)); - when(vertex.getJobVertex()).thenReturn(executionJobVertex); - when(vertex.getPreferredLocationsBasedOnInputs()).thenReturn(preferredLocationFutures); - when(vertex.getJobId()).thenReturn(new JobID()); - when(vertex.getJobvertexId()).thenReturn(jid); - when(vertex.getParallelSubtaskIndex()).thenReturn(taskIndex); - when(vertex.getTotalNumberOfParallelSubtasks()).thenReturn(numTasks); - when(vertex.getMaxParallelism()).thenReturn(numTasks); - when(vertex.toString()).thenReturn("TEST-VERTEX"); - - Execution execution = mock(Execution.class); - when(execution.getVertex()).thenReturn(vertex); - - return execution; - } - - // -------------------------------------------------------------------------------------------- - - public static boolean areAllDistinct(Object ... obj) { - if (obj == null) { - return true; - } - - HashSet set = new HashSet(); - Collections.addAll(set, obj); - - return set.size() == obj.length; - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolInteractionsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolInteractionsTest.java index fe0d279ef2..a281b9f393 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolInteractionsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolInteractionsTest.java @@ -26,7 +26,6 @@ import org.apache.flink.runtime.executiongraph.TestingComponentMainThreadExecuto import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway; import org.apache.flink.runtime.jobmanager.scheduler.DummyScheduledUnit; import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit; -import org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils; import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway; import org.apache.flink.runtime.jobmaster.JobMasterId; import org.apache.flink.runtime.jobmaster.LogicalSlot; @@ -55,6 +54,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; import java.util.function.Consumer; +import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getExecution; import static org.apache.flink.runtime.jobmaster.slotpool.AvailableSlotsTest.DEFAULT_TESTING_PROFILE; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -96,7 +96,7 @@ public class SlotPoolInteractionsTest extends TestLogger { CompletableFuture future = testMainThreadExecutor.execute(() -> scheduler.allocateSlot( new SlotRequestId(), - new ScheduledUnit(SchedulerTestUtils.getDummyTask()), + new ScheduledUnit(getExecution()), SlotProfile.noLocality(DEFAULT_TESTING_PROFILE), fastTimeout)); @@ -124,7 +124,7 @@ public class SlotPoolInteractionsTest extends TestLogger { SlotRequestId requestId = new SlotRequestId(); CompletableFuture future = testMainThreadExecutor.execute(() -> scheduler.allocateSlot( requestId, - new ScheduledUnit(SchedulerTestUtils.getDummyTask()), + new ScheduledUnit(getExecution()), SlotProfile.noLocality(DEFAULT_TESTING_PROFILE), fastTimeout)); @@ -221,7 +221,7 @@ public class SlotPoolInteractionsTest extends TestLogger { SlotRequestId requestId = new SlotRequestId(); CompletableFuture future = testMainThreadExecutor.execute(() -> scheduler.allocateSlot( requestId, - new ScheduledUnit(SchedulerTestUtils.getDummyTask()), + new ScheduledUnit(getExecution()), SlotProfile.noLocality(DEFAULT_TESTING_PROFILE), fastTimeout)); -- Gitee From 11fa0283d2abf7bfeca1a846a8163174dbfc3080 Mon Sep 17 00:00:00 2001 From: felixzheng Date: Sun, 23 Feb 2020 10:47:03 +0800 Subject: [PATCH 050/885] [hotfix] [k8s] Normalize name or package class of the test classes --- .../flink/kubernetes/{ => cli}/KubernetesSessionCliTest.java | 3 +-- ...abric8ClientTest.java => Fabric8FlinkKubeClientTest.java} | 2 +- .../flink/kubernetes/{ => utils}/KubernetesUtilsTest.java | 5 ++--- 3 files changed, 4 insertions(+), 6 deletions(-) rename flink-kubernetes/src/test/java/org/apache/flink/kubernetes/{ => cli}/KubernetesSessionCliTest.java (99%) rename flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/{Fabric8ClientTest.java => Fabric8FlinkKubeClientTest.java} (99%) rename flink-kubernetes/src/test/java/org/apache/flink/kubernetes/{ => utils}/KubernetesUtilsTest.java (99%) diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesSessionCliTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/cli/KubernetesSessionCliTest.java similarity index 99% rename from flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesSessionCliTest.java rename to flink-kubernetes/src/test/java/org/apache/flink/kubernetes/cli/KubernetesSessionCliTest.java index 90a213ba07..8fa097b362 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesSessionCliTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/cli/KubernetesSessionCliTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.kubernetes; +package org.apache.flink.kubernetes.cli; import org.apache.flink.client.cli.CliArgsException; import org.apache.flink.client.deployment.ClusterClientFactory; @@ -28,7 +28,6 @@ import org.apache.flink.configuration.DeploymentOptions; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.MemorySize; import org.apache.flink.configuration.TaskManagerOptions; -import org.apache.flink.kubernetes.cli.KubernetesSessionCli; import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; import org.apache.flink.kubernetes.executors.KubernetesSessionClusterExecutor; diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8ClientTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java similarity index 99% rename from flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8ClientTest.java rename to flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java index 0a477b9142..18e0485c3b 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8ClientTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java @@ -56,7 +56,7 @@ import static org.junit.Assert.assertTrue; /** * Tests for Fabric implementation of {@link FlinkKubeClient}. */ -public class Fabric8ClientTest extends KubernetesTestBase { +public class Fabric8FlinkKubeClientTest extends KubernetesTestBase { private FlinkKubeClient flinkKubeClient; diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesUtilsTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/utils/KubernetesUtilsTest.java similarity index 99% rename from flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesUtilsTest.java rename to flink-kubernetes/src/test/java/org/apache/flink/kubernetes/utils/KubernetesUtilsTest.java index 1853374fdb..64ed94f08e 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesUtilsTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/utils/KubernetesUtilsTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.kubernetes; +package org.apache.flink.kubernetes.utils; import org.apache.flink.api.common.resources.CPUResource; import org.apache.flink.configuration.BlobServerOptions; @@ -27,7 +27,6 @@ import org.apache.flink.configuration.CoreOptions; import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.configuration.MemorySize; import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; -import org.apache.flink.kubernetes.utils.KubernetesUtils; import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec; import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils; @@ -58,7 +57,7 @@ public class KubernetesUtilsTest extends TestLogger { private static final String java = "$JAVA_HOME/bin/java"; private static final String classpath = "-classpath $FLINK_CLASSPATH"; private static final String jvmOpts = "-Djvm"; - private static final String mainClass = "org.apache.flink.kubernetes.KubernetesUtilsTest"; + private static final String mainClass = "org.apache.flink.kubernetes.utils.KubernetesUtilsTest"; private static final String mainClassArgs = "--job-id=1 -Dtest.key=value"; // Logging variables -- Gitee From 0743b437c764a20b72ba4b14ad1e8f08755c2108 Mon Sep 17 00:00:00 2001 From: felixzheng Date: Sun, 23 Feb 2020 12:38:36 +0800 Subject: [PATCH 051/885] [FLINK-16194][k8s] Remove the existing decorator patterns --- .../KubernetesClusterDescriptor.java | 20 +- .../kubeclient/Fabric8FlinkKubeClient.java | 138 +--------- .../kubeclient/FlinkKubeClient.java | 31 --- .../decorators/ConfigMapDecorator.java | 83 ------ .../kubeclient/decorators/Decorator.java | 45 ---- .../FlinkMasterDeploymentDecorator.java | 170 ------------- .../decorators/InitializerDecorator.java | 73 ------ .../kubeclient/decorators/LabelBuilder.java | 66 ----- .../decorators/OwnerReferenceDecorator.java | 75 ------ .../decorators/ServiceDecorator.java | 105 -------- .../decorators/TaskManagerPodDecorator.java | 118 --------- .../resources/KubernetesConfigMap.java | 33 --- .../resources/KubernetesDeployment.java | 34 --- .../flink/kubernetes/KubernetesTestBase.java | 95 ------- .../Fabric8FlinkKubeClientTest.java | 236 +----------------- 15 files changed, 5 insertions(+), 1317 deletions(-) delete mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/ConfigMapDecorator.java delete mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/Decorator.java delete mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/FlinkMasterDeploymentDecorator.java delete mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitializerDecorator.java delete mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/LabelBuilder.java delete mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/OwnerReferenceDecorator.java delete mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/ServiceDecorator.java delete mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/TaskManagerPodDecorator.java delete mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesConfigMap.java delete mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesDeployment.java diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesClusterDescriptor.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesClusterDescriptor.java index acef01c002..eb62393912 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesClusterDescriptor.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesClusterDescriptor.java @@ -36,7 +36,6 @@ import org.apache.flink.kubernetes.configuration.KubernetesConfigOptionsInternal import org.apache.flink.kubernetes.entrypoint.KubernetesSessionClusterEntrypoint; import org.apache.flink.kubernetes.kubeclient.Endpoint; import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient; -import org.apache.flink.kubernetes.kubeclient.resources.KubernetesService; import org.apache.flink.kubernetes.utils.Constants; import org.apache.flink.kubernetes.utils.KubernetesUtils; import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; @@ -177,24 +176,7 @@ public class KubernetesClusterDescriptor implements ClusterDescriptor { } try { - final KubernetesService internalSvc = client.createInternalService(clusterId).get(); - // Update the service id in Flink config, it will be used for gc. - final String serviceId = internalSvc.getInternalResource().getMetadata().getUid(); - if (serviceId != null) { - flinkConfig.setString(KubernetesConfigOptionsInternal.SERVICE_ID, serviceId); - } else { - throw new ClusterDeploymentException("Get service id failed."); - } - - // Create the rest service when exposed type is not ClusterIp. - final String restSvcExposedType = flinkConfig.getString(KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE); - if (!restSvcExposedType.equals(KubernetesConfigOptions.ServiceExposedType.ClusterIP.toString())) { - client.createRestService(clusterId).get(); - } - - client.createConfigMap(); - client.createFlinkMasterDeployment(clusterSpecification); - + // todo return createClusterClientProvider(clusterId); } catch (Exception e) { client.handleException(e); diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java index 5d8d5e1656..1698e3ab97 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java @@ -18,48 +18,29 @@ package org.apache.flink.kubernetes.kubeclient; -import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.RestOptions; import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; -import org.apache.flink.kubernetes.kubeclient.decorators.ConfigMapDecorator; -import org.apache.flink.kubernetes.kubeclient.decorators.Decorator; -import org.apache.flink.kubernetes.kubeclient.decorators.FlinkMasterDeploymentDecorator; -import org.apache.flink.kubernetes.kubeclient.decorators.InitializerDecorator; -import org.apache.flink.kubernetes.kubeclient.decorators.OwnerReferenceDecorator; -import org.apache.flink.kubernetes.kubeclient.decorators.ServiceDecorator; -import org.apache.flink.kubernetes.kubeclient.decorators.TaskManagerPodDecorator; -import org.apache.flink.kubernetes.kubeclient.resources.ActionWatcher; -import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap; -import org.apache.flink.kubernetes.kubeclient.resources.KubernetesDeployment; import org.apache.flink.kubernetes.kubeclient.resources.KubernetesPod; import org.apache.flink.kubernetes.kubeclient.resources.KubernetesService; import org.apache.flink.kubernetes.utils.Constants; -import org.apache.flink.util.TimeUtils; -import org.apache.flink.util.function.FunctionUtils; -import io.fabric8.kubernetes.api.model.ConfigMap; import io.fabric8.kubernetes.api.model.Pod; import io.fabric8.kubernetes.api.model.Service; import io.fabric8.kubernetes.api.model.ServicePort; -import io.fabric8.kubernetes.api.model.apps.Deployment; import io.fabric8.kubernetes.client.KubernetesClient; import io.fabric8.kubernetes.client.KubernetesClientException; -import io.fabric8.kubernetes.client.Watch; import io.fabric8.kubernetes.client.Watcher; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.annotation.Nullable; -import java.time.Duration; import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -76,100 +57,17 @@ public class Fabric8FlinkKubeClient implements FlinkKubeClient { private final String clusterId; private final String nameSpace; - private final List> configMapDecorators = new ArrayList<>(); - private final List> internalServiceDecorators = new ArrayList<>(); - private final List> restServiceDecorators = new ArrayList<>(); - private final List> flinkMasterDeploymentDecorators = new ArrayList<>(); - private final List> taskManagerPodDecorators = new ArrayList<>(); - public Fabric8FlinkKubeClient(Configuration flinkConfig, KubernetesClient client) { this.flinkConfig = checkNotNull(flinkConfig); this.internalClient = checkNotNull(client); this.clusterId = checkNotNull(flinkConfig.getString(KubernetesConfigOptions.CLUSTER_ID)); this.nameSpace = flinkConfig.getString(KubernetesConfigOptions.NAMESPACE); - - initialize(); - } - - private void initialize() { - this.configMapDecorators.add(new InitializerDecorator<>(Constants.CONFIG_MAP_PREFIX + clusterId)); - this.configMapDecorators.add(new OwnerReferenceDecorator<>()); - this.configMapDecorators.add(new ConfigMapDecorator()); - - this.internalServiceDecorators.add(new InitializerDecorator<>(clusterId)); - this.internalServiceDecorators.add(new ServiceDecorator( - KubernetesConfigOptions.ServiceExposedType.ClusterIP, - false)); - - this.restServiceDecorators.add(new InitializerDecorator<>(clusterId + Constants.FLINK_REST_SERVICE_SUFFIX)); - final String exposedType = flinkConfig.getString(KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE); - this.restServiceDecorators.add(new ServiceDecorator( - KubernetesConfigOptions.ServiceExposedType.valueOf(exposedType), - true)); - this.restServiceDecorators.add(new OwnerReferenceDecorator<>()); - - this.flinkMasterDeploymentDecorators.add(new InitializerDecorator<>(clusterId, Constants.APPS_API_VERSION)); - this.flinkMasterDeploymentDecorators.add(new OwnerReferenceDecorator<>(Constants.APPS_API_VERSION)); - - this.taskManagerPodDecorators.add(new InitializerDecorator<>()); - this.taskManagerPodDecorators.add(new OwnerReferenceDecorator<>()); - } - - @Override - public void createConfigMap() { - KubernetesConfigMap configMap = new KubernetesConfigMap(this.flinkConfig); - - for (Decorator c : this.configMapDecorators) { - configMap = c.decorate(configMap); - } - - LOG.debug("Create config map with data size {}", configMap.getInternalResource().getData().size()); - this.internalClient.configMaps().create(configMap.getInternalResource()); - } - - @Override - public CompletableFuture createInternalService(String clusterId) { - return createService(clusterId, this.internalServiceDecorators); - } - - @Override - public CompletableFuture createRestService(String clusterId) { - return createService(clusterId + Constants.FLINK_REST_SERVICE_SUFFIX, this.restServiceDecorators); - } - - @Override - public void createFlinkMasterDeployment(ClusterSpecification clusterSpecification) { - KubernetesDeployment deployment = new KubernetesDeployment(this.flinkConfig); - - for (Decorator d : this.flinkMasterDeploymentDecorators) { - deployment = d.decorate(deployment); - } - - deployment = new FlinkMasterDeploymentDecorator(clusterSpecification).decorate(deployment); - - LOG.debug("Create Flink Master deployment with spec: {}", deployment.getInternalResource().getSpec()); - - this.internalClient - .apps() - .deployments() - .inNamespace(this.nameSpace) - .create(deployment.getInternalResource()); } @Override public void createTaskManagerPod(TaskManagerPodParameter parameter) { - KubernetesPod pod = new KubernetesPod(this.flinkConfig); - - for (Decorator d : this.taskManagerPodDecorators) { - pod = d.decorate(pod); - } - - pod = new TaskManagerPodDecorator(parameter).decorate(pod); - - LOG.debug("Create TaskManager pod with spec: {}", pod.getInternalResource().getSpec()); - - this.internalClient.pods().inNamespace(this.nameSpace).create(pod.getInternalResource()); + // todo } @Override @@ -290,40 +188,6 @@ public class Fabric8FlinkKubeClient implements FlinkKubeClient { this.internalClient.close(); } - private CompletableFuture createService( - String serviceName, - List> serviceDecorators) { - KubernetesService kubernetesService = new KubernetesService(this.flinkConfig); - for (Decorator d : serviceDecorators) { - kubernetesService = d.decorate(kubernetesService); - } - - LOG.debug("Create service {} with spec: {}", serviceName, kubernetesService.getInternalResource().getSpec()); - - this.internalClient.services().create(kubernetesService.getInternalResource()); - - final ActionWatcher watcher = new ActionWatcher<>( - Watcher.Action.ADDED, - kubernetesService.getInternalResource()); - - final Watch watchConnectionManager = this.internalClient - .services() - .inNamespace(this.nameSpace) - .withName(serviceName) - .watch(watcher); - - final Duration timeout = TimeUtils.parseDuration( - flinkConfig.get(KubernetesConfigOptions.SERVICE_CREATE_TIMEOUT)); - - return CompletableFuture.supplyAsync( - FunctionUtils.uncheckedSupplier(() -> { - final Service createdService = watcher.await(timeout.toMillis(), TimeUnit.MILLISECONDS); - watchConnectionManager.close(); - - return new KubernetesService(this.flinkConfig, createdService); - })); - } - private KubernetesService getService(String serviceName) { final Service service = this .internalClient diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkKubeClient.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkKubeClient.java index 6a84a3e68e..69492b3283 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkKubeClient.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkKubeClient.java @@ -18,7 +18,6 @@ package org.apache.flink.kubernetes.kubeclient; -import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.kubernetes.kubeclient.resources.KubernetesPod; import org.apache.flink.kubernetes.kubeclient.resources.KubernetesService; @@ -26,42 +25,12 @@ import javax.annotation.Nullable; import java.util.List; import java.util.Map; -import java.util.concurrent.CompletableFuture; /** * The client to talk with kubernetes. */ public interface FlinkKubeClient extends AutoCloseable { - /** - * Create kubernetes config map, include flink-conf.yaml, log4j.properties. - */ - void createConfigMap() throws Exception; - - /** - * Create kubernetes service for internal use. This will be set to jobmanager rpc address. - * It is the owner of all resources. After deletion, all other resource will be deleted by gc. - * - * @param clusterId cluster id - * @return A CompletableFuture is returned and could be used to wait for service ready. - */ - CompletableFuture createInternalService(String clusterId) throws Exception; - - /** - * Create kubernetes service for rest port. This will be used by client to interact with flink cluster. - * - * @param clusterId cluster id - * @return A CompletableFuture is returned and could be used to wait for service ready. - */ - CompletableFuture createRestService(String clusterId) throws Exception; - - /** - * Create flink master deployment with replication of 1. - * - * @param clusterSpec {@link ClusterSpecification} to create the flink master deployment. - */ - void createFlinkMasterDeployment(ClusterSpecification clusterSpec); - /** * Create task manager pod. * diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/ConfigMapDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/ConfigMapDecorator.java deleted file mode 100644 index b6705d05fe..0000000000 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/ConfigMapDecorator.java +++ /dev/null @@ -1,83 +0,0 @@ -/* - * 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.kubernetes.kubeclient.decorators; - -import org.apache.flink.client.cli.CliFrontend; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; -import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap; -import org.apache.flink.kubernetes.utils.KubernetesUtils; -import org.apache.flink.util.Preconditions; - -import io.fabric8.kubernetes.api.model.ConfigMap; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.io.FileNotFoundException; -import java.util.HashMap; -import java.util.Map; - -import static org.apache.flink.configuration.GlobalConfiguration.FLINK_CONF_FILENAME; -import static org.apache.flink.kubernetes.utils.Constants.CONFIG_FILE_LOG4J_NAME; -import static org.apache.flink.kubernetes.utils.Constants.CONFIG_FILE_LOGBACK_NAME; - -/** - * Decorate config map for flink configuration and log files. - */ -public class ConfigMapDecorator extends Decorator { - - private static final Logger LOG = LoggerFactory.getLogger(ConfigMapDecorator.class); - - @Override - protected ConfigMap decorateInternalResource(ConfigMap resource, Configuration flinkConfig) { - - final String confDir = CliFrontend.getConfigurationDirectoryFromEnv(); - Preconditions.checkArgument(confDir != null); - - final StringBuilder flinkConfContent = new StringBuilder(); - final Map flinkConfigMap = new HashMap<>(flinkConfig.toMap()); - // Remove some keys should not be taken to jobmanager and taskmanager. - flinkConfigMap.remove(KubernetesConfigOptions.KUBE_CONFIG_FILE.key()); - flinkConfigMap.forEach( - (k, v) -> flinkConfContent.append(k).append(": ").append(v).append(System.lineSeparator()) - ); - - final Map configMap = resource.getData() == null ? new HashMap<>() : resource.getData(); - configMap.put(FLINK_CONF_FILENAME, flinkConfContent.toString()); - - final String log4jFile = confDir + File.separator + CONFIG_FILE_LOG4J_NAME; - try { - final String log4jContent = KubernetesUtils.getContentFromFile(log4jFile); - configMap.put(CONFIG_FILE_LOG4J_NAME, log4jContent); - } catch (FileNotFoundException e) { - LOG.info("File {} will not be added to configMap, {}", log4jFile, e.getMessage()); - } - - final String logbackFile = confDir + File.separator + CONFIG_FILE_LOGBACK_NAME; - try { - final String logbackContent = KubernetesUtils.getContentFromFile(logbackFile); - configMap.put(CONFIG_FILE_LOGBACK_NAME, logbackContent); - } catch (FileNotFoundException e) { - LOG.info("File {} will not be added to configMap, {}", logbackFile, e.getMessage()); - } - resource.setData(configMap); - return resource; - } -} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/Decorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/Decorator.java deleted file mode 100644 index 542845dc00..0000000000 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/Decorator.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * 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.kubernetes.kubeclient.decorators; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.kubernetes.kubeclient.resources.KubernetesResource; - -/** - * Abstract decorator for add features to resource such as deployment/pod/service. - */ -public abstract class Decorator> { - - /** - * Decorate the internal resource. - */ - protected abstract R decorateInternalResource(R resource, Configuration flinkConfig); - - /** - * Extract real resource from resource, decorate and put it back. - */ - public T decorate(T resource) { - - R internalResource = resource.getInternalResource(); - R decoratedInternalResource = decorateInternalResource(internalResource, resource.getFlinkConfig()); - resource.setInternalResource(decoratedInternalResource); - - return resource; - } -} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/FlinkMasterDeploymentDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/FlinkMasterDeploymentDecorator.java deleted file mode 100644 index 05d25baf88..0000000000 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/FlinkMasterDeploymentDecorator.java +++ /dev/null @@ -1,170 +0,0 @@ -/* - * 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.kubernetes.kubeclient.decorators; - -import org.apache.flink.client.cli.CliFrontend; -import org.apache.flink.client.deployment.ClusterSpecification; -import org.apache.flink.configuration.BlobServerOptions; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.JobManagerOptions; -import org.apache.flink.configuration.ResourceManagerOptions; -import org.apache.flink.configuration.RestOptions; -import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; -import org.apache.flink.kubernetes.configuration.KubernetesConfigOptionsInternal; -import org.apache.flink.kubernetes.kubeclient.resources.KubernetesDeployment; -import org.apache.flink.kubernetes.utils.Constants; -import org.apache.flink.kubernetes.utils.KubernetesUtils; -import org.apache.flink.runtime.clusterframework.BootstrapTools; - -import io.fabric8.kubernetes.api.model.Container; -import io.fabric8.kubernetes.api.model.ContainerBuilder; -import io.fabric8.kubernetes.api.model.ContainerPortBuilder; -import io.fabric8.kubernetes.api.model.EnvVar; -import io.fabric8.kubernetes.api.model.EnvVarBuilder; -import io.fabric8.kubernetes.api.model.EnvVarSourceBuilder; -import io.fabric8.kubernetes.api.model.LocalObjectReference; -import io.fabric8.kubernetes.api.model.PodSpec; -import io.fabric8.kubernetes.api.model.PodSpecBuilder; -import io.fabric8.kubernetes.api.model.ResourceRequirements; -import io.fabric8.kubernetes.api.model.Volume; -import io.fabric8.kubernetes.api.model.apps.Deployment; -import io.fabric8.kubernetes.api.model.apps.DeploymentSpecBuilder; - -import java.io.File; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -import static org.apache.flink.kubernetes.configuration.KubernetesConfigOptions.CONTAINER_IMAGE_PULL_SECRETS; -import static org.apache.flink.kubernetes.utils.Constants.API_VERSION; -import static org.apache.flink.kubernetes.utils.Constants.ENV_FLINK_POD_IP_ADDRESS; -import static org.apache.flink.kubernetes.utils.Constants.POD_IP_FIELD_PATH; -import static org.apache.flink.util.Preconditions.checkArgument; -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * Flink master specific deployment configuration. - */ -public class FlinkMasterDeploymentDecorator extends Decorator { - - private static final String CONTAINER_NAME = "flink-job-manager"; - - private final ClusterSpecification clusterSpecification; - - public FlinkMasterDeploymentDecorator(ClusterSpecification clusterSpecification) { - this.clusterSpecification = clusterSpecification; - } - - @Override - protected Deployment decorateInternalResource(Deployment deployment, Configuration flinkConfig) { - final String clusterId = flinkConfig.getString(KubernetesConfigOptions.CLUSTER_ID); - checkNotNull(clusterId, "ClusterId must be specified!"); - - final int blobServerPort = KubernetesUtils.parsePort(flinkConfig, BlobServerOptions.PORT); - checkArgument(blobServerPort > 0, "%s should not be 0.", BlobServerOptions.PORT.key()); - - final String mainClass = flinkConfig.getString(KubernetesConfigOptionsInternal.ENTRY_POINT_CLASS); - checkNotNull(mainClass, "Main class must be specified!"); - - final String confDir = CliFrontend.getConfigurationDirectoryFromEnv(); - final boolean hasLogback = new File(confDir, Constants.CONFIG_FILE_LOGBACK_NAME).exists(); - final boolean hasLog4j = new File(confDir, Constants.CONFIG_FILE_LOG4J_NAME).exists(); - - final Map labels = new LabelBuilder() - .withExist(deployment.getMetadata().getLabels()) - .withJobManagerComponent() - .toLabels(); - - deployment.getMetadata().setLabels(labels); - - final Volume configMapVolume = KubernetesUtils.getConfigMapVolume(clusterId, hasLogback, hasLog4j); - - final Container container = createJobManagerContainer(flinkConfig, mainClass, hasLogback, hasLog4j, blobServerPort); - - final String serviceAccount = flinkConfig.getString(KubernetesConfigOptions.JOB_MANAGER_SERVICE_ACCOUNT); - - final LocalObjectReference[] imagePullSecrets = KubernetesUtils.parseImagePullSecrets(flinkConfig.get(CONTAINER_IMAGE_PULL_SECRETS)); - - final PodSpec podSpec = new PodSpecBuilder() - .withServiceAccountName(serviceAccount) - .withVolumes(configMapVolume) - .withContainers(container) - .withImagePullSecrets(imagePullSecrets) - .build(); - - deployment.setSpec(new DeploymentSpecBuilder() - .withReplicas(1) - .withNewTemplate().withNewMetadata().withLabels(labels).endMetadata() - .withSpec(podSpec).endTemplate() - .withNewSelector().addToMatchLabels(labels).endSelector().build()); - return deployment; - } - - private Container createJobManagerContainer( - Configuration flinkConfig, - String mainClass, - boolean hasLogback, - boolean hasLog4j, - int blobServerPort) { - final String flinkConfDirInPod = flinkConfig.getString(KubernetesConfigOptions.FLINK_CONF_DIR); - final String logDirInPod = flinkConfig.getString(KubernetesConfigOptions.FLINK_LOG_DIR); - final String startCommand = KubernetesUtils.getJobManagerStartCommand( - flinkConfig, - clusterSpecification.getMasterMemoryMB(), - flinkConfDirInPod, - logDirInPod, - hasLogback, - hasLog4j, - mainClass, - null); - - final ResourceRequirements requirements = KubernetesUtils.getResourceRequirements( - clusterSpecification.getMasterMemoryMB(), - flinkConfig.getDouble(KubernetesConfigOptions.JOB_MANAGER_CPU)); - - return new ContainerBuilder() - .withName(CONTAINER_NAME) - .withCommand(flinkConfig.getString(KubernetesConfigOptions.KUBERNETES_ENTRY_PATH)) - .withArgs(Arrays.asList("/bin/bash", "-c", startCommand)) - .withImage(flinkConfig.getString(KubernetesConfigOptions.CONTAINER_IMAGE)) - .withImagePullPolicy(flinkConfig.getString(KubernetesConfigOptions.CONTAINER_IMAGE_PULL_POLICY)) - .withResources(requirements) - .withPorts(Arrays.asList( - new ContainerPortBuilder().withContainerPort(flinkConfig.getInteger(RestOptions.PORT)).build(), - new ContainerPortBuilder().withContainerPort(flinkConfig.getInteger(JobManagerOptions.PORT)).build(), - new ContainerPortBuilder().withContainerPort(blobServerPort).build())) - .withEnv(buildEnvForContainer(flinkConfig)) - .withVolumeMounts(KubernetesUtils.getConfigMapVolumeMount(flinkConfDirInPod, hasLogback, hasLog4j)) - .build(); - } - - private List buildEnvForContainer(Configuration flinkConfig) { - List envList = - BootstrapTools.getEnvironmentVariables(ResourceManagerOptions.CONTAINERIZED_MASTER_ENV_PREFIX, flinkConfig) - .entrySet() - .stream() - .map(kv -> new EnvVar(kv.getKey(), kv.getValue(), null)).collect(Collectors.toList()); - envList.add(new EnvVarBuilder() - .withName(ENV_FLINK_POD_IP_ADDRESS) - .withValueFrom(new EnvVarSourceBuilder().withNewFieldRef(API_VERSION, POD_IP_FIELD_PATH).build()) - .build()); - return envList; - } -} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitializerDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitializerDecorator.java deleted file mode 100644 index 89a3139205..0000000000 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitializerDecorator.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * 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.kubernetes.kubeclient.decorators; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; -import org.apache.flink.kubernetes.kubeclient.resources.KubernetesResource; -import org.apache.flink.kubernetes.utils.Constants; -import org.apache.flink.util.Preconditions; - -import io.fabric8.kubernetes.api.model.HasMetadata; -import io.fabric8.kubernetes.api.model.ObjectMeta; - -import java.util.Map; - -/** - * Initial setup for a kubernetes resource. - */ -public class InitializerDecorator > extends Decorator { - - private final String name; - - private final String apiVersion; - - public InitializerDecorator() { - this(null, Constants.API_VERSION); - } - - public InitializerDecorator(String name) { - this(name, Constants.API_VERSION); - } - - public InitializerDecorator(String name, String apiVersion) { - this.name = name; - this.apiVersion = apiVersion; - } - - @Override - protected R decorateInternalResource(R resource, Configuration flinkConfig) { - final String clusterId = flinkConfig.getString(KubernetesConfigOptions.CLUSTER_ID); - Preconditions.checkNotNull(clusterId, "ClusterId must be specified!"); - - final Map labels = new LabelBuilder() - .withCommon() - .withClusterId(clusterId) - .toLabels(); - - final ObjectMeta meta = new ObjectMeta(); - meta.setName(name == null ? clusterId : name); - meta.setLabels(labels); - meta.setNamespace(flinkConfig.getString(KubernetesConfigOptions.NAMESPACE)); - - resource.setApiVersion(apiVersion); - resource.setMetadata(meta); - return resource; - } -} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/LabelBuilder.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/LabelBuilder.java deleted file mode 100644 index 6318f14dac..0000000000 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/LabelBuilder.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * 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.kubernetes.kubeclient.decorators; - -import org.apache.flink.kubernetes.utils.Constants; - -import java.util.HashMap; -import java.util.Map; - -/** - * Builder to create label for jobmanager and taskmanager. - */ -public class LabelBuilder { - - private final Map labels; - - public LabelBuilder() { - labels = new HashMap<>(); - } - - private LabelBuilder withLabel(String key, String value) { - this.labels.put(key, value); - return this; - } - - public LabelBuilder withCommon() { - return this.withLabel(Constants.LABEL_TYPE_KEY, Constants.LABEL_TYPE_NATIVE_TYPE); - } - - public LabelBuilder withExist(Map labels) { - this.labels.putAll(labels); - return this; - } - - public LabelBuilder withClusterId(String id) { - return this.withLabel(Constants.LABEL_APP_KEY, id); - } - - public LabelBuilder withJobManagerComponent() { - return this.withLabel(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_JOB_MANAGER); - } - - public LabelBuilder withTaskManagerComponent() { - return this.withLabel(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_TASK_MANAGER); - } - - public Map toLabels() { - return labels; - } -} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/OwnerReferenceDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/OwnerReferenceDecorator.java deleted file mode 100644 index a998ca77b5..0000000000 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/OwnerReferenceDecorator.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * 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.kubernetes.kubeclient.decorators; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; -import org.apache.flink.kubernetes.configuration.KubernetesConfigOptionsInternal; -import org.apache.flink.kubernetes.kubeclient.resources.KubernetesResource; -import org.apache.flink.kubernetes.utils.Constants; -import org.apache.flink.util.Preconditions; - -import io.fabric8.kubernetes.api.model.HasMetadata; -import io.fabric8.kubernetes.api.model.ObjectMeta; -import io.fabric8.kubernetes.api.model.OwnerReferenceBuilder; - -import java.util.Collections; - -/** - * Set kubernetes resource owner reference for GC. - */ -public class OwnerReferenceDecorator> extends Decorator { - - private final String apiVersion; - - public OwnerReferenceDecorator() { - this(Constants.API_VERSION); - } - - public OwnerReferenceDecorator(String apiVersion) { - this.apiVersion = apiVersion; - } - - @Override - protected R decorateInternalResource(R resource, Configuration flinkConfig) { - final String clusterId = flinkConfig.getString(KubernetesConfigOptions.CLUSTER_ID); - Preconditions.checkNotNull(clusterId, "ClusterId must be specified!"); - - final String serviceId = flinkConfig.getString(KubernetesConfigOptionsInternal.SERVICE_ID); - Preconditions.checkNotNull(serviceId, "Service id must be specified!"); - - if (resource.getMetadata() == null) { - resource.setMetadata(new ObjectMeta()); - } - - resource.getMetadata().setOwnerReferences( - Collections.singletonList( - new OwnerReferenceBuilder() - .withName(clusterId) - .withController(true) - .withBlockOwnerDeletion(true) - .withKind("service") - .withApiVersion(apiVersion) - .withUid(serviceId) - .build() - )); - - return resource; - } -} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/ServiceDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/ServiceDecorator.java deleted file mode 100644 index f8c449345f..0000000000 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/ServiceDecorator.java +++ /dev/null @@ -1,105 +0,0 @@ -/* - * 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.kubernetes.kubeclient.decorators; - -import org.apache.flink.configuration.BlobServerOptions; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.JobManagerOptions; -import org.apache.flink.configuration.RestOptions; -import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; -import org.apache.flink.kubernetes.kubeclient.resources.KubernetesService; -import org.apache.flink.kubernetes.utils.Constants; - -import io.fabric8.kubernetes.api.model.Service; -import io.fabric8.kubernetes.api.model.ServicePort; -import io.fabric8.kubernetes.api.model.ServicePortBuilder; -import io.fabric8.kubernetes.api.model.ServiceSpec; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -/** - * Setup services port. - */ -public class ServiceDecorator extends Decorator { - - private final KubernetesConfigOptions.ServiceExposedType exposedType; - - private final boolean onlyRestPort; - - public ServiceDecorator() { - this(false); - } - - public ServiceDecorator(boolean onlyRestPort) { - this(KubernetesConfigOptions.ServiceExposedType.LoadBalancer, onlyRestPort); - } - - public ServiceDecorator(KubernetesConfigOptions.ServiceExposedType exposedType, boolean onlyRestPort) { - this.exposedType = exposedType; - this.onlyRestPort = onlyRestPort; - } - - private ServicePort getServicePort(String name, int port) { - return new ServicePortBuilder() - .withName(name) - .withPort(port) - .build(); - } - - private String getPortName(String portName){ - return portName.replace('.', '-'); - } - - @Override - protected Service decorateInternalResource(Service resource, Configuration flinkConfig) { - - final ServiceSpec spec = resource.getSpec() != null ? resource.getSpec() : new ServiceSpec(); - - spec.setType(exposedType.toString()); - - final List servicePorts = new ArrayList<>(); - servicePorts.add(getServicePort( - getPortName(RestOptions.PORT.key()), - flinkConfig.getInteger(RestOptions.PORT))); - - if (!onlyRestPort) { - servicePorts.add(getServicePort( - getPortName(JobManagerOptions.PORT.key()), - flinkConfig.getInteger(JobManagerOptions.PORT))); - servicePorts.add(getServicePort( - getPortName(BlobServerOptions.PORT.key()), - Constants.BLOB_SERVER_PORT)); - } - - spec.setPorts(servicePorts); - - final Map labels = new LabelBuilder() - .withExist(resource.getMetadata().getLabels()) - .withJobManagerComponent() - .toLabels(); - - spec.setSelector(labels); - - resource.setSpec(spec); - - return resource; - } -} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/TaskManagerPodDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/TaskManagerPodDecorator.java deleted file mode 100644 index b939732d90..0000000000 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/TaskManagerPodDecorator.java +++ /dev/null @@ -1,118 +0,0 @@ -/* - * 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.kubernetes.kubeclient.decorators; - -import org.apache.flink.client.cli.CliFrontend; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.TaskManagerOptions; -import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; -import org.apache.flink.kubernetes.kubeclient.TaskManagerPodParameter; -import org.apache.flink.kubernetes.kubeclient.resources.KubernetesPod; -import org.apache.flink.kubernetes.utils.Constants; -import org.apache.flink.kubernetes.utils.KubernetesUtils; - -import io.fabric8.kubernetes.api.model.Container; -import io.fabric8.kubernetes.api.model.ContainerBuilder; -import io.fabric8.kubernetes.api.model.ContainerPortBuilder; -import io.fabric8.kubernetes.api.model.EnvVar; -import io.fabric8.kubernetes.api.model.LocalObjectReference; -import io.fabric8.kubernetes.api.model.Pod; -import io.fabric8.kubernetes.api.model.PodSpecBuilder; -import io.fabric8.kubernetes.api.model.Volume; - -import java.io.File; -import java.util.Map; -import java.util.stream.Collectors; - -import static org.apache.flink.kubernetes.configuration.KubernetesConfigOptions.CONTAINER_IMAGE_PULL_SECRETS; -import static org.apache.flink.util.Preconditions.checkArgument; -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * Task manager specific pod configuration. - */ -public class TaskManagerPodDecorator extends Decorator { - - private static final String CONTAINER_NAME = "flink-task-manager"; - - private final TaskManagerPodParameter parameter; - - public TaskManagerPodDecorator(TaskManagerPodParameter parameters) { - checkNotNull(parameters); - this.parameter = parameters; - } - - @Override - protected Pod decorateInternalResource(Pod pod, Configuration flinkConfig) { - - final String clusterId = flinkConfig.getString(KubernetesConfigOptions.CLUSTER_ID); - checkNotNull(clusterId, "ClusterId must be specified!"); - - final int taskManagerRpcPort = KubernetesUtils.parsePort(flinkConfig, TaskManagerOptions.RPC_PORT); - checkArgument(taskManagerRpcPort > 0, "%s should not be 0.", TaskManagerOptions.RPC_PORT.key()); - - final String confDir = CliFrontend.getConfigurationDirectoryFromEnv(); - final boolean hasLogback = new File(confDir, Constants.CONFIG_FILE_LOGBACK_NAME).exists(); - final boolean hasLog4j = new File(confDir, Constants.CONFIG_FILE_LOG4J_NAME).exists(); - - final Map labels = new LabelBuilder() - .withExist(pod.getMetadata().getLabels()) - .withTaskManagerComponent() - .toLabels(); - - pod.getMetadata().setLabels(labels); - pod.getMetadata().setName(this.parameter.getPodName()); - - final Volume configMapVolume = KubernetesUtils.getConfigMapVolume(clusterId, hasLogback, hasLog4j); - - final LocalObjectReference[] imagePullSecrets = KubernetesUtils.parseImagePullSecrets(flinkConfig.get(CONTAINER_IMAGE_PULL_SECRETS)); - - pod.setSpec(new PodSpecBuilder() - .withVolumes(configMapVolume) - .withContainers(createTaskManagerContainer(flinkConfig, hasLogback, hasLog4j, taskManagerRpcPort)) - .withImagePullSecrets(imagePullSecrets) - .build()); - return pod; - } - - private Container createTaskManagerContainer( - Configuration flinkConfig, - boolean hasLogBack, - boolean hasLog4j, - int taskManagerRpcPort) { - final String flinkConfDirInPod = flinkConfig.getString(KubernetesConfigOptions.FLINK_CONF_DIR); - return new ContainerBuilder() - .withName(CONTAINER_NAME) - .withCommand(flinkConfig.getString(KubernetesConfigOptions.KUBERNETES_ENTRY_PATH)) - .withArgs(this.parameter.getArgs()) - .withImage(flinkConfig.getString(KubernetesConfigOptions.CONTAINER_IMAGE)) - .withImagePullPolicy(flinkConfig.getString(KubernetesConfigOptions.CONTAINER_IMAGE_PULL_POLICY)) - .withResources(KubernetesUtils.getResourceRequirements( - parameter.getTaskManagerMemoryInMB(), - parameter.getTaskManagerCpus())) - .withPorts(new ContainerPortBuilder().withContainerPort(taskManagerRpcPort).build()) - .withEnv(this.parameter.getEnvironmentVariables() - .entrySet() - .stream() - .map(kv -> new EnvVar(kv.getKey(), kv.getValue(), null)) - .collect(Collectors.toList())) - .withVolumeMounts(KubernetesUtils.getConfigMapVolumeMount(flinkConfDirInPod, hasLogBack, hasLog4j)) - .build(); - } -} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesConfigMap.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesConfigMap.java deleted file mode 100644 index 18f42e3702..0000000000 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesConfigMap.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * 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.kubernetes.kubeclient.resources; - -import org.apache.flink.configuration.Configuration; - -import io.fabric8.kubernetes.api.model.ConfigMap; - -/** - * Represent KubernetesConfigMap resource in kubernetes. - */ -public class KubernetesConfigMap extends KubernetesResource { - - public KubernetesConfigMap(Configuration flinkConfig) { - super(flinkConfig, new ConfigMap()); - } -} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesDeployment.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesDeployment.java deleted file mode 100644 index b4efd5990d..0000000000 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesDeployment.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * 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.kubernetes.kubeclient.resources; - -import org.apache.flink.configuration.Configuration; - -import io.fabric8.kubernetes.api.model.apps.Deployment; -import io.fabric8.kubernetes.api.model.apps.DeploymentBuilder; - -/** - * Represent KubernetesDeployment resource in kubernetes. - */ -public class KubernetesDeployment extends KubernetesResource { - - public KubernetesDeployment(Configuration flinkConfig) { - super(flinkConfig, new DeploymentBuilder().withNewMetadata().endMetadata().build()); - } -} diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesTestBase.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesTestBase.java index 32ce9e8310..07663ae469 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesTestBase.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesTestBase.java @@ -27,34 +27,19 @@ import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; import org.apache.flink.kubernetes.configuration.KubernetesConfigOptionsInternal; import org.apache.flink.kubernetes.kubeclient.Fabric8FlinkKubeClient; import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient; -import org.apache.flink.kubernetes.kubeclient.decorators.Decorator; -import org.apache.flink.kubernetes.kubeclient.decorators.InitializerDecorator; -import org.apache.flink.kubernetes.kubeclient.decorators.OwnerReferenceDecorator; -import org.apache.flink.kubernetes.kubeclient.decorators.ServiceDecorator; -import org.apache.flink.kubernetes.kubeclient.resources.KubernetesService; import org.apache.flink.kubernetes.utils.Constants; import org.apache.flink.runtime.clusterframework.BootstrapTools; import org.apache.flink.test.util.TestBaseUtils; import org.apache.flink.util.TestLogger; -import io.fabric8.kubernetes.api.model.LoadBalancerIngress; -import io.fabric8.kubernetes.api.model.LoadBalancerStatus; -import io.fabric8.kubernetes.api.model.Service; -import io.fabric8.kubernetes.api.model.ServiceStatusBuilder; -import io.fabric8.kubernetes.api.model.WatchEvent; import io.fabric8.kubernetes.client.KubernetesClient; import org.junit.Before; import org.junit.Rule; import org.junit.rules.TemporaryFolder; -import javax.annotation.Nullable; - import java.io.File; import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; -import java.util.List; import java.util.Map; /** @@ -79,8 +64,6 @@ public class KubernetesTestBase extends TestLogger { protected static final String MOCK_SERVICE_ID = "mock-uuid-of-service"; - protected static final String MOCK_SERVICE_HOST_NAME = "mock-host-name-of-service"; - protected static final String MOCK_SERVICE_IP = "192.168.0.1"; protected static final String FLINK_MASTER_ENV_KEY = "LD_LIBRARY_PATH"; @@ -105,11 +88,6 @@ public class KubernetesTestBase extends TestLogger { Map map = new HashMap<>(); map.put(ConfigConstants.ENV_FLINK_CONF_DIR, flinkConfDir.toString()); TestBaseUtils.setEnv(map); - - // Set mock requests. - mockInternalServiceActionWatch(); - mockRestServiceActionWatcher(CLUSTER_ID); - mockGetRestService(CLUSTER_ID, MOCK_SERVICE_HOST_NAME, MOCK_SERVICE_IP); } protected FlinkKubeClient getFabric8FlinkKubeClient(){ @@ -124,79 +102,6 @@ public class KubernetesTestBase extends TestLogger { return server.getClient().inNamespace(NAMESPACE); } - protected void mockRestServiceActionWatcher(String clusterId) { - String serviceName = clusterId + Constants.FLINK_REST_SERVICE_SUFFIX; - - String path = String.format("/api/v1/namespaces/%s/services?fieldSelector=metadata.name%%3D%s&watch=true", - NAMESPACE, serviceName); - server.expect() - .withPath(path) - .andUpgradeToWebSocket() - .open() - .waitFor(1000) - .andEmit(new WatchEvent(getMockRestService(MOCK_SERVICE_HOST_NAME, MOCK_SERVICE_IP), "ADDED")) - .done() - .once(); - } - - private void mockInternalServiceActionWatch() { - - String path = String.format("/api/v1/namespaces/%s/services?fieldSelector=metadata.name%%3D%s&watch=true", - NAMESPACE, CLUSTER_ID); - server.expect() - .withPath(path) - .andUpgradeToWebSocket() - .open() - .waitFor(1000) - .andEmit(new WatchEvent(getMockInternalService(), "ADDED")) - .done() - .once(); - } - - protected void mockGetRestService(String clusterId, @Nullable String hostname, @Nullable String ip) { - String serviceName = clusterId + Constants.FLINK_REST_SERVICE_SUFFIX; - - String path = String.format("/api/v1/namespaces/%s/services/%s", NAMESPACE, serviceName); - server.expect() - .withPath(path) - .andReturn(200, getMockRestService(hostname, ip)) - .always(); - } - - private Service getMockRestService(@Nullable String hostname, @Nullable String ip) { - List> restServiceDecorators = new ArrayList<>(); - restServiceDecorators.add(new InitializerDecorator<>(CLUSTER_ID + Constants.FLINK_REST_SERVICE_SUFFIX)); - String exposedType = FLINK_CONFIG.getString(KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE); - restServiceDecorators.add(new ServiceDecorator( - KubernetesConfigOptions.ServiceExposedType.valueOf(exposedType), true)); - restServiceDecorators.add(new OwnerReferenceDecorator<>()); - - KubernetesService kubernetesService = new KubernetesService(FLINK_CONFIG); - for (Decorator d : restServiceDecorators) { - kubernetesService = d.decorate(kubernetesService); - } - - Service service = kubernetesService.getInternalResource(); - service.setStatus(new ServiceStatusBuilder() - .withLoadBalancer(new LoadBalancerStatus(Collections.singletonList( - new LoadBalancerIngress(hostname, ip)))).build()); - return service; - } - - private Service getMockInternalService() { - List> internalServiceDecorators = new ArrayList<>(); - internalServiceDecorators.add(new InitializerDecorator<>(CLUSTER_ID)); - internalServiceDecorators.add(new ServiceDecorator( - KubernetesConfigOptions.ServiceExposedType.ClusterIP, true)); - - KubernetesService kubernetesService = new KubernetesService(FLINK_CONFIG); - for (Decorator d : internalServiceDecorators) { - kubernetesService = d.decorate(kubernetesService); - } - kubernetesService.getInternalResource().getMetadata().setUid(MOCK_SERVICE_ID); - return kubernetesService.getInternalResource(); - } - protected Map getCommonLabels() { Map labels = new HashMap<>(); labels.put(Constants.LABEL_TYPE_KEY, Constants.LABEL_TYPE_NATIVE_TYPE); diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java index 18e0485c3b..8537d14d58 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java @@ -18,40 +18,13 @@ package org.apache.flink.kubernetes.kubeclient; -import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.kubernetes.KubernetesTestBase; -import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; -import org.apache.flink.kubernetes.utils.Constants; -import io.fabric8.kubernetes.api.model.ConfigMap; -import io.fabric8.kubernetes.api.model.Container; -import io.fabric8.kubernetes.api.model.ContainerPort; -import io.fabric8.kubernetes.api.model.EnvVar; -import io.fabric8.kubernetes.api.model.EnvVarBuilder; -import io.fabric8.kubernetes.api.model.Pod; -import io.fabric8.kubernetes.api.model.PodSpec; -import io.fabric8.kubernetes.api.model.Service; -import io.fabric8.kubernetes.api.model.ServicePort; -import io.fabric8.kubernetes.api.model.apps.Deployment; import io.fabric8.kubernetes.client.KubernetesClient; -import org.hamcrest.Matchers; import org.junit.Before; import org.junit.Test; -import java.io.File; import java.io.IOException; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -import static org.apache.flink.configuration.GlobalConfiguration.FLINK_CONF_FILENAME; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.assertTrue; /** * Tests for Fabric implementation of {@link FlinkKubeClient}. @@ -69,221 +42,18 @@ public class Fabric8FlinkKubeClientTest extends KubernetesTestBase { kubeClient = getKubeClient(); } - @Test - public void testCreateConfigMap() throws Exception { - flinkKubeClient.createConfigMap(); - - final List configMaps = kubeClient.configMaps().list().getItems(); - assertEquals(1, configMaps.size()); - - // Check labels - final ConfigMap configMap = configMaps.get(0); - assertEquals(Constants.CONFIG_MAP_PREFIX + CLUSTER_ID, configMap.getMetadata().getName()); - final Map labels = getCommonLabels(); - assertEquals(labels, configMap.getMetadata().getLabels()); - - // Check owner reference - assertEquals(1, configMap.getMetadata().getOwnerReferences().size()); - assertEquals(MOCK_SERVICE_ID, configMap.getMetadata().getOwnerReferences().get(0).getUid()); - - // Check data - assertEquals(1, configMap.getData().size()); - assertThat(configMap.getData().get(FLINK_CONF_FILENAME), - Matchers.containsString(KubernetesConfigOptions.CLUSTER_ID.key())); - assertThat(configMap.getData().get(FLINK_CONF_FILENAME), - Matchers.containsString(KubernetesConfigOptions.CONTAINER_IMAGE.key())); - } - - @Test - public void testCreateInternalService() throws Exception { - flinkKubeClient.createInternalService(CLUSTER_ID); - - final List services = kubeClient.services().list().getItems(); - assertEquals(1, services.size()); - - final Service service = services.get(0); - assertEquals(CLUSTER_ID, service.getMetadata().getName()); - final Map labels = getCommonLabels(); - assertEquals(labels, service.getMetadata().getLabels()); - - assertEquals(0, service.getMetadata().getOwnerReferences().size()); - - assertEquals(KubernetesConfigOptions.ServiceExposedType.ClusterIP.toString(), service.getSpec().getType()); - - // The selector labels should contain jobmanager component - labels.put(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_JOB_MANAGER); - assertEquals(labels, service.getSpec().getSelector()); - - assertThat(service.getSpec().getPorts().stream().map(ServicePort::getPort).collect(Collectors.toList()), - Matchers.hasItems(8081, 6123, 6124)); - - // Internal service will be deleted. Other resources are deleted by gc. - flinkKubeClient.stopAndCleanupCluster(CLUSTER_ID); - assertEquals(0, kubeClient.services().list().getItems().size()); - } - - @Test - public void testCreateRestService() throws Exception { - flinkKubeClient.createRestService(CLUSTER_ID).get(); - - final List services = kubeClient.services().list().getItems(); - assertEquals(1, services.size()); - - final Service service = services.get(0); - assertEquals(CLUSTER_ID + Constants.FLINK_REST_SERVICE_SUFFIX, service.getMetadata().getName()); - final Map labels = getCommonLabels(); - assertEquals(labels, service.getMetadata().getLabels()); - - assertEquals(1, service.getMetadata().getOwnerReferences().size()); - assertEquals(MOCK_SERVICE_ID, service.getMetadata().getOwnerReferences().get(0).getUid()); - - assertEquals(KubernetesConfigOptions.ServiceExposedType.LoadBalancer.toString(), service.getSpec().getType()); - - labels.put(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_JOB_MANAGER); - assertEquals(labels, service.getSpec().getSelector()); - - assertThat(service.getSpec().getPorts().stream().map(ServicePort::getPort).collect(Collectors.toList()), - Matchers.hasItems(8081)); - - final Endpoint endpoint = flinkKubeClient.getRestEndpoint(CLUSTER_ID); - assertEquals(MOCK_SERVICE_IP, endpoint.getAddress()); - assertEquals(8081, endpoint.getPort()); - } - - @Test - public void testCreateFlinkMasterDeployment() { - final ClusterSpecification clusterSpecification = new ClusterSpecification.ClusterSpecificationBuilder() - .setMasterMemoryMB(1234) - .createClusterSpecification(); - - flinkKubeClient.createFlinkMasterDeployment(clusterSpecification); - - final List deployments = kubeClient.apps().deployments().list().getItems(); - assertEquals(1, deployments.size()); - - final Deployment deployment = deployments.get(0); - assertEquals(CLUSTER_ID, deployment.getMetadata().getName()); - final Map labels = getCommonLabels(); - labels.put(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_JOB_MANAGER); - assertEquals(labels, deployment.getMetadata().getLabels()); - - assertEquals(1, deployment.getMetadata().getOwnerReferences().size()); - assertEquals(MOCK_SERVICE_ID, deployment.getMetadata().getOwnerReferences().get(0).getUid()); - - final PodSpec jmPodSpec = deployment.getSpec().getTemplate().getSpec(); - assertEquals("default", jmPodSpec.getServiceAccountName()); - assertEquals(1, jmPodSpec.getVolumes().size()); - assertEquals(1, jmPodSpec.getContainers().size()); - final Container jmContainer = jmPodSpec.getContainers().get(0); - - assertEquals(clusterSpecification.getMasterMemoryMB() + Constants.RESOURCE_UNIT_MB, - jmContainer.getResources().getRequests().get(Constants.RESOURCE_NAME_MEMORY).getAmount()); - assertEquals(clusterSpecification.getMasterMemoryMB() + Constants.RESOURCE_UNIT_MB, - jmContainer.getResources().getLimits().get(Constants.RESOURCE_NAME_MEMORY).getAmount()); - - assertThat(jmContainer.getPorts().stream().map(ContainerPort::getContainerPort).collect(Collectors.toList()), - Matchers.hasItems(8081, 6123, 6124)); - - assertEquals(1, jmContainer.getVolumeMounts().size()); - final String mountPath = FLINK_CONFIG.getString(KubernetesConfigOptions.FLINK_CONF_DIR); - assertEquals(new File(mountPath, FLINK_CONF_FILENAME).getPath(), - jmContainer.getVolumeMounts().get(0).getMountPath()); - assertEquals(FLINK_CONF_FILENAME, jmContainer.getVolumeMounts().get(0).getSubPath()); - - EnvVar masterEnv = new EnvVar(FLINK_MASTER_ENV_KEY, FLINK_MASTER_ENV_VALUE, null); - assertTrue( - "Environment " + masterEnv.toString() + " should be set.", - jmContainer.getEnv().contains(masterEnv)); - } - @Test public void testCreateTaskManagerPod() { - final String podName = "taskmanager-1"; - final List commands = Arrays.asList("/bin/bash", "-c", "start-command-of-taskmanager"); - final int tmMem = 1234; - final double tmCpu = 1.2; - final Map env = new HashMap<>(); - env.put("RESOURCE_ID", podName); - TaskManagerPodParameter parameter = new TaskManagerPodParameter( - podName, - commands, - tmMem, - tmCpu, - env); - flinkKubeClient.createTaskManagerPod(parameter); - - final List pods = kubeClient.pods().list().getItems(); - assertEquals(1, pods.size()); - - final Pod tmPod = pods.get(0); - assertEquals(podName, tmPod.getMetadata().getName()); - final Map labels = getCommonLabels(); - labels.put(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_TASK_MANAGER); - assertEquals(labels, tmPod.getMetadata().getLabels()); - - assertEquals(1, tmPod.getMetadata().getOwnerReferences().size()); - assertEquals(MOCK_SERVICE_ID, tmPod.getMetadata().getOwnerReferences().get(0).getUid()); - - assertEquals(1, tmPod.getSpec().getContainers().size()); - final Container tmContainer = tmPod.getSpec().getContainers().get(0); - assertEquals(CONTAINER_IMAGE, tmContainer.getImage()); - assertEquals(commands, tmContainer.getArgs()); - - assertEquals(tmMem + Constants.RESOURCE_UNIT_MB, - tmContainer.getResources().getRequests().get(Constants.RESOURCE_NAME_MEMORY).getAmount()); - assertEquals(tmMem + Constants.RESOURCE_UNIT_MB, - tmContainer.getResources().getLimits().get(Constants.RESOURCE_NAME_MEMORY).getAmount()); - assertEquals(String.valueOf(tmCpu), - tmContainer.getResources().getRequests().get(Constants.RESOURCE_NAME_CPU).getAmount()); - assertEquals(String.valueOf(tmCpu), - tmContainer.getResources().getRequests().get(Constants.RESOURCE_NAME_CPU).getAmount()); - - assertThat(tmContainer.getEnv(), Matchers.contains( - new EnvVarBuilder().withName("RESOURCE_ID").withValue(podName).build())); - - assertThat(tmContainer.getPorts().stream().map(ContainerPort::getContainerPort).collect(Collectors.toList()), - Matchers.hasItems(6122)); - - assertEquals(1, tmContainer.getVolumeMounts().size()); - final String mountPath = FLINK_CONFIG.getString(KubernetesConfigOptions.FLINK_CONF_DIR); - assertEquals(new File(mountPath, FLINK_CONF_FILENAME).getPath(), - tmContainer.getVolumeMounts().get(0).getMountPath()); - assertEquals(FLINK_CONF_FILENAME, tmContainer.getVolumeMounts().get(0).getSubPath()); - - // Stop the pod - flinkKubeClient.stopPod(podName); - assertEquals(0, kubeClient.pods().list().getItems().size()); + // todo } @Test public void testServiceLoadBalancerWithNoIP() throws Exception { - final String hostName = "test-host-name"; - final Endpoint endpoint = getRestEndpoint(hostName, ""); - assertEquals(hostName, endpoint.getAddress()); - assertEquals(8081, endpoint.getPort()); + // todo } @Test public void testServiceLoadBalancerEmptyHostAndIP() throws Exception { - final Endpoint endpoint1 = getRestEndpoint("", ""); - assertNull(endpoint1); - - final Endpoint endpoint2 = getRestEndpoint(null, null); - assertNull(endpoint2); - } - - private Endpoint getRestEndpoint(String hostName, String ip) throws Exception { - final String clusterId = "flink-on-k8s-cluster-test"; - mockRestServiceActionWatcher(clusterId); - mockGetRestService(clusterId, hostName, ip); - - flinkKubeClient.createRestService(clusterId).get(); - - final Service services = kubeClient.services() - .withName(clusterId + Constants.FLINK_REST_SERVICE_SUFFIX) - .get(); - assertNotNull(services); - - return flinkKubeClient.getRestEndpoint(clusterId); + // todo } } -- Gitee From 20d0990f54272039e033f308cbb8a81fd0c7c000 Mon Sep 17 00:00:00 2001 From: felixzheng Date: Sun, 23 Feb 2020 13:26:18 +0800 Subject: [PATCH 052/885] [FLINK-16194][k8s] Refactor and simplify KubernetesTestBase - make KubernetesTestBase contains only the common logic of the client-side and cluster-side - remove some unnecessary methods of getting kube client and mocking Service's ADD event --- .../KubernetesClusterDescriptorTest.java | 28 +++------ .../KubernetesResourceManagerTest.java | 57 +++++++++--------- .../flink/kubernetes/KubernetesTestBase.java | 58 +++++++------------ .../Fabric8FlinkKubeClientTest.java | 13 +---- 4 files changed, 60 insertions(+), 96 deletions(-) diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesClusterDescriptorTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesClusterDescriptorTest.java index de83b150ae..4522385251 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesClusterDescriptorTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesClusterDescriptorTest.java @@ -25,15 +25,12 @@ import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.TaskManagerOptions; -import org.apache.flink.kubernetes.configuration.KubernetesConfigOptionsInternal; -import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient; import org.apache.flink.kubernetes.utils.Constants; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; import io.fabric8.kubernetes.api.model.Container; import io.fabric8.kubernetes.api.model.EnvVar; import io.fabric8.kubernetes.api.model.Service; -import io.fabric8.kubernetes.api.model.ServiceList; import io.fabric8.kubernetes.api.model.apps.Deployment; import io.fabric8.kubernetes.client.KubernetesClient; import org.junit.Test; @@ -50,6 +47,8 @@ import static org.junit.Assert.assertTrue; */ public class KubernetesClusterDescriptorTest extends KubernetesTestBase { + private static final String MOCK_SERVICE_IP = "192.168.0.1"; + private final ClusterSpecification clusterSpecification = new ClusterSpecification.ClusterSpecificationBuilder() .createClusterSpecification(); @@ -57,15 +56,9 @@ public class KubernetesClusterDescriptorTest extends KubernetesTestBase { public void testDeploySessionCluster() throws Exception { final ClusterClient clusterClient = deploySessionCluster(); // Check updated flink config options - assertEquals(String.valueOf(Constants.BLOB_SERVER_PORT), FLINK_CONFIG.getString(BlobServerOptions.PORT)); - assertEquals(String.valueOf(Constants.TASK_MANAGER_RPC_PORT), FLINK_CONFIG.getString(TaskManagerOptions.RPC_PORT)); - assertEquals(CLUSTER_ID + "." + NAMESPACE, FLINK_CONFIG.getString(JobManagerOptions.ADDRESS)); - assertEquals(MOCK_SERVICE_ID, FLINK_CONFIG.getString(KubernetesConfigOptionsInternal.SERVICE_ID)); - - final KubernetesClient kubeClient = server.getClient(); - final ServiceList serviceList = kubeClient.services().list(); - assertEquals(2, serviceList.getItems().size()); - assertEquals(CLUSTER_ID, serviceList.getItems().get(0).getMetadata().getName()); + assertEquals(String.valueOf(Constants.BLOB_SERVER_PORT), flinkConfig.getString(BlobServerOptions.PORT)); + assertEquals(String.valueOf(Constants.TASK_MANAGER_RPC_PORT), flinkConfig.getString(TaskManagerOptions.RPC_PORT)); + assertEquals(CLUSTER_ID + "." + NAMESPACE, flinkConfig.getString(JobManagerOptions.ADDRESS)); final Deployment jmDeployment = kubeClient .apps() @@ -93,7 +86,7 @@ public class KubernetesClusterDescriptorTest extends KubernetesTestBase { @Test public void testDeployHighAvailabilitySessionCluster() throws ClusterDeploymentException { - FLINK_CONFIG.setString(HighAvailabilityOptions.HA_MODE, HighAvailabilityMode.ZOOKEEPER.toString()); + flinkConfig.setString(HighAvailabilityOptions.HA_MODE, HighAvailabilityMode.ZOOKEEPER.toString()); final ClusterClient clusterClient = deploySessionCluster(); @@ -121,8 +114,7 @@ public class KubernetesClusterDescriptorTest extends KubernetesTestBase { @Test public void testKillCluster() throws Exception { - final FlinkKubeClient flinkKubeClient = getFabric8FlinkKubeClient(); - final KubernetesClusterDescriptor descriptor = new KubernetesClusterDescriptor(FLINK_CONFIG, flinkKubeClient); + final KubernetesClusterDescriptor descriptor = new KubernetesClusterDescriptor(flinkConfig, flinkKubeClient); final ClusterSpecification clusterSpecification = new ClusterSpecification.ClusterSpecificationBuilder() .createClusterSpecification(); @@ -137,14 +129,10 @@ public class KubernetesClusterDescriptorTest extends KubernetesTestBase { // Mock kubernetes server do not delete the rest service by gc, so the rest service still exist. final List services = kubeClient.services().list().getItems(); assertEquals(1, services.size()); - assertEquals( - MOCK_SERVICE_ID, - services.get(0).getMetadata().getOwnerReferences().get(0).getUid()); } private ClusterClient deploySessionCluster() throws ClusterDeploymentException { - final FlinkKubeClient flinkKubeClient = getFabric8FlinkKubeClient(); - final KubernetesClusterDescriptor descriptor = new KubernetesClusterDescriptor(FLINK_CONFIG, flinkKubeClient); + final KubernetesClusterDescriptor descriptor = new KubernetesClusterDescriptor(flinkConfig, flinkKubeClient); final ClusterClient clusterClient = descriptor .deploySessionCluster(clusterSpecification) diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesResourceManagerTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesResourceManagerTest.java index 08d8f51f5e..90c033d4e1 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesResourceManagerTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesResourceManagerTest.java @@ -63,7 +63,8 @@ import io.fabric8.kubernetes.api.model.EnvVarBuilder; import io.fabric8.kubernetes.api.model.Pod; import io.fabric8.kubernetes.api.model.PodList; import io.fabric8.kubernetes.api.model.PodStatusBuilder; -import io.fabric8.kubernetes.client.KubernetesClient; +import io.fabric8.kubernetes.api.model.apps.Deployment; +import io.fabric8.kubernetes.api.model.apps.DeploymentBuilder; import org.hamcrest.Matchers; import org.junit.After; import org.junit.Assert; @@ -90,25 +91,30 @@ import static org.junit.Assert.assertTrue; public class KubernetesResourceManagerTest extends KubernetesTestBase { private static final Time TIMEOUT = Time.seconds(10L); + private static final String JOB_MANAGER_HOST = "jm-host1"; private TestingFatalErrorHandler testingFatalErrorHandler; - private final String jobManagerHost = "jm-host1"; - - private Configuration flinkConfig; - private TestingKubernetesResourceManager resourceManager; - private FlinkKubeClient flinkKubeClient; - @Before public void setup() throws Exception { - testingFatalErrorHandler = new TestingFatalErrorHandler(); - flinkConfig = new Configuration(FLINK_CONFIG); + super.setup(); + flinkConfig.set(TaskManagerOptions.TOTAL_PROCESS_MEMORY, MemorySize.parse("1024m")); + flinkConfig.setString(TaskManagerOptions.RPC_PORT, String.valueOf(Constants.TASK_MANAGER_RPC_PORT)); + + testingFatalErrorHandler = new TestingFatalErrorHandler(); - flinkKubeClient = getFabric8FlinkKubeClient(); resourceManager = createAndStartResourceManager(flinkConfig); + + final Deployment mockDeployment = new DeploymentBuilder() + .editOrNewMetadata() + .withName(CLUSTER_ID) + .withUid(CLUSTER_ID) + .endMetadata() + .build(); + kubeClient.apps().deployments().inNamespace(NAMESPACE).create(mockDeployment); } @After @@ -178,8 +184,7 @@ public class KubernetesResourceManagerTest extends KubernetesTestBase { public void testStartAndStopWorker() throws Exception { registerSlotRequest(); - final KubernetesClient client = getKubeClient(); - final PodList list = client.pods().list(); + final PodList list = kubeClient.pods().list(); assertEquals(1, list.getItems().size()); final Pod pod = list.getItems().get(0); @@ -215,7 +220,7 @@ public class KubernetesResourceManagerTest extends KubernetesTestBase { return null; }); unregisterAndReleaseFuture.get(); - assertEquals(0, client.pods().list().getItems().size()); + assertEquals(0, kubeClient.pods().list().getItems().size()); assertEquals(0, resourceManager.getWorkerNodes().size()); } @@ -223,38 +228,37 @@ public class KubernetesResourceManagerTest extends KubernetesTestBase { public void testTaskManagerPodTerminated() throws Exception { registerSlotRequest(); - final KubernetesClient client = getKubeClient(); - final Pod pod1 = client.pods().list().getItems().get(0); + final Pod pod1 = kubeClient.pods().list().getItems().get(0); final String taskManagerPrefix = CLUSTER_ID + "-taskmanager-1-"; resourceManager.onAdded(Collections.singletonList(new KubernetesPod(flinkConfig, pod1))); // General modification event resourceManager.onModified(Collections.singletonList(new KubernetesPod(flinkConfig, pod1))); - assertEquals(1, client.pods().list().getItems().size()); - assertEquals(taskManagerPrefix + 1, client.pods().list().getItems().get(0).getMetadata().getName()); + assertEquals(1, kubeClient.pods().list().getItems().size()); + assertEquals(taskManagerPrefix + 1, kubeClient.pods().list().getItems().get(0).getMetadata().getName()); // Terminate the pod. terminatePod(pod1); resourceManager.onModified(Collections.singletonList(new KubernetesPod(flinkConfig, pod1))); // Old pod should be deleted and a new task manager should be created - assertEquals(1, client.pods().list().getItems().size()); - final Pod pod2 = client.pods().list().getItems().get(0); + assertEquals(1, kubeClient.pods().list().getItems().size()); + final Pod pod2 = kubeClient.pods().list().getItems().get(0); assertEquals(taskManagerPrefix + 2, pod2.getMetadata().getName()); // Error happens in the pod. resourceManager.onAdded(Collections.singletonList(new KubernetesPod(flinkConfig, pod2))); terminatePod(pod2); resourceManager.onError(Collections.singletonList(new KubernetesPod(flinkConfig, pod2))); - final Pod pod3 = client.pods().list().getItems().get(0); + final Pod pod3 = kubeClient.pods().list().getItems().get(0); assertEquals(taskManagerPrefix + 3, pod3.getMetadata().getName()); // Delete the pod. resourceManager.onAdded(Collections.singletonList(new KubernetesPod(flinkConfig, pod3))); terminatePod(pod3); resourceManager.onDeleted(Collections.singletonList(new KubernetesPod(flinkConfig, pod3))); - assertEquals(taskManagerPrefix + 4, client.pods().list().getItems().get(0).getMetadata().getName()); + assertEquals(taskManagerPrefix + 4, kubeClient.pods().list().getItems().get(0).getMetadata().getName()); } @Test @@ -267,8 +271,7 @@ public class KubernetesResourceManagerTest extends KubernetesTestBase { 1024, 1, new HashMap<>())); - final KubernetesClient client = getKubeClient(); - assertEquals(1, client.pods().list().getItems().size()); + assertEquals(1, kubeClient.pods().list().getItems().size()); // Call initialize method to recover worker nodes from previous attempt. resourceManager.initialize(); @@ -277,12 +280,12 @@ public class KubernetesResourceManagerTest extends KubernetesTestBase { // Register the previous taskmanager, no new pod should be created registerTaskExecutor(new ResourceID(previewPodName)); registerSlotRequest(); - assertEquals(1, client.pods().list().getItems().size()); + assertEquals(1, kubeClient.pods().list().getItems().size()); // Register a new slot request, a new taskmanger pod will be created with attempt2 registerSlotRequest(); - assertEquals(2, client.pods().list().getItems().size()); - assertThat(client.pods().list().getItems().stream() + assertEquals(2, kubeClient.pods().list().getItems().size()); + assertThat(kubeClient.pods().list().getItems().stream() .map(e -> e.getMetadata().getName()) .collect(Collectors.toList()), Matchers.containsInAnyOrder(taskManagerPrefix + "1-1", taskManagerPrefix + "2-1")); @@ -341,7 +344,7 @@ public class KubernetesResourceManagerTest extends KubernetesTestBase { private void registerSlotRequest() throws Exception { CompletableFuture registerSlotRequestFuture = resourceManager.runInMainThread(() -> { resourceManager.getSlotManager().registerSlotRequest( - new SlotRequest(new JobID(), new AllocationID(), ResourceProfile.UNKNOWN, jobManagerHost)); + new SlotRequest(new JobID(), new AllocationID(), ResourceProfile.UNKNOWN, JOB_MANAGER_HOST)); return null; }); registerSlotRequestFuture.get(); diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesTestBase.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesTestBase.java index 07663ae469..cf6a578c44 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesTestBase.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesTestBase.java @@ -18,13 +18,9 @@ package org.apache.flink.kubernetes; -import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.ResourceManagerOptions; -import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; -import org.apache.flink.kubernetes.configuration.KubernetesConfigOptionsInternal; import org.apache.flink.kubernetes.kubeclient.Fabric8FlinkKubeClient; import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient; import org.apache.flink.kubernetes.utils.Constants; @@ -46,6 +42,12 @@ import java.util.Map; * Base test class for Kubernetes. */ public class KubernetesTestBase extends TestLogger { + + protected static final String NAMESPACE = "test"; + protected static final String CLUSTER_ID = "my-flink-cluster1"; + protected static final String CONTAINER_IMAGE = "flink-k8s-test:latest"; + protected static final String CONTAINER_IMAGE_PULL_POLICY = "IfNotPresent"; + @Rule public MixedKubernetesServer server = new MixedKubernetesServer(true, true); @@ -54,52 +56,32 @@ public class KubernetesTestBase extends TestLogger { private File flinkConfDir; - protected static final String NAMESPACE = "test"; - - protected static final Configuration FLINK_CONFIG = new Configuration(); + protected final Configuration flinkConfig = new Configuration(); - protected static final String CLUSTER_ID = "my-flink-cluster1"; + protected KubernetesClient kubeClient; - protected static final String CONTAINER_IMAGE = "flink-k8s-test:latest"; - - protected static final String MOCK_SERVICE_ID = "mock-uuid-of-service"; - - protected static final String MOCK_SERVICE_IP = "192.168.0.1"; - - protected static final String FLINK_MASTER_ENV_KEY = "LD_LIBRARY_PATH"; - - protected static final String FLINK_MASTER_ENV_VALUE = "/usr/lib/native"; + protected FlinkKubeClient flinkKubeClient; @Before - public void setUp() throws IOException { - FLINK_CONFIG.setString(KubernetesConfigOptions.NAMESPACE, NAMESPACE); - FLINK_CONFIG.setString(KubernetesConfigOptions.CLUSTER_ID, CLUSTER_ID); - FLINK_CONFIG.setString(KubernetesConfigOptions.CONTAINER_IMAGE, CONTAINER_IMAGE); - FLINK_CONFIG.setString(KubernetesConfigOptionsInternal.SERVICE_ID, MOCK_SERVICE_ID); - FLINK_CONFIG.setString(KubernetesConfigOptionsInternal.ENTRY_POINT_CLASS, "main-class"); - FLINK_CONFIG.setString(BlobServerOptions.PORT, String.valueOf(Constants.BLOB_SERVER_PORT)); - FLINK_CONFIG.setString(TaskManagerOptions.RPC_PORT, String.valueOf(Constants.TASK_MANAGER_RPC_PORT)); - FLINK_CONFIG.setString( - ResourceManagerOptions.CONTAINERIZED_MASTER_ENV_PREFIX + FLINK_MASTER_ENV_KEY, - FLINK_MASTER_ENV_VALUE); + public void setup() throws Exception { + flinkConfig.setString(KubernetesConfigOptions.NAMESPACE, NAMESPACE); + flinkConfig.setString(KubernetesConfigOptions.CLUSTER_ID, CLUSTER_ID); + flinkConfig.setString(KubernetesConfigOptions.CONTAINER_IMAGE, CONTAINER_IMAGE); + flinkConfig.setString(KubernetesConfigOptions.CONTAINER_IMAGE_PULL_POLICY, CONTAINER_IMAGE_PULL_POLICY); flinkConfDir = temporaryFolder.newFolder().getAbsoluteFile(); - BootstrapTools.writeConfiguration(new Configuration(), new File(flinkConfDir, "flink-conf.yaml")); + writeFlinkConfiguration(); + Map map = new HashMap<>(); map.put(ConfigConstants.ENV_FLINK_CONF_DIR, flinkConfDir.toString()); TestBaseUtils.setEnv(map); - } - - protected FlinkKubeClient getFabric8FlinkKubeClient(){ - return getFabric8FlinkKubeClient(FLINK_CONFIG); - } - protected FlinkKubeClient getFabric8FlinkKubeClient(Configuration flinkConfig){ - return new Fabric8FlinkKubeClient(flinkConfig, server.getClient().inNamespace(NAMESPACE)); + kubeClient = server.getClient().inNamespace(NAMESPACE); + flinkKubeClient = new Fabric8FlinkKubeClient(flinkConfig, kubeClient); } - protected KubernetesClient getKubeClient() { - return server.getClient().inNamespace(NAMESPACE); + protected void writeFlinkConfiguration() throws IOException { + BootstrapTools.writeConfiguration(this.flinkConfig, new File(flinkConfDir, "flink-conf.yaml")); } protected Map getCommonLabels() { diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java index 8537d14d58..54251b3677 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java @@ -20,26 +20,17 @@ package org.apache.flink.kubernetes.kubeclient; import org.apache.flink.kubernetes.KubernetesTestBase; -import io.fabric8.kubernetes.client.KubernetesClient; import org.junit.Before; import org.junit.Test; -import java.io.IOException; - /** * Tests for Fabric implementation of {@link FlinkKubeClient}. */ public class Fabric8FlinkKubeClientTest extends KubernetesTestBase { - private FlinkKubeClient flinkKubeClient; - - private KubernetesClient kubeClient; - @Before - public void setUp() throws IOException { - super.setUp(); - flinkKubeClient = getFabric8FlinkKubeClient(); - kubeClient = getKubeClient(); + public void setup() throws Exception { + super.setup(); } @Test -- Gitee From 8c7a17e6ef7e3da884e377437563d2508193a56d Mon Sep 17 00:00:00 2001 From: felixzheng Date: Sun, 23 Feb 2020 14:34:19 +0800 Subject: [PATCH 053/885] [FLINK-16194][k8s] Remove Flink Configuration out of KubernetesResource --- .../kubeclient/Fabric8FlinkKubeClient.java | 12 ++++++------ .../kubeclient/resources/KubernetesPod.java | 10 ++-------- .../kubeclient/resources/KubernetesResource.java | 11 +---------- .../kubeclient/resources/KubernetesService.java | 10 ++-------- .../KubernetesResourceManagerTest.java | 16 ++++++++-------- 5 files changed, 19 insertions(+), 40 deletions(-) diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java index 1698e3ab97..bfa6507281 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java @@ -124,7 +124,7 @@ public class Fabric8FlinkKubeClient implements FlinkKubeClient { return podList .stream() - .map(e -> new KubernetesPod(flinkConfig, e)) + .map(KubernetesPod::new) .collect(Collectors.toList()); } @@ -158,16 +158,16 @@ public class Fabric8FlinkKubeClient implements FlinkKubeClient { LOG.debug("Received {} event for pod {}, details: {}", action, pod.getMetadata().getName(), pod.getStatus()); switch (action) { case ADDED: - callbackHandler.onAdded(Collections.singletonList(new KubernetesPod(flinkConfig, pod))); + callbackHandler.onAdded(Collections.singletonList(new KubernetesPod(pod))); break; case MODIFIED: - callbackHandler.onModified(Collections.singletonList(new KubernetesPod(flinkConfig, pod))); + callbackHandler.onModified(Collections.singletonList(new KubernetesPod(pod))); break; case ERROR: - callbackHandler.onError(Collections.singletonList(new KubernetesPod(flinkConfig, pod))); + callbackHandler.onError(Collections.singletonList(new KubernetesPod(pod))); break; case DELETED: - callbackHandler.onDeleted(Collections.singletonList(new KubernetesPod(flinkConfig, pod))); + callbackHandler.onDeleted(Collections.singletonList(new KubernetesPod(pod))); break; default: LOG.debug("Ignore handling {} event for pod {}", action, pod.getMetadata().getName()); @@ -202,7 +202,7 @@ public class Fabric8FlinkKubeClient implements FlinkKubeClient { return null; } - return new KubernetesService(this.flinkConfig, service); + return new KubernetesService(service); } /** diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesPod.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesPod.java index cef3c290f2..388b742d87 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesPod.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesPod.java @@ -18,8 +18,6 @@ package org.apache.flink.kubernetes.kubeclient.resources; -import org.apache.flink.configuration.Configuration; - import io.fabric8.kubernetes.api.model.Pod; /** @@ -27,12 +25,8 @@ import io.fabric8.kubernetes.api.model.Pod; */ public class KubernetesPod extends KubernetesResource { - public KubernetesPod(Configuration flinkConfig) { - this(flinkConfig, new Pod()); - } - - public KubernetesPod(Configuration flinkConfig, Pod pod) { - super(flinkConfig, pod); + public KubernetesPod(Pod pod) { + super(pod); } public String getName() { diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesResource.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesResource.java index e4ddc434ae..8dbd133bc6 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesResource.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesResource.java @@ -18,8 +18,6 @@ package org.apache.flink.kubernetes.kubeclient.resources; -import org.apache.flink.configuration.Configuration; - /** * Represent a kubernetes resource. */ @@ -27,17 +25,10 @@ public abstract class KubernetesResource { private T internalResource; - private final Configuration flinkConfig; - - public KubernetesResource(Configuration flinkConfig, T internalResource) { - this.flinkConfig = flinkConfig; + public KubernetesResource(T internalResource) { this.internalResource = internalResource; } - public Configuration getFlinkConfig() { - return flinkConfig; - } - public T getInternalResource() { return internalResource; } diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesService.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesService.java index 5d84b8f87a..64bdd10b79 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesService.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesService.java @@ -18,8 +18,6 @@ package org.apache.flink.kubernetes.kubeclient.resources; -import org.apache.flink.configuration.Configuration; - import io.fabric8.kubernetes.api.model.Service; /** @@ -27,11 +25,7 @@ import io.fabric8.kubernetes.api.model.Service; */ public class KubernetesService extends KubernetesResource { - public KubernetesService(Configuration flinkConfig) { - super(flinkConfig, new Service()); - } - - public KubernetesService(Configuration flinkConfig, Service service) { - super(flinkConfig, service); + public KubernetesService(Service internalResource) { + super(internalResource); } } diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesResourceManagerTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesResourceManagerTest.java index 90c033d4e1..429ed2a763 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesResourceManagerTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesResourceManagerTest.java @@ -208,7 +208,7 @@ public class KubernetesResourceManagerTest extends KubernetesTestBase { final String confDirOption = "--configDir " + flinkConfig.getString(KubernetesConfigOptions.FLINK_CONF_DIR); assertTrue(tmContainer.getArgs().get(2).contains(confDirOption)); - resourceManager.onAdded(Collections.singletonList(new KubernetesPod(flinkConfig, pod))); + resourceManager.onAdded(Collections.singletonList(new KubernetesPod(pod))); final ResourceID resourceID = new ResourceID(podName); assertThat(resourceManager.getWorkerNodes().keySet(), Matchers.contains(resourceID)); @@ -231,16 +231,16 @@ public class KubernetesResourceManagerTest extends KubernetesTestBase { final Pod pod1 = kubeClient.pods().list().getItems().get(0); final String taskManagerPrefix = CLUSTER_ID + "-taskmanager-1-"; - resourceManager.onAdded(Collections.singletonList(new KubernetesPod(flinkConfig, pod1))); + resourceManager.onAdded(Collections.singletonList(new KubernetesPod(pod1))); // General modification event - resourceManager.onModified(Collections.singletonList(new KubernetesPod(flinkConfig, pod1))); + resourceManager.onModified(Collections.singletonList(new KubernetesPod(pod1))); assertEquals(1, kubeClient.pods().list().getItems().size()); assertEquals(taskManagerPrefix + 1, kubeClient.pods().list().getItems().get(0).getMetadata().getName()); // Terminate the pod. terminatePod(pod1); - resourceManager.onModified(Collections.singletonList(new KubernetesPod(flinkConfig, pod1))); + resourceManager.onModified(Collections.singletonList(new KubernetesPod(pod1))); // Old pod should be deleted and a new task manager should be created assertEquals(1, kubeClient.pods().list().getItems().size()); @@ -248,16 +248,16 @@ public class KubernetesResourceManagerTest extends KubernetesTestBase { assertEquals(taskManagerPrefix + 2, pod2.getMetadata().getName()); // Error happens in the pod. - resourceManager.onAdded(Collections.singletonList(new KubernetesPod(flinkConfig, pod2))); + resourceManager.onAdded(Collections.singletonList(new KubernetesPod(pod2))); terminatePod(pod2); - resourceManager.onError(Collections.singletonList(new KubernetesPod(flinkConfig, pod2))); + resourceManager.onError(Collections.singletonList(new KubernetesPod(pod2))); final Pod pod3 = kubeClient.pods().list().getItems().get(0); assertEquals(taskManagerPrefix + 3, pod3.getMetadata().getName()); // Delete the pod. - resourceManager.onAdded(Collections.singletonList(new KubernetesPod(flinkConfig, pod3))); + resourceManager.onAdded(Collections.singletonList(new KubernetesPod(pod3))); terminatePod(pod3); - resourceManager.onDeleted(Collections.singletonList(new KubernetesPod(flinkConfig, pod3))); + resourceManager.onDeleted(Collections.singletonList(new KubernetesPod(pod3))); assertEquals(taskManagerPrefix + 4, kubeClient.pods().list().getItems().get(0).getMetadata().getName()); } -- Gitee From 22735e3f3fdafc86b9f3c20b361a3830b30d44e3 Mon Sep 17 00:00:00 2001 From: felixzheng Date: Sun, 23 Feb 2020 14:46:20 +0800 Subject: [PATCH 054/885] [FLINK-16194][k8s] Introduce Kubernetes parameters parsing/verifying/managing tool --- .../kubernetes/KubernetesResourceManager.java | 17 +- .../AbstractKubernetesParameters.java | 145 +++++++++++++++++ .../KubernetesJobManagerParameters.java | 107 ++++++++++++ .../parameters/KubernetesParameters.java | 81 ++++++++++ .../KubernetesTaskManagerParameters.java | 99 ++++++++++++ .../kubernetes/utils/KubernetesUtils.java | 13 ++ .../KubernetesJobManagerParametersTest.java | 153 ++++++++++++++++++ .../KubernetesTaskManagerParametersTest.java | 110 +++++++++++++ 8 files changed, 710 insertions(+), 15 deletions(-) create mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/AbstractKubernetesParameters.java create mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java create mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesParameters.java create mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParameters.java create mode 100644 flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParametersTest.java create mode 100644 flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParametersTest.java diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManager.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManager.java index fdf3afc4c3..e8fd05f34d 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManager.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManager.java @@ -139,7 +139,7 @@ public class KubernetesResourceManager extends ActiveResourceManager podList = kubeClient.getPodsWithLabels(getTaskManagerLabels()); + final List podList = kubeClient.getPodsWithLabels(KubernetesUtils.getTaskManagerLabels(clusterId)); for (KubernetesPod pod : podList) { final KubernetesWorkerNode worker = new KubernetesWorkerNode(new ResourceID(pod.getName())); workerNodes.put(worker.getResourceID(), worker); @@ -320,19 +320,6 @@ public class KubernetesResourceManager extends ActiveResourceManager getTaskManagerLabels() { - final Map labels = new HashMap<>(); - labels.put(Constants.LABEL_TYPE_KEY, Constants.LABEL_TYPE_NATIVE_TYPE); - labels.put(Constants.LABEL_APP_KEY, clusterId); - labels.put(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_TASK_MANAGER); - return labels; - } - protected FlinkKubeClient createFlinkKubeClient() { return KubeClientFactory.fromConfiguration(flinkConfig); } diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/AbstractKubernetesParameters.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/AbstractKubernetesParameters.java new file mode 100644 index 0000000000..93b8524424 --- /dev/null +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/AbstractKubernetesParameters.java @@ -0,0 +1,145 @@ +/* + * 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.kubernetes.kubeclient.parameters; + +import org.apache.flink.client.cli.CliFrontend; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; +import org.apache.flink.kubernetes.utils.Constants; +import org.apache.flink.runtime.clusterframework.BootstrapTools; + +import io.fabric8.kubernetes.api.model.LocalObjectReference; + +import java.io.File; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.kubernetes.configuration.KubernetesConfigOptions.CONTAINER_IMAGE_PULL_SECRETS; +import static org.apache.flink.kubernetes.utils.Constants.CONFIG_FILE_LOG4J_NAME; +import static org.apache.flink.kubernetes.utils.Constants.CONFIG_FILE_LOGBACK_NAME; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Abstract class for the {@link KubernetesParameters}. + */ +public abstract class AbstractKubernetesParameters implements KubernetesParameters { + + protected final Configuration flinkConfig; + + public AbstractKubernetesParameters(Configuration flinkConfig) { + this.flinkConfig = checkNotNull(flinkConfig); + } + + public Configuration getFlinkConfiguration() { + return flinkConfig; + } + + @Override + public String getClusterId() { + final String clusterId = flinkConfig.getString(KubernetesConfigOptions.CLUSTER_ID); + checkNotNull(clusterId, "ClusterId must be specified."); + + return clusterId; + } + + @Override + public String getNamespace() { + final String namespace = flinkConfig.getString(KubernetesConfigOptions.NAMESPACE); + checkArgument(!namespace.trim().isEmpty(), "Invalid " + KubernetesConfigOptions.NAMESPACE + "."); + + return namespace; + } + + @Override + public String getImage() { + final String containerImage = flinkConfig.getString(KubernetesConfigOptions.CONTAINER_IMAGE); + checkArgument(!containerImage.trim().isEmpty(), + "Invalid " + KubernetesConfigOptions.CONTAINER_IMAGE + "."); + return containerImage; + } + + @Override + public String getImagePullPolicy() { + return flinkConfig.getString(KubernetesConfigOptions.CONTAINER_IMAGE_PULL_POLICY); + } + + @Override + public LocalObjectReference[] getImagePullSecrets() { + final List imagePullSecrets = flinkConfig.get(CONTAINER_IMAGE_PULL_SECRETS); + + if (imagePullSecrets == null) { + return new LocalObjectReference[0]; + } else { + return imagePullSecrets.stream() + .map(String::trim) + .filter(secret -> !secret.isEmpty()) + .map(LocalObjectReference::new) + .toArray(LocalObjectReference[]::new); + } + } + + @Override + public Map getCommonLabels() { + Map commonLabels = new HashMap<>(); + commonLabels.put(Constants.LABEL_TYPE_KEY, Constants.LABEL_TYPE_NATIVE_TYPE); + commonLabels.put(Constants.LABEL_APP_KEY, getClusterId()); + + return commonLabels; + } + + @Override + public String getFlinkConfDirInPod() { + return flinkConfig.getString(KubernetesConfigOptions.FLINK_CONF_DIR); + } + + @Override + public String getFlinkLogDirInPod() { + return flinkConfig.getString(KubernetesConfigOptions.FLINK_LOG_DIR); + } + + @Override + public String getContainerEntrypoint() { + return flinkConfig.getString(KubernetesConfigOptions.KUBERNETES_ENTRY_PATH); + } + + @Override + public boolean hasLogback() { + final String confDir = CliFrontend.getConfigurationDirectoryFromEnv(); + final File logbackFile = new File(confDir, CONFIG_FILE_LOGBACK_NAME); + return logbackFile.exists(); + } + + @Override + public boolean hasLog4j() { + final String confDir = CliFrontend.getConfigurationDirectoryFromEnv(); + final File log4jFile = new File(confDir, CONFIG_FILE_LOG4J_NAME); + return log4jFile.exists(); + } + + /** + * Extract container customized environment variable properties with a given name prefix. + * @param envPrefix the given property name prefix + * @return a Map storing with customized environment variable key/value pairs. + */ + protected Map getPrefixedEnvironments(String envPrefix) { + return BootstrapTools.getEnvironmentVariables(envPrefix, flinkConfig); + } +} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java new file mode 100644 index 0000000000..bc44444266 --- /dev/null +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java @@ -0,0 +1,107 @@ +/* + * 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.kubernetes.kubeclient.parameters; + +import org.apache.flink.client.deployment.ClusterSpecification; +import org.apache.flink.configuration.BlobServerOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.ResourceManagerOptions; +import org.apache.flink.configuration.RestOptions; +import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; +import org.apache.flink.kubernetes.configuration.KubernetesConfigOptionsInternal; +import org.apache.flink.kubernetes.utils.Constants; +import org.apache.flink.kubernetes.utils.KubernetesUtils; + +import java.util.Map; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A utility class helps parse, verify, and manage the Kubernetes side parameters + * that are used for constructing the JobManager Pod and all accompanying resources + * connected to it. + */ +public class KubernetesJobManagerParameters extends AbstractKubernetesParameters { + + public static final String JOB_MANAGER_MAIN_CONTAINER_NAME = "flink-job-manager"; + + private final ClusterSpecification clusterSpecification; + + public KubernetesJobManagerParameters(Configuration flinkConfig, ClusterSpecification clusterSpecification) { + super(flinkConfig); + this.clusterSpecification = checkNotNull(clusterSpecification); + } + + @Override + public Map getLabels() { + Map labels = getCommonLabels(); + labels.put(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_JOB_MANAGER); + + return labels; + } + + @Override + public Map getEnvironments() { + return getPrefixedEnvironments(ResourceManagerOptions.CONTAINERIZED_MASTER_ENV_PREFIX); + } + + public String getJobManagerMainContainerName() { + return JOB_MANAGER_MAIN_CONTAINER_NAME; + } + + public int getJobManagerMemoryMB() { + return clusterSpecification.getMasterMemoryMB(); + } + + public double getJobManagerCPU() { + return flinkConfig.getDouble(KubernetesConfigOptions.JOB_MANAGER_CPU); + } + + public int getRestPort() { + return flinkConfig.getInteger(RestOptions.PORT); + } + + public int getRPCPort() { + return flinkConfig.getInteger(JobManagerOptions.PORT); + } + + public int getBlobServerPort() { + final int blobServerPort = KubernetesUtils.parsePort(flinkConfig, BlobServerOptions.PORT); + checkArgument(blobServerPort > 0, "%s should not be 0.", BlobServerOptions.PORT.key()); + return blobServerPort; + } + + public String getServiceAccount() { + return flinkConfig.getString(KubernetesConfigOptions.JOB_MANAGER_SERVICE_ACCOUNT); + } + + public String getEntrypointClass() { + final String entrypointClass = flinkConfig.getString(KubernetesConfigOptionsInternal.ENTRY_POINT_CLASS); + checkNotNull(entrypointClass, KubernetesConfigOptionsInternal.ENTRY_POINT_CLASS + " must be specified!"); + + return entrypointClass; + } + + public String getRestServiceExposedType() { + final String exposedType = flinkConfig.getString(KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE); + return KubernetesConfigOptions.ServiceExposedType.valueOf(exposedType).name(); + } +} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesParameters.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesParameters.java new file mode 100644 index 0000000000..2ce4c89967 --- /dev/null +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesParameters.java @@ -0,0 +1,81 @@ +/* + * 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.kubernetes.kubeclient.parameters; + +import io.fabric8.kubernetes.api.model.LocalObjectReference; + +import java.util.Map; + +/** + * A common collection of parameters that is used to construct the JobManager/TaskManager Pods, + * including the accompanying Kubernetes resources that together represent a Flink application. + */ +public interface KubernetesParameters { + + String getClusterId(); + + String getNamespace(); + + String getImage(); + + String getImagePullPolicy(); + + LocalObjectReference[] getImagePullSecrets(); + + /** + * A common collection of labels that are attached to every created Kubernetes resources. + * This can include the Deployment, the Pod(s), the ConfigMap(s), and the Service(s), etc. + */ + Map getCommonLabels(); + + /** + * A collection of labels that are attached to the JobManager and TaskManager Pod(s). + */ + Map getLabels(); + + /** + * A collection of customized environments that are attached to the JobManager and TaskManager Container(s). + */ + Map getEnvironments(); + + /** + * Directory in Pod that stores the flink-conf.yaml, log4j.properties, and the logback.xml. + */ + String getFlinkConfDirInPod(); + + /** + * Directory in Pod that saves the log files. + */ + String getFlinkLogDirInPod(); + + /** + * The docker entrypoint that starts processes in the container. + */ + String getContainerEntrypoint(); + + /** + * Whether the logback.xml is located. + */ + boolean hasLogback(); + + /** + * Whether the log4j.properties is located. + */ + boolean hasLog4j(); +} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParameters.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParameters.java new file mode 100644 index 0000000000..4fcadd0714 --- /dev/null +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParameters.java @@ -0,0 +1,99 @@ +/* + * 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.kubernetes.kubeclient.parameters; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.kubernetes.utils.KubernetesUtils; +import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; + +import java.util.Map; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A utility class helps parse, verify, and manage the Kubernetes side parameters + * that are used for constructing the TaskManager Pod. + */ +public class KubernetesTaskManagerParameters extends AbstractKubernetesParameters { + + public static final String TASK_MANAGER_MAIN_CONTAINER_NAME = "flink-task-manager"; + + private final String podName; + + private final int taskManagerMemoryMB; + + private final String dynamicProperties; + + private final ContaineredTaskManagerParameters containeredTaskManagerParameters; + + public KubernetesTaskManagerParameters( + Configuration flinkConfig, + String podName, + int taskManagerMemoryMB, + String dynamicProperties, + ContaineredTaskManagerParameters containeredTaskManagerParameters) { + super(flinkConfig); + this.podName = checkNotNull(podName); + this.taskManagerMemoryMB = taskManagerMemoryMB; + this.dynamicProperties = checkNotNull(dynamicProperties); + this.containeredTaskManagerParameters = checkNotNull(containeredTaskManagerParameters); + } + + @Override + public Map getLabels() { + return KubernetesUtils.getTaskManagerLabels(getClusterId()); + } + + @Override + public Map getEnvironments() { + return this.containeredTaskManagerParameters.taskManagerEnv(); + } + + public String getTaskManagerMainContainerName() { + return TASK_MANAGER_MAIN_CONTAINER_NAME; + } + + public String getPodName() { + return podName; + } + + public int getTaskManagerMemoryMB() { + return taskManagerMemoryMB; + } + + public double getTaskManagerCPU() { + return containeredTaskManagerParameters.getTaskExecutorProcessSpec().getCpuCores().getValue().doubleValue(); + } + + public int getRPCPort() { + final int taskManagerRpcPort = KubernetesUtils.parsePort(flinkConfig, TaskManagerOptions.RPC_PORT); + checkArgument(taskManagerRpcPort > 0, "%s should not be 0.", TaskManagerOptions.RPC_PORT.key()); + return taskManagerRpcPort; + } + + public String getDynamicProperties() { + return dynamicProperties; + } + + public ContaineredTaskManagerParameters getContaineredTaskManagerParameters() { + return containeredTaskManagerParameters; + } +} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java index 97b716a09b..c3e49726f2 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java @@ -270,6 +270,19 @@ public class KubernetesUtils { return volumeMounts; } + /** + * Get task manager labels for the current Flink cluster. They could be used to watch the pods status. + * + * @return Task manager labels. + */ + public static Map getTaskManagerLabels(String clusterId) { + final Map labels = new HashMap<>(); + labels.put(Constants.LABEL_TYPE_KEY, Constants.LABEL_TYPE_NATIVE_TYPE); + labels.put(Constants.LABEL_APP_KEY, clusterId); + labels.put(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_TASK_MANAGER); + return labels; + } + /** * Get resource requirements from memory and cpu. * diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParametersTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParametersTest.java new file mode 100644 index 0000000000..46e714d47f --- /dev/null +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParametersTest.java @@ -0,0 +1,153 @@ +/* + * 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.kubernetes.kubeclient.parameters; + +import org.apache.flink.client.deployment.ClusterSpecification; +import org.apache.flink.configuration.BlobServerOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.ResourceManagerOptions; +import org.apache.flink.configuration.RestOptions; +import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; +import org.apache.flink.kubernetes.configuration.KubernetesConfigOptionsInternal; +import org.apache.flink.util.FlinkRuntimeException; + +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.CoreMatchers.containsString; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; + +/** + * General tests for the {@link KubernetesJobManagerParameters}. + */ +public class KubernetesJobManagerParametersTest { + + private static final int JOB_MANAGER_MEMORY = 768; + private static final double JOB_MANAGER_CPU = 2.0; + + private final Configuration flinkConfig = new Configuration(); + + private final ClusterSpecification clusterSpecification = new ClusterSpecification.ClusterSpecificationBuilder() + .setMasterMemoryMB(JOB_MANAGER_MEMORY) + .setTaskManagerMemoryMB(1024) + .setSlotsPerTaskManager(1) + .createClusterSpecification(); + + private final KubernetesJobManagerParameters kubernetesJobManagerParameters = + new KubernetesJobManagerParameters(flinkConfig, clusterSpecification); + + @Test + public void testGetEnvironments() { + final Map expectedEnvironments = new HashMap<>(); + expectedEnvironments.put("k1", "v1"); + expectedEnvironments.put("k2", "v2"); + + expectedEnvironments.forEach((k, v) -> + flinkConfig.setString(ResourceManagerOptions.CONTAINERIZED_MASTER_ENV_PREFIX + k, v)); + + final Map resultEnvironments = kubernetesJobManagerParameters.getEnvironments(); + + assertEquals(expectedEnvironments, resultEnvironments); + } + + @Test + public void testGetJobManagerMemoryMB() { + assertEquals(JOB_MANAGER_MEMORY, kubernetesJobManagerParameters.getJobManagerMemoryMB()); + } + + @Test + public void testGetJobManagerCPU() { + flinkConfig.set(KubernetesConfigOptions.JOB_MANAGER_CPU, JOB_MANAGER_CPU); + assertEquals(JOB_MANAGER_CPU, kubernetesJobManagerParameters.getJobManagerCPU(), 0.00001); + } + + @Test + public void testGetRestPort() { + flinkConfig.set(RestOptions.PORT, 12345); + assertEquals(12345, kubernetesJobManagerParameters.getRestPort()); + } + + @Test + public void testGetRpcPort() { + flinkConfig.set(JobManagerOptions.PORT, 1234); + assertEquals(1234, kubernetesJobManagerParameters.getRPCPort()); + } + + @Test + public void testGetBlobServerPort() { + flinkConfig.set(BlobServerOptions.PORT, "2345"); + assertEquals(2345, kubernetesJobManagerParameters.getBlobServerPort()); + } + + @Test + public void testGetBlobServerPortException1() { + flinkConfig.set(BlobServerOptions.PORT, "1000-2000"); + + try { + kubernetesJobManagerParameters.getBlobServerPort(); + fail("Should fail with an exception."); + } catch (FlinkRuntimeException e) { + assertThat( + e.getMessage(), + containsString(BlobServerOptions.PORT.key() + + " should be specified to a fixed port. Do not support a range of ports.")); + } + } + + @Test + public void testGetBlobServerPortException2() { + flinkConfig.set(BlobServerOptions.PORT, "0"); + + try { + kubernetesJobManagerParameters.getBlobServerPort(); + fail("Should fail with an exception."); + } catch (IllegalArgumentException e) { + assertThat( + e.getMessage(), + containsString(BlobServerOptions.PORT.key() + + " should not be 0.")); + } + } + + @Test + public void testGetServiceAccount() { + flinkConfig.set(KubernetesConfigOptions.JOB_MANAGER_SERVICE_ACCOUNT, "flink"); + assertEquals("flink", kubernetesJobManagerParameters.getServiceAccount()); + } + + @Test + public void testGetEntrypointMainClass() { + final String entrypointClass = "org.flink.kubernetes.Entrypoint"; + flinkConfig.set(KubernetesConfigOptionsInternal.ENTRY_POINT_CLASS, entrypointClass); + assertEquals(entrypointClass, kubernetesJobManagerParameters.getEntrypointClass()); + } + + @Test + public void testGetRestServiceExposedType() { + flinkConfig.set(KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE, + KubernetesConfigOptions.ServiceExposedType.NodePort.name()); + assertEquals(KubernetesConfigOptions.ServiceExposedType.NodePort.name(), + kubernetesJobManagerParameters.getRestServiceExposedType()); + } +} diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParametersTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParametersTest.java new file mode 100644 index 0000000000..f3784c6caa --- /dev/null +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParametersTest.java @@ -0,0 +1,110 @@ +/* + * 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.kubernetes.kubeclient.parameters; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.ResourceManagerOptions; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; +import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec; +import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils; + +import org.junit.Before; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +/** + * General tests for the {@link KubernetesTaskManagerParameters}. + */ +public class KubernetesTaskManagerParametersTest { + + private static final int TASK_MANAGER_MEMORY = 1024; + private static final double TASK_MANAGER_CPU = 1.2; + private static final int RPC_PORT = 13001; + + private static final String POD_NAME = "task-manager-pod-1"; + private static final String DYNAMIC_PROPERTIES = "-Dkey.b='b2'"; + + private final Map customizedEnvs = new HashMap() { + { + put("key1", "value1"); + put("key2", "value2"); + } + }; + + private KubernetesTaskManagerParameters kubernetesTaskManagerParameters; + + @Before + public void setup() { + final Configuration flinkConfig = new Configuration(); + flinkConfig.set(TaskManagerOptions.CPU_CORES, TASK_MANAGER_CPU); + flinkConfig.set(TaskManagerOptions.TOTAL_PROCESS_MEMORY, MemorySize.parse(TASK_MANAGER_MEMORY + "m")); + flinkConfig.set(TaskManagerOptions.RPC_PORT, String.valueOf(RPC_PORT)); + + customizedEnvs.forEach((k, v) -> + flinkConfig.setString(ResourceManagerOptions.CONTAINERIZED_TASK_MANAGER_ENV_PREFIX + k, v)); + + final TaskExecutorProcessSpec taskExecutorProcessSpec = + TaskExecutorProcessUtils.processSpecFromConfig(flinkConfig); + final ContaineredTaskManagerParameters containeredTaskManagerParameters = + ContaineredTaskManagerParameters.create(flinkConfig, taskExecutorProcessSpec, + flinkConfig.getInteger(TaskManagerOptions.NUM_TASK_SLOTS)); + + this.kubernetesTaskManagerParameters = new KubernetesTaskManagerParameters(flinkConfig, + POD_NAME, + TASK_MANAGER_MEMORY, + DYNAMIC_PROPERTIES, + containeredTaskManagerParameters); + } + + @Test + public void testGetEnvironments() { + assertEquals(customizedEnvs, kubernetesTaskManagerParameters.getEnvironments()); + } + + @Test + public void testGetPodName() { + assertEquals(POD_NAME, kubernetesTaskManagerParameters.getPodName()); + } + + @Test + public void testGetTaskManagerMemoryMB() { + assertEquals(TASK_MANAGER_MEMORY, kubernetesTaskManagerParameters.getTaskManagerMemoryMB()); + } + + @Test + public void testGetTaskManagerCPU() { + assertEquals(TASK_MANAGER_CPU, kubernetesTaskManagerParameters.getTaskManagerCPU(), 0.000000000001); + } + + @Test + public void testGetRpcPort() { + assertEquals(RPC_PORT, kubernetesTaskManagerParameters.getRPCPort()); + } + + @Test + public void testGetDynamicProperties() { + assertEquals(DYNAMIC_PROPERTIES, kubernetesTaskManagerParameters.getDynamicProperties()); + } +} -- Gitee From a50435d443395141fc526c6584ec9841e5968713 Mon Sep 17 00:00:00 2001 From: felixzheng Date: Sun, 23 Feb 2020 15:03:30 +0800 Subject: [PATCH 055/885] [FLINK-16194][k8s] Introduce the monadic step decorator design and the InitJobManagerDecorator and the TaskManagerDecorator --- .../flink/kubernetes/kubeclient/FlinkPod.java | 98 +++++++++++ .../AbstractKubernetesStepDecorator.java | 91 +++++++++++ .../decorators/InitJobManagerDecorator.java | 121 ++++++++++++++ .../decorators/InitTaskManagerDecorator.java | 100 ++++++++++++ .../decorators/KubernetesStepDecorator.java | 47 ++++++ .../kubernetes/utils/KubernetesUtils.java | 13 -- .../KubernetesJobManagerTestBase.java | 79 +++++++++ .../KubernetesTaskManagerTestBase.java | 83 ++++++++++ .../InitJobManagerDecoratorTest.java | 152 ++++++++++++++++++ .../InitTaskManagerDecoratorTest.java | 148 +++++++++++++++++ .../kubernetes/utils/KubernetesUtilsTest.java | 24 --- 11 files changed, 919 insertions(+), 37 deletions(-) create mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkPod.java create mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/AbstractKubernetesStepDecorator.java create mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecorator.java create mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecorator.java create mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/KubernetesStepDecorator.java create mode 100644 flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/KubernetesJobManagerTestBase.java create mode 100644 flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/KubernetesTaskManagerTestBase.java create mode 100644 flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecoratorTest.java create mode 100644 flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecoratorTest.java diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkPod.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkPod.java new file mode 100644 index 0000000000..f301d8fdd1 --- /dev/null +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkPod.java @@ -0,0 +1,98 @@ +/* + * 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.kubernetes.kubeclient; + +import io.fabric8.kubernetes.api.model.Container; +import io.fabric8.kubernetes.api.model.ContainerBuilder; +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.PodBuilder; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A collection of variables that composes a JobManager/TaskManager Pod. This can include + * the Pod, the main Container, and the InitContainer, etc. + */ +public class FlinkPod { + + private Pod pod; + + private Container mainContainer; + + public FlinkPod(Pod pod, Container mainContainer) { + this.pod = pod; + this.mainContainer = mainContainer; + } + + public Pod getPod() { + return pod; + } + + public void setPod(Pod pod) { + this.pod = pod; + } + + public Container getMainContainer() { + return mainContainer; + } + + public void setMainContainer(Container mainContainer) { + this.mainContainer = mainContainer; + } + + /** + * Builder for creating a {@link FlinkPod}. + */ + public static class Builder { + + private Pod pod; + private Container mainContainer; + + public Builder() { + this.pod = new PodBuilder() + .withNewMetadata() + .endMetadata() + .withNewSpec() + .endSpec() + .build(); + + this.mainContainer = new ContainerBuilder().build(); + } + + public Builder(FlinkPod flinkPod) { + checkNotNull(flinkPod); + this.pod = checkNotNull(flinkPod.getPod()); + this.mainContainer = checkNotNull(flinkPod.getMainContainer()); + } + + public Builder withPod(Pod pod) { + this.pod = checkNotNull(pod); + return this; + } + + public Builder withMainContainer(Container mainContainer) { + this.mainContainer = checkNotNull(mainContainer); + return this; + } + + public FlinkPod build() { + return new FlinkPod(this.pod, this.mainContainer); + } + } +} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/AbstractKubernetesStepDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/AbstractKubernetesStepDecorator.java new file mode 100644 index 0000000000..dbe6427296 --- /dev/null +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/AbstractKubernetesStepDecorator.java @@ -0,0 +1,91 @@ +/* + * 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.kubernetes.kubeclient.decorators; + +import org.apache.flink.kubernetes.kubeclient.FlinkPod; + +import io.fabric8.kubernetes.api.model.HasMetadata; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; + +/** + * An abstract {@link KubernetesStepDecorator} contains common implementations for different plug-in features. + */ +public abstract class AbstractKubernetesStepDecorator implements KubernetesStepDecorator { + + /** + * Apply transformations on the given FlinkPod in accordance to this feature. + * Note that we should return a FlinkPod that keeps all of the properties of the passed FlinkPod object. + * + *

    So this is correct: + * + *

    +	 * {@code
    +	 *
    +	 * Pod decoratedPod = new PodBuilder(pod) // Keeps the original state
    +	 *     ...
    +	 *     .build()
    +	 *
    +	 * Container decoratedContainer = new ContainerBuilder(container) // Keeps the original state
    +	 *     ...
    +	 *     .build()
    +	 *
    +	 * FlinkPod decoratedFlinkPod = new FlinkPodBuilder(flinkPod) // Keeps the original state
    +	 *     ...
    +	 *     .build()
    +	 *
    +	 * }
    +	 * 
    + * + *

    And this is the incorrect: + * + *

    +	 * {@code
    +	 *
    +	 * Pod decoratedPod = new PodBuilder() // Loses the original state
    +	 *     ...
    +	 *     .build()
    +	 *
    +	 * Container decoratedContainer = new ContainerBuilder() // Loses the original state
    +	 *     ...
    +	 *     .build()
    +	 *
    +	 * FlinkPod decoratedFlinkPod = new FlinkPodBuilder() // Loses the original state
    +	 *     ...
    +	 *     .build()
    +	 *
    +	 * }
    +	 * 
    + */ + @Override + public FlinkPod decorateFlinkPod(FlinkPod flinkPod) { + return flinkPod; + } + + /** + * Note that the method could have a side effect of modifying the Flink Configuration object, such as + * update the JobManager address. + */ + @Override + public List buildAccompanyingKubernetesResources() throws IOException { + return Collections.emptyList(); + } +} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecorator.java new file mode 100644 index 0000000000..bef6b24b1b --- /dev/null +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecorator.java @@ -0,0 +1,121 @@ +/* + * 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.kubernetes.kubeclient.decorators; + +import org.apache.flink.kubernetes.kubeclient.FlinkPod; +import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters; +import org.apache.flink.kubernetes.utils.KubernetesUtils; + +import io.fabric8.kubernetes.api.model.Container; +import io.fabric8.kubernetes.api.model.ContainerBuilder; +import io.fabric8.kubernetes.api.model.ContainerPort; +import io.fabric8.kubernetes.api.model.ContainerPortBuilder; +import io.fabric8.kubernetes.api.model.EnvVar; +import io.fabric8.kubernetes.api.model.EnvVarBuilder; +import io.fabric8.kubernetes.api.model.EnvVarSourceBuilder; +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.PodBuilder; +import io.fabric8.kubernetes.api.model.ResourceRequirements; + +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.kubernetes.utils.Constants.API_VERSION; +import static org.apache.flink.kubernetes.utils.Constants.ENV_FLINK_POD_IP_ADDRESS; +import static org.apache.flink.kubernetes.utils.Constants.POD_IP_FIELD_PATH; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * An initializer for the JobManager {@link org.apache.flink.kubernetes.kubeclient.FlinkPod}. + */ +public class InitJobManagerDecorator extends AbstractKubernetesStepDecorator { + + private final KubernetesJobManagerParameters kubernetesJobManagerParameters; + + public InitJobManagerDecorator(KubernetesJobManagerParameters kubernetesJobManagerParameters) { + this.kubernetesJobManagerParameters = checkNotNull(kubernetesJobManagerParameters); + } + + @Override + public FlinkPod decorateFlinkPod(FlinkPod flinkPod) { + final Pod basicPod = new PodBuilder(flinkPod.getPod()) + .withApiVersion(API_VERSION) + .editOrNewMetadata() + .withLabels(kubernetesJobManagerParameters.getLabels()) + .endMetadata() + .editOrNewSpec() + .withServiceAccountName(kubernetesJobManagerParameters.getServiceAccount()) + .withImagePullSecrets(kubernetesJobManagerParameters.getImagePullSecrets()) + .endSpec() + .build(); + + final Container basicMainContainer = decorateMainContainer(flinkPod.getMainContainer()); + + return new FlinkPod.Builder(flinkPod) + .withPod(basicPod) + .withMainContainer(basicMainContainer) + .build(); + } + + private Container decorateMainContainer(Container container) { + final ResourceRequirements requirements = KubernetesUtils.getResourceRequirements( + kubernetesJobManagerParameters.getJobManagerMemoryMB(), + kubernetesJobManagerParameters.getJobManagerCPU()); + + return new ContainerBuilder(container) + .withName(kubernetesJobManagerParameters.getJobManagerMainContainerName()) + .withImage(kubernetesJobManagerParameters.getImage()) + .withImagePullPolicy(kubernetesJobManagerParameters.getImagePullPolicy()) + .withResources(requirements) + .withPorts(getContainerPorts()) + .withEnv(getCustomizedEnvs()) + .addNewEnv() + .withName(ENV_FLINK_POD_IP_ADDRESS) + .withValueFrom(new EnvVarSourceBuilder() + .withNewFieldRef(API_VERSION, POD_IP_FIELD_PATH) + .build()) + .endEnv() + .build(); + } + + private List getContainerPorts() { + return Arrays.asList( + new ContainerPortBuilder() + .withContainerPort(kubernetesJobManagerParameters.getRestPort()) + .build(), + new ContainerPortBuilder() + .withContainerPort(kubernetesJobManagerParameters.getRPCPort()) + .build(), + new ContainerPortBuilder(). + withContainerPort(kubernetesJobManagerParameters.getBlobServerPort()) + .build()); + } + + private List getCustomizedEnvs() { + return kubernetesJobManagerParameters.getEnvironments() + .entrySet() + .stream() + .map(kv -> new EnvVarBuilder() + .withName(kv.getKey()) + .withValue(kv.getValue()) + .build()) + .collect(Collectors.toList()); + } +} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecorator.java new file mode 100644 index 0000000000..660e005e3c --- /dev/null +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecorator.java @@ -0,0 +1,100 @@ +/* + * 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.kubernetes.kubeclient.decorators; + +import org.apache.flink.kubernetes.kubeclient.FlinkPod; +import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesTaskManagerParameters; +import org.apache.flink.kubernetes.utils.Constants; +import org.apache.flink.kubernetes.utils.KubernetesUtils; + +import io.fabric8.kubernetes.api.model.Container; +import io.fabric8.kubernetes.api.model.ContainerBuilder; +import io.fabric8.kubernetes.api.model.ContainerPortBuilder; +import io.fabric8.kubernetes.api.model.EnvVar; +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.PodBuilder; +import io.fabric8.kubernetes.api.model.ResourceRequirements; + +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.kubernetes.utils.Constants.ENV_FLINK_POD_NAME; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * An initializer for the TaskManager {@link org.apache.flink.kubernetes.kubeclient.FlinkPod}. + */ +public class InitTaskManagerDecorator extends AbstractKubernetesStepDecorator { + + private final KubernetesTaskManagerParameters kubernetesTaskManagerParameters; + + public InitTaskManagerDecorator(KubernetesTaskManagerParameters kubernetesTaskManagerParameters) { + this.kubernetesTaskManagerParameters = checkNotNull(kubernetesTaskManagerParameters); + } + + @Override + public FlinkPod decorateFlinkPod(FlinkPod flinkPod) { + final Pod basicPod = new PodBuilder(flinkPod.getPod()) + .withApiVersion(Constants.API_VERSION) + .editOrNewMetadata() + .withName(kubernetesTaskManagerParameters.getPodName()) + .withLabels(kubernetesTaskManagerParameters.getLabels()) + .endMetadata() + .editOrNewSpec() + .withImagePullSecrets(kubernetesTaskManagerParameters.getImagePullSecrets()) + .endSpec() + .build(); + + final Container basicMainContainer = decorateMainContainer(flinkPod.getMainContainer()); + + return new FlinkPod.Builder(flinkPod) + .withPod(basicPod) + .withMainContainer(basicMainContainer) + .build(); + } + + private Container decorateMainContainer(Container container) { + final ResourceRequirements resourceRequirements = KubernetesUtils.getResourceRequirements( + kubernetesTaskManagerParameters.getTaskManagerMemoryMB(), + kubernetesTaskManagerParameters.getTaskManagerCPU()); + + return new ContainerBuilder(container) + .withName(kubernetesTaskManagerParameters.getTaskManagerMainContainerName()) + .withImage(kubernetesTaskManagerParameters.getImage()) + .withImagePullPolicy(kubernetesTaskManagerParameters.getImagePullPolicy()) + .withResources(resourceRequirements) + .withPorts(new ContainerPortBuilder() + .withContainerPort(kubernetesTaskManagerParameters.getRPCPort()) + .build()) + .withEnv(getCustomizedEnvs()) + .addNewEnv() + .withName(ENV_FLINK_POD_NAME) + .withValue(kubernetesTaskManagerParameters.getPodName()) + .endEnv() + .build(); + } + + private List getCustomizedEnvs() { + return kubernetesTaskManagerParameters.getEnvironments() + .entrySet() + .stream() + .map(kv -> new EnvVar(kv.getKey(), kv.getValue(), null)) + .collect(Collectors.toList()); + } +} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/KubernetesStepDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/KubernetesStepDecorator.java new file mode 100644 index 0000000000..7f1e17f064 --- /dev/null +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/KubernetesStepDecorator.java @@ -0,0 +1,47 @@ +/* + * 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.kubernetes.kubeclient.decorators; + +import org.apache.flink.kubernetes.kubeclient.FlinkPod; + +import io.fabric8.kubernetes.api.model.HasMetadata; + +import java.io.IOException; +import java.util.List; + +/** + * A set of functions that together represent a feature in pods that are deployed for + * the JobManager(s) or the TaskManager(s), which provides an extension to the way the + * given Flink application works. + */ +public interface KubernetesStepDecorator { + + /** + * Apply transformations to the given FlinkPod in accordance with this feature. This can include adding + * labels/annotations, mounting volumes, and setting startup command or parameters, etc. + */ + FlinkPod decorateFlinkPod(FlinkPod flinkPod); + + /** + * Build the accompanying Kubernetes resources that should be introduced to support this feature. This could + * only be applicable on the client-side submission process. + */ + List buildAccompanyingKubernetesResources() throws IOException; + +} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java index c3e49726f2..92d6514387 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java @@ -30,7 +30,6 @@ import org.apache.flink.util.FlinkRuntimeException; import io.fabric8.kubernetes.api.model.ConfigMapVolumeSourceBuilder; import io.fabric8.kubernetes.api.model.KeyToPath; -import io.fabric8.kubernetes.api.model.LocalObjectReference; import io.fabric8.kubernetes.api.model.Quantity; import io.fabric8.kubernetes.api.model.ResourceRequirements; import io.fabric8.kubernetes.api.model.ResourceRequirementsBuilder; @@ -302,18 +301,6 @@ public class KubernetesUtils { .build(); } - public static LocalObjectReference[] parseImagePullSecrets(List imagePullSecrets) { - if (imagePullSecrets == null) { - return new LocalObjectReference[0]; - } else { - return imagePullSecrets.stream() - .map(String::trim) - .filter(secret -> !secret.isEmpty()) - .map(LocalObjectReference::new) - .toArray(LocalObjectReference[]::new); - } - } - private static String getJavaOpts(Configuration flinkConfig, ConfigOption configOption) { String baseJavaOpts = flinkConfig.getString(CoreOptions.FLINK_JVM_OPTIONS); diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/KubernetesJobManagerTestBase.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/KubernetesJobManagerTestBase.java new file mode 100644 index 0000000000..73ada490ac --- /dev/null +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/KubernetesJobManagerTestBase.java @@ -0,0 +1,79 @@ +/* + * 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.kubernetes.kubeclient; + +import org.apache.flink.client.deployment.ClusterSpecification; +import org.apache.flink.configuration.BlobServerOptions; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.ResourceManagerOptions; +import org.apache.flink.configuration.RestOptions; +import org.apache.flink.kubernetes.KubernetesTestBase; +import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; +import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters; + +import org.junit.Before; + +import java.util.HashMap; +import java.util.Map; + +/** + * Base test class for the JobManager side. + */ +public class KubernetesJobManagerTestBase extends KubernetesTestBase { + + protected static final double JOB_MANAGER_CPU = 2.0; + protected static final int JOB_MANAGER_MEMORY = 768; + + protected static final int REST_PORT = 9081; + protected static final int RPC_PORT = 7123; + protected static final int BLOB_SERVER_PORT = 8346; + + protected final Map customizedEnvs = new HashMap() { + { + put("key1", "value1"); + put("key2", "value2"); + } + }; + + protected KubernetesJobManagerParameters kubernetesJobManagerParameters; + + protected FlinkPod baseFlinkPod; + + @Before + public void setup() throws Exception { + super.setup(); + + this.flinkConfig.set(RestOptions.PORT, REST_PORT); + this.flinkConfig.set(JobManagerOptions.PORT, RPC_PORT); + this.flinkConfig.set(BlobServerOptions.PORT, Integer.toString(BLOB_SERVER_PORT)); + this.flinkConfig.set(KubernetesConfigOptions.JOB_MANAGER_CPU, JOB_MANAGER_CPU); + this.customizedEnvs.forEach((k, v) -> + this.flinkConfig.setString(ResourceManagerOptions.CONTAINERIZED_MASTER_ENV_PREFIX + k, v)); + + final ClusterSpecification clusterSpecification = new ClusterSpecification.ClusterSpecificationBuilder() + .setMasterMemoryMB(JOB_MANAGER_MEMORY) + .setTaskManagerMemoryMB(1024) + .setSlotsPerTaskManager(3) + .createClusterSpecification(); + + this.kubernetesJobManagerParameters = new KubernetesJobManagerParameters(flinkConfig, clusterSpecification); + + this.baseFlinkPod = new FlinkPod.Builder().build(); + } +} diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/KubernetesTaskManagerTestBase.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/KubernetesTaskManagerTestBase.java new file mode 100644 index 0000000000..a1799a073a --- /dev/null +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/KubernetesTaskManagerTestBase.java @@ -0,0 +1,83 @@ +/* + * 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.kubernetes.kubeclient; + +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.ResourceManagerOptions; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.kubernetes.KubernetesTestBase; +import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesTaskManagerParameters; +import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; +import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec; +import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils; + +import org.junit.Before; + +import java.util.HashMap; +import java.util.Map; + +/** + * Base test class for the TaskManager side. + */ +public class KubernetesTaskManagerTestBase extends KubernetesTestBase { + + protected static final int RPC_PORT = 12345; + + protected static final String POD_NAME = "taskmanager-pod-1"; + private static final String DYNAMIC_PROPERTIES = ""; + + protected static final int TOTAL_PROCESS_MEMORY = 1024; + protected static final double TASK_MANAGER_CPU = 2.0; + + protected final Map customizedEnvs = new HashMap() { + { + put("key1", "value1"); + put("key2", "value2"); + } + }; + + protected TaskExecutorProcessSpec taskExecutorProcessSpec; + + protected ContaineredTaskManagerParameters containeredTaskManagerParameters; + + protected KubernetesTaskManagerParameters kubernetesTaskManagerParameters; + + protected FlinkPod baseFlinkPod = new FlinkPod.Builder().build(); + + @Before + public void setup() throws Exception { + super.setup(); + + flinkConfig.set(TaskManagerOptions.RPC_PORT, String.valueOf(RPC_PORT)); + flinkConfig.set(TaskManagerOptions.CPU_CORES, TASK_MANAGER_CPU); + flinkConfig.set(TaskManagerOptions.TOTAL_PROCESS_MEMORY, MemorySize.parse(TOTAL_PROCESS_MEMORY + "m")); + customizedEnvs.forEach((k, v) -> + flinkConfig.setString(ResourceManagerOptions.CONTAINERIZED_TASK_MANAGER_ENV_PREFIX + k, v)); + + taskExecutorProcessSpec = TaskExecutorProcessUtils.processSpecFromConfig(flinkConfig); + containeredTaskManagerParameters = ContaineredTaskManagerParameters.create(flinkConfig, taskExecutorProcessSpec, + flinkConfig.getInteger(TaskManagerOptions.NUM_TASK_SLOTS)); + kubernetesTaskManagerParameters = new KubernetesTaskManagerParameters( + flinkConfig, + POD_NAME, + TOTAL_PROCESS_MEMORY, + DYNAMIC_PROPERTIES, + containeredTaskManagerParameters); + } +} diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecoratorTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecoratorTest.java new file mode 100644 index 0000000000..57a09c2f2f --- /dev/null +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecoratorTest.java @@ -0,0 +1,152 @@ +/* + * 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.kubernetes.kubeclient.decorators; + +import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; +import org.apache.flink.kubernetes.kubeclient.FlinkPod; +import org.apache.flink.kubernetes.kubeclient.KubernetesJobManagerTestBase; +import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters; +import org.apache.flink.kubernetes.utils.Constants; + +import io.fabric8.kubernetes.api.model.Container; +import io.fabric8.kubernetes.api.model.ContainerPort; +import io.fabric8.kubernetes.api.model.ContainerPortBuilder; +import io.fabric8.kubernetes.api.model.EnvVar; +import io.fabric8.kubernetes.api.model.LocalObjectReference; +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.Quantity; +import io.fabric8.kubernetes.api.model.ResourceRequirements; +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * General tests for the {@link InitJobManagerDecorator}. + */ +public class InitJobManagerDecoratorTest extends KubernetesJobManagerTestBase { + + private static final String SERVICE_ACCOUNT_NAME = "service-test"; + private static final List IMAGE_PULL_SECRETS = Arrays.asList("s1", "s2", "s3"); + + private Pod resultPod; + private Container resultMainContainer; + + @Before + public void setup() throws Exception { + super.setup(); + this.flinkConfig.set(KubernetesConfigOptions.JOB_MANAGER_SERVICE_ACCOUNT, SERVICE_ACCOUNT_NAME); + this.flinkConfig.set(KubernetesConfigOptions.CONTAINER_IMAGE_PULL_SECRETS, IMAGE_PULL_SECRETS); + + final InitJobManagerDecorator initJobManagerDecorator = + new InitJobManagerDecorator(this.kubernetesJobManagerParameters); + final FlinkPod resultFlinkPod = initJobManagerDecorator.decorateFlinkPod(this.baseFlinkPod); + + this.resultPod = resultFlinkPod.getPod(); + this.resultMainContainer = resultFlinkPod.getMainContainer(); + } + + @Test + public void testApiVersion() { + assertEquals(Constants.API_VERSION, this.resultPod.getApiVersion()); + } + + @Test + public void testMainContainerName() { + assertEquals(KubernetesJobManagerParameters.JOB_MANAGER_MAIN_CONTAINER_NAME, this.resultMainContainer.getName()); + } + + @Test + public void testMainContainerImage() { + assertEquals(CONTAINER_IMAGE, this.resultMainContainer.getImage()); + assertEquals(CONTAINER_IMAGE_PULL_POLICY, this.resultMainContainer.getImagePullPolicy()); + } + + @Test + public void testMainContainerResourceRequirements() { + final ResourceRequirements resourceRequirements = this.resultMainContainer.getResources(); + + final Map requests = resourceRequirements.getRequests(); + assertEquals(Double.toString(JOB_MANAGER_CPU), requests.get("cpu").getAmount()); + assertEquals(JOB_MANAGER_MEMORY + "Mi", requests.get("memory").getAmount()); + + final Map limits = resourceRequirements.getLimits(); + assertEquals(Double.toString(JOB_MANAGER_CPU), limits.get("cpu").getAmount()); + assertEquals(JOB_MANAGER_MEMORY + "Mi", limits.get("memory").getAmount()); + } + + @Test + public void testMainContainerPorts() { + final List expectedContainerPorts = Arrays.asList( + new ContainerPortBuilder() + .withContainerPort(REST_PORT) + .build(), + new ContainerPortBuilder() + .withContainerPort(RPC_PORT) + .build(), + new ContainerPortBuilder() + .withContainerPort(BLOB_SERVER_PORT) + .build()); + + assertEquals(expectedContainerPorts, this.resultMainContainer.getPorts()); + } + + @Test + public void testMainContainerEnv() { + final List envVars = this.resultMainContainer.getEnv(); + + final Map envs = new HashMap<>(); + envVars.forEach(env -> envs.put(env.getName(), env.getValue())); + this.customizedEnvs.forEach((k, v) -> assertEquals(envs.get(k), v)); + + assertTrue(envVars.stream().anyMatch(env -> env.getName().equals(Constants.ENV_FLINK_POD_IP_ADDRESS) + && env.getValueFrom().getFieldRef().getApiVersion().equals(Constants.API_VERSION) + && env.getValueFrom().getFieldRef().getFieldPath().equals(Constants.POD_IP_FIELD_PATH))); + } + + @Test + public void testPodLabels() { + final Map expectedLabels = new HashMap<>(getCommonLabels()); + expectedLabels.put(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_JOB_MANAGER); + + assertEquals(expectedLabels, this.resultPod.getMetadata().getLabels()); + } + + @Test + public void testPodServiceAccountName() { + assertEquals(SERVICE_ACCOUNT_NAME, this.resultPod.getSpec().getServiceAccountName()); + } + + @Test + public void testImagePullSecrets() { + final List resultSecrets = this.resultPod.getSpec().getImagePullSecrets() + .stream() + .map(LocalObjectReference::getName) + .collect(Collectors.toList()); + + assertEquals(IMAGE_PULL_SECRETS, resultSecrets); + } +} diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecoratorTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecoratorTest.java new file mode 100644 index 0000000000..2c1cd145f2 --- /dev/null +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecoratorTest.java @@ -0,0 +1,148 @@ +/* + * 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.kubernetes.kubeclient.decorators; + +import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; +import org.apache.flink.kubernetes.kubeclient.FlinkPod; +import org.apache.flink.kubernetes.kubeclient.KubernetesTaskManagerTestBase; +import org.apache.flink.kubernetes.utils.Constants; + +import io.fabric8.kubernetes.api.model.Container; +import io.fabric8.kubernetes.api.model.ContainerPort; +import io.fabric8.kubernetes.api.model.ContainerPortBuilder; +import io.fabric8.kubernetes.api.model.EnvVar; +import io.fabric8.kubernetes.api.model.LocalObjectReference; +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.Quantity; +import io.fabric8.kubernetes.api.model.ResourceRequirements; +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.junit.Assert.assertEquals; + +/** + * General tests for the {@link InitJobManagerDecorator}. + */ +public class InitTaskManagerDecoratorTest extends KubernetesTaskManagerTestBase { + + private static final List IMAGE_PULL_SECRETS = Arrays.asList("s1", "s2", "s3"); + + private Pod resultPod; + private Container resultMainContainer; + + @Before + public void setup() throws Exception { + super.setup(); + this.flinkConfig.set(KubernetesConfigOptions.CONTAINER_IMAGE_PULL_SECRETS, IMAGE_PULL_SECRETS); + + final InitTaskManagerDecorator initTaskManagerDecorator = + new InitTaskManagerDecorator(kubernetesTaskManagerParameters); + + final FlinkPod resultFlinkPod = initTaskManagerDecorator.decorateFlinkPod(this.baseFlinkPod); + this.resultPod = resultFlinkPod.getPod(); + this.resultMainContainer = resultFlinkPod.getMainContainer(); + } + + @Test + public void testApiVersion() { + assertEquals(Constants.API_VERSION, this.resultPod.getApiVersion()); + } + + @Test + public void testMainContainerName() { + assertEquals( + kubernetesTaskManagerParameters.getTaskManagerMainContainerName(), + this.resultMainContainer.getName()); + } + + @Test + public void testMainContainerImage() { + assertEquals(CONTAINER_IMAGE, this.resultMainContainer.getImage()); + } + + @Test + public void testMainContainerImagePullPolicy() { + assertEquals(CONTAINER_IMAGE_PULL_POLICY, this.resultMainContainer.getImagePullPolicy()); + } + + @Test + public void testMainContainerResourceRequirements() { + final ResourceRequirements resourceRequirements = this.resultMainContainer.getResources(); + + final Map requests = resourceRequirements.getRequests(); + assertEquals(Double.toString(TASK_MANAGER_CPU), requests.get("cpu").getAmount()); + assertEquals(TOTAL_PROCESS_MEMORY + "Mi", requests.get("memory").getAmount()); + + final Map limits = resourceRequirements.getLimits(); + assertEquals(Double.toString(TASK_MANAGER_CPU), limits.get("cpu").getAmount()); + assertEquals(TOTAL_PROCESS_MEMORY + "Mi", limits.get("memory").getAmount()); + } + + @Test + public void testMainContainerPorts() { + final List expectedContainerPorts = Collections.singletonList( + new ContainerPortBuilder() + .withContainerPort(RPC_PORT) + .build()); + + assertEquals(expectedContainerPorts, this.resultMainContainer.getPorts()); + } + + @Test + public void testMainContainerEnv() { + final Map expectedEnvVars = new HashMap<>(customizedEnvs); + expectedEnvVars.put(Constants.ENV_FLINK_POD_NAME, POD_NAME); + + final Map resultEnvVars = this.resultMainContainer.getEnv() + .stream() + .collect(Collectors.toMap(EnvVar::getName, EnvVar::getValue)); + + assertEquals(expectedEnvVars, resultEnvVars); + } + + @Test + public void testPodName() { + assertEquals(POD_NAME, this.resultPod.getMetadata().getName()); + } + + @Test + public void testPodLabels() { + final Map expectedLabels = new HashMap<>(getCommonLabels()); + expectedLabels.put(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_TASK_MANAGER); + + assertEquals(expectedLabels, this.resultPod.getMetadata().getLabels()); + } + + @Test + public void testImagePullSecrets() { + final List resultSecrets = this.resultPod.getSpec().getImagePullSecrets() + .stream() + .map(LocalObjectReference::getName) + .collect(Collectors.toList()); + + assertEquals(IMAGE_PULL_SECRETS, resultSecrets); + } +} diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/utils/KubernetesUtilsTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/utils/KubernetesUtilsTest.java index 64ed94f08e..ca66c600eb 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/utils/KubernetesUtilsTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/utils/KubernetesUtilsTest.java @@ -33,20 +33,13 @@ import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils; import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.TestLogger; -import io.fabric8.kubernetes.api.model.LocalObjectReference; import org.junit.Test; -import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; -import java.util.stream.IntStream; -import static org.apache.flink.kubernetes.configuration.KubernetesConfigOptions.CONTAINER_IMAGE_PULL_SECRETS; -import static org.apache.flink.kubernetes.utils.KubernetesUtils.parseImagePullSecrets; import static org.hamcrest.CoreMatchers.containsString; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; -import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; /** @@ -245,23 +238,6 @@ public class KubernetesUtilsTest extends TestLogger { } } - @Test - public void testParseImagePullSecrets() { - final Configuration cfg = new Configuration(); - final LocalObjectReference[] noSecrets = parseImagePullSecrets(cfg.get(CONTAINER_IMAGE_PULL_SECRETS)); - assertEquals(0, noSecrets.length); - - cfg.set(CONTAINER_IMAGE_PULL_SECRETS, Collections.singletonList("s1")); - final LocalObjectReference[] oneSecret = parseImagePullSecrets(cfg.get(CONTAINER_IMAGE_PULL_SECRETS)); - assertTrue(oneSecret.length == 1 && oneSecret[0].getName().equals("s1")); - - cfg.set(CONTAINER_IMAGE_PULL_SECRETS, Arrays.asList("s1", "s2", "s3", "s4")); - final LocalObjectReference[] commonSeparatedSecrets = parseImagePullSecrets(cfg.get(CONTAINER_IMAGE_PULL_SECRETS)); - final String[] expectedSecrets = new String[]{"s1", "s2", "s3", "s4"}; - assertEquals(4, commonSeparatedSecrets.length); - IntStream.range(0, 3).forEach(i -> assertEquals(expectedSecrets[i], commonSeparatedSecrets[i].getName())); - } - @Test public void testCheckWithDynamicPort() { testCheckAndUpdatePortConfigOption("0", "6123", "6123"); -- Gitee From 7f19c9cf14162c0f1b49e0b7a9bd7a0134cc7e3e Mon Sep 17 00:00:00 2001 From: felixzheng Date: Sun, 23 Feb 2020 15:19:52 +0800 Subject: [PATCH 056/885] [FLINK-16194][k8s] Introduce Java command decorator for the main Container of the JobManager and the TaskManger Pod --- .../JavaCmdJobManagerDecorator.java | 99 +++++++ .../JavaCmdTaskManagerDecorator.java | 107 ++++++++ .../kubernetes/utils/KubernetesUtils.java | 93 +++---- .../flink/kubernetes/KubernetesTestBase.java | 2 +- .../flink/kubernetes/KubernetesTestUtils.java | 35 +++ .../JavaCmdJobManagerDecoratorTest.java | 241 ++++++++++++++++++ .../JavaCmdTaskManagerDecoratorTest.java | 241 ++++++++++++++++++ .../kubernetes/utils/KubernetesUtilsTest.java | 224 ---------------- 8 files changed, 753 insertions(+), 289 deletions(-) create mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/JavaCmdJobManagerDecorator.java create mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/JavaCmdTaskManagerDecorator.java create mode 100644 flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesTestUtils.java create mode 100644 flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/JavaCmdJobManagerDecoratorTest.java create mode 100644 flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/JavaCmdTaskManagerDecoratorTest.java diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/JavaCmdJobManagerDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/JavaCmdJobManagerDecorator.java new file mode 100644 index 0000000000..11900f4e9f --- /dev/null +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/JavaCmdJobManagerDecorator.java @@ -0,0 +1,99 @@ +/* + * 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.kubernetes.kubeclient.decorators; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.kubernetes.kubeclient.FlinkPod; +import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters; +import org.apache.flink.kubernetes.utils.KubernetesUtils; +import org.apache.flink.runtime.clusterframework.BootstrapTools; + +import io.fabric8.kubernetes.api.model.Container; +import io.fabric8.kubernetes.api.model.ContainerBuilder; + +import java.util.Arrays; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Attach the jvm command and args to the main container for running the JobManager code. + */ +public class JavaCmdJobManagerDecorator extends AbstractKubernetesStepDecorator { + + private final KubernetesJobManagerParameters kubernetesJobManagerParameters; + + public JavaCmdJobManagerDecorator(KubernetesJobManagerParameters kubernetesJobManagerParameters) { + this.kubernetesJobManagerParameters = checkNotNull(kubernetesJobManagerParameters); + } + + @Override + public FlinkPod decorateFlinkPod(FlinkPod flinkPod) { + final String startCommand = getJobManagerStartCommand( + kubernetesJobManagerParameters.getFlinkConfiguration(), + kubernetesJobManagerParameters.getJobManagerMemoryMB(), + kubernetesJobManagerParameters.getFlinkConfDirInPod(), + kubernetesJobManagerParameters.getFlinkLogDirInPod(), + kubernetesJobManagerParameters.hasLogback(), + kubernetesJobManagerParameters.hasLog4j(), + kubernetesJobManagerParameters.getEntrypointClass()); + + final Container mainContainerWithStartCmd = new ContainerBuilder(flinkPod.getMainContainer()) + .withCommand(kubernetesJobManagerParameters.getContainerEntrypoint()) + .withArgs(Arrays.asList("/bin/bash", "-c", startCommand)) + .build(); + + return new FlinkPod.Builder(flinkPod) + .withMainContainer(mainContainerWithStartCmd) + .build(); + } + + /** + * Generates the shell command to start a jobmanager for kubernetes. + * + * @param flinkConfig The Flink configuration. + * @param jobManagerMemoryMb JobManager heap size. + * @param configDirectory The configuration directory for the flink-conf.yaml + * @param logDirectory The log directory. + * @param hasLogback Uses logback? + * @param hasLog4j Uses log4j? + * @param mainClass The main class to start with. + * @return A String containing the job manager startup command. + */ + private static String getJobManagerStartCommand( + Configuration flinkConfig, + int jobManagerMemoryMb, + String configDirectory, + String logDirectory, + boolean hasLogback, + boolean hasLog4j, + String mainClass) { + final int heapSize = BootstrapTools.calculateHeapSize(jobManagerMemoryMb, flinkConfig); + final String jvmMemOpts = String.format("-Xms%sm -Xmx%sm", heapSize, heapSize); + return KubernetesUtils.getCommonStartCommand( + flinkConfig, + KubernetesUtils.ClusterComponent.JOB_MANAGER, + jvmMemOpts, + configDirectory, + logDirectory, + hasLogback, + hasLog4j, + mainClass, + null); + } +} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/JavaCmdTaskManagerDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/JavaCmdTaskManagerDecorator.java new file mode 100644 index 0000000000..34332d62ef --- /dev/null +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/JavaCmdTaskManagerDecorator.java @@ -0,0 +1,107 @@ +/* + * 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.kubernetes.kubeclient.decorators; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.kubernetes.kubeclient.FlinkPod; +import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesTaskManagerParameters; +import org.apache.flink.kubernetes.taskmanager.KubernetesTaskExecutorRunner; +import org.apache.flink.kubernetes.utils.KubernetesUtils; +import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; +import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec; +import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils; +import org.apache.flink.runtime.entrypoint.parser.CommandLineOptions; + +import io.fabric8.kubernetes.api.model.Container; +import io.fabric8.kubernetes.api.model.ContainerBuilder; + +import java.util.Arrays; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Attach the jvm command and args to the main container for running the TaskManager code. + */ +public class JavaCmdTaskManagerDecorator extends AbstractKubernetesStepDecorator { + + private final KubernetesTaskManagerParameters kubernetesTaskManagerParameters; + + public JavaCmdTaskManagerDecorator(KubernetesTaskManagerParameters kubernetesTaskManagerParameters) { + this.kubernetesTaskManagerParameters = checkNotNull(kubernetesTaskManagerParameters); + } + + @Override + public FlinkPod decorateFlinkPod(FlinkPod flinkPod) { + final Container mainContainerWithStartCmd = new ContainerBuilder(flinkPod.getMainContainer()) + .withCommand(kubernetesTaskManagerParameters.getContainerEntrypoint()) + .withArgs(Arrays.asList("/bin/bash", "-c", getTaskManagerStartCommand())) + .build(); + + return new FlinkPod.Builder(flinkPod) + .withMainContainer(mainContainerWithStartCmd) + .build(); + } + + private String getTaskManagerStartCommand() { + final String confDirInPod = kubernetesTaskManagerParameters.getFlinkConfDirInPod(); + + final String logDirInPod = kubernetesTaskManagerParameters.getFlinkLogDirInPod(); + + final String mainClassArgs = "--" + CommandLineOptions.CONFIG_DIR_OPTION.getLongOpt() + " " + + confDirInPod + " " + kubernetesTaskManagerParameters.getDynamicProperties(); + + return getTaskManagerStartCommand( + kubernetesTaskManagerParameters.getFlinkConfiguration(), + kubernetesTaskManagerParameters.getContaineredTaskManagerParameters(), + confDirInPod, + logDirInPod, + kubernetesTaskManagerParameters.hasLogback(), + kubernetesTaskManagerParameters.hasLog4j(), + KubernetesTaskExecutorRunner.class.getCanonicalName(), + mainClassArgs); + } + + private static String getTaskManagerStartCommand( + Configuration flinkConfig, + ContaineredTaskManagerParameters tmParams, + String configDirectory, + String logDirectory, + boolean hasLogback, + boolean hasLog4j, + String mainClass, + String mainArgs) { + final TaskExecutorProcessSpec taskExecutorProcessSpec = tmParams.getTaskExecutorProcessSpec(); + final String jvmMemOpts = TaskExecutorProcessUtils.generateJvmParametersStr(taskExecutorProcessSpec); + String args = TaskExecutorProcessUtils.generateDynamicConfigsStr(taskExecutorProcessSpec); + if (mainArgs != null) { + args += " " + mainArgs; + } + + return KubernetesUtils.getCommonStartCommand( + flinkConfig, + KubernetesUtils.ClusterComponent.TASK_MANAGER, + jvmMemOpts, + configDirectory, + logDirectory, + hasLogback, + hasLog4j, + mainClass, + args); + } +} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java index 92d6514387..d5bd27a940 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java @@ -127,43 +127,6 @@ public class KubernetesUtils { } } - /** - * Generates the shell command to start a job manager for kubernetes. - * - * @param flinkConfig The Flink configuration. - * @param jobManagerMemoryMb JobManager heap size. - * @param configDirectory The configuration directory for the flink-conf.yaml - * @param logDirectory The log directory. - * @param hasLogback Uses logback? - * @param hasLog4j Uses log4j? - * @param mainClass The main class to start with. - * @param mainArgs The args for main class. - * @return A String containing the job manager startup command. - */ - public static String getJobManagerStartCommand( - Configuration flinkConfig, - int jobManagerMemoryMb, - String configDirectory, - String logDirectory, - boolean hasLogback, - boolean hasLog4j, - String mainClass, - @Nullable String mainArgs) { - final int heapSize = BootstrapTools.calculateHeapSize(jobManagerMemoryMb, flinkConfig); - final String jvmMemOpts = String.format("-Xms%sm -Xmx%sm", heapSize, heapSize); - return getCommonStartCommand( - flinkConfig, - ClusterComponent.JOB_MANAGER, - jvmMemOpts, - configDirectory, - logDirectory, - hasLogback, - hasLog4j, - mainClass, - mainArgs - ); - } - /** * Generates the shell command to start a task manager for kubernetes. * @@ -301,32 +264,7 @@ public class KubernetesUtils { .build(); } - private static String getJavaOpts(Configuration flinkConfig, ConfigOption configOption) { - String baseJavaOpts = flinkConfig.getString(CoreOptions.FLINK_JVM_OPTIONS); - - if (flinkConfig.getString(configOption).length() > 0) { - return baseJavaOpts + " " + flinkConfig.getString(configOption); - } else { - return baseJavaOpts; - } - } - - private static String getLogging(String logFile, String confDir, boolean hasLogback, boolean hasLog4j) { - StringBuilder logging = new StringBuilder(); - if (hasLogback || hasLog4j) { - logging.append("-Dlog.file=").append(logFile); - if (hasLogback) { - logging.append(" -Dlogback.configurationFile=file:").append(confDir).append("/logback.xml"); - } - if (hasLog4j) { - logging.append(" -Dlog4j.configuration=file:").append(confDir).append("/log4j.properties"); - logging.append(" -Dlog4j.configurationFile=file:").append(confDir).append("/log4j.properties"); - } - } - return logging.toString(); - } - - private static String getCommonStartCommand( + public static String getCommonStartCommand( Configuration flinkConfig, ClusterComponent mode, String jvmMemOpts, @@ -368,7 +306,34 @@ public class KubernetesUtils { return BootstrapTools.getStartCommand(commandTemplate, startCommandValues); } - private enum ClusterComponent { + private static String getJavaOpts(Configuration flinkConfig, ConfigOption configOption) { + String baseJavaOpts = flinkConfig.getString(CoreOptions.FLINK_JVM_OPTIONS); + + if (flinkConfig.getString(configOption).length() > 0) { + return baseJavaOpts + " " + flinkConfig.getString(configOption); + } else { + return baseJavaOpts; + } + } + + private static String getLogging(String logFile, String confDir, boolean hasLogback, boolean hasLog4j) { + StringBuilder logging = new StringBuilder(); + if (hasLogback || hasLog4j) { + logging.append("-Dlog.file=").append(logFile); + if (hasLogback) { + logging.append(" -Dlogback.configurationFile=file:").append(confDir).append("/logback.xml"); + } + if (hasLog4j) { + logging.append(" -Dlog4j.configurationFile=file:").append(confDir).append("/log4j.properties"); + } + } + return logging.toString(); + } + + /** + * Cluster components. + */ + public enum ClusterComponent { JOB_MANAGER, TASK_MANAGER } diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesTestBase.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesTestBase.java index cf6a578c44..05de0368ef 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesTestBase.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesTestBase.java @@ -54,7 +54,7 @@ public class KubernetesTestBase extends TestLogger { @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); - private File flinkConfDir; + protected File flinkConfDir; protected final Configuration flinkConfig = new Configuration(); diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesTestUtils.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesTestUtils.java new file mode 100644 index 0000000000..f9dea847bc --- /dev/null +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesTestUtils.java @@ -0,0 +1,35 @@ +/* + * 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.kubernetes; + +import org.apache.flink.shaded.guava18.com.google.common.io.Files; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; + +/** + * Utilities for the Kubernetes tests. + */ +public class KubernetesTestUtils { + + public static void createTemporyFile(String data, File directory, String fileName) throws IOException { + Files.write(data, new File(directory, fileName), StandardCharsets.UTF_8); + } +} diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/JavaCmdJobManagerDecoratorTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/JavaCmdJobManagerDecoratorTest.java new file mode 100644 index 0000000000..36ccd1ef09 --- /dev/null +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/JavaCmdJobManagerDecoratorTest.java @@ -0,0 +1,241 @@ +/* + * 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.kubernetes.kubeclient.decorators; + +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.kubernetes.KubernetesTestUtils; +import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; +import org.apache.flink.kubernetes.configuration.KubernetesConfigOptionsInternal; +import org.apache.flink.kubernetes.entrypoint.KubernetesSessionClusterEntrypoint; +import org.apache.flink.kubernetes.kubeclient.FlinkPod; +import org.apache.flink.kubernetes.kubeclient.KubernetesJobManagerTestBase; + +import io.fabric8.kubernetes.api.model.Container; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; + +/** + * General tests for the {@link JavaCmdJobManagerDecorator}. + */ +public class JavaCmdJobManagerDecoratorTest extends KubernetesJobManagerTestBase { + + private static final String KUBERNETES_ENTRY_PATH = "/opt/bin/start.sh"; + private static final String FLINK_CONF_DIR_IN_POD = "/opt/flink/flink-conf-"; + private static final String FLINK_LOG_DIR_IN_POD = "/opt/flink/flink-log-"; + private static final String ENTRY_POINT_CLASS = KubernetesSessionClusterEntrypoint.class.getCanonicalName(); + + private static final String java = "$JAVA_HOME/bin/java"; + private static final String classpath = "-classpath $FLINK_CLASSPATH"; + private static final String jvmOpts = "-Djvm"; + + // Logging variables + private static final String logback = + String.format("-Dlogback.configurationFile=file:%s/logback.xml", FLINK_CONF_DIR_IN_POD); + private static final String log4j = + String.format("-Dlog4j.configurationFile=file:%s/log4j.properties", FLINK_CONF_DIR_IN_POD); + private static final String jmLogfile = String.format("-Dlog.file=%s/jobmanager.log", FLINK_LOG_DIR_IN_POD); + private static final String jmLogRedirects = + String.format("1> %s/jobmanager.out 2> %s/jobmanager.err", + FLINK_LOG_DIR_IN_POD, FLINK_LOG_DIR_IN_POD); + + // Memory variables + private static final String jmJvmMem = String.format("-Xms%dm -Xmx%dm", + JOB_MANAGER_MEMORY - 600, JOB_MANAGER_MEMORY - 600); + + private JavaCmdJobManagerDecorator javaCmdJobManagerDecorator; + + @Before + public void setup() throws Exception { + super.setup(); + + flinkConfig.set(KubernetesConfigOptions.FLINK_CONF_DIR, FLINK_CONF_DIR_IN_POD); + flinkConfig.set(KubernetesConfigOptions.FLINK_LOG_DIR, FLINK_LOG_DIR_IN_POD); + flinkConfig.set(KubernetesConfigOptionsInternal.ENTRY_POINT_CLASS, ENTRY_POINT_CLASS); + flinkConfig.set(KubernetesConfigOptions.KUBERNETES_ENTRY_PATH, KUBERNETES_ENTRY_PATH); + + this.javaCmdJobManagerDecorator = new JavaCmdJobManagerDecorator(kubernetesJobManagerParameters); + } + + @Test + public void testWhetherContainerOrPodIsReplaced() { + final FlinkPod resultFlinkPod = javaCmdJobManagerDecorator.decorateFlinkPod(baseFlinkPod); + assertEquals(baseFlinkPod.getPod(), resultFlinkPod.getPod()); + assertNotEquals(baseFlinkPod.getMainContainer(), resultFlinkPod.getMainContainer()); + } + + @Test + public void testStartCommandWithoutLog4jAndLogback() { + final Container resultMainContainer = + javaCmdJobManagerDecorator.decorateFlinkPod(baseFlinkPod).getMainContainer(); + + assertEquals(Collections.singletonList(KUBERNETES_ENTRY_PATH), resultMainContainer.getCommand()); + + final String expectedCommand = getJobManagerExpectedCommand("", ""); + final List expectedArgs = Arrays.asList("/bin/bash", "-c", expectedCommand); + + assertEquals(expectedArgs, resultMainContainer.getArgs()); + } + + @Test + public void testStartCommandWithLog4j() throws IOException { + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "log4j.properties"); + + final Container resultMainContainer = + javaCmdJobManagerDecorator.decorateFlinkPod(baseFlinkPod).getMainContainer(); + + assertEquals(Collections.singletonList(KUBERNETES_ENTRY_PATH), resultMainContainer.getCommand()); + + final String expectedCommand = getJobManagerExpectedCommand("", log4j); + final List expectedArgs = Arrays.asList("/bin/bash", "-c", expectedCommand); + assertEquals(expectedArgs, resultMainContainer.getArgs()); + } + + @Test + public void testStartCommandWithLogback() throws IOException { + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "logback.xml"); + + final Container resultMainContainer = + javaCmdJobManagerDecorator.decorateFlinkPod(baseFlinkPod).getMainContainer(); + + assertEquals(Collections.singletonList(KUBERNETES_ENTRY_PATH), resultMainContainer.getCommand()); + + final String expectedCommand = getJobManagerExpectedCommand("", logback); + final List expectedArgs = Arrays.asList("/bin/bash", "-c", expectedCommand); + assertEquals(expectedArgs, resultMainContainer.getArgs()); + } + + @Test + public void testStartCommandWithLog4jAndLogback() throws IOException { + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "log4j.properties"); + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "logback.xml"); + + final Container resultMainContainer = + javaCmdJobManagerDecorator.decorateFlinkPod(baseFlinkPod).getMainContainer(); + + assertEquals(Collections.singletonList(KUBERNETES_ENTRY_PATH), resultMainContainer.getCommand()); + + final String expectedCommand = + getJobManagerExpectedCommand("", logback + " " + log4j); + final List expectedArgs = Arrays.asList("/bin/bash", "-c", expectedCommand); + assertEquals(expectedArgs, resultMainContainer.getArgs()); + } + + @Test + public void testStartCommandWithLogAndJVMOpts() throws IOException { + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "log4j.properties"); + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "logback.xml"); + + flinkConfig.set(CoreOptions.FLINK_JVM_OPTIONS, jvmOpts); + final Container resultMainContainer = + javaCmdJobManagerDecorator.decorateFlinkPod(baseFlinkPod).getMainContainer(); + + assertEquals(Collections.singletonList(KUBERNETES_ENTRY_PATH), resultMainContainer.getCommand()); + + final String expectedCommand = + getJobManagerExpectedCommand(jvmOpts, logback + " " + log4j); + final List expectedArgs = Arrays.asList("/bin/bash", "-c", expectedCommand); + assertEquals(expectedArgs, resultMainContainer.getArgs()); + } + + @Test + public void testStartCommandWithLogAndJMOpts() throws IOException { + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "log4j.properties"); + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "logback.xml"); + + flinkConfig.set(CoreOptions.FLINK_JM_JVM_OPTIONS, jvmOpts); + final Container resultMainContainer = + javaCmdJobManagerDecorator.decorateFlinkPod(baseFlinkPod).getMainContainer(); + + assertEquals(Collections.singletonList(KUBERNETES_ENTRY_PATH), resultMainContainer.getCommand()); + final String expectedCommand = getJobManagerExpectedCommand(jvmOpts, logback + " " + log4j); + final List expectedArgs = Arrays.asList("/bin/bash", "-c", expectedCommand); + assertEquals(expectedArgs, resultMainContainer.getArgs()); + } + + @Test + public void testContainerStartCommandTemplate1() throws IOException { + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "log4j.properties"); + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "logback.xml"); + + final String containerStartCommandTemplate = + "%java% 1 %classpath% 2 %jvmmem% %jvmopts% %logging% %class% %args% %redirects%"; + this.flinkConfig.set(KubernetesConfigOptions.CONTAINER_START_COMMAND_TEMPLATE, + containerStartCommandTemplate); + + final String jmJvmOpts = "-DjmJvm"; + this.flinkConfig.setString(CoreOptions.FLINK_JVM_OPTIONS, jvmOpts); + this.flinkConfig.setString(CoreOptions.FLINK_JM_JVM_OPTIONS, jmJvmOpts); + + final Container resultMainContainer = + javaCmdJobManagerDecorator.decorateFlinkPod(baseFlinkPod).getMainContainer(); + + assertEquals(Collections.singletonList(KUBERNETES_ENTRY_PATH), resultMainContainer.getCommand()); + + final String expectedCommand = java + " 1 " + classpath + " 2 " + jmJvmMem + + " " + jvmOpts + " " + jmJvmOpts + + " " + jmLogfile + " " + logback + " " + log4j + + " " + ENTRY_POINT_CLASS + " " + jmLogRedirects; + + final List expectedArgs = Arrays.asList("/bin/bash", "-c", expectedCommand); + assertEquals(resultMainContainer.getArgs(), expectedArgs); + } + + @Test + public void testContainerStartCommandTemplate2() throws IOException { + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "log4j.properties"); + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "logback.xml"); + + final String containerStartCommandTemplate = + "%java% %jvmmem% %logging% %jvmopts% %class% %args% %redirects%"; + this.flinkConfig.set(KubernetesConfigOptions.CONTAINER_START_COMMAND_TEMPLATE, + containerStartCommandTemplate); + + final String jmJvmOpts = "-DjmJvm"; + this.flinkConfig.setString(CoreOptions.FLINK_JVM_OPTIONS, jvmOpts); + this.flinkConfig.setString(CoreOptions.FLINK_JM_JVM_OPTIONS, jmJvmOpts); + + final Container resultMainContainer = + javaCmdJobManagerDecorator.decorateFlinkPod(baseFlinkPod).getMainContainer(); + + assertEquals(Collections.singletonList(KUBERNETES_ENTRY_PATH), resultMainContainer.getCommand()); + + final String expectedCommand = java + " " + jmJvmMem + + " " + jmLogfile + " " + logback + " " + log4j + + " " + jvmOpts + " " + jmJvmOpts + + " " + ENTRY_POINT_CLASS + " " + jmLogRedirects; + + final List expectedArgs = Arrays.asList("/bin/bash", "-c", expectedCommand); + assertEquals(resultMainContainer.getArgs(), expectedArgs); + } + + private String getJobManagerExpectedCommand(String jvmAllOpts, String logging) { + return java + " " + classpath + " " + jmJvmMem + + (jvmAllOpts.isEmpty() ? "" : " " + jvmAllOpts) + + (logging.isEmpty() ? "" : " " + jmLogfile + " " + logging) + + " " + ENTRY_POINT_CLASS + " " + jmLogRedirects; + } +} diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/JavaCmdTaskManagerDecoratorTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/JavaCmdTaskManagerDecoratorTest.java new file mode 100644 index 0000000000..f3653a8e4b --- /dev/null +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/JavaCmdTaskManagerDecoratorTest.java @@ -0,0 +1,241 @@ +/* + * 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.kubernetes.kubeclient.decorators; + +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.kubernetes.KubernetesTestUtils; +import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; +import org.apache.flink.kubernetes.kubeclient.FlinkPod; +import org.apache.flink.kubernetes.kubeclient.KubernetesTaskManagerTestBase; +import org.apache.flink.kubernetes.taskmanager.KubernetesTaskExecutorRunner; +import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils; + +import io.fabric8.kubernetes.api.model.Container; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; + +/** + * General tests for the{@link JavaCmdTaskManagerDecorator}. + */ +public class JavaCmdTaskManagerDecoratorTest extends KubernetesTaskManagerTestBase { + + private static final String KUBERNETES_ENTRY_PATH = "/opt/flink/bin/start.sh"; + private static final String FLINK_CONF_DIR_IN_POD = "/opt/flink/flink-conf-"; + private static final String FLINK_LOG_DIR_IN_POD = "/opt/flink/flink-log-"; + + private static final String java = "$JAVA_HOME/bin/java"; + private static final String classpath = "-classpath $FLINK_CLASSPATH"; + private static final String jvmOpts = "-Djvm"; + + private static final String tmJvmMem = + "-Xmx251658235 -Xms251658235 -XX:MaxDirectMemorySize=211392922 -XX:MaxMetaspaceSize=100663296"; + + private static final String mainClass = KubernetesTaskExecutorRunner.class.getCanonicalName(); + private String mainClassArgs; + + // Logging variables + private static final String logback = + String.format("-Dlogback.configurationFile=file:%s/logback.xml", FLINK_CONF_DIR_IN_POD); + private static final String log4j = + String.format("-Dlog4j.configurationFile=file:%s/log4j.properties", FLINK_CONF_DIR_IN_POD); + private static final String tmLogfile = + String.format("-Dlog.file=%s/taskmanager.log", FLINK_LOG_DIR_IN_POD); + private static final String tmLogRedirects = String.format( + "1> %s/taskmanager.out 2> %s/taskmanager.err", + FLINK_LOG_DIR_IN_POD, + FLINK_LOG_DIR_IN_POD); + + private JavaCmdTaskManagerDecorator javaCmdTaskManagerDecorator; + + @Before + public void setup() throws Exception { + super.setup(); + + flinkConfig.setString(KubernetesConfigOptions.KUBERNETES_ENTRY_PATH, KUBERNETES_ENTRY_PATH); + flinkConfig.set(KubernetesConfigOptions.FLINK_CONF_DIR, FLINK_CONF_DIR_IN_POD); + flinkConfig.set(KubernetesConfigOptions.FLINK_LOG_DIR, FLINK_LOG_DIR_IN_POD); + + this.mainClassArgs = String.format( + "%s--configDir %s", + TaskExecutorProcessUtils.generateDynamicConfigsStr(taskExecutorProcessSpec), + FLINK_CONF_DIR_IN_POD); + + this.javaCmdTaskManagerDecorator = new JavaCmdTaskManagerDecorator(this.kubernetesTaskManagerParameters); + } + + @Test + public void testWhetherContainerOrPodIsUpdated() { + final FlinkPod resultFlinkPod = javaCmdTaskManagerDecorator.decorateFlinkPod(this.baseFlinkPod); + assertEquals(this.baseFlinkPod.getPod(), resultFlinkPod.getPod()); + assertNotEquals(this.baseFlinkPod.getMainContainer(), resultFlinkPod.getMainContainer()); + } + + @Test + public void testStartCommandWithoutLog4jAndLogback() { + final Container resultMainContainer = + javaCmdTaskManagerDecorator.decorateFlinkPod(this.baseFlinkPod).getMainContainer(); + assertEquals(Collections.singletonList(KUBERNETES_ENTRY_PATH), resultMainContainer.getCommand()); + + final String expectedCommand = getTaskManagerExpectedCommand("", ""); + final List expectedArgs = Arrays.asList("/bin/bash", "-c", expectedCommand); + assertEquals(expectedArgs, resultMainContainer.getArgs()); + } + + @Test + public void testStartCommandWithLog4j() throws IOException { + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "log4j.properties"); + + final Container resultMainContainer = + javaCmdTaskManagerDecorator.decorateFlinkPod(this.baseFlinkPod).getMainContainer(); + + assertEquals(Collections.singletonList(KUBERNETES_ENTRY_PATH), resultMainContainer.getCommand()); + + final String expectedCommand = getTaskManagerExpectedCommand("", log4j); + final List expectedArgs = Arrays.asList("/bin/bash", "-c", expectedCommand); + assertEquals(expectedArgs, resultMainContainer.getArgs()); + } + + @Test + public void testStartCommandWithLogback() throws IOException { + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "logback.xml"); + + final Container resultMainContainer = + javaCmdTaskManagerDecorator.decorateFlinkPod(this.baseFlinkPod).getMainContainer(); + + assertEquals(Collections.singletonList(KUBERNETES_ENTRY_PATH), resultMainContainer.getCommand()); + + final String expectedCommand = getTaskManagerExpectedCommand("", logback); + final List expectedArgs = Arrays.asList("/bin/bash", "-c", expectedCommand); + assertEquals(expectedArgs, resultMainContainer.getArgs()); + } + + @Test + public void testStartCommandWithLog4jAndLogback() throws IOException { + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "logback.xml"); + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "log4j.properties"); + + final Container resultMainContainer = + javaCmdTaskManagerDecorator.decorateFlinkPod(this.baseFlinkPod).getMainContainer(); + assertEquals(Collections.singletonList(KUBERNETES_ENTRY_PATH), resultMainContainer.getCommand()); + + final String expectedCommand = getTaskManagerExpectedCommand("", logback + " " + log4j); + final List expectedArgs = Arrays.asList("/bin/bash", "-c", expectedCommand); + assertEquals(expectedArgs, resultMainContainer.getArgs()); + } + + @Test + public void testStartCommandWithLogAndJVMOpts() throws IOException { + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "log4j.properties"); + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "logback.xml"); + + flinkConfig.set(CoreOptions.FLINK_JVM_OPTIONS, jvmOpts); + final Container resultMainContainer = + javaCmdTaskManagerDecorator.decorateFlinkPod(baseFlinkPod).getMainContainer(); + + assertEquals(Collections.singletonList(KUBERNETES_ENTRY_PATH), resultMainContainer.getCommand()); + + final String expectedCommand = + getTaskManagerExpectedCommand(jvmOpts, logback + " " + log4j); + final List expectedArgs = Arrays.asList("/bin/bash", "-c", expectedCommand); + assertEquals(expectedArgs, resultMainContainer.getArgs()); + } + + @Test + public void testStartCommandWithLogAndJMOpts() throws IOException { + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "log4j.properties"); + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "logback.xml"); + + flinkConfig.set(CoreOptions.FLINK_TM_JVM_OPTIONS, jvmOpts); + final Container resultMainContainer = + javaCmdTaskManagerDecorator.decorateFlinkPod(baseFlinkPod).getMainContainer(); + + assertEquals(Collections.singletonList(KUBERNETES_ENTRY_PATH), resultMainContainer.getCommand()); + final String expectedCommand = getTaskManagerExpectedCommand(jvmOpts, logback + " " + log4j); + final List expectedArgs = Arrays.asList("/bin/bash", "-c", expectedCommand); + assertEquals(expectedArgs, resultMainContainer.getArgs()); + } + + @Test + public void testContainerStartCommandTemplate1() throws IOException { + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "log4j.properties"); + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "logback.xml"); + + final String containerStartCommandTemplate = + "%java% 1 %classpath% 2 %jvmmem% %jvmopts% %logging% %class% %args% %redirects%"; + this.flinkConfig.set(KubernetesConfigOptions.CONTAINER_START_COMMAND_TEMPLATE, + containerStartCommandTemplate); + + final String tmJvmOpts = "-DjmJvm"; + this.flinkConfig.setString(CoreOptions.FLINK_JVM_OPTIONS, jvmOpts); + this.flinkConfig.setString(CoreOptions.FLINK_TM_JVM_OPTIONS, tmJvmOpts); + + final Container resultMainContainer = + javaCmdTaskManagerDecorator.decorateFlinkPod(baseFlinkPod).getMainContainer(); + assertEquals(Collections.singletonList(KUBERNETES_ENTRY_PATH), resultMainContainer.getCommand()); + + final String expectedCommand = java + " 1 " + classpath + " 2 " + tmJvmMem + + " " + jvmOpts + " " + tmJvmOpts + + " " + tmLogfile + " " + logback + " " + log4j + + " " + mainClass + " " + mainClassArgs + " " + tmLogRedirects; + final List expectedArgs = Arrays.asList("/bin/bash", "-c", expectedCommand); + assertEquals(resultMainContainer.getArgs(), expectedArgs); + } + + @Test + public void testContainerStartCommandTemplate2() throws IOException { + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "log4j.properties"); + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "logback.xml"); + + final String containerStartCommandTemplate = + "%java% %jvmmem% %logging% %jvmopts% %class% %args% %redirects%"; + this.flinkConfig.set(KubernetesConfigOptions.CONTAINER_START_COMMAND_TEMPLATE, + containerStartCommandTemplate); + + final String tmJvmOpts = "-DjmJvm"; + this.flinkConfig.setString(CoreOptions.FLINK_JVM_OPTIONS, jvmOpts); + this.flinkConfig.setString(CoreOptions.FLINK_TM_JVM_OPTIONS, tmJvmOpts); + + final Container resultMainContainer = + javaCmdTaskManagerDecorator.decorateFlinkPod(baseFlinkPod).getMainContainer(); + + assertEquals(Collections.singletonList(KUBERNETES_ENTRY_PATH), resultMainContainer.getCommand()); + + final String expectedCommand = java + " " + tmJvmMem + + " " + tmLogfile + " " + logback + " " + log4j + + " " + jvmOpts + " " + tmJvmOpts + " " + mainClass + + " " + mainClassArgs + " " + tmLogRedirects; + final List expectedArgs = Arrays.asList("/bin/bash", "-c", expectedCommand); + assertEquals(resultMainContainer.getArgs(), expectedArgs); + } + + private String getTaskManagerExpectedCommand(String jvmAllOpts, String logging) { + return java + " " + classpath + " " + tmJvmMem + + (jvmAllOpts.isEmpty() ? "" : " " + jvmAllOpts) + + (logging.isEmpty() ? "" : " " + tmLogfile + " " + logging) + + " " + mainClass + " " + mainClassArgs + " " + tmLogRedirects; + } +} diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/utils/KubernetesUtilsTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/utils/KubernetesUtilsTest.java index ca66c600eb..2bb83ac6b5 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/utils/KubernetesUtilsTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/utils/KubernetesUtilsTest.java @@ -18,25 +18,16 @@ package org.apache.flink.kubernetes.utils; -import org.apache.flink.api.common.resources.CPUResource; import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.CoreOptions; import org.apache.flink.configuration.HighAvailabilityOptions; -import org.apache.flink.configuration.MemorySize; -import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; -import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; -import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec; -import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils; import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.TestLogger; import org.junit.Test; -import java.util.HashMap; - import static org.hamcrest.CoreMatchers.containsString; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; @@ -47,169 +38,6 @@ import static org.junit.Assert.fail; */ public class KubernetesUtilsTest extends TestLogger { - private static final String java = "$JAVA_HOME/bin/java"; - private static final String classpath = "-classpath $FLINK_CLASSPATH"; - private static final String jvmOpts = "-Djvm"; - private static final String mainClass = "org.apache.flink.kubernetes.utils.KubernetesUtilsTest"; - private static final String mainClassArgs = "--job-id=1 -Dtest.key=value"; - - // Logging variables - private static final String confDirInPod = "/opt/flink/conf"; - private static final String logDirInPod = "/opt/flink/log"; - private static final String logback = String.format("-Dlogback.configurationFile=file:%s/logback.xml", confDirInPod); - private static final String log4j = String.format("-Dlog4j.configuration=file:%s/log4j.properties -Dlog4j.configurationFile=file:%s/log4j.properties", confDirInPod, confDirInPod); - private static final String jmLogfile = String.format("-Dlog.file=%s/jobmanager.log", logDirInPod); - private static final String jmLogRedirects = String.format("1> %s/jobmanager.out 2> %s/jobmanager.err", logDirInPod, logDirInPod); - private static final String tmLogfile = String.format("-Dlog.file=%s/taskmanager.log", logDirInPod); - private static final String tmLogRedirects = String.format("1> %s/taskmanager.out 2> %s/taskmanager.err", logDirInPod, logDirInPod); - - // Memory variables - private static final int jobManagerMem = 768; - private static final String jmJvmMem = "-Xms168m -Xmx168m"; - - private static final TaskExecutorProcessSpec TASK_EXECUTOR_PROCESS_SPEC = new TaskExecutorProcessSpec( - new CPUResource(1.0), - new MemorySize(0), // frameworkHeapSize - new MemorySize(0), // frameworkOffHeapSize - new MemorySize(111), // taskHeapSize - new MemorySize(0), // taskOffHeapSize - new MemorySize(222), // networkMemSize - new MemorySize(0), // managedMemorySize - new MemorySize(333), // jvmMetaspaceSize - new MemorySize(0)); // jvmOverheadSize - - private static final String tmJvmMem = "-Xmx111 -Xms111 -XX:MaxDirectMemorySize=222 -XX:MaxMetaspaceSize=333"; - private static final String tmMemDynamicProperties = - TaskExecutorProcessUtils.generateDynamicConfigsStr(TASK_EXECUTOR_PROCESS_SPEC).trim(); - - @Test - public void testGetJobManagerStartCommand() { - final Configuration cfg = new Configuration(); - - final String jmJvmOpts = "-DjmJvm"; - - assertEquals( - getJobManagerExpectedCommand("", "", mainClassArgs), - getJobManagerStartCommand(cfg, false, false, mainClassArgs)); - - // logback only - assertEquals( - getJobManagerExpectedCommand("", logback, mainClassArgs), - getJobManagerStartCommand(cfg, true, false, mainClassArgs)); - - // log4j only - assertEquals( - getJobManagerExpectedCommand("", log4j, mainClassArgs), - getJobManagerStartCommand(cfg, false, true, mainClassArgs)); - - // logback + log4j - assertEquals( - getJobManagerExpectedCommand("", logback + " " + log4j, mainClassArgs), - getJobManagerStartCommand(cfg, true, true, mainClassArgs)); - - // logback + log4j, different JVM opts - cfg.setString(CoreOptions.FLINK_JVM_OPTIONS, jvmOpts); - assertEquals( - getJobManagerExpectedCommand(jvmOpts, logback + " " + log4j, mainClassArgs), - getJobManagerStartCommand(cfg, true, true, mainClassArgs)); - - // logback + log4j,different TM JVM opts - cfg.setString(CoreOptions.FLINK_JM_JVM_OPTIONS, jmJvmOpts); - assertEquals( - getJobManagerExpectedCommand(jvmOpts + " " + jmJvmOpts, logback + " " + log4j, mainClassArgs), - getJobManagerStartCommand(cfg, true, true, mainClassArgs)); - - // no args - assertEquals( - getJobManagerExpectedCommand(jvmOpts + " " + jmJvmOpts, logback + " " + log4j, ""), - getJobManagerStartCommand(cfg, true, true, "")); - - // now try some configurations with different container-start-command-template - - cfg.setString(KubernetesConfigOptions.CONTAINER_START_COMMAND_TEMPLATE, - "%java% 1 %classpath% 2 %jvmmem% %jvmopts% %logging% %class% %args% %redirects%"); - assertEquals( - java + " 1 " + classpath + " 2 " + jmJvmMem + - " " + jvmOpts + " " + jmJvmOpts + // jvmOpts - " " + jmLogfile + " " + logback + " " + log4j + - " " + mainClass + " " + mainClassArgs + " " + jmLogRedirects, - getJobManagerStartCommand(cfg, true, true, mainClassArgs)); - - cfg.setString(KubernetesConfigOptions.CONTAINER_START_COMMAND_TEMPLATE, - "%java% %jvmmem% %logging% %jvmopts% %class% %args% %redirects%"); - assertEquals( - java + " " + jmJvmMem + - " " + jmLogfile + " " + logback + " " + log4j + - " " + jvmOpts + " " + jmJvmOpts + // jvmOpts - " " + mainClass + " " + mainClassArgs + " " + jmLogRedirects, - getJobManagerStartCommand(cfg, true, true, mainClassArgs)); - - } - - @Test - public void testGetTaskManagerStartCommand() { - final Configuration cfg = new Configuration(); - - final String tmJvmOpts = "-DtmJvm"; // if set - - assertEquals( - getTaskManagerExpectedCommand("", "", mainClassArgs), - getTaskManagerStartCommand(cfg, false, false, mainClassArgs)); - - // logback only - assertEquals( - getTaskManagerExpectedCommand("", logback, mainClassArgs), - getTaskManagerStartCommand(cfg, true, false, mainClassArgs)); - - // log4j only - assertEquals( - getTaskManagerExpectedCommand("", log4j, mainClassArgs), - getTaskManagerStartCommand(cfg, false, true, mainClassArgs)); - - // logback + log4j - assertEquals( - getTaskManagerExpectedCommand("", logback + " " + log4j, mainClassArgs), - getTaskManagerStartCommand(cfg, true, true, mainClassArgs)); - - // logback + log4j, different JVM opts - cfg.setString(CoreOptions.FLINK_JVM_OPTIONS, jvmOpts); - assertEquals( - getTaskManagerExpectedCommand(jvmOpts, logback + " " + log4j, mainClassArgs), - getTaskManagerStartCommand(cfg, true, true, mainClassArgs)); - - // logback + log4j, different TM JVM opts - cfg.setString(CoreOptions.FLINK_TM_JVM_OPTIONS, tmJvmOpts); - assertEquals( - getTaskManagerExpectedCommand(jvmOpts + " " + tmJvmOpts, logback + " " + log4j, mainClassArgs), - getTaskManagerStartCommand(cfg, true, true, mainClassArgs)); - - // no args - assertEquals( - getTaskManagerExpectedCommand(jvmOpts + " " + tmJvmOpts, logback + " " + log4j, ""), - getTaskManagerStartCommand(cfg, true, true, null)); - - // now try some configurations with different container-start-command-template - - cfg.setString(KubernetesConfigOptions.CONTAINER_START_COMMAND_TEMPLATE, - "%java% 1 %classpath% 2 %jvmmem% %jvmopts% %logging% %class% %args% %redirects%"); - assertEquals( - java + " 1 " + classpath + " 2 " + tmJvmMem + - " " + jvmOpts + " " + tmJvmOpts + // jvmOpts - " " + tmLogfile + " " + logback + " " + log4j + - " " + mainClass + " " + tmMemDynamicProperties + " " + mainClassArgs + " " + tmLogRedirects, - getTaskManagerStartCommand(cfg, true, true, mainClassArgs)); - - cfg.setString(KubernetesConfigOptions.CONTAINER_START_COMMAND_TEMPLATE, - "%java% %jvmmem% %logging% %jvmopts% %class% %redirects%"); - assertEquals( - java + " " + tmJvmMem + - " " + tmLogfile + " " + logback + " " + log4j + - " " + jvmOpts + " " + tmJvmOpts + // jvmOpts - " " + mainClass + " " + tmLogRedirects, - getTaskManagerStartCommand(cfg, true, true, mainClassArgs)); - - } - @Test public void testParsePortRange() { final Configuration cfg = new Configuration(); @@ -257,56 +85,4 @@ public class KubernetesUtilsTest extends TestLogger { Integer.valueOf(fallbackPort)); assertEquals(expectedPort, cfg.get(HighAvailabilityOptions.HA_JOB_MANAGER_PORT_RANGE)); } - - private String getJobManagerExpectedCommand(String jvmAllOpts, String logging, String mainClassArgs) { - return java + " " + classpath + " " + jmJvmMem + - (jvmAllOpts.isEmpty() ? "" : " " + jvmAllOpts) + - (logging.isEmpty() ? "" : " " + jmLogfile + " " + logging) + - " " + mainClass + (mainClassArgs.isEmpty() ? "" : " " + mainClassArgs) + " " + jmLogRedirects; - } - - private String getTaskManagerExpectedCommand(String jvmAllOpts, String logging, String mainClassArgs) { - return java + " " + classpath + " " + tmJvmMem + - (jvmAllOpts.isEmpty() ? "" : " " + jvmAllOpts) + - (logging.isEmpty() ? "" : " " + tmLogfile + " " + logging) + - " " + mainClass + " " + tmMemDynamicProperties + (mainClassArgs.isEmpty() ? "" : " " + mainClassArgs) + " " + tmLogRedirects; - } - - private String getJobManagerStartCommand( - Configuration cfg, - boolean hasLogBack, - boolean hasLog4j, - String mainClassArgs) { - return KubernetesUtils.getJobManagerStartCommand( - cfg, - jobManagerMem, - confDirInPod, - logDirInPod, - hasLogBack, - hasLog4j, - mainClass, - mainClassArgs - ); - } - - private String getTaskManagerStartCommand( - Configuration cfg, - boolean hasLogBack, - boolean hasLog4j, - String mainClassArgs) { - - final ContaineredTaskManagerParameters containeredParams = - new ContaineredTaskManagerParameters(TASK_EXECUTOR_PROCESS_SPEC, 4, new HashMap<>()); - - return KubernetesUtils.getTaskManagerStartCommand( - cfg, - containeredParams, - confDirInPod, - logDirInPod, - hasLogBack, - hasLog4j, - mainClass, - mainClassArgs - ); - } } -- Gitee From 4d8281a3c89514e16a923e20440095e974b9d91d Mon Sep 17 00:00:00 2001 From: felixzheng Date: Sun, 23 Feb 2020 15:32:15 +0800 Subject: [PATCH 057/885] [FLINK-16194][k8s] Introduce the internal and the external Service decorator --- .../decorators/AbstractServiceDecorator.java | 91 +++++++++++++++++++ .../decorators/ExternalServiceDecorator.java | 59 ++++++++++++ .../decorators/InternalServiceDecorator.java | 82 +++++++++++++++++ .../kubernetes/utils/KubernetesUtils.java | 14 +++ .../ExternalServiceDecoratorTest.java | 90 ++++++++++++++++++ .../InternalServiceDecoratorTest.java | 91 +++++++++++++++++++ 6 files changed, 427 insertions(+) create mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/AbstractServiceDecorator.java create mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecorator.java create mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InternalServiceDecorator.java create mode 100644 flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecoratorTest.java create mode 100644 flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InternalServiceDecoratorTest.java diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/AbstractServiceDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/AbstractServiceDecorator.java new file mode 100644 index 0000000000..77a584bce2 --- /dev/null +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/AbstractServiceDecorator.java @@ -0,0 +1,91 @@ +/* + * 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.kubernetes.kubeclient.decorators; + +import org.apache.flink.configuration.RestOptions; +import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters; +import org.apache.flink.kubernetes.utils.Constants; + +import io.fabric8.kubernetes.api.model.HasMetadata; +import io.fabric8.kubernetes.api.model.Service; +import io.fabric8.kubernetes.api.model.ServiceBuilder; +import io.fabric8.kubernetes.api.model.ServicePort; +import io.fabric8.kubernetes.api.model.ServicePortBuilder; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * An abstract class containing some common implementations for the internal/external Services. + */ +public abstract class AbstractServiceDecorator extends AbstractKubernetesStepDecorator { + + protected final KubernetesJobManagerParameters kubernetesJobManagerParameters; + + public AbstractServiceDecorator(KubernetesJobManagerParameters kubernetesJobManagerParameters) { + this.kubernetesJobManagerParameters = checkNotNull(kubernetesJobManagerParameters); + } + + @Override + public List buildAccompanyingKubernetesResources() throws IOException { + final Service service = new ServiceBuilder() + .withApiVersion(Constants.API_VERSION) + .withNewMetadata() + .withName(getServiceName()) + .withLabels(kubernetesJobManagerParameters.getCommonLabels()) + .endMetadata() + .withNewSpec() + .withType(getServiceType()) + .withPorts(getServicePorts()) + .withSelector(kubernetesJobManagerParameters.getLabels()) + .endSpec() + .build(); + + return Collections.singletonList(service); + } + + protected abstract String getServiceType(); + + protected abstract String getServiceName(); + + protected List getServicePorts() { + final List servicePorts = new ArrayList<>(); + + servicePorts.add(getServicePort( + getPortName(RestOptions.PORT.key()), + kubernetesJobManagerParameters.getRestPort())); + + return servicePorts; + } + + protected static ServicePort getServicePort(String name, int port) { + return new ServicePortBuilder() + .withName(name) + .withPort(port) + .build(); + } + + protected static String getPortName(String portName){ + return portName.replace('.', '-'); + } +} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecorator.java new file mode 100644 index 0000000000..21ee943f2f --- /dev/null +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecorator.java @@ -0,0 +1,59 @@ +/* + * 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.kubernetes.kubeclient.decorators; + +import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; +import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters; +import org.apache.flink.kubernetes.utils.KubernetesUtils; + +import io.fabric8.kubernetes.api.model.HasMetadata; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; + +/** + * Creates an external Service to expose the rest port of the Flink JobManager(s). + */ +public class ExternalServiceDecorator extends AbstractServiceDecorator { + + public ExternalServiceDecorator(KubernetesJobManagerParameters kubernetesJobManagerParameters) { + super(kubernetesJobManagerParameters); + } + + @Override + public List buildAccompanyingKubernetesResources() throws IOException { + if (kubernetesJobManagerParameters.getRestServiceExposedType().equals( + KubernetesConfigOptions.ServiceExposedType.ClusterIP.name())) { + return Collections.emptyList(); + } + + return super.buildAccompanyingKubernetesResources(); + } + + @Override + protected String getServiceType() { + return kubernetesJobManagerParameters.getRestServiceExposedType(); + } + + @Override + protected String getServiceName() { + return KubernetesUtils.getRestServiceName(kubernetesJobManagerParameters.getClusterId()); + } +} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InternalServiceDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InternalServiceDecorator.java new file mode 100644 index 0000000000..971ee94c82 --- /dev/null +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InternalServiceDecorator.java @@ -0,0 +1,82 @@ +/* + * 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.kubernetes.kubeclient.decorators; + +import org.apache.flink.configuration.BlobServerOptions; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; +import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters; +import org.apache.flink.kubernetes.utils.KubernetesUtils; + +import io.fabric8.kubernetes.api.model.HasMetadata; +import io.fabric8.kubernetes.api.model.ServicePort; + +import java.io.IOException; +import java.util.List; + +/** + * Creates an internal Service which forwards the requests from the TaskManager(s) to the + * active JobManager. + * Note that only the non-HA scenario relies on this Service for internal communication, since + * in the HA mode, the TaskManager(s) directly connects to the JobManager via IP address. + */ +public class InternalServiceDecorator extends AbstractServiceDecorator { + + public InternalServiceDecorator(KubernetesJobManagerParameters kubernetesJobManagerParameters) { + super(kubernetesJobManagerParameters); + } + + @Override + public List buildAccompanyingKubernetesResources() throws IOException { + final String serviceName = getServiceName(); + + // Set job manager address to namespaced service name + final String namespace = kubernetesJobManagerParameters.getNamespace(); + kubernetesJobManagerParameters.getFlinkConfiguration() + .setString(JobManagerOptions.ADDRESS, serviceName + "." + namespace); + + return super.buildAccompanyingKubernetesResources(); + } + + @Override + protected List getServicePorts() { + final List servicePorts = super.getServicePorts(); + + servicePorts.add(getServicePort( + getPortName(JobManagerOptions.PORT.key()), + kubernetesJobManagerParameters.getRPCPort())); + servicePorts.add(getServicePort( + getPortName(BlobServerOptions.PORT.key()), + kubernetesJobManagerParameters.getBlobServerPort())); + + return servicePorts; + } + + @Override + protected String getServiceType() { + return KubernetesConfigOptions.ServiceExposedType.ClusterIP.name(); + } + + @Override + protected String getServiceName() { + return KubernetesUtils.getInternalServiceName(kubernetesJobManagerParameters.getClusterId()); + } +} + + diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java index d5bd27a940..4d48e4c097 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java @@ -232,6 +232,20 @@ public class KubernetesUtils { return volumeMounts; } + /** + * Generate name of the internal Service. + */ + public static String getInternalServiceName(String clusterId) { + return clusterId; + } + + /** + * Generate name of the external Service. + */ + public static String getRestServiceName(String clusterId) { + return clusterId + Constants.FLINK_REST_SERVICE_SUFFIX; + } + /** * Get task manager labels for the current Flink cluster. They could be used to watch the pods status. * diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecoratorTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecoratorTest.java new file mode 100644 index 0000000000..c368d4132d --- /dev/null +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecoratorTest.java @@ -0,0 +1,90 @@ +/* + * 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.kubernetes.kubeclient.decorators; + +import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; +import org.apache.flink.kubernetes.kubeclient.KubernetesJobManagerTestBase; +import org.apache.flink.kubernetes.utils.Constants; +import org.apache.flink.kubernetes.utils.KubernetesUtils; + +import io.fabric8.kubernetes.api.model.HasMetadata; +import io.fabric8.kubernetes.api.model.Service; +import io.fabric8.kubernetes.api.model.ServicePort; +import io.fabric8.kubernetes.api.model.ServicePortBuilder; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import static junit.framework.TestCase.assertTrue; +import static org.junit.Assert.assertEquals; + +/** + * General tests for the {@link ExternalServiceDecorator}. + */ +public class ExternalServiceDecoratorTest extends KubernetesJobManagerTestBase { + + private ExternalServiceDecorator externalServiceDecorator; + + @Before + public void setup() throws Exception { + super.setup(); + this.externalServiceDecorator = new ExternalServiceDecorator(this.kubernetesJobManagerParameters); + } + + @Test + public void testBuildAccompanyingKubernetesResources() throws IOException { + final List resources = this.externalServiceDecorator.buildAccompanyingKubernetesResources(); + assertEquals(1, resources.size()); + + final Service restService = (Service) resources.get(0); + + assertEquals(Constants.API_VERSION, restService.getApiVersion()); + + assertEquals(KubernetesUtils.getRestServiceName(CLUSTER_ID), restService.getMetadata().getName()); + + final Map expectedLabels = getCommonLabels(); + assertEquals(expectedLabels, restService.getMetadata().getLabels()); + + assertEquals("LoadBalancer", restService.getSpec().getType()); + + List expectedServicePorts = Collections.singletonList( + new ServicePortBuilder() + .withName("rest-port") + .withPort(REST_PORT) + .build()); + assertEquals(expectedServicePorts, restService.getSpec().getPorts()); + + expectedLabels.put(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_JOB_MANAGER); + assertEquals(expectedLabels, restService.getSpec().getSelector()); + } + + @Test + public void testSetServiceExposedType() throws IOException { + this.flinkConfig.set(KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE, "NodePort"); + List resources = this.externalServiceDecorator.buildAccompanyingKubernetesResources(); + assertEquals("NodePort", ((Service) resources.get(0)).getSpec().getType()); + + this.flinkConfig.set(KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE, "ClusterIP"); + assertTrue(this.externalServiceDecorator.buildAccompanyingKubernetesResources().isEmpty()); + } +} diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InternalServiceDecoratorTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InternalServiceDecoratorTest.java new file mode 100644 index 0000000000..857ae4eada --- /dev/null +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InternalServiceDecoratorTest.java @@ -0,0 +1,91 @@ +/* + * 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.kubernetes.kubeclient.decorators; + +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.kubernetes.kubeclient.KubernetesJobManagerTestBase; +import org.apache.flink.kubernetes.utils.Constants; +import org.apache.flink.kubernetes.utils.KubernetesUtils; + +import io.fabric8.kubernetes.api.model.HasMetadata; +import io.fabric8.kubernetes.api.model.Service; +import io.fabric8.kubernetes.api.model.ServicePort; +import io.fabric8.kubernetes.api.model.ServicePortBuilder; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +/** + * General tests for the {@link InternalServiceDecorator}. + */ +public class InternalServiceDecoratorTest extends KubernetesJobManagerTestBase { + + private InternalServiceDecorator internalServiceDecorator; + + @Before + public void setup() throws Exception { + super.setup(); + this.internalServiceDecorator = new InternalServiceDecorator(this.kubernetesJobManagerParameters); + } + + @Test + public void testBuildAccompanyingKubernetesResources() throws IOException { + final List resources = this.internalServiceDecorator.buildAccompanyingKubernetesResources(); + assertEquals(1, resources.size()); + + assertEquals( + KubernetesUtils.getInternalServiceName(CLUSTER_ID) + "." + NAMESPACE, + this.flinkConfig.getString(JobManagerOptions.ADDRESS)); + + final Service internalService = (Service) resources.get(0); + + assertEquals(Constants.API_VERSION, internalService.getApiVersion()); + + assertEquals(KubernetesUtils.getInternalServiceName(CLUSTER_ID), internalService.getMetadata().getName()); + + final Map expectedLabels = getCommonLabels(); + assertEquals(expectedLabels, internalService.getMetadata().getLabels()); + + assertEquals("ClusterIP", internalService.getSpec().getType()); + + List expectedServicePorts = Arrays.asList( + new ServicePortBuilder() + .withName("rest-port") + .withPort(REST_PORT) + .build(), + new ServicePortBuilder() + .withName("jobmanager-rpc-port") + .withPort(RPC_PORT) + .build(), + new ServicePortBuilder() + .withName("blob-server-port") + .withPort(BLOB_SERVER_PORT) + .build()); + assertEquals(expectedServicePorts, internalService.getSpec().getPorts()); + + expectedLabels.put(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_JOB_MANAGER); + assertEquals(expectedLabels, internalService.getSpec().getSelector()); + } +} -- Gitee From d29628ca6d64581f8627f81e81f05fa457a6b68d Mon Sep 17 00:00:00 2001 From: felixzheng Date: Sun, 23 Feb 2020 15:35:13 +0800 Subject: [PATCH 058/885] [FLINK-16194][k8s] Introduce decorator for mounting configuration files such as flink-conf.yaml, log4j.properties, and logback.xml --- .../decorators/FlinkConfMountDecorator.java | 189 +++++++++++++++ .../kubernetes/utils/KubernetesUtils.java | 76 ------ .../FlinkConfMountDecoratorTest.java | 217 ++++++++++++++++++ 3 files changed, 406 insertions(+), 76 deletions(-) create mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/FlinkConfMountDecorator.java create mode 100644 flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/FlinkConfMountDecoratorTest.java diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/FlinkConfMountDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/FlinkConfMountDecorator.java new file mode 100644 index 0000000000..2f5a329e02 --- /dev/null +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/FlinkConfMountDecorator.java @@ -0,0 +1,189 @@ +/* + * 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.kubernetes.kubeclient.decorators; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.client.cli.CliFrontend; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; +import org.apache.flink.kubernetes.kubeclient.FlinkPod; +import org.apache.flink.kubernetes.kubeclient.parameters.AbstractKubernetesParameters; +import org.apache.flink.kubernetes.utils.Constants; + +import org.apache.flink.shaded.guava18.com.google.common.io.Files; + +import io.fabric8.kubernetes.api.model.ConfigMap; +import io.fabric8.kubernetes.api.model.ConfigMapBuilder; +import io.fabric8.kubernetes.api.model.Container; +import io.fabric8.kubernetes.api.model.ContainerBuilder; +import io.fabric8.kubernetes.api.model.HasMetadata; +import io.fabric8.kubernetes.api.model.KeyToPath; +import io.fabric8.kubernetes.api.model.KeyToPathBuilder; +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.PodBuilder; +import io.fabric8.kubernetes.api.model.Volume; +import io.fabric8.kubernetes.api.model.VolumeBuilder; + +import java.io.File; +import java.io.IOException; +import java.io.PrintWriter; +import java.io.StringWriter; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.flink.configuration.GlobalConfiguration.FLINK_CONF_FILENAME; +import static org.apache.flink.kubernetes.utils.Constants.CONFIG_FILE_LOG4J_NAME; +import static org.apache.flink.kubernetes.utils.Constants.CONFIG_FILE_LOGBACK_NAME; +import static org.apache.flink.kubernetes.utils.Constants.CONFIG_MAP_PREFIX; +import static org.apache.flink.kubernetes.utils.Constants.FLINK_CONF_VOLUME; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Mounts the log4j.properties, logback.xml, and flink-conf.yaml configuration on the JobManager or TaskManager pod. + */ +public class FlinkConfMountDecorator extends AbstractKubernetesStepDecorator { + + private final AbstractKubernetesParameters kubernetesComponentConf; + + public FlinkConfMountDecorator(AbstractKubernetesParameters kubernetesComponentConf) { + this.kubernetesComponentConf = checkNotNull(kubernetesComponentConf); + } + + @Override + public FlinkPod decorateFlinkPod(FlinkPod flinkPod) { + final Pod mountedPod = decoratePod(flinkPod.getPod()); + + final Container mountedMainContainer = new ContainerBuilder(flinkPod.getMainContainer()) + .addNewVolumeMount() + .withName(FLINK_CONF_VOLUME) + .withMountPath(kubernetesComponentConf.getFlinkConfDirInPod()) + .endVolumeMount() + .build(); + + return new FlinkPod.Builder(flinkPod) + .withPod(mountedPod) + .withMainContainer(mountedMainContainer) + .build(); + } + + private Pod decoratePod(Pod pod) { + final List keyToPaths = getLocalLogConfFiles().stream() + .map(file -> new KeyToPathBuilder() + .withKey(file.getName()) + .withPath(file.getName()) + .build()) + .collect(Collectors.toList()); + keyToPaths.add(new KeyToPathBuilder() + .withKey(FLINK_CONF_FILENAME) + .withPath(FLINK_CONF_FILENAME) + .build()); + + final Volume flinkConfVolume = new VolumeBuilder() + .withName(FLINK_CONF_VOLUME) + .withNewConfigMap() + .withName(getFlinkConfConfigMapName(kubernetesComponentConf.getClusterId())) + .withItems(keyToPaths) + .endConfigMap() + .build(); + + return new PodBuilder(pod) + .editSpec() + .addNewVolumeLike(flinkConfVolume) + .endVolume() + .endSpec() + .build(); + } + + @Override + public List buildAccompanyingKubernetesResources() throws IOException { + final String clusterId = kubernetesComponentConf.getClusterId(); + + final Map data = new HashMap<>(); + + final List localLogFiles = getLocalLogConfFiles(); + for (File file : localLogFiles) { + data.put(file.getName(), Files.toString(file, StandardCharsets.UTF_8)); + } + + final Map propertiesMap = getClusterSidePropertiesMap(kubernetesComponentConf.getFlinkConfiguration()); + data.put(FLINK_CONF_FILENAME, getFlinkConfData(propertiesMap)); + + final ConfigMap flinkConfConfigMap = new ConfigMapBuilder() + .withApiVersion(Constants.API_VERSION) + .withNewMetadata() + .withName(getFlinkConfConfigMapName(clusterId)) + .withLabels(kubernetesComponentConf.getCommonLabels()) + .endMetadata() + .addToData(data) + .build(); + + return Collections.singletonList(flinkConfConfigMap); + } + + /** + * Get properties map for the cluster-side after removal of some keys. + */ + private Map getClusterSidePropertiesMap(Configuration flinkConfig) { + final Map propertiesMap = flinkConfig.toMap(); + + // remove kubernetes.config.file + propertiesMap.remove(KubernetesConfigOptions.KUBE_CONFIG_FILE.key()); + return propertiesMap; + } + + @VisibleForTesting + String getFlinkConfData(Map propertiesMap) throws IOException { + try (StringWriter sw = new StringWriter(); + PrintWriter out = new PrintWriter(sw)) { + propertiesMap.forEach((k, v) -> { + out.print(k); + out.print(": "); + out.println(v); + }); + + return sw.toString(); + } + } + + private List getLocalLogConfFiles() { + final String confDir = CliFrontend.getConfigurationDirectoryFromEnv(); + final File logbackFile = new File(confDir, CONFIG_FILE_LOGBACK_NAME); + final File log4jFile = new File(confDir, CONFIG_FILE_LOG4J_NAME); + + List localLogConfFiles = new ArrayList<>(); + if (logbackFile.exists()) { + localLogConfFiles.add(logbackFile); + } + if (log4jFile.exists()) { + localLogConfFiles.add(log4jFile); + } + + return localLogConfFiles; + } + + @VisibleForTesting + String getFlinkConfConfigMapName(String clusterId) { + return CONFIG_MAP_PREFIX + clusterId; + } +} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java index 4d48e4c097..37aa241589 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java @@ -28,14 +28,9 @@ import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec; import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils; import org.apache.flink.util.FlinkRuntimeException; -import io.fabric8.kubernetes.api.model.ConfigMapVolumeSourceBuilder; -import io.fabric8.kubernetes.api.model.KeyToPath; import io.fabric8.kubernetes.api.model.Quantity; import io.fabric8.kubernetes.api.model.ResourceRequirements; import io.fabric8.kubernetes.api.model.ResourceRequirementsBuilder; -import io.fabric8.kubernetes.api.model.Volume; -import io.fabric8.kubernetes.api.model.VolumeMount; -import io.fabric8.kubernetes.api.model.VolumeMountBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -47,16 +42,9 @@ import java.io.FileInputStream; import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStreamReader; -import java.util.ArrayList; import java.util.HashMap; -import java.util.List; import java.util.Map; -import static org.apache.flink.configuration.GlobalConfiguration.FLINK_CONF_FILENAME; -import static org.apache.flink.kubernetes.utils.Constants.CONFIG_FILE_LOG4J_NAME; -import static org.apache.flink.kubernetes.utils.Constants.CONFIG_FILE_LOGBACK_NAME; -import static org.apache.flink.kubernetes.utils.Constants.CONFIG_MAP_PREFIX; -import static org.apache.flink.kubernetes.utils.Constants.FLINK_CONF_VOLUME; import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -168,70 +156,6 @@ public class KubernetesUtils { ); } - /** - * Get config map volume for job manager and task manager pod. - * - * @param clusterId Cluster id. - * @param hasLogback Uses logback? - * @param hasLog4j Uses log4j? - * @return Config map volume. - */ - public static Volume getConfigMapVolume(String clusterId, boolean hasLogback, boolean hasLog4j) { - final Volume configMapVolume = new Volume(); - configMapVolume.setName(FLINK_CONF_VOLUME); - - final List items = new ArrayList<>(); - items.add(new KeyToPath(FLINK_CONF_FILENAME, null, FLINK_CONF_FILENAME)); - - if (hasLogback) { - items.add(new KeyToPath(CONFIG_FILE_LOGBACK_NAME, null, CONFIG_FILE_LOGBACK_NAME)); - } - - if (hasLog4j) { - items.add(new KeyToPath(CONFIG_FILE_LOG4J_NAME, null, CONFIG_FILE_LOG4J_NAME)); - } - - configMapVolume.setConfigMap(new ConfigMapVolumeSourceBuilder() - .withName(CONFIG_MAP_PREFIX + clusterId) - .withItems(items) - .build()); - return configMapVolume; - } - - /** - * Get config map volume for job manager and task manager pod. - * - * @param flinkConfDirInPod Flink conf directory that will be mounted in the pod. - * @param hasLogback Uses logback? - * @param hasLog4j Uses log4j? - * @return Volume mount list. - */ - public static List getConfigMapVolumeMount(String flinkConfDirInPod, boolean hasLogback, boolean hasLog4j) { - final List volumeMounts = new ArrayList<>(); - volumeMounts.add(new VolumeMountBuilder() - .withName(FLINK_CONF_VOLUME) - .withMountPath(new File(flinkConfDirInPod, FLINK_CONF_FILENAME).getPath()) - .withSubPath(FLINK_CONF_FILENAME).build()); - - if (hasLogback) { - volumeMounts.add(new VolumeMountBuilder() - .withName(FLINK_CONF_VOLUME) - .withMountPath(new File(flinkConfDirInPod, CONFIG_FILE_LOGBACK_NAME).getPath()) - .withSubPath(CONFIG_FILE_LOGBACK_NAME) - .build()); - } - - if (hasLog4j) { - volumeMounts.add(new VolumeMountBuilder() - .withName(FLINK_CONF_VOLUME) - .withMountPath(new File(flinkConfDirInPod, CONFIG_FILE_LOG4J_NAME).getPath()) - .withSubPath(CONFIG_FILE_LOG4J_NAME) - .build()); - } - - return volumeMounts; - } - /** * Generate name of the internal Service. */ diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/FlinkConfMountDecoratorTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/FlinkConfMountDecoratorTest.java new file mode 100644 index 0000000000..2537ac3955 --- /dev/null +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/FlinkConfMountDecoratorTest.java @@ -0,0 +1,217 @@ +/* + * 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.kubernetes.kubeclient.decorators; + +import org.apache.flink.kubernetes.KubernetesTestUtils; +import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; +import org.apache.flink.kubernetes.kubeclient.FlinkPod; +import org.apache.flink.kubernetes.kubeclient.KubernetesJobManagerTestBase; +import org.apache.flink.kubernetes.utils.Constants; + +import io.fabric8.kubernetes.api.model.ConfigMap; +import io.fabric8.kubernetes.api.model.Container; +import io.fabric8.kubernetes.api.model.HasMetadata; +import io.fabric8.kubernetes.api.model.KeyToPath; +import io.fabric8.kubernetes.api.model.KeyToPathBuilder; +import io.fabric8.kubernetes.api.model.Volume; +import io.fabric8.kubernetes.api.model.VolumeBuilder; +import io.fabric8.kubernetes.api.model.VolumeMount; +import io.fabric8.kubernetes.api.model.VolumeMountBuilder; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.configuration.GlobalConfiguration.FLINK_CONF_FILENAME; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertTrue; + +/** + * General tests for the {@link FlinkConfMountDecorator}. + */ +public class FlinkConfMountDecoratorTest extends KubernetesJobManagerTestBase { + + private static final String FLINK_CONF_DIR_IN_POD = "/opt/flink/flink-conf-"; + + private FlinkConfMountDecorator flinkConfMountDecorator; + + @Before + public void setup() throws Exception { + super.setup(); + + this.flinkConfig.set(KubernetesConfigOptions.FLINK_CONF_DIR, FLINK_CONF_DIR_IN_POD); + + this.flinkConfMountDecorator = new FlinkConfMountDecorator(kubernetesJobManagerParameters); + } + + @Test + public void testWhetherPodOrContainerIsDecorated() { + final FlinkPod resultFlinkPod = flinkConfMountDecorator.decorateFlinkPod(baseFlinkPod); + assertNotEquals(baseFlinkPod.getPod(), resultFlinkPod.getPod()); + assertNotEquals(baseFlinkPod.getMainContainer(), resultFlinkPod.getMainContainer()); + } + + @Test + public void testConfigMap() throws IOException { + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "log4j.properties"); + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "logback.xml"); + + final List additionalResources = flinkConfMountDecorator.buildAccompanyingKubernetesResources(); + assertEquals(1, additionalResources.size()); + + final ConfigMap resultConfigMap = (ConfigMap) additionalResources.get(0); + + assertEquals(Constants.API_VERSION, resultConfigMap.getApiVersion()); + + assertEquals(flinkConfMountDecorator.getFlinkConfConfigMapName(CLUSTER_ID), + resultConfigMap.getMetadata().getName()); + assertEquals(getCommonLabels(), resultConfigMap.getMetadata().getLabels()); + + Map resultDatas = resultConfigMap.getData(); + assertEquals("some data", resultDatas.get("logback.xml")); + assertEquals("some data", resultDatas.get("log4j.properties")); + assertTrue(resultDatas.get(FLINK_CONF_FILENAME).contains(KubernetesConfigOptions.FLINK_CONF_DIR.key() + + ": " + FLINK_CONF_DIR_IN_POD)); + } + + @Test + public void testDecoratedFlinkPodWithoutLog4jAndLogback() { + final FlinkPod resultFlinkPod = flinkConfMountDecorator.decorateFlinkPod(baseFlinkPod); + + final List expectedKeyToPaths = Collections.singletonList( + new KeyToPathBuilder() + .withKey(FLINK_CONF_FILENAME) + .withPath(FLINK_CONF_FILENAME) + .build()); + final List expectedVolumes = Collections.singletonList( + new VolumeBuilder() + .withName(Constants.FLINK_CONF_VOLUME) + .withNewConfigMap() + .withName(flinkConfMountDecorator.getFlinkConfConfigMapName(CLUSTER_ID)) + .withItems(expectedKeyToPaths) + .endConfigMap() + .build()); + assertEquals(expectedVolumes, resultFlinkPod.getPod().getSpec().getVolumes()); + + final List expectedVolumeMounts = Collections.singletonList( + new VolumeMountBuilder() + .withName(Constants.FLINK_CONF_VOLUME) + .withMountPath(FLINK_CONF_DIR_IN_POD) + .build()); + assertEquals(expectedVolumeMounts, resultFlinkPod.getMainContainer().getVolumeMounts()); + } + + @Test + public void testDecoratedFlinkPodWithLog4j() throws IOException { + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "log4j.properties"); + + final FlinkPod resultFlinkPod = flinkConfMountDecorator.decorateFlinkPod(baseFlinkPod); + + final List expectedKeyToPaths = Arrays.asList( + new KeyToPathBuilder() + .withKey("log4j.properties") + .withPath("log4j.properties") + .build(), + new KeyToPathBuilder() + .withKey(FLINK_CONF_FILENAME) + .withPath(FLINK_CONF_FILENAME) + .build()); + final List expectedVolumes = Collections.singletonList( + new VolumeBuilder() + .withName(Constants.FLINK_CONF_VOLUME) + .withNewConfigMap() + .withName(flinkConfMountDecorator.getFlinkConfConfigMapName(CLUSTER_ID)) + .withItems(expectedKeyToPaths) + .endConfigMap() + .build()); + assertEquals(expectedVolumes, resultFlinkPod.getPod().getSpec().getVolumes()); + } + + @Test + public void testDecoratedFlinkPodWithLogback() throws IOException { + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "logback.xml"); + + final FlinkPod resultFlinkPod = flinkConfMountDecorator.decorateFlinkPod(baseFlinkPod); + + final List expectedKeyToPaths = Arrays.asList( + new KeyToPathBuilder() + .withKey("logback.xml") + .withPath("logback.xml") + .build(), + new KeyToPathBuilder() + .withKey(FLINK_CONF_FILENAME) + .withPath(FLINK_CONF_FILENAME) + .build()); + final List expectedVolumes = Collections.singletonList( + new VolumeBuilder() + .withName(Constants.FLINK_CONF_VOLUME) + .withNewConfigMap() + .withName(flinkConfMountDecorator.getFlinkConfConfigMapName(CLUSTER_ID)) + .withItems(expectedKeyToPaths) + .endConfigMap() + .build()); + assertEquals(expectedVolumes, resultFlinkPod.getPod().getSpec().getVolumes()); + } + + @Test + public void testDecoratedFlinkPodWithLog4jAndLogback() throws IOException { + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "log4j.properties"); + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "logback.xml"); + + final FlinkPod resultFlinkPod = flinkConfMountDecorator.decorateFlinkPod(baseFlinkPod); + + final List expectedKeyToPaths = Arrays.asList( + new KeyToPathBuilder() + .withKey("logback.xml") + .withPath("logback.xml") + .build(), + new KeyToPathBuilder() + .withKey("log4j.properties") + .withPath("log4j.properties") + .build(), + new KeyToPathBuilder() + .withKey(FLINK_CONF_FILENAME) + .withPath(FLINK_CONF_FILENAME) + .build()); + final List expectedVolumes = Collections.singletonList( + new VolumeBuilder() + .withName(Constants.FLINK_CONF_VOLUME) + .withNewConfigMap() + .withName(flinkConfMountDecorator.getFlinkConfConfigMapName(CLUSTER_ID)) + .withItems(expectedKeyToPaths) + .endConfigMap() + .build()); + assertEquals(expectedVolumes, resultFlinkPod.getPod().getSpec().getVolumes()); + } + + @Test + public void testDecoratedFlinkContainer() { + final Container resultMainContainer = flinkConfMountDecorator.decorateFlinkPod(baseFlinkPod).getMainContainer(); + assertEquals(1, resultMainContainer.getVolumeMounts().size()); + + final VolumeMount volumeMount = resultMainContainer.getVolumeMounts().get(0); + assertEquals(Constants.FLINK_CONF_VOLUME, volumeMount.getName()); + assertEquals(FLINK_CONF_DIR_IN_POD, volumeMount.getMountPath()); + } +} -- Gitee From 10f53290dd07a64013623d7269c9d599dd7c2482 Mon Sep 17 00:00:00 2001 From: felixzheng Date: Sun, 23 Feb 2020 15:49:11 +0800 Subject: [PATCH 059/885] [FLINK-16194][k8s] Introduce Factories that chain the decorators together to construct all the client/cluster-side Kubernetes resources --- docs/ops/deployment/native_kubernetes.md | 8 +- docs/ops/deployment/native_kubernetes.zh.md | 8 +- .../test-scripts/test_kubernetes_session.sh | 2 +- flink-kubernetes/pom.xml | 1 + .../KubernetesJobManagerSpecification.java | 47 ++++ .../factory/KubernetesJobManagerFactory.java | 106 +++++++++ .../factory/KubernetesTaskManagerFactory.java | 57 +++++ .../kubernetes/utils/KubernetesUtils.java | 13 ++ .../src/main/resources/META-INF/NOTICE | 1 + .../KubernetesResourceManagerTest.java | 4 +- .../KubernetesJobManagerFactoryTest.java | 203 ++++++++++++++++++ .../KubernetesTaskManagerFactoryTest.java | 75 +++++++ 12 files changed, 514 insertions(+), 11 deletions(-) create mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/KubernetesJobManagerSpecification.java create mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactory.java create mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesTaskManagerFactory.java create mode 100644 flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactoryTest.java create mode 100644 flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesTaskManagerFactoryTest.java diff --git a/docs/ops/deployment/native_kubernetes.md b/docs/ops/deployment/native_kubernetes.md index 908b8c62c1..d543609222 100644 --- a/docs/ops/deployment/native_kubernetes.md +++ b/docs/ops/deployment/native_kubernetes.md @@ -126,12 +126,12 @@ $ echo 'stop' | ./bin/kubernetes-session.sh -Dkubernetes.cluster-id= #### Manual Resource Cleanup -Flink uses [Kubernetes ownerReference's](https://kubernetes.io/docs/concepts/workloads/controllers/garbage-collection/) to cleanup all cluster components. -All the Flink created resources, including `ConfigMap`, `Service`, `Deployment`, `Pod`, have been set the ownerReference to `service/`. -When the service is deleted, all other resource will be deleted automatically. +Flink uses [Kubernetes OwnerReference's](https://kubernetes.io/docs/concepts/workloads/controllers/garbage-collection/) to cleanup all cluster components. +All the Flink created resources, including `ConfigMap`, `Service`, `Pod`, have been set the OwnerReference to `deployment/`. +When the deployment is deleted, all other resources will be deleted automatically. {% highlight bash %} -$ kubectl delete service/ +$ kubectl delete deployment/ {% endhighlight %} ## Log Files diff --git a/docs/ops/deployment/native_kubernetes.zh.md b/docs/ops/deployment/native_kubernetes.zh.md index 10d47e02a7..1337dc0c86 100644 --- a/docs/ops/deployment/native_kubernetes.zh.md +++ b/docs/ops/deployment/native_kubernetes.zh.md @@ -126,12 +126,12 @@ $ echo 'stop' | ./bin/kubernetes-session.sh -Dkubernetes.cluster-id= #### Manual Resource Cleanup -Flink uses [Kubernetes ownerReference's](https://kubernetes.io/docs/concepts/workloads/controllers/garbage-collection/) to cleanup all cluster components. -All the Flink created resources, including `ConfigMap`, `Service`, `Deployment`, `Pod`, have been set the ownerReference to `service/`. -When the service is deleted, all other resource will be deleted automatically. +Flink uses [Kubernetes OwnerReference's](https://kubernetes.io/docs/concepts/workloads/controllers/garbage-collection/) to cleanup all cluster components. +All the Flink created resources, including `ConfigMap`, `Service`, `Pod`, have been set the OwnerReference to `deployment/`. +When the deployment is deleted, all other resources will be deleted automatically. {% highlight bash %} -$ kubectl delete service/ +$ kubectl delete deployment/ {% endhighlight %} ## Log Files diff --git a/flink-end-to-end-tests/test-scripts/test_kubernetes_session.sh b/flink-end-to-end-tests/test-scripts/test_kubernetes_session.sh index ecdf5b88c5..007cac3a0f 100755 --- a/flink-end-to-end-tests/test-scripts/test_kubernetes_session.sh +++ b/flink-end-to-end-tests/test-scripts/test_kubernetes_session.sh @@ -27,7 +27,7 @@ OUTPUT_PATH="/tmp/wc_out" ARGS="--output ${OUTPUT_PATH}" function cleanup { - kubectl delete service ${CLUSTER_ID} + kubectl delete deployment ${CLUSTER_ID} kubectl delete clusterrolebinding ${CLUSTER_ROLE_BINDING} stop_kubernetes } diff --git a/flink-kubernetes/pom.xml b/flink-kubernetes/pom.xml index 6e1e3f53af..6c1898068e 100644 --- a/flink-kubernetes/pom.xml +++ b/flink-kubernetes/pom.xml @@ -121,6 +121,7 @@ under the License. io.fabric8:kubernetes-client io.fabric8:kubernetes-model io.fabric8:kubernetes-model-common + io.fabric8:zjsonpatch com.fasterxml.jackson.core:jackson-core diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/KubernetesJobManagerSpecification.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/KubernetesJobManagerSpecification.java new file mode 100644 index 0000000000..4b573ad4b2 --- /dev/null +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/KubernetesJobManagerSpecification.java @@ -0,0 +1,47 @@ +/* + * 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.kubernetes.kubeclient; + +import io.fabric8.kubernetes.api.model.HasMetadata; +import io.fabric8.kubernetes.api.model.apps.Deployment; + +import java.util.List; + +/** + * Composition of the created Kubernetes components that represents a Flink application. + */ +public class KubernetesJobManagerSpecification { + + private Deployment deployment; + + private List accompanyingResources; + + public KubernetesJobManagerSpecification(Deployment deployment, List accompanyingResources) { + this.deployment = deployment; + this.accompanyingResources = accompanyingResources; + } + + public Deployment getDeployment() { + return deployment; + } + + public List getAccompanyingResources() { + return accompanyingResources; + } +} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactory.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactory.java new file mode 100644 index 0000000000..f1961915f9 --- /dev/null +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactory.java @@ -0,0 +1,106 @@ +/* + * 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.kubernetes.kubeclient.factory; + +import org.apache.flink.kubernetes.kubeclient.FlinkPod; +import org.apache.flink.kubernetes.kubeclient.KubernetesJobManagerSpecification; +import org.apache.flink.kubernetes.kubeclient.decorators.ExternalServiceDecorator; +import org.apache.flink.kubernetes.kubeclient.decorators.FlinkConfMountDecorator; +import org.apache.flink.kubernetes.kubeclient.decorators.InitJobManagerDecorator; +import org.apache.flink.kubernetes.kubeclient.decorators.InternalServiceDecorator; +import org.apache.flink.kubernetes.kubeclient.decorators.JavaCmdJobManagerDecorator; +import org.apache.flink.kubernetes.kubeclient.decorators.KubernetesStepDecorator; +import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters; +import org.apache.flink.kubernetes.utils.Constants; +import org.apache.flink.kubernetes.utils.KubernetesUtils; + +import io.fabric8.kubernetes.api.model.Container; +import io.fabric8.kubernetes.api.model.HasMetadata; +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.PodBuilder; +import io.fabric8.kubernetes.api.model.apps.Deployment; +import io.fabric8.kubernetes.api.model.apps.DeploymentBuilder; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * Utility class for constructing all the Kubernetes components on the client-side. This can + * include the Deployment, the ConfigMap(s), and the Service(s). + */ +public class KubernetesJobManagerFactory { + + public static KubernetesJobManagerSpecification createJobManagerComponent( + KubernetesJobManagerParameters kubernetesJobManagerParameters) throws IOException { + FlinkPod flinkPod = new FlinkPod.Builder().build(); + List accompanyingResources = new ArrayList<>(); + + final KubernetesStepDecorator[] stepDecorators = new KubernetesStepDecorator[] { + new InitJobManagerDecorator(kubernetesJobManagerParameters), + new JavaCmdJobManagerDecorator(kubernetesJobManagerParameters), + new InternalServiceDecorator(kubernetesJobManagerParameters), + new ExternalServiceDecorator(kubernetesJobManagerParameters), + new FlinkConfMountDecorator(kubernetesJobManagerParameters)}; + + for (KubernetesStepDecorator stepDecorator: stepDecorators) { + flinkPod = stepDecorator.decorateFlinkPod(flinkPod); + accompanyingResources.addAll(stepDecorator.buildAccompanyingKubernetesResources()); + } + + final Deployment deployment = createJobManagerDeployment(flinkPod, kubernetesJobManagerParameters); + + return new KubernetesJobManagerSpecification(deployment, accompanyingResources); + } + + private static Deployment createJobManagerDeployment( + FlinkPod flinkPod, + KubernetesJobManagerParameters kubernetesJobManagerParameters) { + final Container resolvedMainContainer = flinkPod.getMainContainer(); + + final Pod resolvedPod = new PodBuilder(flinkPod.getPod()) + .editOrNewSpec() + .addToContainers(resolvedMainContainer) + .endSpec() + .build(); + + final Map labels = resolvedPod.getMetadata().getLabels(); + + return new DeploymentBuilder() + .withApiVersion(Constants.APPS_API_VERSION) + .editOrNewMetadata() + .withName(KubernetesUtils.getDeploymentName(kubernetesJobManagerParameters.getClusterId())) + .withLabels(kubernetesJobManagerParameters.getLabels()) + .endMetadata() + .editOrNewSpec() + .withReplicas(1) + .editOrNewTemplate() + .editOrNewMetadata() + .withLabels(labels) + .endMetadata() + .withSpec(resolvedPod.getSpec()) + .endTemplate() + .editOrNewSelector() + .addToMatchLabels(labels) + .endSelector() + .endSpec() + .build(); + } +} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesTaskManagerFactory.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesTaskManagerFactory.java new file mode 100644 index 0000000000..be91bbe4d3 --- /dev/null +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesTaskManagerFactory.java @@ -0,0 +1,57 @@ +/* + * 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.kubernetes.kubeclient.factory; + +import org.apache.flink.kubernetes.kubeclient.FlinkPod; +import org.apache.flink.kubernetes.kubeclient.decorators.FlinkConfMountDecorator; +import org.apache.flink.kubernetes.kubeclient.decorators.InitTaskManagerDecorator; +import org.apache.flink.kubernetes.kubeclient.decorators.JavaCmdTaskManagerDecorator; +import org.apache.flink.kubernetes.kubeclient.decorators.KubernetesStepDecorator; +import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesTaskManagerParameters; +import org.apache.flink.kubernetes.kubeclient.resources.KubernetesPod; + +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.PodBuilder; + +/** + * Utility class for constructing the TaskManager Pod on the JobManager. + */ +public class KubernetesTaskManagerFactory { + + public static KubernetesPod buildTaskManagerComponent(KubernetesTaskManagerParameters kubernetesTaskManagerParameters) { + FlinkPod flinkPod = new FlinkPod.Builder().build(); + + final KubernetesStepDecorator[] stepDecorators = new KubernetesStepDecorator[] { + new InitTaskManagerDecorator(kubernetesTaskManagerParameters), + new JavaCmdTaskManagerDecorator(kubernetesTaskManagerParameters), + new FlinkConfMountDecorator(kubernetesTaskManagerParameters)}; + + for (KubernetesStepDecorator stepDecorator: stepDecorators) { + flinkPod = stepDecorator.decorateFlinkPod(flinkPod); + } + + final Pod resolvedPod = new PodBuilder(flinkPod.getPod()) + .editOrNewSpec() + .addToContainers(flinkPod.getMainContainer()) + .endSpec() + .build(); + + return new KubernetesPod(resolvedPod); + } +} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java index 37aa241589..b7ca83c220 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java @@ -28,9 +28,13 @@ import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec; import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils; import org.apache.flink.util.FlinkRuntimeException; +import io.fabric8.kubernetes.api.model.HasMetadata; +import io.fabric8.kubernetes.api.model.OwnerReference; +import io.fabric8.kubernetes.api.model.OwnerReferenceBuilder; import io.fabric8.kubernetes.api.model.Quantity; import io.fabric8.kubernetes.api.model.ResourceRequirements; import io.fabric8.kubernetes.api.model.ResourceRequirementsBuilder; +import io.fabric8.kubernetes.api.model.apps.Deployment; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -42,7 +46,9 @@ import java.io.FileInputStream; import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStreamReader; +import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -170,6 +176,13 @@ public class KubernetesUtils { return clusterId + Constants.FLINK_REST_SERVICE_SUFFIX; } + /** + * Generate name of the Deployment. + */ + public static String getDeploymentName(String clusterId) { + return clusterId; + } + /** * Get task manager labels for the current Flink cluster. They could be used to watch the pods status. * diff --git a/flink-kubernetes/src/main/resources/META-INF/NOTICE b/flink-kubernetes/src/main/resources/META-INF/NOTICE index 980bc80c01..9af3fc4c3d 100644 --- a/flink-kubernetes/src/main/resources/META-INF/NOTICE +++ b/flink-kubernetes/src/main/resources/META-INF/NOTICE @@ -17,6 +17,7 @@ This project bundles the following dependencies under the Apache Software Licens - io.fabric8:kubernetes-client:4.5.2 - io.fabric8:kubernetes-model:4.5.2 - io.fabric8:kubernetes-model-common:4.5.2 +- io.fabric8:zjsonpatch:0.3.0 - org.yaml:snakeyaml:1.23 This project bundles the following dependencies under the BSD License. diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesResourceManagerTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesResourceManagerTest.java index 429ed2a763..834873cde9 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesResourceManagerTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesResourceManagerTest.java @@ -28,6 +28,7 @@ import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient; import org.apache.flink.kubernetes.kubeclient.TaskManagerPodParameter; import org.apache.flink.kubernetes.kubeclient.resources.KubernetesPod; import org.apache.flink.kubernetes.utils.Constants; +import org.apache.flink.kubernetes.utils.KubernetesUtils; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; @@ -110,8 +111,7 @@ public class KubernetesResourceManagerTest extends KubernetesTestBase { final Deployment mockDeployment = new DeploymentBuilder() .editOrNewMetadata() - .withName(CLUSTER_ID) - .withUid(CLUSTER_ID) + .withName(KubernetesUtils.getDeploymentName(CLUSTER_ID)) .endMetadata() .build(); kubeClient.apps().deployments().inNamespace(NAMESPACE).create(mockDeployment); diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactoryTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactoryTest.java new file mode 100644 index 0000000000..054b645b5d --- /dev/null +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactoryTest.java @@ -0,0 +1,203 @@ +/* + * 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.kubernetes.kubeclient.factory; + +import org.apache.flink.kubernetes.KubernetesTestUtils; +import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; +import org.apache.flink.kubernetes.configuration.KubernetesConfigOptionsInternal; +import org.apache.flink.kubernetes.entrypoint.KubernetesSessionClusterEntrypoint; +import org.apache.flink.kubernetes.kubeclient.KubernetesJobManagerSpecification; +import org.apache.flink.kubernetes.kubeclient.KubernetesJobManagerTestBase; +import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters; +import org.apache.flink.kubernetes.utils.Constants; +import org.apache.flink.kubernetes.utils.KubernetesUtils; + +import io.fabric8.kubernetes.api.model.ConfigMap; +import io.fabric8.kubernetes.api.model.Container; +import io.fabric8.kubernetes.api.model.HasMetadata; +import io.fabric8.kubernetes.api.model.PodSpec; +import io.fabric8.kubernetes.api.model.Quantity; +import io.fabric8.kubernetes.api.model.Service; +import io.fabric8.kubernetes.api.model.apps.Deployment; +import io.fabric8.kubernetes.api.model.apps.DeploymentSpec; +import org.junit.Before; +import org.junit.Test; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.flink.configuration.GlobalConfiguration.FLINK_CONF_FILENAME; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +/** + * General tests for the {@link KubernetesJobManagerFactory}. + */ +public class KubernetesJobManagerFactoryTest extends KubernetesJobManagerTestBase { + + private static final String SERVICE_ACCOUNT_NAME = "service-test"; + private static final String ENTRY_POINT_CLASS = KubernetesSessionClusterEntrypoint.class.getCanonicalName(); + + private KubernetesJobManagerSpecification kubernetesJobManagerSpecification; + + @Before + public void setup() throws Exception { + super.setup(); + + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "logback.xml"); + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "log4j.properties"); + + flinkConfig.set(KubernetesConfigOptionsInternal.ENTRY_POINT_CLASS, ENTRY_POINT_CLASS); + flinkConfig.set(KubernetesConfigOptions.JOB_MANAGER_SERVICE_ACCOUNT, SERVICE_ACCOUNT_NAME); + + this.kubernetesJobManagerSpecification = + KubernetesJobManagerFactory.createJobManagerComponent(kubernetesJobManagerParameters); + } + + @Test + public void testDeploymentMetadata() { + final Deployment resultDeployment = this.kubernetesJobManagerSpecification.getDeployment(); + assertEquals(Constants.APPS_API_VERSION, resultDeployment.getApiVersion()); + assertEquals(KubernetesUtils.getDeploymentName(CLUSTER_ID), resultDeployment.getMetadata().getName()); + final Map expectedLabels = getCommonLabels(); + expectedLabels.put(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_JOB_MANAGER); + assertEquals(expectedLabels, resultDeployment.getMetadata().getLabels()); + } + + @Test + public void testDeploymentSpec() { + final DeploymentSpec resultDeploymentSpec = this.kubernetesJobManagerSpecification.getDeployment().getSpec(); + assertEquals(1, resultDeploymentSpec.getReplicas().intValue()); + + final Map expectedLabels = new HashMap<>(getCommonLabels()); + expectedLabels.put(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_JOB_MANAGER); + + assertEquals(expectedLabels, resultDeploymentSpec.getTemplate().getMetadata().getLabels()); + assertEquals(expectedLabels, resultDeploymentSpec.getSelector().getMatchLabels()); + + assertNotNull(resultDeploymentSpec.getTemplate().getSpec()); + } + + @Test + public void testPodSpec() { + final PodSpec resultPodSpec = + this.kubernetesJobManagerSpecification.getDeployment().getSpec().getTemplate().getSpec(); + + assertEquals(1, resultPodSpec.getContainers().size()); + assertEquals(SERVICE_ACCOUNT_NAME, resultPodSpec.getServiceAccountName()); + assertEquals(1, resultPodSpec.getVolumes().size()); + + final Container resultedMainContainer = resultPodSpec.getContainers().get(0); + assertEquals(KubernetesJobManagerParameters.JOB_MANAGER_MAIN_CONTAINER_NAME, resultedMainContainer.getName()); + assertEquals(CONTAINER_IMAGE, resultedMainContainer.getImage()); + assertEquals(CONTAINER_IMAGE_PULL_POLICY, resultedMainContainer.getImagePullPolicy()); + + assertEquals(3, resultedMainContainer.getEnv().size()); + assertTrue(resultedMainContainer.getEnv() + .stream() + .anyMatch(envVar -> envVar.getName().equals("key1"))); + + assertEquals(3, resultedMainContainer.getPorts().size()); + + final Map requests = resultedMainContainer.getResources().getRequests(); + assertEquals(Double.toString(JOB_MANAGER_CPU), requests.get("cpu").getAmount()); + assertEquals(JOB_MANAGER_MEMORY + "Mi", requests.get("memory").getAmount()); + + assertEquals(1, resultedMainContainer.getCommand().size()); + assertEquals(3, resultedMainContainer.getArgs().size()); + + assertEquals(1, resultedMainContainer.getVolumeMounts().size()); + } + + @Test + public void testAdditionalResourcesSize() { + final List resultAdditionalResources = this.kubernetesJobManagerSpecification.getAccompanyingResources(); + assertEquals(3, resultAdditionalResources.size()); + + final List resultServices = resultAdditionalResources + .stream() + .filter(x -> x instanceof Service) + .collect(Collectors.toList()); + assertEquals(2, resultServices.size()); + + final List resultConfigMaps = resultAdditionalResources + .stream() + .filter(x -> x instanceof ConfigMap) + .collect(Collectors.toList()); + assertEquals(1, resultConfigMaps.size()); + } + + @Test + public void testServices() { + final List resultServices = this.kubernetesJobManagerSpecification.getAccompanyingResources() + .stream() + .filter(x -> x instanceof Service) + .map(x -> (Service) x) + .collect(Collectors.toList()); + + assertEquals(2, resultServices.size()); + + final List internalServiceCandidates = resultServices + .stream() + .filter(x -> x.getMetadata().getName().equals(KubernetesUtils.getInternalServiceName(CLUSTER_ID))) + .collect(Collectors.toList()); + assertEquals(1, internalServiceCandidates.size()); + + final List restServiceCandidates = resultServices + .stream() + .filter(x -> x.getMetadata().getName().equals(KubernetesUtils.getRestServiceName(CLUSTER_ID))) + .collect(Collectors.toList()); + assertEquals(1, restServiceCandidates.size()); + + final Service resultInternalService = internalServiceCandidates.get(0); + assertEquals(2, resultInternalService.getMetadata().getLabels().size()); + + assertEquals(resultInternalService.getSpec().getType(), "ClusterIP"); + assertEquals(3, resultInternalService.getSpec().getPorts().size()); + assertEquals(3, resultInternalService.getSpec().getSelector().size()); + + final Service resultRestService = restServiceCandidates.get(0); + assertEquals(2, resultRestService.getMetadata().getLabels().size()); + + assertEquals(resultRestService.getSpec().getType(), "LoadBalancer"); + assertEquals(1, resultRestService.getSpec().getPorts().size()); + assertEquals(3, resultRestService.getSpec().getSelector().size()); + } + + @Test + public void testFlinkConfConfigMap() { + final ConfigMap resultConfigMap = (ConfigMap) this.kubernetesJobManagerSpecification.getAccompanyingResources() + .stream() + .filter(x -> x instanceof ConfigMap) + .collect(Collectors.toList()) + .get(0); + + assertEquals(2, resultConfigMap.getMetadata().getLabels().size()); + + final Map resultDatas = resultConfigMap.getData(); + assertEquals(3, resultDatas.size()); + assertEquals("some data", resultDatas.get("log4j.properties")); + assertEquals("some data", resultDatas.get("logback.xml")); + assertTrue(resultDatas.get(FLINK_CONF_FILENAME) + .contains(KubernetesConfigOptionsInternal.ENTRY_POINT_CLASS.key() + ": " + ENTRY_POINT_CLASS)); + } +} diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesTaskManagerFactoryTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesTaskManagerFactoryTest.java new file mode 100644 index 0000000000..8d37b621dd --- /dev/null +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesTaskManagerFactoryTest.java @@ -0,0 +1,75 @@ +/* + * 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.kubernetes.kubeclient.factory; + +import org.apache.flink.kubernetes.KubernetesTestUtils; +import org.apache.flink.kubernetes.kubeclient.KubernetesTaskManagerTestBase; +import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesTaskManagerParameters; + +import io.fabric8.kubernetes.api.model.Container; +import io.fabric8.kubernetes.api.model.Pod; +import org.junit.Before; +import org.junit.Test; + +import java.util.List; + +import static org.junit.Assert.assertEquals; + +/** + * General tests for the {@link KubernetesTaskManagerFactory}. + */ +public class KubernetesTaskManagerFactoryTest extends KubernetesTaskManagerTestBase { + + private Pod resultPod; + + @Before + public void setup() throws Exception { + super.setup(); + + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "logback.xml"); + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "log4j.properties"); + + this.resultPod = + KubernetesTaskManagerFactory.buildTaskManagerComponent(kubernetesTaskManagerParameters).getInternalResource(); + } + + @Test + public void testPod() { + assertEquals(POD_NAME, this.resultPod.getMetadata().getName()); + assertEquals(3, this.resultPod.getMetadata().getLabels().size()); + assertEquals(1, this.resultPod.getSpec().getVolumes().size()); + } + + @Test + public void testContainer() { + final List resultContainers = this.resultPod.getSpec().getContainers(); + assertEquals(1, resultContainers.size()); + + final Container resultMainContainer = resultContainers.get(0); + assertEquals( + KubernetesTaskManagerParameters.TASK_MANAGER_MAIN_CONTAINER_NAME, + resultMainContainer.getName()); + assertEquals(CONTAINER_IMAGE, resultMainContainer.getImage()); + assertEquals(CONTAINER_IMAGE_PULL_POLICY, resultMainContainer.getImagePullPolicy()); + assertEquals(1, resultMainContainer.getPorts().size()); + assertEquals(1, resultMainContainer.getCommand().size()); + assertEquals(3, resultMainContainer.getArgs().size()); + assertEquals(1, resultMainContainer.getVolumeMounts().size()); + } +} -- Gitee From f6ad9685bcb8365277ffc2fa3fe1d54b2d445a28 Mon Sep 17 00:00:00 2001 From: felixzheng Date: Sun, 23 Feb 2020 16:53:48 +0800 Subject: [PATCH 060/885] [FLINK-16194][k8s] Rework FlinkKubeClient to use the new decorator pattern This closes #11233 . --- .../KubernetesClusterDescriptor.java | 16 +- .../kubernetes/KubernetesResourceManager.java | 54 +---- .../KubernetesConfigOptionsInternal.java | 6 - .../kubeclient/Fabric8FlinkKubeClient.java | 83 ++++++- .../kubeclient/FlinkKubeClient.java | 10 +- .../kubeclient/TaskManagerPodParameter.java | 71 ------ .../factory/KubernetesTaskManagerFactory.java | 2 +- .../kubeclient/resources/ActionWatcher.java | 63 ----- .../kubernetes/utils/KubernetesUtils.java | 50 ---- .../KubernetesClusterDescriptorTest.java | 55 ++++- .../KubernetesResourceManagerTest.java | 21 +- .../Fabric8FlinkKubeClientTest.java | 215 +++++++++++++++++- .../KubernetesTaskManagerFactoryTest.java | 2 +- 13 files changed, 376 insertions(+), 272 deletions(-) delete mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/TaskManagerPodParameter.java delete mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/ActionWatcher.java diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesClusterDescriptor.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesClusterDescriptor.java index eb62393912..cae0c869e4 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesClusterDescriptor.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesClusterDescriptor.java @@ -36,6 +36,9 @@ import org.apache.flink.kubernetes.configuration.KubernetesConfigOptionsInternal import org.apache.flink.kubernetes.entrypoint.KubernetesSessionClusterEntrypoint; import org.apache.flink.kubernetes.kubeclient.Endpoint; import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient; +import org.apache.flink.kubernetes.kubeclient.KubernetesJobManagerSpecification; +import org.apache.flink.kubernetes.kubeclient.factory.KubernetesJobManagerFactory; +import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters; import org.apache.flink.kubernetes.utils.Constants; import org.apache.flink.kubernetes.utils.KubernetesUtils; import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; @@ -163,10 +166,6 @@ public class KubernetesClusterDescriptor implements ClusterDescriptor { TaskManagerOptions.RPC_PORT, Constants.TASK_MANAGER_RPC_PORT); - // Set jobmanager address to namespaced service name - final String nameSpace = flinkConfig.getString(KubernetesConfigOptions.NAMESPACE); - flinkConfig.setString(JobManagerOptions.ADDRESS, clusterId + "." + nameSpace); - if (HighAvailabilityMode.isHighAvailabilityModeActivated(flinkConfig)) { flinkConfig.setString(HighAvailabilityOptions.HA_CLUSTER_ID, clusterId); KubernetesUtils.checkAndUpdatePortConfigOption( @@ -176,7 +175,14 @@ public class KubernetesClusterDescriptor implements ClusterDescriptor { } try { - // todo + final KubernetesJobManagerParameters kubernetesJobManagerParameters = + new KubernetesJobManagerParameters(flinkConfig, clusterSpecification); + + final KubernetesJobManagerSpecification kubernetesJobManagerSpec = + KubernetesJobManagerFactory.createJobManagerComponent(kubernetesJobManagerParameters); + + client.createJobManagerComponent(kubernetesJobManagerSpec); + return createClusterClientProvider(clusterId); } catch (Exception e) { client.handleException(e); diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManager.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManager.java index e8fd05f34d..d3c65e1e83 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManager.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManager.java @@ -24,10 +24,9 @@ import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient; import org.apache.flink.kubernetes.kubeclient.KubeClientFactory; -import org.apache.flink.kubernetes.kubeclient.TaskManagerPodParameter; +import org.apache.flink.kubernetes.kubeclient.factory.KubernetesTaskManagerFactory; +import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesTaskManagerParameters; import org.apache.flink.kubernetes.kubeclient.resources.KubernetesPod; -import org.apache.flink.kubernetes.taskmanager.KubernetesTaskExecutorRunner; -import org.apache.flink.kubernetes.utils.Constants; import org.apache.flink.kubernetes.utils.KubernetesUtils; import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.clusterframework.BootstrapTools; @@ -36,7 +35,6 @@ import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.entrypoint.ClusterInformation; -import org.apache.flink.runtime.entrypoint.parser.CommandLineOptions; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup; @@ -53,8 +51,6 @@ import org.slf4j.LoggerFactory; import javax.annotation.Nullable; -import java.io.File; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -89,8 +85,6 @@ public class KubernetesResourceManager extends ActiveResourceManager taskManagerStartCommand; - /** The number of pods requested, but not yet granted. */ private int numPendingPodRequests = 0; @@ -126,8 +120,6 @@ public class KubernetesResourceManager extends ActiveResourceManager env = new HashMap<>(); - env.put(Constants.ENV_FLINK_POD_NAME, podName); - env.putAll(taskManagerParameters.taskManagerEnv()); + final String dynamicProperties = + BootstrapTools.getDynamicPropertiesAsString(flinkClientConfig, flinkConfig); - final TaskManagerPodParameter parameter = new TaskManagerPodParameter( + final KubernetesTaskManagerParameters kubernetesTaskManagerParameters = new KubernetesTaskManagerParameters( + flinkConfig, podName, - taskManagerStartCommand, defaultMemoryMB, - defaultCpus, - env); + dynamicProperties, + taskManagerParameters); + + final KubernetesPod taskManagerPod = + KubernetesTaskManagerFactory.createTaskManagerComponent(kubernetesTaskManagerParameters); log.info("TaskManager {} will be started with {}.", podName, taskExecutorProcessSpec); - kubeClient.createTaskManagerPod(parameter); + kubeClient.createTaskManagerPod(taskManagerPod); } /** @@ -296,30 +290,6 @@ public class KubernetesResourceManager extends ActiveResourceManager getTaskManagerStartCommand() { - final String confDir = flinkConfig.getString(KubernetesConfigOptions.FLINK_CONF_DIR); - final boolean hasLogback = new File(confDir, Constants.CONFIG_FILE_LOGBACK_NAME).exists(); - final boolean hasLog4j = new File(confDir, Constants.CONFIG_FILE_LOG4J_NAME).exists(); - - final String logDir = flinkConfig.getString(KubernetesConfigOptions.FLINK_LOG_DIR); - - final String mainClassArgs = "--" + CommandLineOptions.CONFIG_DIR_OPTION.getLongOpt() + " " + - flinkConfig.getString(KubernetesConfigOptions.FLINK_CONF_DIR) + " " + - BootstrapTools.getDynamicPropertiesAsString(flinkClientConfig, flinkConfig); - - final String command = KubernetesUtils.getTaskManagerStartCommand( - flinkConfig, - taskManagerParameters, - confDir, - logDir, - hasLogback, - hasLog4j, - KubernetesTaskExecutorRunner.class.getCanonicalName(), - mainClassArgs); - - return Arrays.asList("/bin/bash", "-c", command); - } - protected FlinkKubeClient createFlinkKubeClient() { return KubeClientFactory.fromConfiguration(flinkConfig); } diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptionsInternal.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptionsInternal.java index ac04cae78a..404f3ff0b3 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptionsInternal.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptionsInternal.java @@ -28,12 +28,6 @@ import org.apache.flink.configuration.ConfigOptions; @Internal public class KubernetesConfigOptionsInternal { - public static final ConfigOption SERVICE_ID = ConfigOptions - .key("kubernetes.internal.service.id") - .stringType() - .noDefaultValue() - .withDescription("The service id will be set in configuration after created. It will be used for gc."); - public static final ConfigOption ENTRY_POINT_CLASS = ConfigOptions .key("kubernetes.internal.jobmanager.entrypoint.class") .stringType() diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java index bfa6507281..e347e870b6 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java @@ -24,11 +24,15 @@ import org.apache.flink.configuration.RestOptions; import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; import org.apache.flink.kubernetes.kubeclient.resources.KubernetesPod; import org.apache.flink.kubernetes.kubeclient.resources.KubernetesService; -import org.apache.flink.kubernetes.utils.Constants; +import org.apache.flink.kubernetes.utils.KubernetesUtils; +import io.fabric8.kubernetes.api.model.HasMetadata; +import io.fabric8.kubernetes.api.model.OwnerReference; +import io.fabric8.kubernetes.api.model.OwnerReferenceBuilder; import io.fabric8.kubernetes.api.model.Pod; import io.fabric8.kubernetes.api.model.Service; import io.fabric8.kubernetes.api.model.ServicePort; +import io.fabric8.kubernetes.api.model.apps.Deployment; import io.fabric8.kubernetes.client.KubernetesClient; import io.fabric8.kubernetes.client.KubernetesClientException; import io.fabric8.kubernetes.client.Watcher; @@ -66,8 +70,52 @@ public class Fabric8FlinkKubeClient implements FlinkKubeClient { } @Override - public void createTaskManagerPod(TaskManagerPodParameter parameter) { - // todo + public void createJobManagerComponent(KubernetesJobManagerSpecification kubernetesJMSpec) { + final Deployment deployment = kubernetesJMSpec.getDeployment(); + final List accompanyingResources = kubernetesJMSpec.getAccompanyingResources(); + + // create Deployment + LOG.debug("Start to create deployment with spec {}", deployment.getSpec().toString()); + final Deployment createdDeployment = this.internalClient + .apps() + .deployments() + .inNamespace(this.nameSpace) + .create(deployment); + + // Note that we should use the uid of the created Deployment for the OwnerReference. + setOwnerReference(createdDeployment, accompanyingResources); + + this.internalClient + .resourceList(accompanyingResources) + .inNamespace(this.nameSpace) + .createOrReplace(); + } + + @Override + public void createTaskManagerPod(KubernetesPod kubernetesPod) { + final Deployment masterDeployment = this.internalClient + .apps() + .deployments() + .inNamespace(this.nameSpace) + .withName(KubernetesUtils.getDeploymentName(clusterId)) + .get(); + + if (masterDeployment == null) { + throw new RuntimeException( + "Failed to find Deployment named " + clusterId + " in namespace " + this.nameSpace); + } + + // Note that we should use the uid of the master Deployment for the OwnerReference. + setOwnerReference(masterDeployment, Collections.singletonList(kubernetesPod.getInternalResource())); + + LOG.debug("Start to create pod with metadata {}, spec {}", + kubernetesPod.getInternalResource().getMetadata(), + kubernetesPod.getInternalResource().getSpec()); + + this.internalClient + .pods() + .inNamespace(this.nameSpace) + .create(kubernetesPod.getInternalResource()); } @Override @@ -83,7 +131,7 @@ public class Fabric8FlinkKubeClient implements FlinkKubeClient { // Return the service.namespace directly when use ClusterIP. if (serviceExposedType.equals(KubernetesConfigOptions.ServiceExposedType.ClusterIP.toString())) { - return new Endpoint(clusterId + "." + nameSpace, restPort); + return new Endpoint(KubernetesUtils.getInternalServiceName(clusterId) + "." + nameSpace, restPort); } KubernetesService restService = getRestService(clusterId); @@ -130,7 +178,13 @@ public class Fabric8FlinkKubeClient implements FlinkKubeClient { @Override public void stopAndCleanupCluster(String clusterId) { - this.internalClient.services().inNamespace(this.nameSpace).withName(clusterId).cascading(true).delete(); + this.internalClient + .apps() + .deployments() + .inNamespace(this.nameSpace) + .withName(KubernetesUtils.getDeploymentName(clusterId)) + .cascading(true) + .delete(); } @Override @@ -138,16 +192,16 @@ public class Fabric8FlinkKubeClient implements FlinkKubeClient { LOG.error("A Kubernetes exception occurred.", e); } - @Override @Nullable + @Override public KubernetesService getInternalService(String clusterId) { - return getService(clusterId); + return getService(KubernetesUtils.getInternalServiceName(clusterId)); } @Override @Nullable public KubernetesService getRestService(String clusterId) { - return getService(clusterId + Constants.FLINK_REST_SERVICE_SUFFIX); + return getService(KubernetesUtils.getRestServiceName(clusterId)); } @Override @@ -188,6 +242,19 @@ public class Fabric8FlinkKubeClient implements FlinkKubeClient { this.internalClient.close(); } + private void setOwnerReference(Deployment deployment, List resources) { + final OwnerReference deploymentOwnerReference = new OwnerReferenceBuilder() + .withName(deployment.getMetadata().getName()) + .withApiVersion(deployment.getApiVersion()) + .withUid(deployment.getMetadata().getUid()) + .withKind(deployment.getKind()) + .withController(true) + .withBlockOwnerDeletion(true) + .build(); + resources.forEach(resource -> + resource.getMetadata().setOwnerReferences(Collections.singletonList(deploymentOwnerReference))); + } + private KubernetesService getService(String serviceName) { final Service service = this .internalClient diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkKubeClient.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkKubeClient.java index 69492b3283..1fc01cd5d3 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkKubeClient.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkKubeClient.java @@ -32,11 +32,15 @@ import java.util.Map; public interface FlinkKubeClient extends AutoCloseable { /** - * Create task manager pod. + * Create the Master components, this can include the Deployment, the ConfigMap(s), and the Service(s). * - * @param parameter {@link TaskManagerPodParameter} to create a taskmanager pod. */ - void createTaskManagerPod(TaskManagerPodParameter parameter); + void createJobManagerComponent(KubernetesJobManagerSpecification kubernetesJMSpec); + + /** + * Create task manager pod. + */ + void createTaskManagerPod(KubernetesPod kubernetesPod); /** * Stop a specified pod by name. diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/TaskManagerPodParameter.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/TaskManagerPodParameter.java deleted file mode 100644 index 72c2da9487..0000000000 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/TaskManagerPodParameter.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * 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.kubernetes.kubeclient; - -import java.util.List; -import java.util.Map; - -/** - * Parameters to create a taskmanager pod. - */ -public class TaskManagerPodParameter { - - private final String podName; - - private final List args; - - private final int taskManagerMemoryInMB; - - private final double taskManagerCpus; - - private final Map environmentVariables; - - public TaskManagerPodParameter( - String podName, - List args, - int taskManagerMemoryInMB, - double taskManagerCpus, - Map environmentVariables) { - this.podName = podName; - this.args = args; - this.taskManagerMemoryInMB = taskManagerMemoryInMB; - this.taskManagerCpus = taskManagerCpus; - this.environmentVariables = environmentVariables; - } - - public String getPodName() { - return podName; - } - - public List getArgs() { - return args; - } - - public Map getEnvironmentVariables() { - return environmentVariables; - } - - public int getTaskManagerMemoryInMB() { - return taskManagerMemoryInMB; - } - - public double getTaskManagerCpus() { - return taskManagerCpus; - } -} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesTaskManagerFactory.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesTaskManagerFactory.java index be91bbe4d3..60c2d8846a 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesTaskManagerFactory.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesTaskManagerFactory.java @@ -34,7 +34,7 @@ import io.fabric8.kubernetes.api.model.PodBuilder; */ public class KubernetesTaskManagerFactory { - public static KubernetesPod buildTaskManagerComponent(KubernetesTaskManagerParameters kubernetesTaskManagerParameters) { + public static KubernetesPod createTaskManagerComponent(KubernetesTaskManagerParameters kubernetesTaskManagerParameters) { FlinkPod flinkPod = new FlinkPod.Builder().build(); final KubernetesStepDecorator[] stepDecorators = new KubernetesStepDecorator[] { diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/ActionWatcher.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/ActionWatcher.java deleted file mode 100644 index b5123ef2cb..0000000000 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/resources/ActionWatcher.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * 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.kubernetes.kubeclient.resources; - -import io.fabric8.kubernetes.api.model.HasMetadata; -import io.fabric8.kubernetes.client.KubernetesClientException; -import io.fabric8.kubernetes.client.KubernetesClientTimeoutException; -import io.fabric8.kubernetes.client.Watcher; - -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; - -/** - * Watch a specific action. - */ -public class ActionWatcher implements Watcher { - private final CountDownLatch latch = new CountDownLatch(1); - private final AtomicReference reference = new AtomicReference<>(); - private final T resource; - private final Action expectedAction; - - public ActionWatcher(Action expectedAction, T resource) { - this.resource = resource; - this.expectedAction = expectedAction; - } - - @Override - public void eventReceived(Action action, T resource) { - if (action == this.expectedAction) { - this.reference.set(resource); - this.latch.countDown(); - } - } - - @Override - public void onClose(KubernetesClientException e) { - } - - public T await(long amount, TimeUnit timeUnit) throws InterruptedException { - if (this.latch.await(amount, timeUnit)) { - return this.reference.get(); - } else { - throw new KubernetesClientTimeoutException(this.resource, amount, timeUnit); - } - } -} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java index b7ca83c220..a7286f7d79 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java @@ -23,18 +23,11 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; import org.apache.flink.runtime.clusterframework.BootstrapTools; -import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; -import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec; -import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils; import org.apache.flink.util.FlinkRuntimeException; -import io.fabric8.kubernetes.api.model.HasMetadata; -import io.fabric8.kubernetes.api.model.OwnerReference; -import io.fabric8.kubernetes.api.model.OwnerReferenceBuilder; import io.fabric8.kubernetes.api.model.Quantity; import io.fabric8.kubernetes.api.model.ResourceRequirements; import io.fabric8.kubernetes.api.model.ResourceRequirementsBuilder; -import io.fabric8.kubernetes.api.model.apps.Deployment; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,9 +39,7 @@ import java.io.FileInputStream; import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStreamReader; -import java.util.Collections; import java.util.HashMap; -import java.util.List; import java.util.Map; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -121,47 +112,6 @@ public class KubernetesUtils { } } - /** - * Generates the shell command to start a task manager for kubernetes. - * - * @param flinkConfig The Flink configuration. - * @param tmParams Parameters for the task manager. - * @param configDirectory The configuration directory for the flink-conf.yaml - * @param logDirectory The log directory. - * @param hasLogback Uses logback? - * @param hasLog4j Uses log4j? - * @param mainClass The main class to start with. - * @param mainArgs The args for main class. - * @return A String containing the task manager startup command. - */ - public static String getTaskManagerStartCommand( - Configuration flinkConfig, - ContaineredTaskManagerParameters tmParams, - String configDirectory, - String logDirectory, - boolean hasLogback, - boolean hasLog4j, - String mainClass, - @Nullable String mainArgs) { - final TaskExecutorProcessSpec taskExecutorProcessSpec = tmParams.getTaskExecutorProcessSpec(); - final String jvmMemOpts = TaskExecutorProcessUtils.generateJvmParametersStr(taskExecutorProcessSpec); - String args = TaskExecutorProcessUtils.generateDynamicConfigsStr(taskExecutorProcessSpec); - if (mainArgs != null) { - args += " " + mainArgs; - } - return getCommonStartCommand( - flinkConfig, - ClusterComponent.TASK_MANAGER, - jvmMemOpts, - configDirectory, - logDirectory, - hasLogback, - hasLog4j, - mainClass, - args - ); - } - /** * Generate name of the internal Service. */ diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesClusterDescriptorTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesClusterDescriptorTest.java index 4522385251..fdbd0897b1 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesClusterDescriptorTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesClusterDescriptorTest.java @@ -26,16 +26,22 @@ import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.kubernetes.utils.Constants; +import org.apache.flink.kubernetes.utils.KubernetesUtils; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; import io.fabric8.kubernetes.api.model.Container; import io.fabric8.kubernetes.api.model.EnvVar; +import io.fabric8.kubernetes.api.model.LoadBalancerIngress; +import io.fabric8.kubernetes.api.model.LoadBalancerStatus; import io.fabric8.kubernetes.api.model.Service; +import io.fabric8.kubernetes.api.model.ServiceBuilder; +import io.fabric8.kubernetes.api.model.ServiceStatusBuilder; import io.fabric8.kubernetes.api.model.apps.Deployment; import io.fabric8.kubernetes.client.KubernetesClient; +import org.junit.Before; import org.junit.Test; -import java.util.List; +import java.util.Collections; import java.util.stream.Collectors; import static org.apache.flink.kubernetes.utils.Constants.ENV_FLINK_POD_IP_ADDRESS; @@ -46,19 +52,27 @@ import static org.junit.Assert.assertTrue; * Tests for the {@link KubernetesClusterDescriptor}. */ public class KubernetesClusterDescriptorTest extends KubernetesTestBase { - + private static final String MOCK_SERVICE_HOST_NAME = "mock-host-name-of-service"; private static final String MOCK_SERVICE_IP = "192.168.0.1"; private final ClusterSpecification clusterSpecification = new ClusterSpecification.ClusterSpecificationBuilder() .createClusterSpecification(); + @Before + public void setup() throws Exception { + super.setup(); + + mockGetRestServiceWithLoadBalancer(MOCK_SERVICE_HOST_NAME, MOCK_SERVICE_IP); + } + @Test public void testDeploySessionCluster() throws Exception { final ClusterClient clusterClient = deploySessionCluster(); // Check updated flink config options assertEquals(String.valueOf(Constants.BLOB_SERVER_PORT), flinkConfig.getString(BlobServerOptions.PORT)); assertEquals(String.valueOf(Constants.TASK_MANAGER_RPC_PORT), flinkConfig.getString(TaskManagerOptions.RPC_PORT)); - assertEquals(CLUSTER_ID + "." + NAMESPACE, flinkConfig.getString(JobManagerOptions.ADDRESS)); + assertEquals(KubernetesUtils.getInternalServiceName(CLUSTER_ID) + "." + + NAMESPACE, flinkConfig.getString(JobManagerOptions.ADDRESS)); final Deployment jmDeployment = kubeClient .apps() @@ -87,7 +101,6 @@ public class KubernetesClusterDescriptorTest extends KubernetesTestBase { @Test public void testDeployHighAvailabilitySessionCluster() throws ClusterDeploymentException { flinkConfig.setString(HighAvailabilityOptions.HA_MODE, HighAvailabilityMode.ZOOKEEPER.toString()); - final ClusterClient clusterClient = deploySessionCluster(); final KubernetesClient kubeClient = server.getClient(); @@ -126,9 +139,10 @@ public class KubernetesClusterDescriptorTest extends KubernetesTestBase { descriptor.killCluster(CLUSTER_ID); - // Mock kubernetes server do not delete the rest service by gc, so the rest service still exist. - final List services = kubeClient.services().list().getItems(); - assertEquals(1, services.size()); + // Mock kubernetes server do not delete the accompanying resources by gc. + assertTrue(kubeClient.apps().deployments().list().getItems().isEmpty()); + assertEquals(2, kubeClient.services().list().getItems().size()); + assertEquals(1, kubeClient.configMaps().list().getItems().size()); } private ClusterClient deploySessionCluster() throws ClusterDeploymentException { @@ -144,4 +158,31 @@ public class KubernetesClusterDescriptorTest extends KubernetesTestBase { return clusterClient; } + + private void mockGetRestServiceWithLoadBalancer(String hostname, String ip) { + final String restServiceName = KubernetesUtils.getRestServiceName(CLUSTER_ID); + + final String path = String.format("/api/v1/namespaces/%s/services/%s", NAMESPACE, restServiceName); + server.expect() + .get() + .withPath(path) + .andReturn(200, buildMockRestService(hostname, ip)) + .always(); + } + + private Service buildMockRestService(String hostname, String ip) { + final Service service = new ServiceBuilder() + .editOrNewMetadata() + .withName(KubernetesUtils.getRestServiceName(CLUSTER_ID)) + .endMetadata() + .editOrNewSpec() + .endSpec() + .build(); + + service.setStatus(new ServiceStatusBuilder() + .withLoadBalancer(new LoadBalancerStatus(Collections.singletonList( + new LoadBalancerIngress(hostname, ip)))).build()); + + return service; + } } diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesResourceManagerTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesResourceManagerTest.java index 834873cde9..2fa8ca6951 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesResourceManagerTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesResourceManagerTest.java @@ -25,7 +25,6 @@ import org.apache.flink.configuration.MemorySize; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient; -import org.apache.flink.kubernetes.kubeclient.TaskManagerPodParameter; import org.apache.flink.kubernetes.kubeclient.resources.KubernetesPod; import org.apache.flink.kubernetes.utils.Constants; import org.apache.flink.kubernetes.utils.KubernetesUtils; @@ -62,6 +61,7 @@ import io.fabric8.kubernetes.api.model.ContainerStateBuilder; import io.fabric8.kubernetes.api.model.ContainerStatusBuilder; import io.fabric8.kubernetes.api.model.EnvVarBuilder; import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.PodBuilder; import io.fabric8.kubernetes.api.model.PodList; import io.fabric8.kubernetes.api.model.PodStatusBuilder; import io.fabric8.kubernetes.api.model.apps.Deployment; @@ -72,9 +72,7 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; @@ -265,12 +263,17 @@ public class KubernetesResourceManagerTest extends KubernetesTestBase { public void testGetWorkerNodesFromPreviousAttempts() throws Exception { // Prepare pod of previous attempt final String previewPodName = CLUSTER_ID + "-taskmanager-1-1"; - flinkKubeClient.createTaskManagerPod(new TaskManagerPodParameter( - previewPodName, - new ArrayList<>(), - 1024, - 1, - new HashMap<>())); + + final Pod mockTaskManagerPod = new PodBuilder() + .editOrNewMetadata() + .withName(previewPodName) + .withLabels(KubernetesUtils.getTaskManagerLabels(CLUSTER_ID)) + .endMetadata() + .editOrNewSpec() + .endSpec() + .build(); + + flinkKubeClient.createTaskManagerPod(new KubernetesPod(mockTaskManagerPod)); assertEquals(1, kubeClient.pods().list().getItems().size()); // Call initialize method to recover worker nodes from previous attempt. diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java index 54251b3677..351f7644e6 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClientTest.java @@ -18,33 +18,236 @@ package org.apache.flink.kubernetes.kubeclient; +import org.apache.flink.client.deployment.ClusterSpecification; +import org.apache.flink.configuration.BlobServerOptions; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.RestOptions; import org.apache.flink.kubernetes.KubernetesTestBase; +import org.apache.flink.kubernetes.KubernetesTestUtils; +import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; +import org.apache.flink.kubernetes.configuration.KubernetesConfigOptionsInternal; +import org.apache.flink.kubernetes.entrypoint.KubernetesSessionClusterEntrypoint; +import org.apache.flink.kubernetes.kubeclient.factory.KubernetesJobManagerFactory; +import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters; +import org.apache.flink.kubernetes.kubeclient.resources.KubernetesPod; +import org.apache.flink.kubernetes.utils.KubernetesUtils; +import io.fabric8.kubernetes.api.model.ConfigMap; +import io.fabric8.kubernetes.api.model.HasMetadata; +import io.fabric8.kubernetes.api.model.LoadBalancerIngress; +import io.fabric8.kubernetes.api.model.LoadBalancerStatus; +import io.fabric8.kubernetes.api.model.OwnerReference; +import io.fabric8.kubernetes.api.model.Pod; +import io.fabric8.kubernetes.api.model.PodBuilder; +import io.fabric8.kubernetes.api.model.Service; +import io.fabric8.kubernetes.api.model.ServiceBuilder; +import io.fabric8.kubernetes.api.model.ServiceStatusBuilder; +import io.fabric8.kubernetes.api.model.apps.Deployment; import org.junit.Before; import org.junit.Test; +import javax.annotation.Nullable; + +import java.util.Collections; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + /** * Tests for Fabric implementation of {@link FlinkKubeClient}. */ public class Fabric8FlinkKubeClientTest extends KubernetesTestBase { + private static final int REST_PORT = 9081; + private static final int RPC_PORT = 7123; + private static final int BLOB_SERVER_PORT = 8346; + + private static final double JOB_MANAGER_CPU = 2.0; + private static final int JOB_MANAGER_MEMORY = 768; + + private static final String SERVICE_ACCOUNT_NAME = "service-test"; + + private static final String ENTRY_POINT_CLASS = KubernetesSessionClusterEntrypoint.class.getCanonicalName(); + + private KubernetesJobManagerSpecification kubernetesJobManagerSpecification; @Before public void setup() throws Exception { super.setup(); + + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "logback.xml"); + KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "log4j.properties"); + + flinkConfig.set(KubernetesConfigOptions.CONTAINER_IMAGE_PULL_POLICY, CONTAINER_IMAGE_PULL_POLICY); + flinkConfig.set(KubernetesConfigOptionsInternal.ENTRY_POINT_CLASS, ENTRY_POINT_CLASS); + flinkConfig.set(RestOptions.PORT, REST_PORT); + flinkConfig.set(JobManagerOptions.PORT, RPC_PORT); + flinkConfig.set(BlobServerOptions.PORT, Integer.toString(BLOB_SERVER_PORT)); + flinkConfig.set(KubernetesConfigOptions.JOB_MANAGER_CPU, JOB_MANAGER_CPU); + flinkConfig.set(KubernetesConfigOptions.JOB_MANAGER_SERVICE_ACCOUNT, SERVICE_ACCOUNT_NAME); + + final ClusterSpecification clusterSpecification = new ClusterSpecification.ClusterSpecificationBuilder() + .setMasterMemoryMB(JOB_MANAGER_MEMORY) + .setTaskManagerMemoryMB(1000) + .setSlotsPerTaskManager(3) + .createClusterSpecification(); + + final KubernetesJobManagerParameters kubernetesJobManagerParameters = + new KubernetesJobManagerParameters(flinkConfig, clusterSpecification); + this.kubernetesJobManagerSpecification = + KubernetesJobManagerFactory.createJobManagerComponent(kubernetesJobManagerParameters); } @Test - public void testCreateTaskManagerPod() { - // todo + public void testCreateFlinkMasterComponent() throws Exception { + flinkKubeClient.createJobManagerComponent(this.kubernetesJobManagerSpecification); + + final List resultedDeployments = kubeClient.apps().deployments() + .inNamespace(NAMESPACE) + .list() + .getItems(); + assertEquals(1, resultedDeployments.size()); + + final List resultedConfigMaps = kubeClient.configMaps() + .inNamespace(NAMESPACE) + .list() + .getItems(); + assertEquals(1, resultedConfigMaps.size()); + + final List resultedServices = kubeClient.services() + .inNamespace(NAMESPACE) + .list() + .getItems(); + assertEquals(2, resultedServices.size()); + + testOwnerReferenceSetting(resultedDeployments.get(0), resultedConfigMaps); + testOwnerReferenceSetting(resultedDeployments.get(0), resultedServices); + } + + private void testOwnerReferenceSetting( + HasMetadata ownerReference, + List resources) { + resources.forEach(resource -> { + List ownerReferences = resource.getMetadata().getOwnerReferences(); + assertEquals(1, ownerReferences.size()); + assertEquals(ownerReference.getMetadata().getUid(), ownerReferences.get(0).getUid()); + }); + } + + @Test + public void testCreateFlinkTaskManagerPod() throws Exception { + this.flinkKubeClient.createJobManagerComponent(this.kubernetesJobManagerSpecification); + + final KubernetesPod kubernetesPod = new KubernetesPod(new PodBuilder() + .editOrNewMetadata() + .withName("mock-task-manager-pod") + .endMetadata() + .editOrNewSpec() + .endSpec() + .build()); + this.flinkKubeClient.createTaskManagerPod(kubernetesPod); + + final Pod resultTaskManagerPod = + this.kubeClient.pods().inNamespace(NAMESPACE).withName("mock-task-manager-pod").get(); + + assertEquals( + this.kubeClient.apps().deployments().inNamespace(NAMESPACE).list().getItems().get(0).getMetadata().getUid(), + resultTaskManagerPod.getMetadata().getOwnerReferences().get(0).getUid()); } @Test - public void testServiceLoadBalancerWithNoIP() throws Exception { - // todo + public void testStopPod() { + final String podName = "pod-for-delete"; + final Pod pod = new PodBuilder() + .editOrNewMetadata() + .withName(podName) + .endMetadata() + .editOrNewSpec() + .endSpec() + .build(); + + this.kubeClient.pods().inNamespace(NAMESPACE).create(pod); + assertNotNull(this.kubeClient.pods().inNamespace(NAMESPACE).withName(podName).get()); + + this.flinkKubeClient.stopPod(podName); + assertNull(this.kubeClient.pods().inNamespace(NAMESPACE).withName(podName).get()); } @Test - public void testServiceLoadBalancerEmptyHostAndIP() throws Exception { - // todo + public void testServiceLoadBalancerWithNoIP() { + final String hostName = "test-host-name"; + mockRestServiceWithLB(hostName, ""); + + final Endpoint resultEndpoint = flinkKubeClient.getRestEndpoint(CLUSTER_ID); + + assertEquals(hostName, resultEndpoint.getAddress()); + assertEquals(REST_PORT, resultEndpoint.getPort()); + } + + @Test + public void testServiceLoadBalancerEmptyHostAndIP() { + mockRestServiceWithLB("", ""); + + final Endpoint resultEndpoint1 = flinkKubeClient.getRestEndpoint(CLUSTER_ID); + assertNull(resultEndpoint1); + } + + @Test + public void testServiceLoadBalancerNullHostAndIP() { + mockRestServiceWithLB(null, null); + + final Endpoint resultEndpoint2 = flinkKubeClient.getRestEndpoint(CLUSTER_ID); + assertNull(resultEndpoint2); + } + + @Test + public void testStopAndCleanupCluster() throws Exception { + this.flinkKubeClient.createJobManagerComponent(this.kubernetesJobManagerSpecification); + + final KubernetesPod kubernetesPod = new KubernetesPod(new PodBuilder() + .editOrNewMetadata() + .withName("mock-task-manager-pod") + .endMetadata() + .editOrNewSpec() + .endSpec() + .build()); + this.flinkKubeClient.createTaskManagerPod(kubernetesPod); + + assertEquals(1, this.kubeClient.apps().deployments().inNamespace(NAMESPACE).list().getItems().size()); + assertEquals(1, this.kubeClient.configMaps().inNamespace(NAMESPACE).list().getItems().size()); + assertEquals(2, this.kubeClient.services().inNamespace(NAMESPACE).list().getItems().size()); + assertEquals(1, this.kubeClient.pods().inNamespace(NAMESPACE).list().getItems().size()); + + this.flinkKubeClient.stopAndCleanupCluster(CLUSTER_ID); + assertTrue(this.kubeClient.apps().deployments().inNamespace(NAMESPACE).list().getItems().isEmpty()); + } + + private void mockRestServiceWithLB(@Nullable String hostname, @Nullable String ip) { + final String restServiceName = KubernetesUtils.getRestServiceName(CLUSTER_ID); + + final String path = String.format("/api/v1/namespaces/%s/services/%s", NAMESPACE, restServiceName); + server.expect() + .withPath(path) + .andReturn(200, buildMockRestServiceWithLB(hostname, ip)) + .always(); + + final Service resultService = kubeClient.services() + .inNamespace(NAMESPACE) + .withName(KubernetesUtils.getRestServiceName(CLUSTER_ID)) + .get(); + assertNotNull(resultService); + } + + private Service buildMockRestServiceWithLB(@Nullable String hostname, @Nullable String ip) { + final Service service = new ServiceBuilder() + .build(); + + service.setStatus(new ServiceStatusBuilder() + .withLoadBalancer(new LoadBalancerStatus(Collections.singletonList( + new LoadBalancerIngress(hostname, ip)))).build()); + + return service; } } diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesTaskManagerFactoryTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesTaskManagerFactoryTest.java index 8d37b621dd..b54d0839ca 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesTaskManagerFactoryTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesTaskManagerFactoryTest.java @@ -46,7 +46,7 @@ public class KubernetesTaskManagerFactoryTest extends KubernetesTaskManagerTestB KubernetesTestUtils.createTemporyFile("some data", flinkConfDir, "log4j.properties"); this.resultPod = - KubernetesTaskManagerFactory.buildTaskManagerComponent(kubernetesTaskManagerParameters).getInternalResource(); + KubernetesTaskManagerFactory.createTaskManagerComponent(kubernetesTaskManagerParameters).getInternalResource(); } @Test -- Gitee From dd8ce6c19eefeaf60c085365fe8bea960c1dace8 Mon Sep 17 00:00:00 2001 From: tison Date: Thu, 5 Mar 2020 02:00:32 +0800 Subject: [PATCH 061/885] [FLINK-16194][k8s] Make FlinkPod a value class --- .../apache/flink/kubernetes/kubeclient/FlinkPod.java | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkPod.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkPod.java index f301d8fdd1..72c4db6ff2 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkPod.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkPod.java @@ -31,9 +31,9 @@ import static org.apache.flink.util.Preconditions.checkNotNull; */ public class FlinkPod { - private Pod pod; + private final Pod pod; - private Container mainContainer; + private final Container mainContainer; public FlinkPod(Pod pod, Container mainContainer) { this.pod = pod; @@ -44,18 +44,10 @@ public class FlinkPod { return pod; } - public void setPod(Pod pod) { - this.pod = pod; - } - public Container getMainContainer() { return mainContainer; } - public void setMainContainer(Container mainContainer) { - this.mainContainer = mainContainer; - } - /** * Builder for creating a {@link FlinkPod}. */ -- Gitee From 3408eade79f59f7f9228c39ccc976ed5baab0581 Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Wed, 26 Feb 2020 19:59:33 +0530 Subject: [PATCH 062/885] [FLINK-15584][table-planner] Give nested data type of ROWs in ValidationException --- .../flink/table/sinks/TableSinkUtils.scala | 4 ++-- .../validation/InsertIntoValidationTest.scala | 19 +++++++++++++++++++ 2 files changed, 21 insertions(+), 2 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/sinks/TableSinkUtils.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/sinks/TableSinkUtils.scala index 17641315d8..cde3111141 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/sinks/TableSinkUtils.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/sinks/TableSinkUtils.scala @@ -55,10 +55,10 @@ object TableSinkUtils { // format table and table sink schema strings val srcSchema = srcFieldNames.zip(srcFieldTypes) - .map { case (n, t) => s"$n: ${t.getTypeClass.getSimpleName}" } + .map { case (n, t) => s"$n: $t" } .mkString("[", ", ", "]") val sinkSchema = sinkFieldNames.zip(sinkFieldTypes) - .map { case (n, t) => s"$n: ${t.getTypeClass.getSimpleName}" } + .map { case (n, t) => s"$n: $t" } .mkString("[", ", ", "]") throw new ValidationException( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/sql/validation/InsertIntoValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/sql/validation/InsertIntoValidationTest.scala index faccc9f5be..d8915dffee 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/sql/validation/InsertIntoValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/sql/validation/InsertIntoValidationTest.scala @@ -74,4 +74,23 @@ class InsertIntoValidationTest extends TableTestBase { // must fail because partial insert is not supported yet. util.tableEnv.sqlUpdate(sql) } + + @Test + def testValidationExceptionMessage(): Unit = { + expectedException.expect(classOf[ValidationException]) + expectedException.expectMessage("TableSink schema: [a: Integer, b: Row" + + "(f0: Integer, f1: Integer, f2: Integer)]") + val util = batchTestUtil() + util.addTable[(Int, Long, String)]("sourceTable", 'a, 'b, 'c) + val fieldNames = Array("a", "b") + val fieldTypes: Array[TypeInformation[_]] = Array(Types.INT, Types.ROW + (Types.INT, Types.INT, Types.INT)) + val sink = new MemoryTableSourceSinkUtil.UnsafeMemoryAppendTableSink + util.tableEnv.registerTableSink("targetTable", sink.configure(fieldNames, + fieldTypes)) + + val sql = "INSERT INTO targetTable SELECT a, b FROM sourceTable" + + util.tableEnv.sqlUpdate(sql) + } } -- Gitee From 0362d200e3cd9ed86fd363f0c48f1a7d2d7e852f Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Thu, 5 Mar 2020 09:56:30 +0800 Subject: [PATCH 063/885] [FLINK-16414][table] Fix sql validation failed when using udaf/udtf has no getResultType (#11302) --- .../functions/FunctionDefinitionUtil.java | 10 +-- .../functions/FunctionDefinitionUtilTest.java | 67 +++++++++++++++++-- 2 files changed, 66 insertions(+), 11 deletions(-) diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/functions/FunctionDefinitionUtil.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/functions/FunctionDefinitionUtil.java index da798b6951..374b138d7b 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/functions/FunctionDefinitionUtil.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/functions/FunctionDefinitionUtil.java @@ -45,7 +45,7 @@ public class FunctionDefinitionUtil { return new TableFunctionDefinition( name, t, - t.getResultType() + UserDefinedFunctionHelper.getReturnTypeOfTableFunction(t) ); } else if (udf instanceof AggregateFunction) { AggregateFunction a = (AggregateFunction) udf; @@ -53,8 +53,8 @@ public class FunctionDefinitionUtil { return new AggregateFunctionDefinition( name, a, - a.getAccumulatorType(), - a.getResultType() + UserDefinedFunctionHelper.getAccumulatorTypeOfAggregateFunction(a), + UserDefinedFunctionHelper.getReturnTypeOfAggregateFunction(a) ); } else if (udf instanceof TableAggregateFunction) { TableAggregateFunction a = (TableAggregateFunction) udf; @@ -62,8 +62,8 @@ public class FunctionDefinitionUtil { return new TableAggregateFunctionDefinition( name, a, - a.getAccumulatorType(), - a.getResultType() + UserDefinedFunctionHelper.getAccumulatorTypeOfAggregateFunction(a), + UserDefinedFunctionHelper.getReturnTypeOfAggregateFunction(a) ); } else { throw new UnsupportedOperationException( diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/functions/FunctionDefinitionUtilTest.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/functions/FunctionDefinitionUtilTest.java index b37ed8be16..691734cc6a 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/functions/FunctionDefinitionUtilTest.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/functions/FunctionDefinitionUtilTest.java @@ -40,32 +40,56 @@ public class FunctionDefinitionUtilTest { @Test public void testTableFunction() { - FunctionDefinition fd = FunctionDefinitionUtil.createFunctionDefinition( + FunctionDefinition fd1 = FunctionDefinitionUtil.createFunctionDefinition( "test", TestTableFunction.class.getName() ); - assertTrue(((TableFunctionDefinition) fd).getTableFunction() instanceof TestTableFunction); + assertTrue(((TableFunctionDefinition) fd1).getTableFunction() instanceof TestTableFunction); + + FunctionDefinition fd2 = FunctionDefinitionUtil.createFunctionDefinition( + "test", + TestTableFunctionWithoutResultType.class.getName() + ); + + assertTrue(((TableFunctionDefinition) fd2).getTableFunction() instanceof TestTableFunctionWithoutResultType); } @Test public void testAggregateFunction() { - FunctionDefinition fd = FunctionDefinitionUtil.createFunctionDefinition( + FunctionDefinition fd1 = FunctionDefinitionUtil.createFunctionDefinition( "test", TestAggFunction.class.getName() ); - assertTrue(((AggregateFunctionDefinition) fd).getAggregateFunction() instanceof TestAggFunction); + assertTrue(((AggregateFunctionDefinition) fd1).getAggregateFunction() instanceof TestAggFunction); + + FunctionDefinition fd2 = FunctionDefinitionUtil.createFunctionDefinition( + "test", + TestAggFunctionWithoutResultType.class.getName() + ); + + assertTrue(((AggregateFunctionDefinition) fd2).getAggregateFunction() + instanceof TestAggFunctionWithoutResultType); + } @Test public void testTableAggregateFunction() { - FunctionDefinition fd = FunctionDefinitionUtil.createFunctionDefinition( + FunctionDefinition fd1 = FunctionDefinitionUtil.createFunctionDefinition( "test", TestTableAggFunction.class.getName() ); - assertTrue(((TableAggregateFunctionDefinition) fd).getTableAggregateFunction() instanceof TestTableAggFunction); + assertTrue(((TableAggregateFunctionDefinition) fd1).getTableAggregateFunction() instanceof TestTableAggFunction); + + FunctionDefinition fd2 = FunctionDefinitionUtil.createFunctionDefinition( + "test", + TestTableAggFunctionWithoutResultType.class.getName() + ); + + assertTrue(((TableAggregateFunctionDefinition) fd2).getTableAggregateFunction() + instanceof TestTableAggFunctionWithoutResultType); } /** @@ -85,6 +109,12 @@ public class FunctionDefinitionUtilTest { } } + /** + * Test function. + */ + public static class TestTableFunctionWithoutResultType extends TableFunction { + } + /** * Test function. */ @@ -110,6 +140,21 @@ public class FunctionDefinitionUtilTest { } } + /** + * Test function. + */ + public static class TestAggFunctionWithoutResultType extends AggregateFunction { + @Override + public Long createAccumulator() { + return null; + } + + @Override + public Long getValue(Long accumulator) { + return null; + } + } + /** * Test function. */ @@ -129,4 +174,14 @@ public class FunctionDefinitionUtilTest { return TypeInformation.of(Object.class); } } + + /** + * Test function. + */ + public static class TestTableAggFunctionWithoutResultType extends TableAggregateFunction { + @Override + public Long createAccumulator() { + return null; + } + } } -- Gitee From 59b65844e285009b60dd9481dd2aec0458dd7094 Mon Sep 17 00:00:00 2001 From: Qingsheng Ren Date: Thu, 27 Feb 2020 15:54:33 +0800 Subject: [PATCH 064/885] [FLINK-16133][docs-zh] Translate /ops/filesystems/azure.zh.md This closes #11232 --- docs/ops/filesystems/azure.md | 4 ++-- docs/ops/filesystems/azure.zh.md | 40 ++++++++++++++------------------ 2 files changed, 19 insertions(+), 25 deletions(-) diff --git a/docs/ops/filesystems/azure.md b/docs/ops/filesystems/azure.md index d63c9d63a6..e24b1125b3 100644 --- a/docs/ops/filesystems/azure.md +++ b/docs/ops/filesystems/azure.md @@ -53,7 +53,7 @@ env.setStateBackend(new FsStateBackend("wasb://@$.blob.core.windows.net: {% endhighlight %} -Alternatively, the the filesystem can be configured to read the Azure Blob Storage key from an +Alternatively, the filesystem can be configured to read the Azure Blob Storage key from an environment variable `AZURE_STORAGE_KEY` by setting the following configuration keys in `flink-conf.yaml`. diff --git a/docs/ops/filesystems/azure.zh.md b/docs/ops/filesystems/azure.zh.md index 4d81b5596b..0257bd8769 100644 --- a/docs/ops/filesystems/azure.zh.md +++ b/docs/ops/filesystems/azure.zh.md @@ -1,6 +1,6 @@ --- -title: "Azure Blob Storage" -nav-title: Azure Blob Storage +title: "Azure Blob 存储" +nav-title: Azure Blob 存储 nav-parent_id: filesystems nav-pos: 3 --- @@ -23,60 +23,54 @@ specific language governing permissions and limitations under the License. --> -[Azure Blob Storage](https://docs.microsoft.com/en-us/azure/storage/) is a Microsoft-managed service providing cloud storage for a variety of use cases. -You can use Azure Blob Storage with Flink for **reading** and **writing data** as well in conjunction with the [streaming **state backends**]({{ site.baseurl }}/ops/state/state_backends.html) +[Azure Blob 存储](https://docs.microsoft.com/en-us/azure/storage/) 是一项由 Microsoft 管理的服务,能提供多种应用场景下的云存储。 +Azure Blob 存储可与 Flink 一起使用以**读取**和**写入数据**,以及与[流 State Backend]({{ site.baseurl }}/zh/ops/state/state_backends.html) 结合使用。 * This will be replaced by the TOC {:toc} -You can use Azure Blob Storage objects like regular files by specifying paths in the following format: +通过以下格式指定路径,Azure Blob 存储对象可类似于普通文件使用: {% highlight plain %} wasb://@$.blob.core.windows.net/ -// SSL encrypted access +// SSL 加密访问 wasbs://@$.blob.core.windows.net/ {% endhighlight %} -See below for how to use Azure Blob Storage in a Flink job: +参见以下代码了解如何在 Flink 作业中使用 Azure Blob 存储: {% highlight java %} -// Read from Azure Blob storage +// 读取 Azure Blob 存储 env.readTextFile("wasb://@$.blob.core.windows.net/"); -// Write to Azure Blob storage +// 写入 Azure Blob 存储 stream.writeAsText("wasb://@$.blob.core.windows.net/") -// Use Azure Blob Storage as FsStatebackend +// 将 Azure Blob 存储用作 FsStatebackend env.setStateBackend(new FsStateBackend("wasb://@$.blob.core.windows.net/")); {% endhighlight %} -### Shaded Hadoop Azure Blob Storage file system +### Shaded Hadoop Azure Blob 存储文件系统 -To use `flink-azure-fs-hadoop,` copy the respective JAR file from the `opt` directory to the `plugins` directory of your Flink distribution before starting Flink, e.g. +为使用 flink-azure-fs-hadoop,在启动 Flink 之前,将对应的 JAR 文件从 opt 目录复制到 Flink 发行版中的 plugin 目录下的一个文件夹中,例如: {% highlight bash %} mkdir ./plugins/azure-fs-hadoop cp ./opt/flink-azure-fs-hadoop-{{ site.version }}.jar ./plugins/azure-fs-hadoop/ {% endhighlight %} -`flink-azure-fs-hadoop` registers default FileSystem wrappers for URIs with the *wasb://* and *wasbs://* (SSL encrypted access) scheme. +`flink-azure-fs-hadoop` 为使用 *wasb://* 和 *wasbs://* (SSL 加密访问) 的 URI 注册了默认的文件系统包装器。 -### Credentials Configuration - -Hadoop's Azure Filesystem supports configuration of credentials via the Hadoop configuration as -outlined in the [Hadoop Azure Blob Storage documentation](https://hadoop.apache.org/docs/current/hadoop-azure/index.html#Configuring_Credentials). -For convenience Flink forwards all Flink configurations with a key prefix of `fs.azure` to the -Hadoop configuration of the filesystem. Consequentially, the azure blob storage key can be configured -in `flink-conf.yaml` via: +### 凭据配置 +Hadoop 的 Azure 文件系统支持通过 Hadoop 配置来配置凭据,如 [Hadoop Azure Blob Storage 文档](https://hadoop.apache.org/docs/current/hadoop-azure/index.html#Configuring_Credentials) 所述。 +为方便起见,Flink 将所有的 Flink 配置添加 `fs.azure` 键前缀后转发至文件系统的 Hadoop 配置中。因此,可通过以下方法在 `flink-conf.yaml` 中配置 Azure Blob 存储密钥: {% highlight yaml %} fs.azure.account.key..blob.core.windows.net: {% endhighlight %} -Alternatively, the the filesystem can be configured to read the Azure Blob Storage key from an -environment variable `AZURE_STORAGE_KEY` by setting the following configuration keys in -`flink-conf.yaml`. +或者通过在 `flink-conf.yaml` 中设置以下配置键,将文件系统配置为从环境变量 `AZURE_STORAGE_KEY` 读取 Azure Blob 存储密钥: {% highlight yaml %} fs.azure.account.keyprovider..blob.core.windows.net: org.apache.flink.fs.azurefs.EnvironmentVariableKeyProvider -- Gitee From 0428b103663ccd489f3f7eff81ec838a6bfcc7ca Mon Sep 17 00:00:00 2001 From: Qingsheng Ren Date: Tue, 25 Feb 2020 15:04:43 +0800 Subject: [PATCH 065/885] [FLINK-16132][docs-zh] Translate /ops/filesystems/oss.zh.md into Chinese This closes #11231 --- docs/ops/filesystems/oss.md | 4 ++-- docs/ops/filesystems/oss.zh.md | 44 +++++++++++++++++----------------- 2 files changed, 24 insertions(+), 24 deletions(-) diff --git a/docs/ops/filesystems/oss.md b/docs/ops/filesystems/oss.md index 6b2be11cd5..c34bd19a41 100644 --- a/docs/ops/filesystems/oss.md +++ b/docs/ops/filesystems/oss.md @@ -52,7 +52,7 @@ env.setStateBackend(new FsStateBackend("oss:///")); ### Shaded Hadoop OSS file system -To use `flink-oss-fs-hadoop,` copy the respective JAR file from the `opt` directory to a directory in `plugins` directory of your Flink distribution before starting Flink, e.g. +To use `flink-oss-fs-hadoop`, copy the respective JAR file from the `opt` directory to a directory in `plugins` directory of your Flink distribution before starting Flink, e.g. {% highlight bash %} mkdir ./plugins/oss-fs-hadoop @@ -82,7 +82,7 @@ An alternative `CredentialsProvider` can also be configured in the `flink-conf.y # Read Credentials from OSS_ACCESS_KEY_ID and OSS_ACCESS_KEY_SECRET fs.oss.credentials.provider: com.aliyun.oss.common.auth.EnvironmentVariableCredentialsProvider {% endhighlight %} -Other credential providers can be found under https://github.com/aliyun/aliyun-oss-java-sdk/tree/master/src/main/java/com/aliyun/oss/common/auth. +Other credential providers can be found under [here](https://github.com/aliyun/aliyun-oss-java-sdk/tree/master/src/main/java/com/aliyun/oss/common/auth). diff --git a/docs/ops/filesystems/oss.zh.md b/docs/ops/filesystems/oss.zh.md index a93754662b..91bb1739aa 100644 --- a/docs/ops/filesystems/oss.zh.md +++ b/docs/ops/filesystems/oss.zh.md @@ -1,6 +1,6 @@ --- -title: "Aliyun 对象存储服务 (OSS)" -nav-title: Aliyun OSS +title: "阿里云对象存储服务 (OSS)" +nav-title: 阿里云 OSS nav-parent_id: filesystems nav-pos: 2 --- @@ -23,66 +23,66 @@ specific language governing permissions and limitations under the License. --> -## OSS: Object Storage Service +## OSS:对象存储服务 -[Aliyun Object Storage Service](https://www.aliyun.com/product/oss) (Aliyun OSS) is widely used, particularly popular among China’s cloud users, and it provides cloud object storage for a variety of use cases. -You can use OSS with Flink for **reading** and **writing data** as well in conjunction with the [streaming **state backends**]({{ site.baseurl }}/ops/state/state_backends.html) +[阿里云对象存储服务](https://www.aliyun.com/product/oss) (Aliyun OSS) 使用广泛,尤其在中国云用户中十分流行,能提供多种应用场景下的云对象存储。OSS 可与 Flink 一起使用以读取与存储数据,以及与[流 State Backend]({{ site.baseurl }}/zh/ops/state/state_backends.html) 结合使用。 * This will be replaced by the TOC {:toc} -You can use OSS objects like regular files by specifying paths in the following format: +通过以下格式指定路径,OSS 对象可类似于普通文件使用: {% highlight plain %} oss:/// {% endhighlight %} -Below shows how to use OSS in a Flink job: +以下代码展示了如何在 Flink 作业中使用 OSS: {% highlight java %} -// Read from OSS bucket +// 读取 OSS bucket env.readTextFile("oss:///"); -// Write to OSS bucket +// 写入 OSS bucket stream.writeAsText("oss:///") -// Use OSS as FsStatebackend +// 将 OSS 用作 FsStatebackend env.setStateBackend(new FsStateBackend("oss:///")); {% endhighlight %} -### Shaded Hadoop OSS file system +### Shaded Hadoop OSS 文件系统 -To use `flink-oss-fs-hadoop,` copy the respective JAR file from the `opt` directory to a directory in `plugins` directory of your Flink distribution before starting Flink, e.g. +为使用 `flink-oss-fs-hadoop`,在启动 Flink 之前,将对应的 JAR 文件从 `opt` 目录复制到 Flink 发行版中的 `plugin` 目录下的一个文件夹中,例如: {% highlight bash %} mkdir ./plugins/oss-fs-hadoop cp ./opt/flink-oss-fs-hadoop-{{ site.version }}.jar ./plugins/oss-fs-hadoop/ {% endhighlight %} -`flink-oss-fs-hadoop` registers default FileSystem wrappers for URIs with the *oss://* scheme. +`flink-oss-fs-hadoop` 为使用 *oss://* scheme 的 URI 注册了默认的文件系统包装器。 -#### Configurations setup +#### 配置设置 -After setting up the OSS FileSystem wrapper, you need to add some configurations to make sure that Flink is allowed to access your OSS buckets. +在设置好 OSS 文件系统包装器之后,需要添加一些配置以保证 Flink 有权限访问 OSS buckets。 -To allow for easy adoption, you can use the same configuration keys in `flink-conf.yaml` as in Hadoop's `core-site.xml` +为了简单使用,可直接在 `flink-conf.yaml` 中使用与 Hadoop `core-site.xml` 相同的配置关键字。 -You can see the configuration keys in the [Hadoop OSS documentation](http://hadoop.apache.org/docs/current/hadoop-aliyun/tools/hadoop-aliyun/index.html). +可在 [Hadoop OSS 文档](http://hadoop.apache.org/docs/current/hadoop-aliyun/tools/hadoop-aliyun/index.html) 中查看配置关键字。 -There are some required configurations that must be added to `flink-conf.yaml` (**Other configurations defined in Hadoop OSS documentation are advanced configurations which used by performance tuning**): +一些配置必须添加至 `flink-conf.yaml` (**在 Hadoop OSS 文档中定义的其它配置为用作性能调优的高级配置**): {% highlight yaml %} -fs.oss.endpoint: Aliyun OSS endpoint to connect to +fs.oss.endpoint: 连接的 Aliyun OSS endpoint fs.oss.accessKeyId: Aliyun access key ID fs.oss.accessKeySecret: Aliyun access key secret {% endhighlight %} -An alternative `CredentialsProvider` can also be configured in the `flink-conf.yaml`, e.g. +备选的 `CredentialsProvider` 也可在 `flink-conf.yaml` 中配置,例如: {% highlight yaml %} -# Read Credentials from OSS_ACCESS_KEY_ID and OSS_ACCESS_KEY_SECRET +# 从 OSS_ACCESS_KEY_ID 和 OSS_ACCESS_KEY_SECRET 读取凭据 (Credentials) fs.oss.credentials.provider: com.aliyun.oss.common.auth.EnvironmentVariableCredentialsProvider {% endhighlight %} -Other credential providers can be found under https://github.com/aliyun/aliyun-oss-java-sdk/tree/master/src/main/java/com/aliyun/oss/common/auth. + +其余的凭据提供者(credential providers)可在[这里](https://github.com/aliyun/aliyun-oss-java-sdk/tree/master/src/main/java/com/aliyun/oss/common/auth)中找到。 -- Gitee From 5781c67a4c0c2a097c8c6659c155ba94948062d1 Mon Sep 17 00:00:00 2001 From: gauss Date: Sun, 23 Feb 2020 13:10:16 +0800 Subject: [PATCH 066/885] [FLINK-16089][docs-zh] Translate "Data Type" page of Table API & SQL into Chinese This closes #11190 --- docs/dev/table/types.zh.md | 675 ++++++++++++++++--------------------- 1 file changed, 290 insertions(+), 385 deletions(-) diff --git a/docs/dev/table/types.zh.md b/docs/dev/table/types.zh.md index 4dc42af427..252ff021bf 100644 --- a/docs/dev/table/types.zh.md +++ b/docs/dev/table/types.zh.md @@ -1,5 +1,5 @@ --- -title: "Data Types" +title: "数据类型" nav-parent_id: tableapi nav-pos: 20 --- @@ -22,62 +22,47 @@ specific language governing permissions and limitations under the License. --> -Due to historical reasons, before Flink 1.9, Flink's Table & SQL API data types were -tightly coupled to Flink's `TypeInformation`. `TypeInformation` is used in the DataStream -and DataSet API and is sufficient to describe all information needed to serialize and -deserialize JVM-based objects in a distributed setting. +由于历史原因,在 Flink 1.9 之前,Flink Table & SQL API 的数据类型与 Flink 的 `TypeInformation` 耦合紧密。`TypeInformation` 在 DataStream 和 DataSet API 中被使用,并且足以用来用于描述分布式环境中 JVM 对象的序列化和反序列化操作所需的全部信息。 -However, `TypeInformation` was not designed to represent logical types independent of -an actual JVM class. In the past, it was difficult to map SQL standard types to this -abstraction. Furthermore, some types were not SQL-compliant and introduced without a -bigger picture in mind. +然而,`TypeInformation` 并不是为独立于 JVM class 的逻辑类型而设计的。之前很难将 SQL 的标准类型映射到 `TypeInformation` 抽象。此外,有一些类型并不是兼容 SQL 的并且引入的时候没有长远规划过。 -Starting with Flink 1.9, the Table & SQL API will receive a new type system that serves as a long-term -solution for API stability and standard compliance. +从 Flink 1.9 开始,Table & SQL API 开始启用一种新的类型系统作为长期解决方案,用来保持 API 稳定性和 SQL 标准的兼容性。 -Reworking the type system is a major effort that touches almost all user-facing interfaces. Therefore, its -introduction spans multiple releases, and the community aims to finish this effort by Flink 1.10. +重新设计类型系统是一项涉及几乎所有的面向用户接口的重大工作。因此,它的引入跨越多个版本,社区的目标是在 Flink 1.10 完成这项工作。 -Due to the simultaneous addition of a new planner for table programs (see [FLINK-11439](https://issues.apache.org/jira/browse/FLINK-11439)), -not every combination of planner and data type is supported. Furthermore, planners might not support every -data type with the desired precision or parameter. +同时由于为 Table 编程添加了新的 Planner 详见([FLINK-11439](https://issues.apache.org/jira/browse/FLINK-11439)), 并不是每种 Planner 都支持所有的数据类型。此外,Planner 对于数据类型的精度和参数化支持也可能是不完整的。 -Attention Please see the planner compatibility table and limitations -section before using a data type. +注意 在使用数据类型之前请参阅 Planner 的兼容性表和局限性章节。 * This will be replaced by the TOC {:toc} -Data Type +数据类型 --------- -A *data type* describes the logical type of a value in the table ecosystem. It can be used to declare input and/or -output types of operations. +*数据类型* 描述 Table 编程环境中的值的逻辑类型。它可以被用来声明操作的输入输出类型。 -Flink's data types are similar to the SQL standard's *data type* terminology but also contain information -about the nullability of a value for efficient handling of scalar expressions. +Flink 的数据类型和 SQL 标准的 *数据类型* 术语类似,但也包含了可空属性,可以被用于标量表达式(scalar expression)的优化。 -Examples of data types are: +数据类型的示例: - `INT` - `INT NOT NULL` - `INTERVAL DAY TO SECOND(3)` - `ROW, myOtherField TIMESTAMP(3)>` -A list of all pre-defined data types can be found [below](#list-of-data-types). +全部的预定义数据类型见[下面](#数据类型列表)列表。 -### Data Types in the Table API +### Table API 的数据类型 -Users of the JVM-based API work with instances of `org.apache.flink.table.types.DataType` within the Table API or when -defining connectors, catalogs, or user-defined functions. +JVM API 的用户可以在 Table API 中使用 `org.apache.flink.table.types.DataType` 的实例,以及定义连接器(Connector)、Catalog 或者用户自定义函数(User-Defined Function)。 -A `DataType` instance has two responsibilities: -- **Declaration of a logical type** which does not imply a concrete physical representation for transmission -or storage but defines the boundaries between JVM-based languages and the table ecosystem. -- *Optional:* **Giving hints about the physical representation of data to the planner** which is useful at the edges to other APIs . +一个 `DataType` 实例有两个作用: +- **逻辑类型的声明**,它不表达具体物理类型的存储和转换,但是定义了基于 JVM 的语言和 Table 编程环境之间的边界。 +- *可选的:* **向 Planner 提供有关数据的物理表示的提示**,这对于边界 API 很有用。 -For JVM-based languages, all pre-defined data types are available in `org.apache.flink.table.api.DataTypes`. +对于基于 JVM 的语言,所有预定义的数据类型都在 `org.apache.flink.table.api.DataTypes` 里提供。 -It is recommended to add a star import to your table programs for having a fluent API: +建议使用星号将全部的 API 导入到 Table 程序中以便于使用:
    @@ -99,71 +84,61 @@ val t: DataType = INTERVAL(DAY(), SECOND(3));
    -#### Physical Hints +#### 物理提示 -Physical hints are required at the edges of the table ecosystem where the SQL-based type system ends and -programming-specific data types are required. Hints indicate the data format that an implementation -expects. +在 Table 编程环境中,基于 SQL 的类型系统与程序指定的数据类型之间需要物理提示。该提示指出了实现预期的数据格式。 -For example, a data source could express that it produces values for logical `TIMESTAMP`s using a `java.sql.Timestamp` class -instead of using `java.time.LocalDateTime` which would be the default. With this information, the runtime is able to convert -the produced class into its internal data format. In return, a data sink can declare the data format it consumes from the runtime. +例如,Data Source 能够使用类 `java.sql.Timestamp` 来表达逻辑上的 `TIMESTAMP` 产生的值,而不是使用缺省的 `java.time.LocalDateTime`。有了这些信息,运行时就能够将产生的类转换为其内部数据格式。反过来,Data Sink 可以声明它从运行时消费的数据格式。 -Here are some examples of how to declare a bridging conversion class: +下面是一些如何声明桥接转换类的示例:
    {% highlight java %} -// tell the runtime to not produce or consume java.time.LocalDateTime instances -// but java.sql.Timestamp +// 告诉运行时不要产生或者消费 java.time.LocalDateTime 实例 +// 而是使用 java.sql.Timestamp DataType t = DataTypes.TIMESTAMP(3).bridgedTo(java.sql.Timestamp.class); -// tell the runtime to not produce or consume boxed integer arrays -// but primitive int arrays +// 告诉运行时不要产生或者消费装箱的整数数组 +// 而是使用基本数据类型的整数数组 DataType t = DataTypes.ARRAY(DataTypes.INT().notNull()).bridgedTo(int[].class); {% endhighlight %}
    {% highlight scala %} -// tell the runtime to not produce or consume java.time.LocalDateTime instances -// but java.sql.Timestamp +// 告诉运行时不要产生或者消费 java.time.LocalDateTime 实例 +// 而是使用 java.sql.Timestamp val t: DataType = DataTypes.TIMESTAMP(3).bridgedTo(classOf[java.sql.Timestamp]); -// tell the runtime to not produce or consume boxed integer arrays -// but primitive int arrays +// 告诉运行时不要产生或者消费装箱的整数数组 +// 而是使用基本数据类型的整数数组 val t: DataType = DataTypes.ARRAY(DataTypes.INT().notNull()).bridgedTo(classOf[Array[Int]]); {% endhighlight %}
    -Attention Please note that physical hints are usually only required if the -API is extended. Users of predefined sources/sinks/functions do not need to define such hints. Hints within -a table program (e.g. `field.cast(TIMESTAMP(3).bridgedTo(Timestamp.class))`) are ignored. +注意 请注意,通常只有在扩展 API 时才需要物理提示。 +预定义的 Source、Sink、Function 的用户不需要定义这样的提示。在 Table 编程中(例如 `field.cast(TIMESTAMP(3).bridgedTo(Timestamp.class))`)这些提示将被忽略。 -Planner Compatibility +Planner 兼容性 --------------------- -As mentioned in the introduction, reworking the type system will span multiple releases, and the support of each data -type depends on the used planner. This section aims to summarize the most significant differences. +正如简介里提到的,重新开发类型系统将跨越多个版本,每个数据类型的支持取决于使用的 Planner。本节旨在总结最重要的差异。 -### Old Planner +### 旧的 Planner -Flink's old planner, introduced before Flink 1.9, primarily supports type information. It has only limited -support for data types. It is possible to declare data types that can be translated into type information such that the -old planner understands them. +Flink 1.9 之前引入的旧的 Planner 主要支持类型信息(Type Information),它只对数据类型提供有限的支持,可以声明能够转换为类型信息的数据类型,以便旧的 Planner 能够理解它们。 -The following table summarizes the difference between data type and type information. Most simple types, as well as the -row type remain the same. Time types, array types, and the decimal type need special attention. Other hints as the ones -mentioned are not allowed. +下表总结了数据类型和类型信息之间的区别。大多数简单类型以及 Row 类型保持不变。Time 类型、 Array 类型和 Decimal 类型需要特别注意。不允许使用其他的类型提示。 -For the *Type Information* column the table omits the prefix `org.apache.flink.table.api.Types`. +对于 *类型信息* 列,该表省略了前缀 `org.apache.flink.table.api.Types`。 -For the *Data Type Representation* column the table omits the prefix `org.apache.flink.table.api.DataTypes`. +对于 *数据类型表示* 列,该表省略了前缀 `org.apache.flink.table.api.DataTypes`。 -| Type Information | Java Expression String | Data Type Representation | Remarks for Data Type | +| 类型信息 | Java 表达式字符串 | 数据类型表示 | 数据类型备注 | |:-----------------|:-----------------------|:-------------------------|:----------------------| | `STRING()` | `STRING` | `STRING()` | | | `BOOLEAN()` | `BOOLEAN` | `BOOLEAN()` | | @@ -174,35 +149,33 @@ For the *Data Type Representation* column the table omits the prefix `org.apache | `FLOAT()` | `FLOAT` | `FLOAT()` | | | `DOUBLE()` | `DOUBLE` | `DOUBLE()` | | | `ROW(...)` | `ROW<...>` | `ROW(...)` | | -| `BIG_DEC()` | `DECIMAL` | [`DECIMAL()`] | Not a 1:1 mapping as precision and scale are ignored and Java's variable precision and scale are used. | +| `BIG_DEC()` | `DECIMAL` | [`DECIMAL()`] | 不是 1:1 的映射,因为精度和小数位被忽略,Java 的可变精度和小数位被使用。 | | `SQL_DATE()` | `SQL_DATE` | `DATE()`
    `.bridgedTo(java.sql.Date.class)` | | | `SQL_TIME()` | `SQL_TIME` | `TIME(0)`
    `.bridgedTo(java.sql.Time.class)` | | | `SQL_TIMESTAMP()` | `SQL_TIMESTAMP` | `TIMESTAMP(3)`
    `.bridgedTo(java.sql.Timestamp.class)` | | | `INTERVAL_MONTHS()` | `INTERVAL_MONTHS` | `INTERVAL(MONTH())`
    `.bridgedTo(Integer.class)` | | | `INTERVAL_MILLIS()` | `INTERVAL_MILLIS` | `INTERVAL(DataTypes.SECOND(3))`
    `.bridgedTo(Long.class)` | | -| `PRIMITIVE_ARRAY(...)` | `PRIMITIVE_ARRAY<...>` | `ARRAY(DATATYPE.notNull()`
    `.bridgedTo(PRIMITIVE.class))` | Applies to all JVM primitive types except for `byte`. | +| `PRIMITIVE_ARRAY(...)` | `PRIMITIVE_ARRAY<...>` | `ARRAY(DATATYPE.notNull()`
    `.bridgedTo(PRIMITIVE.class))` | 应用于除 `byte` 外的全部 JVM 基本数据类型。 | | `PRIMITIVE_ARRAY(BYTE())` | `PRIMITIVE_ARRAY` | `BYTES()` | | | `OBJECT_ARRAY(...)` | `OBJECT_ARRAY<...>` | `ARRAY(`
    `DATATYPE.bridgedTo(OBJECT.class))` | | | `MULTISET(...)` | | `MULTISET(...)` | | | `MAP(..., ...)` | `MAP<...,...>` | `MAP(...)` | | -| other generic types | | `RAW(...)` | | +| 其他通用类型 | | `RAW(...)` | | -Attention If there is a problem with the new type system. Users -can fallback to type information defined in `org.apache.flink.table.api.Types` at any time. +注意 如果对于新的类型系统有任何疑问,用户可以随时切换到 `org.apache.flink.table.api.Types` 中定义的 type information。 -### New Blink Planner +### 新的 Blink Planner -The new Blink planner supports all of types of the old planner. This includes in particular -the listed Java expression strings and type information. +新的 Blink Planner 支持旧的 Planner 的全部类型,尤其包括列出的 Java 表达式字符串和类型信息。 -The following data types are supported: +支持以下数据类型: -| Data Type | Remarks for Data Type | +| 数据类型 | 数据类型的备注 | |:----------|:----------------------| -| `STRING` | `CHAR` and `VARCHAR` are not supported yet. | +| `STRING` | `CHAR` 和 `VARCHAR` 暂不支持。 | | `BOOLEAN` | | -| `BYTES` | `BINARY` and `VARBINARY` are not supported yet. | -| `DECIMAL` | Supports fixed precision and scale. | +| `BYTES` | `BINARY` 和 `VARBINARY` 暂不支持。 | +| `DECIMAL` | 支持固定精度和小数位数。 | | `TINYINT` | | | `SMALLINT` | | | `INTEGER` | | @@ -210,40 +183,37 @@ The following data types are supported: | `FLOAT` | | | `DOUBLE` | | | `DATE` | | -| `TIME` | Supports only a precision of `0`. | -| `TIMESTAMP` | Supports only a precision of `3`. | -| `TIMESTAMP WITH LOCAL TIME ZONE` | Supports only a precision of `3`. | -| `INTERVAL` | Supports only interval of `MONTH` and `SECOND(3)`. | +| `TIME` | 支持的精度仅为 `0`。 | +| `TIMESTAMP` | 支持的精度仅为 `3`。 | +| `TIMESTAMP WITH LOCAL TIME ZONE` | 支持的精度仅为 `3`。 | +| `INTERVAL` | 仅支持 `MONTH` 和 `SECOND(3)` 区间。 | | `ARRAY` | | | `MULTISET` | | | `MAP` | | | `ROW` | | | `RAW` | | -Limitations +局限性 ----------- -**Java Expression String**: Java expression strings in the Table API such as `table.select("field.cast(STRING)")` -have not been updated to the new type system yet. Use the string representations declared in -the [old planner section](#old-planner). +**Java 表达式字符串**:Table API 中的 Java 表达式字符串,例如 `table.select("field.cast(STRING)")`,尚未被更新到新的类型系统中,使用[旧的 Planner 章节](#旧的-planner)中声明的字符串来表示。 -**Connector Descriptors and SQL Client**: Descriptor string representations have not been updated to the new -type system yet. Use the string representation declared in the [Connect to External Systems section](./connect.html#type-strings) +**连接器描述符和 SQL 客户端**:描述符字符串的表示形式尚未更新到新的类型系统。使用在[连接到外部系统章节](./connect.html#type-strings)中声明的字符串表示。 -**User-defined Functions**: User-defined functions cannot declare a data type yet. +**用户自定义函数**:用户自定义函数尚不能声明数据类型。 -List of Data Types +数据类型列表 ------------------ -This section lists all pre-defined data types. For the JVM-based Table API those types are also available in `org.apache.flink.table.api.DataTypes`. +本节列出了所有预定义的数据类型。对于基于 JVM 的 Table API,这些类型也可以从 `org.apache.flink.table.api.DataTypes` 中找到。 -### Character Strings +### 字符串 #### `CHAR` -Data type of a fixed-length character string. +固定长度字符串的数据类型。 -**Declaration** +**声明**
    @@ -262,21 +232,20 @@ DataTypes.CHAR(n)
    -The type can be declared using `CHAR(n)` where `n` is the number of code points. `n` must have a value between `1` -and `2,147,483,647` (both inclusive). If no length is specified, `n` is equal to `1`. +此类型用 `CHAR(n)` 声明,其中 `n` 表示字符数量。`n` 的值必须在 `1` 和 `2,147,483,647` 之间(含边界值)。如果未指定长度,`n` 等于 `1`。 -**Bridging to JVM Types** +**JVM 类型** -| Java Type | Input | Output | Remarks | +| Java 类型 | 输入 | 输出 | 备注 | |:-------------------|:-----:|:------:|:------------------------| -|`java.lang.String` | X | X | *Default* | -|`byte[]` | X | X | Assumes UTF-8 encoding. | +|`java.lang.String` | X | X | *缺省* | +|`byte[]` | X | X | 假设使用 UTF-8 编码。 | #### `VARCHAR` / `STRING` -Data type of a variable-length character string. +可变长度字符串的数据类型。 -**Declaration** +**声明**
    @@ -299,25 +268,24 @@ DataTypes.STRING()
    -The type can be declared using `VARCHAR(n)` where `n` is the maximum number of code points. `n` must have a value -between `1` and `2,147,483,647` (both inclusive). If no length is specified, `n` is equal to `1`. +此类型用 `VARCHAR(n)` 声明,其中 `n` 表示最大的字符数量。`n` 的值必须在 `1` 和 `2,147,483,647` 之间(含边界值)。如果未指定长度,`n` 等于 `1`。 -`STRING` is a synonym for `VARCHAR(2147483647)`. +`STRING` 等价于 `VARCHAR(2147483647)`. -**Bridging to JVM Types** +**JVM 类型** -| Java Type | Input | Output | Remarks | +| Java 类型 | 输入 | 输出 | 备注 | |:-------------------|:-----:|:------:|:------------------------| -|`java.lang.String` | X | X | *Default* | -|`byte[]` | X | X | Assumes UTF-8 encoding. | +|`java.lang.String` | X | X | *缺省* | +|`byte[]` | X | X | 假设使用 UTF-8 编码。 | -### Binary Strings +### 二进制字符串 #### `BINARY` -Data type of a fixed-length binary string (=a sequence of bytes). +固定长度二进制字符串的数据类型(=字节序列)。 -**Declaration** +**声明**
    @@ -336,20 +304,19 @@ DataTypes.BINARY(n)
    -The type can be declared using `BINARY(n)` where `n` is the number of bytes. `n` must have a value -between `1` and `2,147,483,647` (both inclusive). If no length is specified, `n` is equal to `1`. +此类型用 `BINARY(n)` 声明,其中 `n` 是字节数量。`n` 的值必须在 `1` 和 `2,147,483,647` 之间(含边界值)。如果未指定长度,`n` 等于 `1`。 -**Bridging to JVM Types** +**JVM 类型** -| Java Type | Input | Output | Remarks | +| Java 类型 | 输入 | 输出 | 备注 | |:-------------------|:-----:|:------:|:------------------------| -|`byte[]` | X | X | *Default* | +|`byte[]` | X | X | *缺省* | #### `VARBINARY` / `BYTES` -Data type of a variable-length binary string (=a sequence of bytes). +可变长度二进制字符串的数据类型(=字节序列)。 -**Declaration** +**声明**
    @@ -372,25 +339,23 @@ DataTypes.BYTES()
    -The type can be declared using `VARBINARY(n)` where `n` is the maximum number of bytes. `n` must -have a value between `1` and `2,147,483,647` (both inclusive). If no length is specified, `n` is -equal to `1`. +此类型用 `VARBINARY(n)` 声明,其中 `n` 是最大的字节数量。`n` 的值必须在 `1` 和 `2,147,483,647` 之间(含边界值)。如果未指定长度,`n` 等于 `1`。 -`BYTES` is a synonym for `VARBINARY(2147483647)`. +`BYTES` 等价于 `VARBINARY(2147483647)`。 -**Bridging to JVM Types** +**JVM 类型** -| Java Type | Input | Output | Remarks | +| Java 类型 | 输入 | 输出 | 备注 | |:-------------------|:-----:|:------:|:------------------------| -|`byte[]` | X | X | *Default* | +|`byte[]` | X | X | *缺省* | -### Exact Numerics +### 精确数值 #### `DECIMAL` -Data type of a decimal number with fixed precision and scale. +精度和小数位数固定的十进制数字的数据类型。 -**Declaration** +**声明**
    @@ -418,25 +383,21 @@ DataTypes.DECIMAL(p, s)
    -The type can be declared using `DECIMAL(p, s)` where `p` is the number of digits in a -number (*precision*) and `s` is the number of digits to the right of the decimal point -in a number (*scale*). `p` must have a value between `1` and `38` (both inclusive). `s` -must have a value between `0` and `p` (both inclusive). The default value for `p` is 10. -The default value for `s` is `0`. +此类型用 `DECIMAL(p, s)` 声明,其中 `p` 是数字的位数(*精度*),`s` 是数字中小数点右边的位数(*尾数*)。`p` 的值必须介于 `1` 和 `38` 之间(含边界值)。`s` 的值必须介于 `0` 和 `p` 之间(含边界值)。其中 `p` 的缺省值是 `10`,`s` 的缺省值是 `0`。 -`NUMERIC(p, s)` and `DEC(p, s)` are synonyms for this type. +`NUMERIC(p, s)` 和 `DEC(p, s)` 都等价于这个类型。 -**Bridging to JVM Types** +**JVM 类型** -| Java Type | Input | Output | Remarks | +| Java 类型 | 输入 | 输出 | 备注 | |:----------------------|:-----:|:------:|:------------------------| -|`java.math.BigDecimal` | X | X | *Default* | +|`java.math.BigDecimal` | X | X | *缺省* | #### `TINYINT` -Data type of a 1-byte signed integer with values from `-128` to `127`. +1 字节有符号整数的数据类型,其值从 `-128` to `127`。 -**Declaration** +**声明**
    @@ -454,18 +415,18 @@ DataTypes.TINYINT()
    -**Bridging to JVM Types** +**JVM 类型** -| Java Type | Input | Output | Remarks | +| Java 类型 | 输入 | 输出 | 备注 | |:-------------------|:-----:|:------:|:---------------------------------------------| -|`java.lang.Byte` | X | X | *Default* | -|`byte` | X | (X) | Output only if type is not nullable. | +|`java.lang.Byte` | X | X | *缺省* | +|`byte` | X | (X) | 仅当类型不可为空时才输出。 | #### `SMALLINT` -Data type of a 2-byte signed integer with values from `-32,768` to `32,767`. +2 字节有符号整数的数据类型,其值从 `-32,768` 到 `32,767`。 -**Declaration** +**声明**
    @@ -483,18 +444,18 @@ DataTypes.SMALLINT()
    -**Bridging to JVM Types** +**JVM 类型** -| Java Type | Input | Output | Remarks | +| Java 类型 | 输入 | 输出 | 备注 | |:-------------------|:-----:|:------:|:---------------------------------------------| -|`java.lang.Short` | X | X | *Default* | -|`short` | X | (X) | Output only if type is not nullable. | +|`java.lang.Short` | X | X | *缺省* | +|`short` | X | (X) | 仅当类型不可为空时才输出。 | #### `INT` -Data type of a 4-byte signed integer with values from `-2,147,483,648` to `2,147,483,647`. +4 字节有符号整数的数据类型,其值从 `-2,147,483,648` 到 `2,147,483,647`。 -**Declaration** +**声明**
    @@ -514,21 +475,20 @@ DataTypes.INT()
    -`INTEGER` is a synonym for this type. +`INTEGER` 等价于此类型。 -**Bridging to JVM Types** +**JVM 类型** -| Java Type | Input | Output | Remarks | +| Java 类型 | 输入 | 输出 | 备注 | |:-------------------|:-----:|:------:|:---------------------------------------------| -|`java.lang.Integer` | X | X | *Default* | -|`int` | X | (X) | Output only if type is not nullable. | +|`java.lang.Integer` | X | X | *缺省* | +|`int` | X | (X) | 仅当类型不可为空时才输出。 | #### `BIGINT` -Data type of an 8-byte signed integer with values from `-9,223,372,036,854,775,808` to -`9,223,372,036,854,775,807`. +8 字节有符号整数的数据类型,其值从 `-9,223,372,036,854,775,808` 到 `9,223,372,036,854,775,807`。 -**Declaration** +**声明**
    @@ -546,22 +506,22 @@ DataTypes.BIGINT()
    -**Bridging to JVM Types** +**JVM 类型** -| Java Type | Input | Output | Remarks | +| Java 类型 | 输入 | 输出 | 备注 | |:-------------------|:-----:|:------:|:---------------------------------------------| -|`java.lang.Long` | X | X | *Default* | -|`long` | X | (X) | Output only if type is not nullable. | +|`java.lang.Long` | X | X | *缺省* | +|`long` | X | (X) | 仅当类型不可为空时才输出。 | -### Approximate Numerics +### 近似数值 #### `FLOAT` -Data type of a 4-byte single precision floating point number. +4 字节单精度浮点数的数据类型。 -Compared to the SQL standard, the type does not take parameters. +与 SQL 标准相比,该类型不带参数。 -**Declaration** +**声明**
    @@ -579,18 +539,18 @@ DataTypes.FLOAT()
    -**Bridging to JVM Types** +**JVM 类型** -| Java Type | Input | Output | Remarks | +| Java 类型 | 输入 | 输出 | 备注 | |:-------------------|:-----:|:------:|:---------------------------------------------| -|`java.lang.Float` | X | X | *Default* | -|`float` | X | (X) | Output only if type is not nullable. | +|`java.lang.Float` | X | X | *缺省* | +|`float` | X | (X) | 仅当类型不可为空时才输出。 | #### `DOUBLE` -Data type of an 8-byte double precision floating point number. +8 字节双精度浮点数的数据类型。 -**Declaration** +**声明**
    @@ -610,25 +570,24 @@ DataTypes.DOUBLE()
    -`DOUBLE PRECISION` is a synonym for this type. +`DOUBLE PRECISION` 等价于此类型。 -**Bridging to JVM Types** +**JVM 类型** -| Java Type | Input | Output | Remarks | +| Java 类型 | 输入 | 输出 | 备注 | |:-------------------|:-----:|:------:|:---------------------------------------------| -|`java.lang.Double` | X | X | *Default* | -|`double` | X | (X) | Output only if type is not nullable. | +|`java.lang.Double` | X | X | *缺省* | +|`double` | X | (X) | 仅当类型不可为空时才输出。 | -### Date and Time +### 日期和时间 #### `DATE` -Data type of a date consisting of `year-month-day` with values ranging from `0000-01-01` -to `9999-12-31`. +日期的数据类型由 `year-month-day` 组成,范围从 `0000-01-01` 到 `9999-12-31`。 -Compared to the SQL standard, the range starts at year `0000`. +与 SQL 标准相比,年的范围从 `0000` 开始。 -**Declaration** +**声明**
    @@ -646,25 +605,22 @@ DataTypes.DATE()
    -**Bridging to JVM Types** +**JVM 类型** -| Java Type | Input | Output | Remarks | +| Java 类型 | 输入 | 输出 | 备注 | |:---------------------|:-----:|:------:|:---------------------------------------------| -|`java.time.LocalDate` | X | X | *Default* | +|`java.time.LocalDate` | X | X | *缺省* | |`java.sql.Date` | X | X | | -|`java.lang.Integer` | X | X | Describes the number of days since epoch. | -|`int` | X | (X) | Describes the number of days since epoch.
    Output only if type is not nullable. | +|`java.lang.Integer` | X | X | 描述从 Epoch 算起的天数。 | +|`int` | X | (X) | 描述从 Epoch 算起的天数。
    仅当类型不可为空时才输出。 | #### `TIME` -Data type of a time *without* time zone consisting of `hour:minute:second[.fractional]` with -up to nanosecond precision and values ranging from `00:00:00.000000000` to -`23:59:59.999999999`. +*不带*时区的时间数据类型,由 `hour:minute:second[.fractional]` 组成,精度达到纳秒,范围从 `00:00:00.000000000` 到 `23:59:59.999999999`。 -Compared to the SQL standard, leap seconds (`23:59:60` and `23:59:61`) are not supported as -the semantics are closer to `java.time.LocalTime`. A time *with* time zone is not provided. +与 SQL 标准相比,不支持闰秒(`23:59:60` 和 `23:59:61`),语义上更接近于 `java.time.LocalTime`。没有提供*带有*时区的时间。 -**Declaration** +**声明**
    @@ -683,35 +639,28 @@ DataTypes.TIME(p)
    -The type can be declared using `TIME(p)` where `p` is the number of digits of fractional -seconds (*precision*). `p` must have a value between `0` and `9` (both inclusive). If no -precision is specified, `p` is equal to `0`. +此类型用 `TIME(p)` 声明,其中 `p` 是秒的小数部分的位数(*精度*)。`p` 的值必须介于 `0` 和 `9` 之间(含边界值)。如果未指定精度,则 `p` 等于 `0`。 -**Bridging to JVM Types** +**JVM 类型** -| Java Type | Input | Output | Remarks | +| Java 类型 |输入 |输出 |备注 | |:---------------------|:-----:|:------:|:----------------------------------------------------| -|`java.time.LocalTime` | X | X | *Default* | +|`java.time.LocalTime` | X | X | *缺省* | |`java.sql.Time` | X | X | | -|`java.lang.Integer` | X | X | Describes the number of milliseconds of the day. | -|`int` | X | (X) | Describes the number of milliseconds of the day.
    Output only if type is not nullable. | -|`java.lang.Long` | X | X | Describes the number of nanoseconds of the day. | -|`long` | X | (X) | Describes the number of nanoseconds of the day.
    Output only if type is not nullable. | +|`java.lang.Integer` | X | X | 描述自当天以来的毫秒数。 | +|`int` | X | (X) | 描述自当天以来的毫秒数。
    仅当类型不可为空时才输出。 | +|`java.lang.Long` | X | X | 描述自当天以来的纳秒数。 | +|`long` | X | (X) | 描述自当天以来的纳秒数。
    仅当类型不可为空时才输出。 | #### `TIMESTAMP` -Data type of a timestamp *without* time zone consisting of `year-month-day hour:minute:second[.fractional]` -with up to nanosecond precision and values ranging from `0000-01-01 00:00:00.000000000` to -`9999-12-31 23:59:59.999999999`. +*不带*时区的时间戳数据类型,由 `year-month-day hour:minute:second[.fractional]` 组成,精度达到纳秒,范围从 `0000-01-01 00:00:00.000000000` 到 `9999-12-31 23:59:59.999999999`。 -Compared to the SQL standard, leap seconds (`23:59:60` and `23:59:61`) are not supported as -the semantics are closer to `java.time.LocalDateTime`. +与 SQL 标准相比,不支持闰秒(`23:59:60` 和 `23:59:61`),语义上更接近于 `java.time.LocalDateTime`。 -A conversion from and to `BIGINT` (a JVM `long` type) is not supported as this would imply a time -zone. However, this type is time zone free. For more `java.time.Instant`-like semantics use -`TIMESTAMP WITH LOCAL TIME ZONE`. +不支持和 `BIGINT`(JVM `long` 类型)互相转换,因为这意味着有时区,然而此类型是无时区的。对于语义上更接近于 `java.time.Instant` 的需求请使用 `TIMESTAMP WITH LOCAL TIME ZONE`。 -**Declaration** +**声明**
    @@ -733,33 +682,27 @@ DataTypes.TIMESTAMP(p)
    -The type can be declared using `TIMESTAMP(p)` where `p` is the number of digits of fractional -seconds (*precision*). `p` must have a value between `0` and `9` (both inclusive). If no precision -is specified, `p` is equal to `6`. +此类型用 `TIMESTAMP(p)` 声明,其中 `p` 是秒的小数部分的位数(*精度*)。`p` 的值必须介于 `0` 和 `9` 之间(含边界值)。如果未指定精度,则 `p` 等于 `6`。 -`TIMESTAMP(p) WITHOUT TIME ZONE` is a synonym for this type. +`TIMESTAMP(p) WITHOUT TIME ZONE` 等价于此类型。 -**Bridging to JVM Types** +**JVM 类型** -| Java Type | Input | Output | Remarks | +| Java 类型 | 输入 | 输出 | 备注 | |:-------------------------|:-----:|:------:|:----------------------------------------------------| -|`java.time.LocalDateTime` | X | X | *Default* | +|`java.time.LocalDateTime` | X | X | *缺省* | |`java.sql.Timestamp` | X | X | | #### `TIMESTAMP WITH TIME ZONE` -Data type of a timestamp *with* time zone consisting of `year-month-day hour:minute:second[.fractional] zone` -with up to nanosecond precision and values ranging from `0000-01-01 00:00:00.000000000 +14:59` to -`9999-12-31 23:59:59.999999999 -14:59`. +*带有*时区的时间戳数据类型,由 `year-month-day hour:minute:second[.fractional] zone` 组成,精度达到纳秒,范围从 `0000-01-01 00:00:00.000000000 +14:59` 到 +`9999-12-31 23:59:59.999999999 -14:59`。 -Compared to the SQL standard, leap seconds (`23:59:60` and `23:59:61`) are not supported as the semantics -are closer to `java.time.OffsetDateTime`. +与 SQL 标准相比,不支持闰秒(`23:59:60` 和 `23:59:61`),语义上更接近于 `java.time.OffsetDateTime`。 -Compared to `TIMESTAMP WITH LOCAL TIME ZONE`, the time zone offset information is physically -stored in every datum. It is used individually for every computation, visualization, or communication -to external systems. +与 `TIMESTAMP WITH LOCAL TIME ZONE` 相比,时区偏移信息物理存储在每个数据中。它单独用于每次计算、可视化或者与外部系统的通信。 -**Declaration** +**声明**
    @@ -778,34 +721,27 @@ DataTypes.TIMESTAMP_WITH_TIME_ZONE(p)
    -The type can be declared using `TIMESTAMP(p) WITH TIME ZONE` where `p` is the number of digits of -fractional seconds (*precision*). `p` must have a value between `0` and `9` (both inclusive). If no -precision is specified, `p` is equal to `6`. +此类型用 `TIMESTAMP(p) WITH TIME ZONE` 声明,其中 `p` 是秒的小数部分的位数(*精度*)。`p` 的值必须介于 `0` 和 `9` 之间(含边界值)。如果未指定精度,则 `p` 等于 `6`。 -**Bridging to JVM Types** +**JVM 类型** -| Java Type | Input | Output | Remarks | +| Java 类型 | 输入 | 输出 | 备注 | |:--------------------------|:-----:|:------:|:---------------------| -|`java.time.OffsetDateTime` | X | X | *Default* | -|`java.time.ZonedDateTime` | X | | Ignores the zone ID. | +|`java.time.OffsetDateTime` | X | X | *缺省* | +|`java.time.ZonedDateTime` | X | | 忽略时区 ID。 | #### `TIMESTAMP WITH LOCAL TIME ZONE` -Data type of a timestamp *with local* time zone consisting of `year-month-day hour:minute:second[.fractional] zone` -with up to nanosecond precision and values ranging from `0000-01-01 00:00:00.000000000 +14:59` to -`9999-12-31 23:59:59.999999999 -14:59`. +*带有本地*时区的时间戳数据类型,由 `year-month-day hour:minute:second[.fractional] zone` 组成,精度达到纳秒,范围从 `0000-01-01 00:00:00.000000000 +14:59` 到 +`9999-12-31 23:59:59.999999999 -14:59`。 -Leap seconds (`23:59:60` and `23:59:61`) are not supported as the semantics are closer to `java.time.OffsetDateTime`. +不支持闰秒(`23:59:60` 和 `23:59:61`),语义上更接近于 `java.time.OffsetDateTime`。 -Compared to `TIMESTAMP WITH TIME ZONE`, the time zone offset information is not stored physically -in every datum. Instead, the type assumes `java.time.Instant` semantics in UTC time zone at -the edges of the table ecosystem. Every datum is interpreted in the local time zone configured in -the current session for computation and visualization. +与 `TIMESTAMP WITH TIME ZONE` 相比,时区偏移信息并非物理存储在每个数据中。相反,此类型在 Table 编程环境的 UTC 时区中采用 `java.time.Instant` 语义。每个数据都在当前会话中配置的本地时区中进行解释,以便用于计算和可视化。 -This type fills the gap between time zone free and time zone mandatory timestamp types by allowing -the interpretation of UTC timestamps according to the configured session time zone. +此类型允许根据配置的会话时区来解释 UTC 时间戳,从而填补了时区无关和时区相关的时间戳类型之间的鸿沟。 -**Declaration** +**声明**
    @@ -824,37 +760,32 @@ DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(p)
    -The type can be declared using `TIMESTAMP(p) WITH LOCAL TIME ZONE` where `p` is the number -of digits of fractional seconds (*precision*). `p` must have a value between `0` and `9` -(both inclusive). If no precision is specified, `p` is equal to `6`. +此类型用 `TIMESTAMP(p) WITH LOCAL TIME ZONE` 声明,其中 `p` 是秒的小数部分的位数(*精度*)。`p` 的值必须介于 `0` 和 `9` 之间(含边界值)。如果未指定精度,则 `p` 等于 `6`。 -**Bridging to JVM Types** +**JVM 类型** -| Java Type | Input | Output | Remarks | +| Java 类型 |输入 |输出 |备注 | |:-------------------|:-----:|:------:|:--------------------------------------------------| -|`java.time.Instant` | X | X | *Default* | -|`java.lang.Integer` | X | X | Describes the number of seconds since epoch. | -|`int` | X | (X) | Describes the number of seconds since epoch.
    Output only if type is not nullable. | -|`java.lang.Long` | X | X | Describes the number of milliseconds since epoch. | -|`long` | X | (X) | Describes the number of milliseconds since epoch.
    Output only if type is not nullable. | +|`java.time.Instant` | X | X | *缺省* | +|`java.lang.Integer` | X | X | 描述从 Epoch 算起的秒数。 | +|`int` | X | (X) | 描述从 Epoch 算起的秒数。
    仅当类型不可为空时才输出。 | +|`java.lang.Long` | X | X | 描述从 Epoch 算起的毫秒数。 | +|`long` | X | (X) | 描述从 Epoch 算起的毫秒数。
    仅当类型不可为空时才输出。 | #### `INTERVAL YEAR TO MONTH` -Data type for a group of year-month interval types. +一组 Year-Month Interval 数据类型。 -The type must be parameterized to one of the following resolutions: -- interval of years, -- interval of years to months, -- or interval of months. +此类型必被参数化为以下情况中的一种: +- Year 时间间隔、 +- Year-Month 时间间隔、 +- Month 时间间隔。 -An interval of year-month consists of `+years-months` with values ranging from `-9999-11` to -`+9999-11`. +Year-Month Interval 由 `+years-months` 组成,其范围从 `-9999-11` 到 `+9999-11`。 -The value representation is the same for all types of resolutions. For example, an interval -of months of 50 is always represented in an interval-of-years-to-months format (with default -year precision): `+04-02`. +所有类型的表达能力均相同。例如,Month 时间间隔下的 `50` 等价于 Year-Month 时间间隔(缺省年份精度)下的 `+04-02`。 -**Declaration** +**声明**
    @@ -878,40 +809,37 @@ DataTypes.INTERVAL(DataTypes.MONTH())
    -The type can be declared using the above combinations where `p` is the number of digits of years -(*year precision*). `p` must have a value between `1` and `4` (both inclusive). If no year precision -is specified, `p` is equal to `2`. +可以使用以上组合来声明类型,其中 `p` 是年数(*年精度*)的位数。`p` 的值必须介于 `1` 和 `4` 之间(含边界值)。如果未指定年精度,`p` 则等于 `2`。 -**Bridging to JVM Types** +**JVM 类型** -| Java Type | Input | Output | Remarks | +| Java 类型 | 输入 | 输出 | 备注 | |:-------------------|:-----:|:------:|:-----------------------------------| -|`java.time.Period` | X | X | Ignores the `days` part. *Default* | -|`java.lang.Integer` | X | X | Describes the number of months. | -|`int` | X | (X) | Describes the number of months.
    Output only if type is not nullable. | +|`java.time.Period` | X | X | 忽略 `days` 部分。 *缺省* | +|`java.lang.Integer` | X | X | 描述月的数量。 | +|`int` | X | (X) | 描述月的数量。
    仅当类型不可为空时才输出。 | #### `INTERVAL DAY TO MONTH` -Data type for a group of day-time interval types. +一组 Day-Time Interval 数据类型。 -The type must be parameterized to one of the following resolutions with up to nanosecond precision: -- interval of days, -- interval of days to hours, -- interval of days to minutes, -- interval of days to seconds, -- interval of hours, -- interval of hours to minutes, -- interval of hours to seconds, -- interval of minutes, -- interval of minutes to seconds, -- or interval of seconds. +此类型达到纳秒精度,必被参数化为以下情况中的一种: +- Day 时间间隔、 +- Day-Hour 时间间隔、 +- Day-Minute 时间间隔、 +- Day-Second 时间间隔、 +- Hour 时间间隔、 +- Hour-Minute 时间间隔、 +- Hour-Second 时间间隔、 +- Minute 时间间隔、 +- Minute-Second 时间间隔、 +- Second 时间间隔。 -An interval of day-time consists of `+days hours:months:seconds.fractional` with values ranging from -`-999999 23:59:59.999999999` to `+999999 23:59:59.999999999`. The value representation is the same -for all types of resolutions. For example, an interval of seconds of 70 is always represented in -an interval-of-days-to-seconds format (with default precisions): `+00 00:01:10.000000`. +Day-Time 时间间隔由 `+days hours:months:seconds.fractional` 组成,其范围从 `-999999 23:59:59.999999999` 到 `+999999 23:59:59.999999999`。 -**Declaration** +所有类型的表达能力均相同。例如,Second 时间间隔下的 `70` 等价于 Day-Second 时间间隔(缺省精度)下的 `+00 00:01:10.000000`。 + +**声明**
    @@ -951,30 +879,25 @@ DataTypes.INTERVAL(DataTypes.SECOND(p2))
    -The type can be declared using the above combinations where `p1` is the number of digits of days -(*day precision*) and `p2` is the number of digits of fractional seconds (*fractional precision*). -`p1` must have a value between `1` and `6` (both inclusive). `p2` must have a value between `0` -and `9` (both inclusive). If no `p1` is specified, it is equal to `2` by default. If no `p2` is -specified, it is equal to `6` by default. +可以使用以上组合来声明类型,其中 `p1` 是天数(*天精度*)的位数,`p2` 是秒的小数部分的位数(*小数精度*)。`p1` 的值必须介于 `1` 和之间 `6`(含边界值),`p2` 的值必须介于 `0` 和之间 `9`(含边界值)。如果 `p1` 未指定值,则缺省等于 `2`,如果 `p2` 未指定值,则缺省等于 `6`。 -**Bridging to JVM Types** +**JVM 类型** -| Java Type | Input | Output | Remarks | +| Java 类型 | 输入 | 输出 | 备注 | |:--------------------|:-----:|:------:|:--------------------------------------| -|`java.time.Duration` | X | X | *Default* | -|`java.lang.Long` | X | X | Describes the number of milliseconds. | -|`long` | X | (X) | Describes the number of milliseconds.
    Output only if type is not nullable. | +|`java.time.Duration` | X | X | *缺省* | +|`java.lang.Long` | X | X | 描述毫秒数。 | +|`long` | X | (X) | 描述毫秒数。
    仅当类型不可为空时才输出。 | -### Constructured Data Types +### 结构化的数据类型 #### `ARRAY` -Data type of an array of elements with same subtype. +具有相同子类型元素的数组的数据类型。 -Compared to the SQL standard, the maximum cardinality of an array cannot be specified but is -fixed at `2,147,483,647`. Also, any valid type is supported as a subtype. +与 SQL 标准相比,无法指定数组的最大长度,而是被固定为 `2,147,483,647`。另外,任何有效类型都可以作为子类型。 -**Declaration** +**声明**
    @@ -993,28 +916,25 @@ DataTypes.ARRAY(t)
    -The type can be declared using `ARRAY` where `t` is the data type of the contained -elements. +此类型用 `ARRAY` 声明,其中 `t` 是所包含元素的数据类型。 -`t ARRAY` is a synonym for being closer to the SQL standard. For example, `INT ARRAY` is -equivalent to `ARRAY`. +`t ARRAY` 接近等价于 SQL 标准。例如,`INT ARRAY` 等价于 `ARRAY`。 -**Bridging to JVM Types** +**JVM 类型** -| Java Type | Input | Output | Remarks | +| Java 类型 | 输入 | 输出 | 备注 | |:----------|:-----:|:------:|:----------------------------------| -|*t*`[]` | (X) | (X) | Depends on the subtype. *Default* | +|*t*`[]` | (X) | (X) | 依赖于子类型。 *缺省* | #### `MAP` -Data type of an associative array that maps keys (including `NULL`) to values (including `NULL`). A map -cannot contain duplicate keys; each key can map to at most one value. +将键(包括 `NULL`)映射到值(包括 `NULL`)的关联数组的数据类型。映射不能包含重复的键;每个键最多可以映射到一个值。 -There is no restriction of element types; it is the responsibility of the user to ensure uniqueness. +元素类型没有限制;确保唯一性是用户的责任。 -The map type is an extension to the SQL standard. +Map 类型是 SQL 标准的扩展。 -**Declaration** +**声明**
    @@ -1032,24 +952,22 @@ DataTypes.MAP(kt, vt)
    -The type can be declared using `MAP` where `kt` is the data type of the key elements -and `vt` is the data type of the value elements. +此类型用 `MAP` 声明,其中 `kt` 是键的数据类型,`vt` 是值的数据类型。 -**Bridging to JVM Types** +**JVM 类型** -| Java Type | Input | Output | Remarks | +| Java 类型 | 输入 | 输出 | 备注 | |:--------------------------------------|:-----:|:------:|:----------| -| `java.util.Map` | X | X | *Default* | -| *subclass* of `java.util.Map` | X | | | +| `java.util.Map` | X | X | *缺省* | +| `java.util.Map` 的*子类型* | X | | | #### `MULTISET` -Data type of a multiset (=bag). Unlike a set, it allows for multiple instances for each of its -elements with a common subtype. Each unique value (including `NULL`) is mapped to some multiplicity. +多重集合的数据类型(=bag)。与集合不同的是,它允许每个具有公共子类型的元素有多个实例。每个唯一值(包括 `NULL`)都映射到某种多重性。 -There is no restriction of element types; it is the responsibility of the user to ensure uniqueness. +元素类型没有限制;确保唯一性是用户的责任。 -**Declaration** +**声明**
    @@ -1068,33 +986,28 @@ DataTypes.MULTISET(t)
    -The type can be declared using `MULTISET` where `t` is the data type -of the contained elements. +此类型用 `MULTISET` 声明,其中 `t` 是所包含元素的数据类型。 -`t MULTISET` is a synonym for being closer to the SQL standard. For example, `INT MULTISET` is -equivalent to `MULTISET`. +`t MULTISET` 接近等价于 SQL 标准。例如,`INT MULTISET` 等价于 `MULTISET`。 -**Bridging to JVM Types** +**JVM 类型** -| Java Type | Input | Output | Remarks | +| Java 类型 | 输入 | 输出 | 备注 | |:-------------------------------------|:-----:|:------:|:---------------------------------------------------------| -|`java.util.Map` | X | X | Assigns each value to an integer multiplicity. *Default* | -| *subclass* of `java.util.Map` | X | | Assigns each value to an integer multiplicity. | +|`java.util.Map` | X | X | 将每个值可多重地分配给一个整数 *缺省* | +|`java.util.Map` 的*子类型*| X | | 将每个值可多重地分配给一个整数 | #### `ROW` -Data type of a sequence of fields. +字段序列的数据类型。 -A field consists of a field name, field type, and an optional description. The most specific type -of a row of a table is a row type. In this case, each column of the row corresponds to the field -of the row type that has the same ordinal position as the column. +字段由字段名称、字段类型和可选的描述组成。表中的行的是最特殊的类型是 Row 类型。在这种情况下,行中的每一列对应于相同位置的列的 Row 类型的字段。 -Compared to the SQL standard, an optional field description simplifies the handling with complex -structures. +与 SQL 标准相比,可选的字段描述简化了复杂结构的处理。 -A row type is similar to the `STRUCT` type known from other non-standard-compliant frameworks. +Row 类型类似于其他非标准兼容框架中的 `STRUCT` 类型。 -**Declaration** +**声明**
    @@ -1117,25 +1030,23 @@ DataTypes.ROW(DataTypes.FIELD(n0, t0, d0), DataTypes.FIELD(n1, t1, d1), ...)
    -The type can be declared using `ROW` where `n` is the unique name of -a field, `t` is the logical type of a field, `d` is the description of a field. +此类型用 `ROW` 声明,其中 `n` 是唯一的字段名称,`t` 是字段的逻辑类型,`d` 是字段的描述。 -`ROW(...)` is a synonym for being closer to the SQL standard. For example, `ROW(myField INT, myOtherField BOOLEAN)` is -equivalent to `ROW`. +`ROW(...)` 接近等价于 SQL 标准。例如,`ROW(myField INT, myOtherField BOOLEAN)` 等价于 `ROW`。 -**Bridging to JVM Types** +**JVM 类型** -| Java Type | Input | Output | Remarks | +| Java 类型 | 输入 | 输出 | 备注 | |:----------------------------|:-----:|:------:|:------------------------| -|`org.apache.flink.types.Row` | X | X | *Default* | +|`org.apache.flink.types.Row` | X | X | *缺省* | -### Other Data Types +### 其他数据类型 #### `BOOLEAN` -Data type of a boolean with a (possibly) three-valued logic of `TRUE`, `FALSE`, and `UNKNOWN`. +(可能)具有 `TRUE`、`FALSE` 和 `UNKNOWN` 三值逻辑的布尔数据类型。 -**Declaration** +**声明**
    @@ -1153,26 +1064,24 @@ DataTypes.BOOLEAN()
    -**Bridging to JVM Types** +**JVM 类型** -| Java Type | Input | Output | Remarks | +| Java 类型 | 输入 | 输出 | 备注 | |:-------------------|:-----:|:------:|:-------------------------------------| -|`java.lang.Boolean` | X | X | *Default* | -|`boolean` | X | (X) | Output only if type is not nullable. | +|`java.lang.Boolean` | X | X | *缺省* | +|`boolean` | X | (X) | 仅当类型不可为空时才输出。 | #### `NULL` -Data type for representing untyped `NULL` values. +表示空类型 `NULL` 值的数据类型。 -The null type is an extension to the SQL standard. A null type has no other value -except `NULL`, thus, it can be cast to any nullable type similar to JVM semantics. +NULL 类型是 SQL 标准的扩展。NULL 类型除 `NULL` 值以外没有其他值,因此可以将其强制转换为 JVM 里的任何可空类型。 -This type helps in representing unknown types in API calls that use a `NULL` literal -as well as bridging to formats such as JSON or Avro that define such a type as well. +此类型有助于使用 `NULL` 字面量表示 `API` 调用中的未知类型,以及桥接到定义该类型的 JSON 或 Avro 等格式。 -This type is not very useful in practice and is just mentioned here for completeness. +这种类型在实践中不是很有用,为完整起见仅在此提及。 -**Declaration** +**声明**
    @@ -1190,21 +1099,20 @@ DataTypes.NULL()
    -**Bridging to JVM Types** +**JVM 类型** -| Java Type | Input | Output | Remarks | +| Java 类型 | 输入 | 输出 | 备注 | |:------------------|:-----:|:------:|:-------------------------------------| -|`java.lang.Object` | X | X | *Default* | -|*any class* | | (X) | Any non-primitive type. | +|`java.lang.Object` | X | X | *缺省* | +|*任何类型* | | (X) | 任何非基本数据类型 | #### `RAW` -Data type of an arbitrary serialized type. This type is a black box within the table ecosystem -and is only deserialized at the edges. +任意序列化类型的数据类型。此类型是 Table 编程环境中的黑箱,仅在边缘反序列化。 -The raw type is an extension to the SQL standard. +Raw 类型是 SQL 标准的扩展。 -**Declaration** +**声明**
    @@ -1224,18 +1132,15 @@ DataTypes.RAW(typeInfo)
    -The type can be declared using `RAW('class', 'snapshot')` where `class` is the originating class and -`snapshot` is the serialized `TypeSerializerSnapshot` in Base64 encoding. Usually, the type string is not -declared directly but is generated while persisting the type. +此类型用 `RAW('class', 'snapshot')` 声明,其中 `class` 是原始类,`snapshot` 是 Base64 编码的序列化的 `TypeSerializerSnapshot`。通常,类型字符串不是直接声明的,而是在保留类型时生成的。 -In the API, the `RAW` type can be declared either by directly supplying a `Class` + `TypeSerializer` or -by passing `TypeInformation` and let the framework extract `Class` + `TypeSerializer` from there. +在 API 中,可以通过直接提供 `Class` + `TypeSerializer` 或通过传递 `TypeInformation` 并让框架从那里提取 `Class` + `TypeSerializer` 来声明 `RAW` 类型。 -**Bridging to JVM Types** +**JVM 类型** -| Java Type | Input | Output | Remarks | +| Java 类型 | 输入 | 输出 | 备注 | |:------------------|:-----:|:------:|:-------------------------------------------| -|*class* | X | X | Originating class or subclasses (for input) or superclasses (for output). *Default* | +|*类型* | X | X | 原始类或子类(用于输入)或超类(用于输出)。 *缺省* | |`byte[]` | | X | | {% top %} -- Gitee From 059e71d607405446c40a1b380452f3e3189c94ae Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Thu, 5 Mar 2020 12:30:59 +0800 Subject: [PATCH 067/885] [FLINK-16084][docs-zh] Translate /dev/table/streaming/time_attributes.zh.md (#11102) --- .../dev/table/streaming/time_attributes.zh.md | 139 +++++++++--------- 1 file changed, 68 insertions(+), 71 deletions(-) diff --git a/docs/dev/table/streaming/time_attributes.zh.md b/docs/dev/table/streaming/time_attributes.zh.md index 3b9c892eeb..5846028c16 100644 --- a/docs/dev/table/streaming/time_attributes.zh.md +++ b/docs/dev/table/streaming/time_attributes.zh.md @@ -1,5 +1,5 @@ --- -title: "Time Attributes" +title: "时间属性" nav-parent_id: streaming_tableapi nav-pos: 2 --- @@ -22,29 +22,29 @@ specific language governing permissions and limitations under the License. --> -Flink is able to process streaming data based on different notions of *time*. +Flink 可以基于几种不同的 *时间* 概念来处理数据。 -- *Processing time* refers to the system time of the machine (also known as "wall-clock time") that is executing the respective operation. -- *Event time* refers to the processing of streaming data based on timestamps which are attached to each row. The timestamps can encode when an event happened. -- *Ingestion time* is the time that events enter Flink; internally, it is treated similarly to event time. +- *处理时间* 指的是执行具体操作时的机器时间(也称作"挂钟时间") +- *事件时间* 指的是数据本身携带的时间。这个时间是在事件产生时的时间。 +- *摄入时间* 指的是数据进入 Flink 的时间;在系统内部,会把它当做事件时间来处理。 -For more information about time handling in Flink, see the introduction about [Event Time and Watermarks]({{ site.baseurl }}/dev/event_time.html). +对于时间相关的更多信息,可以参考 [事件时间和Watermark]({{ site.baseurl }}/zh/dev/event_time.html)。 -This page explains how time attributes can be defined for time-based operations in Flink's Table API & SQL. +本页面说明了如何在 Flink Table API & SQL 里面定义时间以及相关的操作。 * This will be replaced by the TOC {:toc} -Introduction to Time Attributes +时间属性介绍 ------------------------------- -Time-based operations such as windows in both the [Table API]({{ site.baseurl }}/dev/table/tableApi.html#group-windows) and [SQL]({{ site.baseurl }}/dev/table/sql/queries.html#group-windows) require information about the notion of time and its origin. Therefore, tables can offer *logical time attributes* for indicating time and accessing corresponding timestamps in table programs. +像窗口(在 [Table API]({{ site.baseurl }}/zh/dev/table/tableApi.html#group-windows) 和 [SQL]({{ site.baseurl }}/zh/dev/table/sql/queries.html#group-windows) )这种基于时间的操作,需要有时间信息。因此,Table API 中的表就需要提供*逻辑时间属性*来表示时间,以及支持时间相关的操作。 -Time attributes can be part of every table schema. They are defined when creating a table from a CREATE TABLE DDL or a `DataStream` or are pre-defined when using a `TableSource`. Once a time attribute has been defined at the beginning, it can be referenced as a field and can be used in time-based operations. +每种类型的表都可以有时间属性,可以在用CREATE TABLE DDL创建表的时候指定、也可以在 `DataStream` 中指定、也可以在定义 `TableSource` 时指定。一旦时间属性定义好,它就可以像普通列一样使用,也可以在时间相关的操作中使用。 -As long as a time attribute is not modified and is simply forwarded from one part of the query to another, it remains a valid time attribute. Time attributes behave like regular timestamps and can be accessed for calculations. If a time attribute is used in a calculation, it will be materialized and becomes a regular timestamp. Regular timestamps do not cooperate with Flink's time and watermarking system and thus can not be used for time-based operations anymore. +只要时间属性没有被修改,而是简单地从一个表传递到另一个表,它就仍然是一个有效的时间属性。时间属性可以像普通的时间戳的列一样被使用和计算。一旦时间属性被用在了计算中,它就会被物化,进而变成一个普通的时间戳。普通的时间戳是无法跟 Flink 的时间以及watermark等一起使用的,所以普通的时间戳就无法用在时间相关的操作中。 -Table programs require that the corresponding time characteristic has been specified for the streaming environment: +Table API 程序需要在 streaming environment 中指定时间属性:
    @@ -53,7 +53,7 @@ final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEn env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime); // default -// alternatively: +// 或者: // env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime); // env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); {% endhighlight %} @@ -64,7 +64,7 @@ val env = StreamExecutionEnvironment.getExecutionEnvironment env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime) // default -// alternatively: +// 或者: // env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime) // env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) {% endhighlight %} @@ -75,30 +75,30 @@ env = StreamExecutionEnvironment.get_execution_environment() env.set_stream_time_characteristic(TimeCharacteristic.ProcessingTime) # default -# alternatively: +# 或者: # env.set_stream_time_characteristic(TimeCharacteristic.IngestionTime) # env.set_stream_time_characteristic(TimeCharacteristic.EventTime) {% endhighlight %}
    -Processing time +处理时间 --------------- -Processing time allows a table program to produce results based on the time of the local machine. It is the simplest notion of time but does not provide determinism. It neither requires timestamp extraction nor watermark generation. +处理时间是基于机器的本地时间来处理数据,它是最简单的一种时间概念,但是它不能提供确定性。它既不需要从数据里获取时间,也不需要生成 watermark。 -There are three ways to define a processing time attribute. +共有三种方法可以定义处理时间。 -### Defining in create table DDL +### 在创建表的 DDL 中定义 -The processing time attribute is defined as a computed column in create table DDL using the system `PROCTIME()` function. Please see [CREATE TABLE DDL]({{ site.baseurl }}/dev/table/sql/create.html#create-table) for more information about computed column. +处理时间属性可以在创建表的 DDL 中用计算列的方式定义,用 `PROCTIME()` 就可以定义处理时间。关于计算列,更多信息可以参考:[CREATE TABLE DDL]({{ site.baseurl }}/zh/dev/table/sql/create.html#create-table) {% highlight sql %} CREATE TABLE user_actions ( user_name STRING, data STRING, - user_action_time AS PROCTIME() -- declare an additional field as a processing time attribute + user_action_time AS PROCTIME() -- 声明一个额外的列作为处理时间属性 ) WITH ( ... ); @@ -110,16 +110,16 @@ GROUP BY TUMBLE(user_action_time, INTERVAL '10' MINUTE); {% endhighlight %} -### During DataStream-to-Table Conversion +### 在 DataStream 到 Table 转换时定义 -The processing time attribute is defined with the `.proctime` property during schema definition. The time attribute must only extend the physical schema by an additional logical field. Thus, it can only be defined at the end of the schema definition. +处理时间属性可以在 schema 定义的时候用 `.proctime` 后缀来定义。时间属性一定不能定义在一个已有字段上,所以它只能定义在 schem 定义的最后。
    {% highlight java %} DataStream> stream = ...; -// declare an additional logical field as a processing time attribute +// 声明一个额外的字段作为时间属性字段 Table table = tEnv.fromDataStream(stream, "user_name, data, user_action_time.proctime"); WindowedTable windowedTable = table.window(Tumble.over("10.minutes").on("user_action_time").as("userActionWindow")); @@ -129,7 +129,7 @@ WindowedTable windowedTable = table.window(Tumble.over("10.minutes").on("user_ac {% highlight scala %} val stream: DataStream[(String, String)] = ... -// declare an additional logical field as a processing time attribute +// 声明一个额外的字段作为时间属性字段 val table = tEnv.fromDataStream(stream, 'UserActionTimestamp, 'user_name, 'data, 'user_action_time.proctime) val windowedTable = table.window(Tumble over 10.minutes on 'user_action_time as 'userActionWindow) @@ -137,14 +137,14 @@ val windowedTable = table.window(Tumble over 10.minutes on 'user_action_time as
    -### Using a TableSource +### 使用 TableSource 定义 -The processing time attribute is defined by a `TableSource` that implements the `DefinedProctimeAttribute` interface. The logical time attribute is appended to the physical schema defined by the return type of the `TableSource`. +处理时间属性可以在实现了 `DefinedProctimeAttribute` 的 `TableSource` 中定义。逻辑的时间属性会放在 `TableSource` 已有物理字段的最后
    {% highlight java %} -// define a table source with a processing attribute +// 定义一个由处理时间属性的 table source public class UserActionSource implements StreamTableSource, DefinedProctimeAttribute { @Override @@ -163,7 +163,7 @@ public class UserActionSource implements StreamTableSource, DefinedProctime @Override public String getProctimeAttribute() { - // field with this name will be appended as a third field + // 这个名字的列会被追加到最后,作为第三列 return "user_action_time"; } } @@ -178,7 +178,7 @@ WindowedTable windowedTable = tEnv
    {% highlight scala %} -// define a table source with a processing attribute +// 定义一个由处理时间属性的 table source class UserActionSource extends StreamTableSource[Row] with DefinedProctimeAttribute { override def getReturnType = { @@ -194,7 +194,7 @@ class UserActionSource extends StreamTableSource[Row] with DefinedProctimeAttrib } override def getProctimeAttribute = { - // field with this name will be appended as a third field + // 这个名字的列会被追加到最后,作为第三列 "user_action_time" } } @@ -209,20 +209,20 @@ val windowedTable = tEnv
    -Event time +事件时间 ---------- -Event time allows a table program to produce results based on the time that is contained in every record. This allows for consistent results even in case of out-of-order events or late events. It also ensures replayable results of the table program when reading records from persistent storage. +事件时间允许程序按照数据中包含的时间来处理,这样可以在有乱序或者晚到的数据的情况下产生一致的处理结果。它可以保证从外部存储读取数据后产生可以复现(replayable)的结果。 -Additionally, event time allows for unified syntax for table programs in both batch and streaming environments. A time attribute in a streaming environment can be a regular field of a record in a batch environment. +除此之外,事件时间可以让程序在流式和批式作业中使用同样的语法。在流式程序中的事件时间属性,在批式程序中就是一个正常的时间字段。 -In order to handle out-of-order events and distinguish between on-time and late events in streaming, Flink needs to extract timestamps from events and make some kind of progress in time (so-called [watermarks]({{ site.baseurl }}/dev/event_time.html)). +为了能够处理乱序的事件,并且区分正常到达和晚到的事件,Flink 需要从事件中获取事件时间并且产生 watermark([watermarks]({{ site.baseurl }}/zh/dev/event_time.html))。 -An event time attribute can be defined either in create table DDL or during DataStream-to-Table conversion or by using a TableSource. +事件时间属性也有类似于处理时间的三种定义方式:在DDL中定义、在 DataStream 到 Table 转换时定义、用 TableSource 定义。 -### Defining in create table DDL +### 在 DDL 中定义 -The event time attribute is defined using WATERMARK statement in CREATE TABLE DDL. A watermark statement defines a watermark generation expression on an existing event time field, which marks the event time field as event time attribute. Please see [CREATE TABLE DDL]({{ site.baseurl }}/dev/table/sql/create.html#create-table) for more information about watermark statement and watermark strategies. +事件时间属性可以用 WATERMARK 语句在 CREATE TABLE DDL 中进行定义。WATERMARK 语句在一个已有字段上定义一个 watermark 生成表达式,同时标记这个已有字段为时间属性字段。更多信息可以参考:[CREATE TABLE DDL]({{ site.baseurl }}/zh/dev/table/sql/create.html#create-table) {% highlight sql %} @@ -230,7 +230,7 @@ CREATE TABLE user_actions ( user_name STRING, data STRING, user_action_time TIMESTAMP(3), - -- declare user_action_time as event time attribute and use 5 seconds delayed watermark strategy + -- 声明 user_action_time 是事件时间属性,并且用 延迟 5 秒的策略来生成 watermark WATERMARK FOR user_action_time AS user_action_time - INTERVAL '5' SECOND ) WITH ( ... @@ -243,16 +243,16 @@ GROUP BY TUMBLE(user_action_time, INTERVAL '10' MINUTE); {% endhighlight %} -### During DataStream-to-Table Conversion +### 在 DataStream 到 Table 转换时定义 -The event time attribute is defined with the `.rowtime` property during schema definition. [Timestamps and watermarks]({{ site.baseurl }}/dev/event_time.html) must have been assigned in the `DataStream` that is converted. +事件时间属性可以用 `.rowtime` 后缀在定义 `DataStream` schema 的时候来定义。[时间戳和 watermark]({{ site.baseurl }}/zh/dev/event_time.html) 在这之前一定是在 `DataStream` 上已经定义好了。 -There are two ways of defining the time attribute when converting a `DataStream` into a `Table`. Depending on whether the specified `.rowtime` field name exists in the schema of the `DataStream` or not, the timestamp field is either +在从 `DataStream` 到 `Table` 转换时定义事件时间属性有两种方式。取决于用 `.rowtime` 后缀修饰的字段名字是否是已有字段,事件时间字段可以是: -- appended as a new field to the schema or -- replaces an existing field. +- 在 schema 的结尾追加一个新的字段 +- 替换一个已经存在的字段。 -In either case the event time timestamp field will hold the value of the `DataStream` event time timestamp. +不管在哪种情况下,事件时间字段都表示 `DataStream` 中定义的事件的时间戳。
    @@ -260,20 +260,19 @@ In either case the event time timestamp field will hold the value of the `DataSt // Option 1: -// extract timestamp and assign watermarks based on knowledge of the stream +// 基于 stream 中的事件产生时间戳和 watermark DataStream> stream = inputStream.assignTimestampsAndWatermarks(...); -// declare an additional logical field as an event time attribute +// 声明一个额外的逻辑字段作为事件时间属性 Table table = tEnv.fromDataStream(stream, "user_name, data, user_action_time.rowtime"); // Option 2: -// extract timestamp from first field, and assign watermarks based on knowledge of the stream +// 从第一个字段获取事件时间,并且产生 watermark DataStream> stream = inputStream.assignTimestampsAndWatermarks(...); -// the first field has been used for timestamp extraction, and is no longer necessary -// replace first field with a logical event time attribute +// 第一个字段已经用作事件时间抽取了,不用再用一个新字段来表示事件时间了 Table table = tEnv.fromDataStream(stream, "user_action_time.rowtime, user_name, data"); // Usage: @@ -286,20 +285,19 @@ WindowedTable windowedTable = table.window(Tumble.over("10.minutes").on("user_ac // Option 1: -// extract timestamp and assign watermarks based on knowledge of the stream +// 基于 stream 中的事件产生时间戳和 watermark val stream: DataStream[(String, String)] = inputStream.assignTimestampsAndWatermarks(...) -// declare an additional logical field as an event time attribute +// 声明一个额外的逻辑字段作为事件时间属性 val table = tEnv.fromDataStream(stream, 'user_name, 'data, 'user_action_time.rowtime) // Option 2: -// extract timestamp from first field, and assign watermarks based on knowledge of the stream +// 从第一个字段获取事件时间,并且产生 watermark val stream: DataStream[(Long, String, String)] = inputStream.assignTimestampsAndWatermarks(...) -// the first field has been used for timestamp extraction, and is no longer necessary -// replace first field with a logical event time attribute +// 第一个字段已经用作事件时间抽取了,不用再用一个新字段来表示事件时间了 val table = tEnv.fromDataStream(stream, 'user_action_time.rowtime, 'user_name, 'data) // Usage: @@ -309,19 +307,18 @@ val windowedTable = table.window(Tumble over 10.minutes on 'user_action_time as
    -### Using a TableSource +### 使用 TableSource 定义 -The event time attribute is defined by a `TableSource` that implements the `DefinedRowtimeAttributes` interface. The `getRowtimeAttributeDescriptors()` method returns a list of `RowtimeAttributeDescriptor` for describing the final name of a time attribute, a timestamp extractor to derive the values of the attribute, and the watermark strategy associated with the attribute. +事件时间属性可以在实现了 `DefinedRowTimeAttributes` 的 `TableSource` 中定义。`getRowtimeAttributeDescriptors()` 方法返回 `RowtimeAttributeDescriptor` 的列表,包含了描述事件时间属性的字段名字、如何计算事件时间、以及 watermark 生成策略等信息。 -Please make sure that the `DataStream` returned by the `getDataStream()` method is aligned with the defined time attribute. -The timestamps of the `DataStream` (the ones which are assigned by a `TimestampAssigner`) are only considered if a `StreamRecordTimestamp` timestamp extractor is defined. -Watermarks of a `DataStream` are only preserved if a `PreserveWatermarks` watermark strategy is defined. -Otherwise, only the values of the `TableSource`'s rowtime attribute are relevant. +同时需要确保 `getDataStream` 返回的 `DataStream` 已经定义好了时间属性。 +只有在定义了 `StreamRecordTimestamp` 时间戳分配器的时候,才认为 `DataStream` 是有时间戳信息的。 +只有定义了 `PreserveWatermarks` watermark 生成策略的 `DataStream` 的 watermark 才会被保留。反之,则只有时间字段的值是生效的。
    {% highlight java %} -// define a table source with a rowtime attribute +// 定义一个有事件时间属性的 table source public class UserActionSource implements StreamTableSource, DefinedRowtimeAttributes { @Override @@ -334,17 +331,17 @@ public class UserActionSource implements StreamTableSource, DefinedRowtimeA @Override public DataStream getDataStream(StreamExecutionEnvironment execEnv) { - // create stream + // 构造 DataStream // ... - // assign watermarks based on the "user_action_time" attribute + // 基于 "user_action_time" 定义 watermark DataStream stream = inputStream.assignTimestampsAndWatermarks(...); return stream; } @Override public List getRowtimeAttributeDescriptors() { - // Mark the "user_action_time" attribute as event-time attribute. - // We create one attribute descriptor of "user_action_time". + // 标记 "user_action_time" 字段是事件时间字段 + // 给 "user_action_time" 构造一个时间属性描述符 RowtimeAttributeDescriptor rowtimeAttrDescr = new RowtimeAttributeDescriptor( "user_action_time", new ExistingField("user_action_time"), @@ -364,7 +361,7 @@ WindowedTable windowedTable = tEnv
    {% highlight scala %} -// define a table source with a rowtime attribute +// 定义一个有事件时间属性的 table source class UserActionSource extends StreamTableSource[Row] with DefinedRowtimeAttributes { override def getReturnType = { @@ -374,16 +371,16 @@ class UserActionSource extends StreamTableSource[Row] with DefinedRowtimeAttribu } override def getDataStream(execEnv: StreamExecutionEnvironment): DataStream[Row] = { - // create stream + // 构造 DataStream // ... - // assign watermarks based on the "user_action_time" attribute + // 基于 "user_action_time" 定义 watermark val stream = inputStream.assignTimestampsAndWatermarks(...) stream } override def getRowtimeAttributeDescriptors: util.List[RowtimeAttributeDescriptor] = { - // Mark the "user_action_time" attribute as event-time attribute. - // We create one attribute descriptor of "user_action_time". + // 标记 "user_action_time" 字段是事件时间字段 + // 给 "user_action_time" 构造一个时间属性描述符 val rowtimeAttrDescr = new RowtimeAttributeDescriptor( "user_action_time", new ExistingField("user_action_time"), -- Gitee From b4730308a75f5a99f930622ccdd5289380c5d73c Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Thu, 5 Mar 2020 17:01:00 +0800 Subject: [PATCH 068/885] [FLINK-16418][hive] Hide hive version to avoid user confuse (#11304) --- docs/dev/table/hive/hive_catalog.md | 1 - docs/dev/table/hive/hive_catalog.zh.md | 1 - docs/dev/table/hive/hive_functions.md | 1 - docs/dev/table/hive/hive_functions.zh.md | 1 - docs/dev/table/hive/index.md | 1 - docs/dev/table/hive/index.zh.md | 1 - docs/dev/table/modules.md | 1 - docs/dev/table/modules.zh.md | 1 - docs/dev/table/sqlClient.md | 2 -- docs/dev/table/sqlClient.zh.md | 2 -- .../apache/flink/table/catalog/hive/HiveCatalog.java | 4 ++++ .../org/apache/flink/table/module/hive/HiveModule.java | 4 ++++ .../flink/table/module/hive/HiveModuleDescriptor.java | 8 ++++---- .../flink/table/module/hive/HiveModuleFactoryTest.java | 6 ++---- .../apache/flink/table/module/hive/HiveModuleTest.java | 10 +++++----- 15 files changed, 19 insertions(+), 25 deletions(-) diff --git a/docs/dev/table/hive/hive_catalog.md b/docs/dev/table/hive/hive_catalog.md index 92b05cbbcb..d2e7d5a652 100644 --- a/docs/dev/table/hive/hive_catalog.md +++ b/docs/dev/table/hive/hive_catalog.md @@ -153,7 +153,6 @@ catalogs: - name: myhive type: hive hive-conf-dir: /opt/hive-conf # contains hive-site.xml - hive-version: 2.3.4 {% endhighlight %} diff --git a/docs/dev/table/hive/hive_catalog.zh.md b/docs/dev/table/hive/hive_catalog.zh.md index 92b05cbbcb..d2e7d5a652 100644 --- a/docs/dev/table/hive/hive_catalog.zh.md +++ b/docs/dev/table/hive/hive_catalog.zh.md @@ -153,7 +153,6 @@ catalogs: - name: myhive type: hive hive-conf-dir: /opt/hive-conf # contains hive-site.xml - hive-version: 2.3.4 {% endhighlight %} diff --git a/docs/dev/table/hive/hive_functions.md b/docs/dev/table/hive/hive_functions.md index e2b7ac51b6..537d684aa3 100644 --- a/docs/dev/table/hive/hive_functions.md +++ b/docs/dev/table/hive/hive_functions.md @@ -54,7 +54,6 @@ modules: type: core - name: myhive type: hive - hive-version: 2.3.4 {% endhighlight %}
    diff --git a/docs/dev/table/hive/hive_functions.zh.md b/docs/dev/table/hive/hive_functions.zh.md index e2b7ac51b6..537d684aa3 100644 --- a/docs/dev/table/hive/hive_functions.zh.md +++ b/docs/dev/table/hive/hive_functions.zh.md @@ -54,7 +54,6 @@ modules: type: core - name: myhive type: hive - hive-version: 2.3.4 {% endhighlight %} diff --git a/docs/dev/table/hive/index.md b/docs/dev/table/hive/index.md index 42e825633f..f1e8eed108 100644 --- a/docs/dev/table/hive/index.md +++ b/docs/dev/table/hive/index.md @@ -374,7 +374,6 @@ catalogs: - name: myhive type: hive hive-conf-dir: /opt/hive-conf - hive-version: 2.3.4 {% endhighlight %} diff --git a/docs/dev/table/hive/index.zh.md b/docs/dev/table/hive/index.zh.md index 925dce1326..2da1559a23 100644 --- a/docs/dev/table/hive/index.zh.md +++ b/docs/dev/table/hive/index.zh.md @@ -374,7 +374,6 @@ catalogs: - name: myhive type: hive hive-conf-dir: /opt/hive-conf - hive-version: 2.3.4 {% endhighlight %} diff --git a/docs/dev/table/modules.md b/docs/dev/table/modules.md index ec39a08c05..e38e2438ed 100644 --- a/docs/dev/table/modules.md +++ b/docs/dev/table/modules.md @@ -105,7 +105,6 @@ modules: type: core - name: myhive type: hive - hive-version: 1.2.1 {% endhighlight %} diff --git a/docs/dev/table/modules.zh.md b/docs/dev/table/modules.zh.md index 8367865bed..10eb32b0b1 100644 --- a/docs/dev/table/modules.zh.md +++ b/docs/dev/table/modules.zh.md @@ -105,7 +105,6 @@ modules: type: core - name: myhive type: hive - hive-version: 1.2.1 {% endhighlight %} diff --git a/docs/dev/table/sqlClient.md b/docs/dev/table/sqlClient.md index d35a054b16..ed4ff80f9f 100644 --- a/docs/dev/table/sqlClient.md +++ b/docs/dev/table/sqlClient.md @@ -211,7 +211,6 @@ catalogs: property-version: 1 default-database: mydb2 hive-conf-dir: ... - hive-version: 1.2.1 # Properties that change the fundamental execution behavior of a table program. @@ -448,7 +447,6 @@ catalogs: type: hive property-version: 1 default-database: mydb2 - hive-version: 1.2.1 hive-conf-dir: - name: catalog_2 type: hive diff --git a/docs/dev/table/sqlClient.zh.md b/docs/dev/table/sqlClient.zh.md index bce976afa9..93d506b8ac 100644 --- a/docs/dev/table/sqlClient.zh.md +++ b/docs/dev/table/sqlClient.zh.md @@ -211,7 +211,6 @@ catalogs: property-version: 1 default-database: mydb2 hive-conf-dir: ... - hive-version: 1.2.1 # Properties that change the fundamental execution behavior of a table program. @@ -448,7 +447,6 @@ catalogs: type: hive property-version: 1 default-database: mydb2 - hive-version: 1.2.1 hive-conf-dir: - name: catalog_2 type: hive diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalog.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalog.java index 83a5299884..8cf3abcf2e 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalog.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalog.java @@ -136,6 +136,10 @@ public class HiveCatalog extends AbstractCatalog { private HiveMetastoreClientWrapper client; + public HiveCatalog(String catalogName, @Nullable String defaultDatabase, @Nullable String hiveConfDir) { + this(catalogName, defaultDatabase, hiveConfDir, HiveShimLoader.getHiveVersion()); + } + public HiveCatalog(String catalogName, @Nullable String defaultDatabase, @Nullable String hiveConfDir, String hiveVersion) { this(catalogName, defaultDatabase == null ? DEFAULT_DB : defaultDatabase, diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/module/hive/HiveModule.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/module/hive/HiveModule.java index 2a3e922afa..e34965bf62 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/module/hive/HiveModule.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/module/hive/HiveModule.java @@ -53,6 +53,10 @@ public class HiveModule implements Module { private final String hiveVersion; private final HiveShim hiveShim; + public HiveModule() { + this(HiveShimLoader.getHiveVersion()); + } + public HiveModule(String hiveVersion) { checkArgument(!StringUtils.isNullOrWhitespaceOnly(hiveVersion), "hiveVersion cannot be null"); diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/module/hive/HiveModuleDescriptor.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/module/hive/HiveModuleDescriptor.java index 0acd653f53..894097773c 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/module/hive/HiveModuleDescriptor.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/module/hive/HiveModuleDescriptor.java @@ -20,13 +20,11 @@ package org.apache.flink.table.module.hive; import org.apache.flink.table.descriptors.DescriptorProperties; import org.apache.flink.table.descriptors.ModuleDescriptor; -import org.apache.flink.util.StringUtils; import java.util.Map; import static org.apache.flink.table.module.hive.HiveModuleDescriptorValidator.MODULE_HIVE_VERSION; import static org.apache.flink.table.module.hive.HiveModuleDescriptorValidator.MODULE_TYPE_HIVE; -import static org.apache.flink.util.Preconditions.checkArgument; /** * Module descriptor for {@link HiveModule}. @@ -34,10 +32,12 @@ import static org.apache.flink.util.Preconditions.checkArgument; public class HiveModuleDescriptor extends ModuleDescriptor { private String hiveVersion; + public HiveModuleDescriptor() { + this(null); + } + public HiveModuleDescriptor(String hiveVersion) { super(MODULE_TYPE_HIVE); - - checkArgument(!StringUtils.isNullOrWhitespaceOnly(hiveVersion)); this.hiveVersion = hiveVersion; } diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/module/hive/HiveModuleFactoryTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/module/hive/HiveModuleFactoryTest.java index 22e3adba04..03485c2a84 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/module/hive/HiveModuleFactoryTest.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/module/hive/HiveModuleFactoryTest.java @@ -35,11 +35,9 @@ import static org.junit.Assert.assertEquals; public class HiveModuleFactoryTest { @Test public void test() { - final String hiveVersion = "2.3.4"; + final HiveModule expected = new HiveModule(); - final HiveModule expected = new HiveModule(hiveVersion); - - final ModuleDescriptor moduleDescriptor = new HiveModuleDescriptor(hiveVersion); + final ModuleDescriptor moduleDescriptor = new HiveModuleDescriptor(); final Map properties = moduleDescriptor.toProperties(); diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/module/hive/HiveModuleTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/module/hive/HiveModuleTest.java index 4acc6166fb..50a5494dbb 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/module/hive/HiveModuleTest.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/module/hive/HiveModuleTest.java @@ -83,7 +83,7 @@ public class HiveModuleTest { @Test public void testHiveBuiltInFunction() { - FunctionDefinition fd = new HiveModule(HiveShimLoader.getHiveVersion()).getFunctionDefinition("reverse").get(); + FunctionDefinition fd = new HiveModule().getFunctionDefinition("reverse").get(); ScalarFunction func = ((ScalarFunctionDefinition) fd).getScalarFunction(); HiveSimpleUDF udf = (HiveSimpleUDF) func; @@ -102,7 +102,7 @@ public class HiveModuleTest { @Test public void testNonExistFunction() { - assertFalse(new HiveModule(HiveShimLoader.getHiveVersion()).getFunctionDefinition("nonexist").isPresent()); + assertFalse(new HiveModule().getFunctionDefinition("nonexist").isPresent()); } @Test @@ -110,7 +110,7 @@ public class HiveModuleTest { TableEnvironment tEnv = HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(); tEnv.unloadModule("core"); - tEnv.loadModule("hive", new HiveModule(HiveShimLoader.getHiveVersion())); + tEnv.loadModule("hive", new HiveModule()); List results = TableUtils.collectToList(tEnv.sqlQuery("select concat('an', 'bn')")); assertEquals("[anbn]", results.toString()); @@ -134,7 +134,7 @@ public class HiveModuleTest { TableEnvironment tEnv = HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(); tEnv.unloadModule("core"); - tEnv.loadModule("hive", new HiveModule(HiveShimLoader.getHiveVersion())); + tEnv.loadModule("hive", new HiveModule()); List results = TableUtils.collectToList(tEnv.sqlQuery("select negative(5.1)")); @@ -143,7 +143,7 @@ public class HiveModuleTest { @Test public void testBlackList() { - HiveModule hiveModule = new HiveModule(HiveShimLoader.getHiveVersion()); + HiveModule hiveModule = new HiveModule(); assertFalse(hiveModule.listFunctions().removeAll(HiveModule.BUILT_IN_FUNC_BLACKLIST)); for (String banned : HiveModule.BUILT_IN_FUNC_BLACKLIST) { assertFalse(hiveModule.getFunctionDefinition(banned).isPresent()); -- Gitee From 0296dfa2e7eb7f69beb0f0fd038e6879e5f1dab3 Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Fri, 28 Feb 2020 15:44:35 +0100 Subject: [PATCH 069/885] [FLINK-16122][AZP] Upload artifacts file This closes #11306 --- flink-end-to-end-tests/run-nightly-tests.sh | 12 ++++++++++++ .../test-scripts/test-runner-common.sh | 3 +++ tools/azure-pipelines/jobs-template.yml | 16 +++++++++++++++- tools/travis_watchdog.sh | 14 ++++++++++++-- 4 files changed, 42 insertions(+), 3 deletions(-) diff --git a/flink-end-to-end-tests/run-nightly-tests.sh b/flink-end-to-end-tests/run-nightly-tests.sh index ad6b86c694..624bd6dcc0 100755 --- a/flink-end-to-end-tests/run-nightly-tests.sh +++ b/flink-end-to-end-tests/run-nightly-tests.sh @@ -220,6 +220,8 @@ fi ARTIFACTS_DIR="${HERE}/artifacts" mkdir -p $ARTIFACTS_DIR || { echo "FAILURE: cannot create log directory '${ARTIFACTS_DIR}'." ; exit 1; } +env > $ARTIFACTS_DIR/environment + LOG4J_PROPERTIES=${HERE}/../tools/log4j-travis.properties MVN_LOGGING_OPTIONS="-Dlog.dir=${ARTIFACTS_DIR} -Dlog4j.configurationFile=file://$LOG4J_PROPERTIES -Dorg.slf4j.simpleLogger.log.org.apache.maven.cli.transfer.Slf4jMavenTransferListener=warn" @@ -232,4 +234,14 @@ mvn ${MVN_COMMON_OPTIONS} ${MVN_LOGGING_OPTIONS} ${PROFILE} verify -pl ${e2e_mod EXIT_CODE=$? +# On Azure, publish ARTIFACTS_FILE as a build artifact +if [ ! -z "$TF_BUILD" ] ; then + echo "COMPRESSING build artifacts." + ARTIFACTS_FILE=${BUILD_BUILDNUMBER}.tgz + tar -zcvf ${ARTIFACTS_FILE} $ARTIFACTS_DIR + mkdir artifact-dir + cp ${ARTIFACTS_FILE} artifact-dir/ + echo "##vso[task.setvariable variable=ARTIFACT_DIR]$(pwd)/artifact-dir" +fi + exit $EXIT_CODE diff --git a/flink-end-to-end-tests/test-scripts/test-runner-common.sh b/flink-end-to-end-tests/test-scripts/test-runner-common.sh index ecddbd9fe8..7e09e515e8 100644 --- a/flink-end-to-end-tests/test-scripts/test-runner-common.sh +++ b/flink-end-to-end-tests/test-scripts/test-runner-common.sh @@ -48,10 +48,13 @@ function run_test { echo "[FAIL] Test script contains errors." post_test_validation 1 "$description" "$skip_check_exceptions" } + # set a trap to catch a test execution error trap 'test_error' ERR ${command} exit_code="$?" + # remove trap for test execution + trap - ERR post_test_validation ${exit_code} "$description" "$skip_check_exceptions" } diff --git a/tools/azure-pipelines/jobs-template.yml b/tools/azure-pipelines/jobs-template.yml index 4d8bf6504a..8cba11efe7 100644 --- a/tools/azure-pipelines/jobs-template.yml +++ b/tools/azure-pipelines/jobs-template.yml @@ -126,7 +126,13 @@ jobs: - task: PublishTestResults@2 inputs: testResultsFormat: 'JUnit' - + # upload debug artifacts + - task: PublishPipelineArtifact@1 + condition: and(succeededOrFailed(), not(eq('$(ARTIFACT_DIR)', ''))) + displayName: Upload Logs + inputs: + path: $(ARTIFACT_DIR) + artifact: logs-${{parameters.stage_name}}-$(ARTIFACT_NAME) - job: e2e_${{parameters.stage_name}} # uncomment below condition to run the e2e tests only on request. @@ -163,4 +169,12 @@ jobs: # displayName: Test - precommit - script: FLINK_DIR=`pwd`/build-target flink-end-to-end-tests/run-nightly-tests.sh displayName: Run e2e tests + # upload debug artifacts + - task: PublishPipelineArtifact@1 + condition: and(succeededOrFailed(), not(eq('$(ARTIFACT_DIR)', ''))) + displayName: Upload Logs + inputs: + path: $(ARTIFACT_DIR) + artifact: logs-${{parameters.stage_name}}-e2e + diff --git a/tools/travis_watchdog.sh b/tools/travis_watchdog.sh index 595e9f4be6..4a1bb67a63 100755 --- a/tools/travis_watchdog.sh +++ b/tools/travis_watchdog.sh @@ -44,11 +44,11 @@ MAX_NO_OUTPUT=${1:-300} SLEEP_TIME=20 # Maximum times to retry uploading artifacts file to transfer.sh -TRANSFER_UPLOAD_MAX_RETRIES=10 +TRANSFER_UPLOAD_MAX_RETRIES=2 # The delay between two retries to upload artifacts file to transfer.sh. The default exponential # backoff algorithm should be too long for the last several retries. -TRANSFER_UPLOAD_RETRY_DELAY=15 +TRANSFER_UPLOAD_RETRY_DELAY=5 LOG4J_PROPERTIES=${HERE}/log4j-travis.properties @@ -144,6 +144,16 @@ upload_artifacts_s3() { artifacts upload --bucket $UPLOAD_BUCKET --key $UPLOAD_ACCESS_KEY --secret $UPLOAD_SECRET_KEY --target-paths $UPLOAD_TARGET_PATH $ARTIFACTS_FILE fi + # On Azure, publish ARTIFACTS_FILE as a build artifact + if [ ! -z "$TF_BUILD" ] ; then + ARTIFACT_DIR="$(pwd)/artifact-dir" + mkdir $ARTIFACT_DIR + cp $ARTIFACTS_FILE $ARTIFACT_DIR/ + + echo "##vso[task.setvariable variable=ARTIFACT_DIR]$ARTIFACT_DIR" + echo "##vso[task.setvariable variable=ARTIFACT_NAME]$(echo $MODULE | tr -dc '[:alnum:]\n\r')" + fi + # upload to https://transfer.sh echo "Uploading to transfer.sh" curl --retry ${TRANSFER_UPLOAD_MAX_RETRIES} --retry-delay ${TRANSFER_UPLOAD_RETRY_DELAY} --upload-file $ARTIFACTS_FILE --max-time 60 https://transfer.sh -- Gitee From 2e4a642235a680baa3bc3115fd42f449ad590384 Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Wed, 4 Mar 2020 10:36:36 +0100 Subject: [PATCH 070/885] [hotfix][AZP] Shorten cron build job names --- tools/azure-pipelines/build-apache-repo.yml | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tools/azure-pipelines/build-apache-repo.yml b/tools/azure-pipelines/build-apache-repo.yml index e1a8bfa628..559c84d420 100644 --- a/tools/azure-pipelines/build-apache-repo.yml +++ b/tools/azure-pipelines/build-apache-repo.yml @@ -44,13 +44,13 @@ variables: stages: # CI / PR triggered stage: - - stage: ci_build + - stage: ci displayName: "CI build (custom builders)" condition: not(eq(variables['Build.Reason'], in('Schedule', 'Manual'))) jobs: - template: jobs-template.yml parameters: - stage_name: ci_build + stage_name: ci test_pool_definition: name: Default e2e_pool_definition: @@ -67,7 +67,7 @@ stages: jobs: - template: jobs-template.yml parameters: - stage_name: cron_build_hadoop241 + stage_name: cron_hadoop241 test_pool_definition: name: Default e2e_pool_definition: @@ -78,7 +78,7 @@ stages: jdk: jdk8 - template: jobs-template.yml parameters: - stage_name: cron_build_scala212 + stage_name: cron_scala212 test_pool_definition: name: Default e2e_pool_definition: @@ -89,7 +89,7 @@ stages: jdk: jdk8 - template: jobs-template.yml parameters: - stage_name: cron_build_jdk11 + stage_name: cron_jdk11 test_pool_definition: name: Default e2e_pool_definition: @@ -100,7 +100,7 @@ stages: jdk: jdk11 - template: jobs-template.yml parameters: - stage_name: cron_build_hadoopfree + stage_name: cron_hadoopfree test_pool_definition: name: Default e2e_pool_definition: -- Gitee From be0eb5477732b8a8bb77476e1c850fbd40d851c7 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Wed, 4 Mar 2020 16:37:52 +0100 Subject: [PATCH 071/885] [FLINK-16400][s3] Fixing YarnFileStageTestS3ITCase for direct Hadoop access. --- .../java/org/apache/flink/yarn/YarnFileStageTestS3ITCase.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTestS3ITCase.java b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTestS3ITCase.java index 6cf6884afd..11d48bd697 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTestS3ITCase.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTestS3ITCase.java @@ -20,6 +20,7 @@ package org.apache.flink.yarn; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.fs.hdfs.HadoopFileSystem; @@ -137,6 +138,7 @@ public class YarnFileStageTestS3ITCase extends TestLogger { final Configuration conf = new Configuration(); conf.setString(ConfigConstants.HDFS_SITE_CONFIG, hadoopConfig.getAbsolutePath()); + conf.set(CoreOptions.ALLOWED_FALLBACK_FILESYSTEMS, "s3;s3a;s3n"); FileSystem.initialize(conf); } -- Gitee From 3e66fcfd25f6a24e6d2b2da161683e1556242733 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Wed, 4 Mar 2020 19:16:31 +0100 Subject: [PATCH 072/885] [FLINK-16400][filesystem] Deprecating filesystem kind. Also removing invalid HdfsKindTest cases. --- .../org/apache/flink/core/fs/FileSystem.java | 2 + .../flink/runtime/fs/hdfs/HdfsKindTest.java | 42 ------------------- 2 files changed, 2 insertions(+), 42 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java index ba5c0c0350..0985a8b95e 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java @@ -760,7 +760,9 @@ public abstract class FileSystem { /** * Gets a description of the characteristics of this file system. + * @deprecated this method is not used anymore. */ + @Deprecated public abstract FileSystemKind getKind(); // ------------------------------------------------------------------------ diff --git a/flink-filesystems/flink-hadoop-fs/src/test/java/org/apache/flink/runtime/fs/hdfs/HdfsKindTest.java b/flink-filesystems/flink-hadoop-fs/src/test/java/org/apache/flink/runtime/fs/hdfs/HdfsKindTest.java index 69ecdb851b..afa5c3ab7c 100644 --- a/flink-filesystems/flink-hadoop-fs/src/test/java/org/apache/flink/runtime/fs/hdfs/HdfsKindTest.java +++ b/flink-filesystems/flink-hadoop-fs/src/test/java/org/apache/flink/runtime/fs/hdfs/HdfsKindTest.java @@ -44,48 +44,6 @@ public class HdfsKindTest extends TestLogger { assertEquals(FileSystemKind.FILE_SYSTEM, fs.getKind()); } - @Test - public void testS3Kind() throws IOException { - try { - Class.forName("org.apache.hadoop.fs.s3.S3FileSystem"); - } catch (ClassNotFoundException ignored) { - // not in the classpath, cannot run this test - log.info("Skipping test 'testS3Kind()' because the S3 file system is not in the class path"); - return; - } - - final FileSystem s3 = new Path("s3://myId:mySecret@bucket/some/bucket/some/object").getFileSystem(); - assertEquals(FileSystemKind.OBJECT_STORE, s3.getKind()); - } - - @Test - public void testS3nKind() throws IOException { - try { - Class.forName("org.apache.hadoop.fs.s3native.NativeS3FileSystem"); - } catch (ClassNotFoundException ignored) { - // not in the classpath, cannot run this test - log.info("Skipping test 'testS3nKind()' because the Native S3 file system is not in the class path"); - return; - } - - final FileSystem s3n = new Path("s3n://myId:mySecret@bucket/some/bucket/some/object").getFileSystem(); - assertEquals(FileSystemKind.OBJECT_STORE, s3n.getKind()); - } - - @Test - public void testS3aKind() throws IOException { - try { - Class.forName("org.apache.hadoop.fs.s3a.S3AFileSystem"); - } catch (ClassNotFoundException ignored) { - // not in the classpath, cannot run this test - log.info("Skipping test 'testS3aKind()' because the S3AFileSystem is not in the class path"); - return; - } - - final FileSystem s3a = new Path("s3a://myId:mySecret@bucket/some/bucket/some/object").getFileSystem(); - assertEquals(FileSystemKind.OBJECT_STORE, s3a.getKind()); - } - @Test public void testS3fileSystemSchemes() { assertEquals(FileSystemKind.OBJECT_STORE, HadoopFileSystem.getKindForScheme("s3")); -- Gitee From 377024ba85058e3ba5fa092aaf5c92be33d7ae09 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Thu, 5 Mar 2020 19:40:20 +0800 Subject: [PATCH 073/885] [FLINK-16269][FLINK-16108][table-planner-blink] Fix schema of query and sink do not match when generic or POJO type is requested (#11236) This also fixes failure of StreamSQLExample when running in blink planner. --- .../types/inference/TypeTransformations.java | 8 ++ .../LegacyRawTypeTransformation.java | 46 +++++++ .../inference/TypeTransformationsTest.java | 27 ++++ .../planner/codegen/SinkCodeGenerator.scala | 9 +- .../table/planner/sinks/TableSinkUtils.scala | 122 ++++++++++++------ .../planner/runtime/utils/JavaPojos.java | 81 ++++++++++++ .../sql/StreamTableEnvironmentITCase.scala | 96 ++++++++++++++ .../runtime/utils/StreamTestSink.scala | 8 ++ 8 files changed, 353 insertions(+), 44 deletions(-) create mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/transforms/LegacyRawTypeTransformation.java create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/StreamTableEnvironmentITCase.scala diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/TypeTransformations.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/TypeTransformations.java index 0bb8fa6ca4..99e4ff9a81 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/TypeTransformations.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/TypeTransformations.java @@ -22,6 +22,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.inference.transforms.DataTypeConversionClassTransformation; import org.apache.flink.table.types.inference.transforms.LegacyDecimalTypeTransformation; +import org.apache.flink.table.types.inference.transforms.LegacyRawTypeTransformation; import org.apache.flink.table.types.logical.LogicalTypeRoot; import java.sql.Date; @@ -58,6 +59,13 @@ public class TypeTransformations { return LegacyDecimalTypeTransformation.INSTANCE; } + /** + * Returns a type transformation that transforms LEGACY('RAW', ...) type to the RAW(..., ?) type. + */ + public static TypeTransformation legacyRawToTypeInfoRaw() { + return LegacyRawTypeTransformation.INSTANCE; + } + /** * Returns a type transformation that transforms data type to nullable data type but keeps * other information unchanged. diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/transforms/LegacyRawTypeTransformation.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/transforms/LegacyRawTypeTransformation.java new file mode 100644 index 0000000000..c03ff7549c --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/transforms/LegacyRawTypeTransformation.java @@ -0,0 +1,46 @@ +/* + * 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.table.types.inference.transforms; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.inference.TypeTransformation; +import org.apache.flink.table.types.logical.LegacyTypeInformationType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; + +/** + * This type transformation transforms the LEGACY('RAW', ...) type to the RAW(..., ?) type. + */ +public class LegacyRawTypeTransformation implements TypeTransformation { + + public static final TypeTransformation INSTANCE = new LegacyRawTypeTransformation(); + + @Override + public DataType transform(DataType typeToTransform) { + LogicalType logicalType = typeToTransform.getLogicalType(); + if (logicalType instanceof LegacyTypeInformationType && logicalType.getTypeRoot() == LogicalTypeRoot.RAW) { + TypeInformation typeInfo = ((LegacyTypeInformationType) logicalType).getTypeInformation(); + DataType rawType = DataTypes.RAW(typeInfo).bridgedTo(typeInfo.getTypeClass()); + return logicalType.isNullable() ? rawType : rawType.notNull(); + } + return typeToTransform; + } +} diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/TypeTransformationsTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/TypeTransformationsTest.java index f4b2815168..64c0342f63 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/TypeTransformationsTest.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/TypeTransformationsTest.java @@ -18,7 +18,9 @@ package org.apache.flink.table.types.inference; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.utils.DataTypeUtils; @@ -31,6 +33,7 @@ import java.sql.Time; import java.sql.Timestamp; import static org.apache.flink.table.types.inference.TypeTransformations.legacyDecimalToDefaultDecimal; +import static org.apache.flink.table.types.inference.TypeTransformations.legacyRawToTypeInfoRaw; import static org.apache.flink.table.types.inference.TypeTransformations.timeToSqlTypes; import static org.apache.flink.table.types.inference.TypeTransformations.toNullable; import static org.junit.Assert.assertEquals; @@ -84,6 +87,26 @@ public class TypeTransformationsTest { assertEquals(expected, DataTypeUtils.transform(dataType, legacyDecimalToDefaultDecimal())); } + @Test + public void testLegacyRawToTypeInfoRaw() { + DataType dataType = DataTypes.ROW( + DataTypes.FIELD("a", DataTypes.STRING()), + DataTypes.FIELD("b", DataTypes.DECIMAL(10, 3)), + DataTypes.FIELD("c", createLegacyRaw()), + DataTypes.FIELD("d", DataTypes.ARRAY(createLegacyRaw())) + ); + + TypeInformation typeInformation = TypeExtractor.getForClass(TypeTransformationsTest.class); + DataType expected = DataTypes.ROW( + DataTypes.FIELD("a", DataTypes.STRING()), + DataTypes.FIELD("b", DataTypes.DECIMAL(10, 3)), + DataTypes.FIELD("c", DataTypes.RAW(typeInformation)), + DataTypes.FIELD("d", DataTypes.ARRAY(DataTypes.RAW(typeInformation))) + ); + + assertEquals(expected, DataTypeUtils.transform(dataType, legacyRawToTypeInfoRaw())); + } + @Test public void testToNullable() { DataType dataType = DataTypes.ROW( @@ -110,4 +133,8 @@ public class TypeTransformationsTest { private static DataType createLegacyDecimal() { return TypeConversions.fromLegacyInfoToDataType(Types.BIG_DEC); } + + private static DataType createLegacyRaw() { + return TypeConversions.fromLegacyInfoToDataType(Types.GENERIC(TypeTransformationsTest.class)); + } } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/SinkCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/SinkCodeGenerator.scala index 9a807f112a..2cc9a65a53 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/SinkCodeGenerator.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/SinkCodeGenerator.scala @@ -20,6 +20,7 @@ package org.apache.flink.table.planner.codegen import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.typeinfo.{TypeInformation, Types} +import org.apache.flink.api.common.typeutils.CompositeType import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2} import org.apache.flink.api.java.typeutils.{PojoTypeInfo, TupleTypeInfo} import org.apache.flink.api.java.typeutils.runtime.TupleSerializerBase @@ -33,6 +34,8 @@ import org.apache.flink.table.planner.codegen.OperatorCodeGenerator.generateColl import org.apache.flink.table.planner.sinks.TableSinkUtils import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter.fromDataTypeToTypeInfo +import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType +import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo import org.apache.flink.table.sinks.TableSink import org.apache.flink.table.types.logical.RowType @@ -83,8 +86,12 @@ object SinkCodeGenerator { inputRowType, inputTerm, inputFieldMapping = Option(mapping)) + val outputRowType = RowType.of( + (0 until pojo.getArity) + .map(pojo.getTypeAt) + .map(fromTypeInfoToLogicalType): _*) val conversion = resultGenerator.generateConverterResultExpression( - inputRowType, + outputRowType, classOf[GenericRow]) afterIndexModify = CodeGenUtils.newName("afterIndexModify") s""" diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sinks/TableSinkUtils.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sinks/TableSinkUtils.scala index 63e6bbe21b..90955ee9e1 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sinks/TableSinkUtils.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sinks/TableSinkUtils.scala @@ -19,7 +19,7 @@ package org.apache.flink.table.planner.sinks import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2} -import org.apache.flink.api.java.typeutils.{GenericTypeInfo, TupleTypeInfo} +import org.apache.flink.api.java.typeutils.{GenericTypeInfo, PojoTypeInfo, TupleTypeInfo} import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo import org.apache.flink.table.api._ import org.apache.flink.table.catalog.{CatalogTable, ObjectIdentifier} @@ -27,17 +27,17 @@ import org.apache.flink.table.dataformat.BaseRow import org.apache.flink.table.operations.CatalogSinkModifyOperation import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.utils.RelOptUtils +import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter.fromDataTypeToTypeInfo import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo import org.apache.flink.table.sinks._ import org.apache.flink.table.types.DataType -import org.apache.flink.table.types.inference.TypeTransformations.{legacyDecimalToDefaultDecimal, toNullable} +import org.apache.flink.table.types.inference.TypeTransformations.{legacyDecimalToDefaultDecimal, toNullable, legacyRawToTypeInfoRaw} import org.apache.flink.table.types.logical.utils.{LogicalTypeCasts, LogicalTypeChecks} -import org.apache.flink.table.types.logical.{LegacyTypeInformationType, LogicalType, RowType} +import org.apache.flink.table.types.logical.{LegacyTypeInformationType, LogicalType, RowType, TypeInformationRawType} import org.apache.flink.table.types.utils.DataTypeUtils import org.apache.flink.table.types.utils.TypeConversions.{fromLegacyInfoToDataType, fromLogicalToDataType} import org.apache.flink.table.utils.{TableSchemaUtils, TypeMappingUtils} import org.apache.flink.types.Row - import org.apache.calcite.rel.RelNode import _root_.scala.collection.JavaConversions._ @@ -64,7 +64,7 @@ object TableSinkUtils { val queryLogicalType = FlinkTypeFactory.toLogicalRowType(query.getRowType) val sinkLogicalType = DataTypeUtils // we recognize legacy decimal is the same to default decimal - .transform(sinkSchema.toRowDataType, legacyDecimalToDefaultDecimal) + .transform(sinkSchema.toRowDataType, legacyDecimalToDefaultDecimal, legacyRawToTypeInfoRaw) .getLogicalType .asInstanceOf[RowType] if (LogicalTypeCasts.supportsImplicitCast(queryLogicalType, sinkLogicalType)) { @@ -189,13 +189,47 @@ object TableSinkUtils { queryLogicalType, withChangeFlag) if (LogicalTypeChecks.isCompositeType(requestedOutputType.getLogicalType)) { - DataTypeUtils.expandCompositeTypeToSchema(requestedOutputType) + // if the requested output type is POJO, then we should ignore the POJO fields order, + // and infer the sink schema via field names, see expandPojoTypeToSchema(). + fromDataTypeToTypeInfo(requestedOutputType) match { + case pj: PojoTypeInfo[_] => expandPojoTypeToSchema(pj, queryLogicalType) + case _ => DataTypeUtils.expandCompositeTypeToSchema(requestedOutputType) + } } else { // atomic type TableSchema.builder().field("f0", requestedOutputType).build() } } + /** + * Expands a [[PojoTypeInfo]] to a corresponding [[TableSchema]]. + * This is a special handling for [[PojoTypeInfo]], because fields of [[PojoTypeInfo]] is not + * in the defined order but the alphabet order. In order to match the query schema, we should + * reorder the Pojo schema. + */ + private def expandPojoTypeToSchema( + pojo: PojoTypeInfo[_], + queryLogicalType: RowType): TableSchema = { + val fieldNames = queryLogicalType.getFieldNames + // reorder pojo fields according to the query schema + val reorderedFields = fieldNames.map(name => { + val index = pojo.getFieldIndex(name) + if (index < 0) { + throw new TableException(s"$name is not found in ${pojo.toString}") + } + val fieldTypeInfo = pojo.getPojoFieldAt(index).getTypeInformation + val fieldDataType = fieldTypeInfo match { + case nestedPojo: PojoTypeInfo[_] => + val nestedLogicalType = queryLogicalType.getFields()(index).getType.asInstanceOf[RowType] + expandPojoTypeToSchema(nestedPojo, nestedLogicalType).toRowDataType + case _ => + fromLegacyInfoToDataType(fieldTypeInfo) + } + DataTypes.FIELD(name, fieldDataType) + }) + DataTypeUtils.expandCompositeTypeToSchema(DataTypes.ROW(reorderedFields: _*)) + } + /** * Inferences the physical data type of [[TableSink]], the physical data type ignores * the change flag field. @@ -209,45 +243,47 @@ object TableSinkUtils { consumedDataType: DataType, queryLogicalType: RowType, withChangeFlag: Boolean): DataType = { - consumedDataType.getLogicalType match { - case lt: LegacyTypeInformationType[_] => - val requestedTypeInfo = if (withChangeFlag) { - lt.getTypeInformation match { - // Scala tuple - case t: CaseClassTypeInfo[_] - if t.getTypeClass == classOf[(_, _)] && t.getTypeAt(0) == Types.BOOLEAN => - t.getTypeAt[Any](1) - // Java tuple - case t: TupleTypeInfo[_] - if t.getTypeClass == classOf[JTuple2[_, _]] && t.getTypeAt(0) == Types.BOOLEAN => - t.getTypeAt[Any](1) - case _ => throw new TableException( - "Don't support " + consumedDataType + " conversion for the retract sink") - } - } else { - lt.getTypeInformation - } - // The tpe may been inferred by invoking [[TypeExtractor.createTypeInfo]] based the - // class of the resulting type. For example, converts the given [[Table]] into - // an append [[DataStream]]. If the class is Row, then the return type only is - // [[GenericTypeInfo[Row]]. So it should convert to the [[RowTypeInfo]] in order - // to better serialize performance. - requestedTypeInfo match { - case gt: GenericTypeInfo[Row] if gt.getTypeClass == classOf[Row] => - fromLogicalToDataType(queryLogicalType).bridgedTo(classOf[Row]) - case gt: GenericTypeInfo[BaseRow] if gt.getTypeClass == classOf[BaseRow] => - fromLogicalToDataType(queryLogicalType).bridgedTo(classOf[BaseRow]) - case bt: BaseRowTypeInfo => - val fields = bt.getFieldNames.zip(bt.getLogicalTypes).map { case (n, t) => - DataTypes.FIELD(n, fromLogicalToDataType(t)) - } - DataTypes.ROW(fields: _*).bridgedTo(classOf[BaseRow]) - case _ => - fromLegacyInfoToDataType(requestedTypeInfo) - } + val consumedTypeInfo = consumedDataType.getLogicalType match { + case lt: LegacyTypeInformationType[_] => Some(lt.getTypeInformation) + case _ => None + } + if (consumedTypeInfo.isEmpty) { + return consumedDataType + } + val requestedTypeInfo = if (withChangeFlag) { + consumedTypeInfo.get match { + // Scala tuple + case t: CaseClassTypeInfo[_] + if t.getTypeClass == classOf[(_, _)] && t.getTypeAt(0) == Types.BOOLEAN => + t.getTypeAt[Any](1) + // Java tuple + case t: TupleTypeInfo[_] + if t.getTypeClass == classOf[JTuple2[_, _]] && t.getTypeAt(0) == Types.BOOLEAN => + t.getTypeAt[Any](1) + case _ => throw new TableException( + "Don't support " + consumedDataType + " conversion for the retract sink") + } + } else { + consumedTypeInfo.get + } + // The tpe may been inferred by invoking [[TypeExtractor.createTypeInfo]] based the + // class of the resulting type. For example, converts the given [[Table]] into + // an append [[DataStream]]. If the class is Row, then the return type only is + // [[GenericTypeInfo[Row]]. So it should convert to the [[RowTypeInfo]] in order + // to better serialize performance. + requestedTypeInfo match { + case gt: GenericTypeInfo[Row] if gt.getTypeClass == classOf[Row] => + fromLogicalToDataType(queryLogicalType).bridgedTo(classOf[Row]) + case gt: GenericTypeInfo[BaseRow] if gt.getTypeClass == classOf[BaseRow] => + fromLogicalToDataType(queryLogicalType).bridgedTo(classOf[BaseRow]) + case bt: BaseRowTypeInfo => + val fields = bt.getFieldNames.zip(bt.getLogicalTypes).map { case (n, t) => + DataTypes.FIELD(n, fromLogicalToDataType(t)) + } + DataTypes.ROW(fields: _*).bridgedTo(classOf[BaseRow]) case _ => - consumedDataType + fromLegacyInfoToDataType(requestedTypeInfo) } } diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaPojos.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaPojos.java index d8c4537c9f..a64fe052ea 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaPojos.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaPojos.java @@ -20,6 +20,8 @@ package org.apache.flink.table.planner.runtime.utils; import java.io.Serializable; import java.sql.Timestamp; +import java.util.Map; +import java.util.TreeMap; /** * POJOs for Table API testing. @@ -42,4 +44,83 @@ public class JavaPojos { '}'; } } + + /** + * Nested POJO. + */ + public static class Order { + public Long user; + public ProductItem product; + public int amount; + + public Order() { + } + + public Order(Long user, ProductItem product, int amount) { + this.user = user; + this.product = product; + this.amount = amount; + } + + @Override + public String toString() { + return "Order{" + + "user=" + user + + ", product='" + product + '\'' + + ", amount=" + amount + + '}'; + } + } + + /** + * Simple POJO. + */ + public static class ProductItem { + public String name; + public Long id; + + public ProductItem() { + } + + public ProductItem(String name, Long id) { + this.name = name; + this.id = id; + } + + @Override + public String toString() { + return "Product{" + + "name='" + name + '\'' + + ", id=" + id + + '}'; + } + } + + /** + * POJO with a RAW type. + */ + public static class Device { + public Long deviceId; + public String deviceName; + // raw type + public TreeMap metrics; + + public Device() { + } + + public Device(Long deviceId, String deviceName, Map metrics) { + this.deviceId = deviceId; + this.deviceName = deviceName; + this.metrics = new TreeMap<>(metrics); + } + + @Override + public String toString() { + return "Device{" + + "deviceId=" + deviceId + + ", deviceName='" + deviceName + '\'' + + ", metrics=" + metrics + + '}'; + } + } } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/StreamTableEnvironmentITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/StreamTableEnvironmentITCase.scala new file mode 100644 index 0000000000..d26c86484e --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/StreamTableEnvironmentITCase.scala @@ -0,0 +1,96 @@ +/* + * 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.table.planner.runtime.stream.sql + +import org.apache.flink.api.scala._ +import org.apache.flink.streaming.api.scala.DataStream +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.planner.runtime.utils.JavaPojos.{Device, Order, ProductItem} +import org.apache.flink.table.planner.runtime.utils.{StreamingTestBase, StringSink} + +import org.junit.Assert.assertEquals +import org.junit.Test + +import java.util.Collections + +/** + * Integration tests for methods on [[org.apache.flink.table.api.scala.StreamTableEnvironment]]. + */ +class StreamTableEnvironmentITCase extends StreamingTestBase { + + @Test + def testToAppendStreamWithPojoType(): Unit = { + val orderA = env.fromCollection(Seq( + new Order(1L, new ProductItem("beer", 10L), 3), + new Order(1L, new ProductItem("diaper", 11L), 4), + new Order(3L, new ProductItem("rubber", 12L), 2))) + + val orderB: DataStream[Order] = env.fromCollection(Seq( + new Order(2L, new ProductItem("pen", 13L), 3), + new Order(2L, new ProductItem("rubber", 12L), 3), + new Order(4L, new ProductItem("beer", 10L), 1))) + + // convert DataStream to Table + val tableA = tEnv.fromDataStream(orderA, 'user, 'product, 'amount) + // register DataStream as Table + tEnv.createTemporaryView("OrderB", orderB, 'user, 'product, 'amount) + + // union the two tables + val result = tEnv.sqlQuery( + s""" + |SELECT * FROM $tableA WHERE amount > 2 + |UNION ALL + |SELECT * FROM OrderB WHERE amount < 2 + """.stripMargin) + + val sink = new StringSink[Order]() + result.toAppendStream[Order].addSink(sink) + + env.execute() + + val expected = List( + "Order{user=1, product='Product{name='beer', id=10}', amount=3}", + "Order{user=1, product='Product{name='diaper', id=11}', amount=4}", + "Order{user=4, product='Product{name='beer', id=10}', amount=1}") + assertEquals(expected.sorted, sink.getResults.sorted) + } + + @Test + def testToAppendStreamWithRawType(): Unit = { + val devices = env.fromCollection(Seq( + new Device(1L, "device1", Collections.singletonMap("A", 10)), + new Device(2L, "device2", Collections.emptyMap()), + new Device(3L, "device3", Collections.singletonMap("B", 20)) + )) + + // register DataStream as Table + tEnv.createTemporaryView("devices", devices,'deviceId, 'deviceName, 'metrics) + + val result = tEnv.sqlQuery("SELECT * FROM devices WHERE deviceId >= 2") + val sink = new StringSink[Device]() + result.toAppendStream[Device].addSink(sink) + + env.execute() + + val expected = List( + "Device{deviceId=2, deviceName='device2', metrics={}}", + "Device{deviceId=3, deviceName='device3', metrics={B=20}}") + assertEquals(expected.sorted, sink.getResults.sorted) + } +} diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamTestSink.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamTestSink.scala index 24fb71b85e..1ddf312684 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamTestSink.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamTestSink.scala @@ -127,6 +127,14 @@ abstract class AbstractExactlyOnceSink[T] extends RichSinkFunction[T] with Check } } +final class StringSink[T] extends AbstractExactlyOnceSink[T]() { + override def invoke(value: T) { + localResults += value.toString + } + + override def getResults: List[String] = super.getResults +} + final class TestingAppendBaseRowSink( rowTypeInfo: BaseRowTypeInfo, tz: TimeZone) extends AbstractExactlyOnceSink[BaseRow] { -- Gitee From ef8402ee0c119a1dafe42157d3cbfef4c84dff4c Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 27 Feb 2020 16:44:24 +0100 Subject: [PATCH 074/885] [FLINK-16177][refactor][checkpointing] Remove unnecessary test constructor from PendingCheckpoint. The constructor is adding trivial benefit and is only used once. --- .../runtime/checkpoint/PendingCheckpoint.java | 23 +------------------ .../checkpoint/PendingCheckpointTest.java | 3 ++- 2 files changed, 3 insertions(+), 23 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java index a596acba93..ae9027da2e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java @@ -117,30 +117,9 @@ public class PendingCheckpoint { private volatile ScheduledFuture cancellerHandle; - private CheckpointException failureCause = null; + private CheckpointException failureCause; // -------------------------------------------------------------------------------------------- - public PendingCheckpoint( - JobID jobId, - long checkpointId, - long checkpointTimestamp, - Map verticesToConfirm, - Collection masterStateIdentifiers, - CheckpointProperties props, - CheckpointStorageLocation targetLocation, - Executor executor) { - - this( - jobId, - checkpointId, - checkpointTimestamp, - verticesToConfirm, - masterStateIdentifiers, - props, - targetLocation, - executor, - new CompletableFuture<>()); - } public PendingCheckpoint( JobID jobId, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java index cdda728bf7..c78c077833 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java @@ -462,7 +462,8 @@ public class PendingCheckpointTest { masterStateIdentifiers, props, location, - executor); + executor, + new CompletableFuture<>()); } @SuppressWarnings("unchecked") -- Gitee From 196196620915397d7d45bbb43a31d5db2627f109 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Mon, 24 Feb 2020 20:25:57 +0100 Subject: [PATCH 075/885] [FLINK-16177][refactor][checkpointing] Introduce a new MetadataV3Serializer for the latest version --- .../flink/runtime/checkpoint/Checkpoints.java | 6 +- .../metadata/MetadataSerializers.java | 18 +- .../metadata/MetadataV2Serializer.java | 500 +--------------- .../metadata/MetadataV2V3SerializerBase.java | 555 ++++++++++++++++++ .../metadata/MetadataV3Serializer.java | 65 ++ ...est.java => MetadataV3SerializerTest.java} | 4 +- .../streaming/util/OperatorSnapshotUtil.java | 22 +- 7 files changed, 653 insertions(+), 517 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2V3SerializerBase.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV3Serializer.java rename flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/metadata/{MetadataV2SerializerTest.java => MetadataV3SerializerTest.java} (98%) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/Checkpoints.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/Checkpoints.java index 225b7b9f6d..17db5e6ce6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/Checkpoints.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/Checkpoints.java @@ -23,7 +23,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.checkpoint.metadata.CheckpointMetadata; import org.apache.flink.runtime.checkpoint.metadata.MetadataSerializer; import org.apache.flink.runtime.checkpoint.metadata.MetadataSerializers; -import org.apache.flink.runtime.checkpoint.metadata.MetadataV2Serializer; +import org.apache.flink.runtime.checkpoint.metadata.MetadataV3Serializer; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.OperatorID; @@ -85,8 +85,8 @@ public class Checkpoints { // write generic header out.writeInt(HEADER_MAGIC_NUMBER); - out.writeInt(MetadataV2Serializer.VERSION); - MetadataV2Serializer.serialize(checkpointMetadata, out); + out.writeInt(MetadataV3Serializer.VERSION); + MetadataV3Serializer.serialize(checkpointMetadata, out); } // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataSerializers.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataSerializers.java index ba28b7f6ca..b8863bd2b7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataSerializers.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataSerializers.java @@ -28,19 +28,18 @@ import java.util.Map; */ public class MetadataSerializers { - private static final Map SERIALIZERS = new HashMap<>(2); + private static final Map SERIALIZERS = new HashMap<>(3); static { - SERIALIZERS.put(MetadataV1Serializer.VERSION, MetadataV1Serializer.INSTANCE); - SERIALIZERS.put(MetadataV2Serializer.VERSION, MetadataV2Serializer.INSTANCE); + registerSerializer(MetadataV1Serializer.INSTANCE); + registerSerializer(MetadataV2Serializer.INSTANCE); + registerSerializer(MetadataV3Serializer.INSTANCE); } - private MetadataSerializers() { - throw new AssertionError(); + private static void registerSerializer(MetadataSerializer serializer) { + SERIALIZERS.put(serializer.getVersion(), serializer); } - // ------------------------------------------------------------------------ - /** * Returns the {@link MetadataSerializer} for the given savepoint version. * @@ -56,4 +55,9 @@ public class MetadataSerializers { throw new IllegalArgumentException("Unrecognized checkpoint version number: " + version); } } + + // ------------------------------------------------------------------------ + + /** Utility method class, not meant to be instantiated. */ + private MetadataSerializers() {} } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2Serializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2Serializer.java index c2ac522065..bddceb84fa 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2Serializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2Serializer.java @@ -19,76 +19,25 @@ package org.apache.flink.runtime.checkpoint.metadata; import org.apache.flink.annotation.Internal; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.core.fs.Path; -import org.apache.flink.runtime.checkpoint.MasterState; -import org.apache.flink.runtime.checkpoint.OperatorState; -import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; -import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.flink.runtime.state.IncrementalRemoteKeyedStateHandle; -import org.apache.flink.runtime.state.KeyGroupRange; -import org.apache.flink.runtime.state.KeyGroupRangeOffsets; -import org.apache.flink.runtime.state.KeyGroupsStateHandle; -import org.apache.flink.runtime.state.KeyedStateHandle; -import org.apache.flink.runtime.state.OperatorStateHandle; -import org.apache.flink.runtime.state.OperatorStreamStateHandle; -import org.apache.flink.runtime.state.StateHandleID; -import org.apache.flink.runtime.state.StreamStateHandle; -import org.apache.flink.runtime.state.filesystem.FileStateHandle; -import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; import java.io.DataInputStream; -import java.io.DataOutputStream; import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.UUID; /** * (De)serializer for checkpoint metadata format version 2. + * This format was introduced with Apache Flink 1.3.0. * - *

    This format version adds - * - *

    Basic checkpoint metadata layout: - *

    - *  +--------------+---------------+-----------------+
    - *  | checkpointID | master states | operator states |
    - *  +--------------+---------------+-----------------+
    - *
    - *  Master state:
    - *  +--------------+---------------------+---------+------+---------------+
    - *  | magic number | num remaining bytes | version | name | payload bytes |
    - *  +--------------+---------------------+---------+------+---------------+
    - * 
    + *

    See {@link MetadataV2V3SerializerBase} for a description of the format layout. */ @Internal -public class MetadataV2Serializer implements MetadataSerializer { +public class MetadataV2Serializer extends MetadataV2V3SerializerBase implements MetadataSerializer{ /** The metadata format version. */ public static final int VERSION = 2; - /** Random magic number for consistency checks. */ - private static final int MASTER_STATE_MAGIC_NUMBER = 0xc96b1696; - - private static final byte NULL_HANDLE = 0; - private static final byte BYTE_STREAM_STATE_HANDLE = 1; - private static final byte FILE_STREAM_STATE_HANDLE = 2; - private static final byte KEY_GROUPS_HANDLE = 3; - private static final byte PARTITIONABLE_OPERATOR_STATE_HANDLE = 4; - private static final byte INCREMENTAL_KEY_GROUPS_HANDLE = 5; - /** The singleton instance of the serializer. */ public static final MetadataV2Serializer INSTANCE = new MetadataV2Serializer(); - // ------------------------------------------------------------------------ - /** Singleton, not meant to be instantiated. */ private MetadataV2Serializer() {} @@ -98,448 +47,11 @@ public class MetadataV2Serializer implements MetadataSerializer { } // ------------------------------------------------------------------------ - // (De)serialization entry points + // Deserialization entry point // ------------------------------------------------------------------------ - public static void serialize(CheckpointMetadata checkpointMetadata, DataOutputStream dos) throws IOException { - // first: checkpoint ID - dos.writeLong(checkpointMetadata.getCheckpointId()); - - // second: master state - final Collection masterStates = checkpointMetadata.getMasterStates(); - dos.writeInt(masterStates.size()); - for (MasterState ms : masterStates) { - serializeMasterState(ms, dos); - } - - // third: operator states - Collection operatorStates = checkpointMetadata.getOperatorStates(); - dos.writeInt(operatorStates.size()); - - for (OperatorState operatorState : operatorStates) { - // Operator ID - dos.writeLong(operatorState.getOperatorID().getLowerPart()); - dos.writeLong(operatorState.getOperatorID().getUpperPart()); - - // Parallelism - int parallelism = operatorState.getParallelism(); - dos.writeInt(parallelism); - dos.writeInt(operatorState.getMaxParallelism()); - dos.writeInt(1); - - // Sub task states - Map subtaskStateMap = operatorState.getSubtaskStates(); - dos.writeInt(subtaskStateMap.size()); - for (Map.Entry entry : subtaskStateMap.entrySet()) { - dos.writeInt(entry.getKey()); - serializeSubtaskState(entry.getValue(), dos); - } - } - } - @Override - public CheckpointMetadata deserialize(DataInputStream dis, ClassLoader cl) throws IOException { - // first: checkpoint ID - final long checkpointId = dis.readLong(); - if (checkpointId < 0) { - throw new IOException("invalid checkpoint ID: " + checkpointId); - } - - // second: master state - final List masterStates; - final int numMasterStates = dis.readInt(); - - if (numMasterStates == 0) { - masterStates = Collections.emptyList(); - } - else if (numMasterStates > 0) { - masterStates = new ArrayList<>(numMasterStates); - for (int i = 0; i < numMasterStates; i++) { - masterStates.add(deserializeMasterState(dis)); - } - } - else { - throw new IOException("invalid number of master states: " + numMasterStates); - } - - // third: operator states - int numTaskStates = dis.readInt(); - List operatorStates = new ArrayList<>(numTaskStates); - - for (int i = 0; i < numTaskStates; i++) { - OperatorID jobVertexId = new OperatorID(dis.readLong(), dis.readLong()); - int parallelism = dis.readInt(); - int maxParallelism = dis.readInt(); - int chainLength = dis.readInt(); - - // Add task state - OperatorState taskState = new OperatorState(jobVertexId, parallelism, maxParallelism); - operatorStates.add(taskState); - - // Sub task states - int numSubTaskStates = dis.readInt(); - - for (int j = 0; j < numSubTaskStates; j++) { - int subtaskIndex = dis.readInt(); - - OperatorSubtaskState subtaskState = deserializeSubtaskState(dis); - taskState.putState(subtaskIndex, subtaskState); - } - } - - return new CheckpointMetadata(checkpointId, operatorStates, masterStates); - } - - // ------------------------------------------------------------------------ - // master state (de)serialization methods - // ------------------------------------------------------------------------ - - private static void serializeMasterState(MasterState state, DataOutputStream dos) throws IOException { - // magic number for error detection - dos.writeInt(MASTER_STATE_MAGIC_NUMBER); - - // for safety, we serialize first into an array and then write the array and its - // length into the checkpoint - final ByteArrayOutputStream baos = new ByteArrayOutputStream(); - final DataOutputStream out = new DataOutputStream(baos); - - out.writeInt(state.version()); - out.writeUTF(state.name()); - - final byte[] bytes = state.bytes(); - out.writeInt(bytes.length); - out.write(bytes, 0, bytes.length); - - out.close(); - byte[] data = baos.toByteArray(); - - dos.writeInt(data.length); - dos.write(data, 0, data.length); - } - - private MasterState deserializeMasterState(DataInputStream dis) throws IOException { - final int magicNumber = dis.readInt(); - if (magicNumber != MASTER_STATE_MAGIC_NUMBER) { - throw new IOException("incorrect magic number in master styte byte sequence"); - } - - final int numBytes = dis.readInt(); - if (numBytes <= 0) { - throw new IOException("found zero or negative length for master state bytes"); - } - - final byte[] data = new byte[numBytes]; - dis.readFully(data); - - final DataInputStream in = new DataInputStream(new ByteArrayInputStream(data)); - - final int version = in.readInt(); - final String name = in.readUTF(); - - final byte[] bytes = new byte[in.readInt()]; - in.readFully(bytes); - - // check that the data is not corrupt - if (in.read() != -1) { - throw new IOException("found trailing bytes in master state"); - } - - return new MasterState(name, bytes, version); - } - - // ------------------------------------------------------------------------ - // task state (de)serialization methods - // ------------------------------------------------------------------------ - - private static T extractSingleton(Collection collection) { - if (collection == null || collection.isEmpty()) { - return null; - } - - if (collection.size() == 1) { - return collection.iterator().next(); - } else { - throw new IllegalStateException("Expected singleton collection, but found size: " + collection.size()); - } - } - - private static void serializeSubtaskState(OperatorSubtaskState subtaskState, DataOutputStream dos) throws IOException { - - dos.writeLong(-1); - - int len = 0; - dos.writeInt(len); - - OperatorStateHandle operatorStateBackend = extractSingleton(subtaskState.getManagedOperatorState()); - - len = operatorStateBackend != null ? 1 : 0; - dos.writeInt(len); - if (len == 1) { - serializeOperatorStateHandle(operatorStateBackend, dos); - } - - OperatorStateHandle operatorStateFromStream = extractSingleton(subtaskState.getRawOperatorState()); - - len = operatorStateFromStream != null ? 1 : 0; - dos.writeInt(len); - if (len == 1) { - serializeOperatorStateHandle(operatorStateFromStream, dos); - } - - KeyedStateHandle keyedStateBackend = extractSingleton(subtaskState.getManagedKeyedState()); - serializeKeyedStateHandle(keyedStateBackend, dos); - - KeyedStateHandle keyedStateStream = extractSingleton(subtaskState.getRawKeyedState()); - serializeKeyedStateHandle(keyedStateStream, dos); - } - - private static OperatorSubtaskState deserializeSubtaskState(DataInputStream dis) throws IOException { - // Duration field has been removed from SubtaskState, do not remove - long ignoredDuration = dis.readLong(); - - final int numLegacyTaskStates = dis.readInt(); - if (numLegacyTaskStates > 0) { - throw new IOException( - "Legacy state (from Flink <= 1.1, created through the 'Checkpointed' interface) is " + - "no longer supported."); - } - - int len = dis.readInt(); - OperatorStateHandle operatorStateBackend = len == 0 ? null : deserializeOperatorStateHandle(dis); - - len = dis.readInt(); - OperatorStateHandle operatorStateStream = len == 0 ? null : deserializeOperatorStateHandle(dis); - - KeyedStateHandle keyedStateBackend = deserializeKeyedStateHandle(dis); - - KeyedStateHandle keyedStateStream = deserializeKeyedStateHandle(dis); - - return new OperatorSubtaskState( - operatorStateBackend, - operatorStateStream, - keyedStateBackend, - keyedStateStream); - } - - @VisibleForTesting - public static void serializeKeyedStateHandle( - KeyedStateHandle stateHandle, DataOutputStream dos) throws IOException { - - if (stateHandle == null) { - dos.writeByte(NULL_HANDLE); - } else if (stateHandle instanceof KeyGroupsStateHandle) { - KeyGroupsStateHandle keyGroupsStateHandle = (KeyGroupsStateHandle) stateHandle; - - dos.writeByte(KEY_GROUPS_HANDLE); - dos.writeInt(keyGroupsStateHandle.getKeyGroupRange().getStartKeyGroup()); - dos.writeInt(keyGroupsStateHandle.getKeyGroupRange().getNumberOfKeyGroups()); - for (int keyGroup : keyGroupsStateHandle.getKeyGroupRange()) { - dos.writeLong(keyGroupsStateHandle.getOffsetForKeyGroup(keyGroup)); - } - serializeStreamStateHandle(keyGroupsStateHandle.getDelegateStateHandle(), dos); - } else if (stateHandle instanceof IncrementalRemoteKeyedStateHandle) { - IncrementalRemoteKeyedStateHandle incrementalKeyedStateHandle = - (IncrementalRemoteKeyedStateHandle) stateHandle; - - dos.writeByte(INCREMENTAL_KEY_GROUPS_HANDLE); - - dos.writeLong(incrementalKeyedStateHandle.getCheckpointId()); - dos.writeUTF(String.valueOf(incrementalKeyedStateHandle.getBackendIdentifier())); - dos.writeInt(incrementalKeyedStateHandle.getKeyGroupRange().getStartKeyGroup()); - dos.writeInt(incrementalKeyedStateHandle.getKeyGroupRange().getNumberOfKeyGroups()); - - serializeStreamStateHandle(incrementalKeyedStateHandle.getMetaStateHandle(), dos); - - serializeStreamStateHandleMap(incrementalKeyedStateHandle.getSharedState(), dos); - serializeStreamStateHandleMap(incrementalKeyedStateHandle.getPrivateState(), dos); - } else { - throw new IllegalStateException("Unknown KeyedStateHandle type: " + stateHandle.getClass()); - } - } - - private static void serializeStreamStateHandleMap( - Map map, - DataOutputStream dos) throws IOException { - dos.writeInt(map.size()); - for (Map.Entry entry : map.entrySet()) { - dos.writeUTF(entry.getKey().toString()); - serializeStreamStateHandle(entry.getValue(), dos); - } - } - - private static Map deserializeStreamStateHandleMap( - DataInputStream dis) throws IOException { - - final int size = dis.readInt(); - Map result = new HashMap<>(size); - - for (int i = 0; i < size; ++i) { - StateHandleID stateHandleID = new StateHandleID(dis.readUTF()); - StreamStateHandle stateHandle = deserializeStreamStateHandle(dis); - result.put(stateHandleID, stateHandle); - } - - return result; - } - - @VisibleForTesting - public static KeyedStateHandle deserializeKeyedStateHandle(DataInputStream dis) throws IOException { - final int type = dis.readByte(); - if (NULL_HANDLE == type) { - - return null; - } else if (KEY_GROUPS_HANDLE == type) { - - int startKeyGroup = dis.readInt(); - int numKeyGroups = dis.readInt(); - KeyGroupRange keyGroupRange = - KeyGroupRange.of(startKeyGroup, startKeyGroup + numKeyGroups - 1); - long[] offsets = new long[numKeyGroups]; - for (int i = 0; i < numKeyGroups; ++i) { - offsets[i] = dis.readLong(); - } - KeyGroupRangeOffsets keyGroupRangeOffsets = new KeyGroupRangeOffsets( - keyGroupRange, offsets); - StreamStateHandle stateHandle = deserializeStreamStateHandle(dis); - return new KeyGroupsStateHandle(keyGroupRangeOffsets, stateHandle); - } else if (INCREMENTAL_KEY_GROUPS_HANDLE == type) { - - long checkpointId = dis.readLong(); - String backendId = dis.readUTF(); - int startKeyGroup = dis.readInt(); - int numKeyGroups = dis.readInt(); - KeyGroupRange keyGroupRange = - KeyGroupRange.of(startKeyGroup, startKeyGroup + numKeyGroups - 1); - - StreamStateHandle metaDataStateHandle = deserializeStreamStateHandle(dis); - Map sharedStates = deserializeStreamStateHandleMap(dis); - Map privateStates = deserializeStreamStateHandleMap(dis); - - UUID uuid; - - try { - uuid = UUID.fromString(backendId); - } catch (Exception ex) { - // compatibility with old format pre FLINK-6964: - uuid = UUID.nameUUIDFromBytes(backendId.getBytes(StandardCharsets.UTF_8)); - } - - return new IncrementalRemoteKeyedStateHandle( - uuid, - keyGroupRange, - checkpointId, - sharedStates, - privateStates, - metaDataStateHandle); - } else { - throw new IllegalStateException("Reading invalid KeyedStateHandle, type: " + type); - } - } - - @VisibleForTesting - public static void serializeOperatorStateHandle( - OperatorStateHandle stateHandle, DataOutputStream dos) throws IOException { - - if (stateHandle != null) { - dos.writeByte(PARTITIONABLE_OPERATOR_STATE_HANDLE); - Map partitionOffsetsMap = - stateHandle.getStateNameToPartitionOffsets(); - dos.writeInt(partitionOffsetsMap.size()); - for (Map.Entry entry : partitionOffsetsMap.entrySet()) { - dos.writeUTF(entry.getKey()); - - OperatorStateHandle.StateMetaInfo stateMetaInfo = entry.getValue(); - - int mode = stateMetaInfo.getDistributionMode().ordinal(); - dos.writeByte(mode); - - long[] offsets = stateMetaInfo.getOffsets(); - dos.writeInt(offsets.length); - for (long offset : offsets) { - dos.writeLong(offset); - } - } - serializeStreamStateHandle(stateHandle.getDelegateStateHandle(), dos); - } else { - dos.writeByte(NULL_HANDLE); - } - } - - @VisibleForTesting - public static OperatorStateHandle deserializeOperatorStateHandle( - DataInputStream dis) throws IOException { - - final int type = dis.readByte(); - if (NULL_HANDLE == type) { - return null; - } else if (PARTITIONABLE_OPERATOR_STATE_HANDLE == type) { - int mapSize = dis.readInt(); - Map offsetsMap = new HashMap<>(mapSize); - for (int i = 0; i < mapSize; ++i) { - String key = dis.readUTF(); - - int modeOrdinal = dis.readByte(); - OperatorStateHandle.Mode mode = OperatorStateHandle.Mode.values()[modeOrdinal]; - - long[] offsets = new long[dis.readInt()]; - for (int j = 0; j < offsets.length; ++j) { - offsets[j] = dis.readLong(); - } - - OperatorStateHandle.StateMetaInfo metaInfo = - new OperatorStateHandle.StateMetaInfo(offsets, mode); - offsetsMap.put(key, metaInfo); - } - StreamStateHandle stateHandle = deserializeStreamStateHandle(dis); - return new OperatorStreamStateHandle(offsetsMap, stateHandle); - } else { - throw new IllegalStateException("Reading invalid OperatorStateHandle, type: " + type); - } - } - - @VisibleForTesting - public static void serializeStreamStateHandle( - StreamStateHandle stateHandle, DataOutputStream dos) throws IOException { - - if (stateHandle == null) { - dos.writeByte(NULL_HANDLE); - - } else if (stateHandle instanceof FileStateHandle) { - dos.writeByte(FILE_STREAM_STATE_HANDLE); - FileStateHandle fileStateHandle = (FileStateHandle) stateHandle; - dos.writeLong(stateHandle.getStateSize()); - dos.writeUTF(fileStateHandle.getFilePath().toString()); - - } else if (stateHandle instanceof ByteStreamStateHandle) { - dos.writeByte(BYTE_STREAM_STATE_HANDLE); - ByteStreamStateHandle byteStreamStateHandle = (ByteStreamStateHandle) stateHandle; - dos.writeUTF(byteStreamStateHandle.getHandleName()); - byte[] internalData = byteStreamStateHandle.getData(); - dos.writeInt(internalData.length); - dos.write(byteStreamStateHandle.getData()); - } else { - throw new IOException("Unknown implementation of StreamStateHandle: " + stateHandle.getClass()); - } - - dos.flush(); - } - - public static StreamStateHandle deserializeStreamStateHandle(DataInputStream dis) throws IOException { - final int type = dis.read(); - if (NULL_HANDLE == type) { - return null; - } else if (FILE_STREAM_STATE_HANDLE == type) { - long size = dis.readLong(); - String pathString = dis.readUTF(); - return new FileStateHandle(new Path(pathString), size); - } else if (BYTE_STREAM_STATE_HANDLE == type) { - String handleName = dis.readUTF(); - int numBytes = dis.readInt(); - byte[] data = new byte[numBytes]; - dis.readFully(data); - return new ByteStreamStateHandle(handleName, data); - } else { - throw new IOException("Unknown implementation of StreamStateHandle, code: " + type); - } + public CheckpointMetadata deserialize(DataInputStream dis, ClassLoader classLoader) throws IOException { + return deserializeMetadata(dis); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2V3SerializerBase.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2V3SerializerBase.java new file mode 100644 index 0000000000..7e66448355 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2V3SerializerBase.java @@ -0,0 +1,555 @@ +/* + * 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.runtime.checkpoint.metadata; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.checkpoint.MasterState; +import org.apache.flink.runtime.checkpoint.OperatorState; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.state.IncrementalRemoteKeyedStateHandle; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.KeyGroupRangeOffsets; +import org.apache.flink.runtime.state.KeyGroupsStateHandle; +import org.apache.flink.runtime.state.KeyedStateHandle; +import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.OperatorStreamStateHandle; +import org.apache.flink.runtime.state.StateHandleID; +import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.runtime.state.filesystem.FileStateHandle; +import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; + +import javax.annotation.Nullable; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; + +/** + * Base (De)serializer for checkpoint metadata format version 2 and 3. + * + *

    The difference between versions 2 and 3 is minor. Version 3 includes + * operator coordinator state for each operator, and drops some minor unused fields. + * + *

    Basic checkpoint metadata layout: + *

    + *  +--------------+---------------+-----------------+
    + *  | checkpointID | master states | operator states |
    + *  +--------------+---------------+-----------------+
    + *
    + *  Master state:
    + *  +--------------+---------------------+---------+------+---------------+
    + *  | magic number | num remaining bytes | version | name | payload bytes |
    + *  +--------------+---------------------+---------+------+---------------+
    + * 
    + */ +@Internal +public abstract class MetadataV2V3SerializerBase { + + /** Random magic number for consistency checks. */ + private static final int MASTER_STATE_MAGIC_NUMBER = 0xc96b1696; + + private static final byte NULL_HANDLE = 0; + private static final byte BYTE_STREAM_STATE_HANDLE = 1; + private static final byte FILE_STREAM_STATE_HANDLE = 2; + private static final byte KEY_GROUPS_HANDLE = 3; + private static final byte PARTITIONABLE_OPERATOR_STATE_HANDLE = 4; + private static final byte INCREMENTAL_KEY_GROUPS_HANDLE = 5; + + // ------------------------------------------------------------------------ + // (De)serialization entry points + // ------------------------------------------------------------------------ + + protected void serializeMetadata(CheckpointMetadata checkpointMetadata, DataOutputStream dos) throws IOException { + // first: checkpoint ID + dos.writeLong(checkpointMetadata.getCheckpointId()); + + // second: master state + final Collection masterStates = checkpointMetadata.getMasterStates(); + dos.writeInt(masterStates.size()); + for (MasterState ms : masterStates) { + serializeMasterState(ms, dos); + } + + // third: operator states + Collection operatorStates = checkpointMetadata.getOperatorStates(); + dos.writeInt(operatorStates.size()); + + for (OperatorState operatorState : operatorStates) { + serializeOperatorState(operatorState, dos); + } + } + + protected CheckpointMetadata deserializeMetadata(DataInputStream dis) throws IOException { + // first: checkpoint ID + final long checkpointId = dis.readLong(); + if (checkpointId < 0) { + throw new IOException("invalid checkpoint ID: " + checkpointId); + } + + // second: master state + final List masterStates; + final int numMasterStates = dis.readInt(); + + if (numMasterStates == 0) { + masterStates = Collections.emptyList(); + } + else if (numMasterStates > 0) { + masterStates = new ArrayList<>(numMasterStates); + for (int i = 0; i < numMasterStates; i++) { + masterStates.add(deserializeMasterState(dis)); + } + } + else { + throw new IOException("invalid number of master states: " + numMasterStates); + } + + // third: operator states + final int numTaskStates = dis.readInt(); + final List operatorStates = new ArrayList<>(numTaskStates); + + for (int i = 0; i < numTaskStates; i++) { + operatorStates.add(deserializeOperatorState(dis)); + } + + return new CheckpointMetadata(checkpointId, operatorStates, masterStates); + } + + // ------------------------------------------------------------------------ + // master state (de)serialization methods + // ------------------------------------------------------------------------ + + protected void serializeMasterState(MasterState state, DataOutputStream dos) throws IOException { + // magic number for error detection + dos.writeInt(MASTER_STATE_MAGIC_NUMBER); + + // for safety, we serialize first into an array and then write the array and its + // length into the checkpoint + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final DataOutputStream out = new DataOutputStream(baos); + + out.writeInt(state.version()); + out.writeUTF(state.name()); + + final byte[] bytes = state.bytes(); + out.writeInt(bytes.length); + out.write(bytes, 0, bytes.length); + + out.close(); + byte[] data = baos.toByteArray(); + + dos.writeInt(data.length); + dos.write(data, 0, data.length); + } + + protected MasterState deserializeMasterState(DataInputStream dis) throws IOException { + final int magicNumber = dis.readInt(); + if (magicNumber != MASTER_STATE_MAGIC_NUMBER) { + throw new IOException("incorrect magic number in master styte byte sequence"); + } + + final int numBytes = dis.readInt(); + if (numBytes <= 0) { + throw new IOException("found zero or negative length for master state bytes"); + } + + final byte[] data = new byte[numBytes]; + dis.readFully(data); + + final DataInputStream in = new DataInputStream(new ByteArrayInputStream(data)); + + final int version = in.readInt(); + final String name = in.readUTF(); + + final byte[] bytes = new byte[in.readInt()]; + in.readFully(bytes); + + // check that the data is not corrupt + if (in.read() != -1) { + throw new IOException("found trailing bytes in master state"); + } + + return new MasterState(name, bytes, version); + } + + // ------------------------------------------------------------------------ + // operator state (de)serialization methods + // ------------------------------------------------------------------------ + + protected void serializeOperatorState(OperatorState operatorState, DataOutputStream dos) throws IOException { + // Operator ID + dos.writeLong(operatorState.getOperatorID().getLowerPart()); + dos.writeLong(operatorState.getOperatorID().getUpperPart()); + + // Parallelism + int parallelism = operatorState.getParallelism(); + dos.writeInt(parallelism); + dos.writeInt(operatorState.getMaxParallelism()); + + // this field was "chain length" before Flink 1.3, and it is still part + // of the format, despite being unused + dos.writeInt(1); + + // Sub task states + Map subtaskStateMap = operatorState.getSubtaskStates(); + dos.writeInt(subtaskStateMap.size()); + for (Map.Entry entry : subtaskStateMap.entrySet()) { + dos.writeInt(entry.getKey()); + serializeSubtaskState(entry.getValue(), dos); + } + } + + protected OperatorState deserializeOperatorState(DataInputStream dis) throws IOException { + final OperatorID jobVertexId = new OperatorID(dis.readLong(), dis.readLong()); + final int parallelism = dis.readInt(); + final int maxParallelism = dis.readInt(); + + // this field was "chain length" before Flink 1.3, and it is still part + // of the format, despite being unused + dis.readInt(); + + // Add task state + final OperatorState taskState = new OperatorState(jobVertexId, parallelism, maxParallelism); + + // Sub task states + final int numSubTaskStates = dis.readInt(); + + for (int j = 0; j < numSubTaskStates; j++) { + final int subtaskIndex = dis.readInt(); + final OperatorSubtaskState subtaskState = deserializeSubtaskState(dis); + taskState.putState(subtaskIndex, subtaskState); + } + + return taskState; + } + + // ------------------------------------------------------------------------ + // operator subtask state (de)serialization methods + // ------------------------------------------------------------------------ + + protected void serializeSubtaskState(OperatorSubtaskState subtaskState, DataOutputStream dos) throws IOException { + + dos.writeLong(-1); + + int len = 0; + dos.writeInt(len); + + OperatorStateHandle operatorStateBackend = extractSingleton(subtaskState.getManagedOperatorState()); + + len = operatorStateBackend != null ? 1 : 0; + dos.writeInt(len); + if (len == 1) { + serializeOperatorStateHandle(operatorStateBackend, dos); + } + + OperatorStateHandle operatorStateFromStream = extractSingleton(subtaskState.getRawOperatorState()); + + len = operatorStateFromStream != null ? 1 : 0; + dos.writeInt(len); + if (len == 1) { + serializeOperatorStateHandle(operatorStateFromStream, dos); + } + + KeyedStateHandle keyedStateBackend = extractSingleton(subtaskState.getManagedKeyedState()); + serializeKeyedStateHandle(keyedStateBackend, dos); + + KeyedStateHandle keyedStateStream = extractSingleton(subtaskState.getRawKeyedState()); + serializeKeyedStateHandle(keyedStateStream, dos); + } + + protected OperatorSubtaskState deserializeSubtaskState(DataInputStream dis) throws IOException { + // Duration field has been removed from SubtaskState, do not remove + long ignoredDuration = dis.readLong(); + + final int numLegacyTaskStates = dis.readInt(); + if (numLegacyTaskStates > 0) { + throw new IOException( + "Legacy state (from Flink <= 1.1, created through the 'Checkpointed' interface) is " + + "no longer supported."); + } + + int len = dis.readInt(); + OperatorStateHandle operatorStateBackend = len == 0 ? null : deserializeOperatorStateHandle(dis); + + len = dis.readInt(); + OperatorStateHandle operatorStateStream = len == 0 ? null : deserializeOperatorStateHandle(dis); + + KeyedStateHandle keyedStateBackend = deserializeKeyedStateHandle(dis); + + KeyedStateHandle keyedStateStream = deserializeKeyedStateHandle(dis); + + return new OperatorSubtaskState( + operatorStateBackend, + operatorStateStream, + keyedStateBackend, + keyedStateStream); + } + + @VisibleForTesting + public static void serializeKeyedStateHandle( + KeyedStateHandle stateHandle, DataOutputStream dos) throws IOException { + + if (stateHandle == null) { + dos.writeByte(NULL_HANDLE); + } else if (stateHandle instanceof KeyGroupsStateHandle) { + KeyGroupsStateHandle keyGroupsStateHandle = (KeyGroupsStateHandle) stateHandle; + + dos.writeByte(KEY_GROUPS_HANDLE); + dos.writeInt(keyGroupsStateHandle.getKeyGroupRange().getStartKeyGroup()); + dos.writeInt(keyGroupsStateHandle.getKeyGroupRange().getNumberOfKeyGroups()); + for (int keyGroup : keyGroupsStateHandle.getKeyGroupRange()) { + dos.writeLong(keyGroupsStateHandle.getOffsetForKeyGroup(keyGroup)); + } + serializeStreamStateHandle(keyGroupsStateHandle.getDelegateStateHandle(), dos); + } else if (stateHandle instanceof IncrementalRemoteKeyedStateHandle) { + IncrementalRemoteKeyedStateHandle incrementalKeyedStateHandle = + (IncrementalRemoteKeyedStateHandle) stateHandle; + + dos.writeByte(INCREMENTAL_KEY_GROUPS_HANDLE); + + dos.writeLong(incrementalKeyedStateHandle.getCheckpointId()); + dos.writeUTF(String.valueOf(incrementalKeyedStateHandle.getBackendIdentifier())); + dos.writeInt(incrementalKeyedStateHandle.getKeyGroupRange().getStartKeyGroup()); + dos.writeInt(incrementalKeyedStateHandle.getKeyGroupRange().getNumberOfKeyGroups()); + + serializeStreamStateHandle(incrementalKeyedStateHandle.getMetaStateHandle(), dos); + + serializeStreamStateHandleMap(incrementalKeyedStateHandle.getSharedState(), dos); + serializeStreamStateHandleMap(incrementalKeyedStateHandle.getPrivateState(), dos); + } else { + throw new IllegalStateException("Unknown KeyedStateHandle type: " + stateHandle.getClass()); + } + } + + private static void serializeStreamStateHandleMap( + Map map, + DataOutputStream dos) throws IOException { + + dos.writeInt(map.size()); + for (Map.Entry entry : map.entrySet()) { + dos.writeUTF(entry.getKey().toString()); + serializeStreamStateHandle(entry.getValue(), dos); + } + } + + private static Map deserializeStreamStateHandleMap( + DataInputStream dis) throws IOException { + + final int size = dis.readInt(); + Map result = new HashMap<>(size); + + for (int i = 0; i < size; ++i) { + StateHandleID stateHandleID = new StateHandleID(dis.readUTF()); + StreamStateHandle stateHandle = deserializeStreamStateHandle(dis); + result.put(stateHandleID, stateHandle); + } + + return result; + } + + @VisibleForTesting + public static KeyedStateHandle deserializeKeyedStateHandle(DataInputStream dis) throws IOException { + final int type = dis.readByte(); + if (NULL_HANDLE == type) { + + return null; + } else if (KEY_GROUPS_HANDLE == type) { + + int startKeyGroup = dis.readInt(); + int numKeyGroups = dis.readInt(); + KeyGroupRange keyGroupRange = + KeyGroupRange.of(startKeyGroup, startKeyGroup + numKeyGroups - 1); + long[] offsets = new long[numKeyGroups]; + for (int i = 0; i < numKeyGroups; ++i) { + offsets[i] = dis.readLong(); + } + KeyGroupRangeOffsets keyGroupRangeOffsets = new KeyGroupRangeOffsets( + keyGroupRange, offsets); + StreamStateHandle stateHandle = deserializeStreamStateHandle(dis); + return new KeyGroupsStateHandle(keyGroupRangeOffsets, stateHandle); + } else if (INCREMENTAL_KEY_GROUPS_HANDLE == type) { + + long checkpointId = dis.readLong(); + String backendId = dis.readUTF(); + int startKeyGroup = dis.readInt(); + int numKeyGroups = dis.readInt(); + KeyGroupRange keyGroupRange = + KeyGroupRange.of(startKeyGroup, startKeyGroup + numKeyGroups - 1); + + StreamStateHandle metaDataStateHandle = deserializeStreamStateHandle(dis); + Map sharedStates = deserializeStreamStateHandleMap(dis); + Map privateStates = deserializeStreamStateHandleMap(dis); + + UUID uuid; + + try { + uuid = UUID.fromString(backendId); + } catch (Exception ex) { + // compatibility with old format pre FLINK-6964: + uuid = UUID.nameUUIDFromBytes(backendId.getBytes(StandardCharsets.UTF_8)); + } + + return new IncrementalRemoteKeyedStateHandle( + uuid, + keyGroupRange, + checkpointId, + sharedStates, + privateStates, + metaDataStateHandle); + } else { + throw new IllegalStateException("Reading invalid KeyedStateHandle, type: " + type); + } + } + + @VisibleForTesting + public static void serializeOperatorStateHandle( + OperatorStateHandle stateHandle, DataOutputStream dos) throws IOException { + + if (stateHandle != null) { + dos.writeByte(PARTITIONABLE_OPERATOR_STATE_HANDLE); + Map partitionOffsetsMap = + stateHandle.getStateNameToPartitionOffsets(); + dos.writeInt(partitionOffsetsMap.size()); + for (Map.Entry entry : partitionOffsetsMap.entrySet()) { + dos.writeUTF(entry.getKey()); + + OperatorStateHandle.StateMetaInfo stateMetaInfo = entry.getValue(); + + int mode = stateMetaInfo.getDistributionMode().ordinal(); + dos.writeByte(mode); + + long[] offsets = stateMetaInfo.getOffsets(); + dos.writeInt(offsets.length); + for (long offset : offsets) { + dos.writeLong(offset); + } + } + serializeStreamStateHandle(stateHandle.getDelegateStateHandle(), dos); + } else { + dos.writeByte(NULL_HANDLE); + } + } + + @VisibleForTesting + public static OperatorStateHandle deserializeOperatorStateHandle( + DataInputStream dis) throws IOException { + + final int type = dis.readByte(); + if (NULL_HANDLE == type) { + return null; + } else if (PARTITIONABLE_OPERATOR_STATE_HANDLE == type) { + int mapSize = dis.readInt(); + Map offsetsMap = new HashMap<>(mapSize); + for (int i = 0; i < mapSize; ++i) { + String key = dis.readUTF(); + + int modeOrdinal = dis.readByte(); + OperatorStateHandle.Mode mode = OperatorStateHandle.Mode.values()[modeOrdinal]; + + long[] offsets = new long[dis.readInt()]; + for (int j = 0; j < offsets.length; ++j) { + offsets[j] = dis.readLong(); + } + + OperatorStateHandle.StateMetaInfo metaInfo = + new OperatorStateHandle.StateMetaInfo(offsets, mode); + offsetsMap.put(key, metaInfo); + } + StreamStateHandle stateHandle = deserializeStreamStateHandle(dis); + return new OperatorStreamStateHandle(offsetsMap, stateHandle); + } else { + throw new IllegalStateException("Reading invalid OperatorStateHandle, type: " + type); + } + } + + @VisibleForTesting + public static void serializeStreamStateHandle( + StreamStateHandle stateHandle, DataOutputStream dos) throws IOException { + + if (stateHandle == null) { + dos.writeByte(NULL_HANDLE); + + } else if (stateHandle instanceof FileStateHandle) { + dos.writeByte(FILE_STREAM_STATE_HANDLE); + FileStateHandle fileStateHandle = (FileStateHandle) stateHandle; + dos.writeLong(stateHandle.getStateSize()); + dos.writeUTF(fileStateHandle.getFilePath().toString()); + + } else if (stateHandle instanceof ByteStreamStateHandle) { + dos.writeByte(BYTE_STREAM_STATE_HANDLE); + ByteStreamStateHandle byteStreamStateHandle = (ByteStreamStateHandle) stateHandle; + dos.writeUTF(byteStreamStateHandle.getHandleName()); + byte[] internalData = byteStreamStateHandle.getData(); + dos.writeInt(internalData.length); + dos.write(byteStreamStateHandle.getData()); + } else { + throw new IOException("Unknown implementation of StreamStateHandle: " + stateHandle.getClass()); + } + + dos.flush(); + } + + public static StreamStateHandle deserializeStreamStateHandle(DataInputStream dis) throws IOException { + final int type = dis.read(); + if (NULL_HANDLE == type) { + return null; + } else if (FILE_STREAM_STATE_HANDLE == type) { + long size = dis.readLong(); + String pathString = dis.readUTF(); + return new FileStateHandle(new Path(pathString), size); + } else if (BYTE_STREAM_STATE_HANDLE == type) { + String handleName = dis.readUTF(); + int numBytes = dis.readInt(); + byte[] data = new byte[numBytes]; + dis.readFully(data); + return new ByteStreamStateHandle(handleName, data); + } else { + throw new IOException("Unknown implementation of StreamStateHandle, code: " + type); + } + } + + // ------------------------------------------------------------------------ + // utilities + // ------------------------------------------------------------------------ + + @Nullable + private static T extractSingleton(Collection collection) { + if (collection == null || collection.isEmpty()) { + return null; + } + + if (collection.size() == 1) { + return collection.iterator().next(); + } else { + throw new IllegalStateException("Expected singleton collection, but found size: " + collection.size()); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV3Serializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV3Serializer.java new file mode 100644 index 0000000000..f9e750a89f --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV3Serializer.java @@ -0,0 +1,65 @@ +/* + * 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.runtime.checkpoint.metadata; + +import org.apache.flink.annotation.Internal; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; + +/** + * (De)serializer for checkpoint metadata format version 3. + * This format was introduced with Apache Flink 1.11.0. + * + *

    Compared to format version 2, this drops some unused fields and introduces + * operator coordinator state. + * + *

    See {@link MetadataV2V3SerializerBase} for a description of the format layout. + */ +@Internal +public class MetadataV3Serializer extends MetadataV2V3SerializerBase implements MetadataSerializer { + + /** The metadata format version. */ + public static final int VERSION = 3; + + /** The singleton instance of the serializer. */ + public static final MetadataV3Serializer INSTANCE = new MetadataV3Serializer(); + + /** Singleton, not meant to be instantiated. */ + private MetadataV3Serializer() {} + + @Override + public int getVersion() { + return VERSION; + } + + // ------------------------------------------------------------------------ + // (De)serialization entry points + // ------------------------------------------------------------------------ + + public static void serialize(CheckpointMetadata checkpointMetadata, DataOutputStream dos) throws IOException { + INSTANCE.serializeMetadata(checkpointMetadata, dos); + } + + @Override + public CheckpointMetadata deserialize(DataInputStream dis, ClassLoader classLoader) throws IOException { + return deserializeMetadata(dis); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2SerializerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV3SerializerTest.java similarity index 98% rename from flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2SerializerTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV3SerializerTest.java index c1da0ba856..0147e5495a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2SerializerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV3SerializerTest.java @@ -40,7 +40,7 @@ import static org.junit.Assert.assertEquals; /** * Various tests for the version 2 format serializer of a checkpoint. */ -public class MetadataV2SerializerTest { +public class MetadataV3SerializerTest { @Test public void testCheckpointWithNoState() throws Exception { @@ -122,7 +122,7 @@ public class MetadataV2SerializerTest { Collection operatorStates, Collection masterStates) throws IOException { - MetadataV2Serializer serializer = MetadataV2Serializer.INSTANCE; + MetadataV3Serializer serializer = MetadataV3Serializer.INSTANCE; ByteArrayOutputStreamWithPos baos = new ByteArrayOutputStreamWithPos(); DataOutputStream out = new DataOutputViewStreamWrapper(baos); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OperatorSnapshotUtil.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OperatorSnapshotUtil.java index 3b79fc6c4e..41c8bc1781 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OperatorSnapshotUtil.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OperatorSnapshotUtil.java @@ -20,7 +20,7 @@ package org.apache.flink.streaming.util; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.runtime.checkpoint.StateObjectCollection; -import org.apache.flink.runtime.checkpoint.metadata.MetadataV2Serializer; +import org.apache.flink.runtime.checkpoint.metadata.MetadataV3Serializer; import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.OperatorStateHandle; @@ -55,13 +55,13 @@ public class OperatorSnapshotUtil { dos.writeInt(0); // still required for compatibility - MetadataV2Serializer.serializeStreamStateHandle(null, dos); + MetadataV3Serializer.serializeStreamStateHandle(null, dos); Collection rawOperatorState = state.getRawOperatorState(); if (rawOperatorState != null) { dos.writeInt(rawOperatorState.size()); for (OperatorStateHandle operatorStateHandle : rawOperatorState) { - MetadataV2Serializer.serializeOperatorStateHandle(operatorStateHandle, dos); + MetadataV3Serializer.serializeOperatorStateHandle(operatorStateHandle, dos); } } else { // this means no states, not even an empty list @@ -72,7 +72,7 @@ public class OperatorSnapshotUtil { if (managedOperatorState != null) { dos.writeInt(managedOperatorState.size()); for (OperatorStateHandle operatorStateHandle : managedOperatorState) { - MetadataV2Serializer.serializeOperatorStateHandle(operatorStateHandle, dos); + MetadataV3Serializer.serializeOperatorStateHandle(operatorStateHandle, dos); } } else { // this means no states, not even an empty list @@ -83,7 +83,7 @@ public class OperatorSnapshotUtil { if (rawKeyedState != null) { dos.writeInt(rawKeyedState.size()); for (KeyedStateHandle keyedStateHandle : rawKeyedState) { - MetadataV2Serializer.serializeKeyedStateHandle(keyedStateHandle, dos); + MetadataV3Serializer.serializeKeyedStateHandle(keyedStateHandle, dos); } } else { // this means no operator states, not even an empty list @@ -94,7 +94,7 @@ public class OperatorSnapshotUtil { if (managedKeyedState != null) { dos.writeInt(managedKeyedState.size()); for (KeyedStateHandle keyedStateHandle : managedKeyedState) { - MetadataV2Serializer.serializeKeyedStateHandle(keyedStateHandle, dos); + MetadataV3Serializer.serializeKeyedStateHandle(keyedStateHandle, dos); } } else { // this means no operator states, not even an empty list @@ -113,14 +113,14 @@ public class OperatorSnapshotUtil { dis.readInt(); // still required for compatibility to consume the bytes. - MetadataV2Serializer.deserializeStreamStateHandle(dis); + MetadataV3Serializer.deserializeStreamStateHandle(dis); List rawOperatorState = null; int numRawOperatorStates = dis.readInt(); if (numRawOperatorStates >= 0) { rawOperatorState = new ArrayList<>(); for (int i = 0; i < numRawOperatorStates; i++) { - OperatorStateHandle operatorState = MetadataV2Serializer.deserializeOperatorStateHandle( + OperatorStateHandle operatorState = MetadataV3Serializer.deserializeOperatorStateHandle( dis); rawOperatorState.add(operatorState); } @@ -131,7 +131,7 @@ public class OperatorSnapshotUtil { if (numManagedOperatorStates >= 0) { managedOperatorState = new ArrayList<>(); for (int i = 0; i < numManagedOperatorStates; i++) { - OperatorStateHandle operatorState = MetadataV2Serializer.deserializeOperatorStateHandle( + OperatorStateHandle operatorState = MetadataV3Serializer.deserializeOperatorStateHandle( dis); managedOperatorState.add(operatorState); } @@ -142,7 +142,7 @@ public class OperatorSnapshotUtil { if (numRawKeyedStates >= 0) { rawKeyedState = new ArrayList<>(); for (int i = 0; i < numRawKeyedStates; i++) { - KeyedStateHandle keyedState = MetadataV2Serializer.deserializeKeyedStateHandle( + KeyedStateHandle keyedState = MetadataV3Serializer.deserializeKeyedStateHandle( dis); rawKeyedState.add(keyedState); } @@ -153,7 +153,7 @@ public class OperatorSnapshotUtil { if (numManagedKeyedStates >= 0) { managedKeyedState = new ArrayList<>(); for (int i = 0; i < numManagedKeyedStates; i++) { - KeyedStateHandle keyedState = MetadataV2Serializer.deserializeKeyedStateHandle( + KeyedStateHandle keyedState = MetadataV3Serializer.deserializeKeyedStateHandle( dis); managedKeyedState.add(keyedState); } -- Gitee From a05f4565354479274dfbcd1fa301333ba9c902cf Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Mon, 24 Feb 2020 21:12:59 +0100 Subject: [PATCH 076/885] [FLINK-16177][refactor][checkpointing] Remove unused fields from Metadata Serializer version 3 This also cleans up some code and names in the shared SubtaskState serialization code. --- .../metadata/MetadataV2Serializer.java | 86 +++++++++++++ .../metadata/MetadataV2V3SerializerBase.java | 117 +++++------------- .../metadata/MetadataV3Serializer.java | 48 +++++++ 3 files changed, 163 insertions(+), 88 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2Serializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2Serializer.java index bddceb84fa..bbf25d8906 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2Serializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2Serializer.java @@ -19,9 +19,14 @@ package org.apache.flink.runtime.checkpoint.metadata; import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.checkpoint.OperatorState; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.jobgraph.OperatorID; import java.io.DataInputStream; +import java.io.DataOutputStream; import java.io.IOException; +import java.util.Map; /** * (De)serializer for checkpoint metadata format version 2. @@ -54,4 +59,85 @@ public class MetadataV2Serializer extends MetadataV2V3SerializerBase implements public CheckpointMetadata deserialize(DataInputStream dis, ClassLoader classLoader) throws IOException { return deserializeMetadata(dis); } + + // ------------------------------------------------------------------------ + // version-specific serialization + // ------------------------------------------------------------------------ + + @Override + protected void serializeOperatorState(OperatorState operatorState, DataOutputStream dos) throws IOException { + // Operator ID + dos.writeLong(operatorState.getOperatorID().getLowerPart()); + dos.writeLong(operatorState.getOperatorID().getUpperPart()); + + // Parallelism + int parallelism = operatorState.getParallelism(); + dos.writeInt(parallelism); + dos.writeInt(operatorState.getMaxParallelism()); + + // this field was "chain length" before Flink 1.3, and it is still part + // of the format, despite being unused + dos.writeInt(1); + + // Sub task states + Map subtaskStateMap = operatorState.getSubtaskStates(); + dos.writeInt(subtaskStateMap.size()); + for (Map.Entry entry : subtaskStateMap.entrySet()) { + dos.writeInt(entry.getKey()); + serializeSubtaskState(entry.getValue(), dos); + } + } + + @Override + protected OperatorState deserializeOperatorState(DataInputStream dis) throws IOException { + final OperatorID jobVertexId = new OperatorID(dis.readLong(), dis.readLong()); + final int parallelism = dis.readInt(); + final int maxParallelism = dis.readInt(); + + // this field was "chain length" before Flink 1.3, and it is still part + // of the format, despite being unused + dis.readInt(); + + // Add task state + final OperatorState taskState = new OperatorState(jobVertexId, parallelism, maxParallelism); + + // Sub task states + final int numSubTaskStates = dis.readInt(); + + for (int j = 0; j < numSubTaskStates; j++) { + final int subtaskIndex = dis.readInt(); + final OperatorSubtaskState subtaskState = deserializeSubtaskState(dis); + taskState.putState(subtaskIndex, subtaskState); + } + + return taskState; + } + + @Override + protected void serializeSubtaskState(OperatorSubtaskState subtaskState, DataOutputStream dos) throws IOException { + // write two unused fields for compatibility: + // - "duration" + // - number of legacy states + dos.writeLong(-1); + dos.writeInt(0); + + super.serializeSubtaskState(subtaskState, dos); + } + + @Override + protected OperatorSubtaskState deserializeSubtaskState(DataInputStream dis) throws IOException { + // read two unused fields for compatibility: + // - "duration" + // - number of legacy states + dis.readLong(); + final int numLegacyTaskStates = dis.readInt(); + + if (numLegacyTaskStates > 0) { + throw new IOException( + "Legacy state (from Flink <= 1.1, created through the 'Checkpointed' interface) is " + + "no longer supported."); + } + + return super.deserializeSubtaskState(dis); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2V3SerializerBase.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2V3SerializerBase.java index 7e66448355..3b93b8c031 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2V3SerializerBase.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2V3SerializerBase.java @@ -24,7 +24,6 @@ import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.checkpoint.MasterState; import org.apache.flink.runtime.checkpoint.OperatorState; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; -import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.state.IncrementalRemoteKeyedStateHandle; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.KeyGroupRangeOffsets; @@ -204,113 +203,55 @@ public abstract class MetadataV2V3SerializerBase { // operator state (de)serialization methods // ------------------------------------------------------------------------ - protected void serializeOperatorState(OperatorState operatorState, DataOutputStream dos) throws IOException { - // Operator ID - dos.writeLong(operatorState.getOperatorID().getLowerPart()); - dos.writeLong(operatorState.getOperatorID().getUpperPart()); - - // Parallelism - int parallelism = operatorState.getParallelism(); - dos.writeInt(parallelism); - dos.writeInt(operatorState.getMaxParallelism()); - - // this field was "chain length" before Flink 1.3, and it is still part - // of the format, despite being unused - dos.writeInt(1); - - // Sub task states - Map subtaskStateMap = operatorState.getSubtaskStates(); - dos.writeInt(subtaskStateMap.size()); - for (Map.Entry entry : subtaskStateMap.entrySet()) { - dos.writeInt(entry.getKey()); - serializeSubtaskState(entry.getValue(), dos); - } - } - - protected OperatorState deserializeOperatorState(DataInputStream dis) throws IOException { - final OperatorID jobVertexId = new OperatorID(dis.readLong(), dis.readLong()); - final int parallelism = dis.readInt(); - final int maxParallelism = dis.readInt(); - - // this field was "chain length" before Flink 1.3, and it is still part - // of the format, despite being unused - dis.readInt(); - - // Add task state - final OperatorState taskState = new OperatorState(jobVertexId, parallelism, maxParallelism); + protected abstract void serializeOperatorState(OperatorState operatorState, DataOutputStream dos) throws IOException; - // Sub task states - final int numSubTaskStates = dis.readInt(); - - for (int j = 0; j < numSubTaskStates; j++) { - final int subtaskIndex = dis.readInt(); - final OperatorSubtaskState subtaskState = deserializeSubtaskState(dis); - taskState.putState(subtaskIndex, subtaskState); - } - - return taskState; - } + protected abstract OperatorState deserializeOperatorState(DataInputStream dis) throws IOException; // ------------------------------------------------------------------------ // operator subtask state (de)serialization methods // ------------------------------------------------------------------------ protected void serializeSubtaskState(OperatorSubtaskState subtaskState, DataOutputStream dos) throws IOException { - - dos.writeLong(-1); - - int len = 0; - dos.writeInt(len); - - OperatorStateHandle operatorStateBackend = extractSingleton(subtaskState.getManagedOperatorState()); - - len = operatorStateBackend != null ? 1 : 0; - dos.writeInt(len); - if (len == 1) { - serializeOperatorStateHandle(operatorStateBackend, dos); + final OperatorStateHandle managedOperatorState = extractSingleton(subtaskState.getManagedOperatorState()); + if (managedOperatorState != null) { + dos.writeInt(1); + serializeOperatorStateHandle(managedOperatorState, dos); + } else { + dos.writeInt(0); } - OperatorStateHandle operatorStateFromStream = extractSingleton(subtaskState.getRawOperatorState()); - - len = operatorStateFromStream != null ? 1 : 0; - dos.writeInt(len); - if (len == 1) { - serializeOperatorStateHandle(operatorStateFromStream, dos); + final OperatorStateHandle rawOperatorState = extractSingleton(subtaskState.getRawOperatorState()); + if (rawOperatorState != null) { + dos.writeInt(1); + serializeOperatorStateHandle(rawOperatorState, dos); + } else { + dos.writeInt(0); } - KeyedStateHandle keyedStateBackend = extractSingleton(subtaskState.getManagedKeyedState()); + final KeyedStateHandle keyedStateBackend = extractSingleton(subtaskState.getManagedKeyedState()); serializeKeyedStateHandle(keyedStateBackend, dos); - KeyedStateHandle keyedStateStream = extractSingleton(subtaskState.getRawKeyedState()); + final KeyedStateHandle keyedStateStream = extractSingleton(subtaskState.getRawKeyedState()); serializeKeyedStateHandle(keyedStateStream, dos); } protected OperatorSubtaskState deserializeSubtaskState(DataInputStream dis) throws IOException { - // Duration field has been removed from SubtaskState, do not remove - long ignoredDuration = dis.readLong(); - - final int numLegacyTaskStates = dis.readInt(); - if (numLegacyTaskStates > 0) { - throw new IOException( - "Legacy state (from Flink <= 1.1, created through the 'Checkpointed' interface) is " + - "no longer supported."); - } - - int len = dis.readInt(); - OperatorStateHandle operatorStateBackend = len == 0 ? null : deserializeOperatorStateHandle(dis); - - len = dis.readInt(); - OperatorStateHandle operatorStateStream = len == 0 ? null : deserializeOperatorStateHandle(dis); + final int hasManagedOperatorState = dis.readInt(); + final OperatorStateHandle managedOperatorState = hasManagedOperatorState == 0 ? + null : deserializeOperatorStateHandle(dis); - KeyedStateHandle keyedStateBackend = deserializeKeyedStateHandle(dis); + final int hasRawOperatorState = dis.readInt(); + final OperatorStateHandle rawOperatorState = hasRawOperatorState == 0 ? + null : deserializeOperatorStateHandle(dis); - KeyedStateHandle keyedStateStream = deserializeKeyedStateHandle(dis); + final KeyedStateHandle managedKeyedState = deserializeKeyedStateHandle(dis); + final KeyedStateHandle rawKeyedState = deserializeKeyedStateHandle(dis); return new OperatorSubtaskState( - operatorStateBackend, - operatorStateStream, - keyedStateBackend, - keyedStateStream); + managedOperatorState, + rawOperatorState, + managedKeyedState, + rawKeyedState); } @VisibleForTesting @@ -541,7 +482,7 @@ public abstract class MetadataV2V3SerializerBase { // ------------------------------------------------------------------------ @Nullable - private static T extractSingleton(Collection collection) { + static T extractSingleton(Collection collection) { if (collection == null || collection.isEmpty()) { return null; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV3Serializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV3Serializer.java index f9e750a89f..e304ef41f9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV3Serializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV3Serializer.java @@ -19,10 +19,14 @@ package org.apache.flink.runtime.checkpoint.metadata; import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.checkpoint.OperatorState; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.jobgraph.OperatorID; import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; +import java.util.Map; /** * (De)serializer for checkpoint metadata format version 3. @@ -62,4 +66,48 @@ public class MetadataV3Serializer extends MetadataV2V3SerializerBase implements public CheckpointMetadata deserialize(DataInputStream dis, ClassLoader classLoader) throws IOException { return deserializeMetadata(dis); } + + // ------------------------------------------------------------------------ + // version-specific serialization formats + // ------------------------------------------------------------------------ + + @Override + protected void serializeOperatorState(OperatorState operatorState, DataOutputStream dos) throws IOException { + // Operator ID + dos.writeLong(operatorState.getOperatorID().getLowerPart()); + dos.writeLong(operatorState.getOperatorID().getUpperPart()); + + // Parallelism + dos.writeInt(operatorState.getParallelism()); + dos.writeInt(operatorState.getMaxParallelism()); + + // Sub task states + final Map subtaskStateMap = operatorState.getSubtaskStates(); + dos.writeInt(subtaskStateMap.size()); + for (Map.Entry entry : subtaskStateMap.entrySet()) { + dos.writeInt(entry.getKey()); + serializeSubtaskState(entry.getValue(), dos); + } + } + + @Override + protected OperatorState deserializeOperatorState(DataInputStream dis) throws IOException { + final OperatorID jobVertexId = new OperatorID(dis.readLong(), dis.readLong()); + final int parallelism = dis.readInt(); + final int maxParallelism = dis.readInt(); + + // Add task state + final OperatorState operatorState = new OperatorState(jobVertexId, parallelism, maxParallelism); + + // Sub task states + final int numSubTaskStates = dis.readInt(); + + for (int j = 0; j < numSubTaskStates; j++) { + final int subtaskIndex = dis.readInt(); + final OperatorSubtaskState subtaskState = deserializeSubtaskState(dis); + operatorState.putState(subtaskIndex, subtaskState); + } + + return operatorState; + } } -- Gitee From add072264a4415be774099192408c13cfbfca320 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 27 Feb 2020 15:39:26 +0100 Subject: [PATCH 077/885] [hotfix][checkpointing] Minor code cleanups for 'checkpoint.metadata' test classes - fix checkstyle - Adjust names of tests with renamed main scope classes - remove test util methods that are no longer used after Legacy State was dropped. --- ...2Test.java => CheckpointMetadataTest.java} | 10 +- .../metadata/CheckpointTestUtils.java | 101 +----------------- .../metadata/MetadataV3SerializerTest.java | 4 +- 3 files changed, 10 insertions(+), 105 deletions(-) rename flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/metadata/{MetadataV2Test.java => CheckpointMetadataTest.java} (92%) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2Test.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/metadata/CheckpointMetadataTest.java similarity index 92% rename from flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2Test.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/metadata/CheckpointMetadataTest.java index ff93d5d2d4..f4cc7d4913 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2Test.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/metadata/CheckpointMetadataTest.java @@ -30,13 +30,13 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -public class MetadataV2Test { +/** + * Simple tests for the {@link CheckpointMetadata} data holder class. + */ +public class CheckpointMetadataTest { - /** - * Simple test of savepoint methods. - */ @Test - public void testSavepointV2() throws Exception { + public void testConstructAndDispose() throws Exception { final Random rnd = new Random(); final long checkpointId = rnd.nextInt(Integer.MAX_VALUE) + 1; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/metadata/CheckpointTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/metadata/CheckpointTestUtils.java index b526d0539e..f411a39f62 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/metadata/CheckpointTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/metadata/CheckpointTestUtils.java @@ -22,11 +22,7 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.runtime.checkpoint.MasterState; import org.apache.flink.runtime.checkpoint.OperatorState; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; -import org.apache.flink.runtime.checkpoint.SubtaskState; -import org.apache.flink.runtime.checkpoint.TaskState; -import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.flink.runtime.state.ChainedStateHandle; import org.apache.flink.runtime.state.IncrementalRemoteKeyedStateHandle; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.KeyGroupRangeOffsets; @@ -56,13 +52,6 @@ import static org.junit.Assert.assertEquals; */ public class CheckpointTestUtils { - /** - * Creates a random collection of OperatorState objects containing various types of state handles. - */ - public static Collection createOperatorStates(int numTaskStates, int numSubtasksPerTask) { - return createOperatorStates(new Random(), numTaskStates, numSubtasksPerTask); - } - /** * Creates a random collection of OperatorState objects containing various types of state handles. */ @@ -93,7 +82,7 @@ public class CheckpointTestUtils { OperatorStateHandle operatorStateHandleBackend = null; OperatorStateHandle operatorStateHandleStream = null; - + Map offsetsMap = new HashMap<>(); offsetsMap.put("A", new OperatorStateHandle.StateMetaInfo(new long[]{0, 10, 20}, OperatorStateHandle.Mode.SPLIT_DISTRIBUTE)); offsetsMap.put("B", new OperatorStateHandle.StateMetaInfo(new long[]{30, 40, 50}, OperatorStateHandle.Mode.SPLIT_DISTRIBUTE)); @@ -135,89 +124,6 @@ public class CheckpointTestUtils { return taskStates; } - /** - * Creates a random collection of TaskState objects containing various types of state handles. - */ - public static Collection createTaskStates(int numTaskStates, int numSubtasksPerTask) { - return createTaskStates(new Random(), numTaskStates, numSubtasksPerTask); - } - - /** - * Creates a random collection of TaskState objects containing various types of state handles. - */ - public static Collection createTaskStates( - Random random, - int numTaskStates, - int numSubtasksPerTask) { - - List taskStates = new ArrayList<>(numTaskStates); - - for (int stateIdx = 0; stateIdx < numTaskStates; ++stateIdx) { - - int chainLength = 1 + random.nextInt(8); - - TaskState taskState = new TaskState(new JobVertexID(), numSubtasksPerTask, 128, chainLength); - - int noNonPartitionableStateAtIndex = random.nextInt(chainLength); - int noOperatorStateBackendAtIndex = random.nextInt(chainLength); - int noOperatorStateStreamAtIndex = random.nextInt(chainLength); - - boolean hasKeyedBackend = random.nextInt(4) != 0; - boolean hasKeyedStream = random.nextInt(4) != 0; - - for (int subtaskIdx = 0; subtaskIdx < numSubtasksPerTask; subtaskIdx++) { - - List operatorStatesBackend = new ArrayList<>(chainLength); - List operatorStatesStream = new ArrayList<>(chainLength); - - for (int chainIdx = 0; chainIdx < chainLength; ++chainIdx) { - - StreamStateHandle operatorStateBackend = - new ByteStreamStateHandle("b-" + chainIdx, ("Beautiful-" + chainIdx).getBytes(ConfigConstants.DEFAULT_CHARSET)); - StreamStateHandle operatorStateStream = - new ByteStreamStateHandle("b-" + chainIdx, ("Beautiful-" + chainIdx).getBytes(ConfigConstants.DEFAULT_CHARSET)); - Map offsetsMap = new HashMap<>(); - offsetsMap.put("A", new OperatorStateHandle.StateMetaInfo(new long[]{0, 10, 20}, OperatorStateHandle.Mode.SPLIT_DISTRIBUTE)); - offsetsMap.put("B", new OperatorStateHandle.StateMetaInfo(new long[]{30, 40, 50}, OperatorStateHandle.Mode.SPLIT_DISTRIBUTE)); - offsetsMap.put("C", new OperatorStateHandle.StateMetaInfo(new long[]{60, 70, 80}, OperatorStateHandle.Mode.UNION)); - - if (chainIdx != noOperatorStateBackendAtIndex) { - OperatorStateHandle operatorStateHandleBackend = - new OperatorStreamStateHandle(offsetsMap, operatorStateBackend); - operatorStatesBackend.add(operatorStateHandleBackend); - } - - if (chainIdx != noOperatorStateStreamAtIndex) { - OperatorStateHandle operatorStateHandleStream = - new OperatorStreamStateHandle(offsetsMap, operatorStateStream); - operatorStatesStream.add(operatorStateHandleStream); - } - } - - KeyGroupsStateHandle keyedStateBackend = null; - KeyGroupsStateHandle keyedStateStream = null; - - if (hasKeyedBackend) { - keyedStateBackend = createDummyKeyGroupStateHandle(random); - } - - if (hasKeyedStream) { - keyedStateStream = createDummyKeyGroupStateHandle(random); - } - - taskState.putState(subtaskIdx, new SubtaskState( - new ChainedStateHandle<>(operatorStatesBackend), - new ChainedStateHandle<>(operatorStatesStream), - keyedStateStream, - keyedStateBackend)); - } - - taskStates.add(taskState); - } - - return taskStates; - } - /** * Creates a bunch of random master states. */ @@ -238,7 +144,7 @@ public class CheckpointTestUtils { /** * Asserts that two MasterStates are equal. - * + * *

    The MasterState avoids overriding {@code equals()} on purpose, because equality is not well * defined in the raw contents. */ @@ -251,10 +157,9 @@ public class CheckpointTestUtils { // ------------------------------------------------------------------------ - /** utility class, not meant to be instantiated */ + /** utility class, not meant to be instantiated. */ private CheckpointTestUtils() {} - public static IncrementalRemoteKeyedStateHandle createDummyIncrementalKeyedStateHandle(Random rnd) { return new IncrementalRemoteKeyedStateHandle( createRandomUUID(rnd), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV3SerializerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV3SerializerTest.java index 0147e5495a..9e4456c046 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV3SerializerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV3SerializerTest.java @@ -38,7 +38,7 @@ import java.util.Random; import static org.junit.Assert.assertEquals; /** - * Various tests for the version 2 format serializer of a checkpoint. + * Various tests for the version 3 format serializer of a checkpoint. */ public class MetadataV3SerializerTest { @@ -127,7 +127,7 @@ public class MetadataV3SerializerTest { ByteArrayOutputStreamWithPos baos = new ByteArrayOutputStreamWithPos(); DataOutputStream out = new DataOutputViewStreamWrapper(baos); - serializer.serialize(new CheckpointMetadata(checkpointId, operatorStates, masterStates), out); + MetadataV3Serializer.serialize(new CheckpointMetadata(checkpointId, operatorStates, masterStates), out); out.close(); byte[] bytes = baos.toByteArray(); -- Gitee From 086c0ffe21b3f3dc4235acbe57f42ad2c0a0e003 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 27 Feb 2020 20:30:56 +0100 Subject: [PATCH 078/885] [hotfix][tests] Fix parameter error in CheckpointCoordinatorTestingUtils --- .../runtime/checkpoint/CheckpointCoordinatorTestingUtils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java index 9c5de11c04..14ff4398c7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java @@ -735,7 +735,7 @@ public class CheckpointCoordinatorTestingUtils { return this; } - public CheckpointCoordinatorBuilder setIoExecutor(Executor exioExecutorecutor) { + public CheckpointCoordinatorBuilder setIoExecutor(Executor ioExecutor) { this.ioExecutor = ioExecutor; return this; } -- Gitee From 963974f99d18d6a9f36fa78b792dcc2bc9e53de5 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 27 Feb 2020 15:36:39 +0100 Subject: [PATCH 079/885] [FLINK-16177][checkpointing] Integrate OperatorCoordinator with checkpoints (triggering and committing) Restoring state to OperatorCoordinators is not included in this commit. This closes #11274 --- .../checkpoint/CheckpointCoordinator.java | 34 ++++- .../OperatorCoordinatorCheckpointContext.java | 76 +++++++++ .../OperatorCoordinatorCheckpoints.java | 144 ++++++++++++++++++ .../runtime/checkpoint/OperatorState.java | 27 +++- .../runtime/checkpoint/PendingCheckpoint.java | 66 +++++++- .../metadata/MetadataV3Serializer.java | 7 +- .../executiongraph/ExecutionGraph.java | 21 +++ .../executiongraph/ExecutionJobVertex.java | 10 +- .../coordination/OperatorCoordinator.java | 13 +- .../CheckpointCoordinatorMasterHooksTest.java | 1 + .../CheckpointCoordinatorTestingUtils.java | 7 + ...overStrategyCheckpointCoordinatorTest.java | 2 + .../checkpoint/PendingCheckpointTest.java | 117 +++++++++++++- .../metadata/CheckpointTestUtils.java | 6 + .../coordination/MockOperatorCoordinator.java | 63 ++++++++ .../state/TestingStreamStateHandle.java | 72 +++++++++ 16 files changed, 643 insertions(+), 23 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorCoordinatorCheckpointContext.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorCoordinatorCheckpoints.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/MockOperatorCoordinator.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/state/TestingStreamStateHandle.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java index 070e00c001..4e23df3bbc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java @@ -55,6 +55,8 @@ import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayDeque; +import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.LinkedHashMap; @@ -110,6 +112,9 @@ public class CheckpointCoordinator { /** Tasks who need to be sent a message when a checkpoint is confirmed. */ private final ExecutionVertex[] tasksToCommitTo; + /** The operator coordinators that need to be checkpointed. */ + private final Collection coordinatorsToCheckpoint; + /** Map from checkpoint ID to the pending checkpoint. */ private final Map pendingCheckpoints; @@ -203,6 +208,7 @@ public class CheckpointCoordinator { ExecutionVertex[] tasksToTrigger, ExecutionVertex[] tasksToWaitFor, ExecutionVertex[] tasksToCommitTo, + Collection coordinatorsToCheckpoint, CheckpointIDCounter checkpointIDCounter, CompletedCheckpointStore completedCheckpointStore, StateBackend checkpointStateBackend, @@ -217,6 +223,7 @@ public class CheckpointCoordinator { tasksToTrigger, tasksToWaitFor, tasksToCommitTo, + coordinatorsToCheckpoint, checkpointIDCounter, completedCheckpointStore, checkpointStateBackend, @@ -234,6 +241,7 @@ public class CheckpointCoordinator { ExecutionVertex[] tasksToTrigger, ExecutionVertex[] tasksToWaitFor, ExecutionVertex[] tasksToCommitTo, + Collection coordinatorsToCheckpoint, CheckpointIDCounter checkpointIDCounter, CompletedCheckpointStore completedCheckpointStore, StateBackend checkpointStateBackend, @@ -267,6 +275,7 @@ public class CheckpointCoordinator { this.tasksToTrigger = checkNotNull(tasksToTrigger); this.tasksToWaitFor = checkNotNull(tasksToWaitFor); this.tasksToCommitTo = checkNotNull(tasksToCommitTo); + this.coordinatorsToCheckpoint = Collections.unmodifiableCollection(coordinatorsToCheckpoint); this.pendingCheckpoints = new LinkedHashMap<>(); this.checkpointIdCounter = checkNotNull(checkpointIDCounter); this.completedCheckpointStore = checkNotNull(completedCheckpointStore); @@ -548,8 +557,16 @@ public class CheckpointCoordinator { onCompletionPromise), timer); - pendingCheckpointCompletableFuture - .thenCompose(this::snapshotMasterState) + final CompletableFuture masterStatesComplete = pendingCheckpointCompletableFuture + .thenCompose(this::snapshotMasterState); + + final CompletableFuture coordinatorCheckpointsComplete = pendingCheckpointCompletableFuture + .thenComposeAsync((pendingCheckpoint) -> + OperatorCoordinatorCheckpoints.triggerAndAcknowledgeAllCoordinatorCheckpointsWithCompletion( + coordinatorsToCheckpoint, pendingCheckpoint, timer), + timer); + + CompletableFuture.allOf(masterStatesComplete, coordinatorCheckpointsComplete) .whenCompleteAsync( (ignored, throwable) -> { final PendingCheckpoint checkpoint = @@ -634,6 +651,7 @@ public class CheckpointCoordinator { checkpointID, timestamp, ackTasks, + OperatorCoordinatorCheckpointContext.getIds(coordinatorsToCheckpoint), masterHooks.keySet(), props, checkpointStorageLocation, @@ -1080,15 +1098,23 @@ public class CheckpointCoordinator { LOG.debug(builder.toString()); } - // send the "notify complete" call to all vertices - final long timestamp = completedCheckpoint.getTimestamp(); + // send the "notify complete" call to all vertices, coordinators, etc. + sendAcknowledgeMessages(checkpointId, completedCheckpoint.getTimestamp()); + } + private void sendAcknowledgeMessages(long checkpointId, long timestamp) { + // commit tasks for (ExecutionVertex ev : tasksToCommitTo) { Execution ee = ev.getCurrentExecutionAttempt(); if (ee != null) { ee.notifyCheckpointComplete(checkpointId, timestamp); } } + + // commit coordinators + for (OperatorCoordinatorCheckpointContext coordinatorContext : coordinatorsToCheckpoint) { + coordinatorContext.coordinator().checkpointComplete(checkpointId); + } } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorCoordinatorCheckpointContext.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorCoordinatorCheckpointContext.java new file mode 100644 index 0000000000..16acbb2b75 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorCoordinatorCheckpointContext.java @@ -0,0 +1,76 @@ +/* + * 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.runtime.checkpoint; + +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; + +import java.util.Collection; +import java.util.stream.Collectors; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * An {@link OperatorCoordinator} and its contextual information needed to trigger and + * acknowledge a checkpoint. + */ +public final class OperatorCoordinatorCheckpointContext { + + private final OperatorCoordinator coordinator; + + private final OperatorID operatorId; + + private final int maxParallelism; + + private final int currentParallelism; + + public OperatorCoordinatorCheckpointContext( + OperatorCoordinator coordinator, + OperatorID operatorId, + int maxParallelism, + int currentParallelism) { + + this.coordinator = checkNotNull(coordinator); + this.operatorId = checkNotNull(operatorId); + this.maxParallelism = maxParallelism; + this.currentParallelism = currentParallelism; + } + + public OperatorCoordinator coordinator() { + return coordinator; + } + + public OperatorID operatorId() { + return operatorId; + } + + public int maxParallelism() { + return maxParallelism; + } + + public int currentParallelism() { + return currentParallelism; + } + + public static Collection getIds(Collection infos) { + return infos.stream() + .map(OperatorCoordinatorCheckpointContext::operatorId) + .collect(Collectors.toList()); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorCoordinatorCheckpoints.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorCoordinatorCheckpoints.java new file mode 100644 index 0000000000..cfacec867a --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorCoordinatorCheckpoints.java @@ -0,0 +1,144 @@ +/* + * 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.runtime.checkpoint; + +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.Executor; + +/** + * All the logic related to taking checkpoints of the {@link OperatorCoordinator}s. + * + *

    NOTE: This class has a simplified error handling logic. If one of the several coordinator checkpoints + * fail, no cleanup is triggered for the other concurrent ones. That is okay, since they all produce just byte[] + * as the result. We have to change that once we allow then to create external resources that actually need + * to be cleaned up. + */ +final class OperatorCoordinatorCheckpoints { + + public static CompletableFuture triggerCoordinatorCheckpoint( + final OperatorCoordinatorCheckpointContext coordinatorInfo, + final long checkpointId) throws Exception { + + final CompletableFuture checkpointFuture = + coordinatorInfo.coordinator().checkpointCoordinator(checkpointId); + + return checkpointFuture.thenApply( + (state) -> new CoordinatorSnapshot( + coordinatorInfo, new ByteStreamStateHandle(coordinatorInfo.operatorId().toString(), state)) + ); + } + + public static CompletableFuture triggerAllCoordinatorCheckpoints( + final Collection coordinators, + final long checkpointId) throws Exception { + + final Collection> individualSnapshots = new ArrayList<>(coordinators.size()); + + for (final OperatorCoordinatorCheckpointContext coordinator : coordinators) { + individualSnapshots.add(triggerCoordinatorCheckpoint(coordinator, checkpointId)); + } + + return FutureUtils.combineAll(individualSnapshots).thenApply(AllCoordinatorSnapshots::new); + } + + public static CompletableFuture triggerAndAcknowledgeAllCoordinatorCheckpoints( + final Collection coordinators, + final PendingCheckpoint checkpoint, + final Executor acknowledgeExecutor) throws Exception { + + final CompletableFuture snapshots = + triggerAllCoordinatorCheckpoints(coordinators, checkpoint.getCheckpointId()); + + return snapshots + .thenAcceptAsync( + (allSnapshots) -> { + try { + acknowledgeAllCoordinators(checkpoint, allSnapshots.snapshots); + } + catch (Exception e) { + throw new CompletionException(e); + } + }, + acknowledgeExecutor); + } + + public static CompletableFuture triggerAndAcknowledgeAllCoordinatorCheckpointsWithCompletion( + final Collection coordinators, + final PendingCheckpoint checkpoint, + final Executor acknowledgeExecutor) throws CompletionException { + + try { + return triggerAndAcknowledgeAllCoordinatorCheckpoints(coordinators, checkpoint, acknowledgeExecutor); + } catch (Exception e) { + throw new CompletionException(e); + } + } + + // ------------------------------------------------------------------------ + + private static void acknowledgeAllCoordinators(PendingCheckpoint checkpoint, Collection snapshots) throws CheckpointException { + for (final CoordinatorSnapshot snapshot : snapshots) { + final PendingCheckpoint.TaskAcknowledgeResult result = + checkpoint.acknowledgeCoordinatorState(snapshot.coordinator, snapshot.state); + + if (result != PendingCheckpoint.TaskAcknowledgeResult.SUCCESS) { + throw new CheckpointException("Coordinator state not acknowledged successfully: " + result, + CheckpointFailureReason.TRIGGER_CHECKPOINT_FAILURE); + } + } + } + + // ------------------------------------------------------------------------ + + static final class AllCoordinatorSnapshots { + + private final Collection snapshots; + + AllCoordinatorSnapshots(Collection snapshots) { + this.snapshots = snapshots; + } + + public Iterable snapshots() { + return snapshots; + } + } + + static final class CoordinatorSnapshot { + + final OperatorCoordinatorCheckpointContext coordinator; + final StreamStateHandle state; + + CoordinatorSnapshot(OperatorCoordinatorCheckpointContext coordinator, StreamStateHandle state) { + // if this is not true any more, we need more elaborate dispose/cleanup handling + // see comment above the class. + assert state instanceof ByteStreamStateHandle; + + this.coordinator = coordinator; + this.state = state; + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java index de011c1117..998a3bdbd0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java @@ -21,14 +21,19 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.state.CompositeStateHandle; import org.apache.flink.runtime.state.SharedStateRegistry; +import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.util.Preconditions; +import javax.annotation.Nullable; + import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Objects; +import static org.apache.flink.util.Preconditions.checkState; + /** * Simple container class which contains the raw/managed operator state and key-group state handles from all sub * tasks of an operator and therefore represents the complete state of a logical operator. @@ -43,6 +48,10 @@ public class OperatorState implements CompositeStateHandle { /** The handles to states created by the parallel tasks: subtaskIndex -> subtaskstate. */ private final Map operatorSubtaskStates; + /** The state of the operator coordinator. Null, if no such state exists. */ + @Nullable + private StreamStateHandle coordinatorState; + /** The parallelism of the operator when it was checkpointed. */ private final int parallelism; @@ -87,6 +96,16 @@ public class OperatorState implements CompositeStateHandle { } } + public void setCoordinatorState(@Nullable StreamStateHandle coordinatorState) { + checkState(this.coordinatorState == null, "coordinator state already set"); + this.coordinatorState = coordinatorState; + } + + @Nullable + public StreamStateHandle getCoordinatorState() { + return coordinatorState; + } + public Map getSubtaskStates() { return Collections.unmodifiableMap(operatorSubtaskStates); } @@ -112,6 +131,10 @@ public class OperatorState implements CompositeStateHandle { for (OperatorSubtaskState operatorSubtaskState : operatorSubtaskStates.values()) { operatorSubtaskState.discardState(); } + + if (coordinatorState != null) { + coordinatorState.discardState(); + } } @Override @@ -123,7 +146,7 @@ public class OperatorState implements CompositeStateHandle { @Override public long getStateSize() { - long result = 0L; + long result = coordinatorState == null ? 0L : coordinatorState.getStateSize(); for (int i = 0; i < parallelism; i++) { OperatorSubtaskState operatorSubtaskState = operatorSubtaskStates.get(i); @@ -142,6 +165,7 @@ public class OperatorState implements CompositeStateHandle { return operatorID.equals(other.operatorID) && parallelism == other.parallelism + && Objects.equals(coordinatorState, other.coordinatorState) && operatorSubtaskStates.equals(other.operatorSubtaskStates); } else { return false; @@ -161,6 +185,7 @@ public class OperatorState implements CompositeStateHandle { "operatorID: " + operatorID + ", parallelism: " + parallelism + ", maxParallelism: " + maxParallelism + + ", coordinatorState: " + (coordinatorState == null ? "(none)" : coordinatorState.getStateSize() + " bytes") + ", sub task states: " + operatorSubtaskStates.size() + ", total size (bytes): " + getStateSize() + ')'; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java index ae9027da2e..27a8513b3d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java @@ -27,6 +27,7 @@ import org.apache.flink.runtime.state.CheckpointMetadataOutputStream; import org.apache.flink.runtime.state.CheckpointStorageLocation; import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation; import org.apache.flink.runtime.state.StateUtil; +import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.Preconditions; @@ -38,6 +39,7 @@ import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -88,6 +90,8 @@ public class PendingCheckpoint { private final Map notYetAcknowledgedTasks; + private final Set notYetAcknowledgedOperatorCoordinators; + private final List masterStates; private final Set notYetAcknowledgedMasterStates; @@ -126,6 +130,7 @@ public class PendingCheckpoint { long checkpointId, long checkpointTimestamp, Map verticesToConfirm, + Collection operatorCoordinatorsToConfirm, Collection masterStateIdentifiers, CheckpointProperties props, CheckpointStorageLocation targetLocation, @@ -145,7 +150,10 @@ public class PendingCheckpoint { this.operatorStates = new HashMap<>(); this.masterStates = new ArrayList<>(masterStateIdentifiers.size()); - this.notYetAcknowledgedMasterStates = new HashSet<>(masterStateIdentifiers); + this.notYetAcknowledgedMasterStates = masterStateIdentifiers.isEmpty() + ? Collections.emptySet() : new HashSet<>(masterStateIdentifiers); + this.notYetAcknowledgedOperatorCoordinators = operatorCoordinatorsToConfirm.isEmpty() + ? Collections.emptySet() : new HashSet<>(operatorCoordinatorsToConfirm); this.acknowledgedTasks = new HashSet<>(verticesToConfirm.size()); this.onCompletionPromise = checkNotNull(onCompletionPromise); } @@ -176,6 +184,10 @@ public class PendingCheckpoint { return notYetAcknowledgedTasks.size(); } + public int getNumberOfNonAcknowledgedOperatorCoordinators() { + return notYetAcknowledgedOperatorCoordinators.size(); + } + public int getNumberOfAcknowledgedTasks() { return numAcknowledgedTasks; } @@ -188,11 +200,21 @@ public class PendingCheckpoint { return masterStates; } - public boolean areMasterStatesFullyAcknowledged() { + public boolean isFullyAcknowledged() { + return areTasksFullyAcknowledged() && + areCoordinatorsFullyAcknowledged() && + areMasterStatesFullyAcknowledged(); + } + + boolean areMasterStatesFullyAcknowledged() { return notYetAcknowledgedMasterStates.isEmpty() && !discarded; } - public boolean areTasksFullyAcknowledged() { + boolean areCoordinatorsFullyAcknowledged() { + return notYetAcknowledgedOperatorCoordinators.isEmpty() && !discarded; + } + + boolean areTasksFullyAcknowledged() { return notYetAcknowledgedTasks.isEmpty() && !discarded; } @@ -270,10 +292,8 @@ public class PendingCheckpoint { public CompletedCheckpoint finalizeCheckpoint() throws IOException { synchronized (lock) { - checkState(areMasterStatesFullyAcknowledged(), - "Pending checkpoint has not been fully acknowledged by master states yet."); - checkState(areTasksFullyAcknowledged(), - "Pending checkpoint has not been fully acknowledged by tasks yet."); + checkState(!isDiscarded(), "checkpoint is discarded"); + checkState(isFullyAcknowledged(), "Pending checkpoint has not been fully acknowledged yet"); // make sure we fulfill the promise with an exception if something fails try { @@ -410,6 +430,38 @@ public class PendingCheckpoint { } } + public TaskAcknowledgeResult acknowledgeCoordinatorState( + OperatorCoordinatorCheckpointContext coordinatorInfo, + @Nullable StreamStateHandle stateHandle) { + + synchronized (lock) { + if (discarded) { + return TaskAcknowledgeResult.DISCARDED; + } + + final OperatorID operatorId = coordinatorInfo.operatorId(); + OperatorState operatorState = operatorStates.get(operatorId); + + // sanity check for better error reporting + if (!notYetAcknowledgedOperatorCoordinators.remove(operatorId)) { + return operatorState != null && operatorState.getCoordinatorState() != null + ? TaskAcknowledgeResult.DUPLICATE + : TaskAcknowledgeResult.UNKNOWN; + } + + if (stateHandle != null) { + if (operatorState == null) { + operatorState = new OperatorState( + operatorId, coordinatorInfo.currentParallelism(), coordinatorInfo.maxParallelism()); + operatorStates.put(operatorId, operatorState); + } + operatorState.setCoordinatorState(stateHandle); + } + + return TaskAcknowledgeResult.SUCCESS; + } + } + /** * Acknowledges a master state (state generated on the checkpoint coordinator) to * the pending checkpoint. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV3Serializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV3Serializer.java index e304ef41f9..7db1255e29 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV3Serializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV3Serializer.java @@ -81,6 +81,9 @@ public class MetadataV3Serializer extends MetadataV2V3SerializerBase implements dos.writeInt(operatorState.getParallelism()); dos.writeInt(operatorState.getMaxParallelism()); + // Coordinator state + serializeStreamStateHandle(operatorState.getCoordinatorState(), dos); + // Sub task states final Map subtaskStateMap = operatorState.getSubtaskStates(); dos.writeInt(subtaskStateMap.size()); @@ -96,9 +99,11 @@ public class MetadataV3Serializer extends MetadataV2V3SerializerBase implements final int parallelism = dis.readInt(); final int maxParallelism = dis.readInt(); - // Add task state final OperatorState operatorState = new OperatorState(jobVertexId, parallelism, maxParallelism); + // Coordinator state + operatorState.setCoordinatorState(deserializeStreamStateHandle(dis)); + // Sub task states final int numSubTaskStates = dis.readInt(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index f8d79fa14a..09a6a3aae5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -41,6 +41,7 @@ import org.apache.flink.runtime.checkpoint.CheckpointStatsSnapshot; import org.apache.flink.runtime.checkpoint.CheckpointStatsTracker; import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore; import org.apache.flink.runtime.checkpoint.MasterTriggerRestoreHook; +import org.apache.flink.runtime.checkpoint.OperatorCoordinatorCheckpointContext; import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.concurrent.FutureUtils.ConjunctFuture; @@ -60,10 +61,12 @@ import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobgraph.ScheduleMode; import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup; import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; import org.apache.flink.runtime.query.KvStateLocationRegistry; import org.apache.flink.runtime.scheduler.InternalFailuresListener; import org.apache.flink.runtime.scheduler.adapter.DefaultExecutionTopology; @@ -457,6 +460,8 @@ public class ExecutionGraph implements AccessExecutionGraph { ExecutionVertex[] tasksToWaitFor = collectExecutionVertices(verticesToWaitFor); ExecutionVertex[] tasksToCommitTo = collectExecutionVertices(verticesToCommitTo); + final Collection operatorCoordinators = buildOpCoordinatorCheckpointContexts(); + checkpointStatsTracker = checkNotNull(statsTracker, "CheckpointStatsTracker"); CheckpointFailureManager failureManager = new CheckpointFailureManager( @@ -487,6 +492,7 @@ public class ExecutionGraph implements AccessExecutionGraph { tasksToTrigger, tasksToWaitFor, tasksToCommitTo, + operatorCoordinators, checkpointIDCounter, checkpointStore, checkpointStateBackend, @@ -566,6 +572,21 @@ public class ExecutionGraph implements AccessExecutionGraph { } } + private Collection buildOpCoordinatorCheckpointContexts() { + final ArrayList contexts = new ArrayList<>(); + for (final ExecutionJobVertex vertex : verticesInCreationOrder) { + for (final Map.Entry coordinator : vertex.getOperatorCoordinatorMap().entrySet()) { + contexts.add(new OperatorCoordinatorCheckpointContext( + coordinator.getValue(), + coordinator.getKey(), + vertex.getMaxParallelism(), + vertex.getParallelism())); + } + } + contexts.trimToSize(); + return contexts; + } + // -------------------------------------------------------------------------------------------- // Properties and Status of the Execution Graph // -------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java index 4b606520fc..44d33fae9f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java @@ -249,10 +249,12 @@ public class ExecutionJobVertex implements AccessExecutionJobVertex, Archiveable } try { - this.operatorCoordinators = OperatorCoordinatorUtil.instantiateCoordinators( + final Map coordinators = OperatorCoordinatorUtil.instantiateCoordinators( jobVertex.getOperatorCoordinators(), graph.getUserClassLoader(), (opId) -> new ExecutionJobVertexCoordinatorContext(opId, this)); + + this.operatorCoordinators = Collections.unmodifiableMap(coordinators); } catch (IOException | ClassNotFoundException e) { throw new JobException("Cannot instantiate the coordinator for operator " + getName(), e); @@ -402,8 +404,12 @@ public class ExecutionJobVertex implements AccessExecutionJobVertex, Archiveable return operatorCoordinators.get(operatorId); } + public Map getOperatorCoordinatorMap() { + return operatorCoordinators; + } + public Collection getOperatorCoordinators() { - return Collections.unmodifiableCollection(operatorCoordinators.values()); + return operatorCoordinators.values(); } public Either, PermanentBlobKey> getTaskInformationOrBlobKey() throws IOException { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinator.java index b5951c22d4..eaa24ac2eb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinator.java @@ -67,7 +67,18 @@ public interface OperatorCoordinator extends AutoCloseable { CompletableFuture checkpointCoordinator(long checkpointId) throws Exception; - void checkpointComplete(long checkpointId) throws Exception; + /** + * Notifies the coordinator that the checkpoint with the given checkpointId completes and + * was committed. + * + *

    Important: This method is not supposed to throw an exception, because by the + * time we notify that the checkpoint is complete, the checkpoint is committed and cannot be + * aborted any more. If the coordinator gets into an inconsistent state internally, it should + * fail the job ({@link Context#failJob(Throwable)}) instead. Any exception propagating from + * this method may be treated as a fatal error for the JobManager, crashing the JobManager, + * and leading to an expensive "master failover" procedure. + */ + void checkpointComplete(long checkpointId); void resetToCheckpoint(byte[] checkpointData) throws Exception; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java index e5ea59ad65..cff25719cb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java @@ -460,6 +460,7 @@ public class CheckpointCoordinatorMasterHooksTest { new ExecutionVertex[0], ackVertices, new ExecutionVertex[0], + Collections.emptyList(), new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(10), new MemoryStateBackend(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java index 14ff4398c7..cf36f202ce 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java @@ -659,6 +659,8 @@ public class CheckpointCoordinatorTestingUtils { private ExecutionVertex[] tasksToCommitTo; + private Collection coordinatorsToCheckpoint = Collections.emptyList(); + private CheckpointIDCounter checkpointIDCounter = new StandaloneCheckpointIDCounter(); @@ -718,6 +720,10 @@ public class CheckpointCoordinatorTestingUtils { return this; } + public void setCoordinatorsToCheckpoint(Collection coordinatorsToCheckpoint) { + this.coordinatorsToCheckpoint = coordinatorsToCheckpoint; + } + public CheckpointCoordinatorBuilder setCheckpointIDCounter( CheckpointIDCounter checkpointIDCounter) { this.checkpointIDCounter = checkpointIDCounter; @@ -764,6 +770,7 @@ public class CheckpointCoordinatorTestingUtils { tasksToTrigger, tasksToWaitFor, tasksToCommitTo, + coordinatorsToCheckpoint, checkpointIDCounter, completedCheckpointStore, checkpointStateBackend, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailoverStrategyCheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailoverStrategyCheckpointCoordinatorTest.java index f281ff439c..6d7b6cfb6d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailoverStrategyCheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailoverStrategyCheckpointCoordinatorTest.java @@ -36,6 +36,7 @@ import org.junit.Before; import org.junit.Test; import org.mockito.Mockito; +import java.util.Collections; import java.util.concurrent.ThreadLocalRandom; import static org.junit.Assert.assertEquals; @@ -79,6 +80,7 @@ public class FailoverStrategyCheckpointCoordinatorTest extends TestLogger { new ExecutionVertex[] { executionVertex }, new ExecutionVertex[] { executionVertex }, new ExecutionVertex[] { executionVertex }, + Collections.emptyList(), new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(1), new MemoryStateBackend(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java index c78c077833..f172e51f12 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java @@ -23,6 +23,7 @@ import org.apache.flink.core.fs.Path; import org.apache.flink.core.fs.local.LocalFileSystem; import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.StringSerializer; +import org.apache.flink.runtime.checkpoint.PendingCheckpoint.TaskAcknowledgeResult; import org.apache.flink.runtime.checkpoint.hooks.MasterHooks; import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; @@ -30,10 +31,14 @@ import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.executiongraph.ExecutionVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinator; import org.apache.flink.runtime.state.CheckpointStorageLocationReference; import org.apache.flink.runtime.state.SharedStateRegistry; +import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.runtime.state.TestingStreamStateHandle; import org.apache.flink.runtime.state.filesystem.FsCheckpointStorageLocation; +import org.hamcrest.Matchers; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -46,6 +51,7 @@ import java.io.IOException; import java.lang.reflect.Field; import java.util.ArrayDeque; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -59,11 +65,12 @@ import java.util.concurrent.ScheduledFuture; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.nullable; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyLong; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; @@ -189,7 +196,6 @@ public class PendingCheckpointTest { * Tests that abort discards state. */ @Test - @SuppressWarnings("unchecked") public void testAbortDiscardsState() throws Exception { CheckpointProperties props = new CheckpointProperties(false, CheckpointType.SAVEPOINT, false, false, false, false, false); QueueExecutor executor = new QueueExecutor(); @@ -428,21 +434,109 @@ public class PendingCheckpointTest { assertEquals("state", deserializedState); } + @Test + public void testInitiallyUnacknowledgedCoordinatorStates() throws Exception { + final PendingCheckpoint checkpoint = createPendingCheckpointWithCoordinators( + createOperatorCoordinator(), createOperatorCoordinator()); + + assertEquals(2, checkpoint.getNumberOfNonAcknowledgedOperatorCoordinators()); + assertFalse(checkpoint.isFullyAcknowledged()); + } + + @Test + public void testAcknowledgedCoordinatorStates() throws Exception { + final OperatorCoordinatorCheckpointContext coord1 = createOperatorCoordinator(); + final OperatorCoordinatorCheckpointContext coord2 = createOperatorCoordinator(); + final PendingCheckpoint checkpoint = createPendingCheckpointWithCoordinators(coord1, coord2); + + final TaskAcknowledgeResult ack1 = checkpoint.acknowledgeCoordinatorState(coord1, new TestingStreamStateHandle()); + final TaskAcknowledgeResult ack2 = checkpoint.acknowledgeCoordinatorState(coord2, null); + + assertEquals(TaskAcknowledgeResult.SUCCESS, ack1); + assertEquals(TaskAcknowledgeResult.SUCCESS, ack2); + assertEquals(0, checkpoint.getNumberOfNonAcknowledgedOperatorCoordinators()); + assertTrue(checkpoint.isFullyAcknowledged()); + assertThat(checkpoint.getOperatorStates().keySet(), Matchers.contains(coord1.operatorId())); + } + + @Test + public void testDuplicateAcknowledgeCoordinator() throws Exception { + final OperatorCoordinatorCheckpointContext coordinator = createOperatorCoordinator(); + final PendingCheckpoint checkpoint = createPendingCheckpointWithCoordinators(coordinator); + + checkpoint.acknowledgeCoordinatorState(coordinator, new TestingStreamStateHandle()); + final TaskAcknowledgeResult secondAck = checkpoint.acknowledgeCoordinatorState(coordinator, null); + + assertEquals(TaskAcknowledgeResult.DUPLICATE, secondAck); + } + + @Test + public void testAcknowledgeUnknownCoordinator() throws Exception { + final PendingCheckpoint checkpoint = createPendingCheckpointWithCoordinators(createOperatorCoordinator()); + + final TaskAcknowledgeResult ack = checkpoint.acknowledgeCoordinatorState(createOperatorCoordinator(), null); + + assertEquals(TaskAcknowledgeResult.UNKNOWN, ack); + } + + @Test + public void testDisposeDisposesCoordinatorStates() throws Exception { + final TestingStreamStateHandle handle1 = new TestingStreamStateHandle(); + final TestingStreamStateHandle handle2 = new TestingStreamStateHandle(); + final PendingCheckpoint checkpoint = createPendingCheckpointWithAcknowledgedCoordinators(handle1, handle2); + + checkpoint.abort(CheckpointFailureReason.CHECKPOINT_EXPIRED); + + assertTrue(handle1.isDisposed()); + assertTrue(handle2.isDisposed()); + } + // ------------------------------------------------------------------------ private PendingCheckpoint createPendingCheckpoint(CheckpointProperties props) throws IOException { - return createPendingCheckpoint(props, Collections.emptyList(), Executors.directExecutor()); + return createPendingCheckpoint(props, Collections.emptyList(), Collections.emptyList(), Executors.directExecutor()); } private PendingCheckpoint createPendingCheckpoint(CheckpointProperties props, Executor executor) throws IOException { - return createPendingCheckpoint(props, Collections.emptyList(), executor); + return createPendingCheckpoint(props, Collections.emptyList(), Collections.emptyList(), executor); } private PendingCheckpoint createPendingCheckpoint(CheckpointProperties props, Collection masterStateIdentifiers) throws IOException { - return createPendingCheckpoint(props, masterStateIdentifiers, Executors.directExecutor()); + return createPendingCheckpoint(props, Collections.emptyList(), masterStateIdentifiers, Executors.directExecutor()); } - private PendingCheckpoint createPendingCheckpoint(CheckpointProperties props, Collection masterStateIdentifiers, Executor executor) throws IOException { + private PendingCheckpoint createPendingCheckpointWithCoordinators( + OperatorCoordinatorCheckpointContext... coordinators) throws IOException { + + final PendingCheckpoint checkpoint = createPendingCheckpoint( + CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION), + OperatorCoordinatorCheckpointContext.getIds(Arrays.asList(coordinators)), + Collections.emptyList(), + Executors.directExecutor()); + + checkpoint.acknowledgeTask(ATTEMPT_ID, null, new CheckpointMetrics()); + return checkpoint; + } + + private PendingCheckpoint createPendingCheckpointWithAcknowledgedCoordinators(StreamStateHandle... handles) throws IOException { + OperatorCoordinatorCheckpointContext[] coords = new OperatorCoordinatorCheckpointContext[handles.length]; + for (int i = 0; i < handles.length; i++) { + coords[i] = createOperatorCoordinator(); + } + + final PendingCheckpoint checkpoint = createPendingCheckpointWithCoordinators(coords); + for (int i = 0; i < handles.length; i++) { + checkpoint.acknowledgeCoordinatorState(coords[i], handles[i]); + } + + return checkpoint; + } + + private PendingCheckpoint createPendingCheckpoint( + CheckpointProperties props, + Collection operatorCoordinators, + Collection masterStateIdentifiers, + Executor executor) throws IOException { final Path checkpointDir = new Path(tmpFolder.newFolder().toURI()); final FsCheckpointStorageLocation location = new FsCheckpointStorageLocation( @@ -459,6 +553,7 @@ public class PendingCheckpointTest { 0, 1, ackTasks, + operatorCoordinators, masterStateIdentifiers, props, location, @@ -466,6 +561,14 @@ public class PendingCheckpointTest { new CompletableFuture<>()); } + private static OperatorCoordinatorCheckpointContext createOperatorCoordinator() { + return new OperatorCoordinatorCheckpointContext( + new MockOperatorCoordinator(), + new OperatorID(), + 256, + 50); + } + @SuppressWarnings("unchecked") static void setTaskState(PendingCheckpoint pending, OperatorState state) throws NoSuchFieldException, IllegalAccessException { Field field = PendingCheckpoint.class.getDeclaredField("operatorStates"); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/metadata/CheckpointTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/metadata/CheckpointTestUtils.java index f411a39f62..ef6ea85ac2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/metadata/CheckpointTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/metadata/CheckpointTestUtils.java @@ -66,6 +66,12 @@ public class CheckpointTestUtils { OperatorState taskState = new OperatorState(new OperatorID(), numSubtasksPerTask, 128); + final boolean hasCoordinatorState = random.nextBoolean(); + if (hasCoordinatorState) { + final StreamStateHandle stateHandle = createDummyStreamStateHandle(random); + taskState.setCoordinatorState(stateHandle); + } + boolean hasOperatorStateBackend = random.nextBoolean(); boolean hasOperatorStateStream = random.nextBoolean(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/MockOperatorCoordinator.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/MockOperatorCoordinator.java new file mode 100644 index 0000000000..c4122fc358 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/MockOperatorCoordinator.java @@ -0,0 +1,63 @@ +/* + * 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.runtime.operators.coordination; + +import java.util.concurrent.CompletableFuture; + +/** + * An empty interface implementation of the {@link OperatorCoordinator}. + * If you need a testing stub, use the {@link TestingOperatorCoordinator} instead. + */ +public final class MockOperatorCoordinator implements OperatorCoordinator { + + @Override + public void start() { + throw new UnsupportedOperationException(); + } + + @Override + public void close() { + throw new UnsupportedOperationException(); + } + + @Override + public void handleEventFromOperator(int subtask, OperatorEvent event) { + throw new UnsupportedOperationException(); + } + + @Override + public void subtaskFailed(int subtask) { + throw new UnsupportedOperationException(); + } + + @Override + public CompletableFuture checkpointCoordinator(long checkpointId) { + throw new UnsupportedOperationException(); + } + + @Override + public void checkpointComplete(long checkpointId) { + throw new UnsupportedOperationException(); + } + + @Override + public void resetToCheckpoint(byte[] checkpointData) { + throw new UnsupportedOperationException(); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/TestingStreamStateHandle.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TestingStreamStateHandle.java new file mode 100644 index 0000000000..7a968d2292 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TestingStreamStateHandle.java @@ -0,0 +1,72 @@ +/* + * 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.runtime.state; + +import org.apache.flink.core.fs.FSDataInputStream; + +import javax.annotation.Nullable; + +/** + * A simple test mock for a {@link StreamStateHandle}. + */ +public class TestingStreamStateHandle implements StreamStateHandle { + private static final long serialVersionUID = 1L; + + @Nullable + private final FSDataInputStream inputStream; + + private final long size; + + private boolean disposed; + + public TestingStreamStateHandle() { + this(null, 0L); + } + + public TestingStreamStateHandle(@Nullable FSDataInputStream inputStream, long size) { + this.inputStream = inputStream; + this.size = size; + } + + // ------------------------------------------------------------------------ + + @Override + public FSDataInputStream openInputStream() { + if (inputStream == null) { + throw new UnsupportedOperationException("no input stream provided"); + } + return inputStream; + } + + @Override + public void discardState() { + disposed = true; + } + + @Override + public long getStateSize() { + return size; + } + + // ------------------------------------------------------------------------ + + public boolean isDisposed() { + return disposed; + } +} -- Gitee From b6f07ddfe779698f929959289c317dbafefc4eb5 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Wed, 4 Mar 2020 10:27:41 +0100 Subject: [PATCH 080/885] [FLINK-16410][e2e][build] Add explicit flink-runtime dependency --- .../flink-end-to-end-tests-common/pom.xml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/pom.xml b/flink-end-to-end-tests/flink-end-to-end-tests-common/pom.xml index 66e110b1e3..25092e5893 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/pom.xml +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/pom.xml @@ -55,6 +55,12 @@ under the License. flink-test-utils-junit compile + + org.apache.flink + flink-runtime_${scala.binary.version} + ${project.version} + compile + org.apache.flink -- Gitee From 8587ce8dacb2b6c9fae1d3d3fd923ee0a32b7ee4 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Fri, 14 Feb 2020 12:34:12 +0100 Subject: [PATCH 081/885] [FLINK-16172][build] Add baseline set of allowed unused dependencies --- pom.xml | 28 ++++++++++++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/pom.xml b/pom.xml index 3088d6364f..2590accfe6 100644 --- a/pom.xml +++ b/pom.xml @@ -1761,6 +1761,34 @@ under the License. 1.11.1 + + + + org.apache.flink:force-shading + + com.google.code.findbugs:jsr305 + org.scala-lang:scala-compiler + + org.slf4j:slf4j-api + + log4j:log4j + org.slf4j:slf4j-log4j12 + + org.apache.logging.log4j:log4j-slf4j-impl + org.apache.logging.log4j:log4j-api + org.apache.logging.log4j:log4j-core + org.apache.logging.log4j:log4j-1.2-api + + org.apache.flink:flink-test-utils-junit + junit:junit + org.mockito:mockito-core + org.powermock:powermock-api-mockito2 + org.powermock:powermock-module-junit4 + org.hamcrest:hamcrest-all + + -- Gitee From a762e76c0fa270367a6cf6cebca807ba80e124ae Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Wed, 19 Feb 2020 13:38:34 +0100 Subject: [PATCH 082/885] [FLINK-16173][build] Reduce noise for used undeclared dependencies --- pom.xml | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/pom.xml b/pom.xml index 2590accfe6..90de99ba89 100644 --- a/pom.xml +++ b/pom.xml @@ -1762,6 +1762,16 @@ under the License. + + + org.apache.flink:* + + org.hamcrest:hamcrest-core + + org.powermock:powermock-core + org.powermock:powermock-reflect + org.powermock:powermock-api-support + org.apache.flink:force-shading -- Gitee From 1924f82512e17a43f874b8c86492d9ce6c8d7dd1 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 5 Mar 2020 16:12:37 +0100 Subject: [PATCH 083/885] [FLINK-16189][e2e] Remove test logic from FlinkDistribution The FlinkDistribution is now a simple wrapper, which makes it easier to use in ways that do not match the jUnit resource life-cycle. --- .../tests/util/flink/FlinkDistribution.java | 86 ++++--------------- .../flink/LocalStandaloneFlinkResource.java | 53 ++++++++++-- .../LocalStandaloneFlinkResourceFactory.java | 18 +++- 3 files changed, 83 insertions(+), 74 deletions(-) diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkDistribution.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkDistribution.java index c0cebe0c82..970dbef62f 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkDistribution.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkDistribution.java @@ -25,7 +25,6 @@ import org.apache.flink.configuration.UnmodifiableConfiguration; import org.apache.flink.tests.util.AutoClosableProcess; import org.apache.flink.tests.util.TestUtils; import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.ExternalResource; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; @@ -33,8 +32,6 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMap import okhttp3.OkHttpClient; import okhttp3.Request; import okhttp3.Response; -import org.junit.Assert; -import org.junit.rules.TemporaryFolder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,13 +43,11 @@ import java.io.InputStreamReader; import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; -import java.nio.file.Paths; import java.util.ArrayList; import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.Optional; -import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.function.Consumer; @@ -65,80 +60,30 @@ import java.util.stream.Stream; /** * A wrapper around a Flink distribution. */ -final class FlinkDistribution implements ExternalResource { +final class FlinkDistribution { private static final Logger LOG = LoggerFactory.getLogger(FlinkDistribution.class); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - private final Path logBackupDir; + private final Path opt; + private final Path lib; + private final Path conf; + private final Path log; + private final Path bin; - private final TemporaryFolder temporaryFolder = new TemporaryFolder(); + private final Configuration defaultConfig; - private final Path originalFlinkDir; - private Path opt; - private Path lib; - private Path conf; - private Path log; - private Path bin; - - private Configuration defaultConfig; - - FlinkDistribution() { - final String distDirProperty = System.getProperty("distDir"); - if (distDirProperty == null) { - Assert.fail("The distDir property was not set. You can set it when running maven via -DdistDir= ."); - } - final String backupDirProperty = System.getProperty("logBackupDir"); - logBackupDir = backupDirProperty == null ? null : Paths.get(backupDirProperty); - originalFlinkDir = Paths.get(distDirProperty); - } - - @Override - public void before() throws IOException { - temporaryFolder.create(); - - final Path flinkDir = temporaryFolder.newFolder().toPath(); - - LOG.info("Copying distribution to {}.", flinkDir); - TestUtils.copyDirectory(originalFlinkDir, flinkDir); - - bin = flinkDir.resolve("bin"); - opt = flinkDir.resolve("opt"); - lib = flinkDir.resolve("lib"); - conf = flinkDir.resolve("conf"); - log = flinkDir.resolve("log"); + FlinkDistribution(Path distributionDir) { + bin = distributionDir.resolve("bin"); + opt = distributionDir.resolve("opt"); + lib = distributionDir.resolve("lib"); + conf = distributionDir.resolve("conf"); + log = distributionDir.resolve("log"); defaultConfig = new UnmodifiableConfiguration(GlobalConfiguration.loadConfiguration(conf.toAbsolutePath().toString())); } - @Override - public void afterTestSuccess() { - try { - stopFlinkCluster(); - } catch (IOException e) { - LOG.error("Failure while shutting down Flink cluster.", e); - } - - temporaryFolder.delete(); - } - - @Override - public void afterTestFailure() { - if (logBackupDir != null) { - final UUID id = UUID.randomUUID(); - LOG.info("Backing up logs to {}/{}.", logBackupDir, id); - try { - Files.createDirectories(logBackupDir); - TestUtils.copyDirectory(log, logBackupDir.resolve(id.toString())); - } catch (IOException e) { - LOG.warn("An error occurred while backing up logs.", e); - } - } - - afterTestSuccess(); - } - public void startJobManager() throws IOException { LOG.info("Starting Flink JobManager."); AutoClosableProcess.runBlocking(bin.resolve("jobmanager.sh").toAbsolutePath().toString(), "start"); @@ -331,4 +276,9 @@ final class FlinkDistribution implements ExternalResource { } return matches.stream(); } + + public void copyLogsTo(Path targetDirectory) throws IOException { + Files.createDirectories(targetDirectory); + TestUtils.copyDirectory(log, targetDirectory); + } } diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResource.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResource.java index 1a0fb13f7e..74ccf6e489 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResource.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResource.java @@ -28,13 +28,19 @@ import org.apache.flink.runtime.rest.messages.EmptyMessageParameters; import org.apache.flink.runtime.rest.messages.EmptyRequestBody; import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagersHeaders; import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagersInfo; +import org.apache.flink.tests.util.TestUtils; import org.apache.flink.util.ConfigurationException; +import org.junit.rules.TemporaryFolder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.io.IOException; +import java.nio.file.Path; import java.util.Collections; +import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; @@ -51,16 +57,28 @@ public class LocalStandaloneFlinkResource implements FlinkResource { private static final Logger LOG = LoggerFactory.getLogger(LocalStandaloneFlinkResource.class); - private final FlinkDistribution distribution = new FlinkDistribution(); + private final TemporaryFolder temporaryFolder = new TemporaryFolder(); + private final Path distributionDirectory; + @Nullable + private final Path logBackupDirectory; private final FlinkResourceSetup setup; - LocalStandaloneFlinkResource(FlinkResourceSetup setup) { + private FlinkDistribution distribution; + + LocalStandaloneFlinkResource(Path distributionDirectory, @Nullable Path logBackupDirectory, FlinkResourceSetup setup) { + this.distributionDirectory = distributionDirectory; + this.logBackupDirectory = logBackupDirectory; this.setup = setup; } @Override public void before() throws Exception { - distribution.before(); + temporaryFolder.create(); + Path tmp = temporaryFolder.newFolder().toPath(); + LOG.info("Copying distribution to {}.", tmp); + TestUtils.copyDirectory(distributionDirectory, tmp); + + distribution = new FlinkDistribution(tmp); for (JarMove jarMove : setup.getJarMoveOperations()) { distribution.moveJar(jarMove); } @@ -71,12 +89,37 @@ public class LocalStandaloneFlinkResource implements FlinkResource { @Override public void afterTestSuccess() { - distribution.afterTestSuccess(); + shutdownCluster(); + temporaryFolder.delete(); } @Override public void afterTestFailure() { - distribution.afterTestFailure(); + if (distribution != null) { + shutdownCluster(); + backupLogs(); + } + temporaryFolder.delete(); + } + + private void shutdownCluster() { + try { + distribution.stopFlinkCluster(); + } catch (IOException e) { + LOG.warn("Error while shutting down Flink cluster.", e); + } + } + + private void backupLogs() { + if (logBackupDirectory != null) { + final Path targetDirectory = logBackupDirectory.resolve(UUID.randomUUID().toString()); + try { + distribution.copyLogsTo(targetDirectory); + LOG.info("Backed up logs to {}.", targetDirectory); + } catch (IOException e) { + LOG.warn("An error has occurred while backing up logs to {}.", targetDirectory, e); + } + } } @Override diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResourceFactory.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResourceFactory.java index bbfaef6fd2..e4c1e5c02f 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResourceFactory.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResourceFactory.java @@ -18,9 +18,13 @@ package org.apache.flink.tests.util.flink; +import org.apache.flink.tests.util.parameters.ParameterProperty; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.nio.file.Path; +import java.nio.file.Paths; import java.util.Optional; /** @@ -29,9 +33,21 @@ import java.util.Optional; public final class LocalStandaloneFlinkResourceFactory implements FlinkResourceFactory { private static final Logger LOG = LoggerFactory.getLogger(LocalStandaloneFlinkResourceFactory.class); + private static final ParameterProperty DISTRIBUTION_DIRECTORY = new ParameterProperty<>("distDir", Paths::get); + private static final ParameterProperty DISTRIBUTION_LOG_BACKUP_DIRECTORY = new ParameterProperty<>("logBackupDir", Paths::get); + @Override public Optional create(FlinkResourceSetup setup) { + Optional distributionDirectory = DISTRIBUTION_DIRECTORY.get(); + if (!distributionDirectory.isPresent()) { + LOG.warn("The distDir property was not set. You can set it when running maven via -DdistDir= ."); + return Optional.empty(); + } + Optional logBackupDirectory = DISTRIBUTION_LOG_BACKUP_DIRECTORY.get(); + if (!logBackupDirectory.isPresent()) { + LOG.warn("Property {} not set, logs will not be backed up in case of test failures.", DISTRIBUTION_LOG_BACKUP_DIRECTORY.getPropertyName()); + } LOG.info("Created {}.", LocalStandaloneFlinkResource.class.getSimpleName()); - return Optional.of(new LocalStandaloneFlinkResource(setup)); + return Optional.of(new LocalStandaloneFlinkResource(distributionDirectory.get(), logBackupDirectory.orElse(null), setup)); } } -- Gitee From 2b19918f1fd9e3d056c2b729555b9582f9f30656 Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Wed, 4 Mar 2020 16:18:20 +0100 Subject: [PATCH 084/885] [FLINK-16417][e2e] Slightly increase offheap memory for ConnectedComponents test The calculated off-heap size was 130 MB. The test passes on JDK 11 when setting it to 270 MB. The test fails on JDK 11 when setting it to 135 MB. The test passes on JDK 11 when setting it to 160 MB. This commit sets the memory to 160MB. --- .../test-scripts/test_high_parallelism_iterations.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/flink-end-to-end-tests/test-scripts/test_high_parallelism_iterations.sh b/flink-end-to-end-tests/test-scripts/test_high_parallelism_iterations.sh index a761ff3092..54a441bd7b 100755 --- a/flink-end-to-end-tests/test-scripts/test_high_parallelism_iterations.sh +++ b/flink-end-to-end-tests/test-scripts/test_high_parallelism_iterations.sh @@ -32,6 +32,7 @@ TEST_PROGRAM_JAR=${END_TO_END_DIR}/$TEST/target/$TEST_PROGRAM_NAME.jar set_config_key "taskmanager.numberOfTaskSlots" "$SLOTS_PER_TM" set_config_key "taskmanager.memory.network.min" "160m" set_config_key "taskmanager.memory.network.max" "160m" +set_config_key "taskmanager.memory.framework.off-heap.size" "160m" print_mem_use start_cluster -- Gitee From bd0d631f800cf5b8ee37e37b4798200e8d5d9a44 Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Thu, 5 Mar 2020 09:30:25 +0100 Subject: [PATCH 085/885] [FLINK-16431][AZP] Pass build profile into end to end test script on Azure --- tools/azure-pipelines/jobs-template.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tools/azure-pipelines/jobs-template.yml b/tools/azure-pipelines/jobs-template.yml index 8cba11efe7..a916fc755b 100644 --- a/tools/azure-pipelines/jobs-template.yml +++ b/tools/azure-pipelines/jobs-template.yml @@ -167,7 +167,7 @@ jobs: # TODO remove pre-commit tests script by adding the tests to the nightly script # - script: FLINK_DIR=build-target ./flink-end-to-end-tests/run-pre-commit-tests.sh # displayName: Test - precommit - - script: FLINK_DIR=`pwd`/build-target flink-end-to-end-tests/run-nightly-tests.sh + - script: ${{parameters.environment}} FLINK_DIR=`pwd`/build-target flink-end-to-end-tests/run-nightly-tests.sh displayName: Run e2e tests # upload debug artifacts - task: PublishPipelineArtifact@1 -- Gitee From 1a4fb25df802253be17f27d0f89e51c5762c3d7d Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Thu, 5 Mar 2020 09:30:47 +0100 Subject: [PATCH 086/885] [FLINK-16378][AZP] Disable Docker tests when running with JDK11 --- flink-end-to-end-tests/run-nightly-tests.sh | 40 ++++++++++----------- 1 file changed, 18 insertions(+), 22 deletions(-) diff --git a/flink-end-to-end-tests/run-nightly-tests.sh b/flink-end-to-end-tests/run-nightly-tests.sh index 624bd6dcc0..9174ed654b 100755 --- a/flink-end-to-end-tests/run-nightly-tests.sh +++ b/flink-end-to-end-tests/run-nightly-tests.sh @@ -100,12 +100,26 @@ run_test "Resuming Externalized Checkpoint after terminal failure (rocks, increm run_test "RocksDB Memory Management end-to-end test" "$END_TO_END_DIR/test-scripts/test_rocksdb_state_memory_control.sh" ################################################################################ -# Docker +# Docker / Container / Kubernetes / Mesos tests ################################################################################ -run_test "Running Kerberized YARN on Docker test (default input)" "$END_TO_END_DIR/test-scripts/test_yarn_kerberos_docker.sh" -run_test "Running Kerberized YARN on Docker test (custom fs plugin)" "$END_TO_END_DIR/test-scripts/test_yarn_kerberos_docker.sh dummy-fs" -run_test "Wordcount on Docker test (custom fs plugin)" "$END_TO_END_DIR/test-scripts/test_docker_embedded_job.sh dummy-fs" +# These tests are known to fail on JDK11. See FLINK-13719 +if [[ ${PROFILE} != *"jdk11"* ]]; then + run_test "Wordcount on Docker test (custom fs plugin)" "$END_TO_END_DIR/test-scripts/test_docker_embedded_job.sh dummy-fs" + + run_test "Running Kerberized YARN on Docker test (default input)" "$END_TO_END_DIR/test-scripts/test_yarn_kerberos_docker.sh" + run_test "Running Kerberized YARN on Docker test (custom fs plugin)" "$END_TO_END_DIR/test-scripts/test_yarn_kerberos_docker.sh dummy-fs" + + run_test "Run Kubernetes test" "$END_TO_END_DIR/test-scripts/test_kubernetes_embedded_job.sh" + run_test "Run kubernetes session test" "$END_TO_END_DIR/test-scripts/test_kubernetes_session.sh" + + if [[ $PROFILE == *"include-hadoop"* ]]; then + run_test "Run Mesos WordCount test" "$END_TO_END_DIR/test-scripts/test_mesos_wordcount.sh" + run_test "Run Mesos multiple submission test" "$END_TO_END_DIR/test-scripts/test_mesos_multiple_submissions.sh" + fi + + run_test "Test PubSub connector with Docker based Google PubSub Emulator" "$END_TO_END_DIR/test-scripts/test_streaming_gcp_pubsub.sh" +fi ################################################################################ # High Availability @@ -123,22 +137,6 @@ run_test "Running HA per-job cluster (file, sync) end-to-end test" "$END_TO_END_ run_test "Running HA per-job cluster (rocks, non-incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh rocks true false" "skip_check_exceptions" run_test "Running HA per-job cluster (rocks, incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh rocks true true" "skip_check_exceptions" -################################################################################ -# Kubernetes -################################################################################ - -run_test "Run Kubernetes test" "$END_TO_END_DIR/test-scripts/test_kubernetes_embedded_job.sh" -run_test "Run kubernetes session test" "$END_TO_END_DIR/test-scripts/test_kubernetes_session.sh" - -################################################################################ -# Mesos -################################################################################ - -if [[ $PROFILE == *"include-hadoop"* ]]; then - run_test "Run Mesos WordCount test" "$END_TO_END_DIR/test-scripts/test_mesos_wordcount.sh" - run_test "Run Mesos multiple submission test" "$END_TO_END_DIR/test-scripts/test_mesos_multiple_submissions.sh" -fi - ################################################################################ # Miscellaneous ################################################################################ @@ -170,8 +168,6 @@ run_test "Walkthrough Table Scala nightly end-to-end test" "$END_TO_END_DIR/test run_test "Walkthrough DataStream Java nightly end-to-end test" "$END_TO_END_DIR/test-scripts/test_datastream_walkthroughs.sh java" run_test "Walkthrough DataStream Scala nightly end-to-end test" "$END_TO_END_DIR/test-scripts/test_datastream_walkthroughs.sh scala" -run_test "Test PubSub connector with Docker based Google PubSub Emulator" "$END_TO_END_DIR/test-scripts/test_streaming_gcp_pubsub.sh" - if [[ ${PROFILE} != *"jdk11"* ]]; then run_test "Avro Confluent Schema Registry nightly end-to-end test" "$END_TO_END_DIR/test-scripts/test_confluent_schema_registry.sh" fi -- Gitee From b7f53c27db9f054c6251fd1d2f4900ccebfac82f Mon Sep 17 00:00:00 2001 From: hequn8128 Date: Wed, 26 Feb 2020 16:58:01 +0800 Subject: [PATCH 087/885] [FLINK-16249][python][ml] Add interfaces for Params, ParamInfo and WithParams This closes #11220. --- flink-python/README.md | 2 +- flink-python/docs/index.rst | 1 + flink-python/docs/pyflink.ml.rst | 52 +++ flink-python/pyflink/__init__.py | 13 + flink-python/pyflink/ml/__init__.py | 7 - flink-python/pyflink/ml/api/__init__.py | 24 ++ .../pyflink/ml/{ => api}/ml_environment.py | 0 .../ml/{ => api}/ml_environment_factory.py | 2 +- flink-python/pyflink/ml/api/param/__init__.py | 21 ++ flink-python/pyflink/ml/api/param/base.py | 349 ++++++++++++++++++ flink-python/pyflink/ml/lib/__init__.py | 17 + flink-python/pyflink/ml/lib/param/__init__.py | 21 ++ flink-python/pyflink/ml/lib/param/colname.py | 55 +++ .../pyflink/ml/tests/test_ml_environment.py | 2 +- .../ml/tests/test_ml_environment_factory.py | 2 +- flink-python/pyflink/ml/tests/test_params.py | 187 ++++++++++ flink-python/setup.py | 2 +- 17 files changed, 745 insertions(+), 12 deletions(-) create mode 100644 flink-python/docs/pyflink.ml.rst create mode 100644 flink-python/pyflink/ml/api/__init__.py rename flink-python/pyflink/ml/{ => api}/ml_environment.py (100%) rename flink-python/pyflink/ml/{ => api}/ml_environment_factory.py (98%) create mode 100644 flink-python/pyflink/ml/api/param/__init__.py create mode 100644 flink-python/pyflink/ml/api/param/base.py create mode 100644 flink-python/pyflink/ml/lib/__init__.py create mode 100644 flink-python/pyflink/ml/lib/param/__init__.py create mode 100644 flink-python/pyflink/ml/lib/param/colname.py create mode 100644 flink-python/pyflink/ml/tests/test_params.py diff --git a/flink-python/README.md b/flink-python/README.md index 82ad18e5be..ea7bb162fb 100644 --- a/flink-python/README.md +++ b/flink-python/README.md @@ -16,7 +16,7 @@ The auto-generated Python docs can be found at [https://ci.apache.org/projects/f ## Python Requirements -Apache Flink Python API depends on Py4J (currently version 0.10.8.1), CloudPickle (currently version 1.2.2), python-dateutil(currently version 2.8.0) and Apache Beam (currently version 2.19.0). +Apache Flink Python API depends on Py4J (currently version 0.10.8.1), CloudPickle (currently version 1.2.2), python-dateutil(currently version 2.8.0), Apache Beam (currently version 2.19.0) and jsonpickle (currently 1.2). ## Development Notices diff --git a/flink-python/docs/index.rst b/flink-python/docs/index.rst index 78c2fa1800..e6d6025dea 100644 --- a/flink-python/docs/index.rst +++ b/flink-python/docs/index.rst @@ -28,6 +28,7 @@ Welcome to Flink Python API Docs! pyflink.table pyflink.dataset pyflink.datastream + pyflink.ml Core Classes: diff --git a/flink-python/docs/pyflink.ml.rst b/flink-python/docs/pyflink.ml.rst new file mode 100644 index 0000000000..5b399b306f --- /dev/null +++ b/flink-python/docs/pyflink.ml.rst @@ -0,0 +1,52 @@ +.. ################################################################################ + 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. + ################################################################################ + +pyflink.ml package +===================== + +Module contents +--------------- + +.. automodule:: pyflink.ml + :members: + :undoc-members: + :show-inheritance: + +pyflink.ml.api module +--------------------------- +.. automodule:: pyflink.ml.api + :members: + :undoc-members: + +pyflink.ml.api.param module +--------------------------- +.. automodule:: pyflink.ml.api.param + :members: + :undoc-members: + +pyflink.ml.lib module +------------------------------------- +.. automodule:: pyflink.ml.lib + :members: + :undoc-members: + +pyflink.ml.lib.param module +--------------------------- +.. automodule:: pyflink.ml.lib.param + :members: + :undoc-members: diff --git a/flink-python/pyflink/__init__.py b/flink-python/pyflink/__init__.py index 9d68ffd99b..1fb541b06c 100644 --- a/flink-python/pyflink/__init__.py +++ b/flink-python/pyflink/__init__.py @@ -16,6 +16,7 @@ # limitations under the License. ################################################################################# import sys +from functools import wraps if sys.version_info < (3, 5): raise RuntimeError( @@ -37,3 +38,15 @@ def since(version): f.__doc__ = original_doc.rstrip() + "\n\n%s.. versionadded:: %s" % (indent, version) return f return deco + + +def keyword(func): + """ + A decorator that forces keyword arguments usage and store actual + input keyword arguments in `_input_kwargs`. + """ + @wraps(func) + def wrapper(self, **kwargs): + self._input_kwargs = kwargs + return func(self, **kwargs) + return wrapper diff --git a/flink-python/pyflink/ml/__init__.py b/flink-python/pyflink/ml/__init__.py index 94d36cdd24..65b48d4d79 100644 --- a/flink-python/pyflink/ml/__init__.py +++ b/flink-python/pyflink/ml/__init__.py @@ -15,10 +15,3 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ - -from pyflink.ml.ml_environment import MLEnvironment -from pyflink.ml.ml_environment_factory import MLEnvironmentFactory - -__all__ = [ - "MLEnvironment", "MLEnvironmentFactory" -] diff --git a/flink-python/pyflink/ml/api/__init__.py b/flink-python/pyflink/ml/api/__init__.py new file mode 100644 index 0000000000..faca34a226 --- /dev/null +++ b/flink-python/pyflink/ml/api/__init__.py @@ -0,0 +1,24 @@ +################################################################################ +# 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. +################################################################################ + +from pyflink.ml.api.ml_environment import MLEnvironment +from pyflink.ml.api.ml_environment_factory import MLEnvironmentFactory + +__all__ = [ + "MLEnvironment", "MLEnvironmentFactory" +] diff --git a/flink-python/pyflink/ml/ml_environment.py b/flink-python/pyflink/ml/api/ml_environment.py similarity index 100% rename from flink-python/pyflink/ml/ml_environment.py rename to flink-python/pyflink/ml/api/ml_environment.py diff --git a/flink-python/pyflink/ml/ml_environment_factory.py b/flink-python/pyflink/ml/api/ml_environment_factory.py similarity index 98% rename from flink-python/pyflink/ml/ml_environment_factory.py rename to flink-python/pyflink/ml/api/ml_environment_factory.py index 360b95ccae..0825fb07d0 100644 --- a/flink-python/pyflink/ml/ml_environment_factory.py +++ b/flink-python/pyflink/ml/api/ml_environment_factory.py @@ -17,7 +17,7 @@ ################################################################################ from typing import Optional -from pyflink.ml.ml_environment import MLEnvironment +from pyflink.ml.api.ml_environment import MLEnvironment from pyflink.dataset import ExecutionEnvironment from pyflink.datastream import StreamExecutionEnvironment from pyflink.table import BatchTableEnvironment, StreamTableEnvironment diff --git a/flink-python/pyflink/ml/api/param/__init__.py b/flink-python/pyflink/ml/api/param/__init__.py new file mode 100644 index 0000000000..871d887f69 --- /dev/null +++ b/flink-python/pyflink/ml/api/param/__init__.py @@ -0,0 +1,21 @@ +################################################################################ +# 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. +################################################################################ + +from pyflink.ml.api.param.base import WithParams, Params, ParamInfo, TypeConverters + +__all__ = ['WithParams', 'Params', 'ParamInfo', 'TypeConverters'] diff --git a/flink-python/pyflink/ml/api/param/base.py b/flink-python/pyflink/ml/api/param/base.py new file mode 100644 index 0000000000..b784bc766d --- /dev/null +++ b/flink-python/pyflink/ml/api/param/base.py @@ -0,0 +1,349 @@ +################################################################################ +# 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. +################################################################################ +import array +from typing import TypeVar, Generic + +V = TypeVar('V') + + +class WithParams(Generic[V]): + """ + Parameters are widely used in machine learning realm. This class defines a common + interface to interact with classes with parameters. + """ + + def get_params(self) -> 'Params': + """ + Returns all the parameters. + + :return: all the parameters. + """ + pass + + def set(self, info: 'ParamInfo', value: V) -> 'WithParams': + """ + Set the value of a specific parameter. + + :param info: the info of the specific param to set. + :param value: the value to be set to the specific param. + :return: the WithParams itself. + """ + self.get_params().set(info, value) + return self + + def get(self, info: 'ParamInfo') -> V: + """ + Returns the value of the specific param. + + :param info: the info of the specific param, usually with default value. + :return: the value of the specific param, or default value defined in the \ + ParamInfo if the inner Params doesn't contains this param. + """ + return self.get_params().get(info) + + def _set(self, **kwargs): + """ + Sets user-supplied params. + """ + for param, value in kwargs.items(): + p = getattr(self, param) + if value is not None: + try: + value = p.type_converter(value) + except TypeError as e: + raise TypeError('Invalid param value given for param "%s". %s' % (p.name, e)) + self.get_params().set(p, value) + return self + + +class Params(Generic[V]): + """ + The map-like container class for parameter. This class is provided to unify + the interaction with parameters. + """ + + def __init__(self): + self._param_map = {} + + def set(self, info: 'ParamInfo', value: V) -> 'Params': + """ + Return the number of params. + + :param info: the info of the specific parameter to set. + :param value: the value to be set to the specific parameter. + :return: return the current Params. + """ + self._param_map[info] = value + return self + + def get(self, info: 'ParamInfo') -> V: + """ + Returns the value of the specific parameter, or default value defined in the + info if this Params doesn't have a value set for the parameter. An exception + will be thrown in the following cases because no value could be found for the + specified parameter. + + :param info: the info of the specific parameter to set. + :return: the value of the specific param, or default value defined in the \ + info if this Params doesn't contain the parameter. + """ + if info not in self._param_map: + if not info.is_optional: + raise ValueError("Missing non-optional parameter %s" % info.name) + elif not info.has_default_value: + raise ValueError("Cannot find default value for optional parameter %s" % info.name) + else: + return info.default_value + else: + return self._param_map[info] + + def remove(self, info: 'ParamInfo') -> V: + """ + Removes the specific parameter from this Params. + + :param info: the info of the specific parameter to remove. + :return: the type of the specific parameter. + """ + self._param_map.pop(info) + + def contains(self, info: 'ParamInfo') -> bool: + """ + Check whether this params has a value set for the given `info`. + + :param info: the info of the specific parameter to check. + :return: `True` if this params has a value set for the specified `info`, false otherwise. + """ + return info in self._param_map + + def size(self) -> int: + """ + Return the number of params. + + :return: Return the number of params. + """ + return len(self._param_map) + + def clear(self) -> None: + """ + Removes all of the params. The params will be empty after this call returns. + + :return: None. + """ + self._param_map.clear() + + def is_empty(self) -> bool: + """ + Returns `true` if this params contains no mappings. + + :return: `true` if this params contains no mappings. + """ + return len(self._param_map) == 0 + + def to_json(self) -> str: + """ + Returns a json containing all parameters in this Params. The json should be + human-readable if possible. + + :return: a json containing all parameters in this Params. + """ + import jsonpickle + return str(jsonpickle.encode(self._param_map, keys=True)) + + def load_json(self, json: str) -> 'Params': + """ + Restores the parameters from the given json. The parameters should be exactly + the same with the one who was serialized to the input json after the restoration. + + :param json: the json String to restore from. + :return: the Params. + """ + import jsonpickle + self._param_map.update(jsonpickle.decode(json, keys=True)) + return self + + @staticmethod + def from_json(json) -> 'Params': + """ + Factory method for constructing params. + + :param json: the json string to load. + :return: the `Params` loaded from the json string. + """ + return Params().load_json(json) + + def merge(self, other_params: 'Params') -> 'Params': + """ + Merge other params into this. + + :param other_params: other params. + :return: return this Params. + """ + if other_params is not None: + self._param_map.update(other_params._param_map) + return self + + def clone(self) -> 'Params': + """ + Creates and returns a deep clone of this Params. + + :return: a clone of this Params. + """ + new_params = Params() + new_params._param_map.update(self._param_map) + return new_params + + +class ParamInfo(object): + """ + Definition of a parameter, including name, description, type_converter and so on. + """ + + def __init__(self, name, description, is_optional=True, + has_default_value=False, default_value=None, + type_converter=None): + self.name = str(name) + self.description = str(description) + self.is_optional = is_optional + self.has_default_value = has_default_value + self.default_value = default_value + self.type_converter = TypeConverters.identity if type_converter is None else type_converter + + def __str__(self): + return self.name + + def __repr__(self): + return "Param(name=%r, description=%r)" % (self.name, self.description) + + def __hash__(self): + return hash(str(self.name)) + + def __eq__(self, other): + if isinstance(other, ParamInfo): + return self.name == other.name + else: + return False + + +class TypeConverters(object): + """ + Factory methods for common type conversion functions for `Param.typeConverter`. + The TypeConverter makes PyFlink ML pipeline support more types of parameters. For example, + a list could be a list, a range or an array. Validation can also be done in the converters. + """ + + @staticmethod + def _is_numeric(value): + vtype = type(value) + return vtype in [int, float] or vtype.__name__ == 'long' + + @staticmethod + def _is_integer(value): + return TypeConverters._is_numeric(value) and float(value).is_integer() + + @staticmethod + def _can_convert_to_string(value): + return isinstance(value, str) + + @staticmethod + def identity(value): + """ + Dummy converter that just returns value. + """ + return value + + @staticmethod + def to_list(value): + """ + Convert a value to a list, if possible. + """ + if isinstance(value, list): + return value + elif type(value) in [tuple, range, array.array]: + return list(value) + else: + raise TypeError("Could not convert %s to list" % value) + + @staticmethod + def to_list_float(value): + """ + Convert a value to list of floats, if possible. + """ + value = TypeConverters.to_list(value) + if all(map(lambda v: TypeConverters._is_numeric(v), value)): + return [float(v) for v in value] + raise TypeError("Could not convert %s to list of floats" % value) + + @staticmethod + def to_list_int(value): + """ + Convert a value to list of ints, if possible. + """ + value = TypeConverters.to_list(value) + if all(map(lambda v: TypeConverters._is_integer(v), value)): + return [int(v) for v in value] + raise TypeError("Could not convert %s to list of ints" % value) + + @staticmethod + def to_list_string(value): + """ + Convert a value to list of strings, if possible. + """ + value = TypeConverters.to_list(value) + if all(map(lambda v: TypeConverters._can_convert_to_string(v), value)): + return [TypeConverters.to_string(v) for v in value] + raise TypeError("Could not convert %s to list of strings" % value) + + @staticmethod + def to_float(value: float) -> float: + """ + Convert a value to a float, if possible. + """ + if TypeConverters._is_numeric(value): + return float(value) + else: + raise TypeError("Could not convert %s to float" % value) + + @staticmethod + def to_int(value: int) -> int: + """ + Convert a value to an int, if possible. + """ + if TypeConverters._is_integer(value): + return int(value) + else: + raise TypeError("Could not convert %s to int" % value) + + @staticmethod + def to_string(value: str) -> str: + """ + Convert a value to a string, if possible. + """ + if isinstance(value, str): + return value + else: + raise TypeError("Could not convert %s to string type" % type(value)) + + @staticmethod + def to_boolean(value: bool) -> bool: + """ + Convert a value to a boolean, if possible. + """ + if isinstance(value, bool): + return value + else: + raise TypeError("Boolean Param requires value of type bool. Found %s." % type(value)) diff --git a/flink-python/pyflink/ml/lib/__init__.py b/flink-python/pyflink/ml/lib/__init__.py new file mode 100644 index 0000000000..65b48d4d79 --- /dev/null +++ b/flink-python/pyflink/ml/lib/__init__.py @@ -0,0 +1,17 @@ +################################################################################ +# 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. +################################################################################ diff --git a/flink-python/pyflink/ml/lib/param/__init__.py b/flink-python/pyflink/ml/lib/param/__init__.py new file mode 100644 index 0000000000..e0a2e2c6e5 --- /dev/null +++ b/flink-python/pyflink/ml/lib/param/__init__.py @@ -0,0 +1,21 @@ +################################################################################ +# 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. +################################################################################ + +from pyflink.ml.lib.param.colname import HasSelectedCols, HasOutputCol + +__all__ = ["HasSelectedCols", "HasOutputCol"] diff --git a/flink-python/pyflink/ml/lib/param/colname.py b/flink-python/pyflink/ml/lib/param/colname.py new file mode 100644 index 0000000000..034cd4c58d --- /dev/null +++ b/flink-python/pyflink/ml/lib/param/colname.py @@ -0,0 +1,55 @@ +################################################################################ +# 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. +################################################################################ + +from pyflink.ml.api.param import WithParams, ParamInfo, TypeConverters + + +class HasSelectedCols(WithParams): + """ + An interface for classes with a parameter specifying the name of multiple table columns. + """ + + selected_cols = ParamInfo( + "selectedCols", + "Names of the columns used for processing", + is_optional=False, + type_converter=TypeConverters.to_list_string) + + def set_selected_cols(self, v: list) -> 'HasSelectedCols': + return super().set(self.selected_cols, v) + + def get_selected_cols(self) -> list: + return super().get(self.selected_cols) + + +class HasOutputCol(WithParams): + """ + An interface for classes with a parameter specifying the name of the output column. + """ + + output_col = ParamInfo( + "outputCol", + "Name of the output column", + is_optional=False, + type_converter=TypeConverters.to_string) + + def set_output_col(self, v: str) -> 'HasOutputCol': + return super().set(self.output_col, v) + + def get_output_col(self) -> str: + return super().get(self.output_col) diff --git a/flink-python/pyflink/ml/tests/test_ml_environment.py b/flink-python/pyflink/ml/tests/test_ml_environment.py index 02d76bbff5..78f1258f44 100644 --- a/flink-python/pyflink/ml/tests/test_ml_environment.py +++ b/flink-python/pyflink/ml/tests/test_ml_environment.py @@ -18,7 +18,7 @@ import unittest -from pyflink.ml.ml_environment_factory import MLEnvironment +from pyflink.ml.api.ml_environment_factory import MLEnvironment from pyflink.dataset import ExecutionEnvironment from pyflink.datastream import StreamExecutionEnvironment from pyflink.table import BatchTableEnvironment, StreamTableEnvironment diff --git a/flink-python/pyflink/ml/tests/test_ml_environment_factory.py b/flink-python/pyflink/ml/tests/test_ml_environment_factory.py index 3b13a783db..7f381d1b37 100644 --- a/flink-python/pyflink/ml/tests/test_ml_environment_factory.py +++ b/flink-python/pyflink/ml/tests/test_ml_environment_factory.py @@ -16,7 +16,7 @@ # limitations under the License. ################################################################################ -from pyflink.ml.ml_environment_factory import MLEnvironmentFactory, MLEnvironment +from pyflink.ml.api.ml_environment_factory import MLEnvironmentFactory, MLEnvironment from pyflink.testing.test_case_utils import MLTestCase diff --git a/flink-python/pyflink/ml/tests/test_params.py b/flink-python/pyflink/ml/tests/test_params.py new file mode 100644 index 0000000000..3e7321e80b --- /dev/null +++ b/flink-python/pyflink/ml/tests/test_params.py @@ -0,0 +1,187 @@ +################################################################################ +# 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. +################################################################################ + +import array +import unittest + +from pyflink import keyword +from pyflink.ml.api.param import ParamInfo, TypeConverters, Params +from pyflink.ml.lib.param.colname import HasSelectedCols, HasOutputCol + + +class ParamsTest(unittest.TestCase): + + def test_default_behavior(self): + params = Params() + + not_optinal = ParamInfo("a", "", is_optional=False) + with self.assertRaises(ValueError): + params.get(not_optinal) + + # get optional without default param + optional_without_default = ParamInfo("a", "") + with self.assertRaises(ValueError): + params.get(optional_without_default) + + def test_get_optional_param(self): + param_info = ParamInfo( + "key", + "", + has_default_value=True, + default_value=None, + type_converter=TypeConverters.to_string) + + params = Params() + self.assertIsNone(params.get(param_info)) + + val = "3" + params.set(param_info, val) + self.assertEqual(val, params.get(param_info)) + + params.set(param_info, None) + self.assertIsNone(params.get(param_info)) + + def test_remove_contains_size_clear_is_empty(self): + param_info = ParamInfo( + "key", + "", + has_default_value=True, + default_value=None, + type_converter=TypeConverters.to_string) + + params = Params() + self.assertEqual(params.size(), 0) + self.assertTrue(params.is_empty()) + + val = "3" + params.set(param_info, val) + self.assertEqual(params.size(), 1) + self.assertFalse(params.is_empty()) + + params_json = params.to_json() + params_new = Params.from_json(params_json) + self.assertEqual(params.get(param_info), val) + self.assertEqual(params_new.get(param_info), val) + + params.clear() + self.assertEqual(params.size(), 0) + self.assertTrue(params.is_empty()) + + def test_to_from_json(self): + import jsonpickle + + param_info = ParamInfo( + "key", + "", + has_default_value=True, + default_value=None, + type_converter=TypeConverters.to_string) + + param_info_new = jsonpickle.decode(jsonpickle.encode(param_info)) + self.assertEqual(param_info_new, param_info) + + params = Params() + val = "3" + params.set(param_info, val) + params_new = Params.from_json(params.to_json()) + self.assertEqual(params_new.get(param_info), val) + + +class ParamTypeConversionTests(unittest.TestCase): + """ + Test that param type conversion happens. + """ + + def test_list(self): + l = [0, 1] + for lst_like in [l, range(2), tuple(l), array.array('l', l)]: + converted = TypeConverters.to_list(lst_like) + self.assertEqual(type(converted), list) + self.assertListEqual(converted, l) + + def test_list_float_or_list_int(self): + l = [0, 1] + for lst_like in [l, range(2), tuple(l), array.array('l', l)]: + converted1 = TypeConverters.to_list_float(lst_like) + converted2 = TypeConverters.to_list_int(lst_like) + self.assertEqual(type(converted1), list) + self.assertEqual(type(converted2), list) + self.assertListEqual(converted1, l) + self.assertListEqual(converted2, l) + + def test_list_string(self): + l = ["aa", "bb"] + for lst_like in [l, tuple(l)]: + converted = TypeConverters.to_list_string(lst_like) + self.assertEqual(type(converted), list) + self.assertListEqual(converted, l) + + def test_float(self): + data = 1.45 + converted = TypeConverters.to_float(data) + self.assertEqual(type(converted), float) + self.assertEqual(converted, data) + + def test_int(self): + data = 1234567890 + converted = TypeConverters.to_int(data) + self.assertEqual(type(converted), int) + self.assertEqual(converted, data) + + def test_string(self): + data = "1234567890" + converted = TypeConverters.to_string(data) + self.assertEqual(type(converted), str) + self.assertEqual(converted, data) + + def test_boolean(self): + data = True + converted = TypeConverters.to_boolean(data) + self.assertEqual(type(converted), bool) + self.assertEqual(converted, data) + + +class MockVectorAssembler(HasSelectedCols, HasOutputCol): + + @keyword + def __init__(self, *, selected_cols=None, output_col=None): + self._params = Params() + kwargs = self._input_kwargs + self._set(**kwargs) + + def get_params(self): + return self._params + + +class TestWithParams(unittest.TestCase): + + def test_set_params_with_keyword_arguments(self): + assembler = MockVectorAssembler(selected_cols=["a", "b"], output_col="features") + params = assembler.get_params() + self.assertEqual(params.size(), 2) + self.assertEqual(assembler.get(HasSelectedCols.selected_cols), ["a", "b"]) + self.assertEqual(assembler.get(HasOutputCol.output_col), "features") + + def test_set_params_with_builder_mode(self): + assembler = MockVectorAssembler()\ + .set_selected_cols(["a", "b"])\ + .set_output_col("features") + params = assembler.get_params() + self.assertEqual(params.size(), 2) + self.assertEqual(assembler.get(HasSelectedCols.selected_cols), ["a", "b"]) + self.assertEqual(assembler.get(HasOutputCol.output_col), "features") diff --git a/flink-python/setup.py b/flink-python/setup.py index 32cf406e43..744036e4c8 100644 --- a/flink-python/setup.py +++ b/flink-python/setup.py @@ -224,7 +224,7 @@ run sdist. author_email='dev@flink.apache.org', python_requires='>=3.5', install_requires=['py4j==0.10.8.1', 'python-dateutil==2.8.0', 'apache-beam==2.19.0', - 'cloudpickle==1.2.2', 'avro-python3>=1.8.1,<=1.9.1'], + 'cloudpickle==1.2.2', 'avro-python3>=1.8.1,<=1.9.1', 'jsonpickle==1.2'], tests_require=['pytest==4.4.1'], description='Apache Flink Python API', long_description=long_description, -- Gitee From 1aabab86f9461ff557ef9ffe506bd67556988fc7 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Fri, 6 Mar 2020 09:29:25 +0800 Subject: [PATCH 088/885] [FLINK-11899][parquet] Introduce parquet ColumnarRow split reader This closes #10922 --- flink-formats/flink-parquet/pom.xml | 8 + .../formats/parquet/ParquetTableSource.java | 4 +- .../vector/ParquetColumnarRowSplitReader.java | 307 ++++++++++ .../parquet/vector/ParquetDecimalVector.java | 58 ++ .../parquet/vector/ParquetDictionary.java | 66 +++ .../vector/ParquetSplitReaderUtil.java | 384 ++++++++++++ .../vector/reader/AbstractColumnReader.java | 318 ++++++++++ .../vector/reader/BooleanColumnReader.java | 115 ++++ .../vector/reader/ByteColumnReader.java | 99 ++++ .../vector/reader/BytesColumnReader.java | 98 ++++ .../parquet/vector/reader/ColumnReader.java | 34 ++ .../vector/reader/DoubleColumnReader.java | 101 ++++ .../reader/FixedLenBytesColumnReader.java | 144 +++++ .../vector/reader/FloatColumnReader.java | 101 ++++ .../vector/reader/IntColumnReader.java | 100 ++++ .../vector/reader/LongColumnReader.java | 101 ++++ .../vector/reader/RunLengthDecoder.java | 293 ++++++++++ .../vector/reader/ShortColumnReader.java | 84 +++ .../vector/reader/TimestampColumnReader.java | 120 ++++ .../parquet/utils/ParquetWriterUtil.java | 166 ++++++ .../flink/formats/parquet/utils/TestUtil.java | 2 +- .../ParquetColumnarRowSplitReaderTest.java | 545 ++++++++++++++++++ 22 files changed, 3245 insertions(+), 3 deletions(-) create mode 100644 flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetColumnarRowSplitReader.java create mode 100644 flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetDecimalVector.java create mode 100644 flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetDictionary.java create mode 100644 flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetSplitReaderUtil.java create mode 100644 flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/AbstractColumnReader.java create mode 100644 flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/BooleanColumnReader.java create mode 100644 flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/ByteColumnReader.java create mode 100644 flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/BytesColumnReader.java create mode 100644 flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/ColumnReader.java create mode 100644 flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/DoubleColumnReader.java create mode 100644 flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/FixedLenBytesColumnReader.java create mode 100644 flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/FloatColumnReader.java create mode 100644 flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/IntColumnReader.java create mode 100644 flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/LongColumnReader.java create mode 100644 flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/RunLengthDecoder.java create mode 100644 flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/ShortColumnReader.java create mode 100644 flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/TimestampColumnReader.java create mode 100644 flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/utils/ParquetWriterUtil.java create mode 100644 flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/vector/ParquetColumnarRowSplitReaderTest.java diff --git a/flink-formats/flink-parquet/pom.xml b/flink-formats/flink-parquet/pom.xml index 5e0171d36c..b8a7c605dc 100644 --- a/flink-formats/flink-parquet/pom.xml +++ b/flink-formats/flink-parquet/pom.xml @@ -73,6 +73,14 @@ under the License. true + + org.apache.flink + flink-table-runtime-blink_${scala.binary.version} + ${project.version} + provided + true + + diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetTableSource.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetTableSource.java index a8b90f4737..dd853ad0a2 100644 --- a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetTableSource.java +++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/ParquetTableSource.java @@ -81,12 +81,12 @@ import java.util.List; * *

      * {@code
    - * ParquetTableSource orcSrc = ParquetTableSource.builder()
    + * ParquetTableSource parquetSrc = ParquetTableSource.builder()
      *   .path("file:///my/data/file.parquet")
      *   .schema(messageType)
      *   .build();
      *
    - * tEnv.registerTableSource("parquetTable", orcSrc);
    + * tEnv.registerTableSource("parquetTable", parquetSrc);
      * Table res = tableEnv.sqlQuery("SELECT * FROM parquetTable");
      * }
      * 
    diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetColumnarRowSplitReader.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetColumnarRowSplitReader.java new file mode 100644 index 0000000000..e7bf0a742f --- /dev/null +++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetColumnarRowSplitReader.java @@ -0,0 +1,307 @@ +/* + * 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.formats.parquet.vector; + +import org.apache.flink.formats.parquet.vector.reader.AbstractColumnReader; +import org.apache.flink.formats.parquet.vector.reader.ColumnReader; +import org.apache.flink.table.dataformat.ColumnarRow; +import org.apache.flink.table.dataformat.vector.ColumnVector; +import org.apache.flink.table.dataformat.vector.VectorizedColumnBatch; +import org.apache.flink.table.dataformat.vector.writable.WritableColumnVector; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.PageReadStore; +import org.apache.parquet.filter2.compat.FilterCompat; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.Type; +import org.apache.parquet.schema.Types; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; + +import static org.apache.flink.formats.parquet.vector.ParquetSplitReaderUtil.createColumnReader; +import static org.apache.flink.formats.parquet.vector.ParquetSplitReaderUtil.createWritableColumnVector; +import static org.apache.parquet.filter2.compat.RowGroupFilter.filterRowGroups; +import static org.apache.parquet.format.converter.ParquetMetadataConverter.range; +import static org.apache.parquet.hadoop.ParquetFileReader.readFooter; +import static org.apache.parquet.hadoop.ParquetInputFormat.getFilter; + +/** + * This reader is used to read a {@link VectorizedColumnBatch} from input split. + */ +public class ParquetColumnarRowSplitReader implements Closeable { + + private final boolean utcTimestamp; + + private final MessageType fileSchema; + + private final MessageType requestedSchema; + + /** + * The total number of rows this RecordReader will eventually read. The sum of the rows of all + * the row groups. + */ + private final long totalRowCount; + + private final WritableColumnVector[] writableVectors; + + private final VectorizedColumnBatch columnarBatch; + + private final ColumnarRow row; + + private final LogicalType[] selectedTypes; + + private final int batchSize; + + private ParquetFileReader reader; + + /** + * For each request column, the reader to read this column. This is NULL if this column is + * missing from the file, in which case we populate the attribute with NULL. + */ + private ColumnReader[] columnReaders; + + /** + * The number of rows that have been returned. + */ + private long rowsReturned; + + /** + * The number of rows that have been reading, including the current in flight row group. + */ + private long totalCountLoadedSoFar; + + // the index of the next row to return + private int nextRow; + + // the number of rows in the current batch + private int rowsInBatch; + + public ParquetColumnarRowSplitReader( + boolean utcTimestamp, + Configuration conf, + LogicalType[] selectedTypes, + String[] selectedFieldNames, + ColumnBatchGenerator generator, + int batchSize, + Path path, + long splitStart, + long splitLength) throws IOException { + this.utcTimestamp = utcTimestamp; + this.selectedTypes = selectedTypes; + this.batchSize = batchSize; + // then we need to apply the predicate push down filter + ParquetMetadata footer = readFooter(conf, path, range(splitStart, splitLength)); + MessageType fileSchema = footer.getFileMetaData().getSchema(); + FilterCompat.Filter filter = getFilter(conf); + List blocks = filterRowGroups(filter, footer.getBlocks(), fileSchema); + + this.fileSchema = footer.getFileMetaData().getSchema(); + this.requestedSchema = clipParquetSchema(fileSchema, selectedFieldNames); + this.reader = new ParquetFileReader( + conf, footer.getFileMetaData(), path, blocks, requestedSchema.getColumns()); + + long totalRowCount = 0; + for (BlockMetaData block : blocks) { + totalRowCount += block.getRowCount(); + } + this.totalRowCount = totalRowCount; + this.nextRow = 0; + this.rowsInBatch = 0; + this.rowsReturned = 0; + + checkSchema(); + + this.writableVectors = createWritableVectors(); + this.columnarBatch = generator.generate(createReadableVectors()); + this.row = new ColumnarRow(columnarBatch); + } + + /** + * Clips `parquetSchema` according to `fieldNames`. + */ + private static MessageType clipParquetSchema(GroupType parquetSchema, String[] fieldNames) { + Type[] types = new Type[fieldNames.length]; + for (int i = 0; i < fieldNames.length; ++i) { + String fieldName = fieldNames[i]; + if (parquetSchema.getFieldIndex(fieldName) < 0) { + throw new IllegalArgumentException(fieldName + " does not exist"); + } + types[i] = parquetSchema.getType(fieldName); + } + return Types.buildMessage().addFields(types).named("flink-parquet"); + } + + private WritableColumnVector[] createWritableVectors() { + WritableColumnVector[] columns = new WritableColumnVector[selectedTypes.length]; + for (int i = 0; i < selectedTypes.length; i++) { + columns[i] = createWritableColumnVector( + batchSize, + selectedTypes[i], + requestedSchema.getColumns().get(i).getPrimitiveType()); + } + return columns; + } + + /** + * Create readable vectors from writable vectors. + * Especially for decimal, see {@link ParquetDecimalVector}. + */ + private ColumnVector[] createReadableVectors() { + ColumnVector[] vectors = new ColumnVector[writableVectors.length]; + for (int i = 0; i < writableVectors.length; i++) { + vectors[i] = selectedTypes[i].getTypeRoot() == LogicalTypeRoot.DECIMAL ? + new ParquetDecimalVector(writableVectors[i]) : + writableVectors[i]; + } + return vectors; + } + + private void checkSchema() throws IOException, UnsupportedOperationException { + if (selectedTypes.length != requestedSchema.getFieldCount()) { + throw new RuntimeException("The quality of field type is incompatible with the request schema!"); + } + + /* + * Check that the requested schema is supported. + */ + for (int i = 0; i < requestedSchema.getFieldCount(); ++i) { + Type t = requestedSchema.getFields().get(i); + if (!t.isPrimitive() || t.isRepetition(Type.Repetition.REPEATED)) { + throw new UnsupportedOperationException("Complex types not supported."); + } + + String[] colPath = requestedSchema.getPaths().get(i); + if (fileSchema.containsPath(colPath)) { + ColumnDescriptor fd = fileSchema.getColumnDescription(colPath); + if (!fd.equals(requestedSchema.getColumns().get(i))) { + throw new UnsupportedOperationException("Schema evolution not supported."); + } + } else { + if (requestedSchema.getColumns().get(i).getMaxDefinitionLevel() == 0) { + // Column is missing in data but the required data is non-nullable. This file is invalid. + throw new IOException("Required column is missing in data file. Col: " + Arrays.toString(colPath)); + } + } + } + } + + /** + * Method used to check if the end of the input is reached. + * + * @return True if the end is reached, otherwise false. + * @throws IOException Thrown, if an I/O error occurred. + */ + public boolean reachedEnd() throws IOException { + return !ensureBatch(); + } + + public ColumnarRow nextRecord() { + // return the next row + row.setRowId(this.nextRow++); + return row; + } + + /** + * Checks if there is at least one row left in the batch to return. If no more row are + * available, it reads another batch of rows. + * + * @return Returns true if there is one more row to return, false otherwise. + * @throws IOException throw if an exception happens while reading a batch. + */ + private boolean ensureBatch() throws IOException { + if (nextRow >= rowsInBatch) { + // No more rows available in the Rows array. + nextRow = 0; + // Try to read the next batch if rows from the file. + return nextBatch(); + } + // there is at least one Row left in the Rows array. + return true; + } + + /** + * Advances to the next batch of rows. Returns false if there are no more. + */ + private boolean nextBatch() throws IOException { + for (WritableColumnVector v : writableVectors) { + v.reset(); + } + columnarBatch.setNumRows(0); + if (rowsReturned >= totalRowCount) { + return false; + } + if (rowsReturned == totalCountLoadedSoFar) { + readNextRowGroup(); + } + + int num = (int) Math.min(batchSize, totalCountLoadedSoFar - rowsReturned); + for (int i = 0; i < columnReaders.length; ++i) { + //noinspection unchecked + columnReaders[i].readToVector(num, writableVectors[i]); + } + rowsReturned += num; + columnarBatch.setNumRows(num); + rowsInBatch = num; + return true; + } + + private void readNextRowGroup() throws IOException { + PageReadStore pages = reader.readNextRowGroup(); + if (pages == null) { + throw new IOException("expecting more rows but reached last block. Read " + + rowsReturned + " out of " + totalRowCount); + } + List columns = requestedSchema.getColumns(); + columnReaders = new AbstractColumnReader[columns.size()]; + for (int i = 0; i < columns.size(); ++i) { + columnReaders[i] = createColumnReader( + utcTimestamp, + selectedTypes[i], + columns.get(i), + pages.getPageReader(columns.get(i))); + } + totalCountLoadedSoFar += pages.getRowCount(); + } + + @Override + public void close() throws IOException { + if (reader != null) { + reader.close(); + reader = null; + } + } + + /** + * Interface to gen {@link VectorizedColumnBatch}. + */ + public interface ColumnBatchGenerator { + VectorizedColumnBatch generate(ColumnVector[] readVectors); + } +} diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetDecimalVector.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetDecimalVector.java new file mode 100644 index 0000000000..288aed0c0c --- /dev/null +++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetDecimalVector.java @@ -0,0 +1,58 @@ +/* + * 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.formats.parquet.vector; + +import org.apache.flink.table.dataformat.Decimal; +import org.apache.flink.table.dataformat.vector.BytesColumnVector; +import org.apache.flink.table.dataformat.vector.ColumnVector; +import org.apache.flink.table.dataformat.vector.DecimalColumnVector; +import org.apache.flink.table.dataformat.vector.IntColumnVector; +import org.apache.flink.table.dataformat.vector.LongColumnVector; + +/** + * Parquet write decimal as int32 and int64 and binary, this class wrap the real vector to + * provide {@link DecimalColumnVector} interface. + */ +public class ParquetDecimalVector implements DecimalColumnVector { + + private final ColumnVector vector; + + ParquetDecimalVector(ColumnVector vector) { + this.vector = vector; + } + + @Override + public Decimal getDecimal(int i, int precision, int scale) { + if (Decimal.is32BitDecimal(precision)) { + return Decimal.fromUnscaledLong( + precision, scale, ((IntColumnVector) vector).getInt(i)); + } else if (Decimal.is64BitDecimal(precision)) { + return Decimal.fromUnscaledLong( + precision, scale, ((LongColumnVector) vector).getLong(i)); + } else { + return Decimal.fromUnscaledBytes( + precision, scale, ((BytesColumnVector) vector).getBytes(i).getBytes()); + } + } + + @Override + public boolean isNullAt(int i) { + return vector.isNullAt(i); + } +} diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetDictionary.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetDictionary.java new file mode 100644 index 0000000000..c93025eccc --- /dev/null +++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetDictionary.java @@ -0,0 +1,66 @@ +/* + * 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.formats.parquet.vector; + +import org.apache.flink.table.dataformat.SqlTimestamp; +import org.apache.flink.table.dataformat.vector.Dictionary; + +import static org.apache.flink.formats.parquet.vector.reader.TimestampColumnReader.decodeInt96ToTimestamp; + +/** + * Parquet dictionary. + */ +public final class ParquetDictionary implements Dictionary { + + private org.apache.parquet.column.Dictionary dictionary; + + public ParquetDictionary(org.apache.parquet.column.Dictionary dictionary) { + this.dictionary = dictionary; + } + + @Override + public int decodeToInt(int id) { + return dictionary.decodeToInt(id); + } + + @Override + public long decodeToLong(int id) { + return dictionary.decodeToLong(id); + } + + @Override + public float decodeToFloat(int id) { + return dictionary.decodeToFloat(id); + } + + @Override + public double decodeToDouble(int id) { + return dictionary.decodeToDouble(id); + } + + @Override + public byte[] decodeToBinary(int id) { + return dictionary.decodeToBinary(id).getBytes(); + } + + @Override + public SqlTimestamp decodeToTimestamp(int id) { + return decodeInt96ToTimestamp(true, dictionary, id); + } +} diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetSplitReaderUtil.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetSplitReaderUtil.java new file mode 100644 index 0000000000..dab7ff9f3a --- /dev/null +++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetSplitReaderUtil.java @@ -0,0 +1,384 @@ +/* + * 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.formats.parquet.vector; + +import org.apache.flink.core.fs.Path; +import org.apache.flink.formats.parquet.vector.reader.BooleanColumnReader; +import org.apache.flink.formats.parquet.vector.reader.ByteColumnReader; +import org.apache.flink.formats.parquet.vector.reader.BytesColumnReader; +import org.apache.flink.formats.parquet.vector.reader.ColumnReader; +import org.apache.flink.formats.parquet.vector.reader.DoubleColumnReader; +import org.apache.flink.formats.parquet.vector.reader.FixedLenBytesColumnReader; +import org.apache.flink.formats.parquet.vector.reader.FloatColumnReader; +import org.apache.flink.formats.parquet.vector.reader.IntColumnReader; +import org.apache.flink.formats.parquet.vector.reader.LongColumnReader; +import org.apache.flink.formats.parquet.vector.reader.ShortColumnReader; +import org.apache.flink.formats.parquet.vector.reader.TimestampColumnReader; +import org.apache.flink.table.dataformat.Decimal; +import org.apache.flink.table.dataformat.SqlTimestamp; +import org.apache.flink.table.dataformat.vector.ColumnVector; +import org.apache.flink.table.dataformat.vector.VectorizedColumnBatch; +import org.apache.flink.table.dataformat.vector.heap.HeapBooleanVector; +import org.apache.flink.table.dataformat.vector.heap.HeapByteVector; +import org.apache.flink.table.dataformat.vector.heap.HeapBytesVector; +import org.apache.flink.table.dataformat.vector.heap.HeapDoubleVector; +import org.apache.flink.table.dataformat.vector.heap.HeapFloatVector; +import org.apache.flink.table.dataformat.vector.heap.HeapIntVector; +import org.apache.flink.table.dataformat.vector.heap.HeapLongVector; +import org.apache.flink.table.dataformat.vector.heap.HeapShortVector; +import org.apache.flink.table.dataformat.vector.heap.HeapTimestampVector; +import org.apache.flink.table.dataformat.vector.writable.WritableColumnVector; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.VarBinaryType; +import org.apache.flink.util.Preconditions; + +import org.apache.hadoop.conf.Configuration; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.schema.OriginalType; +import org.apache.parquet.schema.PrimitiveType; + +import java.io.IOException; +import java.math.BigDecimal; +import java.nio.charset.StandardCharsets; +import java.sql.Date; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.flink.table.runtime.functions.SqlDateTimeUtils.dateToInternal; +import static org.apache.parquet.Preconditions.checkArgument; + +/** + * Util for generating {@link ParquetColumnarRowSplitReader}. + */ +public class ParquetSplitReaderUtil { + + /** + * Util for generating partitioned {@link ParquetColumnarRowSplitReader}. + */ + public static ParquetColumnarRowSplitReader genPartColumnarRowReader( + boolean utcTimestamp, + Configuration conf, + String[] fullFieldNames, + DataType[] fullFieldTypes, + Map partitionSpec, + int[] selectedFields, + int batchSize, + Path path, + long splitStart, + long splitLength) throws IOException { + List nonPartNames = Arrays.stream(fullFieldNames) + .filter(n -> !partitionSpec.containsKey(n)) + .collect(Collectors.toList()); + + List selNonPartNames = Arrays.stream(selectedFields) + .mapToObj(i -> fullFieldNames[i]) + .filter(nonPartNames::contains).collect(Collectors.toList()); + + int[] selParquetFields = selNonPartNames.stream() + .mapToInt(nonPartNames::indexOf) + .toArray(); + + ParquetColumnarRowSplitReader.ColumnBatchGenerator gen = readVectors -> { + // create and initialize the row batch + ColumnVector[] vectors = new ColumnVector[selectedFields.length]; + for (int i = 0; i < vectors.length; i++) { + String name = fullFieldNames[selectedFields[i]]; + LogicalType type = fullFieldTypes[selectedFields[i]].getLogicalType(); + vectors[i] = partitionSpec.containsKey(name) ? + createVectorFromConstant(type, partitionSpec.get(name), batchSize) : + readVectors[i]; + } + return new VectorizedColumnBatch(vectors); + }; + + return new ParquetColumnarRowSplitReader( + utcTimestamp, + conf, + Arrays.stream(selParquetFields) + .mapToObj(i -> fullFieldTypes[i].getLogicalType()) + .toArray(LogicalType[]::new), + selNonPartNames.toArray(new String[0]), + gen, + batchSize, + new org.apache.hadoop.fs.Path(path.toUri()), + splitStart, + splitLength); + } + + private static ColumnVector createVectorFromConstant( + LogicalType type, + Object value, + int batchSize) { + switch (type.getTypeRoot()) { + case CHAR: + case VARCHAR: + case BINARY: + case VARBINARY: + HeapBytesVector bsv = new HeapBytesVector(batchSize); + if (value == null) { + bsv.fillWithNulls(); + } else { + bsv.fill(value instanceof byte[] ? + (byte[]) value : + value.toString().getBytes(StandardCharsets.UTF_8)); + } + return bsv; + case BOOLEAN: + HeapBooleanVector bv = new HeapBooleanVector(batchSize); + if (value == null) { + bv.fillWithNulls(); + } else { + bv.fill((boolean) value); + } + return bv; + case TINYINT: + HeapByteVector byteVector = new HeapByteVector(batchSize); + if (value == null) { + byteVector.fillWithNulls(); + } else { + byteVector.fill(((Number) value).byteValue()); + } + return byteVector; + case SMALLINT: + HeapShortVector sv = new HeapShortVector(batchSize); + if (value == null) { + sv.fillWithNulls(); + } else { + sv.fill(((Number) value).shortValue()); + } + return sv; + case INTEGER: + HeapIntVector iv = new HeapIntVector(batchSize); + if (value == null) { + iv.fillWithNulls(); + } else { + iv.fill(((Number) value).intValue()); + } + return iv; + case BIGINT: + HeapLongVector lv = new HeapLongVector(batchSize); + if (value == null) { + lv.fillWithNulls(); + } else { + lv.fill(((Number) value).longValue()); + } + return lv; + case DECIMAL: + DecimalType decimalType = (DecimalType) type; + int precision = decimalType.getPrecision(); + int scale = decimalType.getScale(); + Decimal decimal = value == null ? null : Preconditions.checkNotNull( + Decimal.fromBigDecimal((BigDecimal) value, precision, scale)); + ColumnVector internalVector; + if (Decimal.is32BitDecimal(precision)) { + internalVector = createVectorFromConstant( + new IntType(), + decimal == null ? null : (int) decimal.toUnscaledLong(), + batchSize); + } else if (Decimal.is64BitDecimal(precision)) { + internalVector = createVectorFromConstant( + new BigIntType(), + decimal == null ? null : decimal.toUnscaledLong(), + batchSize); + } else { + internalVector = createVectorFromConstant( + new VarBinaryType(), + decimal == null ? null : decimal.toUnscaledBytes(), + batchSize); + } + return new ParquetDecimalVector(internalVector); + case FLOAT: + HeapFloatVector fv = new HeapFloatVector(batchSize); + if (value == null) { + fv.fillWithNulls(); + } else { + fv.fill(((Number) value).floatValue()); + } + return fv; + case DOUBLE: + HeapDoubleVector dv = new HeapDoubleVector(batchSize); + if (value == null) { + dv.fillWithNulls(); + } else { + dv.fill(((Number) value).doubleValue()); + } + return dv; + case DATE: + if (value instanceof LocalDate) { + value = Date.valueOf((LocalDate) value); + } + return createVectorFromConstant( + new IntType(), + value == null ? null : dateToInternal((Date) value), + batchSize); + case TIMESTAMP_WITHOUT_TIME_ZONE: + HeapTimestampVector tv = new HeapTimestampVector(batchSize); + if (value == null) { + tv.fillWithNulls(); + } else { + tv.fill(SqlTimestamp.fromLocalDateTime((LocalDateTime) value)); + } + return tv; + default: + throw new UnsupportedOperationException("Unsupported type: " + type); + } + } + + public static ColumnReader createColumnReader( + boolean utcTimestamp, + LogicalType fieldType, + ColumnDescriptor descriptor, + PageReader pageReader) throws IOException { + switch (fieldType.getTypeRoot()) { + case BOOLEAN: + return new BooleanColumnReader(descriptor, pageReader); + case TINYINT: + return new ByteColumnReader(descriptor, pageReader); + case DOUBLE: + return new DoubleColumnReader(descriptor, pageReader); + case FLOAT: + return new FloatColumnReader(descriptor, pageReader); + case INTEGER: + case DATE: + case TIME_WITHOUT_TIME_ZONE: + return new IntColumnReader(descriptor, pageReader); + case BIGINT: + return new LongColumnReader(descriptor, pageReader); + case SMALLINT: + return new ShortColumnReader(descriptor, pageReader); + case CHAR: + case VARCHAR: + case BINARY: + case VARBINARY: + return new BytesColumnReader(descriptor, pageReader); + case TIMESTAMP_WITHOUT_TIME_ZONE: + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return new TimestampColumnReader(utcTimestamp, descriptor, pageReader); + case DECIMAL: + switch (descriptor.getPrimitiveType().getPrimitiveTypeName()) { + case INT32: + return new IntColumnReader(descriptor, pageReader); + case INT64: + return new LongColumnReader(descriptor, pageReader); + case BINARY: + return new BytesColumnReader(descriptor, pageReader); + case FIXED_LEN_BYTE_ARRAY: + return new FixedLenBytesColumnReader( + descriptor, pageReader, ((DecimalType) fieldType).getPrecision()); + } + default: + throw new UnsupportedOperationException(fieldType + " is not supported now."); + } + } + + public static WritableColumnVector createWritableColumnVector( + int batchSize, + LogicalType fieldType, + PrimitiveType primitiveType) { + PrimitiveType.PrimitiveTypeName typeName = primitiveType.getPrimitiveTypeName(); + switch (fieldType.getTypeRoot()) { + case BOOLEAN: + checkArgument( + typeName == PrimitiveType.PrimitiveTypeName.BOOLEAN, + "Unexpected type: %s", typeName); + return new HeapBooleanVector(batchSize); + case TINYINT: + checkArgument( + typeName == PrimitiveType.PrimitiveTypeName.INT32, + "Unexpected type: %s", typeName); + return new HeapByteVector(batchSize); + case DOUBLE: + checkArgument( + typeName == PrimitiveType.PrimitiveTypeName.DOUBLE, + "Unexpected type: %s", typeName); + return new HeapDoubleVector(batchSize); + case FLOAT: + checkArgument( + typeName == PrimitiveType.PrimitiveTypeName.FLOAT, + "Unexpected type: %s", typeName); + return new HeapFloatVector(batchSize); + case INTEGER: + case DATE: + case TIME_WITHOUT_TIME_ZONE: + checkArgument( + typeName == PrimitiveType.PrimitiveTypeName.INT32, + "Unexpected type: %s", typeName); + return new HeapIntVector(batchSize); + case BIGINT: + checkArgument( + typeName == PrimitiveType.PrimitiveTypeName.INT64, + "Unexpected type: %s", typeName); + return new HeapLongVector(batchSize); + case SMALLINT: + checkArgument( + typeName == PrimitiveType.PrimitiveTypeName.INT32, + "Unexpected type: %s", typeName); + return new HeapShortVector(batchSize); + case CHAR: + case VARCHAR: + case BINARY: + case VARBINARY: + checkArgument( + typeName == PrimitiveType.PrimitiveTypeName.BINARY, + "Unexpected type: %s", typeName); + return new HeapBytesVector(batchSize); + case TIMESTAMP_WITHOUT_TIME_ZONE: + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + checkArgument( + typeName == PrimitiveType.PrimitiveTypeName.INT96, + "Unexpected type: %s", typeName); + return new HeapTimestampVector(batchSize); + case DECIMAL: + DecimalType decimalType = (DecimalType) fieldType; + if (Decimal.is32BitDecimal(decimalType.getPrecision())) { + checkArgument( + (typeName == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY || + typeName == PrimitiveType.PrimitiveTypeName.INT32) && + primitiveType.getOriginalType() == OriginalType.DECIMAL, + "Unexpected type: %s", typeName); + return new HeapIntVector(batchSize); + } else if (Decimal.is64BitDecimal(decimalType.getPrecision())) { + checkArgument( + (typeName == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY || + typeName == PrimitiveType.PrimitiveTypeName.INT64) && + primitiveType.getOriginalType() == OriginalType.DECIMAL, + "Unexpected type: %s", typeName); + return new HeapLongVector(batchSize); + } else { + checkArgument( + (typeName == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY || + typeName == PrimitiveType.PrimitiveTypeName.BINARY) && + primitiveType.getOriginalType() == OriginalType.DECIMAL, + "Unexpected type: %s", typeName); + return new HeapBytesVector(batchSize); + } + default: + throw new UnsupportedOperationException(fieldType + " is not supported now."); + } + } +} diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/AbstractColumnReader.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/AbstractColumnReader.java new file mode 100644 index 0000000000..786f546cb0 --- /dev/null +++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/AbstractColumnReader.java @@ -0,0 +1,318 @@ +/* + * 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.formats.parquet.vector.reader; + +import org.apache.flink.formats.parquet.vector.ParquetDictionary; +import org.apache.flink.table.dataformat.vector.writable.WritableColumnVector; +import org.apache.flink.table.dataformat.vector.writable.WritableIntVector; + +import org.apache.parquet.Preconditions; +import org.apache.parquet.bytes.ByteBufferInputStream; +import org.apache.parquet.bytes.BytesInput; +import org.apache.parquet.bytes.BytesUtils; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.Dictionary; +import org.apache.parquet.column.Encoding; +import org.apache.parquet.column.impl.ColumnReaderImpl; +import org.apache.parquet.column.page.DataPage; +import org.apache.parquet.column.page.DataPageV1; +import org.apache.parquet.column.page.DataPageV2; +import org.apache.parquet.column.page.DictionaryPage; +import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.column.values.ValuesReader; +import org.apache.parquet.io.ParquetDecodingException; +import org.apache.parquet.schema.PrimitiveType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +import static org.apache.parquet.column.ValuesType.REPETITION_LEVEL; + +/** + * Abstract {@link ColumnReader}. + * See {@link ColumnReaderImpl}, part of the code is referred from Apache Spark and Apache Parquet. + */ +public abstract class AbstractColumnReader + implements ColumnReader { + + private static final Logger LOG = LoggerFactory.getLogger(AbstractColumnReader.class); + + private final PageReader pageReader; + + /** + * The dictionary, if this column has dictionary encoding. + */ + protected final Dictionary dictionary; + + /** + * Maximum definition level for this column. + */ + protected final int maxDefLevel; + + protected final ColumnDescriptor descriptor; + + /** + * Total number of values read. + */ + private long valuesRead; + + /** + * value that indicates the end of the current page. That is, if valuesRead == + * endOfPageValueCount, we are at the end of the page. + */ + private long endOfPageValueCount; + + /** + * If true, the current page is dictionary encoded. + */ + private boolean isCurrentPageDictionaryEncoded; + + /** + * Total values in the current page. + */ + private int pageValueCount; + + /* + * Input streams: + * 1.Run length encoder to encode every data, so we have run length stream to get + * run length information. + * 2.Data maybe is real data, maybe is dictionary ids which need be decode to real + * data from Dictionary. + * + * Run length stream ------> Data stream + * | + * ------> Dictionary ids stream + */ + + /** + * Run length decoder for data and dictionary. + */ + protected RunLengthDecoder runLenDecoder; + + /** + * Data input stream. + */ + ByteBufferInputStream dataInputStream; + + /** + * Dictionary decoder to wrap dictionary ids input stream. + */ + private RunLengthDecoder dictionaryIdsDecoder; + + public AbstractColumnReader( + ColumnDescriptor descriptor, + PageReader pageReader) throws IOException { + this.descriptor = descriptor; + this.pageReader = pageReader; + this.maxDefLevel = descriptor.getMaxDefinitionLevel(); + + DictionaryPage dictionaryPage = pageReader.readDictionaryPage(); + if (dictionaryPage != null) { + try { + this.dictionary = dictionaryPage.getEncoding().initDictionary(descriptor, dictionaryPage); + this.isCurrentPageDictionaryEncoded = true; + } catch (IOException e) { + throw new IOException("could not decode the dictionary for " + descriptor, e); + } + } else { + this.dictionary = null; + this.isCurrentPageDictionaryEncoded = false; + } + /* + * Total number of values in this column (in this row group). + */ + long totalValueCount = pageReader.getTotalValueCount(); + if (totalValueCount == 0) { + throw new IOException("totalValueCount == 0"); + } + } + + protected void checkTypeName(PrimitiveType.PrimitiveTypeName expectedName) { + PrimitiveType.PrimitiveTypeName actualName = descriptor.getPrimitiveType().getPrimitiveTypeName(); + Preconditions.checkArgument( + actualName == expectedName, + "Expected type name: %s, actual type name: %s", + expectedName, + actualName); + } + + /** + * Reads `total` values from this columnReader into column. + */ + @Override + public final void readToVector(int readNumber, VECTOR vector) throws IOException { + int rowId = 0; + WritableIntVector dictionaryIds = null; + if (dictionary != null) { + dictionaryIds = vector.reserveDictionaryIds(readNumber); + } + while (readNumber > 0) { + // Compute the number of values we want to read in this page. + int leftInPage = (int) (endOfPageValueCount - valuesRead); + if (leftInPage == 0) { + DataPage page = pageReader.readPage(); + if (page instanceof DataPageV1) { + readPageV1((DataPageV1) page); + } else if (page instanceof DataPageV2) { + readPageV2((DataPageV2) page); + } else { + throw new RuntimeException("Unsupported page type: " + page.getClass()); + } + leftInPage = (int) (endOfPageValueCount - valuesRead); + } + int num = Math.min(readNumber, leftInPage); + if (isCurrentPageDictionaryEncoded) { + // Read and decode dictionary ids. + runLenDecoder.readDictionaryIds( + num, dictionaryIds, vector, rowId, maxDefLevel, this.dictionaryIdsDecoder); + + if (vector.hasDictionary() || (rowId == 0 && supportLazyDecode())) { + // Column vector supports lazy decoding of dictionary values so just set the dictionary. + // We can't do this if rowId != 0 AND the column doesn't have a dictionary (i.e. some + // non-dictionary encoded values have already been added). + vector.setDictionary(new ParquetDictionary(dictionary)); + } else { + readBatchFromDictionaryIds(rowId, num, vector, dictionaryIds); + } + } else { + if (vector.hasDictionary() && rowId != 0) { + // This batch already has dictionary encoded values but this new page is not. The batch + // does not support a mix of dictionary and not so we will decode the dictionary. + readBatchFromDictionaryIds(0, rowId, vector, vector.getDictionaryIds()); + } + vector.setDictionary(null); + readBatch(rowId, num, vector); + } + + valuesRead += num; + rowId += num; + readNumber -= num; + } + } + + private void readPageV1(DataPageV1 page) throws IOException { + this.pageValueCount = page.getValueCount(); + ValuesReader rlReader = page.getRlEncoding().getValuesReader(descriptor, REPETITION_LEVEL); + + // Initialize the decoders. + if (page.getDlEncoding() != Encoding.RLE && descriptor.getMaxDefinitionLevel() != 0) { + throw new UnsupportedOperationException("Unsupported encoding: " + page.getDlEncoding()); + } + int bitWidth = BytesUtils.getWidthFromMaxInt(descriptor.getMaxDefinitionLevel()); + this.runLenDecoder = new RunLengthDecoder(bitWidth); + try { + BytesInput bytes = page.getBytes(); + ByteBufferInputStream in = bytes.toInputStream(); + rlReader.initFromPage(pageValueCount, in); + this.runLenDecoder.initFromStream(pageValueCount, in); + prepareNewPage(page.getValueEncoding(), in); + } catch (IOException e) { + throw new IOException("could not read page " + page + " in col " + descriptor, e); + } + } + + private void readPageV2(DataPageV2 page) throws IOException { + this.pageValueCount = page.getValueCount(); + + int bitWidth = BytesUtils.getWidthFromMaxInt(descriptor.getMaxDefinitionLevel()); + // do not read the length from the stream. v2 pages handle dividing the page bytes. + this.runLenDecoder = new RunLengthDecoder(bitWidth, false); + this.runLenDecoder.initFromStream( + this.pageValueCount, page.getDefinitionLevels().toInputStream()); + try { + prepareNewPage(page.getDataEncoding(), page.getData().toInputStream()); + } catch (IOException e) { + throw new IOException("could not read page " + page + " in col " + descriptor, e); + } + } + + private void prepareNewPage( + Encoding dataEncoding, + ByteBufferInputStream in) throws IOException { + this.endOfPageValueCount = valuesRead + pageValueCount; + if (dataEncoding.usesDictionary()) { + if (dictionary == null) { + throw new IOException( + "could not read page in col " + descriptor + + " as the dictionary was missing for encoding " + dataEncoding); + } + @SuppressWarnings("deprecation") + Encoding plainDict = Encoding.PLAIN_DICTIONARY; // var to allow warning suppression + if (dataEncoding != plainDict && dataEncoding != Encoding.RLE_DICTIONARY) { + throw new UnsupportedOperationException("Unsupported encoding: " + dataEncoding); + } + this.dataInputStream = null; + this.dictionaryIdsDecoder = new RunLengthDecoder(); + try { + this.dictionaryIdsDecoder.initFromStream(pageValueCount, in); + } catch (IOException e) { + throw new IOException("could not read dictionary in col " + descriptor, e); + } + this.isCurrentPageDictionaryEncoded = true; + } else { + if (dataEncoding != Encoding.PLAIN) { + throw new UnsupportedOperationException("Unsupported encoding: " + dataEncoding); + } + this.dictionaryIdsDecoder = null; + LOG.debug("init from page at offset {} for length {}", in.position(), in.available()); + this.dataInputStream = in.remainingStream(); + this.isCurrentPageDictionaryEncoded = false; + } + + afterReadPage(); + } + + final ByteBuffer readDataBuffer(int length) { + try { + return dataInputStream.slice(length).order(ByteOrder.LITTLE_ENDIAN); + } catch (IOException e) { + throw new ParquetDecodingException("Failed to read " + length + " bytes", e); + } + } + + /** + * After read a page, we may need some initialization. + */ + protected void afterReadPage() {} + + /** + * Support lazy dictionary ids decode. See more in {@link ParquetDictionary}. + * If return false, we will decode all the data first. + */ + protected boolean supportLazyDecode() { + return true; + } + + /** + * Read batch from {@link #runLenDecoder} and {@link #dataInputStream}. + */ + protected abstract void readBatch(int rowId, int num, VECTOR column); + + /** + * Decode dictionary ids to data. + * From {@link #runLenDecoder} and {@link #dictionaryIdsDecoder}. + */ + protected abstract void readBatchFromDictionaryIds( + int rowId, + int num, + VECTOR column, + WritableIntVector dictionaryIds); +} diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/BooleanColumnReader.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/BooleanColumnReader.java new file mode 100644 index 0000000000..504a50479a --- /dev/null +++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/BooleanColumnReader.java @@ -0,0 +1,115 @@ +/* + * 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.formats.parquet.vector.reader; + +import org.apache.flink.table.dataformat.vector.writable.WritableBooleanVector; +import org.apache.flink.table.dataformat.vector.writable.WritableIntVector; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.io.ParquetDecodingException; +import org.apache.parquet.schema.PrimitiveType; + +import java.io.IOException; + +/** + * Boolean {@link ColumnReader}. + */ +public class BooleanColumnReader extends AbstractColumnReader { + + /** + * Parquet use a bit to store booleans, so we need split a byte to 8 boolean. + */ + private int bitOffset; + private byte currentByte = 0; + + public BooleanColumnReader( + ColumnDescriptor descriptor, + PageReader pageReader) throws IOException { + super(descriptor, pageReader); + checkTypeName(PrimitiveType.PrimitiveTypeName.BOOLEAN); + } + + @Override + protected boolean supportLazyDecode() { + return true; + } + + @Override + protected void afterReadPage() { + bitOffset = 0; + currentByte = 0; + } + + @Override + protected void readBatchFromDictionaryIds(int rowId, int num, WritableBooleanVector column, + WritableIntVector dictionaryIds) { + throw new UnsupportedOperationException(); + } + + @Override + protected void readBatch(int rowId, int num, WritableBooleanVector column) { + int left = num; + while (left > 0) { + if (runLenDecoder.currentCount == 0) { + runLenDecoder.readNextGroup(); + } + int n = Math.min(left, runLenDecoder.currentCount); + switch (runLenDecoder.mode) { + case RLE: + if (runLenDecoder.currentValue == maxDefLevel) { + for (int i = 0; i < n; i++) { + column.setBoolean(rowId + i, readBoolean()); + } + } else { + column.setNulls(rowId, n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (runLenDecoder.currentBuffer[runLenDecoder.currentBufferIdx++] == maxDefLevel) { + column.setBoolean(rowId + i, readBoolean()); + } else { + column.setNullAt(rowId + i); + } + } + break; + } + rowId += n; + left -= n; + runLenDecoder.currentCount -= n; + } + } + + private boolean readBoolean() { + if (bitOffset == 0) { + try { + currentByte = (byte) dataInputStream.read(); + } catch (IOException e) { + throw new ParquetDecodingException("Failed to read a byte", e); + } + } + + boolean v = (currentByte & (1 << bitOffset)) != 0; + bitOffset += 1; + if (bitOffset == 8) { + bitOffset = 0; + } + return v; + } +} diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/ByteColumnReader.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/ByteColumnReader.java new file mode 100644 index 0000000000..ada34058a6 --- /dev/null +++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/ByteColumnReader.java @@ -0,0 +1,99 @@ +/* + * 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.formats.parquet.vector.reader; + +import org.apache.flink.table.dataformat.vector.writable.WritableByteVector; +import org.apache.flink.table.dataformat.vector.writable.WritableIntVector; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.schema.PrimitiveType; + +import java.io.IOException; +import java.nio.ByteBuffer; + +/** + * Byte {@link ColumnReader}. Using INT32 to store byte, so just cast int to byte. + */ +public class ByteColumnReader extends AbstractColumnReader { + + public ByteColumnReader( + ColumnDescriptor descriptor, + PageReader pageReader) throws IOException { + super(descriptor, pageReader); + checkTypeName(PrimitiveType.PrimitiveTypeName.INT32); + } + + @Override + protected void readBatch(int rowId, int num, WritableByteVector column) { + int left = num; + while (left > 0) { + if (runLenDecoder.currentCount == 0) { + runLenDecoder.readNextGroup(); + } + int n = Math.min(left, runLenDecoder.currentCount); + switch (runLenDecoder.mode) { + case RLE: + if (runLenDecoder.currentValue == maxDefLevel) { + readBytes(n, column, rowId); + } else { + column.setNulls(rowId, n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (runLenDecoder.currentBuffer[runLenDecoder.currentBufferIdx++] == maxDefLevel) { + column.setByte(rowId + i, readByte()); + } else { + column.setNullAt(rowId + i); + } + } + break; + } + rowId += n; + left -= n; + runLenDecoder.currentCount -= n; + } + } + + @Override + protected void readBatchFromDictionaryIds(int rowId, int num, WritableByteVector column, + WritableIntVector dictionaryIds) { + for (int i = rowId; i < rowId + num; ++i) { + if (!column.isNullAt(i)) { + column.setByte(i, (byte) dictionary.decodeToInt(dictionaryIds.getInt(i))); + } + } + } + + private byte readByte() { + return (byte) readDataBuffer(4).getInt(); + } + + private void readBytes(int total, WritableByteVector c, int rowId) { + // Bytes are stored as a 4-byte little endian int. Just read the first byte. + int requiredBytes = total * 4; + ByteBuffer buffer = readDataBuffer(requiredBytes); + + for (int i = 0; i < total; i += 1) { + c.setByte(rowId + i, buffer.get()); + // skip the next 3 bytes + buffer.position(buffer.position() + 3); + } + } +} diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/BytesColumnReader.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/BytesColumnReader.java new file mode 100644 index 0000000000..3f6a135e37 --- /dev/null +++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/BytesColumnReader.java @@ -0,0 +1,98 @@ +/* + * 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.formats.parquet.vector.reader; + +import org.apache.flink.table.dataformat.vector.writable.WritableBytesVector; +import org.apache.flink.table.dataformat.vector.writable.WritableIntVector; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.schema.PrimitiveType; + +import java.io.IOException; +import java.nio.ByteBuffer; + +/** + * Bytes {@link ColumnReader}. A int length and bytes data. + */ +public class BytesColumnReader extends AbstractColumnReader { + + public BytesColumnReader( + ColumnDescriptor descriptor, + PageReader pageReader) throws IOException { + super(descriptor, pageReader); + checkTypeName(PrimitiveType.PrimitiveTypeName.BINARY); + } + + @Override + protected void readBatch(int rowId, int num, WritableBytesVector column) { + int left = num; + while (left > 0) { + if (runLenDecoder.currentCount == 0) { + runLenDecoder.readNextGroup(); + } + int n = Math.min(left, runLenDecoder.currentCount); + switch (runLenDecoder.mode) { + case RLE: + if (runLenDecoder.currentValue == maxDefLevel) { + readBinary(n, column, rowId); + } else { + column.setNulls(rowId, n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (runLenDecoder.currentBuffer[runLenDecoder.currentBufferIdx++] == maxDefLevel) { + readBinary(1, column, rowId + i); + } else { + column.setNullAt(rowId + i); + } + } + break; + } + rowId += n; + left -= n; + runLenDecoder.currentCount -= n; + } + } + + @Override + protected void readBatchFromDictionaryIds(int rowId, int num, WritableBytesVector column, + WritableIntVector dictionaryIds) { + for (int i = rowId; i < rowId + num; ++i) { + if (!column.isNullAt(i)) { + byte[] bytes = dictionary.decodeToBinary(dictionaryIds.getInt(i)).getBytes(); + column.appendBytes(i, bytes, 0, bytes.length); + } + } + } + + private void readBinary(int total, WritableBytesVector v, int rowId) { + for (int i = 0; i < total; i++) { + int len = readDataBuffer(4).getInt(); + ByteBuffer buffer = readDataBuffer(len); + if (buffer.hasArray()) { + v.appendBytes(rowId + i, buffer.array(), buffer.arrayOffset() + buffer.position(), len); + } else { + byte[] bytes = new byte[len]; + buffer.get(bytes); + v.appendBytes(rowId + i, bytes, 0, bytes.length); + } + } + } +} diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/ColumnReader.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/ColumnReader.java new file mode 100644 index 0000000000..f05dc6cba3 --- /dev/null +++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/ColumnReader.java @@ -0,0 +1,34 @@ +/* + * 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.formats.parquet.vector.reader; + +import org.apache.flink.table.dataformat.vector.writable.WritableColumnVector; + +import java.io.IOException; + +/** + * Read a batch of records for a column to {@link WritableColumnVector} from parquet data file. + */ +public interface ColumnReader { + + /** + * @param readNumber number to read. + * @param vector vector to write. + */ + void readToVector(int readNumber, VECTOR vector) throws IOException; +} diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/DoubleColumnReader.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/DoubleColumnReader.java new file mode 100644 index 0000000000..4999c728ad --- /dev/null +++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/DoubleColumnReader.java @@ -0,0 +1,101 @@ +/* + * 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.formats.parquet.vector.reader; + +import org.apache.flink.table.dataformat.vector.writable.WritableDoubleVector; +import org.apache.flink.table.dataformat.vector.writable.WritableIntVector; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.schema.PrimitiveType; + +import java.io.IOException; +import java.nio.ByteBuffer; + +/** + * Double {@link ColumnReader}. + */ +public class DoubleColumnReader extends AbstractColumnReader { + + public DoubleColumnReader( + ColumnDescriptor descriptor, + PageReader pageReader) throws IOException { + super(descriptor, pageReader); + checkTypeName(PrimitiveType.PrimitiveTypeName.DOUBLE); + } + + @Override + protected void readBatch(int rowId, int num, WritableDoubleVector column) { + int left = num; + while (left > 0) { + if (runLenDecoder.currentCount == 0) { + runLenDecoder.readNextGroup(); + } + int n = Math.min(left, runLenDecoder.currentCount); + switch (runLenDecoder.mode) { + case RLE: + if (runLenDecoder.currentValue == maxDefLevel) { + readDoubles(n, column, rowId); + } else { + column.setNulls(rowId, n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (runLenDecoder.currentBuffer[runLenDecoder.currentBufferIdx++] == maxDefLevel) { + column.setDouble(rowId + i, readDouble()); + } else { + column.setNullAt(rowId + i); + } + } + break; + } + rowId += n; + left -= n; + runLenDecoder.currentCount -= n; + } + } + + @Override + protected void readBatchFromDictionaryIds(int rowId, int num, WritableDoubleVector column, + WritableIntVector dictionaryIds) { + for (int i = rowId; i < rowId + num; ++i) { + if (!column.isNullAt(i)) { + column.setDouble(i, dictionary.decodeToDouble(dictionaryIds.getInt(i))); + } + } + } + + private double readDouble() { + return readDataBuffer(8).getDouble(); + } + + private void readDoubles(int total, WritableDoubleVector c, int rowId) { + int requiredBytes = total * 8; + ByteBuffer buffer = readDataBuffer(requiredBytes); + + if (buffer.hasArray()) { + int offset = buffer.arrayOffset() + buffer.position(); + c.setDoublesFromBinary(rowId, total, buffer.array(), offset); + } else { + for (int i = 0; i < total; i += 1) { + c.setDouble(rowId + i, buffer.getDouble()); + } + } + } +} diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/FixedLenBytesColumnReader.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/FixedLenBytesColumnReader.java new file mode 100644 index 0000000000..835562edc1 --- /dev/null +++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/FixedLenBytesColumnReader.java @@ -0,0 +1,144 @@ +/* + * 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.formats.parquet.vector.reader; + +import org.apache.flink.table.dataformat.Decimal; +import org.apache.flink.table.dataformat.vector.writable.WritableBytesVector; +import org.apache.flink.table.dataformat.vector.writable.WritableColumnVector; +import org.apache.flink.table.dataformat.vector.writable.WritableIntVector; +import org.apache.flink.table.dataformat.vector.writable.WritableLongVector; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.PrimitiveType; + +import java.io.IOException; +import java.nio.ByteBuffer; + +/** + * Fixed length bytes {@link ColumnReader}, just for decimal. + */ +public class FixedLenBytesColumnReader extends AbstractColumnReader { + + private final int precision; + + public FixedLenBytesColumnReader( + ColumnDescriptor descriptor, + PageReader pageReader, + int precision) throws IOException { + super(descriptor, pageReader); + checkTypeName(PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY); + this.precision = precision; + } + + @Override + protected void readBatch(int rowId, int num, VECTOR column) { + int bytesLen = descriptor.getPrimitiveType().getTypeLength(); + if (Decimal.is32BitDecimal(precision)) { + WritableIntVector intVector = (WritableIntVector) column; + for (int i = 0; i < num; i++) { + if (runLenDecoder.readInteger() == maxDefLevel) { + intVector.setInt(rowId + i, (int) heapBinaryToLong(readDataBinary(bytesLen))); + } else { + intVector.setNullAt(rowId + i); + } + } + } else if (Decimal.is64BitDecimal(precision)) { + WritableLongVector longVector = (WritableLongVector) column; + for (int i = 0; i < num; i++) { + if (runLenDecoder.readInteger() == maxDefLevel) { + longVector.setLong(rowId + i, heapBinaryToLong(readDataBinary(bytesLen))); + } else { + longVector.setNullAt(rowId + i); + } + } + } else { + WritableBytesVector bytesVector = (WritableBytesVector) column; + for (int i = 0; i < num; i++) { + if (runLenDecoder.readInteger() == maxDefLevel) { + byte[] bytes = readDataBinary(bytesLen).getBytes(); + bytesVector.appendBytes(rowId + i, bytes, 0, bytes.length); + } else { + bytesVector.setNullAt(rowId + i); + } + } + } + } + + @Override + protected void readBatchFromDictionaryIds( + int rowId, + int num, + VECTOR column, + WritableIntVector dictionaryIds) { + if (Decimal.is32BitDecimal(precision)) { + WritableIntVector intVector = (WritableIntVector) column; + for (int i = rowId; i < rowId + num; ++i) { + if (!intVector.isNullAt(i)) { + Binary v = dictionary.decodeToBinary(dictionaryIds.getInt(i)); + intVector.setInt(i, (int) heapBinaryToLong(v)); + } + } + } else if (Decimal.is64BitDecimal(precision)) { + WritableLongVector longVector = (WritableLongVector) column; + for (int i = rowId; i < rowId + num; ++i) { + if (!longVector.isNullAt(i)) { + Binary v = dictionary.decodeToBinary(dictionaryIds.getInt(i)); + longVector.setLong(i, heapBinaryToLong(v)); + } + } + } else { + WritableBytesVector bytesVector = (WritableBytesVector) column; + for (int i = rowId; i < rowId + num; ++i) { + if (!bytesVector.isNullAt(i)) { + byte[] v = dictionary.decodeToBinary(dictionaryIds.getInt(i)).getBytes(); + bytesVector.appendBytes(i, v, 0, v.length); + } + } + } + } + + private long heapBinaryToLong(Binary binary) { + ByteBuffer buffer = binary.toByteBuffer(); + byte[] bytes = buffer.array(); + int start = buffer.arrayOffset() + buffer.position(); + int end = buffer.arrayOffset() + buffer.limit(); + + long unscaled = 0L; + + for (int i = start; i < end; i++) { + unscaled = (unscaled << 8) | (bytes[i] & 0xff); + } + + int bits = 8 * (end - start); + return (unscaled << (64 - bits)) >> (64 - bits); + } + + private Binary readDataBinary(int len) { + ByteBuffer buffer = readDataBuffer(len); + if (buffer.hasArray()) { + return Binary.fromConstantByteArray( + buffer.array(), buffer.arrayOffset() + buffer.position(), len); + } else { + byte[] bytes = new byte[len]; + buffer.get(bytes); + return Binary.fromConstantByteArray(bytes); + } + } +} diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/FloatColumnReader.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/FloatColumnReader.java new file mode 100644 index 0000000000..71b794a0fb --- /dev/null +++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/FloatColumnReader.java @@ -0,0 +1,101 @@ +/* + * 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.formats.parquet.vector.reader; + +import org.apache.flink.table.dataformat.vector.writable.WritableFloatVector; +import org.apache.flink.table.dataformat.vector.writable.WritableIntVector; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.schema.PrimitiveType; + +import java.io.IOException; +import java.nio.ByteBuffer; + +/** + * Float {@link ColumnReader}. + */ +public class FloatColumnReader extends AbstractColumnReader { + + public FloatColumnReader( + ColumnDescriptor descriptor, + PageReader pageReader) throws IOException { + super(descriptor, pageReader); + checkTypeName(PrimitiveType.PrimitiveTypeName.FLOAT); + } + + @Override + protected void readBatch(int rowId, int num, WritableFloatVector column) { + int left = num; + while (left > 0) { + if (runLenDecoder.currentCount == 0) { + runLenDecoder.readNextGroup(); + } + int n = Math.min(left, runLenDecoder.currentCount); + switch (runLenDecoder.mode) { + case RLE: + if (runLenDecoder.currentValue == maxDefLevel) { + readFloats(n, column, rowId); + } else { + column.setNulls(rowId, n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (runLenDecoder.currentBuffer[runLenDecoder.currentBufferIdx++] == maxDefLevel) { + column.setFloat(rowId + i, readFloat()); + } else { + column.setNullAt(rowId + i); + } + } + break; + } + rowId += n; + left -= n; + runLenDecoder.currentCount -= n; + } + } + + @Override + protected void readBatchFromDictionaryIds(int rowId, int num, WritableFloatVector column, + WritableIntVector dictionaryIds) { + for (int i = rowId; i < rowId + num; ++i) { + if (!column.isNullAt(i)) { + column.setFloat(i, dictionary.decodeToFloat(dictionaryIds.getInt(i))); + } + } + } + + private float readFloat() { + return readDataBuffer(4).getFloat(); + } + + private void readFloats(int total, WritableFloatVector c, int rowId) { + int requiredBytes = total * 4; + ByteBuffer buffer = readDataBuffer(requiredBytes); + + if (buffer.hasArray()) { + int offset = buffer.arrayOffset() + buffer.position(); + c.setFloatsFromBinary(rowId, total, buffer.array(), offset); + } else { + for (int i = 0; i < total; i += 1) { + c.setFloat(rowId + i, buffer.getFloat()); + } + } + } +} diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/IntColumnReader.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/IntColumnReader.java new file mode 100644 index 0000000000..9b07367d5c --- /dev/null +++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/IntColumnReader.java @@ -0,0 +1,100 @@ +/* + * 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.formats.parquet.vector.reader; + +import org.apache.flink.table.dataformat.vector.writable.WritableIntVector; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.schema.PrimitiveType; + +import java.io.IOException; +import java.nio.ByteBuffer; + +/** + * Int {@link ColumnReader}. + */ +public class IntColumnReader extends AbstractColumnReader { + + public IntColumnReader( + ColumnDescriptor descriptor, + PageReader pageReader) throws IOException { + super(descriptor, pageReader); + checkTypeName(PrimitiveType.PrimitiveTypeName.INT32); + } + + @Override + protected void readBatch(int rowId, int num, WritableIntVector column) { + int left = num; + while (left > 0) { + if (runLenDecoder.currentCount == 0) { + runLenDecoder.readNextGroup(); + } + int n = Math.min(left, runLenDecoder.currentCount); + switch (runLenDecoder.mode) { + case RLE: + if (runLenDecoder.currentValue == maxDefLevel) { + readIntegers(n, column, rowId); + } else { + column.setNulls(rowId, n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (runLenDecoder.currentBuffer[runLenDecoder.currentBufferIdx++] == maxDefLevel) { + column.setInt(rowId + i, readInteger()); + } else { + column.setNullAt(rowId + i); + } + } + break; + } + rowId += n; + left -= n; + runLenDecoder.currentCount -= n; + } + } + + @Override + protected void readBatchFromDictionaryIds(int rowId, int num, WritableIntVector column, + WritableIntVector dictionaryIds) { + for (int i = rowId; i < rowId + num; ++i) { + if (!column.isNullAt(i)) { + column.setInt(i, dictionary.decodeToInt(dictionaryIds.getInt(i))); + } + } + } + + private int readInteger() { + return readDataBuffer(4).getInt(); + } + + private void readIntegers(int total, WritableIntVector c, int rowId) { + int requiredBytes = total * 4; + ByteBuffer buffer = readDataBuffer(requiredBytes); + + if (buffer.hasArray()) { + int offset = buffer.arrayOffset() + buffer.position(); + c.setIntsFromBinary(rowId, total, buffer.array(), offset); + } else { + for (int i = 0; i < total; i += 1) { + c.setInt(rowId + i, buffer.getInt()); + } + } + } +} diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/LongColumnReader.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/LongColumnReader.java new file mode 100644 index 0000000000..b68a7326b6 --- /dev/null +++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/LongColumnReader.java @@ -0,0 +1,101 @@ +/* + * 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.formats.parquet.vector.reader; + +import org.apache.flink.table.dataformat.vector.writable.WritableIntVector; +import org.apache.flink.table.dataformat.vector.writable.WritableLongVector; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.schema.PrimitiveType; + +import java.io.IOException; +import java.nio.ByteBuffer; + +/** + * Long {@link ColumnReader}. + */ +public class LongColumnReader extends AbstractColumnReader { + + public LongColumnReader( + ColumnDescriptor descriptor, + PageReader pageReader) throws IOException { + super(descriptor, pageReader); + checkTypeName(PrimitiveType.PrimitiveTypeName.INT64); + } + + @Override + protected void readBatch(int rowId, int num, WritableLongVector column) { + int left = num; + while (left > 0) { + if (runLenDecoder.currentCount == 0) { + runLenDecoder.readNextGroup(); + } + int n = Math.min(left, runLenDecoder.currentCount); + switch (runLenDecoder.mode) { + case RLE: + if (runLenDecoder.currentValue == maxDefLevel) { + readLongs(n, column, rowId); + } else { + column.setNulls(rowId, n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (runLenDecoder.currentBuffer[runLenDecoder.currentBufferIdx++] == maxDefLevel) { + column.setLong(rowId + i, readLong()); + } else { + column.setNullAt(rowId + i); + } + } + break; + } + rowId += n; + left -= n; + runLenDecoder.currentCount -= n; + } + } + + @Override + protected void readBatchFromDictionaryIds(int rowId, int num, WritableLongVector column, + WritableIntVector dictionaryIds) { + for (int i = rowId; i < rowId + num; ++i) { + if (!column.isNullAt(i)) { + column.setLong(i, dictionary.decodeToLong(dictionaryIds.getInt(i))); + } + } + } + + private long readLong() { + return readDataBuffer(8).getLong(); + } + + private void readLongs(int total, WritableLongVector c, int rowId) { + int requiredBytes = total * 8; + ByteBuffer buffer = readDataBuffer(requiredBytes); + + if (buffer.hasArray()) { + int offset = buffer.arrayOffset() + buffer.position(); + c.setLongsFromBinary(rowId, total, buffer.array(), offset); + } else { + for (int i = 0; i < total; i += 1) { + c.setLong(rowId + i, buffer.getLong()); + } + } + } +} diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/RunLengthDecoder.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/RunLengthDecoder.java new file mode 100644 index 0000000000..3337f6de4b --- /dev/null +++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/RunLengthDecoder.java @@ -0,0 +1,293 @@ +/* + * 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.formats.parquet.vector.reader; + +import org.apache.flink.table.dataformat.vector.writable.WritableColumnVector; +import org.apache.flink.table.dataformat.vector.writable.WritableIntVector; + +import org.apache.parquet.Preconditions; +import org.apache.parquet.bytes.ByteBufferInputStream; +import org.apache.parquet.bytes.BytesUtils; +import org.apache.parquet.column.values.bitpacking.BytePacker; +import org.apache.parquet.column.values.bitpacking.Packer; +import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridDecoder; +import org.apache.parquet.io.ParquetDecodingException; + +import java.io.IOException; +import java.nio.ByteBuffer; + +/** + * Run length decoder for data and dictionary ids. + * See https://github.com/apache/parquet-format/blob/master/Encodings.md + * See {@link RunLengthBitPackingHybridDecoder}. + */ +final class RunLengthDecoder { + + /** + * If true, the bit width is fixed. This decoder is used in different places and this also + * controls if we need to read the bitwidth from the beginning of the data stream. + */ + private final boolean fixedWidth; + private final boolean readLength; + + // Encoded data. + private ByteBufferInputStream in; + + // bit/byte width of decoded data and utility to batch unpack them. + private int bitWidth; + private int bytesWidth; + private BytePacker packer; + + // Current decoding mode and values + MODE mode; + int currentCount; + int currentValue; + + // Buffer of decoded values if the values are PACKED. + int[] currentBuffer = new int[16]; + int currentBufferIdx = 0; + + RunLengthDecoder() { + this.fixedWidth = false; + this.readLength = false; + } + + RunLengthDecoder(int bitWidth) { + this.fixedWidth = true; + this.readLength = bitWidth != 0; + initWidthAndPacker(bitWidth); + } + + RunLengthDecoder(int bitWidth, boolean readLength) { + this.fixedWidth = true; + this.readLength = readLength; + initWidthAndPacker(bitWidth); + } + + /** + * Init from input stream. + */ + void initFromStream(int valueCount, ByteBufferInputStream in) throws IOException { + this.in = in; + if (fixedWidth) { + // initialize for repetition and definition levels + if (readLength) { + int length = readIntLittleEndian(); + this.in = in.sliceStream(length); + } + } else { + // initialize for values + if (in.available() > 0) { + initWidthAndPacker(in.read()); + } + } + if (bitWidth == 0) { + // 0 bit width, treat this as an RLE run of valueCount number of 0's. + this.mode = MODE.RLE; + this.currentCount = valueCount; + this.currentValue = 0; + } else { + this.currentCount = 0; + } + } + + /** + * Initializes the internal state for decoding ints of `bitWidth`. + */ + private void initWidthAndPacker(int bitWidth) { + Preconditions.checkArgument(bitWidth >= 0 && bitWidth <= 32, "bitWidth must be >= 0 and <= 32"); + this.bitWidth = bitWidth; + this.bytesWidth = BytesUtils.paddedByteCountFromBits(bitWidth); + this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth); + } + + int readInteger() { + if (this.currentCount == 0) { + this.readNextGroup(); + } + + this.currentCount--; + switch (mode) { + case RLE: + return this.currentValue; + case PACKED: + return this.currentBuffer[currentBufferIdx++]; + } + throw new RuntimeException("Unreachable"); + } + + /** + * Decoding for dictionary ids. The IDs are populated into `values` and the nullability is + * populated into `nulls`. + */ + void readDictionaryIds( + int total, + WritableIntVector values, + WritableColumnVector nulls, + int rowId, + int level, + RunLengthDecoder data) { + int left = total; + while (left > 0) { + if (this.currentCount == 0) { + this.readNextGroup(); + } + int n = Math.min(left, this.currentCount); + switch (mode) { + case RLE: + if (currentValue == level) { + data.readDictionaryIdData(n, values, rowId); + } else { + nulls.setNulls(rowId, n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (currentBuffer[currentBufferIdx++] == level) { + values.setInt(rowId + i, data.readInteger()); + } else { + nulls.setNullAt(rowId + i); + } + } + break; + } + rowId += n; + left -= n; + currentCount -= n; + } + } + + /** + * It is used to decode dictionary IDs. + */ + private void readDictionaryIdData(int total, WritableIntVector c, int rowId) { + int left = total; + while (left > 0) { + if (this.currentCount == 0) { + this.readNextGroup(); + } + int n = Math.min(left, this.currentCount); + switch (mode) { + case RLE: + c.setInts(rowId, n, currentValue); + break; + case PACKED: + c.setInts(rowId, n, currentBuffer, currentBufferIdx); + currentBufferIdx += n; + break; + } + rowId += n; + left -= n; + currentCount -= n; + } + } + + /** + * Reads the next varint encoded int. + */ + private int readUnsignedVarInt() throws IOException { + int value = 0; + int shift = 0; + int b; + do { + b = in.read(); + value |= (b & 0x7F) << shift; + shift += 7; + } while ((b & 0x80) != 0); + return value; + } + + /** + * Reads the next 4 byte little endian int. + */ + private int readIntLittleEndian() throws IOException { + int ch4 = in.read(); + int ch3 = in.read(); + int ch2 = in.read(); + int ch1 = in.read(); + return ((ch1 << 24) + (ch2 << 16) + (ch3 << 8) + ch4); + } + + /** + * Reads the next byteWidth little endian int. + */ + private int readIntLittleEndianPaddedOnBitWidth() throws IOException { + switch (bytesWidth) { + case 0: + return 0; + case 1: + return in.read(); + case 2: { + int ch2 = in.read(); + int ch1 = in.read(); + return (ch1 << 8) + ch2; + } + case 3: { + int ch3 = in.read(); + int ch2 = in.read(); + int ch1 = in.read(); + return (ch1 << 16) + (ch2 << 8) + ch3; + } + case 4: { + return readIntLittleEndian(); + } + } + throw new RuntimeException("Unreachable"); + } + + /** + * Reads the next group. + */ + void readNextGroup() { + try { + int header = readUnsignedVarInt(); + this.mode = (header & 1) == 0 ? MODE.RLE : MODE.PACKED; + switch (mode) { + case RLE: + this.currentCount = header >>> 1; + this.currentValue = readIntLittleEndianPaddedOnBitWidth(); + return; + case PACKED: + int numGroups = header >>> 1; + this.currentCount = numGroups * 8; + + if (this.currentBuffer.length < this.currentCount) { + this.currentBuffer = new int[this.currentCount]; + } + currentBufferIdx = 0; + int valueIndex = 0; + while (valueIndex < this.currentCount) { + // values are bit packed 8 at a time, so reading bitWidth will always work + ByteBuffer buffer = in.slice(bitWidth); + this.packer.unpack8Values(buffer, buffer.position(), this.currentBuffer, valueIndex); + valueIndex += 8; + } + return; + default: + throw new ParquetDecodingException("not a valid mode " + this.mode); + } + } catch (IOException e) { + throw new ParquetDecodingException("Failed to read from input stream", e); + } + } + + enum MODE { + RLE, + PACKED + } +} diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/ShortColumnReader.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/ShortColumnReader.java new file mode 100644 index 0000000000..a98051d589 --- /dev/null +++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/ShortColumnReader.java @@ -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.formats.parquet.vector.reader; + +import org.apache.flink.table.dataformat.vector.writable.WritableIntVector; +import org.apache.flink.table.dataformat.vector.writable.WritableShortVector; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.schema.PrimitiveType; + +import java.io.IOException; + +/** + * Short {@link ColumnReader}. Using INT32 to store short, so just cast int to short. + */ +public class ShortColumnReader extends AbstractColumnReader { + + public ShortColumnReader( + ColumnDescriptor descriptor, + PageReader pageReader) throws IOException { + super(descriptor, pageReader); + checkTypeName(PrimitiveType.PrimitiveTypeName.INT32); + } + + @Override + protected void readBatch(int rowId, int num, WritableShortVector column) { + int left = num; + while (left > 0) { + if (runLenDecoder.currentCount == 0) { + runLenDecoder.readNextGroup(); + } + int n = Math.min(left, runLenDecoder.currentCount); + switch (runLenDecoder.mode) { + case RLE: + if (runLenDecoder.currentValue == maxDefLevel) { + for (int i = 0; i < n; i++) { + column.setShort(rowId + i, (short) readDataBuffer(4).getInt()); + } + } else { + column.setNulls(rowId, n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (runLenDecoder.currentBuffer[runLenDecoder.currentBufferIdx++] == maxDefLevel) { + column.setShort(rowId + i, (short) readDataBuffer(4).getInt()); + } else { + column.setNullAt(rowId + i); + } + } + break; + } + rowId += n; + left -= n; + runLenDecoder.currentCount -= n; + } + } + + @Override + protected void readBatchFromDictionaryIds(int rowId, int num, WritableShortVector column, + WritableIntVector dictionaryIds) { + for (int i = rowId; i < rowId + num; ++i) { + if (!column.isNullAt(i)) { + column.setShort(i, (short) dictionary.decodeToInt(dictionaryIds.getInt(i))); + } + } + } +} diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/TimestampColumnReader.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/TimestampColumnReader.java new file mode 100644 index 0000000000..256ae721e5 --- /dev/null +++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/reader/TimestampColumnReader.java @@ -0,0 +1,120 @@ +/* + * 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.formats.parquet.vector.reader; + +import org.apache.flink.table.dataformat.SqlTimestamp; +import org.apache.flink.table.dataformat.vector.writable.WritableIntVector; +import org.apache.flink.table.dataformat.vector.writable.WritableTimestampVector; + +import org.apache.parquet.Preconditions; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.PrimitiveType; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.sql.Timestamp; +import java.util.concurrent.TimeUnit; + +/** + * Timestamp {@link ColumnReader}. We only support INT96 bytes now, julianDay(4) + nanosOfDay(8). + * See https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#timestamp + * TIMESTAMP_MILLIS and TIMESTAMP_MICROS are the deprecated ConvertedType. + */ +public class TimestampColumnReader extends AbstractColumnReader { + + public static final int JULIAN_EPOCH_OFFSET_DAYS = 2_440_588; + public static final long MILLIS_IN_DAY = TimeUnit.DAYS.toMillis(1); + public static final long NANOS_PER_MILLISECOND = TimeUnit.MILLISECONDS.toNanos(1); + public static final long NANOS_PER_SECOND = TimeUnit.SECONDS.toNanos(1); + + private final boolean utcTimestamp; + + public TimestampColumnReader( + boolean utcTimestamp, + ColumnDescriptor descriptor, + PageReader pageReader) throws IOException { + super(descriptor, pageReader); + this.utcTimestamp = utcTimestamp; + checkTypeName(PrimitiveType.PrimitiveTypeName.INT96); + } + + @Override + protected boolean supportLazyDecode() { + return utcTimestamp; + } + + @Override + protected void readBatch(int rowId, int num, WritableTimestampVector column) { + for (int i = 0; i < num; i++) { + if (runLenDecoder.readInteger() == maxDefLevel) { + ByteBuffer buffer = readDataBuffer(12); + column.setTimestamp(rowId + i, int96ToTimestamp( + utcTimestamp, buffer.getLong(), buffer.getInt())); + } else { + column.setNullAt(rowId + i); + } + } + } + + @Override + protected void readBatchFromDictionaryIds( + int rowId, + int num, + WritableTimestampVector column, + WritableIntVector dictionaryIds) { + for (int i = rowId; i < rowId + num; ++i) { + if (!column.isNullAt(i)) { + column.setTimestamp(i, decodeInt96ToTimestamp( + utcTimestamp, dictionary, dictionaryIds.getInt(i))); + } + } + } + + public static SqlTimestamp decodeInt96ToTimestamp( + boolean utcTimestamp, + org.apache.parquet.column.Dictionary dictionary, + int id) { + Binary binary = dictionary.decodeToBinary(id); + Preconditions.checkArgument( + binary.length() == 12, + "Timestamp with int96 should be 12 bytes."); + ByteBuffer buffer = binary.toByteBuffer().order(ByteOrder.LITTLE_ENDIAN); + return int96ToTimestamp(utcTimestamp, buffer.getLong(), buffer.getInt()); + } + + private static SqlTimestamp int96ToTimestamp( + boolean utcTimestamp, long nanosOfDay, int julianDay) { + long millisecond = julianDayToMillis(julianDay) + (nanosOfDay / NANOS_PER_MILLISECOND); + + if (utcTimestamp) { + int nanoOfMillisecond = (int) (nanosOfDay % NANOS_PER_MILLISECOND); + return SqlTimestamp.fromEpochMillis(millisecond, nanoOfMillisecond); + } else { + Timestamp timestamp = new Timestamp(millisecond); + timestamp.setNanos((int) (nanosOfDay % NANOS_PER_SECOND)); + return SqlTimestamp.fromTimestamp(timestamp); + } + } + + private static long julianDayToMillis(int julianDay) { + return (julianDay - JULIAN_EPOCH_OFFSET_DAYS) * MILLIS_IN_DAY; + } +} diff --git a/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/utils/ParquetWriterUtil.java b/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/utils/ParquetWriterUtil.java new file mode 100644 index 0000000000..f14c29568b --- /dev/null +++ b/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/utils/ParquetWriterUtil.java @@ -0,0 +1,166 @@ +/* + * 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.formats.parquet.utils; + +import org.apache.flink.core.fs.Path; +import org.apache.flink.types.Row; + +import org.apache.hadoop.conf.Configuration; +import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.hadoop.api.WriteSupport; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.io.api.RecordConsumer; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; + +import java.io.File; +import java.io.IOException; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.charset.StandardCharsets; +import java.sql.Timestamp; +import java.time.LocalDateTime; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.UUID; + +import static org.apache.flink.formats.parquet.vector.reader.TimestampColumnReader.JULIAN_EPOCH_OFFSET_DAYS; +import static org.apache.flink.formats.parquet.vector.reader.TimestampColumnReader.MILLIS_IN_DAY; +import static org.apache.flink.formats.parquet.vector.reader.TimestampColumnReader.NANOS_PER_SECOND; + +/** + * Parquet writer util to write Row to file. + */ +public class ParquetWriterUtil { + + public static Path createTempParquetFile(File folder, MessageType schema, List records, int rowGroupSize) throws IOException { + Path path = new Path(folder.getPath(), UUID.randomUUID().toString()); + WriteSupport support = new WriteSupport() { + private RecordConsumer consumer; + + @Override + public WriteContext init(Configuration configuration) { + return new WriteContext(schema, new HashMap<>()); + } + + @Override + public void prepareForWrite(RecordConsumer consumer) { + this.consumer = consumer; + } + + @Override + public void write(Row row) { + consumer.startMessage(); + for (int i = 0; i < row.getArity(); i++) { + PrimitiveType type = schema.getColumns().get(i).getPrimitiveType(); + Object field = row.getField(i); + if (field != null) { + consumer.startField("f" + i, i); + switch (type.getPrimitiveTypeName()) { + case INT64: + consumer.addLong(((Number) field).longValue()); + break; + case INT32: + consumer.addInteger(((Number) field).intValue()); + break; + case BOOLEAN: + consumer.addBoolean((Boolean) field); + break; + case BINARY: + if (field instanceof String) { + field = ((String) field).getBytes(StandardCharsets.UTF_8); + } else if (field instanceof BigDecimal) { + field = ((BigDecimal) field).unscaledValue().toByteArray(); + } + consumer.addBinary(Binary.fromConstantByteArray((byte[]) field)); + break; + case FLOAT: + consumer.addFloat(((Number) field).floatValue()); + break; + case DOUBLE: + consumer.addDouble(((Number) field).doubleValue()); + break; + case INT96: + consumer.addBinary(timestampToInt96((LocalDateTime) field)); + break; + case FIXED_LEN_BYTE_ARRAY: + byte[] bytes = ((BigDecimal) field).unscaledValue().toByteArray(); + byte signByte = (byte) (bytes[0] < 0 ? -1 : 0); + int numBytes = 16; + byte[] newBytes = new byte[numBytes]; + Arrays.fill(newBytes, 0, numBytes - bytes.length, signByte); + System.arraycopy(bytes, 0, newBytes, numBytes - bytes.length, bytes.length); + consumer.addBinary(Binary.fromConstantByteArray(newBytes)); + break; + } + consumer.endField("f" + i, i); + } + } + consumer.endMessage(); + } + }; + ParquetWriter writer = new ParquetWriterBuilder( + new org.apache.hadoop.fs.Path(path.getPath()), support) + .withRowGroupSize(rowGroupSize) + .build(); + + for (Row record : records) { + writer.write(record); + } + + writer.close(); + return path; + } + + private static class ParquetWriterBuilder extends ParquetWriter.Builder { + + private final WriteSupport support; + + private ParquetWriterBuilder(org.apache.hadoop.fs.Path path, WriteSupport support) { + super(path); + this.support = support; + } + + @Override + protected ParquetWriterBuilder self() { + return this; + } + + @Override + protected WriteSupport getWriteSupport(Configuration conf) { + return support; + } + } + + private static Binary timestampToInt96(LocalDateTime time) { + Timestamp timestamp = Timestamp.valueOf(time); + long mills = timestamp.getTime(); + int julianDay = (int) ((mills / MILLIS_IN_DAY) + JULIAN_EPOCH_OFFSET_DAYS); + long nanosOfDay = ((mills % MILLIS_IN_DAY) / 1000) * NANOS_PER_SECOND + timestamp.getNanos(); + + ByteBuffer buf = ByteBuffer.allocate(12); + buf.order(ByteOrder.LITTLE_ENDIAN); + buf.putLong(nanosOfDay); + buf.putInt(julianDay); + buf.flip(); + return Binary.fromConstantByteBuffer(buf); + } +} diff --git a/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/utils/TestUtil.java b/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/utils/TestUtil.java index 6a4f1764c4..9220aaa4ec 100644 --- a/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/utils/TestUtil.java +++ b/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/utils/TestUtil.java @@ -230,7 +230,7 @@ public class TestUtil { return mockContext; } - private static Schema getTestSchema(String schemaName) { + public static Schema getTestSchema(String schemaName) { try { InputStream inputStream = TestUtil.class.getClassLoader() .getResourceAsStream("avro/" + schemaName); diff --git a/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/vector/ParquetColumnarRowSplitReaderTest.java b/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/vector/ParquetColumnarRowSplitReaderTest.java new file mode 100644 index 0000000000..758c267da0 --- /dev/null +++ b/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/vector/ParquetColumnarRowSplitReaderTest.java @@ -0,0 +1,545 @@ +/* + * 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.formats.parquet.vector; + +import org.apache.flink.core.fs.Path; +import org.apache.flink.table.dataformat.ColumnarRow; +import org.apache.flink.table.dataformat.Decimal; +import org.apache.flink.table.dataformat.vector.VectorizedColumnBatch; +import org.apache.flink.table.runtime.functions.SqlDateTimeUtils; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.DateType; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.DoubleType; +import org.apache.flink.table.types.logical.FloatType; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.SmallIntType; +import org.apache.flink.table.types.logical.TimestampType; +import org.apache.flink.table.types.logical.TinyIntType; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.flink.types.Row; + +import org.apache.hadoop.conf.Configuration; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.OriginalType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; +import org.apache.parquet.schema.Types; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.IOException; +import java.math.BigDecimal; +import java.sql.Date; +import java.time.LocalDateTime; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.stream.IntStream; + +import static org.apache.flink.formats.parquet.utils.ParquetWriterUtil.createTempParquetFile; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * Test for {@link ParquetColumnarRowSplitReader}. + */ +@RunWith(Parameterized.class) +public class ParquetColumnarRowSplitReaderTest { + + private static final int FIELD_NUMBER = 15; + private static final LocalDateTime BASE_TIME = LocalDateTime.now(); + + private static final MessageType PARQUET_SCHEMA = new MessageType( + "TOP", + Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.OPTIONAL) + .named("f0"), + Types.primitive(PrimitiveType.PrimitiveTypeName.BOOLEAN, Type.Repetition.OPTIONAL) + .named("f1"), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.OPTIONAL) + .named("f2"), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.OPTIONAL) + .named("f3"), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.OPTIONAL) + .named("f4"), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.OPTIONAL) + .named("f5"), + Types.primitive(PrimitiveType.PrimitiveTypeName.FLOAT, Type.Repetition.OPTIONAL) + .named("f6"), + Types.primitive(PrimitiveType.PrimitiveTypeName.DOUBLE, Type.Repetition.OPTIONAL) + .named("f7"), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT96, Type.Repetition.OPTIONAL) + .named("f8"), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.OPTIONAL) + .precision(5) + .as(OriginalType.DECIMAL) + .named("f9"), + Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.OPTIONAL) + .precision(15) + .as(OriginalType.DECIMAL) + .named("f10"), + Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.OPTIONAL) + .precision(20) + .as(OriginalType.DECIMAL) + .named("f11"), + Types.primitive(PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, Type.Repetition.OPTIONAL) + .length(16) + .precision(5) + .as(OriginalType.DECIMAL) + .named("f12"), + Types.primitive(PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, Type.Repetition.OPTIONAL) + .length(16) + .precision(15) + .as(OriginalType.DECIMAL) + .named("f13"), + Types.primitive(PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, Type.Repetition.OPTIONAL) + .length(16) + .precision(20) + .as(OriginalType.DECIMAL) + .named("f14") + ); + + @ClassRule + public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + private final int rowGroupSize; + + @Parameterized.Parameters(name = "rowGroupSize-{0}") + public static Collection parameters() { + return Arrays.asList(10, 1000); + } + + public ParquetColumnarRowSplitReaderTest(int rowGroupSize) { + this.rowGroupSize = rowGroupSize; + } + + @Test + public void testNormalTypesReadWithSplits() throws IOException { + // prepare parquet file + int number = 10000; + List records = new ArrayList<>(number); + List values = new ArrayList<>(number); + Random random = new Random(); + for (int i = 0; i < number; i++) { + Integer v = random.nextInt(number / 2); + if (v % 10 == 0) { + values.add(null); + records.add(new Row(FIELD_NUMBER)); + } else { + values.add(v); + records.add(newRow(v)); + } + } + + testNormalTypes(number, records, values); + } + + private void testNormalTypes(int number, List records, + List values) throws IOException { + Path testPath = createTempParquetFile( + TEMPORARY_FOLDER.newFolder(), PARQUET_SCHEMA, records, rowGroupSize); + + // test reading and splitting + long fileLen = testPath.getFileSystem().getFileStatus(testPath).getLen(); + int len1 = readSplitAndCheck(0, testPath, 0, fileLen / 3, values); + int len2 = readSplitAndCheck(len1, testPath, fileLen / 3, fileLen * 2 / 3, values); + int len3 = readSplitAndCheck(len1 + len2, testPath, fileLen * 2 / 3, Long.MAX_VALUE, values); + assertEquals(number, len1 + len2 + len3); + } + + private int readSplitAndCheck( + int start, + Path testPath, + long splitStart, + long splitLength, + List values) throws IOException { + LogicalType[] fieldTypes = new LogicalType[]{ + new VarCharType(VarCharType.MAX_LENGTH), + new BooleanType(), + new TinyIntType(), + new SmallIntType(), + new IntType(), + new BigIntType(), + new FloatType(), + new DoubleType(), + new TimestampType(9), + new DecimalType(5, 0), + new DecimalType(15, 0), + new DecimalType(20, 0), + new DecimalType(5, 0), + new DecimalType(15, 0), + new DecimalType(20, 0)}; + + ParquetColumnarRowSplitReader reader = new ParquetColumnarRowSplitReader( + false, + new Configuration(), + fieldTypes, + new String[] { + "f0", "f1", "f2", "f3", "f4", "f5", "f6", "f7", + "f8", "f9", "f10", "f11", "f12", "f13", "f14"}, + VectorizedColumnBatch::new, + 500, + new org.apache.hadoop.fs.Path(testPath.getPath()), + splitStart, + splitLength); + + int i = start; + while (!reader.reachedEnd()) { + ColumnarRow row = reader.nextRecord(); + Integer v = values.get(i); + if (v == null) { + assertTrue(row.isNullAt(0)); + assertTrue(row.isNullAt(1)); + assertTrue(row.isNullAt(2)); + assertTrue(row.isNullAt(3)); + assertTrue(row.isNullAt(4)); + assertTrue(row.isNullAt(5)); + assertTrue(row.isNullAt(6)); + assertTrue(row.isNullAt(7)); + assertTrue(row.isNullAt(8)); + assertTrue(row.isNullAt(9)); + assertTrue(row.isNullAt(10)); + assertTrue(row.isNullAt(11)); + assertTrue(row.isNullAt(12)); + assertTrue(row.isNullAt(13)); + assertTrue(row.isNullAt(14)); + } else { + assertEquals("" + v, row.getString(0).toString()); + assertEquals(v % 2 == 0, row.getBoolean(1)); + assertEquals(v.byteValue(), row.getByte(2)); + assertEquals(v.shortValue(), row.getShort(3)); + assertEquals(v.intValue(), row.getInt(4)); + assertEquals(v.longValue(), row.getLong(5)); + assertEquals(v.floatValue(), row.getFloat(6), 0); + assertEquals(v.doubleValue(), row.getDouble(7), 0); + assertEquals( + toDateTime(v), + row.getTimestamp(8, 9).toLocalDateTime()); + assertEquals(BigDecimal.valueOf(v), row.getDecimal(9, 5, 0).toBigDecimal()); + assertEquals(BigDecimal.valueOf(v), row.getDecimal(10, 15, 0).toBigDecimal()); + assertEquals(BigDecimal.valueOf(v), row.getDecimal(11, 20, 0).toBigDecimal()); + assertEquals(BigDecimal.valueOf(v), row.getDecimal(12, 5, 0).toBigDecimal()); + assertEquals(BigDecimal.valueOf(v), row.getDecimal(13, 15, 0).toBigDecimal()); + assertEquals(BigDecimal.valueOf(v), row.getDecimal(14, 20, 0).toBigDecimal()); + } + i++; + } + reader.close(); + return i - start; + } + + private Row newRow(Integer v) { + return Row.of( + "" + v, + v % 2 == 0, + v, + v, + v, + v.longValue(), + v.floatValue(), + v.doubleValue(), + toDateTime(v), + BigDecimal.valueOf(v), + BigDecimal.valueOf(v), + BigDecimal.valueOf(v), + BigDecimal.valueOf(v), + BigDecimal.valueOf(v), + BigDecimal.valueOf(v)); + } + + private LocalDateTime toDateTime(Integer v) { + v = (v > 0 ? v : -v) % 10000; + return BASE_TIME.plusNanos(v).plusSeconds(v); + } + + @Test + public void testDictionary() throws IOException { + // prepare parquet file + int number = 10000; + List records = new ArrayList<>(number); + List values = new ArrayList<>(number); + Random random = new Random(); + int[] intValues = new int[10]; + // test large values in dictionary + for (int i = 0; i < intValues.length; i++) { + intValues[i] = random.nextInt(); + } + for (int i = 0; i < number; i++) { + Integer v = intValues[random.nextInt(10)]; + if (v == 0) { + values.add(null); + records.add(new Row(FIELD_NUMBER)); + } else { + values.add(v); + records.add(newRow(v)); + } + } + + testNormalTypes(number, records, values); + } + + @Test + public void testPartialDictionary() throws IOException { + // prepare parquet file + int number = 10000; + List records = new ArrayList<>(number); + List values = new ArrayList<>(number); + Random random = new Random(); + int[] intValues = new int[10]; + // test large values in dictionary + for (int i = 0; i < intValues.length; i++) { + intValues[i] = random.nextInt(); + } + for (int i = 0; i < number; i++) { + Integer v = i < 5000 ? intValues[random.nextInt(10)] : i; + if (v == 0) { + values.add(null); + records.add(new Row(FIELD_NUMBER)); + } else { + values.add(v); + records.add(newRow(v)); + } + } + + testNormalTypes(number, records, values); + } + + @Test + public void testContinuousRepetition() throws IOException { + // prepare parquet file + int number = 10000; + List records = new ArrayList<>(number); + List values = new ArrayList<>(number); + Random random = new Random(); + for (int i = 0; i < 100; i++) { + Integer v = random.nextInt(10); + for (int j = 0; j < 100; j++) { + if (v == 0) { + values.add(null); + records.add(new Row(FIELD_NUMBER)); + } else { + values.add(v); + records.add(newRow(v)); + } + } + } + + testNormalTypes(number, records, values); + } + + @Test + public void testLargeValue() throws IOException { + // prepare parquet file + int number = 10000; + List records = new ArrayList<>(number); + List values = new ArrayList<>(number); + Random random = new Random(); + for (int i = 0; i < number; i++) { + Integer v = random.nextInt(); + if (v % 10 == 0) { + values.add(null); + records.add(new Row(FIELD_NUMBER)); + } else { + values.add(v); + records.add(newRow(v)); + } + } + + testNormalTypes(number, records, values); + } + + @Test + public void testProject() throws IOException { + // prepare parquet file + int number = 1000; + List records = new ArrayList<>(number); + for (int i = 0; i < number; i++) { + Integer v = i; + records.add(newRow(v)); + } + + Path testPath = createTempParquetFile( + TEMPORARY_FOLDER.newFolder(), PARQUET_SCHEMA, records, rowGroupSize); + + // test reader + LogicalType[] fieldTypes = new LogicalType[]{ + new DoubleType(), + new TinyIntType(), + new IntType()}; + ParquetColumnarRowSplitReader reader = new ParquetColumnarRowSplitReader( + false, + new Configuration(), + fieldTypes, + new String[] {"f7", "f2", "f4"}, + VectorizedColumnBatch::new, + 500, + new org.apache.hadoop.fs.Path(testPath.getPath()), + 0, + Long.MAX_VALUE); + int i = 0; + while (!reader.reachedEnd()) { + ColumnarRow row = reader.nextRecord(); + assertEquals(i, row.getDouble(0), 0); + assertEquals((byte) i, row.getByte(1)); + assertEquals(i, row.getInt(2)); + i++; + } + reader.close(); + } + + @Test + public void testPartitionValues() throws IOException { + // prepare parquet file + int number = 1000; + List records = new ArrayList<>(number); + for (int i = 0; i < number; i++) { + Integer v = i; + records.add(newRow(v)); + } + + Path testPath = createTempParquetFile( + TEMPORARY_FOLDER.newFolder(), PARQUET_SCHEMA, records, rowGroupSize); + + // test reader + Map partSpec = new HashMap<>(); + partSpec.put("f15", true); + partSpec.put("f16", Date.valueOf("2020-11-23")); + partSpec.put("f17", LocalDateTime.of(1999, 1, 1, 1, 1)); + partSpec.put("f18", 6.6); + partSpec.put("f19", (byte) 9); + partSpec.put("f20", (short) 10); + partSpec.put("f21", 11); + partSpec.put("f22", 12L); + partSpec.put("f23", 13f); + partSpec.put("f24", new BigDecimal(24)); + partSpec.put("f25", new BigDecimal(25)); + partSpec.put("f26", new BigDecimal(26)); + partSpec.put("f27", "f27"); + + innerTestPartitionValues(testPath, partSpec, false); + + for (String k : new ArrayList<>(partSpec.keySet())) { + partSpec.put(k, null); + } + + innerTestPartitionValues(testPath, partSpec, true); + } + + private void innerTestPartitionValues( + Path testPath, + Map partSpec, + boolean nullPartValue) throws IOException { + LogicalType[] fieldTypes = new LogicalType[]{ + new VarCharType(VarCharType.MAX_LENGTH), + new BooleanType(), + new TinyIntType(), + new SmallIntType(), + new IntType(), + new BigIntType(), + new FloatType(), + new DoubleType(), + new TimestampType(9), + new DecimalType(5, 0), + new DecimalType(15, 0), + new DecimalType(20, 0), + new DecimalType(5, 0), + new DecimalType(15, 0), + new DecimalType(20, 0), + new BooleanType(), + new DateType(), + new TimestampType(9), + new DoubleType(), + new TinyIntType(), + new SmallIntType(), + new IntType(), + new BigIntType(), + new FloatType(), + new DecimalType(5, 0), + new DecimalType(15, 0), + new DecimalType(20, 0), + new VarCharType(VarCharType.MAX_LENGTH)}; + ParquetColumnarRowSplitReader reader = ParquetSplitReaderUtil.genPartColumnarRowReader( + false, + new Configuration(), + IntStream.range(0, 28).mapToObj(i -> "f" + i).toArray(String[]::new), + Arrays.stream(fieldTypes) + .map(TypeConversions::fromLogicalToDataType) + .toArray(DataType[]::new), + partSpec, + new int[]{7, 2, 4, 15, 19, 20, 21, 22, 23, 18, 16, 17, 24, 25, 26, 27}, + rowGroupSize, + new Path(testPath.getPath()), + 0, + Long.MAX_VALUE); + int i = 0; + while (!reader.reachedEnd()) { + ColumnarRow row = reader.nextRecord(); + + // common values + assertEquals(i, row.getDouble(0), 0); + assertEquals((byte) i, row.getByte(1)); + assertEquals(i, row.getInt(2)); + + // partition values + if (nullPartValue) { + for (int j = 3; j < 16; j++) { + assertTrue(row.isNullAt(j)); + } + } else { + assertTrue(row.getBoolean(3)); + assertEquals(9, row.getByte(4)); + assertEquals(10, row.getShort(5)); + assertEquals(11, row.getInt(6)); + assertEquals(12, row.getLong(7)); + assertEquals(13, row.getFloat(8), 0); + assertEquals(6.6, row.getDouble(9), 0); + assertEquals( + SqlDateTimeUtils.dateToInternal(Date.valueOf("2020-11-23")), + row.getInt(10)); + assertEquals( + LocalDateTime.of(1999, 1, 1, 1, 1), + row.getTimestamp(11, 9).toLocalDateTime()); + assertEquals( + Decimal.fromBigDecimal(new BigDecimal(24), 5, 0), + row.getDecimal(12, 5, 0)); + assertEquals( + Decimal.fromBigDecimal(new BigDecimal(25), 15, 0), + row.getDecimal(13, 15, 0)); + assertEquals( + Decimal.fromBigDecimal(new BigDecimal(26), 20, 0), + row.getDecimal(14, 20, 0)); + assertEquals("f27", row.getString(15).toString()); + } + + i++; + } + reader.close(); + } +} -- Gitee From 1ec78b7feef1ca703ba5747a4b11e0b10a92eac9 Mon Sep 17 00:00:00 2001 From: Qingsheng Ren Date: Fri, 6 Mar 2020 13:51:49 +0800 Subject: [PATCH 089/885] [FLINK-16129][docs-zh] Translate /ops/filesystems/index.zh.md into Chinese This closes #11167 --- docs/ops/filesystems/index.zh.md | 102 ++++++++++++------------------- 1 file changed, 40 insertions(+), 62 deletions(-) diff --git a/docs/ops/filesystems/index.zh.md b/docs/ops/filesystems/index.zh.md index a27ec72dec..936063e73f 100644 --- a/docs/ops/filesystems/index.zh.md +++ b/docs/ops/filesystems/index.zh.md @@ -1,5 +1,5 @@ --- -title: "File Systems" +title: "文件系统" nav-id: filesystems nav-parent_id: ops nav-show_overview: true @@ -24,104 +24,82 @@ specific language governing permissions and limitations under the License. --> -Apache Flink uses file systems to consume and persistently store data, both for the results of applications and for fault tolerance and recovery. -These are some of most of the popular file systems, including *local*, *hadoop-compatible*, *Amazon S3*, *MapR FS*, *OpenStack Swift FS*, *Aliyun OSS* and *Azure Blob Storage*. +Apache Flink 使用文件系统来消费和持久化地存储数据,以处理应用结果以及容错与恢复。以下是一些最常用的文件系统:*本地存储*,*hadoop-compatible*,*Amazon S3*,*MapR FS*,*OpenStack Swift FS*,*阿里云 OSS* 和 *Azure Blob Storage*。 -The file system used for a particular file is determined by its URI scheme. -For example, `file:///home/user/text.txt` refers to a file in the local file system, while `hdfs://namenode:50010/data/user/text.txt` is a file in a specific HDFS cluster. +文件使用的文件系统通过其 URI Scheme 指定。例如 `file:///home/user/text.txt` 表示一个在本地文件系统中的文件,`hdfs://namenode:50010/data/user/text.txt` 表示一个在指定 HDFS 集群中的文件。 -File system instances are instantiated once per process and then cached/pooled, to avoid configuration overhead per stream creation and to enforce certain constraints, such as connection/stream limits. +文件系统在每个进程实例化一次,然后进行缓存/池化,从而避免每次创建流时的配置开销,并强制执行特定的约束,如连接/流的限制。 * This will be replaced by the TOC {:toc} -## Local File System +## 本地文件系统 -Flink has built-in support for the file system of the local machine, including any NFS or SAN drives mounted into that local file system. -It can be used by default without additional configuration. Local files are referenced with the *file://* URI scheme. +Flink 原生支持本地机器上的文件系统,包括任何挂载到本地文件系统的 NFS 或 SAN 驱动器,默认即可使用,无需额外配置。本地文件可通过 *file://* URI Scheme 引用。 -## Pluggable File Systems +## 外部文件系统 -The Apache Flink project supports the following file systems: +Apache Flink 支持下列文件系统: + - [**Amazon S3**](./s3.html) 对象存储由 `flink-s3-fs-presto` 和 `flink-s3-fs-hadoop` 两种替代实现提供支持。这两种实现都是独立的,没有依赖项。 - - [**Amazon S3**](./s3.html) object storage is supported by two alternative implementations: `flink-s3-fs-presto` and `flink-s3-fs-hadoop`. - Both implementations are self-contained with no dependency footprint. + - **MapR FS** 文件系统适配器已在 Flink 的主发行版中通过 *maprfs://* URI Scheme 支持。MapR 库需要在 classpath 中指定(例如在 `lib` 目录中)。 - - **MapR FS** file system adapter is already supported in the main Flink distribution under the *maprfs://* URI scheme. - You must provide the MapR libraries in the classpath (for example in `lib` directory). + - **OpenStack Swift FS** 由 `flink-swift-fs-hadoop` 支持,并通过 *swift://* URI scheme 使用。该实现基于 [Hadoop Project](https://hadoop.apache.org/),但其是独立的,没有依赖项。 + 将 Flink 作为库使用时,使用该文件系统需要添加相应的 Maven 依赖项(`org.apache.flink:flink-swift-fs-hadoop:{{ site.version }}`)。 - - **OpenStack Swift FS** is supported by `flink-swift-fs-hadoop` and registered under the *swift://* URI scheme. - The implementation is based on the [Hadoop Project](https://hadoop.apache.org/) but is self-contained with no dependency footprint. - To use it when using Flink as a library, add the respective maven dependency (`org.apache.flink:flink-swift-fs-hadoop:{{ site.version }}`). - - - **[Aliyun Object Storage Service](./oss.html)** is supported by `flink-oss-fs-hadoop` and registered under the *oss://* URI scheme. - The implementation is based on the [Hadoop Project](https://hadoop.apache.org/) but is self-contained with no dependency footprint. + - **[阿里云对象存储](./oss.html)**由 `flink-oss-fs-hadoop` 支持,并通过 *oss://* URI scheme 使用。该实现基于 [Hadoop Project](https://hadoop.apache.org/),但其是独立的,没有依赖项。 - - **[Azure Blob Storage](./azure.html)** is supported by `flink-azure-fs-hadoop` and registered under the *wasb(s)://* URI schemes. - The implementation is based on the [Hadoop Project](https://hadoop.apache.org/) but is self-contained with no dependency footprint. + - **[Azure Blob Storage](./azure.html)** 由`flink-azure-fs-hadoop` 支持,并通过 *wasb(s)://* URI scheme 使用。该实现基于 [Hadoop Project](https://hadoop.apache.org/),但其是独立的,没有依赖项。 -Except **MapR FS**, you can and should use any of them as [plugins](../plugins.html). +除 **MapR FS** 之外,上述文件系统可以并且需要作为[插件](../plugins.html)使用。 -To use a pluggable file systems, copy the corresponding JAR file from the `opt` directory to a directory under `plugins` directory -of your Flink distribution before starting Flink, e.g. +使用外部文件系统时,在启动 Flink 之前需将对应的 JAR 文件从 `opt` 目录复制到 Flink 发行版 `plugin` 目录下的某一文件夹中,例如: {% highlight bash %} mkdir ./plugins/s3-fs-hadoop cp ./opt/flink-s3-fs-hadoop-{{ site.version }}.jar ./plugins/s3-fs-hadoop/ {% endhighlight %} -Attention The [plugin](../plugins.html) mechanism for file systems was introduced in Flink version `1.9` to -support dedicated Java class loaders per plugin and to move away from the class shading mechanism. -You can still use the provided file systems (or your own implementations) via the old mechanism by copying the corresponding -JAR file into `lib` directory. However, **since 1.10, s3 plugins must be loaded through the plugin mechanism**; the old -way no longer works as these plugins are not shaded anymore (or more specifically the classes are not relocated since 1.10). +注意 文件系统的[插件](../plugins.html)机制在 Flink 版本 1.9 中引入,以支持每个插件专有 Java 类加载器,并避免类隐藏机制。您仍然可以通过旧机制使用文件系统,即将对应的 JAR 文件复制到 `lib` 目录中,或使用您自己的实现方式,但是从版本 1.10 开始,**S3 插件必须通过插件机制加载**,因为这些插件不再被隐藏(版本 1.10 之后类不再被重定位),旧机制不再可用。 -It's encouraged to use the [plugins](../plugins.html)-based loading mechanism for file systems that support it. Loading file systems components from the `lib` -directory will not supported in future Flink versions. +尽可能通过基于[插件](../plugins.html)的加载机制使用支持的文件系统。未来的 Flink 版本将不再支持通过 `lib` 目录加载文件系统组件。 -## Adding a new pluggable File System implementation +## 添加新的外部文件系统实现 -File systems are represented via the `org.apache.flink.core.fs.FileSystem` class, which captures the ways to access and modify files and objects in that file system. +文件系统由类 `org.apache.flink.core.fs.FileSystem` 表示,该类定义了访问与修改文件系统中文件与对象的方法。 -To add a new file system: +要添加一个新的文件系统: - - Add the File System implementation, which is a subclass of `org.apache.flink.core.fs.FileSystem`. - - Add a factory that instantiates that file system and declares the scheme under which the FileSystem is registered. This must be a subclass of `org.apache.flink.core.fs.FileSystemFactory`. - - Add a service entry. Create a file `META-INF/services/org.apache.flink.core.fs.FileSystemFactory` which contains the class name of your file system factory class - (see the [Java Service Loader docs](https://docs.oracle.com/javase/8/docs/api/java/util/ServiceLoader.html) for more details). + - 添加文件系统实现,它应是 `org.apache.flink.core.fs.FileSystem` 的子类。 + - 添加 Factory 类,以实例化该文件系统并声明文件系统所注册的 scheme, 它应是 `org.apache.flink.core.fs.FileSystemFactory` 的子类。 + - 添加 Service Entry。创建文件 `META-INF/services/org.apache.flink.core.fs.FileSystemFactory`,文件中包含文件系统 Factory 类的类名。 + (更多细节请查看 [Java Service Loader docs](https://docs.oracle.com/javase/8/docs/api/java/util/ServiceLoader.html)) -During plugins discovery, the file system factory class will be loaded by a dedicated Java class loader to avoid class conflicts with other plugins and Flink components. -The same class loader should be used during file system instantiation and the file system operation calls. +在插件检索时,文件系统 Factory 类会由一个专用的 Java 类加载器加载,从而避免与其他类或 Flink 组件冲突。在文件系统实例化和文件系统调用时,应使用该类加载器。 -Warning In practice, it means you should avoid using `Thread.currentThread().getContextClassLoader()` class loader -in your implementation. +警告 实际上这表示您的实现应避免使用 `Thread.currentThread().getContextClassLoader()` 类加载器。 -## Hadoop File System (HDFS) and its other implementations +## Hadoop 文件系统 (HDFS) 及其其他实现 -For all schemes where Flink cannot find a directly supported file system, it falls back to Hadoop. -All Hadoop file systems are automatically available when `flink-runtime` and the Hadoop libraries are on the classpath. -See also **[Hadoop Integration]({{ site.baseurl }}/ops/deployment/hadoop.html)**. +所有 Flink 无法找到直接支持的文件系统均将回退为 Hadoop。 +当 `flink-runtime` 和 Hadoop 类包含在 classpath 中时,所有的 Hadoop 文件系统将自动可用。参见 **[Hadoop 集成]({{ site.baseurl }}/zh/../../s3.zh.mdops/deployment/hadoop.html)**。 -This way, Flink seamlessly supports all of Hadoop file systems implementing the `org.apache.hadoop.fs.FileSystem` interface, -and all Hadoop-compatible file systems (HCFS). - - - HDFS (tested) - - [Google Cloud Storage Connector for Hadoop](https://cloud.google.com/hadoop/google-cloud-storage-connector) (tested) - - [Alluxio](http://alluxio.org/) (tested, see configuration specifics below) - - [XtreemFS](http://www.xtreemfs.org/) (tested) - - FTP via [Hftp](http://hadoop.apache.org/docs/r1.2.1/hftp.html) (not tested) - - HAR (not tested) +因此,Flink 无缝支持所有实现 `org.apache.hadoop.fs.FileSystem` 接口的 Hadoop 文件系统和所有兼容 Hadoop 的文件系统 (Hadoop-compatible file system, HCFS): + - HDFS (已测试) + - [Google Cloud Storage Connector for Hadoop](https://cloud.google.com/hadoop/google-cloud-storage-connector)(已测试) + - [Alluxio](http://alluxio.org/)(已测试,参见下文的配置详细信息) + - [XtreemFS](http://www.xtreemfs.org/)(已测试) + - FTP via [Hftp](http://hadoop.apache.org/docs/r1.2.1/hftp.html)(未测试) + - HAR(未测试) - ... -The Hadoop configuration has to have an entry for the required file system implementation in the `core-site.xml` file. -See example for **[Alluxio]({{ site.baseurl }}/ops/filesystems/#alluxio)**. +Hadoop 配置须在 `core-site.xml` 文件中包含所需文件系统的实现。可查看 **[Alluxio 的示例]({{ site.baseurl }}/zh/ops/filesystems/#alluxio)**。 -We recommend using Flink's built-in file systems unless required otherwise. Using a Hadoop File System directly may be required, -for example, when using that file system for YARN's resource storage, via the `fs.defaultFS` configuration property in Hadoop's `core-site.xml`. +除非有其他的需要,建议使用 Flink 内置的文件系统。在某些情况下,如通过配置 Hadoop `core-site.xml` 中的 `fs.defaultFS` 属性将文件系统作为 YARN 的资源存储时,可能需要直接使用 Hadoop 文件系统。 ### Alluxio -For Alluxio support add the following entry into the `core-site.xml` file: +在 `core-site.xml` 文件中添加以下条目以支持 Alluxio: {% highlight xml %} -- Gitee From 5adbd70a30d72df1b315a36e79ccdf099f3aabc3 Mon Sep 17 00:00:00 2001 From: Qingsheng Ren Date: Sat, 22 Feb 2020 15:53:04 +0800 Subject: [PATCH 090/885] [FLINK-16130][docs-zh] Translate /opt/filesystems/common.zh.md into Chinese This closes #11182 --- docs/ops/filesystems/common.md | 2 +- docs/ops/filesystems/common.zh.md | 41 +++++++++++++------------------ 2 files changed, 18 insertions(+), 25 deletions(-) diff --git a/docs/ops/filesystems/common.md b/docs/ops/filesystems/common.md index a8a371ed6a..468e62502e 100644 --- a/docs/ops/filesystems/common.md +++ b/docs/ops/filesystems/common.md @@ -62,7 +62,7 @@ If the opening of the stream takes longer than `fs..limit.timeout`, the To prevent inactive streams from taking up the full pool (preventing new connections to be opened), you can add an inactivity timeout which forcibly closes them if they do not read/write any bytes for at least that amount of time: `fs..limit.stream-timeout`. -Limit enforcment on a per TaskManager/file system basis. +Limit enforcement on a per TaskManager/file system basis. Because file systems creation occurs per scheme and authority, different authorities have independent connection pools. For example `hdfs://myhdfs:50010/` and `hdfs://anotherhdfs:4399/` will have separate pools. diff --git a/docs/ops/filesystems/common.zh.md b/docs/ops/filesystems/common.zh.md index a8a371ed6a..2c5b1675b5 100644 --- a/docs/ops/filesystems/common.zh.md +++ b/docs/ops/filesystems/common.zh.md @@ -1,5 +1,5 @@ --- -title: "Common Configurations" +title: "通用配置" nav-parent_id: filesystems nav-pos: 0 --- @@ -22,48 +22,41 @@ specific language governing permissions and limitations under the License. --> -Apache Flink provides several standard configuration settings that work across all file system implementations. +Apache Flink 提供了一些对所有文件系统均适用的基本配置。 * This will be replaced by the TOC {:toc} -## Default File System +## 默认文件系统 -A default scheme (and authority) is used if paths to files do not explicitly specify a file system scheme (and authority). +如果文件路径未明确指定文件系统的 scheme(和 authority),将会使用默认的 scheme(和 authority): {% highlight yaml %} fs.default-scheme: {% endhighlight %} -For example, if the default file system configured as `fs.default-scheme: hdfs://localhost:9000/`, then a file path of -`/user/hugo/in.txt` is interpreted as `hdfs://localhost:9000/user/hugo/in.txt`. +例如默认的文件系统配置为 `fs.default-scheme: hdfs://localhost:9000/`,则文件路径 `/user/hugo/in.txt` 将被处理为 `hdfs://localhost:9000/user/hugo/in.txt`。 -## Connection limiting +## 连接限制 -You can limit the total number of connections that a file system can concurrently open which is useful when the file system cannot handle a large number -of concurrent reads/writes or open connections at the same time. +如果文件系统不能处理大量并发读/写操作或连接,可以为文件系统同时打开的总连接数设置上限。 -For example, small HDFS clusters with few RPC handlers can sometimes be overwhelmed by a large Flink job trying to build up many connections during a checkpoint. +例如在一个大型 Flink 任务建立 checkpoint 时,具有少量 RPC handler 的小型 HDFS 集群可能会由于建立了过多的连接而过载。 -To limit a specific file system's connections, add the following entries to the Flink configuration. The file system to be limited is identified by -its scheme. +要限制文件系统的连接数,可将下列配置添加至 Flink 配置中。设置限制的文件系统由其 scheme 指定: {% highlight yaml %} -fs..limit.total: (number, 0/-1 mean no limit) -fs..limit.input: (number, 0/-1 mean no limit) -fs..limit.output: (number, 0/-1 mean no limit) -fs..limit.timeout: (milliseconds, 0 means infinite) -fs..limit.stream-timeout: (milliseconds, 0 means infinite) +fs..limit.total: (数量,0/-1 表示无限制) +fs..limit.input: (数量,0/-1 表示无限制) +fs..limit.output: (数量,0/-1 表示无限制) +fs..limit.timeout: (毫秒,0 表示无穷) +fs..limit.stream-timeout: (毫秒,0 表示无穷) {% endhighlight %} -You can limit the number of input/output connections (streams) separately (`fs..limit.input` and `fs..limit.output`), as well as impose a limit on -the total number of concurrent streams (`fs..limit.total`). If the file system tries to open more streams, the operation blocks until some streams close. -If the opening of the stream takes longer than `fs..limit.timeout`, the stream opening fails. +输入和输出连接(流)的数量可以分别进行限制(`fs..limit.input` 和 `fs..limit.output`),也可以限制并发流的总数(`fs..limit.total`)。如果文件系统尝试打开更多的流,操作将被阻塞直至某些流关闭。如果打开流的时间超过 `fs..limit.timeout`,则流打开失败。 -To prevent inactive streams from taking up the full pool (preventing new connections to be opened), you can add an inactivity timeout which forcibly closes them if they do not read/write any bytes for at least that amount of time: `fs..limit.stream-timeout`. +为避免不活动的流占满整个连接池(阻止新连接的建立),可以在配置中添加无活动超时时间,如果连接至少在 `fs..limit.stream-timeout` 时间内没有读/写操作,则连接会被强制关闭。 -Limit enforcment on a per TaskManager/file system basis. -Because file systems creation occurs per scheme and authority, different -authorities have independent connection pools. For example `hdfs://myhdfs:50010/` and `hdfs://anotherhdfs:4399/` will have separate pools. +连接数是按每个 TaskManager/文件系统来进行限制的。因为文件系统的创建是按照 scheme 和 authority 进行的,所以不同的 authority 具有独立的连接池,例如 `hdfs://myhdfs:50010/` 和 `hdfs://anotherhdfs:4399/` 会有单独的连接池。 {% top %} \ No newline at end of file -- Gitee From 2a3daf3931acd0b6089c802c34e18f3bc59b7bf6 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Fri, 28 Feb 2020 20:57:32 +0800 Subject: [PATCH 091/885] [FLINK-16337][python][table-planner-blink] Add support of vectorized Python UDF in blink planner --- flink-python/pyflink/table/udf.py | 3 + .../functions/python/PythonFunction.java | 7 + .../functions/python/PythonFunctionKind.java | 32 +++ .../python/SimplePythonFunction.java | 13 +- .../logical/PythonCorrelateSplitRule.java | 4 +- .../batch/BatchExecPythonCorrelateRule.java | 4 +- .../stream/StreamExecPythonCorrelateRule.java | 4 +- .../codegen/PythonFunctionCodeGenerator.scala | 12 +- .../plan/nodes/common/CommonPythonBase.scala | 3 +- .../plan/nodes/common/CommonPythonCalc.scala | 21 +- .../physical/stream/StreamExecMatch.scala | 4 +- .../plan/rules/FlinkBatchRuleSets.scala | 1 + .../plan/rules/FlinkStreamRuleSets.scala | 1 + .../rules/logical/PythonCalcSplitRule.scala | 78 +++++-- ...plitPythonConditionFromCorrelateRule.scala | 4 +- .../SplitPythonConditionFromJoinRule.scala | 4 +- .../physical/batch/BatchExecCalcRule.scala | 2 +- .../batch/BatchExecPythonCalcRule.scala | 2 +- .../physical/stream/StreamExecCalcRule.scala | 2 +- .../stream/StreamExecPythonCalcRule.scala | 2 +- .../table/planner/plan/utils/PythonUtil.scala | 50 ++-- .../utils/python/PythonTableUtils.scala | 5 +- .../utils/JavaUserDefinedScalarFunctions.java | 29 +++ .../rules/logical/PythonCalcSplitRuleTest.xml | 220 ++++++++++++++---- .../ExpressionReductionRulesTest.scala | 2 +- .../logical/PythonCalcSplitRuleTest.scala | 60 ++++- .../table/plan/nodes/CommonPythonBase.scala | 3 +- .../utils/JavaUserDefinedScalarFunctions.java | 11 + .../plan/ExpressionReductionRulesTest.scala | 4 +- 29 files changed, 484 insertions(+), 103 deletions(-) create mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/python/PythonFunctionKind.java diff --git a/flink-python/pyflink/table/udf.py b/flink-python/pyflink/table/udf.py index 05f5784608..c130563086 100644 --- a/flink-python/pyflink/table/udf.py +++ b/flink-python/pyflink/table/udf.py @@ -199,6 +199,8 @@ class UserDefinedScalarFunctionWrapper(UserDefinedFunctionWrapper): j_input_types = utils.to_jarray(gateway.jvm.TypeInformation, [_to_java_type(i) for i in self._input_types]) j_result_type = _to_java_type(self._result_type) + j_function_kind = gateway.jvm.org.apache.flink.table.functions.python.\ + PythonFunctionKind.GENERAL if is_blink_planner: PythonTableUtils = gateway.jvm\ .org.apache.flink.table.planner.utils.python.PythonTableUtils @@ -208,6 +210,7 @@ class UserDefinedScalarFunctionWrapper(UserDefinedFunctionWrapper): bytearray(serialized_func), j_input_types, j_result_type, + j_function_kind, self._deterministic, _get_python_env()) else: diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/python/PythonFunction.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/python/PythonFunction.java index cd6bf9c96a..f2b416996a 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/python/PythonFunction.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/python/PythonFunction.java @@ -38,4 +38,11 @@ public interface PythonFunction extends Serializable { * Returns the Python execution environment. */ PythonEnv getPythonEnv(); + + /** + * Returns the kind of the user-defined python function. + */ + default PythonFunctionKind getPythonFunctionKind() { + return PythonFunctionKind.GENERAL; + } } diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/python/PythonFunctionKind.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/python/PythonFunctionKind.java new file mode 100644 index 0000000000..c6918a22f1 --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/python/PythonFunctionKind.java @@ -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.flink.table.functions.python; + +import org.apache.flink.annotation.Internal; + +/** + * Categorizes the Python functions. + */ +@Internal +public enum PythonFunctionKind { + + GENERAL, + + PANDAS +} diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/python/SimplePythonFunction.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/python/SimplePythonFunction.java index 116151ad1d..1cb70f524f 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/python/SimplePythonFunction.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/python/SimplePythonFunction.java @@ -39,9 +39,15 @@ public final class SimplePythonFunction implements PythonFunction { */ private final PythonEnv pythonEnv; - public SimplePythonFunction(byte[] serializedPythonFunction, PythonEnv pythonEnv) { + /** + * The kind of the user-defined python function. + */ + private final PythonFunctionKind pythonFunctionKind; + + public SimplePythonFunction(byte[] serializedPythonFunction, PythonEnv pythonEnv, PythonFunctionKind pythonFunctionKind) { this.serializedPythonFunction = Preconditions.checkNotNull(serializedPythonFunction); this.pythonEnv = Preconditions.checkNotNull(pythonEnv); + this.pythonFunctionKind = Preconditions.checkNotNull(pythonFunctionKind); } @Override @@ -53,4 +59,9 @@ public final class SimplePythonFunction implements PythonFunction { public PythonEnv getPythonEnv() { return pythonEnv; } + + @Override + public PythonFunctionKind getPythonFunctionKind() { + return pythonFunctionKind; + } } diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PythonCorrelateSplitRule.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PythonCorrelateSplitRule.java index cf6421029a..ce4522a5ec 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PythonCorrelateSplitRule.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PythonCorrelateSplitRule.java @@ -92,7 +92,7 @@ public class PythonCorrelateSplitRule extends RelOptRule { } RexNode rexNode = pythonTableFuncScan.getCall(); if (rexNode instanceof RexCall) { - return PythonUtil.isPythonCall(rexNode) && PythonUtil.containsNonPythonCall(rexNode); + return PythonUtil.isPythonCall(rexNode, null) && PythonUtil.containsNonPythonCall(rexNode); } return false; } @@ -196,7 +196,7 @@ public class PythonCorrelateSplitRule extends RelOptRule { ScalarFunctionSplitter splitter = new ScalarFunctionSplitter( primitiveLeftFieldCount, extractedJavaRexCalls, - false + PythonUtil::isNonPythonCall ); RelNode rightNewInput; diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecPythonCorrelateRule.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecPythonCorrelateRule.java index 7fb02cc1b9..4ecb1dfce3 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecPythonCorrelateRule.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecPythonCorrelateRule.java @@ -58,7 +58,7 @@ public class BatchExecPythonCorrelateRule extends ConverterRule { // right node is a table function FlinkLogicalTableFunctionScan scan = (FlinkLogicalTableFunctionScan) right; // return true if the table function is python table function - return PythonUtil.isPythonCall(scan.getCall()); + return PythonUtil.isPythonCall(scan.getCall(), null); } else if (right instanceof FlinkLogicalCalc) { // a filter is pushed above the table function FlinkLogicalCalc calc = (FlinkLogicalCalc) right; @@ -66,7 +66,7 @@ public class BatchExecPythonCorrelateRule extends ConverterRule { if (input instanceof FlinkLogicalTableFunctionScan) { FlinkLogicalTableFunctionScan scan = (FlinkLogicalTableFunctionScan) input; // return true if the table function is python table function - return PythonUtil.isPythonCall(scan.getCall()); + return PythonUtil.isPythonCall(scan.getCall(), null); } } return false; diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecPythonCorrelateRule.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecPythonCorrelateRule.java index 2f1ff2e498..8c2b12e92a 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecPythonCorrelateRule.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecPythonCorrelateRule.java @@ -54,7 +54,7 @@ public class StreamExecPythonCorrelateRule extends ConverterRule { RelNode child = ((RelSubset) calc.getInput()).getOriginal(); if (child instanceof FlinkLogicalTableFunctionScan) { FlinkLogicalTableFunctionScan scan = (FlinkLogicalTableFunctionScan) child; - return PythonUtil.isPythonCall(scan.getCall()); + return PythonUtil.isPythonCall(scan.getCall(), null); } else if (child instanceof FlinkLogicalCalc) { FlinkLogicalCalc childCalc = (FlinkLogicalCalc) child; return findTableFunction(childCalc); @@ -70,7 +70,7 @@ public class StreamExecPythonCorrelateRule extends ConverterRule { // right node is a table function FlinkLogicalTableFunctionScan scan = (FlinkLogicalTableFunctionScan) right; // return true if the table function is python table function - return PythonUtil.isPythonCall(scan.getCall()); + return PythonUtil.isPythonCall(scan.getCall(), null); } else if (right instanceof FlinkLogicalCalc) { // a filter is pushed above the table function return findTableFunction((FlinkLogicalCalc) right); diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/PythonFunctionCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/PythonFunctionCodeGenerator.scala index cd256fe66c..152aafd387 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/PythonFunctionCodeGenerator.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/PythonFunctionCodeGenerator.scala @@ -19,7 +19,7 @@ package org.apache.flink.table.planner.codegen import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.RowTypeInfo -import org.apache.flink.table.functions.python.{PythonEnv, PythonFunction} +import org.apache.flink.table.functions.python.{PythonEnv, PythonFunction, PythonFunctionKind} import org.apache.flink.table.functions.{ScalarFunction, TableFunction, UserDefinedFunction} import org.apache.flink.table.planner.codegen.CodeGenUtils.{newName, primitiveDefaultValue, primitiveTypeTermForType} import org.apache.flink.table.planner.codegen.Indenter.toISC @@ -44,6 +44,7 @@ object PythonFunctionCodeGenerator { * @param serializedScalarFunction serialized Python scalar function * @param inputTypes input data types * @param resultType expected result type + * @param pythonFunctionKind the kind of the Python function * @param deterministic the determinism of the function's results * @param pythonEnv the Python execution environment * @return instance of generated ScalarFunction @@ -54,6 +55,7 @@ object PythonFunctionCodeGenerator { serializedScalarFunction: Array[Byte], inputTypes: Array[TypeInformation[_]], resultType: TypeInformation[_], + pythonFunctionKind: PythonFunctionKind, deterministic: Boolean, pythonEnv: PythonEnv): ScalarFunction = { val funcName = newName(PYTHON_SCALAR_FUNCTION_NAME) @@ -67,6 +69,7 @@ object PythonFunctionCodeGenerator { val typeInfoTypeTerm = classOf[TypeInformation[_]].getCanonicalName val pythonEnvTypeTerm = classOf[PythonEnv].getCanonicalName + val pythonFunctionKindTypeTerm = classOf[PythonFunctionKind].getCanonicalName val resultTypeNameTerm = ctx.addReusableObject(resultType, "resultType", typeInfoTypeTerm) @@ -76,6 +79,8 @@ object PythonFunctionCodeGenerator { val inputTypesCode = inputTypes .map(ctx.addReusableObject(_, "inputType", typeInfoTypeTerm)) .mkString(", ") + val pythonFunctionKindNameTerm = + ctx.addReusableObject(pythonFunctionKind, "pythonFunctionKind", pythonFunctionKindTypeTerm) val funcCode = j""" |public class $funcName extends ${classOf[ScalarFunction].getCanonicalName} @@ -114,6 +119,11 @@ object PythonFunctionCodeGenerator { | } | | @Override + | public $pythonFunctionKindTypeTerm getPythonFunctionKind() { + | return $pythonFunctionKindNameTerm; + | } + | + | @Override | public boolean isDeterministic() { | return $deterministic; | } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonBase.scala index 52e23a1770..745cd3f26f 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonBase.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonBase.scala @@ -74,7 +74,8 @@ trait CommonPythonBase { // the serialized Python function, the Python env, etc val pythonFunction = new SimplePythonFunction( func.asInstanceOf[PythonFunction].getSerializedPythonFunction, - func.asInstanceOf[PythonFunction].getPythonEnv) + func.asInstanceOf[PythonFunction].getPythonEnv, + func.asInstanceOf[PythonFunction].getPythonFunctionKind) new PythonFunctionInfo(pythonFunction, inputs.toArray) } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonCalc.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonCalc.scala index c7d7d30ee4..66c9bc7ecf 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonCalc.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonCalc.scala @@ -24,13 +24,16 @@ import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.operators.OneInputStreamOperator import org.apache.flink.streaming.api.transformations.OneInputTransformation import org.apache.flink.table.dataformat.BaseRow -import org.apache.flink.table.functions.python.PythonFunctionInfo +import org.apache.flink.table.functions.python.{PythonFunctionInfo, PythonFunctionKind} import org.apache.flink.table.planner.calcite.FlinkTypeFactory +import org.apache.flink.table.planner.plan.nodes.common.CommonPythonCalc.ARROW_PYTHON_SCALAR_FUNCTION_OPERATOR_NAME import org.apache.flink.table.planner.plan.nodes.common.CommonPythonCalc.PYTHON_SCALAR_FUNCTION_OPERATOR_NAME +import org.apache.flink.table.planner.plan.utils.PythonUtil.containsPythonCall import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo import org.apache.flink.table.types.logical.RowType import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.mutable trait CommonPythonCalc extends CommonPythonBase { @@ -53,8 +56,13 @@ trait CommonPythonCalc extends CommonPythonBase { outputRowTypeInfo: BaseRowTypeInfo, udfInputOffsets: Array[Int], pythonFunctionInfos: Array[PythonFunctionInfo], - forwardedFields: Array[Int])= { - val clazz = loadClass(PYTHON_SCALAR_FUNCTION_OPERATOR_NAME) + forwardedFields: Array[Int], + isArrow: Boolean)= { + val clazz = if (isArrow) { + loadClass(ARROW_PYTHON_SCALAR_FUNCTION_OPERATOR_NAME) + } else { + loadClass(PYTHON_SCALAR_FUNCTION_OPERATOR_NAME) + } val ctor = clazz.getConstructor( classOf[Configuration], classOf[Array[PythonFunctionInfo]], @@ -103,7 +111,8 @@ trait CommonPythonCalc extends CommonPythonBase { pythonOperatorResultTyeInfo, pythonUdfInputOffsets, pythonFunctionInfos, - forwardedFields) + forwardedFields, + calcProgram.getExprList.asScala.exists(containsPythonCall(_, PythonFunctionKind.PANDAS))) new OneInputTransformation( inputTransform, @@ -118,4 +127,8 @@ trait CommonPythonCalc extends CommonPythonBase { object CommonPythonCalc { val PYTHON_SCALAR_FUNCTION_OPERATOR_NAME = "org.apache.flink.table.runtime.operators.python.scalar.BaseRowPythonScalarFunctionOperator" + + val ARROW_PYTHON_SCALAR_FUNCTION_OPERATOR_NAME = + "org.apache.flink.table.runtime.operators.python.scalar.arrow." + + "BaseRowArrowPythonScalarFunctionOperator" } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecMatch.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecMatch.scala index c3ed8e57d0..9b40944897 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecMatch.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecMatch.scala @@ -76,8 +76,8 @@ class StreamExecMatch( with StreamPhysicalRel with StreamExecNode[BaseRow] { - if (logicalMatch.measures.values().exists(containsPythonCall) || - logicalMatch.patternDefinitions.values().exists(containsPythonCall)) { + if (logicalMatch.measures.values().exists(containsPythonCall(_)) || + logicalMatch.patternDefinitions.values().exists(containsPythonCall(_))) { throw new TableException("Python Function can not be used in MATCH_RECOGNIZE for now.") } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkBatchRuleSets.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkBatchRuleSets.scala index cbf6f58920..104c8f3322 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkBatchRuleSets.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkBatchRuleSets.scala @@ -367,6 +367,7 @@ object FlinkBatchRuleSets { // Rule that splits python ScalarFunctions from java/scala ScalarFunctions PythonCalcSplitRule.SPLIT_CONDITION, PythonCalcSplitRule.SPLIT_PROJECT, + PythonCalcSplitRule.SPLIT_PANDAS_IN_PROJECT, PythonCalcSplitRule.PUSH_CONDITION, PythonCalcSplitRule.REWRITE_PROJECT ) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala index 48e9e28880..44760f6f69 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala @@ -349,6 +349,7 @@ object FlinkStreamRuleSets { // Rule that splits python ScalarFunctions from java/scala ScalarFunctions. PythonCalcSplitRule.SPLIT_CONDITION, PythonCalcSplitRule.SPLIT_PROJECT, + PythonCalcSplitRule.SPLIT_PANDAS_IN_PROJECT, PythonCalcSplitRule.PUSH_CONDITION, PythonCalcSplitRule.REWRITE_PROJECT ) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRule.scala index e0478a85e4..cb98bb5890 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRule.scala @@ -18,13 +18,16 @@ package org.apache.flink.table.planner.plan.rules.logical +import java.util.function.Function + import org.apache.calcite.plan.RelOptRule.{any, operand} import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall} import org.apache.calcite.rex.{RexCall, RexInputRef, RexNode, RexProgram} import org.apache.calcite.sql.validate.SqlValidatorUtil import org.apache.flink.table.functions.ScalarFunction +import org.apache.flink.table.functions.python.PythonFunctionKind import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalCalc -import org.apache.flink.table.planner.plan.utils.PythonUtil.{containsPythonCall, containsNonPythonCall, isPythonCall, isNonPythonCall} +import org.apache.flink.table.planner.plan.utils.PythonUtil.{containsNonPythonCall, containsPythonCall, isNonPythonCall, isPythonCall} import org.apache.flink.table.planner.plan.utils.{InputRefVisitor, RexDefaultVisitor} import scala.collection.JavaConverters._ @@ -52,7 +55,9 @@ abstract class PythonCalcSplitRuleBase(description: String) val splitter = new ScalarFunctionSplitter( extractedFunctionOffset, extractedRexCalls, - isConvertPythonFunction(program)) + new Function[RexCall, Boolean] { + override def apply(rexCall: RexCall): Boolean = needConvertRexCall(program, rexCall) + }) val (bottomCalcCondition, topCalcCondition, topCalcProjects) = split(program, splitter) val accessedFields = extractRefInputFields( @@ -112,9 +117,9 @@ abstract class PythonCalcSplitRuleBase(description: String) } /** - * Returns true if converting Python functions. + * Returns true if need to convert the specified call. */ - def isConvertPythonFunction(program: RexProgram): Boolean + def needConvertRexCall(program: RexProgram, call: RexCall): Boolean /** * Splits the specified [[RexProgram]] using the specified [[ScalarFunctionSplitter]]. @@ -138,10 +143,10 @@ object PythonCalcSplitConditionRule extends PythonCalcSplitRuleBase( // matches if it contains Python functions in condition Option(calc.getProgram.getCondition) - .map(calc.getProgram.expandLocalRef).exists(containsPythonCall) + .map(calc.getProgram.expandLocalRef).exists(containsPythonCall(_)) } - override def isConvertPythonFunction(program: RexProgram): Boolean = true + override def needConvertRexCall(program: RexProgram, call: RexCall): Boolean = isPythonCall(call) override def split(program: RexProgram, splitter: ScalarFunctionSplitter) : (Option[RexNode], Option[RexNode], Seq[RexNode]) = { @@ -150,12 +155,22 @@ object PythonCalcSplitConditionRule extends PythonCalcSplitRuleBase( } } +abstract class PythonCalcSplitProjectionRuleBase(description: String) + extends PythonCalcSplitRuleBase(description) { + + override def split(program: RexProgram, splitter: ScalarFunctionSplitter) + : (Option[RexNode], Option[RexNode], Seq[RexNode]) = { + (Option(program.getCondition).map(program.expandLocalRef), None, + program.getProjectList.map(program.expandLocalRef(_).accept(splitter))) + } +} + /** * Rule that splits [[FlinkLogicalCalc]]s which contain both Java functions and Python functions * in the projection into multiple [[FlinkLogicalCalc]]s. After this rule is applied, it will * only contain Python functions or Java functions in the projection of each [[FlinkLogicalCalc]]. */ -object PythonCalcSplitProjectionRule extends PythonCalcSplitRuleBase( +object PythonCalcSplitProjectionRule extends PythonCalcSplitProjectionRuleBase( "PythonCalcSplitProjectionRule") { override def matches(call: RelOptRuleCall): Boolean = { @@ -163,17 +178,36 @@ object PythonCalcSplitProjectionRule extends PythonCalcSplitRuleBase( val projects = calc.getProgram.getProjectList.map(calc.getProgram.expandLocalRef) // matches if it contains both Python functions and Java functions in the projection - projects.exists(containsPythonCall) && projects.exists(containsNonPythonCall) + projects.exists(containsPythonCall(_)) && projects.exists(containsNonPythonCall) } - override def isConvertPythonFunction(program: RexProgram): Boolean = { - program.getProjectList.map(program.expandLocalRef).exists(isNonPythonCall) + override def needConvertRexCall(program: RexProgram, call: RexCall): Boolean = { + program.getProjectList.map(program.expandLocalRef).exists(isNonPythonCall) == isPythonCall(call) } +} - override def split(program: RexProgram, splitter: ScalarFunctionSplitter) - : (Option[RexNode], Option[RexNode], Seq[RexNode]) = { - (Option(program.getCondition).map(program.expandLocalRef), None, - program.getProjectList.map(program.expandLocalRef(_).accept(splitter))) +/** + * Rule that splits [[FlinkLogicalCalc]]s which contain both general Python functions and + * pandas Python functions in the projection into multiple [[FlinkLogicalCalc]]s. After + * this rule is applied, it will only contain general Python functions or pandas Python + * functions in the projection of each [[FlinkLogicalCalc]]. + */ +object PythonCalcSplitPandasInProjectionRule extends PythonCalcSplitProjectionRuleBase( + "PythonCalcSplitPandasInProjectionRule") { + + override def matches(call: RelOptRuleCall): Boolean = { + val calc: FlinkLogicalCalc = call.rel(0).asInstanceOf[FlinkLogicalCalc] + val projects = calc.getProgram.getProjectList.map(calc.getProgram.expandLocalRef) + + // matches if it contains both general Python functions and + // pandas Python functions in the projection + projects.exists(containsPythonCall(_, PythonFunctionKind.GENERAL)) && + projects.exists(containsPythonCall(_, PythonFunctionKind.PANDAS)) + } + + override def needConvertRexCall(program: RexProgram, call: RexCall): Boolean = { + program.getProjectList.map(program.expandLocalRef).exists( + isPythonCall(_, PythonFunctionKind.GENERAL)) == isPythonCall(call, PythonFunctionKind.PANDAS) } } @@ -191,10 +225,11 @@ object PythonCalcPushConditionRule extends PythonCalcSplitRuleBase( // matches if all the following conditions hold true: // 1) the condition is not null // 2) it contains Python functions in the projection - calc.getProgram.getCondition != null && projects.exists(containsPythonCall) + calc.getProgram.getCondition != null && projects.exists(containsPythonCall(_)) } - override def isConvertPythonFunction(program: RexProgram): Boolean = false + override def needConvertRexCall(program: RexProgram, call: RexCall): Boolean = + isNonPythonCall(call) override def split(program: RexProgram, splitter: ScalarFunctionSplitter) : (Option[RexNode], Option[RexNode], Seq[RexNode]) = { @@ -221,13 +256,13 @@ object PythonCalcRewriteProjectionRule extends PythonCalcSplitRuleBase( // 1) it contains Python functions in the projection // 2) it contains RexNodes besides RexInputRef and RexCall or // not all the RexCalls lying at the end of the project list - projects.exists(containsPythonCall) && + projects.exists(containsPythonCall(_)) && (projects.exists(expr => !expr.isInstanceOf[RexCall] && !expr.isInstanceOf[RexInputRef]) || projects.indexWhere(_.isInstanceOf[RexCall]) < projects.lastIndexWhere(_.isInstanceOf[RexInputRef])) } - override def isConvertPythonFunction(program: RexProgram): Boolean = true + override def needConvertRexCall(program: RexProgram, call: RexCall): Boolean = isPythonCall(call) override def split(program: RexProgram, splitter: ScalarFunctionSplitter) : (Option[RexNode], Option[RexNode], Seq[RexNode]) = { @@ -238,11 +273,11 @@ object PythonCalcRewriteProjectionRule extends PythonCalcSplitRuleBase( private class ScalarFunctionSplitter( extractedFunctionOffset: Int, extractedRexCalls: mutable.ArrayBuffer[RexCall], - convertPythonFunction: Boolean) + needConvertRexCall: Function[RexCall, Boolean]) extends RexDefaultVisitor[RexNode] { override def visitCall(call: RexCall): RexNode = { - visit(if (isPythonCall(call)) convertPythonFunction else !convertPythonFunction, call) + visit(needConvertRexCall(call), call) } override def visitNode(rexNode: RexNode): RexNode = rexNode @@ -299,10 +334,11 @@ private class ExtractedFunctionInputRewriter( object PythonCalcSplitRule { /** * These rules should be applied sequentially in the order of - * SPLIT_CONDITION, SPLIT_PROJECT, PUSH_CONDITION and REWRITE_PROJECT. + * SPLIT_CONDITION, SPLIT_PROJECT, SPLIT_PANDAS_IN_PROJECT, PUSH_CONDITION and REWRITE_PROJECT. */ val SPLIT_CONDITION: RelOptRule = PythonCalcSplitConditionRule val SPLIT_PROJECT: RelOptRule = PythonCalcSplitProjectionRule + val SPLIT_PANDAS_IN_PROJECT: RelOptRule = PythonCalcSplitPandasInProjectionRule val PUSH_CONDITION: RelOptRule = PythonCalcPushConditionRule val REWRITE_PROJECT: RelOptRule = PythonCalcRewriteProjectionRule } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromCorrelateRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromCorrelateRule.scala index dd6adcc02e..a5de6b7f77 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromCorrelateRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromCorrelateRule.scala @@ -55,7 +55,7 @@ class SplitPythonConditionFromCorrelateRule joinType == JoinRelType.INNER && Option(mergedCalc.getProgram.getCondition) .map(mergedCalc.getProgram.expandLocalRef) - .exists(containsPythonCall) + .exists(containsPythonCall(_)) } override def onMatch(call: RelOptRuleCall): Unit = { @@ -97,7 +97,7 @@ class SplitPythonConditionFromCorrelateRule correlate.getRowType.getFieldCount - mergedCalc.getRowType.getFieldCount) val pythonFilters = correlateFilters - .filter(containsPythonCall) + .filter(containsPythonCall(_)) .map(_.accept(inputRefRewriter)) val topCalcCondition = RexUtil.composeConjunction(rexBuilder, pythonFilters) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromJoinRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromJoinRule.scala index 08efa0cec8..f72a5a5018 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromJoinRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromJoinRule.scala @@ -43,7 +43,7 @@ class SplitPythonConditionFromJoinRule extends RelOptRule( val join: FlinkLogicalJoin = call.rel(0).asInstanceOf[FlinkLogicalJoin] val joinType: JoinRelType = join.getJoinType // matches if it is inner join and it contains Python functions in condition - joinType == JoinRelType.INNER && Option(join.getCondition).exists(containsPythonCall) + joinType == JoinRelType.INNER && Option(join.getCondition).exists(containsPythonCall(_)) } override def onMatch(call: RelOptRuleCall): Unit = { @@ -51,7 +51,7 @@ class SplitPythonConditionFromJoinRule extends RelOptRule( val rexBuilder = join.getCluster.getRexBuilder val joinFilters = RelOptUtil.conjunctions(join.getCondition) - val pythonFilters = joinFilters.filter(containsPythonCall) + val pythonFilters = joinFilters.filter(containsPythonCall(_)) val remainingFilters = joinFilters.filter(!containsPythonCall(_)) val newJoinCondition = RexUtil.composeConjunction(rexBuilder, remainingFilters) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecCalcRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecCalcRule.scala index 9163de31a0..20e47c69eb 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecCalcRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecCalcRule.scala @@ -41,7 +41,7 @@ class BatchExecCalcRule override def matches(call: RelOptRuleCall): Boolean = { val calc: FlinkLogicalCalc = call.rel(0).asInstanceOf[FlinkLogicalCalc] val program = calc.getProgram - !program.getExprList.asScala.exists(containsPythonCall) + !program.getExprList.asScala.exists(containsPythonCall(_)) } def convert(rel: RelNode): RelNode = { diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecPythonCalcRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecPythonCalcRule.scala index 874d44b3e3..4666e161aa 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecPythonCalcRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecPythonCalcRule.scala @@ -41,7 +41,7 @@ class BatchExecPythonCalcRule override def matches(call: RelOptRuleCall): Boolean = { val calc: FlinkLogicalCalc = call.rel(0).asInstanceOf[FlinkLogicalCalc] val program = calc.getProgram - program.getExprList.asScala.exists(containsPythonCall) + program.getExprList.asScala.exists(containsPythonCall(_)) } def convert(rel: RelNode): RelNode = { diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecCalcRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecCalcRule.scala index ed9dac733f..16b8e26e33 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecCalcRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecCalcRule.scala @@ -41,7 +41,7 @@ class StreamExecCalcRule override def matches(call: RelOptRuleCall): Boolean = { val calc: FlinkLogicalCalc = call.rel(0).asInstanceOf[FlinkLogicalCalc] val program = calc.getProgram - !program.getExprList.asScala.exists(containsPythonCall) + !program.getExprList.asScala.exists(containsPythonCall(_)) } def convert(rel: RelNode): RelNode = { diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecPythonCalcRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecPythonCalcRule.scala index bae9b99cf5..c99c3b0ff0 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecPythonCalcRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecPythonCalcRule.scala @@ -41,7 +41,7 @@ class StreamExecPythonCalcRule override def matches(call: RelOptRuleCall): Boolean = { val calc: FlinkLogicalCalc = call.rel(0).asInstanceOf[FlinkLogicalCalc] val program = calc.getProgram - program.getExprList.asScala.exists(containsPythonCall) + program.getExprList.asScala.exists(containsPythonCall(_)) } def convert(rel: RelNode): RelNode = { diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/PythonUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/PythonUtil.scala index 877d2a70c9..d924e0e9a8 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/PythonUtil.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/PythonUtil.scala @@ -19,7 +19,8 @@ package org.apache.flink.table.planner.plan.utils import org.apache.calcite.rex.{RexCall, RexNode} -import org.apache.flink.table.functions.python.PythonFunction +import org.apache.flink.table.functions.UserDefinedFunction +import org.apache.flink.table.functions.python.{PythonFunction, PythonFunctionKind} import org.apache.flink.table.planner.functions.utils.{ScalarSqlFunction, TableSqlFunction} import scala.collection.JavaConversions._ @@ -27,12 +28,16 @@ import scala.collection.JavaConversions._ object PythonUtil { /** - * Checks whether it contains Python function call in the specified node. + * Checks whether it contains the specified kind of Python function call in the specified node. + * If the parameter pythonFunctionKind is null, it will return true for any kind of Python + * function. * * @param node the RexNode to check + * @param pythonFunctionKind the kind of the python function * @return true if it contains the Python function call in the specified node. */ - def containsPythonCall(node: RexNode): Boolean = node.accept(new FunctionFinder(true, true)) + def containsPythonCall(node: RexNode, pythonFunctionKind: PythonFunctionKind = null): Boolean = + node.accept(new FunctionFinder(true, Option(pythonFunctionKind), true)) /** * Checks whether it contains non-Python function call in the specified node. @@ -40,15 +45,20 @@ object PythonUtil { * @param node the RexNode to check * @return true if it contains the non-Python function call in the specified node. */ - def containsNonPythonCall(node: RexNode): Boolean = node.accept(new FunctionFinder(false, true)) + def containsNonPythonCall(node: RexNode): Boolean = + node.accept(new FunctionFinder(false, None, true)) /** - * Checks whether the specified node is a Python function call. + * Checks whether the specified node is the specified kind of Python function call. + * If the parameter pythonFunctionKind is null, it will return true for any kind of Python + * function. * * @param node the RexNode to check + * @param pythonFunctionKind the kind of the python function * @return true if the specified node is a Python function call. */ - def isPythonCall(node: RexNode): Boolean = node.accept(new FunctionFinder(true, false)) + def isPythonCall(node: RexNode, pythonFunctionKind: PythonFunctionKind = null): Boolean = + node.accept(new FunctionFinder(true, Option(pythonFunctionKind), false)) /** * Checks whether the specified node is a non-Python function call. @@ -56,27 +66,39 @@ object PythonUtil { * @param node the RexNode to check * @return true if the specified node is a non-Python function call. */ - def isNonPythonCall(node: RexNode): Boolean = node.accept(new FunctionFinder(false, false)) + def isNonPythonCall(node: RexNode): Boolean = node.accept(new FunctionFinder(false, None, false)) /** * Checks whether it contains the specified kind of function in a RexNode. * * @param findPythonFunction true to find python function, false to find non-python function + * @param pythonFunctionKind the kind of the python function * @param recursive whether check the inputs */ - private class FunctionFinder(findPythonFunction: Boolean, recursive: Boolean) + private class FunctionFinder( + findPythonFunction: Boolean, + pythonFunctionKind: Option[PythonFunctionKind], + recursive: Boolean) extends RexDefaultVisitor[Boolean] { /** - * Checks whether the specified rexCall is python function call. + * Checks whether the specified rexCall is a python function call of the specified kind. * * @param rexCall the RexCall to check. - * @return true if it is python function call. + * @return true if it is python function call of the specified kind. */ - private def isPythonRexCall(rexCall: RexCall): Boolean = rexCall.getOperator match { - case sfc: ScalarSqlFunction => sfc.scalarFunction.isInstanceOf[PythonFunction] - case tfc: TableSqlFunction => tfc.udtf.isInstanceOf[PythonFunction] - case _ => false + private def isPythonRexCall(rexCall: RexCall): Boolean = + rexCall.getOperator match { + case sfc: ScalarSqlFunction => isPythonFunction(sfc.scalarFunction) + case tfc: TableSqlFunction => isPythonFunction(tfc.udtf) + case _ => false + } + + private def isPythonFunction(userDefinedFunction: UserDefinedFunction): Boolean = { + userDefinedFunction.isInstanceOf[PythonFunction] && + (pythonFunctionKind.isEmpty || + userDefinedFunction.asInstanceOf[PythonFunction].getPythonFunctionKind == + pythonFunctionKind.get) } override def visitCall(call: RexCall): Boolean = { diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/utils/python/PythonTableUtils.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/utils/python/PythonTableUtils.scala index 88963f2570..e427923645 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/utils/python/PythonTableUtils.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/utils/python/PythonTableUtils.scala @@ -32,7 +32,7 @@ import org.apache.flink.api.java.typeutils.{MapTypeInfo, ObjectArrayTypeInfo, Ro import org.apache.flink.core.io.InputSplit import org.apache.flink.table.api.{TableConfig, TableSchema, Types} import org.apache.flink.table.functions.{ScalarFunction, TableFunction} -import org.apache.flink.table.functions.python.PythonEnv +import org.apache.flink.table.functions.python.{PythonEnv, PythonFunctionKind} import org.apache.flink.table.planner.codegen.{CodeGeneratorContext, PythonFunctionCodeGenerator} import org.apache.flink.table.sources.InputFormatTableSource import org.apache.flink.types.Row @@ -48,6 +48,7 @@ object PythonTableUtils { * @param serializedScalarFunction serialized Python scalar function * @param inputTypes input data types * @param resultType expected result type + * @param pythonFunctionKind the kind of the Python function * @param deterministic the determinism of the function's results * @param pythonEnv the Python execution environment * @return A generated Java ScalarFunction representation for the specified Python ScalarFunction @@ -58,6 +59,7 @@ object PythonTableUtils { serializedScalarFunction: Array[Byte], inputTypes: Array[TypeInformation[_]], resultType: TypeInformation[_], + pythonFunctionKind: PythonFunctionKind, deterministic: Boolean, pythonEnv: PythonEnv): ScalarFunction = PythonFunctionCodeGenerator.generateScalarFunction( @@ -66,6 +68,7 @@ object PythonTableUtils { serializedScalarFunction, inputTypes, resultType, + pythonFunctionKind, deterministic, pythonEnv) diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaUserDefinedScalarFunctions.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaUserDefinedScalarFunctions.java index 75e93affd8..987f4270c1 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaUserDefinedScalarFunctions.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaUserDefinedScalarFunctions.java @@ -28,6 +28,7 @@ import org.apache.flink.table.functions.FunctionContext; import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.table.functions.python.PythonEnv; import org.apache.flink.table.functions.python.PythonFunction; +import org.apache.flink.table.functions.python.PythonFunctionKind; import java.sql.Timestamp; import java.util.Arrays; @@ -273,4 +274,32 @@ public class JavaUserDefinedScalarFunctions { return null; } } + + /** + * Test for Pandas Python Scalar Function. + */ + public static class PandasScalarFunction extends PythonScalarFunction { + public PandasScalarFunction(String name) { + super(name); + } + + @Override + public PythonFunctionKind getPythonFunctionKind() { + return PythonFunctionKind.PANDAS; + } + } + + /** + * Test for Pandas Python Scalar Function. + */ + public static class BooleanPandasScalarFunction extends BooleanPythonScalarFunction { + public BooleanPandasScalarFunction(String name) { + super(name); + } + + @Override + public PythonFunctionKind getPythonFunctionKind() { + return PythonFunctionKind.PANDAS; + } + } } diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRuleTest.xml index dac9ca4ec1..485550f901 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRuleTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRuleTest.xml @@ -16,17 +16,34 @@ See the License for the specific language governing permissions and limitations under the License. --> + + + + + + + + + + + - - - - - - - + - - + - - - + - - + - + + + + + + + + + + + - - - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 0]]> + + + (pandasFunc2($0, $2), 0)]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) +]]> + + + (f0, 0)]) + +- FlinkLogicalCalc(select=[a, b, c, pandasFunc2(a, c) AS f0]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +]]> + + + + + + + + + + + + + + + + + + + + + + - - - - - - - - - 0]]> - (pyFunc2($0, $2), 0)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) ]]> - - diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ExpressionReductionRulesTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ExpressionReductionRulesTest.scala index 13cf899182..9a76699cae 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ExpressionReductionRulesTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ExpressionReductionRulesTest.scala @@ -20,8 +20,8 @@ package org.apache.flink.table.planner.plan.rules.logical import org.apache.flink.api.scala._ import org.apache.flink.table.api.scala._ -import org.apache.flink.table.functions.python.{PythonEnv, PythonFunction} import org.apache.flink.table.functions.ScalarFunction +import org.apache.flink.table.functions.python.{PythonEnv, PythonFunction} import org.apache.flink.table.planner.expressions.utils.{Func1, RichFunc1} import org.apache.flink.table.planner.utils.TableTestBase import org.junit.Test diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRuleTest.scala index d54db3bf64..3a9890efd9 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRuleTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRuleTest.scala @@ -24,7 +24,7 @@ import org.apache.flink.table.api.scala._ import org.apache.flink.table.planner.plan.nodes.FlinkConventions import org.apache.flink.table.planner.plan.optimize.program._ import org.apache.flink.table.planner.plan.rules.{FlinkBatchRuleSets, FlinkStreamRuleSets} -import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.{BooleanPythonScalarFunction, PythonScalarFunction} +import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.{BooleanPandasScalarFunction, BooleanPythonScalarFunction, PandasScalarFunction, PythonScalarFunction} import org.apache.flink.table.planner.utils.TableTestBase import org.junit.{Before, Test} @@ -58,6 +58,10 @@ class PythonCalcSplitRuleTest extends TableTestBase { util.addFunction("pyFunc2", new PythonScalarFunction("pyFunc2")) util.addFunction("pyFunc3", new PythonScalarFunction("pyFunc3")) util.addFunction("pyFunc4", new BooleanPythonScalarFunction("pyFunc4")) + util.addFunction("pandasFunc1", new PandasScalarFunction("pandasFunc1")) + util.addFunction("pandasFunc2", new PandasScalarFunction("pandasFunc2")) + util.addFunction("pandasFunc3", new PandasScalarFunction("pandasFunc3")) + util.addFunction("pandasFunc4", new BooleanPandasScalarFunction("pandasFunc4")) } @Test @@ -120,4 +124,58 @@ class PythonCalcSplitRuleTest extends TableTestBase { val sqlQuery = "SELECT a, pyFunc1(a, c), b FROM MyTable" util.verifyPlan(sqlQuery) } + + @Test + def testPandasFunctionAsInputOfJavaFunction(): Unit = { + val sqlQuery = "SELECT pandasFunc1(a, b) + 1 FROM MyTable" + util.verifyPlan(sqlQuery) + } + + @Test + def testPandasFunctionMixedWithJavaFunction(): Unit = { + val sqlQuery = "SELECT pandasFunc1(a, b), c + 1 FROM MyTable" + util.verifyPlan(sqlQuery) + } + + @Test + def testPandasFunctionMixedWithJavaFunctionInWhereClause(): Unit = { + val sqlQuery = "SELECT pandasFunc1(a, b), c + 1 FROM MyTable WHERE pandasFunc2(a, c) > 0" + util.verifyPlan(sqlQuery) + } + + @Test + def testPandasFunctionInWhereClause(): Unit = { + val sqlQuery = "SELECT pandasFunc1(a, b) FROM MyTable WHERE pandasFunc4(a, c)" + util.verifyPlan(sqlQuery) + } + + @Test + def testChainingPandasFunction(): Unit = { + val sqlQuery = "SELECT pandasFunc3(pandasFunc2(a + pandasFunc1(a, c), b), c) FROM MyTable" + util.verifyPlan(sqlQuery) + } + + @Test + def testOnlyOnePandasFunction(): Unit = { + val sqlQuery = "SELECT pandasFunc1(a, b) FROM MyTable" + util.verifyPlan(sqlQuery) + } + + @Test + def testOnlyOnePandasFunctionInWhereClause(): Unit = { + val sqlQuery = "SELECT a, b FROM MyTable WHERE pandasFunc4(a, c)" + util.verifyPlan(sqlQuery) + } + + @Test + def testPandasFunctionMixedWithGeneralPythonFunction(): Unit = { + val sqlQuery = "SELECT pandasFunc1(a, b), pyFunc1(a, c) + 1 FROM MyTable" + util.verifyPlan(sqlQuery) + } + + @Test + def testPandasFunctionNotChainingWithGeneralPythonFunction(): Unit = { + val sqlQuery = "SELECT pyFunc1(a, pandasFunc1(a, b)) + 1 FROM MyTable" + util.verifyPlan(sqlQuery) + } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonPythonBase.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonPythonBase.scala index b9b98c5c61..784281576e 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonPythonBase.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonPythonBase.scala @@ -72,7 +72,8 @@ trait CommonPythonBase { // the serialized Python function, the Python env, etc val pythonFunction = new SimplePythonFunction( func.asInstanceOf[PythonFunction].getSerializedPythonFunction, - func.asInstanceOf[PythonFunction].getPythonEnv) + func.asInstanceOf[PythonFunction].getPythonEnv, + func.asInstanceOf[PythonFunction].getPythonFunctionKind) new PythonFunctionInfo(pythonFunction, inputs.toArray) } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/utils/JavaUserDefinedScalarFunctions.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/utils/JavaUserDefinedScalarFunctions.java index abc6095664..641e700b23 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/utils/JavaUserDefinedScalarFunctions.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/utils/JavaUserDefinedScalarFunctions.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.table.functions.python.PythonEnv; import org.apache.flink.table.functions.python.PythonFunction; +import org.apache.flink.table.functions.python.PythonFunctionKind; import java.util.Arrays; @@ -117,6 +118,11 @@ public class JavaUserDefinedScalarFunctions { public PythonEnv getPythonEnv() { return null; } + + @Override + public PythonFunctionKind getPythonFunctionKind() { + return PythonFunctionKind.GENERAL; + } } /** @@ -152,5 +158,10 @@ public class JavaUserDefinedScalarFunctions { public PythonEnv getPythonEnv() { return null; } + + @Override + public PythonFunctionKind getPythonFunctionKind() { + return PythonFunctionKind.GENERAL; + } } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/ExpressionReductionRulesTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/ExpressionReductionRulesTest.scala index 080da5d7c4..64e595f2b9 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/ExpressionReductionRulesTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/ExpressionReductionRulesTest.scala @@ -23,7 +23,7 @@ import org.apache.flink.table.api.Types import org.apache.flink.table.api.scala._ import org.apache.flink.table.expressions.utils.{Func1, RichFunc1} import org.apache.flink.table.functions.ScalarFunction -import org.apache.flink.table.functions.python.{PythonEnv, PythonFunction} +import org.apache.flink.table.functions.python.{PythonEnv, PythonFunction, PythonFunctionKind} import org.apache.flink.table.utils.TableTestBase import org.apache.flink.table.utils.TableTestUtil._ @@ -581,4 +581,6 @@ object DeterministicPythonFunc extends ScalarFunction with PythonFunction { override def getSerializedPythonFunction: Array[Byte] = null override def getPythonEnv: PythonEnv = null + + override def getPythonFunctionKind: PythonFunctionKind = PythonFunctionKind.GENERAL } -- Gitee From 3ebc162bae8491401509bd58a310748e11afe28f Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Fri, 28 Feb 2020 20:57:32 +0800 Subject: [PATCH 092/885] [FLINK-16337][python][table-planner] Add support of vectorized Python UDF in old planner This closes #11252. --- flink-python/pyflink/table/udf.py | 1 + .../DataStreamPythonCorrelateRule.java | 4 +- .../logical/PythonCorrelateSplitRule.java | 4 +- .../codegen/PythonFunctionCodeGenerator.scala | 12 +- .../table/plan/nodes/CommonPythonCalc.scala | 49 +++- .../nodes/datastream/DataStreamMatch.scala | 4 +- .../datastream/DataStreamPythonCalc.scala | 34 --- .../table/plan/rules/FlinkRuleSets.scala | 1 + .../plan/rules/dataSet/DataSetCalcRule.scala | 2 +- .../rules/dataSet/DataSetPythonCalcRule.scala | 2 +- .../rules/datastream/DataStreamCalcRule.scala | 2 +- .../datastream/DataStreamPythonCalcRule.scala | 2 +- .../rules/logical/PythonCalcSplitRule.scala | 78 ++++-- ...plitPythonConditionFromCorrelateRule.scala | 4 +- .../SplitPythonConditionFromJoinRule.scala | 4 +- .../flink/table/plan/util/PythonUtil.scala | 51 ++-- .../table/util/python/PythonTableUtils.scala | 5 +- .../utils/JavaUserDefinedScalarFunctions.java | 30 ++- .../plan/ExpressionReductionRulesTest.scala | 2 - .../table/plan/PythonCalcSplitRuleTest.scala | 222 +++++++++++++++++- 20 files changed, 417 insertions(+), 96 deletions(-) diff --git a/flink-python/pyflink/table/udf.py b/flink-python/pyflink/table/udf.py index c130563086..f508b3d2fa 100644 --- a/flink-python/pyflink/table/udf.py +++ b/flink-python/pyflink/table/udf.py @@ -220,6 +220,7 @@ class UserDefinedScalarFunctionWrapper(UserDefinedFunctionWrapper): bytearray(serialized_func), j_input_types, j_result_type, + j_function_kind, self._deterministic, _get_python_env()) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/plan/rules/datastream/DataStreamPythonCorrelateRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/plan/rules/datastream/DataStreamPythonCorrelateRule.java index 0d30f1aeb8..f58c5bb8ab 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/plan/rules/datastream/DataStreamPythonCorrelateRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/plan/rules/datastream/DataStreamPythonCorrelateRule.java @@ -59,12 +59,12 @@ public class DataStreamPythonCorrelateRule extends ConverterRule { if (right instanceof FlinkLogicalTableFunctionScan) { // right node is a python table function FlinkLogicalTableFunctionScan scan = (FlinkLogicalTableFunctionScan) right; - return PythonUtil.isPythonCall(scan.getCall()); + return PythonUtil.isPythonCall(scan.getCall(), null); } else if (right instanceof FlinkLogicalCalc) { // a filter is pushed above the table function FlinkLogicalCalc calc = (FlinkLogicalCalc) right; Option scan = CorrelateUtil.getTableFunctionScan(calc); - return scan.isDefined() && PythonUtil.isPythonCall(scan.get().getCall()); + return scan.isDefined() && PythonUtil.isPythonCall(scan.get().getCall(), null); } return false; } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/plan/rules/logical/PythonCorrelateSplitRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/plan/rules/logical/PythonCorrelateSplitRule.java index b498d7d3fd..8357a926c1 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/plan/rules/logical/PythonCorrelateSplitRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/plan/rules/logical/PythonCorrelateSplitRule.java @@ -99,7 +99,7 @@ public class PythonCorrelateSplitRule extends RelOptRule { } RexNode rexNode = pythonTableFuncScan.getCall(); if (rexNode instanceof RexCall) { - return PythonUtil.isPythonCall(rexNode) && PythonUtil.containsNonPythonCall(rexNode); + return PythonUtil.isPythonCall(rexNode, null) && PythonUtil.containsNonPythonCall(rexNode); } return false; } @@ -203,7 +203,7 @@ public class PythonCorrelateSplitRule extends RelOptRule { ScalarFunctionSplitter splitter = new ScalarFunctionSplitter( primitiveLeftFieldCount, extractedJavaRexCalls, - false + PythonUtil::isNonPythonCall ); RelNode rightNewInput; diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/PythonFunctionCodeGenerator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/PythonFunctionCodeGenerator.scala index 198260ed46..31952219ad 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/PythonFunctionCodeGenerator.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/PythonFunctionCodeGenerator.scala @@ -22,7 +22,7 @@ import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.table.codegen.CodeGenUtils.{newName, primitiveDefaultValue, primitiveTypeTermForTypeInfo} import org.apache.flink.table.codegen.Indenter.toISC import org.apache.flink.table.functions.{ScalarFunction, TableFunction, UserDefinedFunction} -import org.apache.flink.table.functions.python.{PythonEnv, PythonFunction} +import org.apache.flink.table.functions.python.{PythonEnv, PythonFunction, PythonFunctionKind} import org.apache.flink.table.utils.EncodingUtils import org.apache.flink.types.Row @@ -42,6 +42,7 @@ object PythonFunctionCodeGenerator extends Compiler[UserDefinedFunction] { * @param serializedScalarFunction serialized Python scalar function * @param inputTypes input data types * @param resultType expected result type + * @param pythonFunctionKind the kind of the Python function * @param deterministic the determinism of the function's results * @param pythonEnv the Python execution environment * @return instance of generated ScalarFunction @@ -51,6 +52,7 @@ object PythonFunctionCodeGenerator extends Compiler[UserDefinedFunction] { serializedScalarFunction: Array[Byte], inputTypes: Array[TypeInformation[_]], resultType: TypeInformation[_], + pythonFunctionKind: PythonFunctionKind, deterministic: Boolean, pythonEnv: PythonEnv): ScalarFunction = { val funcName = newName(PYTHON_SCALAR_FUNCTION_NAME) @@ -73,6 +75,8 @@ object PythonFunctionCodeGenerator extends Compiler[UserDefinedFunction] { val encodedScalarFunction = EncodingUtils.encodeBytesToBase64(serializedScalarFunction) val encodedPythonEnv = EncodingUtils.encodeObjectToString(pythonEnv) val pythonEnvTypeTerm = classOf[PythonEnv].getCanonicalName + val encodedPythonFunctionKind = EncodingUtils.encodeObjectToString(pythonFunctionKind) + val pythonFunctionKindTypeTerm = classOf[PythonFunctionKind].getCanonicalName val funcCode = j""" |public class $funcName extends ${classOf[ScalarFunction].getCanonicalName} @@ -107,6 +111,12 @@ object PythonFunctionCodeGenerator extends Compiler[UserDefinedFunction] { | } | | @Override + | public $pythonFunctionKindTypeTerm getPythonFunctionKind() { + | return ($pythonFunctionKindTypeTerm) $encodingUtilsTypeTerm.decodeStringToObject( + | "$encodedPythonFunctionKind", $pythonFunctionKindTypeTerm.class); + | } + | + | @Override | public boolean isDeterministic() { | return $deterministic; | } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonPythonCalc.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonPythonCalc.scala index 28a40fb62a..44d318e6df 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonPythonCalc.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonPythonCalc.scala @@ -18,9 +18,16 @@ package org.apache.flink.table.plan.nodes import org.apache.calcite.rex.{RexCall, RexInputRef, RexNode, RexProgram} -import org.apache.flink.table.functions.python.PythonFunctionInfo +import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.operators.OneInputStreamOperator +import org.apache.flink.table.functions.python.{PythonFunctionInfo, PythonFunctionKind} +import org.apache.flink.table.plan.nodes.CommonPythonCalc.{ARROW_PYTHON_SCALAR_FUNCTION_OPERATOR_NAME, PYTHON_SCALAR_FUNCTION_OPERATOR_NAME} +import org.apache.flink.table.plan.util.PythonUtil.containsPythonCall +import org.apache.flink.table.runtime.types.CRow +import org.apache.flink.table.types.logical.RowType import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.mutable trait CommonPythonCalc extends CommonPythonBase { @@ -50,4 +57,44 @@ trait CommonPythonCalc extends CommonPythonBase { .collect { case inputRef: RexInputRef => inputRef.getIndex } .toArray } + + private[flink] def getPythonScalarFunctionOperator( + config: Configuration, + inputRowType: RowType, + outputRowType: RowType, + calcProgram: RexProgram) = { + val clazz = if (calcProgram.getExprList.asScala.exists( + containsPythonCall(_, PythonFunctionKind.PANDAS))) { + loadClass(ARROW_PYTHON_SCALAR_FUNCTION_OPERATOR_NAME) + } else { + loadClass(PYTHON_SCALAR_FUNCTION_OPERATOR_NAME) + } + val ctor = clazz.getConstructor( + classOf[Configuration], + classOf[Array[PythonFunctionInfo]], + classOf[RowType], + classOf[RowType], + classOf[Array[Int]], + classOf[Array[Int]]) + val (udfInputOffsets, pythonFunctionInfos) = + extractPythonScalarFunctionInfos(getPythonRexCalls(calcProgram)) + ctor.newInstance( + config, + pythonFunctionInfos, + inputRowType, + outputRowType, + udfInputOffsets, + getForwardedFields(calcProgram)) + .asInstanceOf[OneInputStreamOperator[CRow, CRow]] + } +} + +object CommonPythonCalc { + val PYTHON_SCALAR_FUNCTION_OPERATOR_NAME = + "org.apache.flink.table.runtime.operators.python.scalar.PythonScalarFunctionOperator" + + val ARROW_PYTHON_SCALAR_FUNCTION_OPERATOR_NAME = + "org.apache.flink.table.runtime.operators.python.scalar.arrow." + + "ArrowPythonScalarFunctionOperator" } + diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamMatch.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamMatch.scala index 67c8d6b2ce..167e54e69b 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamMatch.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamMatch.scala @@ -77,8 +77,8 @@ class DataStreamMatch( with CommonMatchRecognize with DataStreamRel { - if (logicalMatch.measures.values().exists(containsPythonCall) || - logicalMatch.patternDefinitions.values().exists(containsPythonCall)) { + if (logicalMatch.measures.values().exists(containsPythonCall(_)) || + logicalMatch.patternDefinitions.values().exists(containsPythonCall(_))) { throw new TableException("Python Function can not be used in MATCH_RECOGNIZE for now.") } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamPythonCalc.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamPythonCalc.scala index 7eebebc986..37c0eef715 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamPythonCalc.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamPythonCalc.scala @@ -23,14 +23,10 @@ import org.apache.calcite.rel.RelNode import org.apache.calcite.rel.core.Calc import org.apache.calcite.rex.RexProgram import org.apache.flink.api.java.typeutils.RowTypeInfo -import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.datastream.DataStream -import org.apache.flink.streaming.api.operators.OneInputStreamOperator import org.apache.flink.table.api.StreamQueryConfig import org.apache.flink.table.calcite.FlinkTypeFactory -import org.apache.flink.table.functions.python.PythonFunctionInfo import org.apache.flink.table.plan.nodes.CommonPythonCalc -import org.apache.flink.table.plan.nodes.datastream.DataStreamPythonCalc.PYTHON_SCALAR_FUNCTION_OPERATOR_NAME import org.apache.flink.table.plan.schema.RowSchema import org.apache.flink.table.planner.StreamPlanner import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} @@ -101,34 +97,4 @@ class DataStreamPythonCalc( // keep parallelism to ensure order of accumulate and retract messages .setParallelism(inputParallelism) } - - private[flink] def getPythonScalarFunctionOperator( - config: Configuration, - inputRowType: RowType, - outputRowType: RowType, - calcProgram: RexProgram) = { - val clazz = loadClass(PYTHON_SCALAR_FUNCTION_OPERATOR_NAME) - val ctor = clazz.getConstructor( - classOf[Configuration], - classOf[Array[PythonFunctionInfo]], - classOf[RowType], - classOf[RowType], - classOf[Array[Int]], - classOf[Array[Int]]) - val (udfInputOffsets, pythonFunctionInfos) = - extractPythonScalarFunctionInfos(getPythonRexCalls(calcProgram)) - ctor.newInstance( - config, - pythonFunctionInfos, - inputRowType, - outputRowType, - udfInputOffsets, - getForwardedFields(calcProgram)) - .asInstanceOf[OneInputStreamOperator[CRow, CRow]] - } -} - -object DataStreamPythonCalc { - val PYTHON_SCALAR_FUNCTION_OPERATOR_NAME = - "org.apache.flink.table.runtime.operators.python.scalar.PythonScalarFunctionOperator" } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala index 8419e44aa5..7bbfa6a864 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala @@ -157,6 +157,7 @@ object FlinkRuleSets { CalcMergeRule.INSTANCE, PythonCalcSplitRule.SPLIT_CONDITION, PythonCalcSplitRule.SPLIT_PROJECT, + PythonCalcSplitRule.SPLIT_PANDAS_IN_PROJECT, PythonCalcSplitRule.PUSH_CONDITION, PythonCalcSplitRule.REWRITE_PROJECT ) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetCalcRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetCalcRule.scala index 84cd1cf9d8..93a3355879 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetCalcRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetCalcRule.scala @@ -38,7 +38,7 @@ class DataSetCalcRule override def matches(call: RelOptRuleCall): Boolean = { val calc: FlinkLogicalCalc = call.rel(0).asInstanceOf[FlinkLogicalCalc] val program = calc.getProgram - !program.getExprList.asScala.exists(containsPythonCall) + !program.getExprList.asScala.exists(containsPythonCall(_)) } def convert(rel: RelNode): RelNode = { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetPythonCalcRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetPythonCalcRule.scala index b555a960e7..7548ca2ade 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetPythonCalcRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetPythonCalcRule.scala @@ -38,7 +38,7 @@ class DataSetPythonCalcRule override def matches(call: RelOptRuleCall): Boolean = { val calc: FlinkLogicalCalc = call.rel(0).asInstanceOf[FlinkLogicalCalc] val program = calc.getProgram - program.getExprList.asScala.exists(containsPythonCall) + program.getExprList.asScala.exists(containsPythonCall(_)) } def convert(rel: RelNode): RelNode = { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamCalcRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamCalcRule.scala index 30e5ef7128..5e5b19963f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamCalcRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamCalcRule.scala @@ -39,7 +39,7 @@ class DataStreamCalcRule override def matches(call: RelOptRuleCall): Boolean = { val calc: FlinkLogicalCalc = call.rel(0).asInstanceOf[FlinkLogicalCalc] val program = calc.getProgram - !program.getExprList.asScala.exists(containsPythonCall) + !program.getExprList.asScala.exists(containsPythonCall(_)) } def convert(rel: RelNode): RelNode = { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamPythonCalcRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamPythonCalcRule.scala index 6e2bee1687..a9641dfe24 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamPythonCalcRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamPythonCalcRule.scala @@ -39,7 +39,7 @@ class DataStreamPythonCalcRule override def matches(call: RelOptRuleCall): Boolean = { val calc: FlinkLogicalCalc = call.rel(0).asInstanceOf[FlinkLogicalCalc] val program = calc.getProgram - program.getExprList.asScala.exists(containsPythonCall) + program.getExprList.asScala.exists(containsPythonCall(_)) } def convert(rel: RelNode): RelNode = { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/PythonCalcSplitRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/PythonCalcSplitRule.scala index 4c2fad07be..ed810af175 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/PythonCalcSplitRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/PythonCalcSplitRule.scala @@ -18,13 +18,16 @@ package org.apache.flink.table.plan.rules.logical +import java.util.function.Function + import org.apache.calcite.plan.RelOptRule.{any, operand} import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall} import org.apache.calcite.rex.{RexCall, RexInputRef, RexNode, RexProgram} import org.apache.calcite.sql.validate.SqlValidatorUtil import org.apache.flink.table.functions.ScalarFunction +import org.apache.flink.table.functions.python.PythonFunctionKind import org.apache.flink.table.plan.nodes.logical.FlinkLogicalCalc -import org.apache.flink.table.plan.util.PythonUtil.{containsPythonCall, containsNonPythonCall, isPythonCall, isNonPythonCall} +import org.apache.flink.table.plan.util.PythonUtil.{containsNonPythonCall, containsPythonCall, isNonPythonCall, isPythonCall} import org.apache.flink.table.plan.util.{InputRefVisitor, RexDefaultVisitor} import scala.collection.JavaConverters._ @@ -52,7 +55,9 @@ abstract class PythonCalcSplitRuleBase(description: String) val splitter = new ScalarFunctionSplitter( extractedFunctionOffset, extractedRexCalls, - isConvertPythonFunction(program)) + new Function[RexCall, Boolean] { + override def apply(rexCall: RexCall): Boolean = needConvertRexCall(program, rexCall) + }) val (bottomCalcCondition, topCalcCondition, topCalcProjects) = split(program, splitter) val accessedFields = extractRefInputFields( @@ -112,9 +117,9 @@ abstract class PythonCalcSplitRuleBase(description: String) } /** - * Returns true if converting Python functions. + * Returns true if need to convert the specified call. */ - def isConvertPythonFunction(program: RexProgram): Boolean + def needConvertRexCall(program: RexProgram, call: RexCall): Boolean /** * Splits the specified [[RexProgram]] using the specified [[ScalarFunctionSplitter]]. @@ -138,10 +143,10 @@ object PythonCalcSplitConditionRule extends PythonCalcSplitRuleBase( // matches if it contains Python functions in condition Option(calc.getProgram.getCondition) - .map(calc.getProgram.expandLocalRef).exists(containsPythonCall) + .map(calc.getProgram.expandLocalRef).exists(containsPythonCall(_)) } - override def isConvertPythonFunction(program: RexProgram): Boolean = true + override def needConvertRexCall(program: RexProgram, call: RexCall): Boolean = isPythonCall(call) override def split(program: RexProgram, splitter: ScalarFunctionSplitter) : (Option[RexNode], Option[RexNode], Seq[RexNode]) = { @@ -150,12 +155,22 @@ object PythonCalcSplitConditionRule extends PythonCalcSplitRuleBase( } } +abstract class PythonCalcSplitProjectionRuleBase(description: String) + extends PythonCalcSplitRuleBase(description) { + + override def split(program: RexProgram, splitter: ScalarFunctionSplitter) + : (Option[RexNode], Option[RexNode], Seq[RexNode]) = { + (Option(program.getCondition).map(program.expandLocalRef), None, + program.getProjectList.map(program.expandLocalRef(_).accept(splitter))) + } +} + /** * Rule that splits [[FlinkLogicalCalc]]s which contain both Java functions and Python functions * in the projection into multiple [[FlinkLogicalCalc]]s. After this rule is applied, it will * only contain Python functions or Java functions in the projection of each [[FlinkLogicalCalc]]. */ -object PythonCalcSplitProjectionRule extends PythonCalcSplitRuleBase( +object PythonCalcSplitProjectionRule extends PythonCalcSplitProjectionRuleBase( "PythonCalcSplitProjectionRule") { override def matches(call: RelOptRuleCall): Boolean = { @@ -163,17 +178,36 @@ object PythonCalcSplitProjectionRule extends PythonCalcSplitRuleBase( val projects = calc.getProgram.getProjectList.map(calc.getProgram.expandLocalRef) // matches if it contains both Python functions and Java functions in the projection - projects.exists(containsPythonCall) && projects.exists(containsNonPythonCall) + projects.exists(containsPythonCall(_)) && projects.exists(containsNonPythonCall) } - override def isConvertPythonFunction(program: RexProgram): Boolean = { - program.getProjectList.map(program.expandLocalRef).exists(isNonPythonCall) + override def needConvertRexCall(program: RexProgram, call: RexCall): Boolean = { + program.getProjectList.map(program.expandLocalRef).exists(isNonPythonCall) == isPythonCall(call) } +} - override def split(program: RexProgram, splitter: ScalarFunctionSplitter) - : (Option[RexNode], Option[RexNode], Seq[RexNode]) = { - (Option(program.getCondition).map(program.expandLocalRef), None, - program.getProjectList.map(program.expandLocalRef(_).accept(splitter))) +/** + * Rule that splits [[FlinkLogicalCalc]]s which contain both general Python functions and + * pandas Python functions in the projection into multiple [[FlinkLogicalCalc]]s. After + * this rule is applied, it will only contain general Python functions or pandas Python + * functions in the projection of each [[FlinkLogicalCalc]]. + */ +object PythonCalcSplitPandasInProjectionRule extends PythonCalcSplitProjectionRuleBase( + "PythonCalcSplitPandasInProjectionRule") { + + override def matches(call: RelOptRuleCall): Boolean = { + val calc: FlinkLogicalCalc = call.rel(0).asInstanceOf[FlinkLogicalCalc] + val projects = calc.getProgram.getProjectList.map(calc.getProgram.expandLocalRef) + + // matches if it contains both general Python functions and + // pandas Python functions in the projection + projects.exists(containsPythonCall(_, PythonFunctionKind.GENERAL)) && + projects.exists(containsPythonCall(_, PythonFunctionKind.PANDAS)) + } + + override def needConvertRexCall(program: RexProgram, call: RexCall): Boolean = { + program.getProjectList.map(program.expandLocalRef).exists( + isPythonCall(_, PythonFunctionKind.GENERAL)) == isPythonCall(call, PythonFunctionKind.PANDAS) } } @@ -191,10 +225,11 @@ object PythonCalcPushConditionRule extends PythonCalcSplitRuleBase( // matches if all the following conditions hold true: // 1) the condition is not null // 2) it contains Python functions in the projection - calc.getProgram.getCondition != null && projects.exists(containsPythonCall) + calc.getProgram.getCondition != null && projects.exists(containsPythonCall(_)) } - override def isConvertPythonFunction(program: RexProgram): Boolean = false + override def needConvertRexCall(program: RexProgram, call: RexCall): Boolean = + isNonPythonCall(call) override def split(program: RexProgram, splitter: ScalarFunctionSplitter) : (Option[RexNode], Option[RexNode], Seq[RexNode]) = { @@ -221,13 +256,13 @@ object PythonCalcRewriteProjectionRule extends PythonCalcSplitRuleBase( // 1) it contains Python functions in the projection // 2) it contains RexNodes besides RexInputRef and RexCall or // not all the RexCalls lying at the end of the project list - projects.exists(containsPythonCall) && + projects.exists(containsPythonCall(_)) && (projects.exists(expr => !expr.isInstanceOf[RexCall] && !expr.isInstanceOf[RexInputRef]) || projects.indexWhere(_.isInstanceOf[RexCall]) < projects.lastIndexWhere(_.isInstanceOf[RexInputRef])) } - override def isConvertPythonFunction(program: RexProgram): Boolean = true + override def needConvertRexCall(program: RexProgram, call: RexCall): Boolean = isPythonCall(call) override def split(program: RexProgram, splitter: ScalarFunctionSplitter) : (Option[RexNode], Option[RexNode], Seq[RexNode]) = { @@ -238,11 +273,11 @@ object PythonCalcRewriteProjectionRule extends PythonCalcSplitRuleBase( private class ScalarFunctionSplitter( extractedFunctionOffset: Int, extractedRexCalls: mutable.ArrayBuffer[RexCall], - convertPythonFunction: Boolean) + needConvertRexCall: Function[RexCall, Boolean]) extends RexDefaultVisitor[RexNode] { override def visitCall(call: RexCall): RexNode = { - visit(if (isPythonCall(call)) convertPythonFunction else !convertPythonFunction, call) + visit(needConvertRexCall(call), call) } override def visitNode(rexNode: RexNode): RexNode = rexNode @@ -299,10 +334,11 @@ private class ExtractedFunctionInputRewriter( object PythonCalcSplitRule { /** * These rules should be applied sequentially in the order of - * SPLIT_CONDITION, SPLIT_PROJECT, PUSH_CONDITION and REWRITE_PROJECT. + * SPLIT_CONDITION, SPLIT_PROJECT, SPLIT_PANDAS_IN_PROJECT, PUSH_CONDITION and REWRITE_PROJECT. */ val SPLIT_CONDITION: RelOptRule = PythonCalcSplitConditionRule val SPLIT_PROJECT: RelOptRule = PythonCalcSplitProjectionRule + val SPLIT_PANDAS_IN_PROJECT: RelOptRule = PythonCalcSplitPandasInProjectionRule val PUSH_CONDITION: RelOptRule = PythonCalcPushConditionRule val REWRITE_PROJECT: RelOptRule = PythonCalcRewriteProjectionRule } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/SplitPythonConditionFromCorrelateRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/SplitPythonConditionFromCorrelateRule.scala index 35bb7df33b..09a90a68e1 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/SplitPythonConditionFromCorrelateRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/SplitPythonConditionFromCorrelateRule.scala @@ -54,7 +54,7 @@ class SplitPythonConditionFromCorrelateRule joinType == JoinRelType.INNER && Option(mergedCalc.getProgram.getCondition) .map(mergedCalc.getProgram.expandLocalRef) - .exists(containsPythonCall) + .exists(containsPythonCall(_)) } override def onMatch(call: RelOptRuleCall): Unit = { @@ -95,7 +95,7 @@ class SplitPythonConditionFromCorrelateRule correlate.getRowType.getFieldCount - mergedCalc.getRowType.getFieldCount) val pythonFilters = correlateFilters - .filter(containsPythonCall) + .filter(containsPythonCall(_)) .map(_.accept(inputRefRewriter)) val topCalcCondition = RexUtil.composeConjunction(rexBuilder, pythonFilters) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/SplitPythonConditionFromJoinRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/SplitPythonConditionFromJoinRule.scala index 43f971e65c..54bc0b5e9a 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/SplitPythonConditionFromJoinRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/SplitPythonConditionFromJoinRule.scala @@ -43,7 +43,7 @@ class SplitPythonConditionFromJoinRule extends RelOptRule( val join: FlinkLogicalJoin = call.rel(0).asInstanceOf[FlinkLogicalJoin] val joinType: JoinRelType = join.getJoinType // matches if it is inner join and it contains Python functions in condition - joinType == JoinRelType.INNER && Option(join.getCondition).exists(containsPythonCall) + joinType == JoinRelType.INNER && Option(join.getCondition).exists(containsPythonCall(_)) } override def onMatch(call: RelOptRuleCall): Unit = { @@ -51,7 +51,7 @@ class SplitPythonConditionFromJoinRule extends RelOptRule( val rexBuilder = join.getCluster.getRexBuilder val joinFilters = RelOptUtil.conjunctions(join.getCondition) - val pythonFilters = joinFilters.filter(containsPythonCall) + val pythonFilters = joinFilters.filter(containsPythonCall(_)) val remainingFilters = joinFilters.filter(!containsPythonCall(_)) val newJoinCondition = RexUtil.composeConjunction(rexBuilder, remainingFilters) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/util/PythonUtil.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/util/PythonUtil.scala index 1b2ac637df..79bbb84d0e 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/util/PythonUtil.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/util/PythonUtil.scala @@ -15,10 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.table.plan.util import org.apache.calcite.rex.{RexCall, RexNode} -import org.apache.flink.table.functions.python.PythonFunction +import org.apache.flink.table.functions.UserDefinedFunction +import org.apache.flink.table.functions.python.{PythonFunction, PythonFunctionKind} import org.apache.flink.table.functions.utils.{ScalarSqlFunction, TableSqlFunction} import scala.collection.JavaConversions._ @@ -26,12 +28,16 @@ import scala.collection.JavaConversions._ object PythonUtil { /** - * Checks whether it contains Python function call in the specified node. + * Checks whether it contains the specified kind of Python function call in the specified node. + * If the parameter pythonFunctionKind is null, it will return true for any kind of Python + * function. * * @param node the RexNode to check + * @param pythonFunctionKind the kind of the python function * @return true if it contains the Python function call in the specified node. */ - def containsPythonCall(node: RexNode): Boolean = node.accept(new FunctionFinder(true, true)) + def containsPythonCall(node: RexNode, pythonFunctionKind: PythonFunctionKind = null): Boolean = + node.accept(new FunctionFinder(true, Option(pythonFunctionKind), true)) /** * Checks whether it contains non-Python function call in the specified node. @@ -39,15 +45,20 @@ object PythonUtil { * @param node the RexNode to check * @return true if it contains the non-Python function call in the specified node. */ - def containsNonPythonCall(node: RexNode): Boolean = node.accept(new FunctionFinder(false, true)) + def containsNonPythonCall(node: RexNode): Boolean = + node.accept(new FunctionFinder(false, None, true)) /** - * Checks whether the specified node is a Python function call. + * Checks whether the specified node is the specified kind of Python function call. + * If the parameter pythonFunctionKind is null, it will return true for any kind of Python + * function. * * @param node the RexNode to check + * @param pythonFunctionKind the kind of the python function * @return true if the specified node is a Python function call. */ - def isPythonCall(node: RexNode): Boolean = node.accept(new FunctionFinder(true, false)) + def isPythonCall(node: RexNode, pythonFunctionKind: PythonFunctionKind = null): Boolean = + node.accept(new FunctionFinder(true, Option(pythonFunctionKind), false)) /** * Checks whether the specified node is a non-Python function call. @@ -55,27 +66,39 @@ object PythonUtil { * @param node the RexNode to check * @return true if the specified node is a non-Python function call. */ - def isNonPythonCall(node: RexNode): Boolean = node.accept(new FunctionFinder(false, false)) + def isNonPythonCall(node: RexNode): Boolean = node.accept(new FunctionFinder(false, None, false)) /** * Checks whether it contains the specified kind of function in a RexNode. * * @param findPythonFunction true to find python function, false to find non-python function + * @param pythonFunctionKind the kind of the python function * @param recursive whether check the inputs */ - private class FunctionFinder(findPythonFunction: Boolean, recursive: Boolean) + private class FunctionFinder( + findPythonFunction: Boolean, + pythonFunctionKind: Option[PythonFunctionKind], + recursive: Boolean) extends RexDefaultVisitor[Boolean] { /** - * Checks whether the specified rexCall is python function call. + * Checks whether the specified rexCall is a python function call of the specified kind. * * @param rexCall the RexCall to check. - * @return true if it is python function call. + * @return true if it is python function call of the specified kind. */ - private def isPythonRexCall(rexCall: RexCall): Boolean = rexCall.getOperator match { - case sfc: ScalarSqlFunction => sfc.getScalarFunction.isInstanceOf[PythonFunction] - case tfc: TableSqlFunction => tfc.getTableFunction.isInstanceOf[PythonFunction] - case _ => false + private def isPythonRexCall(rexCall: RexCall): Boolean = + rexCall.getOperator match { + case sfc: ScalarSqlFunction => isPythonFunction(sfc.getScalarFunction) + case tfc: TableSqlFunction => isPythonFunction(tfc.getTableFunction) + case _ => false + } + + private def isPythonFunction(userDefinedFunction: UserDefinedFunction): Boolean = { + userDefinedFunction.isInstanceOf[PythonFunction] && + (pythonFunctionKind.isEmpty || + userDefinedFunction.asInstanceOf[PythonFunction].getPythonFunctionKind == + pythonFunctionKind.get) } override def visitCall(call: RexCall): Boolean = { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/util/python/PythonTableUtils.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/util/python/PythonTableUtils.scala index 1cfc1e4fd3..fb451d4192 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/util/python/PythonTableUtils.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/util/python/PythonTableUtils.scala @@ -33,7 +33,7 @@ import org.apache.flink.core.io.InputSplit import org.apache.flink.table.api.{TableSchema, Types} import org.apache.flink.table.codegen.PythonFunctionCodeGenerator import org.apache.flink.table.functions.{ScalarFunction, TableFunction} -import org.apache.flink.table.functions.python.PythonEnv +import org.apache.flink.table.functions.python.{PythonEnv, PythonFunctionKind} import org.apache.flink.table.sources.InputFormatTableSource import org.apache.flink.types.Row @@ -48,6 +48,7 @@ object PythonTableUtils { * @param serializedScalarFunction serialized Python scalar function * @param inputTypes input data types * @param resultType expected result type + * @param pythonFunctionKind the kind of the Python function * @param deterministic the determinism of the function's results * @param pythonEnv the Python execution environment * @return A generated Java ScalarFunction representation for the specified Python ScalarFunction @@ -57,6 +58,7 @@ object PythonTableUtils { serializedScalarFunction: Array[Byte], inputTypes: Array[TypeInformation[_]], resultType: TypeInformation[_], + pythonFunctionKind: PythonFunctionKind, deterministic: Boolean, pythonEnv: PythonEnv): ScalarFunction = PythonFunctionCodeGenerator.generateScalarFunction( @@ -64,6 +66,7 @@ object PythonTableUtils { serializedScalarFunction, inputTypes, resultType, + pythonFunctionKind, deterministic, pythonEnv) diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/utils/JavaUserDefinedScalarFunctions.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/utils/JavaUserDefinedScalarFunctions.java index 641e700b23..7a6fb5ba2a 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/utils/JavaUserDefinedScalarFunctions.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/utils/JavaUserDefinedScalarFunctions.java @@ -118,11 +118,6 @@ public class JavaUserDefinedScalarFunctions { public PythonEnv getPythonEnv() { return null; } - - @Override - public PythonFunctionKind getPythonFunctionKind() { - return PythonFunctionKind.GENERAL; - } } /** @@ -158,10 +153,33 @@ public class JavaUserDefinedScalarFunctions { public PythonEnv getPythonEnv() { return null; } + } + + /** + * Test for Pandas Python Scalar Function. + */ + public static class PandasScalarFunction extends PythonScalarFunction { + public PandasScalarFunction(String name) { + super(name); + } + + @Override + public PythonFunctionKind getPythonFunctionKind() { + return PythonFunctionKind.PANDAS; + } + } + + /** + * Test for Pandas Python Scalar Function. + */ + public static class BooleanPandasScalarFunction extends BooleanPythonScalarFunction { + public BooleanPandasScalarFunction(String name) { + super(name); + } @Override public PythonFunctionKind getPythonFunctionKind() { - return PythonFunctionKind.GENERAL; + return PythonFunctionKind.PANDAS; } } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/ExpressionReductionRulesTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/ExpressionReductionRulesTest.scala index 64e595f2b9..5e0cdba500 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/ExpressionReductionRulesTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/ExpressionReductionRulesTest.scala @@ -581,6 +581,4 @@ object DeterministicPythonFunc extends ScalarFunction with PythonFunction { override def getSerializedPythonFunction: Array[Byte] = null override def getPythonEnv: PythonEnv = null - - override def getPythonFunctionKind: PythonFunctionKind = PythonFunctionKind.GENERAL } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/PythonCalcSplitRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/PythonCalcSplitRuleTest.scala index f89c104c60..d588e79109 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/PythonCalcSplitRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/PythonCalcSplitRuleTest.scala @@ -20,8 +20,8 @@ package org.apache.flink.table.plan import org.apache.flink.api.scala._ import org.apache.flink.table.api.scala._ -import org.apache.flink.table.runtime.utils.JavaUserDefinedScalarFunctions.{BooleanPythonScalarFunction, PythonScalarFunction} -import org.apache.flink.table.utils.TableTestUtil._ +import org.apache.flink.table.runtime.utils.JavaUserDefinedScalarFunctions.{BooleanPandasScalarFunction, BooleanPythonScalarFunction, PandasScalarFunction, PythonScalarFunction} +import org.apache.flink.table.utils.TableTestUtil.{term, _} import org.apache.flink.table.utils.TableTestBase import org.junit.Test @@ -266,4 +266,222 @@ class PythonCalcSplitRuleTest extends TableTestBase { util.verifyTable(resultTable, expected) } + + @Test + def testPandasFunctionAsInputOfJavaFunction(): Unit = { + val util = streamTestUtil() + val table = util.addTable[(Int, Int, Int)]("MyTable", 'a, 'b, 'c) + util.tableEnv.registerFunction("pandasFunc1", new PandasScalarFunction("pandasFunc1")) + + val resultTable = table.select("pandasFunc1(a, b) + 1") + + val expected = unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamPythonCalc", + streamTableNode(table), + term("select", "pandasFunc1(a, b) AS f0") + ), + term("select", "+(f0, 1) AS _c0") + ) + + util.verifyTable(resultTable, expected) + } + + @Test + def testPandasFunctionMixedWithJavaFunction(): Unit = { + val util = streamTestUtil() + val table = util.addTable[(Int, Int, Int)]("MyTable", 'a, 'b, 'c) + util.tableEnv.registerFunction("pandasFunc1", new PandasScalarFunction("pandasFunc1")) + + val resultTable = table.select("pandasFunc1(a, b), c + 1") + + val expected = unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamPythonCalc", + streamTableNode(table), + term("select", "c", "pandasFunc1(a, b) AS f0") + ), + term("select", "f0 AS _c0", "+(c, 1) AS _c1") + ) + + util.verifyTable(resultTable, expected) + } + + @Test + def testPandasFunctionMixedWithJavaFunctionInWhereClause(): Unit = { + val util = streamTestUtil() + val table = util.addTable[(Int, Int, Int)]("MyTable", 'a, 'b, 'c) + util.tableEnv.registerFunction("pandasFunc1", new PandasScalarFunction("pandasFunc1")) + util.tableEnv.registerFunction("pandasFunc2", new PandasScalarFunction("pandasFunc2")) + + val resultTable = table.where("pandasFunc2(a, c) > 0").select("pandasFunc1(a, b), c + 1") + + val expected = unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamPythonCalc", + unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamPythonCalc", + streamTableNode(table), + term("select", "a", "b", "c", "pandasFunc2(a, c) AS f0") + ), + term("select", "c", "a", "b"), + term("where", ">(f0, 0)") + ), + term("select", "c", "pandasFunc1(a, b) AS f0") + ), + term("select", "f0 AS _c0", "+(c, 1) AS _c1") + ) + + util.verifyTable(resultTable, expected) + } + + @Test + def testPandasFunctionInWhereClause(): Unit = { + val util = streamTestUtil() + val table = util.addTable[(Int, Int, Int)]("MyTable", 'a, 'b, 'c) + util.tableEnv.registerFunction("pandasFunc1", new PandasScalarFunction("pandasFunc1")) + util.tableEnv.registerFunction("pandasFunc2", new BooleanPandasScalarFunction("pandasFunc2")) + + val resultTable = table.where("pandasFunc2(a, c)").select("pandasFunc1(a, b)") + + val expected = unaryNode( + "DataStreamPythonCalc", + unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamPythonCalc", + streamTableNode(table), + term("select", "a", "b", "pandasFunc2(a, c) AS f0") + ), + term("select", "a", "b"), + term("where", "f0") + ), + term("select", "pandasFunc1(a, b) AS _c0") + ) + + util.verifyTable(resultTable, expected) + } + + @Test + def testChainingPandasFunction(): Unit = { + val util = streamTestUtil() + val table = util.addTable[(Int, Int, Int)]("MyTable", 'a, 'b, 'c) + util.tableEnv.registerFunction("pandasFunc1", new PandasScalarFunction("pandasFunc1")) + util.tableEnv.registerFunction("pandasFunc2", new PandasScalarFunction("pandasFunc2")) + util.tableEnv.registerFunction("pandasFunc3", new PandasScalarFunction("pandasFunc3")) + + val resultTable = table.select("pandasFunc3(pandasFunc2(a + pandasFunc1(a, c), b), c)") + + val expected = unaryNode( + "DataStreamPythonCalc", + unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamPythonCalc", + streamTableNode(table), + term("select", "b", "c", "a", "pandasFunc1(a, c) AS f0") + ), + term("select", "b", "c", "+(a, f0) AS f0") + ), + term("select", "pandasFunc3(pandasFunc2(f0, b), c) AS _c0") + ) + + util.verifyTable(resultTable, expected) + } + + @Test + def testOnlyOnePandasFunction(): Unit = { + val util = streamTestUtil() + val table = util.addTable[(Int, Int, Int)]("MyTable", 'a, 'b, 'c) + util.tableEnv.registerFunction("pandasFunc1", new PandasScalarFunction("pandasFunc1")) + + val resultTable = table.select("pandasFunc1(a, b)") + + val expected = unaryNode( + "DataStreamPythonCalc", + streamTableNode(table), + term("select", "pandasFunc1(a, b) AS _c0") + ) + + util.verifyTable(resultTable, expected) + } + + @Test + def testOnlyOnePandasFunctionInWhereClause(): Unit = { + val util = streamTestUtil() + val table = util.addTable[(Int, Int, Int)]("MyTable", 'a, 'b, 'c) + util.tableEnv.registerFunction("pandasFunc1", new BooleanPandasScalarFunction("pandasFunc1")) + + val resultTable = table.where("pandasFunc1(a, c)").select("a, b") + + val expected = unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamPythonCalc", + streamTableNode(table), + term("select", "a", "b", "pandasFunc1(a, c) AS f0") + ), + term("select", "a", "b"), + term("where", "f0") + ) + + util.verifyTable(resultTable, expected) + } + + @Test + def testPandasFunctionMixedWithGeneralPythonFunction(): Unit = { + val util = streamTestUtil() + val table = util.addTable[(Int, Int, Int)]("MyTable", 'a, 'b, 'c) + util.tableEnv.registerFunction("pyFunc1", new PythonScalarFunction("pyFunc1")) + util.tableEnv.registerFunction("pandasFunc1", new PandasScalarFunction("pandasFunc1")) + + val resultTable = table.select("pandasFunc1(a, b), pyFunc1(a, c) + 1") + + val expected = unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamPythonCalc", + unaryNode( + "DataStreamPythonCalc", + streamTableNode(table), + term("select", "a", "c", "pandasFunc1(a, b) AS f0") + ), + term("select", "f0", "pyFunc1(a, c) AS f1") + ), + term("select", "f0 AS _c0", "+(f1, 1) AS _c1") + ) + + util.verifyTable(resultTable, expected) + } + + @Test + def testPandasFunctionNotChainingWithGeneralPythonFunction(): Unit = { + val util = streamTestUtil() + val table = util.addTable[(Int, Int, Int)]("MyTable", 'a, 'b, 'c) + util.tableEnv.registerFunction("pyFunc1", new PythonScalarFunction("pyFunc1")) + util.tableEnv.registerFunction("pandasFunc1", new PandasScalarFunction("pandasFunc1")) + + val resultTable = table.select("pyFunc1(a, pandasFunc1(a, b)) + 1") + + val expected = unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamPythonCalc", + unaryNode( + "DataStreamPythonCalc", + streamTableNode(table), + term("select", "a", "pandasFunc1(a, b) AS f0") + ), + term("select", "pyFunc1(a, f0) AS f0") + ), + term("select", "+(f0, 1) AS _c0") + ) + + util.verifyTable(resultTable, expected) + } } -- Gitee From 37578f75e59d61b396194641e8a83d74bfd2c984 Mon Sep 17 00:00:00 2001 From: huangxingbo Date: Thu, 5 Mar 2020 17:40:50 +0800 Subject: [PATCH 093/885] [FLINK-16435][python] Replace since decorator with versionadd to mark the version an API was introduced This closes #11318. --- flink-python/pyflink/__init__.py | 16 --------- flink-python/pyflink/table/catalog.py | 7 ++-- flink-python/pyflink/table/table_config.py | 7 ++-- .../pyflink/table/table_environment.py | 34 ++++++++++++------- 4 files changed, 29 insertions(+), 35 deletions(-) diff --git a/flink-python/pyflink/__init__.py b/flink-python/pyflink/__init__.py index 1fb541b06c..e526d0688d 100644 --- a/flink-python/pyflink/__init__.py +++ b/flink-python/pyflink/__init__.py @@ -24,22 +24,6 @@ if sys.version_info < (3, 5): str(sys.version_info) + '].') -def since(version): - """ - A decorator that annotates a function to append the version the function was added. - """ - import re - indent_p = re.compile(r'\n( +)') - - def deco(f): - original_doc = f.__doc__ or "" - indents = indent_p.findall(original_doc) - indent = ' ' * (min(len(indent) for indent in indents) if indents else 0) - f.__doc__ = original_doc.rstrip() + "\n\n%s.. versionadded:: %s" % (indent, version) - return f - return deco - - def keyword(func): """ A decorator that forces keyword arguments usage and store actual diff --git a/flink-python/pyflink/table/catalog.py b/flink-python/pyflink/table/catalog.py index a3eef3c144..5fcd8593a2 100644 --- a/flink-python/pyflink/table/catalog.py +++ b/flink-python/pyflink/table/catalog.py @@ -18,7 +18,6 @@ from py4j.java_gateway import java_import -from pyflink import since from pyflink.java_gateway import get_gateway from pyflink.table.table_schema import TableSchema @@ -790,21 +789,23 @@ class CatalogFunction(object): else: return None - @since("1.10.0") def is_generic(self): """ Whether or not is the function a flink UDF. :return: Whether is the function a flink UDF. + + .. versionadded:: 1.10.0 """ return self._j_catalog_function.isGeneric() - @since("1.10.0") def get_function_language(self): """ Get the language used for the function definition. :return: the language type of the function definition + + .. versionadded:: 1.10.0 """ return self._j_catalog_function.getFunctionLanguage() diff --git a/flink-python/pyflink/table/table_config.py b/flink-python/pyflink/table/table_config.py index e8cbf47079..72389b41e5 100644 --- a/flink-python/pyflink/table/table_config.py +++ b/flink-python/pyflink/table/table_config.py @@ -18,7 +18,6 @@ from py4j.compat import long -from pyflink import since from pyflink.common import Configuration from pyflink.common.dependency_manager import DependencyManager from pyflink.java_gateway import get_gateway @@ -277,7 +276,6 @@ class TableConfig(object): """ self._j_table_config.setSqlDialect(SqlDialect._to_j_sql_dialect(sql_dialect)) - @since("1.10.0") def set_python_executable(self, python_exec): """ Sets the path of the python interpreter which is used to execute the python udf workers. @@ -312,10 +310,11 @@ class TableConfig(object): :param python_exec: The path of python interpreter. :type python_exec: str + + .. versionadded:: 1.10.0 """ self.get_configuration().set_string(DependencyManager.PYTHON_EXEC, python_exec) - @since("1.10.0") def get_python_executable(self): """ Gets the path of the python interpreter which is used to execute the python udf workers. @@ -323,6 +322,8 @@ class TableConfig(object): :return: The path of the python interpreter which is used to execute the python udf workers. :rtype: str + + .. versionadded:: 1.10.0 """ return self.get_configuration().get_string(DependencyManager.PYTHON_EXEC, None) diff --git a/flink-python/pyflink/table/table_environment.py b/flink-python/pyflink/table/table_environment.py index 6ee05817f2..6035a7752d 100644 --- a/flink-python/pyflink/table/table_environment.py +++ b/flink-python/pyflink/table/table_environment.py @@ -22,7 +22,6 @@ from abc import ABCMeta, abstractmethod from py4j.java_gateway import get_java_class, get_method -from pyflink import since from pyflink.common.dependency_manager import DependencyManager from pyflink.serializers import BatchedSerializer, PickleSerializer from pyflink.table.catalog import Catalog @@ -246,7 +245,6 @@ class TableEnvironment(object): j_table = self._j_tenv.scan(j_table_paths) return Table(j_table) - @since("1.10.0") def from_path(self, path): """ Reads a registered table and returns the resulting :class:`~pyflink.table.Table`. @@ -281,6 +279,7 @@ class TableEnvironment(object): .. seealso:: :func:`use_catalog` .. seealso:: :func:`use_database` + .. versionadded:: 1.10.0 """ return Table(get_method(self._j_tenv, "from")(path)) @@ -319,13 +318,14 @@ class TableEnvironment(object): j_catalog_name_array = self._j_tenv.listCatalogs() return [item for item in j_catalog_name_array] - @since("1.10.0") def list_modules(self): """ Gets the names of all modules registered in this environment. :return: List of module names. :rtype: list[str] + + .. versionadded:: 1.10.0 """ j_module_name_array = self._j_tenv.listModules() return [item for item in j_module_name_array] @@ -360,18 +360,18 @@ class TableEnvironment(object): j_udf_name_array = self._j_tenv.listUserDefinedFunctions() return [item for item in j_udf_name_array] - @since("1.10.0") def list_functions(self): """ Gets the names of all functions in this environment. :return: List of the names of all functions in this environment. :rtype: list[str] + + .. versionadded:: 1.10.0 """ j_function_name_array = self._j_tenv.listFunctions() return [item for item in j_function_name_array] - @since("1.10.0") def list_temporary_tables(self): """ Gets the names of all temporary tables and views available in the current namespace @@ -382,11 +382,11 @@ class TableEnvironment(object): :rtype: list[str] .. seealso:: :func:`list_tables` + .. versionadded:: 1.10.0 """ j_table_name_array = self._j_tenv.listTemporaryTables() return [item for item in j_table_name_array] - @since("1.10.0") def list_temporary_views(self): """ Gets the names of all temporary views available in the current namespace (the current @@ -397,11 +397,11 @@ class TableEnvironment(object): :rtype: list[str] .. seealso:: :func:`list_tables` + .. versionadded:: 1.10.0 """ j_view_name_array = self._j_tenv.listTemporaryViews() return [item for item in j_view_name_array] - @since("1.10.0") def drop_temporary_table(self, table_path): """ Drops a temporary table registered in the given path. @@ -413,10 +413,11 @@ class TableEnvironment(object): :type table_path: str :return: True if a table existed in the given path and was removed. :rtype: bool + + .. versionadded:: 1.10.0 """ return self._j_tenv.dropTemporaryTable(table_path) - @since("1.10.0") def drop_temporary_view(self, view_path): """ Drops a temporary view registered in the given path. @@ -426,6 +427,8 @@ class TableEnvironment(object): :return: True if a view existed in the given path and was removed. :rtype: bool + + .. versionadded:: 1.10.0 """ return self._j_tenv.dropTemporaryView(view_path) @@ -737,7 +740,6 @@ class TableEnvironment(object): .loadClass(function_class_name).newInstance() self._j_tenv.registerFunction(name, java_function) - @since("1.10.0") def register_function(self, name, function): """ Registers a python user-defined function under a unique name. Replaces already existing @@ -765,11 +767,12 @@ class TableEnvironment(object): :type name: str :param function: The python user-defined function to register. :type function: pyflink.table.udf.UserDefinedFunctionWrapper + + .. versionadded:: 1.10.0 """ self._j_tenv.registerFunction(name, function.java_user_defined_function( self._is_blink_planner, self.get_config()._j_table_config)) - @since("1.10.0") def create_temporary_view(self, view_path, table): """ Registers a :class:`~pyflink.table.Table` API object as a temporary view similar to SQL @@ -785,10 +788,11 @@ class TableEnvironment(object): :type view_path: str :param table: The view to register. :type table: pyflink.table.Table + + .. versionadded:: 1.10.0 """ self._j_tenv.createTemporaryView(view_path, table._j_table) - @since("1.10.0") def add_python_file(self, file_path): """ Adds a python dependency which could be python files, python packages or @@ -797,10 +801,11 @@ class TableEnvironment(object): :param file_path: The path of the python dependency. :type file_path: str + + .. versionadded:: 1.10.0 """ self._dependency_manager.add_python_file(file_path) - @since("1.10.0") def set_python_requirements(self, requirements_file_path, requirements_cache_dir=None): """ Specifies a requirements.txt file which defines the third-party dependencies. @@ -834,11 +839,12 @@ class TableEnvironment(object): :param requirements_cache_dir: The path of the local directory which contains the installation packages. :type requirements_cache_dir: str + + .. versionadded:: 1.10.0 """ self._dependency_manager.set_python_requirements(requirements_file_path, requirements_cache_dir) - @since("1.10.0") def add_python_archive(self, archive_path, target_dir=None): """ Adds a python archive file. The file will be extracted to the working directory of @@ -889,6 +895,8 @@ class TableEnvironment(object): :type archive_path: str :param target_dir: Optional, the target dir name that the archive file extracted to. :type target_dir: str + + .. versionadded:: 1.10.0 """ self._dependency_manager.add_python_archive(archive_path, target_dir) -- Gitee From d1f1d40a297dace6942c6d1d49bc4aca264b9c49 Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Thu, 5 Mar 2020 21:55:59 +0100 Subject: [PATCH 094/885] [FLINK-16313] Ignore unstable rocksdb state processing api test --- .../state/api/RocksDBStateBackendReaderKeyedStateITCase.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/RocksDBStateBackendReaderKeyedStateITCase.java b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/RocksDBStateBackendReaderKeyedStateITCase.java index df9fb85c9c..369671662d 100644 --- a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/RocksDBStateBackendReaderKeyedStateITCase.java +++ b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/RocksDBStateBackendReaderKeyedStateITCase.java @@ -22,9 +22,12 @@ import org.apache.flink.contrib.streaming.state.RocksDBStateBackend; import org.apache.flink.runtime.state.StateBackend; import org.apache.flink.runtime.state.memory.MemoryStateBackend; +import org.junit.Ignore; + /** * IT Case for reading state from a RocksDB keyed state backend. */ +@Ignore public class RocksDBStateBackendReaderKeyedStateITCase extends SavepointReaderKeyedStateITCase { @Override protected RocksDBStateBackend getStateBackend() { -- Gitee From ce85bb70b52b43cb0cfa5b3d73f58fc70c33597a Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 6 Mar 2020 14:46:52 +0100 Subject: [PATCH 095/885] [hotfix] In flink-end-to-end-tests-common-kafka, execute dependency plugin in different phase We do this to fix this error message: [ERROR] Failed to execute goal org.apache.maven.plugins:maven-dependency-plugin:3.1.1:copy (copy) on project flink-end-to-end-tests-common-kafka: Artifact has not been packaged yet. When used on reactor artifact, copy should be executed after packaging: see MDEP-187. -> [Help 1] --- .../flink-end-to-end-tests-common-kafka/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/pom.xml b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/pom.xml index 71f524d9ad..b9eab9f456 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/pom.xml +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/pom.xml @@ -141,7 +141,7 @@ under the License. copy - process-test-resources + pre-integration-test copy -- Gitee From 809eb2ab292ad2916d74f4f761b45ec4aa2f5404 Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Thu, 5 Mar 2020 16:47:22 +0100 Subject: [PATCH 096/885] [FLINK-16445][build] Set property japicmp.referenceVersion to 1.10.0 This closes #11324. --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 90de99ba89..21e89ad24f 100644 --- a/pom.xml +++ b/pom.xml @@ -152,7 +152,7 @@ under the License. For Hadoop 2.7, the minor Hadoop version supported for flink-shaded-hadoop-2-uber is 2.7.5 --> 2.7.5 - 1.9.1 + 1.10.0 tools/japicmp-output -- Gitee From 560e8052c8632ae6b1db73dd6152b837402c1290 Mon Sep 17 00:00:00 2001 From: Seth Wiesman Date: Fri, 6 Mar 2020 09:49:47 -0600 Subject: [PATCH 097/885] [FLINK-16313][state-processor-api] Properly dispose of native resources when closing input split This closes #11335. --- .../state/api/input/KeyedStateInputFormat.java | 7 ++++++- .../api/input/operator/StateReaderOperator.java | 16 +++++++++++++++- ...ocksDBStateBackendReaderKeyedStateITCase.java | 3 --- 3 files changed, 21 insertions(+), 5 deletions(-) diff --git a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/input/KeyedStateInputFormat.java b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/input/KeyedStateInputFormat.java index daf62780f9..32df221be1 100644 --- a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/input/KeyedStateInputFormat.java +++ b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/input/KeyedStateInputFormat.java @@ -180,7 +180,12 @@ public class KeyedStateInputFormat extends RichInputFormat imple } public void close() throws Exception { - FunctionUtils.closeFunction(function); + Exception exception = null; + + try { + FunctionUtils.closeFunction(function); + } catch (Exception e) { + // The state backend must always be closed + // to release native resources. + exception = e; + } + + keyedStateBackend.dispose(); + + if (exception != null) { + throw exception; + } } @Override diff --git a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/RocksDBStateBackendReaderKeyedStateITCase.java b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/RocksDBStateBackendReaderKeyedStateITCase.java index 369671662d..df9fb85c9c 100644 --- a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/RocksDBStateBackendReaderKeyedStateITCase.java +++ b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/RocksDBStateBackendReaderKeyedStateITCase.java @@ -22,12 +22,9 @@ import org.apache.flink.contrib.streaming.state.RocksDBStateBackend; import org.apache.flink.runtime.state.StateBackend; import org.apache.flink.runtime.state.memory.MemoryStateBackend; -import org.junit.Ignore; - /** * IT Case for reading state from a RocksDB keyed state backend. */ -@Ignore public class RocksDBStateBackendReaderKeyedStateITCase extends SavepointReaderKeyedStateITCase { @Override protected RocksDBStateBackend getStateBackend() { -- Gitee From a68f5fde472de8a351726239a936ca2acf8073a3 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Mon, 9 Mar 2020 12:05:53 +0800 Subject: [PATCH 098/885] [FLINK-16450][hive] Integrate parquet columnar row reader to hive This closes #11327 --- docs/dev/table/hive/read_write_hive.md | 5 +- docs/dev/table/hive/read_write_hive.zh.md | 5 +- flink-connectors/flink-connector-hive/pom.xml | 24 +++++ .../hive/read/HiveTableInputFormat.java | 97 ++++++++++++------- .../HiveVectorizedParquetSplitReader.java | 91 +++++++++++++++++ .../src/main/resources/META-INF/NOTICE | 13 +++ .../hive/TableEnvHiveConnectorTest.java | 46 +++++++-- 7 files changed, 235 insertions(+), 46 deletions(-) create mode 100644 flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveVectorizedParquetSplitReader.java create mode 100644 flink-connectors/flink-connector-hive/src/main/resources/META-INF/NOTICE diff --git a/docs/dev/table/hive/read_write_hive.md b/docs/dev/table/hive/read_write_hive.md index 07ee3b34d5..8aa83b91a0 100644 --- a/docs/dev/table/hive/read_write_hive.md +++ b/docs/dev/table/hive/read_write_hive.md @@ -171,13 +171,14 @@ It is especially beneficial when a table contains many columns. For queries with LIMIT clause, Flink will limit the number of output records wherever possible to minimize the amount of data transferred across network. -### ORC Vectorized Optimization upon Read +### Vectorized Optimization upon Read Optimization is used automatically when the following conditions are met: +- Format: ORC or Parquet. - Columns without complex data type, like hive types: List, Map, Struct, Union. -This feature is turned on by default. If there is a problem, you can use this config option to close ORC Vectorized Optimization: +This feature is turned on by default. If there is a problem, you can use this config option to close Vectorized Optimization: {% highlight bash %} table.exec.hive.fallback-mapred-reader=true diff --git a/docs/dev/table/hive/read_write_hive.zh.md b/docs/dev/table/hive/read_write_hive.zh.md index 07ee3b34d5..8aa83b91a0 100644 --- a/docs/dev/table/hive/read_write_hive.zh.md +++ b/docs/dev/table/hive/read_write_hive.zh.md @@ -171,13 +171,14 @@ It is especially beneficial when a table contains many columns. For queries with LIMIT clause, Flink will limit the number of output records wherever possible to minimize the amount of data transferred across network. -### ORC Vectorized Optimization upon Read +### Vectorized Optimization upon Read Optimization is used automatically when the following conditions are met: +- Format: ORC or Parquet. - Columns without complex data type, like hive types: List, Map, Struct, Union. -This feature is turned on by default. If there is a problem, you can use this config option to close ORC Vectorized Optimization: +This feature is turned on by default. If there is a problem, you can use this config option to close Vectorized Optimization: {% highlight bash %} table.exec.hive.fallback-mapred-reader=true diff --git a/flink-connectors/flink-connector-hive/pom.xml b/flink-connectors/flink-connector-hive/pom.xml index d02509638d..04efcd952e 100644 --- a/flink-connectors/flink-connector-hive/pom.xml +++ b/flink-connectors/flink-connector-hive/pom.xml @@ -104,6 +104,12 @@ under the License. + + org.apache.flink + flink-parquet_${scala.binary.version} + ${project.version} + + org.apache.flink flink-hadoop-fs @@ -761,6 +767,10 @@ under the License. shade-flink + package + + shade + false @@ -769,6 +779,12 @@ under the License. org.apache.flink:flink-orc_${scala.binary.version} org.apache.flink:flink-orc-nohive_${scala.binary.version} org.apache.flink:flink-hadoop-compatibility_${scala.binary.version} + org.apache.flink:flink-parquet_${scala.binary.version} + org.apache.parquet:parquet-hadoop + org.apache.parquet:parquet-format + org.apache.parquet:parquet-column + org.apache.parquet:parquet-common + org.apache.parquet:parquet-encoding @@ -776,6 +792,14 @@ under the License. org.apache.flink.runtime.fs.hdfs org.apache.flink.connectors.hive.fs.hdfs + + org.apache.parquet + org.apache.flink.shaded.org.apache.parquet + + + shaded.parquet + org.apache.flink.reshaded.parquet + diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveTableInputFormat.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveTableInputFormat.java index f6a23b5ad9..043a68babe 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveTableInputFormat.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveTableInputFormat.java @@ -29,6 +29,7 @@ import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.hive.client.HiveShimLoader; import org.apache.flink.table.dataformat.BaseRow; import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.mapred.InputFormat; @@ -113,9 +114,13 @@ public class HiveTableInputFormat extends HadoopInputFormatCommonBase row1 = new ArrayList<>(Arrays.asList(1, "a", "2018-08-20 00:00:00.1")); List row2 = new ArrayList<>(Arrays.asList(2, "b", "2019-08-26 00:00:00.1")); // some data types are not supported for parquet tables in early versions -- https://issues.apache.org/jira/browse/HIVE-6384 @@ -157,23 +160,37 @@ public class TableEnvHiveConnectorTest { } else { tableSchema = "(i int,s string,ts timestamp)"; } - hiveShell.execute(String.format("create table db1.src %s %s", tableSchema, suffix)); - hiveShell.execute(String.format("create table db1.dest %s %s", tableSchema, suffix)); + + hiveShell.execute(String.format( + "create table db1.src %s partitioned by (p1 string, p2 timestamp) %s", tableSchema, suffix)); + hiveShell.execute(String.format( + "create table db1.dest %s partitioned by (p1 string, p2 timestamp) %s", tableSchema, suffix)); // prepare source data with Hive // TABLE keyword in INSERT INTO is mandatory prior to 1.1.0 - hiveShell.execute(String.format("insert into table db1.src values (%s),(%s)", - toRowValue(row1), toRowValue(row2))); + hiveShell.execute(String.format( + "insert into table db1.src partition(p1='first',p2='2018-08-20 00:00:00.1') values (%s)", + toRowValue(row1))); + hiveShell.execute(String.format( + "insert into table db1.src partition(p1='second',p2='2018-08-26 00:00:00.1') values (%s)", + toRowValue(row2))); + + List expected = Arrays.asList( + String.join("\t", ArrayUtils.concat( + row1.stream().map(Object::toString).toArray(String[]::new), + new String[]{"first", "2018-08-20 00:00:00.1"})), + String.join("\t", ArrayUtils.concat( + row2.stream().map(Object::toString).toArray(String[]::new), + new String[]{"second", "2018-08-26 00:00:00.1"}))); + + verifyFlinkQueryResult(tableEnv.sqlQuery("select * from db1.src"), expected); // populate dest table with source table tableEnv.sqlUpdate("insert into db1.dest select * from db1.src"); tableEnv.execute("test_" + format); // verify data on hive side - verifyHiveQueryResult("select * from db1.dest", - Arrays.asList( - row1.stream().map(Object::toString).collect(Collectors.joining("\t")), - row2.stream().map(Object::toString).collect(Collectors.joining("\t")))); + verifyHiveQueryResult("select * from db1.dest", expected); hiveShell.execute("drop database db1 cascade"); } @@ -523,4 +540,17 @@ public class TableEnvHiveConnectorTest { assertEquals(expected.size(), results.size()); assertEquals(new HashSet<>(expected), new HashSet<>(results)); } + + private void verifyFlinkQueryResult(org.apache.flink.table.api.Table table, List expected) throws Exception { + List rows = TableUtils.collectToList(table); + List results = rows.stream().map(row -> + IntStream.range(0, row.getArity()) + .mapToObj(row::getField) + .map(o -> o instanceof LocalDateTime ? + Timestamp.valueOf((LocalDateTime) o) : o) + .map(Object::toString) + .collect(Collectors.joining("\t"))).collect(Collectors.toList()); + assertEquals(expected.size(), results.size()); + assertEquals(new HashSet<>(expected), new HashSet<>(results)); + } } -- Gitee From e9fd832e1452631f25ff24e1f23ab53763c914dd Mon Sep 17 00:00:00 2001 From: hehuiyuan <471627698@qq.com> Date: Mon, 9 Mar 2020 15:33:32 +0800 Subject: [PATCH 099/885] [FLINK-16463][table-planner-blink] Remove the redundant semicolon in the generated code in CodeGenUtils This closes #11339 --- .../org/apache/flink/table/planner/codegen/CodeGenUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala index 591cd38a47..26306ec0ac 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala @@ -495,7 +495,7 @@ object CodeGenUtils { } } else if (rowClass == classOf[GenericRow] || rowClass == classOf[BoxedWrapperRow]) { val writeField = if (rowClass == classOf[GenericRow]) { - s"$rowTerm.setField($indexTerm, $fieldTerm);" + s"$rowTerm.setField($indexTerm, $fieldTerm)" } else { boxedWrapperRowFieldSetAccess(rowTerm, indexTerm, fieldTerm, fieldType) } -- Gitee From b642005f0cae3001f96db179917cbe6c245b10ab Mon Sep 17 00:00:00 2001 From: tison Date: Fri, 21 Feb 2020 20:08:26 +0800 Subject: [PATCH 100/885] [FLINK-16199][table] Support IS JSON predication for SQL in blink planner This closes #11174 --- .../functions/sql/FlinkSqlOperatorTable.java | 11 ++ .../codegen/calls/FunctionGenerator.scala | 48 +++++++ .../planner/codegen/calls/NotCallGen.scala | 39 ++++++ .../expressions/JsonFunctionsTest.scala | 128 ++++++++++++++++++ 4 files changed, 226 insertions(+) create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/NotCallGen.scala create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/JsonFunctionsTest.scala diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java index 54cf2c58ec..6cae5de44c 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java @@ -32,6 +32,7 @@ import org.apache.calcite.sql.SqlIdentifier; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlOperator; import org.apache.calcite.sql.SqlOperatorBinding; +import org.apache.calcite.sql.SqlPostfixOperator; import org.apache.calcite.sql.SqlSyntax; import org.apache.calcite.sql.fun.SqlStdOperatorTable; import org.apache.calcite.sql.type.InferTypes; @@ -1000,4 +1001,14 @@ public class FlinkSqlOperatorTable extends ReflectiveSqlOperatorTable { public static final SqlAggFunction ROW_NUMBER = SqlStdOperatorTable.ROW_NUMBER; public static final SqlAggFunction LEAD = SqlStdOperatorTable.LEAD; public static final SqlAggFunction LAG = SqlStdOperatorTable.LAG; + + // JSON FUNCTIONS + public static final SqlPostfixOperator IS_JSON_VALUE = SqlStdOperatorTable.IS_JSON_VALUE; + public static final SqlPostfixOperator IS_JSON_OBJECT = SqlStdOperatorTable.IS_JSON_OBJECT; + public static final SqlPostfixOperator IS_JSON_ARRAY = SqlStdOperatorTable.IS_JSON_ARRAY; + public static final SqlPostfixOperator IS_JSON_SCALAR = SqlStdOperatorTable.IS_JSON_SCALAR; + public static final SqlPostfixOperator IS_NOT_JSON_VALUE = SqlStdOperatorTable.IS_NOT_JSON_VALUE; + public static final SqlPostfixOperator IS_NOT_JSON_OBJECT = SqlStdOperatorTable.IS_NOT_JSON_OBJECT; + public static final SqlPostfixOperator IS_NOT_JSON_ARRAY = SqlStdOperatorTable.IS_NOT_JSON_ARRAY; + public static final SqlPostfixOperator IS_NOT_JSON_SCALAR = SqlStdOperatorTable.IS_NOT_JSON_SCALAR; } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/FunctionGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/FunctionGenerator.scala index 4754005562..5e153c3bf2 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/FunctionGenerator.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/FunctionGenerator.scala @@ -753,6 +753,54 @@ object FunctionGenerator { Seq(FLOAT, INTEGER), BuiltInMethods.TRUNCATE_FLOAT) + addSqlFunctionMethod(IS_JSON_VALUE, Seq(CHAR), BuiltInMethod.IS_JSON_VALUE.method) + addSqlFunctionMethod(IS_JSON_VALUE, Seq(VARCHAR), BuiltInMethod.IS_JSON_VALUE.method) + + addSqlFunctionMethod(IS_JSON_OBJECT, Seq(CHAR), BuiltInMethod.IS_JSON_OBJECT.method) + addSqlFunctionMethod(IS_JSON_OBJECT, Seq(VARCHAR), BuiltInMethod.IS_JSON_OBJECT.method) + + addSqlFunctionMethod(IS_JSON_ARRAY, Seq(CHAR), BuiltInMethod.IS_JSON_ARRAY.method) + addSqlFunctionMethod(IS_JSON_ARRAY, Seq(VARCHAR), BuiltInMethod.IS_JSON_ARRAY.method) + + addSqlFunctionMethod(IS_JSON_SCALAR, Seq(CHAR), BuiltInMethod.IS_JSON_SCALAR.method) + addSqlFunctionMethod(IS_JSON_SCALAR, Seq(VARCHAR), BuiltInMethod.IS_JSON_SCALAR.method) + + addSqlFunction(IS_NOT_JSON_VALUE, Seq(CHAR), + new NotCallGen( + new MethodCallGen( + BuiltInMethod.IS_JSON_VALUE.method))) + addSqlFunction(IS_NOT_JSON_VALUE, Seq(VARCHAR), + new NotCallGen( + new MethodCallGen( + BuiltInMethod.IS_JSON_VALUE.method))) + + addSqlFunction(IS_NOT_JSON_OBJECT, Seq(CHAR), + new NotCallGen( + new MethodCallGen( + BuiltInMethod.IS_JSON_OBJECT.method))) + addSqlFunction(IS_NOT_JSON_OBJECT, Seq(VARCHAR), + new NotCallGen( + new MethodCallGen( + BuiltInMethod.IS_JSON_OBJECT.method))) + + addSqlFunction(IS_NOT_JSON_ARRAY, Seq(CHAR), + new NotCallGen( + new MethodCallGen( + BuiltInMethod.IS_JSON_ARRAY.method))) + addSqlFunction(IS_NOT_JSON_ARRAY, Seq(VARCHAR), + new NotCallGen( + new MethodCallGen( + BuiltInMethod.IS_JSON_ARRAY.method))) + + addSqlFunction(IS_NOT_JSON_SCALAR, Seq(CHAR), + new NotCallGen( + new MethodCallGen( + BuiltInMethod.IS_JSON_SCALAR.method))) + addSqlFunction(IS_NOT_JSON_SCALAR, Seq(VARCHAR), + new NotCallGen( + new MethodCallGen( + BuiltInMethod.IS_JSON_SCALAR.method))) + // ---------------------------------------------------------------------------------------------- diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/NotCallGen.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/NotCallGen.scala new file mode 100644 index 0000000000..f670cc2927 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/NotCallGen.scala @@ -0,0 +1,39 @@ +/* + * 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.table.planner.codegen.calls + +import org.apache.flink.table.planner.codegen.{CodeGeneratorContext, GeneratedExpression} +import org.apache.flink.table.types.logical.{BooleanType, LogicalType} + +/** + * Inverts the boolean value of a [[CallGenerator]] result. + */ +class NotCallGen(callGenerator: CallGenerator) extends CallGenerator { + + override def generate( + ctx: CodeGeneratorContext, + operands: Seq[GeneratedExpression], + returnType: LogicalType + ): GeneratedExpression = { + assert(returnType.isInstanceOf[BooleanType]) + + ScalarOperatorGens.generateNot(ctx, callGenerator.generate(ctx, operands, returnType)) + } + +} diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/JsonFunctionsTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/JsonFunctionsTest.scala new file mode 100644 index 0000000000..3a2e925eb0 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/JsonFunctionsTest.scala @@ -0,0 +1,128 @@ +/* + * 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.table.planner.expressions + +import org.apache.flink.api.common.typeinfo.Types +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.table.api.ValidationException +import org.apache.flink.table.planner.expressions.utils.ExpressionTestBase +import org.apache.flink.types.Row +import org.junit.Assert.assertEquals +import org.junit.Test + +class JsonFunctionsTest extends ExpressionTestBase { + + override def testData: Row = { + val testData = new Row(9) + testData.setField(0, "This is a test String.") + testData.setField(1, true) + testData.setField(2, 42.toByte) + testData.setField(3, 43.toShort) + testData.setField(4, 44.toLong) + testData.setField(5, 4.5.toFloat) + testData.setField(6, 4.6) + testData.setField(7, 3) + testData.setField(8, """{ "name" : "flink" }""") + testData + } + + override def typeInfo: RowTypeInfo = { + new RowTypeInfo( + /* 0 */ Types.STRING, + /* 1 */ Types.BOOLEAN, + /* 2 */ Types.BYTE, + /* 3 */ Types.SHORT, + /* 4 */ Types.LONG, + /* 5 */ Types.FLOAT, + /* 6 */ Types.DOUBLE, + /* 7 */ Types.INT, + /* 8 */ Types.STRING) + } + + @Test + def testPredicates(): Unit = { + val malformed = Array(false, false, false, false) + val jsonObject = Array(true, true, false, false) + val jsonArray = Array(true, false, true, false) + val jsonScalar = Array(true, false, false, true) + + // strings + verifyPredicates("'{}'", jsonObject) + verifyPredicates("'[]'", jsonArray) + verifyPredicates("'100'", jsonScalar) + verifyPredicates("'{]'", malformed) + + // valid fields + verifyPredicates("f0", malformed) + verifyPredicates("f8", jsonObject) + + // invalid fields + verifyException("f1", classOf[ValidationException]) + verifyException("f2", classOf[ValidationException]) + verifyException("f3", classOf[ValidationException]) + verifyException("f4", classOf[ValidationException]) + verifyException("f5", classOf[ValidationException]) + verifyException("f6", classOf[ValidationException]) + verifyException("f7", classOf[ValidationException]) + } + + /** + * Utility for verify predicates. + * + * @param candidate to be verified, can be a scalar or a column + * @param expectedValues array of expected values as result of + * (IS_JSON_VALUE, IS_JSON_OBJECT, IS_JSON_ARRAY, IS_JSON_SCALAR) + */ + private def verifyPredicates(candidate: String, expectedValues: Array[Boolean]): Unit = { + assert(expectedValues.length == 4) + + testSqlApi(s"$candidate is json value", expectedValues(0).toString) + testSqlApi(s"$candidate is not json value", (!expectedValues(0)).toString) + testSqlApi(s"$candidate is json object", expectedValues(1).toString) + testSqlApi(s"$candidate is not json object", (!expectedValues(1)).toString) + testSqlApi(s"$candidate is json array", expectedValues(2).toString) + testSqlApi(s"$candidate is not json array", (!expectedValues(2)).toString) + testSqlApi(s"$candidate is json scalar", expectedValues(3).toString) + testSqlApi(s"$candidate is not json scalar", (!expectedValues(3)).toString) + } + + private def verifyException[T <: Exception]( + candidate: String, + expectedException: Class[T] + ): Unit = { + val sqlCandidates = Array( + s"$candidate is json value", + s"$candidate is not json value", + s"$candidate is json object", + s"$candidate is not json object", + s"$candidate is json array", + s"$candidate is not json array", + s"$candidate is json scalar", + s"$candidate is not json scalar") + + for (sql <- sqlCandidates) { + try { + testSqlApi(sql, "null") + } catch { + case e: Exception => assertEquals(e.getClass, expectedException) + } + } + } + +} -- Gitee From 2d08e102f40ca644ee5c84c38ec0bacd586ee288 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Mon, 9 Mar 2020 17:31:08 +0800 Subject: [PATCH 101/885] [FLINK-16500][hive] Hive get primary key should not throw exception when Invalid method name This closes #11352 --- .../table/catalog/hive/client/HiveShimV210.java | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV210.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV210.java index e360ee5a04..5d1bef48e5 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV210.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV210.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hive.metastore.api.EnvironmentContext; import org.apache.hadoop.hive.metastore.api.InvalidOperationException; import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.thrift.TApplicationException; import org.apache.thrift.TException; import java.lang.reflect.InvocationTargetException; @@ -95,8 +96,16 @@ public class HiveShimV210 extends HiveShimV201 { // all pk constraints should have the same name, so let's use the name of the first one String pkName = (String) HiveReflectionUtils.invokeMethod(constraintClz, constraints.get(0), "getPk_name", null, null); return Optional.of(UniqueConstraint.primaryKey(pkName, colNames)); - } catch (Exception e) { - throw new CatalogException("Failed to get PrimaryKey constraints", e); + } catch (Throwable t) { + if (t instanceof InvocationTargetException) { + t = t.getCause(); + } + if (t instanceof TApplicationException && + t.getMessage() != null && + t.getMessage().contains("Invalid method name")) { + return Optional.empty(); + } + throw new CatalogException("Failed to get PrimaryKey constraints", t); } } -- Gitee From cac40c6b2c4aa2b397b6df562aa23bd1f18dd3b0 Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Mon, 9 Mar 2020 12:13:34 +0800 Subject: [PATCH 102/885] [FLINK-16467][core] Fix MemorySizeTest#testToHumanReadableString failure due to different string formats in different locales. --- .../src/main/java/org/apache/flink/configuration/MemorySize.java | 1 + 1 file changed, 1 insertion(+) diff --git a/flink-core/src/main/java/org/apache/flink/configuration/MemorySize.java b/flink-core/src/main/java/org/apache/flink/configuration/MemorySize.java index b98b370b74..e92c99ca09 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/MemorySize.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/MemorySize.java @@ -191,6 +191,7 @@ public class MemorySize implements java.io.Serializable, Comparable } else { double approximate = 1.0 * bytes / highestUnit.getMultiplier(); return String.format( + Locale.ROOT, "%.3f%s (%d bytes)", approximate, highestUnit.getUnits()[1], -- Gitee From aaf63f0f969063ed5c557f3eb0a6ca64629dd96a Mon Sep 17 00:00:00 2001 From: tison Date: Wed, 4 Mar 2020 23:10:54 +0800 Subject: [PATCH 103/885] [FLINK-16427][api] Don't throw ProgramInvocationException in RemoteStreamEnvironment ProgramInvocationException is in flink-clients and we want to eventually turn around the dependencies, that is flink-clients should depend on flink-streaming-java (or some streaming translation package) and not the other way round, as it currently is. --- .../api/environment/RemoteStreamEnvironment.java | 12 +----------- 1 file changed, 1 insertion(+), 11 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java index 82800fa7bb..560633e42d 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java @@ -21,7 +21,6 @@ import org.apache.flink.annotation.Public; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.java.RemoteEnvironmentConfigUtils; -import org.apache.flink.client.program.ProgramInvocationException; import org.apache.flink.configuration.ConfigUtils; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.DeploymentOptions; @@ -215,16 +214,7 @@ public class RemoteStreamEnvironment extends StreamExecutionEnvironment { @Override public JobExecutionResult execute(StreamGraph streamGraph) throws Exception { - try { - return super.execute(streamGraph); - } - catch (ProgramInvocationException e) { - throw e; - } - catch (Exception e) { - String term = e.getMessage() == null ? "." : (": " + e.getMessage()); - throw new ProgramInvocationException("The program execution failed" + term, e); - } + return super.execute(streamGraph); } @Override -- Gitee From f21fa9675b15b8a9673fa529a1d368427f846161 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Tue, 10 Mar 2020 11:07:44 +0800 Subject: [PATCH 104/885] [FLINK-16273][python] Set io.netty.tryReflectionSetAccessible to true by default (#11341) --- .../AbstractArrowPythonScalarFunctionRunner.java | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/AbstractArrowPythonScalarFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/AbstractArrowPythonScalarFunctionRunner.java index 033d8d78f2..80f7ded5aa 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/AbstractArrowPythonScalarFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/AbstractArrowPythonScalarFunctionRunner.java @@ -46,6 +46,18 @@ public abstract class AbstractArrowPythonScalarFunctionRunner extends Abstra private static final String SCHEMA_ARROW_CODER_URN = "flink:coder:schema:scalar_function:arrow:v1"; + static { + // Arrow requires the property io.netty.tryReflectionSetAccessible to + // be set to true for JDK >= 9. Please refer to ARROW-5412 for more details. + if (System.getProperty("io.netty.tryReflectionSetAccessible") == null) { + System.setProperty("io.netty.tryReflectionSetAccessible", "true"); + } else if (!io.netty.util.internal.PlatformDependent.hasDirectBufferNoCleanerConstructor()) { + throw new RuntimeException("Vectorized Python UDF depends on " + + "DirectByteBuffer.(long, int) which is not available. Please set the " + + "system property 'io.netty.tryReflectionSetAccessible' to 'true'."); + } + } + /** * Max number of elements to include in an arrow batch. */ -- Gitee From 5a81fa0766abe3ef6c0e6b8f9217e9adcd18095f Mon Sep 17 00:00:00 2001 From: "Jiangjie (Becket) Qin" Date: Sun, 8 Dec 2019 18:21:17 +0800 Subject: [PATCH 105/885] [FLINK-15131][connector/source] Add the APIs for Source (FLIP-27). Boundedness - A enum indicating whether the input stream is Bounded or Unbounded Source - A factory style class that helps create SplitEnumerator and SourceReader at runtime. SourceSplit - An interface for all the split types. SplitEnumerator - Discover the splits and assign them to the SourceReaders SplitEnumeratorContext - Provide necessary information to the SplitEnumerator to assign splits and send custom events to the the SourceReaders. SplitAssignment - A container class holding the source split assignment for each subtask. SourceReader - Read the records from the splits assigned by the SplitEnumerator. ReaderInfo - A container class about reader information. SourceReaderContext - Provide necessary function to the SourceReader to communicate with SplitEnumerator. SourceOutput - A collector style interface to take the records and timestamps emit by the SourceReader. WatermarkOutput - An interface for emitting watermark and indicate idleness of the source. Watermark - A new Watermark class in the package org.apache.flink.api.common.eventtime to replace the existing Watermark class. --- .../flink/api/common/eventtime/Watermark.java | 106 ++++++++++++++++++ .../api/common/eventtime/WatermarkOutput.java | 44 ++++++++ .../api/connector/source/Boundedness.java | 55 +++++++++ .../api/connector/source/ReaderInfo.java | 51 +++++++++ .../flink/api/connector/source/Source.java | 94 ++++++++++++++++ .../api/connector/source/SourceEvent.java | 31 +++++ .../api/connector/source/SourceOutput.java | 45 ++++++++ .../api/connector/source/SourceReader.java | 93 +++++++++++++++ .../connector/source/SourceReaderContext.java | 45 ++++++++ .../api/connector/source/SourceSplit.java | 34 ++++++ .../api/connector/source/SplitEnumerator.java | 78 +++++++++++++ .../source/SplitEnumeratorContext.java | 96 ++++++++++++++++ .../connector/source/SplitsAssignment.java | 51 +++++++++ 13 files changed, 823 insertions(+) create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/eventtime/Watermark.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/eventtime/WatermarkOutput.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/connector/source/Boundedness.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/connector/source/ReaderInfo.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/connector/source/Source.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/connector/source/SourceEvent.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/connector/source/SourceOutput.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/connector/source/SourceReader.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/connector/source/SourceReaderContext.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/connector/source/SourceSplit.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/connector/source/SplitEnumerator.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/connector/source/SplitEnumeratorContext.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/connector/source/SplitsAssignment.java diff --git a/flink-core/src/main/java/org/apache/flink/api/common/eventtime/Watermark.java b/flink-core/src/main/java/org/apache/flink/api/common/eventtime/Watermark.java new file mode 100644 index 0000000000..8a44ebc212 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/eventtime/Watermark.java @@ -0,0 +1,106 @@ +/* + 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.api.common.eventtime; + +import org.apache.flink.annotation.Public; + +import java.io.Serializable; +import java.text.SimpleDateFormat; +import java.util.Date; + +/** + * Watermarks are the progress indicators in the data streams. A watermark signifies + * that no events with a timestamp smaller or equal to the watermark's time will occur after the + * water. A watermark with timestamp T indicates that the stream's event time has progressed + * to time T. + * + *

    Watermarks are created at the sources and propagate through the streams and operators. + * + *

    In some cases a watermark is only a heuristic, meaning some events with a lower timestamp + * may still follow. In that case, it is up to the logic of the operators to decide what to do + * with the "late events". Operators can for example ignore these late events, route them to a + * different stream, or send update to their previously emitted results. + * + *

    When a source reaches the end of the input, it emits a final watermark with timestamp + * {@code Long.MAX_VALUE}, indicating the "end of time". + * + *

    Note: A stream's time starts with a watermark of {@code Long.MIN_VALUE}. That means that all records + * in the stream with a timestamp of {@code Long.MIN_VALUE} are immediately late. + */ +@Public +public final class Watermark implements Serializable { + + private static final long serialVersionUID = 1L; + + /** Thread local formatter for stringifying the timestamps. */ + private static final ThreadLocal TS_FORMATTER = ThreadLocal.withInitial( + () -> new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS")); + + // ------------------------------------------------------------------------ + + /** The watermark that signifies end-of-event-time. */ + public static final Watermark MAX_WATERMARK = new Watermark(Long.MAX_VALUE); + + // ------------------------------------------------------------------------ + + /** The timestamp of the watermark in milliseconds. */ + private final long timestamp; + + /** + * Creates a new watermark with the given timestamp in milliseconds. + */ + public Watermark(long timestamp) { + this.timestamp = timestamp; + } + + /** + * Returns the timestamp associated with this Watermark. + */ + public long getTimestamp() { + return timestamp; + } + + /** + * Formats the timestamp of this watermark, assuming it is a millisecond timestamp. + * The returned format is "yyyy-MM-dd HH:mm:ss.SSS". + */ + public String getFormattedTimestamp() { + return TS_FORMATTER.get().format(new Date(timestamp)); + } + + // ------------------------------------------------------------------------ + + @Override + public boolean equals(Object o) { + return this == o || + o != null && + o.getClass() == Watermark.class && + ((Watermark) o).timestamp == this.timestamp; + } + + @Override + public int hashCode() { + return Long.hashCode(timestamp); + } + + @Override + public String toString() { + return "Watermark @ " + timestamp + " (" + getFormattedTimestamp() + ')'; + } +} diff --git a/flink-core/src/main/java/org/apache/flink/api/common/eventtime/WatermarkOutput.java b/flink-core/src/main/java/org/apache/flink/api/common/eventtime/WatermarkOutput.java new file mode 100644 index 0000000000..a1f0d9d5d7 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/eventtime/WatermarkOutput.java @@ -0,0 +1,44 @@ +/* + 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.api.common.eventtime; + +import org.apache.flink.annotation.Public; + +/** + * An output for watermarks. The output accepts watermarks and idleness (inactivity) status. + */ +@Public +public interface WatermarkOutput { + + /** + * Emits the given watermark. + * + *

    Emitting a watermark also implicitly marks the stream as active, ending + * previously marked idleness. + */ + void emitWatermark(Watermark watermark); + + /** + * Marks this output as idle, meaning that downstream operations do not + * wait for watermarks from this output. + * + *

    An output becomes active again as soon as the next watermark is emitted. + */ + void markIdle(); +} diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/Boundedness.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/Boundedness.java new file mode 100644 index 0000000000..5b5c4a66f6 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/Boundedness.java @@ -0,0 +1,55 @@ +/* + 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.api.connector.source; + +import org.apache.flink.annotation.Public; + +/** + * The boundedness of a stream. A stream could either be "bounded" (a stream with finite records) or + * "unbounded" (a stream with infinite records). + */ +@Public +public enum Boundedness { + /** + * A BOUNDED stream is a stream with finite records. + * + *

    In the context of sources, a BOUNDED stream expects the source to put a boundary of the + * records it emits. Such boundaries could be number of records, number of bytes, elapsed time, + * and so on. Such indication of how to bound a stream is typically passed to the sources via + * configurations. When the sources emit a BOUNDED stream, Flink may leverage this property to + * do specific optimizations in the execution. + * + *

    Unlike unbounded streams, the bounded streams are usually order insensitive. That means + * the source implementations may not have to keep track of the event times or watermarks. + * Instead, a higher throughput would be preferred. + */ + BOUNDED, + + /** + * A CONTINUOUS_UNBOUNDED stream is a stream with infinite records. + * + *

    In the context of sources, an infinite stream expects the source implementation to run + * without an upfront indication to Flink that they will eventually stop. The sources may + * eventually be terminated when users cancel the jobs or some source-specific condition is met. + * + *

    A CONTINUOUS_UNBOUNDED stream may also eventually stop at some point. But before that + * happens, Flink always assumes the sources are going to run forever. + */ + CONTINUOUS_UNBOUNDED +} diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/ReaderInfo.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/ReaderInfo.java new file mode 100644 index 0000000000..f899b12b9c --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/ReaderInfo.java @@ -0,0 +1,51 @@ +/* + 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.api.connector.source; + +import org.apache.flink.annotation.Public; + +import java.io.Serializable; + +/** + * A container class hosting the information of a {@link SourceReader}. + */ +@Public +public final class ReaderInfo implements Serializable { + private final int subtaskId; + private final String location; + + public ReaderInfo(int subtaskId, String location) { + this.subtaskId = subtaskId; + this.location = location; + } + + /** + * @return the ID of the subtask that runs the source reader. + */ + public int getSubtaskId() { + return subtaskId; + } + + /** + * @return the location of the subtask that runs this source reader. + */ + public String getLocation() { + return location; + } +} diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/Source.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/Source.java new file mode 100644 index 0000000000..1b8a1ca1a2 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/Source.java @@ -0,0 +1,94 @@ +/* + 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.api.connector.source; + +import org.apache.flink.annotation.Public; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import java.io.IOException; +import java.io.Serializable; + +/** + * The interface for Source. It acts like a factory class that helps construct + * the {@link SplitEnumerator} and {@link SourceReader} and corresponding + * serializers. + * + * @param The type of records produced by the source. + * @param The type of splits handled by the source. + * @param The type of the enumerator checkpoints. + */ +@Public +public interface Source extends Serializable { + + /** + * Get the boundedness of this source. + * + * @return the boundedness of this source. + */ + Boundedness getBoundedness(); + + /** + * Creates a new reader to read data from the spits it gets assigned. + * The reader starts fresh and does not have any state to resume. + * + * @param readerContext The {@link SourceReaderContext context} for the source reader. + * @return A new SourceReader. + */ + SourceReader createReader(SourceReaderContext readerContext); + + /** + * Creates a new SplitEnumerator for this source, starting a new input. + * + * @param enumContext The {@link SplitEnumeratorContext context} for the split enumerator. + * @return A new SplitEnumerator. + */ + SplitEnumerator createEnumerator(SplitEnumeratorContext enumContext); + + /** + * Restores an enumerator from a checkpoint. + * + * @param enumContext The {@link SplitEnumeratorContext context} for the restored split enumerator. + * @param checkpoint The checkpoint to restore the SplitEnumerator from. + * @return A SplitEnumerator restored from the given checkpoint. + */ + SplitEnumerator restoreEnumerator( + SplitEnumeratorContext enumContext, + EnumChkT checkpoint) throws IOException; + + // ------------------------------------------------------------------------ + // serializers for the metadata + // ------------------------------------------------------------------------ + + /** + * Creates a serializer for the source splits. Splits are serialized when sending them + * from enumerator to reader, and when checkpointing the reader's current state. + * + * @return The serializer for the split type. + */ + SimpleVersionedSerializer getSplitSerializer(); + + /** + * Creates the serializer for the {@link SplitEnumerator} checkpoint. + * The serializer is used for the result of the {@link SplitEnumerator#snapshotState()} + * method. + * + * @return The serializer for the SplitEnumerator checkpoint. + */ + SimpleVersionedSerializer getEnumeratorCheckpointSerializer(); +} diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/SourceEvent.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/SourceEvent.java new file mode 100644 index 0000000000..2377bb76ce --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/SourceEvent.java @@ -0,0 +1,31 @@ +/* + 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.api.connector.source; + +import org.apache.flink.annotation.Public; + +import java.io.Serializable; + +/** + * An base class for the events passed between the SourceReaders and Enumerators. + */ +@Public +public interface SourceEvent extends Serializable { + +} diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/SourceOutput.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/SourceOutput.java new file mode 100644 index 0000000000..4d9f065ed6 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/SourceOutput.java @@ -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.flink.api.connector.source; + +import org.apache.flink.annotation.Public; +import org.apache.flink.api.common.eventtime.WatermarkOutput; + +/** + * The interface provided by Flink task to the {@link SourceReader} to emit records + * to downstream operators for message processing. + */ +@Public +public interface SourceOutput extends WatermarkOutput { + + /** + * Emit a record without a timestamp. Equivalent to {@link #collect(Object, long) collect(timestamp, null)}; + * + * @param record the record to emit. + */ + void collect(T record) throws Exception; + + /** + * Emit a record with timestamp. + * + * @param record the record to emit. + * @param timestamp the timestamp of the record. + */ + void collect(T record, long timestamp) throws Exception; +} diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/SourceReader.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/SourceReader.java new file mode 100644 index 0000000000..24d1bee9b6 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/SourceReader.java @@ -0,0 +1,93 @@ +/* + 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.api.connector.source; + +import org.apache.flink.annotation.Public; + +import java.io.Serializable; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +/** + * The interface for a source reader which is responsible for reading the records from + * the source splits assigned by {@link SplitEnumerator}. + * + * @param The type of the record emitted by this source reader. + * @param The type of the the source splits. + */ +@Public +public interface SourceReader extends Serializable, AutoCloseable { + + /** + * Start the reader. + */ + void start(); + + /** + * Poll the next available record into the {@link SourceOutput}. + * + *

    The implementation must make sure this method is non-blocking. + * + *

    Although the implementation can emit multiple records into the given SourceOutput, + * it is recommended not doing so. Instead, emit one record into the SourceOutput + * and return a {@link Status#AVAILABLE_NOW} to let the caller thread + * know there are more records available. + * + * @return The {@link Status} of the SourceReader after the method invocation. + */ + Status pollNext(SourceOutput sourceOutput) throws Exception; + + /** + * Checkpoint on the state of the source. + * + * @return the state of the source. + */ + List snapshotState(); + + /** + * @return a future that will be completed once there is a record available to poll. + */ + CompletableFuture isAvailable(); + + /** + * Adds a list of splits for this reader to read. + * + * @param splits The splits assigned by the split enumerator. + */ + void addSplits(List splits); + + /** + * Handle a source event sent by the {@link SplitEnumerator}. + * + * @param sourceEvent the event sent by the {@link SplitEnumerator}. + */ + void handleSourceEvents(SourceEvent sourceEvent); + + /** + * The status of this reader. + */ + enum Status { + /** The next record is available right now. */ + AVAILABLE_NOW, + /** The next record will be available later. */ + AVAILABLE_LATER, + /** The source reader has completed all the reading work. */ + FINISHED + } +} diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/SourceReaderContext.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/SourceReaderContext.java new file mode 100644 index 0000000000..f355306a07 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/SourceReaderContext.java @@ -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.flink.api.connector.source; + +import org.apache.flink.annotation.Public; +import org.apache.flink.metrics.MetricGroup; + +import java.util.concurrent.CompletableFuture; + +/** + * The class that expose some context from runtime to the {@link SourceReader}. + */ +@Public +public interface SourceReaderContext { + + /** + * @return The metric group this source belongs to. + */ + MetricGroup metricGroup(); + + /** + * Send a source event to the source coordinator. + * + * @param sourceEvent the source event to coordinator. + * @return a completable future which will be completed either the event has been successfully sent + * or failed. + */ + CompletableFuture sendSourceEventToCoordinator(SourceEvent sourceEvent); +} diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/SourceSplit.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/SourceSplit.java new file mode 100644 index 0000000000..d317f0ec1f --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/SourceSplit.java @@ -0,0 +1,34 @@ +/* + 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.api.connector.source; + +import org.apache.flink.annotation.Public; + +/** + * An interface for all the Split types to extend. + */ +@Public +public interface SourceSplit { + + /** + * Get the split id of this source split. + * @return id of this source split. + */ + String splitId(); +} diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/SplitEnumerator.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/SplitEnumerator.java new file mode 100644 index 0000000000..bdaee36da2 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/SplitEnumerator.java @@ -0,0 +1,78 @@ +/* + 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.api.connector.source; + +import org.apache.flink.annotation.Public; + +import java.io.IOException; +import java.util.List; + +/** + * A interface of a split enumerator responsible for the followings: + * 1. discover the splits for the {@link SourceReader} to read. + * 2. assign the splits to the source reader. + */ +@Public +public interface SplitEnumerator extends AutoCloseable { + + /** + * Start the split enumerator. + * + *

    The default behavior does nothing. + */ + void start(); + + /** + * Handles the source event from the source reader. + * + * @param subtaskId the subtask id of the source reader who sent the source event. + * @param sourceEvent the source event from the source reader. + */ + void handleSourceEvent(int subtaskId, SourceEvent sourceEvent); + + /** + * Add a split back to the split enumerator. It will only happen when a {@link SourceReader} fails + * and there are splits assigned to it after the last successful checkpoint. + * + * @param splits The split to add back to the enumerator for reassignment. + * @param subtaskId The id of the subtask to which the returned splits belong. + */ + void addSplitsBack(List splits, int subtaskId); + + /** + * Add a new source reader with the given subtask ID. + * + * @param subtaskId the subtask ID of the new source reader. + */ + void addReader(int subtaskId); + + /** + * Checkpoints the state of this split enumerator. + * + * @return an object containing the state of the split enumerator. + */ + CheckpointT snapshotState(); + + /** + * Called to close the enumerator, in case it holds on to any resources, like threads or + * network connections. + */ + @Override + void close() throws IOException; +} diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/SplitEnumeratorContext.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/SplitEnumeratorContext.java new file mode 100644 index 0000000000..0dd004adad --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/SplitEnumeratorContext.java @@ -0,0 +1,96 @@ +/* + 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.api.connector.source; + +import org.apache.flink.annotation.Public; +import org.apache.flink.metrics.MetricGroup; + +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.function.BiConsumer; + +/** + * A context class for the {@link SplitEnumerator}. This class serves the following purposes: + * 1. Host information necessary for the SplitEnumerator to make split assignment decisions. + * 2. Accept and track the split assignment from the enumerator. + * 3. Provide a managed threading model so the split enumerators do not need to create their + * own internal threads. + * + * @param the type of the splits. + */ +@Public +public interface SplitEnumeratorContext { + + MetricGroup metricGroup(); + + /** + * Send a source event to a source reader. The source reader is identified by its subtask id. + * + * @param subtaskId the subtask id of the source reader to send this event to. + * @param event the source event to send. + */ + void sendEventToSourceReader(int subtaskId, SourceEvent event); + + /** + * Get the number of subtasks. + * + * @return the number of subtasks. + */ + int numSubtasks(); + + /** + * Get the currently registered readers. The mapping is from subtask id to the reader info. + * + * @return the currently registered readers. + */ + Map registeredReaders(); + + /** + * Assign the splits. + * + * @param newSplitAssignments the new split assignments to add. + */ + void assignSplits(SplitsAssignment newSplitAssignments); + + /** + * Invoke the callable and handover the return value to the handler which will be executed + * by the source coordinator. + * + *

    It is important to make sure that the callable should not modify + * any shared state. Otherwise the there might be unexpected behavior. + * + * @param callable a callable to call. + * @param handler a handler that handles the return value of or the exception thrown from the callable. + */ + void callAsync(Callable callable, BiConsumer handler); + + /** + * Invoke the callable and handover the return value to the handler which will be executed + * by the source coordinator. + * + *

    It is important to make sure that the callable should not modify + * any shared state. Otherwise the there might be unexpected behavior. + * + * @param callable the callable to call. + * @param handler a handler that handles the return value of or the exception thrown from the callable. + * @param initialDelay the initial delay of calling the callable. + * @param period the period between two invocations of the callable. + */ + void callAsync(Callable callable, BiConsumer handler, long initialDelay, long period); +} diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/SplitsAssignment.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/SplitsAssignment.java new file mode 100644 index 0000000000..aad3577d95 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/SplitsAssignment.java @@ -0,0 +1,51 @@ +/* + 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.api.connector.source; + +import org.apache.flink.annotation.Public; + +import java.util.List; +import java.util.Map; + +/** + * A class containing the splits assignment to the source readers. + * + *

    The assignment is always incremental. In another word, splits in the assignment are simply + * added to the existing assignment. + */ +@Public +public final class SplitsAssignment { + private final Map> assignment; + + public SplitsAssignment(Map> assignment) { + this.assignment = assignment; + } + + /** + * @return A mapping from subtask ID to their split assignment. + */ + public Map> assignment() { + return assignment; + } + + @Override + public String toString() { + return assignment.toString(); + } +} -- Gitee From 8f8e35815c917616f98c13d056f20fefe36098f3 Mon Sep 17 00:00:00 2001 From: jingwen-ywb Date: Wed, 19 Feb 2020 21:08:11 +0800 Subject: [PATCH 106/885] [FLINK-16167][python][doc] Improve python_shell document This closes #11142. --- docs/ops/python_shell.md | 26 +++++++++++--------------- docs/ops/python_shell.zh.md | 23 ++++++++++------------- 2 files changed, 21 insertions(+), 28 deletions(-) diff --git a/docs/ops/python_shell.md b/docs/ops/python_shell.md index a2faea246d..ac571c0ce0 100644 --- a/docs/ops/python_shell.md +++ b/docs/ops/python_shell.md @@ -34,20 +34,16 @@ $ python --version # the version printed here must be 3.5+ {% endhighlight %} -Note Using Python UDF in Python Shell requires apache-beam 2.19.0. Run the following command to confirm that it meets the requirements before run the Shell in local mode: +To use the shell with an integrated Flink cluster, you can simply install PyFlink with PyPi and execute the shell directly: {% highlight bash %} -$ python -m pip install apache-beam==2.19.0 +# install PyFlink +$ python -m pip install apache-flink +# execute the shell +$ pyflink-shell.sh local {% endhighlight %} -To use the shell with an integrated Flink cluster just execute: - -{% highlight bash %} -bin/pyflink-shell.sh local -{% endhighlight %} - -in the root directory of your binary Flink directory. To run the Shell on a -cluster, please see the Setup section below. +To run the shell on a cluster, please see the Setup section below. ## Usage @@ -130,7 +126,7 @@ The example below is a simple program in the Python shell: To get an overview of what options the Python Shell provides, please use {% highlight bash %} -bin/pyflink-shell.sh --help +pyflink-shell.sh --help {% endhighlight %} ### Local @@ -138,7 +134,7 @@ bin/pyflink-shell.sh --help To use the shell with an integrated Flink cluster just execute: {% highlight bash %} -bin/pyflink-shell.sh local +pyflink-shell.sh local {% endhighlight %} @@ -148,7 +144,7 @@ To use it with a running cluster, please start the Python shell with the keyword and supply the host and port of the JobManager with: {% highlight bash %} -bin/pyflink-shell.sh remote +pyflink-shell.sh remote {% endhighlight %} ### Yarn Python Shell cluster @@ -163,7 +159,7 @@ For example, to start a Yarn cluster for the Python Shell with two TaskManagers use the following: {% highlight bash %} -bin/pyflink-shell.sh yarn -n 2 +pyflink-shell.sh yarn -n 2 {% endhighlight %} For all other options, see the full reference at the bottom. @@ -175,7 +171,7 @@ If you have previously deployed a Flink cluster using the Flink Yarn Session, the Python shell can connect with it using the following command: {% highlight bash %} -bin/pyflink-shell.sh yarn +pyflink-shell.sh yarn {% endhighlight %} diff --git a/docs/ops/python_shell.zh.md b/docs/ops/python_shell.zh.md index 09dbccc514..88846d8685 100644 --- a/docs/ops/python_shell.zh.md +++ b/docs/ops/python_shell.zh.md @@ -34,16 +34,13 @@ $ python --version # the version printed here must be 3.5+ {% endhighlight %} -注意 在Python Shell中使用Python UDF依赖apache-beam 2.19.0。 在以本地模式执行之前,执行以下命令以确认环境满足需求: +你可以通过PyPi安装PyFlink,然后使用Python Shell: {% highlight bash %} -$ python -m pip install apache-beam==2.19.0 -{% endhighlight %} - -为了使用Flink的Python Shell,你只需要在Flink的binary目录下执行: - -{% highlight bash %} -bin/pyflink-shell.sh local +# 安装 PyFlink +$ python -m pip install apache-flink +# 执行脚本 +$ pyflink-shell.sh local {% endhighlight %} 关于如何在一个Cluster集群上运行Python shell,可以参考启动章节介绍。 @@ -129,7 +126,7 @@ bin/pyflink-shell.sh local 查看Python Shell提供的可选参数,可以使用: {% highlight bash %} -bin/pyflink-shell.sh --help +pyflink-shell.sh --help {% endhighlight %} ### Local @@ -137,7 +134,7 @@ bin/pyflink-shell.sh --help Python Shell运行在local模式下,只需要执行: {% highlight bash %} -bin/pyflink-shell.sh local +pyflink-shell.sh local {% endhighlight %} @@ -147,7 +144,7 @@ Python Shell运行在一个指定的JobManager上,通过关键字`remote`和 的地址和端口号来进行指定: {% highlight bash %} -bin/pyflink-shell.sh remote +pyflink-shell.sh remote {% endhighlight %} ### Yarn Python Shell cluster @@ -157,7 +154,7 @@ Python Shell可以运行在YARN集群之上。Python shell在Yarn上部署一个 例如,在一个部署了两个TaskManager的Yarn集群上运行Python Shell: {% highlight bash %} -bin/pyflink-shell.sh yarn -n 2 +pyflink-shell.sh yarn -n 2 {% endhighlight %} 关于所有可选的参数,可以查看本页面底部的完整说明。 @@ -168,7 +165,7 @@ bin/pyflink-shell.sh yarn -n 2 如果你已经通过Flink Yarn Session部署了一个Flink集群,能够通过以下的命令连接到这个集群: {% highlight bash %} -bin/pyflink-shell.sh yarn +pyflink-shell.sh yarn {% endhighlight %} -- Gitee From 6f19fd2a72e85c9954d4e1830e15637180fee8ec Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Tue, 10 Mar 2020 19:36:15 +0800 Subject: [PATCH 107/885] [FLINK-16483][python] Add Python building blocks to make sure the basic functionality of vectorized Python UDF could work (#11342) --- .../pyflink/fn_execution/ResettableIO.py | 72 ++++++++ .../pyflink/fn_execution/coder_impl.py | 43 +++++ flink-python/pyflink/fn_execution/coders.py | 42 ++++- .../pyflink/table/tests/test_pandas_udf.py | 162 ++++++++++++++++++ flink-python/pyflink/table/udf.py | 41 +++-- flink-python/setup.py | 3 +- ...stractArrowPythonScalarFunctionRunner.java | 1 + 7 files changed, 351 insertions(+), 13 deletions(-) create mode 100644 flink-python/pyflink/fn_execution/ResettableIO.py create mode 100644 flink-python/pyflink/table/tests/test_pandas_udf.py diff --git a/flink-python/pyflink/fn_execution/ResettableIO.py b/flink-python/pyflink/fn_execution/ResettableIO.py new file mode 100644 index 0000000000..ecca3d34e4 --- /dev/null +++ b/flink-python/pyflink/fn_execution/ResettableIO.py @@ -0,0 +1,72 @@ +################################################################################ +# 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. +################################################################################ +import io + + +class ResettableIO(io.RawIOBase): + """ + Raw I/O implementation the input and output stream is resettable. + """ + + def set_input_bytes(self, b): + self._input_bytes = b + self._input_offset = 0 + + def readinto(self, b): + """ + Read up to len(b) bytes into the writable buffer *b* and return + the number of bytes read. If no bytes are available, None is returned. + """ + input_len = len(self._input_bytes) + output_buffer_len = len(b) + remaining = input_len - self._input_offset + + if remaining >= output_buffer_len: + b[:] = self._input_bytes[self._input_offset:self._input_offset + output_buffer_len] + self._input_offset += output_buffer_len + return output_buffer_len + elif remaining > 0: + b[:remaining] = self._input_bytes[self._input_offset:self._input_offset + remaining] + self._input_offset = input_len + return remaining + else: + return None + + def set_output_stream(self, output_stream): + self._output_stream = output_stream + + def write(self, b): + """ + Write the given bytes or pyarrow.Buffer object *b* to the underlying + output stream and return the number of bytes written. + """ + if isinstance(b, bytes): + self._output_stream.write(b) + else: + # pyarrow.Buffer + self._output_stream.write(b.to_pybytes()) + return len(b) + + def seekable(self): + return False + + def readable(self): + return True + + def writable(self): + return True diff --git a/flink-python/pyflink/fn_execution/coder_impl.py b/flink-python/pyflink/fn_execution/coder_impl.py index 56f1d5e0ce..ea2db30809 100644 --- a/flink-python/pyflink/fn_execution/coder_impl.py +++ b/flink-python/pyflink/fn_execution/coder_impl.py @@ -19,8 +19,11 @@ import datetime import decimal import struct +import pyarrow as pa + from apache_beam.coders.coder_impl import StreamCoderImpl +from pyflink.fn_execution.ResettableIO import ResettableIO from pyflink.table import Row @@ -373,3 +376,43 @@ class TimestampCoderImpl(StreamCoderImpl): second, microsecond = (milliseconds // 1000, milliseconds % 1000 * 1000 + nanoseconds // 1000) return datetime.datetime.utcfromtimestamp(second).replace(microsecond=microsecond) + + +class ArrowCoderImpl(StreamCoderImpl): + + def __init__(self, schema): + self._schema = schema + self._resettable_io = ResettableIO() + self._batch_reader = ArrowCoderImpl._load_from_stream(self._resettable_io) + self._batch_writer = pa.RecordBatchStreamWriter(self._resettable_io, self._schema) + + def encode_to_stream(self, cols, out_stream, nested): + self._resettable_io.set_output_stream(out_stream) + self._batch_writer.write_batch(self._create_batch(cols)) + + def decode_from_stream(self, in_stream, nested): + self._resettable_io.set_input_bytes(in_stream.read_all()) + # there is only arrow batch in the underlying input stream + table = pa.Table.from_batches([next(self._batch_reader)]) + return [c.to_pandas(date_as_object=True) for c in table.itercolumns()] + + @staticmethod + def _load_from_stream(stream): + reader = pa.ipc.open_stream(stream) + for batch in reader: + yield batch + + def _create_batch(self, cols): + def create_array(s, t): + try: + return pa.Array.from_pandas(s, mask=s.isnull(), type=t) + except pa.ArrowException as e: + error_msg = "Exception thrown when converting pandas.Series (%s) to " \ + "pyarrow.Array (%s)." + raise RuntimeError(error_msg % (s.dtype, t), e) + + arrays = [create_array(cols[i], self._schema.types[i]) for i in range(0, len(self._schema))] + return pa.RecordBatch.from_arrays(arrays, self._schema) + + def __repr__(self): + return 'ArrowCoderImpl[%s]' % self._schema diff --git a/flink-python/pyflink/fn_execution/coders.py b/flink-python/pyflink/fn_execution/coders.py index e87c760fed..ae6e3df93f 100644 --- a/flink-python/pyflink/fn_execution/coders.py +++ b/flink-python/pyflink/fn_execution/coders.py @@ -21,6 +21,7 @@ from abc import ABC import datetime import decimal +import pyarrow as pa from apache_beam.coders import Coder from apache_beam.coders.coders import FastCoder from apache_beam.typehints import typehints @@ -31,12 +32,13 @@ from pyflink.table import Row FLINK_SCALAR_FUNCTION_SCHEMA_CODER_URN = "flink:coder:schema:scalar_function:v1" FLINK_TABLE_FUNCTION_SCHEMA_CODER_URN = "flink:coder:schema:table_function:v1" +FLINK_SCALAR_FUNCTION_SCHEMA_ARROW_CODER_URN = "flink:coder:schema:scalar_function:arrow:v1" __all__ = ['FlattenRowCoder', 'RowCoder', 'BigIntCoder', 'TinyIntCoder', 'BooleanCoder', 'SmallIntCoder', 'IntCoder', 'FloatCoder', 'DoubleCoder', 'BinaryCoder', 'CharCoder', 'DateCoder', 'TimeCoder', - 'TimestampCoder', 'ArrayCoder', 'MapCoder', 'DecimalCoder'] + 'TimestampCoder', 'ArrayCoder', 'MapCoder', 'DecimalCoder', 'ArrowCoder'] class TableFunctionRowCoder(FastCoder): @@ -374,6 +376,44 @@ class TimestampCoder(DeterministicCoder): return datetime.datetime +class ArrowCoder(DeterministicCoder): + """ + Coder for Arrow. + """ + def __init__(self, schema): + self._schema = schema + + def _create_impl(self): + return coder_impl.ArrowCoderImpl(self._schema) + + def to_type_hint(self): + import pandas as pd + return pd.Series + + @Coder.register_urn(FLINK_SCALAR_FUNCTION_SCHEMA_ARROW_CODER_URN, + flink_fn_execution_pb2.Schema) + def _pickle_from_runner_api_parameter(schema_proto, unused_components, unused_context): + def _to_arrow_type(field): + if field.type.type_name == flink_fn_execution_pb2.Schema.TypeName.TINYINT: + return pa.field(field.name, pa.int8(), field.type.nullable) + elif field.type.type_name == flink_fn_execution_pb2.Schema.TypeName.SMALLINT: + return pa.field(field.name, pa.int16(), field.type.nullable) + elif field.type.type_name == flink_fn_execution_pb2.Schema.TypeName.INT: + return pa.field(field.name, pa.int32(), field.type.nullable) + elif field.type.type_name == flink_fn_execution_pb2.Schema.TypeName.BIGINT: + return pa.field(field.name, pa.int64(), field.type.nullable) + else: + raise ValueError("field_type %s is not supported." % field.type) + + def _to_arrow_schema(row_schema): + return pa.schema([_to_arrow_type(f) for f in row_schema.fields]) + + return ArrowCoder(_to_arrow_schema(schema_proto)) + + def __repr__(self): + return 'ArrowCoder[%s]' % self._schema + + type_name = flink_fn_execution_pb2.Schema.TypeName _type_name_mappings = { type_name.TINYINT: TinyIntCoder(), diff --git a/flink-python/pyflink/table/tests/test_pandas_udf.py b/flink-python/pyflink/table/tests/test_pandas_udf.py new file mode 100644 index 0000000000..188285d5be --- /dev/null +++ b/flink-python/pyflink/table/tests/test_pandas_udf.py @@ -0,0 +1,162 @@ +################################################################################ +# 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. +################################################################################ + +import unittest + +from pyflink.table import DataTypes +from pyflink.table.tests.test_udf import SubtractOne +from pyflink.table.udf import udf +from pyflink.testing import source_sink_utils +from pyflink.testing.test_case_utils import PyFlinkStreamTableTestCase, \ + PyFlinkBlinkBatchTableTestCase, PyFlinkBlinkStreamTableTestCase + + +class PandasUDFTests(unittest.TestCase): + + def test_non_exist_udf_type(self): + with self.assertRaisesRegex(ValueError, + 'The udf_type must be one of \'general, pandas\''): + udf(lambda i: i + 1, DataTypes.BIGINT(), DataTypes.BIGINT(), udf_type="non-exist") + + +class PandasUDFITTests(object): + + def test_basic_functionality(self): + # pandas UDF + self.t_env.register_function( + "add_one", + udf(lambda i: i + 1, DataTypes.BIGINT(), DataTypes.BIGINT(), udf_type="pandas")) + + self.t_env.register_function("add", add) + + # general Python UDF + self.t_env.register_function( + "subtract_one", udf(SubtractOne(), DataTypes.BIGINT(), DataTypes.BIGINT())) + + table_sink = source_sink_utils.TestAppendSink( + ['a', 'b', 'c', 'd'], + [DataTypes.BIGINT(), DataTypes.BIGINT(), DataTypes.BIGINT(), DataTypes.BIGINT()]) + self.t_env.register_table_sink("Results", table_sink) + + t = self.t_env.from_elements([(1, 2, 3), (2, 5, 6), (3, 1, 9)], ['a', 'b', 'c']) + t.where("add_one(b) <= 3") \ + .select("a, b + 1, add(a + 1, subtract_one(c)) + 2, add(add_one(a), 1L)") \ + .insert_into("Results") + self.t_env.execute("test") + actual = source_sink_utils.results() + self.assert_equals(actual, ["1,3,6,3", "3,2,14,5"]) + + def test_all_data_types(self): + import pandas as pd + import numpy as np + + def tinyint_func(tinyint_param): + assert isinstance(tinyint_param, pd.Series) + assert isinstance(tinyint_param[0], np.int8), \ + 'tinyint_param of wrong type %s !' % type(tinyint_param[0]) + return tinyint_param + + def smallint_func(smallint_param): + assert isinstance(smallint_param, pd.Series) + assert isinstance(smallint_param[0], np.int16), \ + 'smallint_param of wrong type %s !' % type(smallint_param[0]) + assert smallint_param[0] == 32767, 'smallint_param of wrong value %s' % smallint_param + return smallint_param + + def int_func(int_param): + assert isinstance(int_param, pd.Series) + assert isinstance(int_param[0], np.int32), \ + 'int_param of wrong type %s !' % type(int_param[0]) + assert int_param[0] == -2147483648, 'int_param of wrong value %s' % int_param + return int_param + + def bigint_func(bigint_param): + assert isinstance(bigint_param, pd.Series) + assert isinstance(bigint_param[0], np.int64), \ + 'bigint_param of wrong type %s !' % type(bigint_param[0]) + return bigint_param + + self.t_env.register_function( + "tinyint_func", + udf(tinyint_func, [DataTypes.TINYINT()], DataTypes.TINYINT(), udf_type="pandas")) + + self.t_env.register_function( + "smallint_func", + udf(smallint_func, [DataTypes.SMALLINT()], DataTypes.SMALLINT(), udf_type="pandas")) + + self.t_env.register_function( + "int_func", + udf(int_func, [DataTypes.INT()], DataTypes.INT(), udf_type="pandas")) + + self.t_env.register_function( + "bigint_func", + udf(bigint_func, [DataTypes.BIGINT()], DataTypes.BIGINT(), udf_type="pandas")) + + table_sink = source_sink_utils.TestAppendSink( + ['a', 'b', 'c', 'd'], + [DataTypes.TINYINT(), DataTypes.SMALLINT(), DataTypes.INT(), DataTypes.BIGINT()]) + self.t_env.register_table_sink("Results", table_sink) + + t = self.t_env.from_elements( + [(1, 32767, -2147483648, 1)], + DataTypes.ROW( + [DataTypes.FIELD("a", DataTypes.TINYINT()), + DataTypes.FIELD("b", DataTypes.SMALLINT()), + DataTypes.FIELD("c", DataTypes.INT()), + DataTypes.FIELD("d", DataTypes.BIGINT())])) + + t.select("tinyint_func(a)," + "smallint_func(b)," + "int_func(c)," + "bigint_func(d)") \ + .insert_into("Results") + self.t_env.execute("test") + actual = source_sink_utils.results() + self.assert_equals(actual, + ["1,32767,-2147483648,1"]) + + +class StreamPandasUDFITTests(PandasUDFITTests, + PyFlinkStreamTableTestCase): + pass + + +class BlinkBatchPandasUDFITTests(PandasUDFITTests, + PyFlinkBlinkBatchTableTestCase): + pass + + +class BlinkStreamPandasUDFITTests(PandasUDFITTests, + PyFlinkBlinkStreamTableTestCase): + pass + +@udf(input_types=[DataTypes.BIGINT(), DataTypes.BIGINT()], result_type=DataTypes.BIGINT(), + udf_type='pandas') +def add(i, j): + return i + j + + +if __name__ == '__main__': + import unittest + + try: + import xmlrunner + testRunner = xmlrunner.XMLTestRunner(output='target/test-reports') + except ImportError: + testRunner = None + unittest.main(testRunner=testRunner, verbosity=2) diff --git a/flink-python/pyflink/table/udf.py b/flink-python/pyflink/table/udf.py index f508b3d2fa..091c495336 100644 --- a/flink-python/pyflink/table/udf.py +++ b/flink-python/pyflink/table/udf.py @@ -172,7 +172,7 @@ class UserDefinedScalarFunctionWrapper(UserDefinedFunctionWrapper): Wrapper for Python user-defined scalar function. """ - def __init__(self, func, input_types, result_type, deterministic, name): + def __init__(self, func, input_types, result_type, udf_type, deterministic, name): super(UserDefinedScalarFunctionWrapper, self).__init__( func, input_types, deterministic, name) @@ -180,6 +180,7 @@ class UserDefinedScalarFunctionWrapper(UserDefinedFunctionWrapper): raise TypeError( "Invalid returnType: returnType should be DataType but is {}".format(result_type)) self._result_type = result_type + self._udf_type = udf_type self._judf_placeholder = None def java_user_defined_function(self, is_blink_planner, table_config): @@ -188,6 +189,18 @@ class UserDefinedScalarFunctionWrapper(UserDefinedFunctionWrapper): return self._judf_placeholder def _create_judf(self, is_blink_planner, table_config): + gateway = get_gateway() + + def get_python_function_kind(udf_type): + JPythonFunctionKind = gateway.jvm.org.apache.flink.table.functions.python.\ + PythonFunctionKind + if udf_type == "general": + return JPythonFunctionKind.GENERAL + elif udf_type == "pandas": + return JPythonFunctionKind.PANDAS + else: + raise TypeError("Unsupported udf_type: %s." % udf_type) + func = self._func if not isinstance(self._func, UserDefinedFunction): func = DelegatingScalarFunction(self._func) @@ -195,12 +208,10 @@ class UserDefinedScalarFunctionWrapper(UserDefinedFunctionWrapper): import cloudpickle serialized_func = cloudpickle.dumps(func) - gateway = get_gateway() j_input_types = utils.to_jarray(gateway.jvm.TypeInformation, [_to_java_type(i) for i in self._input_types]) j_result_type = _to_java_type(self._result_type) - j_function_kind = gateway.jvm.org.apache.flink.table.functions.python.\ - PythonFunctionKind.GENERAL + j_function_kind = get_python_function_kind(self._udf_type) if is_blink_planner: PythonTableUtils = gateway.jvm\ .org.apache.flink.table.planner.utils.python.PythonTableUtils @@ -297,15 +308,17 @@ def _get_python_env(): return gateway.jvm.org.apache.flink.table.functions.python.PythonEnv(exec_type) -def _create_udf(f, input_types, result_type, deterministic, name): - return UserDefinedScalarFunctionWrapper(f, input_types, result_type, deterministic, name) +def _create_udf(f, input_types, result_type, udf_type, deterministic, name): + return UserDefinedScalarFunctionWrapper( + f, input_types, result_type, udf_type, deterministic, name) def _create_udtf(f, input_types, result_types, deterministic, name): return UserDefinedTableFunctionWrapper(f, input_types, result_types, deterministic, name) -def udf(f=None, input_types=None, result_type=None, deterministic=None, name=None): +def udf(f=None, input_types=None, result_type=None, deterministic=None, name=None, + udf_type="general"): """ Helper method for creating a user-defined function. @@ -330,21 +343,27 @@ def udf(f=None, input_types=None, result_type=None, deterministic=None, name=Non :type input_types: list[DataType] or DataType :param result_type: the result data type. :type result_type: DataType - :param name: the function name. - :type name: str :param deterministic: the determinism of the function's results. True if and only if a call to this function is guaranteed to always return the same result given the same parameters. (default True) :type deterministic: bool + :param name: the function name. + :type name: str + :param udf_type: the type of the python function, available value: general, pandas, + (default: general) + :type udf_type: str :return: UserDefinedScalarFunctionWrapper or function. :rtype: UserDefinedScalarFunctionWrapper or function """ + if udf_type not in ('general', 'pandas'): + raise ValueError("The udf_type must be one of 'general, pandas', got %s." % udf_type) + # decorator if f is None: return functools.partial(_create_udf, input_types=input_types, result_type=result_type, - deterministic=deterministic, name=name) + udf_type=udf_type, deterministic=deterministic, name=name) else: - return _create_udf(f, input_types, result_type, deterministic, name) + return _create_udf(f, input_types, result_type, udf_type, deterministic, name) def udtf(f=None, input_types=None, result_types=None, deterministic=None, name=None): diff --git a/flink-python/setup.py b/flink-python/setup.py index 744036e4c8..ca4b5985fa 100644 --- a/flink-python/setup.py +++ b/flink-python/setup.py @@ -224,7 +224,8 @@ run sdist. author_email='dev@flink.apache.org', python_requires='>=3.5', install_requires=['py4j==0.10.8.1', 'python-dateutil==2.8.0', 'apache-beam==2.19.0', - 'cloudpickle==1.2.2', 'avro-python3>=1.8.1,<=1.9.1', 'jsonpickle==1.2'], + 'cloudpickle==1.2.2', 'avro-python3>=1.8.1,<=1.9.1', 'jsonpickle==1.2', + 'pandas>=0.23.4,<=0.25.3', 'pyarrow>=0.15.1,<=0.16.0'], tests_require=['pytest==4.4.1'], description='Apache Flink Python API', long_description=long_description, diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/AbstractArrowPythonScalarFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/AbstractArrowPythonScalarFunctionRunner.java index 80f7ded5aa..d218c8fddf 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/AbstractArrowPythonScalarFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/AbstractArrowPythonScalarFunctionRunner.java @@ -174,6 +174,7 @@ public abstract class AbstractArrowPythonScalarFunctionRunner extends Abstra private void finishCurrentBatch() throws Exception { if (currentBatchCount > 0) { arrowWriter.finish(); + // the batch of elements sent out as one row should be serialized into one arrow batch arrowStreamWriter.writeBatch(); arrowWriter.reset(); -- Gitee From 79de2ea5ab64de03b46e8ad6a0df3bbde986d124 Mon Sep 17 00:00:00 2001 From: Sivaprasanna S Date: Wed, 4 Mar 2020 17:04:21 +0530 Subject: [PATCH 108/885] [FLINK-16371][fs-connector] Make CompressWriterFactory serializable --- .../compress/CompressWriterFactory.java | 98 ++++++++++++++----- .../writers/HadoopCompressionBulkWriter.java | 30 +++--- .../writers/NoCompressionBulkWriter.java | 10 +- .../compress/CompressWriterFactoryTest.java | 80 +++++++++------ .../compress/CustomCompressionCodec.java | 32 ++++++ 5 files changed, 177 insertions(+), 73 deletions(-) create mode 100644 flink-formats/flink-compress/src/test/java/org/apache/flink/formats/compress/CustomCompressionCodec.java diff --git a/flink-formats/flink-compress/src/main/java/org/apache/flink/formats/compress/CompressWriterFactory.java b/flink-formats/flink-compress/src/main/java/org/apache/flink/formats/compress/CompressWriterFactory.java index 8d66524359..76dd4a96b8 100644 --- a/flink-formats/flink-compress/src/main/java/org/apache/flink/formats/compress/CompressWriterFactory.java +++ b/flink-formats/flink-compress/src/main/java/org/apache/flink/formats/compress/CompressWriterFactory.java @@ -24,57 +24,111 @@ import org.apache.flink.core.fs.FSDataOutputStream; import org.apache.flink.formats.compress.extractor.Extractor; import org.apache.flink.formats.compress.writers.HadoopCompressionBulkWriter; import org.apache.flink.formats.compress.writers.NoCompressionBulkWriter; -import org.apache.flink.util.Preconditions; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.CompressionCodecFactory; import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import static org.apache.flink.util.Preconditions.checkNotNull; /** - * A factory that creates a {@link BulkWriter} implementation that compresses the written data. + * A factory that creates for {@link BulkWriter bulk writers} that, when provided + * with a {@link CompressionCodec}, they compress the data they write. If no codec is + * provided, the data is written in bulk but uncompressed. * * @param The type of element to write. */ @PublicEvolving public class CompressWriterFactory implements BulkWriter.Factory { - private Extractor extractor; - private CompressionCodec hadoopCodec; + private final Extractor extractor; + private final Map hadoopConfigMap; + + private transient CompressionCodec hadoopCodec; + + private String hadoopCodecName; + private String codecExtension; + /** + * Creates a new CompressWriterFactory using the given {@link Extractor} to assemble + * either {@link HadoopCompressionBulkWriter} or {@link NoCompressionBulkWriter} + * based on whether a Hadoop CompressionCodec name is specified. + * + * @param extractor Extractor to extract the element + */ public CompressWriterFactory(Extractor extractor) { - this.extractor = Preconditions.checkNotNull(extractor, "extractor cannot be null"); + this.extractor = checkNotNull(extractor, "Extractor cannot be null"); + this.hadoopConfigMap = new HashMap<>(); } - public CompressWriterFactory withHadoopCompression(String hadoopCodecName) { - return withHadoopCompression(hadoopCodecName, new Configuration()); + /** + * Compresses the data using the provided Hadoop {@link CompressionCodec}. + * + * @param codecName Simple/complete name or alias of the CompressionCodec + * @return the instance of CompressionWriterFactory + * @throws IOException + */ + public CompressWriterFactory withHadoopCompression(String codecName) throws IOException { + return withHadoopCompression(codecName, new Configuration()); } - public CompressWriterFactory withHadoopCompression(String hadoopCodecName, Configuration hadoopConfiguration) { - return withHadoopCompression(new CompressionCodecFactory(hadoopConfiguration).getCodecByName(hadoopCodecName)); - } + /** + * Compresses the data using the provided Hadoop {@link CompressionCodec} and {@link Configuration}. + * + * @param codecName Simple/complete name or alias of the CompressionCodec + * @param hadoopConfig Hadoop Configuration + * @return the instance of CompressionWriterFactory + * @throws IOException + */ + public CompressWriterFactory withHadoopCompression(String codecName, Configuration hadoopConfig) throws IOException { + this.codecExtension = getHadoopCodecExtension(codecName, hadoopConfig); + this.hadoopCodecName = codecName; + + for (Map.Entry entry : hadoopConfig) { + hadoopConfigMap.put(entry.getKey(), entry.getValue()); + } - public CompressWriterFactory withHadoopCompression(CompressionCodec hadoopCodec) { - this.hadoopCodec = Preconditions.checkNotNull(hadoopCodec, "hadoopCodec cannot be null"); return this; } @Override public BulkWriter create(FSDataOutputStream out) throws IOException { - try { - return (hadoopCodec != null) - ? new HadoopCompressionBulkWriter<>(out, extractor, hadoopCodec) - : new NoCompressionBulkWriter<>(out, extractor); - } catch (Exception e) { - throw new IOException(e.getLocalizedMessage(), e); + if (hadoopCodecName == null || hadoopCodecName.trim().isEmpty()) { + return new NoCompressionBulkWriter<>(out, extractor); } + + initializeCompressionCodec(); + + return new HadoopCompressionBulkWriter<>(hadoopCodec.createOutputStream(out), extractor); } - public String codecExtension() { - return (hadoopCodec != null) - ? hadoopCodec.getDefaultExtension() - : ""; + public String getExtension() { + return (hadoopCodecName != null) ? this.codecExtension : ""; } + private void initializeCompressionCodec() { + if (hadoopCodec == null) { + Configuration conf = new Configuration(); + + for (Map.Entry entry : hadoopConfigMap.entrySet()) { + conf.set(entry.getKey(), entry.getValue()); + } + + hadoopCodec = new CompressionCodecFactory(conf).getCodecByName(this.hadoopCodecName); + } + } + + private String getHadoopCodecExtension(String hadoopCodecName, Configuration conf) throws IOException { + CompressionCodec codec = new CompressionCodecFactory(conf).getCodecByName(hadoopCodecName); + + if (codec == null) { + throw new IOException("Unable to load the provided Hadoop codec [" + hadoopCodecName + "]"); + } + + return codec.getDefaultExtension(); + } } diff --git a/flink-formats/flink-compress/src/main/java/org/apache/flink/formats/compress/writers/HadoopCompressionBulkWriter.java b/flink-formats/flink-compress/src/main/java/org/apache/flink/formats/compress/writers/HadoopCompressionBulkWriter.java index 798c2912f7..e62a0f0d77 100644 --- a/flink-formats/flink-compress/src/main/java/org/apache/flink/formats/compress/writers/HadoopCompressionBulkWriter.java +++ b/flink-formats/flink-compress/src/main/java/org/apache/flink/formats/compress/writers/HadoopCompressionBulkWriter.java @@ -19,48 +19,42 @@ package org.apache.flink.formats.compress.writers; import org.apache.flink.api.common.serialization.BulkWriter; -import org.apache.flink.core.fs.FSDataOutputStream; import org.apache.flink.formats.compress.extractor.Extractor; -import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.CompressionOutputStream; import java.io.IOException; +import static org.apache.flink.util.Preconditions.checkNotNull; + /** - * A {@link BulkWriter} implementation that compresses data using Hadoop codecs. + * A {@link BulkWriter} implementation that writes data that have been + * compressed using Hadoop {@link org.apache.hadoop.io.compress.CompressionCodec}. * * @param The type of element to write. */ public class HadoopCompressionBulkWriter implements BulkWriter { - private Extractor extractor; - private FSDataOutputStream outputStream; - private CompressionOutputStream compressor; + private final Extractor extractor; + private final CompressionOutputStream out; - public HadoopCompressionBulkWriter( - FSDataOutputStream outputStream, - Extractor extractor, - CompressionCodec compressionCodec) throws Exception { - this.outputStream = outputStream; - this.extractor = extractor; - this.compressor = compressionCodec.createOutputStream(outputStream); + public HadoopCompressionBulkWriter(CompressionOutputStream out, Extractor extractor) { + this.out = checkNotNull(out); + this.extractor = checkNotNull(extractor); } @Override public void addElement(T element) throws IOException { - compressor.write(extractor.extract(element)); + out.write(extractor.extract(element)); } @Override public void flush() throws IOException { - compressor.flush(); - outputStream.flush(); + out.flush(); } @Override public void finish() throws IOException { - compressor.finish(); - outputStream.sync(); + out.finish(); } } diff --git a/flink-formats/flink-compress/src/main/java/org/apache/flink/formats/compress/writers/NoCompressionBulkWriter.java b/flink-formats/flink-compress/src/main/java/org/apache/flink/formats/compress/writers/NoCompressionBulkWriter.java index 11cbbbf49f..0935f867db 100644 --- a/flink-formats/flink-compress/src/main/java/org/apache/flink/formats/compress/writers/NoCompressionBulkWriter.java +++ b/flink-formats/flink-compress/src/main/java/org/apache/flink/formats/compress/writers/NoCompressionBulkWriter.java @@ -24,6 +24,8 @@ import org.apache.flink.formats.compress.extractor.Extractor; import java.io.IOException; +import static org.apache.flink.util.Preconditions.checkNotNull; + /** * A {@link BulkWriter} implementation that does not compress data. This is essentially a no-op * writer for use with {@link org.apache.flink.formats.compress.CompressWriterFactory} for the case @@ -33,12 +35,12 @@ import java.io.IOException; */ public class NoCompressionBulkWriter implements BulkWriter { - private Extractor extractor; - private FSDataOutputStream outputStream; + private final Extractor extractor; + private final FSDataOutputStream outputStream; public NoCompressionBulkWriter(FSDataOutputStream outputStream, Extractor extractor) { - this.outputStream = outputStream; - this.extractor = extractor; + this.outputStream = checkNotNull(outputStream); + this.extractor = checkNotNull(extractor); } @Override diff --git a/flink-formats/flink-compress/src/test/java/org/apache/flink/formats/compress/CompressWriterFactoryTest.java b/flink-formats/flink-compress/src/test/java/org/apache/flink/formats/compress/CompressWriterFactoryTest.java index 2feab41403..dc46e860d8 100644 --- a/flink-formats/flink-compress/src/test/java/org/apache/flink/formats/compress/CompressWriterFactoryTest.java +++ b/flink-formats/flink-compress/src/test/java/org/apache/flink/formats/compress/CompressWriterFactoryTest.java @@ -30,12 +30,9 @@ import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.util.TestLogger; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.io.compress.BZip2Codec; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.CompressionCodecFactory; -import org.apache.hadoop.io.compress.DefaultCodec; -import org.apache.hadoop.io.compress.DeflateCodec; -import org.apache.hadoop.io.compress.GzipCodec; +import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -43,6 +40,7 @@ import org.junit.rules.TemporaryFolder; import java.io.BufferedReader; import java.io.File; import java.io.FileInputStream; +import java.io.IOException; import java.io.InputStreamReader; import java.util.Arrays; import java.util.List; @@ -59,62 +57,86 @@ public class CompressWriterFactoryTest extends TestLogger { @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + private static Configuration confWithCustomCodec; + + @BeforeClass + public static void before() { + confWithCustomCodec = new Configuration(); + confWithCustomCodec.set("io.compression.codecs", "org.apache.flink.formats.compress.CustomCompressionCodec"); + } @Test - public void testBzip2CompressByName() throws Exception { + public void testBzip2CompressByAlias() throws Exception { testCompressByName("Bzip2"); } @Test - public void testBzip2CompressCodec() throws Exception { - BZip2Codec codec = new BZip2Codec(); - codec.setConf(new Configuration()); - testCompressCodec(codec); + public void testBzip2CompressByName() throws Exception { + testCompressByName("Bzip2Codec"); } @Test - public void testGzipCompressByName() throws Exception { + public void testGzipCompressByAlias() throws Exception { testCompressByName("Gzip"); } @Test - public void testGzipCompressCodec() throws Exception { - GzipCodec codec = new GzipCodec(); - codec.setConf(new Configuration()); - testCompressCodec(codec); + public void testGzipCompressByName() throws Exception { + testCompressByName("GzipCodec"); + } + + @Test + public void testDeflateCompressByAlias() throws Exception { + testCompressByName("deflate"); } @Test - public void testDeflateCompressByName() throws Exception { - DeflateCodec codec = new DeflateCodec(); - codec.setConf(new Configuration()); - testCompressCodec(codec); + public void testDeflateCompressByClassName() throws Exception { + testCompressByName("org.apache.hadoop.io.compress.DeflateCodec"); } @Test public void testDefaultCompressByName() throws Exception { - DefaultCodec codec = new DefaultCodec(); - codec.setConf(new Configuration()); - testCompressCodec(codec); + testCompressByName("DefaultCodec"); } - private void testCompressByName(String codec) throws Exception { - CompressWriterFactory writer = CompressWriters.forExtractor(new DefaultExtractor()).withHadoopCompression(codec); - List lines = Arrays.asList("line1", "line2", "line3"); + @Test + public void testDefaultCompressByClassName() throws Exception { + testCompressByName("org.apache.hadoop.io.compress.DefaultCodec"); + } - File directory = prepareCompressedFile(writer, lines); + @Test(expected = IOException.class) + public void testCompressFailureWithUnknownCodec() throws Exception { + testCompressByName("com.bla.bla.UnknownCodec"); + } + + @Test + public void testCustomCompressionCodecByClassName() throws Exception { + testCompressByName("org.apache.flink.formats.compress.CustomCompressionCodec", confWithCustomCodec); + } - validateResults(directory, lines, new CompressionCodecFactory(new Configuration()).getCodecByName(codec)); + @Test + public void testCustomCompressionCodecByAlias() throws Exception { + testCompressByName("CustomCompressionCodec", confWithCustomCodec); + } + + @Test + public void testCustomCompressionCodecByName() throws Exception { + testCompressByName("CustomCompression", confWithCustomCodec); } - private void testCompressCodec(CompressionCodec codec) throws Exception { + private void testCompressByName(String codec) throws Exception { + testCompressByName(codec, new Configuration()); + } - CompressWriterFactory writer = CompressWriters.forExtractor(new DefaultExtractor()).withHadoopCompression(codec); + private void testCompressByName(String codec, Configuration conf) throws Exception { + CompressWriterFactory writer = CompressWriters.forExtractor(new DefaultExtractor()) + .withHadoopCompression(codec, conf); List lines = Arrays.asList("line1", "line2", "line3"); File directory = prepareCompressedFile(writer, lines); - validateResults(directory, lines, codec); + validateResults(directory, lines, new CompressionCodecFactory(conf).getCodecByName(codec)); } private File prepareCompressedFile(CompressWriterFactory writer, List lines) throws Exception { diff --git a/flink-formats/flink-compress/src/test/java/org/apache/flink/formats/compress/CustomCompressionCodec.java b/flink-formats/flink-compress/src/test/java/org/apache/flink/formats/compress/CustomCompressionCodec.java new file mode 100644 index 0000000000..e82a01872e --- /dev/null +++ b/flink-formats/flink-compress/src/test/java/org/apache/flink/formats/compress/CustomCompressionCodec.java @@ -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.flink.formats.compress; + +import org.apache.hadoop.io.compress.BZip2Codec; +import org.apache.hadoop.io.compress.CompressionCodec; + +/** + * Just a dummy class which extends BZip2Codec to verify + * that custom Hadoop {@link CompressionCodec} can also + * be successfully loaded using the class name or alias, + * just like the default ones. + */ +public class CustomCompressionCodec extends BZip2Codec { + // Seriously nothing happens here +} -- Gitee From 33c6d4b635807148e0a823bbbb597b0791dbc864 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Tue, 10 Mar 2020 12:02:54 +0100 Subject: [PATCH 109/885] [FLINK-16371][fs-connector] Add ITCase This closes #11307. --- .../compress/CompressionFactoryITCase.java | 120 ++++++++++++++++++ 1 file changed, 120 insertions(+) create mode 100644 flink-formats/flink-compress/src/test/java/org/apache/flink/formats/compress/CompressionFactoryITCase.java diff --git a/flink-formats/flink-compress/src/test/java/org/apache/flink/formats/compress/CompressionFactoryITCase.java b/flink-formats/flink-compress/src/test/java/org/apache/flink/formats/compress/CompressionFactoryITCase.java new file mode 100644 index 0000000000..b9ce9f6f2b --- /dev/null +++ b/flink-formats/flink-compress/src/test/java/org/apache/flink/formats/compress/CompressionFactoryITCase.java @@ -0,0 +1,120 @@ +/* + * 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.formats.compress; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.core.fs.Path; +import org.apache.flink.formats.compress.extractor.DefaultExtractor; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink; +import org.apache.flink.streaming.util.FiniteTestSource; +import org.apache.flink.test.util.AbstractTestBase; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.hadoop.io.compress.CompressionCodecFactory; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.Timeout; + +import java.io.BufferedReader; +import java.io.File; +import java.io.FileInputStream; +import java.io.InputStreamReader; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +/** + * Integration test case for writing bulk encoded files with the + * {@link StreamingFileSink} and Hadoop Compression Codecs. + */ +public class CompressionFactoryITCase extends AbstractTestBase { + + private final Configuration configuration = new Configuration(); + + private static final String TEST_CODEC_NAME = "Bzip2"; + + private final List testData = Arrays.asList( + "line1", + "line2", + "line3" + ); + + @Rule + public final Timeout timeoutPerTest = Timeout.seconds(20); + + @Test + public void testWriteCompressedFile() throws Exception { + final File folder = TEMPORARY_FOLDER.newFolder(); + final Path testPath = Path.fromLocalFile(folder); + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + env.enableCheckpointing(100); + + DataStream stream = env.addSource( + new FiniteTestSource<>(testData), + TypeInformation.of(String.class) + ); + + stream.map(str -> str).addSink( + StreamingFileSink.forBulkFormat( + testPath, + CompressWriters.forExtractor(new DefaultExtractor()).withHadoopCompression(TEST_CODEC_NAME) + ).build()); + + env.execute(); + + validateResults(folder, testData, new CompressionCodecFactory(configuration).getCodecByName(TEST_CODEC_NAME)); + } + + private List readFile(File file, CompressionCodec codec) throws Exception { + try ( + FileInputStream inputStream = new FileInputStream(file); + InputStreamReader readerStream = new InputStreamReader(codec.createInputStream(inputStream)); + BufferedReader reader = new BufferedReader(readerStream) + ) { + return reader.lines().collect(Collectors.toList()); + } + } + + private void validateResults(File folder, List expected, CompressionCodec codec) throws Exception { + File[] buckets = folder.listFiles(); + assertNotNull(buckets); + assertEquals(1, buckets.length); + + final File[] partFiles = buckets[0].listFiles(); + assertNotNull(partFiles); + assertEquals(2, partFiles.length); + + for (File partFile : partFiles) { + assertTrue(partFile.length() > 0); + + final List fileContent = readFile(partFile, codec); + assertEquals(expected, fileContent); + } + } +} + -- Gitee From dae17509743a46c8a30fb0e3def3b7e3a51ccba5 Mon Sep 17 00:00:00 2001 From: Roman Khachatryan Date: Fri, 24 Jan 2020 20:01:12 +0100 Subject: [PATCH 110/885] [FLINK-15782][connectors/jdbc] rename package-private JDBC classes Motivation: Future classes for JDBC XA and JDBC DML make uppercase convention inconvenient --- ...mat.java => AbstractJdbcOutputFormat.java} | 12 +++------ .../api/java/io/jdbc/JDBCAppendTableSink.java | 8 +++--- .../io/jdbc/JDBCAppendTableSinkBuilder.java | 4 +-- .../api/java/io/jdbc/JDBCLookupFunction.java | 4 +-- .../api/java/io/jdbc/JDBCLookupOptions.java | 2 +- .../api/java/io/jdbc/JDBCOutputFormat.java | 2 +- .../api/java/io/jdbc/JDBCTableSource.java | 2 +- .../api/java/io/jdbc/JDBCUpsertTableSink.java | 26 +++++++++---------- ...inkFunction.java => JdbcSinkFunction.java} | 4 +-- .../{JDBCTypeUtil.java => JdbcTypeUtil.java} | 4 +-- ...ormat.java => JdbcUpsertOutputFormat.java} | 14 +++++----- ...ction.java => JdbcUpsertSinkFunction.java} | 6 ++--- .../java/io/jdbc/JDBCAppendTableSinkTest.java | 4 +-- .../io/jdbc/JDBCUpsertTableSinkITCase.java | 2 +- ...ypeUtilTest.java => JdbcTypeUtilTest.java} | 4 +-- ...t.java => JdbcUpsertOutputFormatTest.java} | 8 +++--- 16 files changed, 50 insertions(+), 56 deletions(-) rename flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/{AbstractJDBCOutputFormat.java => AbstractJdbcOutputFormat.java} (89%) rename flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/{JDBCSinkFunction.java => JdbcSinkFunction.java} (94%) rename flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/{JDBCTypeUtil.java => JdbcTypeUtil.java} (99%) rename flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/{JDBCUpsertOutputFormat.java => JdbcUpsertOutputFormat.java} (95%) rename flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/{JDBCUpsertSinkFunction.java => JdbcUpsertSinkFunction.java} (92%) rename flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/{JDBCTypeUtilTest.java => JdbcTypeUtilTest.java} (94%) rename flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/{JDBCUpsertOutputFormatTest.java => JdbcUpsertOutputFormatTest.java} (95%) diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/AbstractJDBCOutputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/AbstractJdbcOutputFormat.java similarity index 89% rename from flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/AbstractJDBCOutputFormat.java rename to flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/AbstractJdbcOutputFormat.java index 451bafff68..647c968b9c 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/AbstractJDBCOutputFormat.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/AbstractJdbcOutputFormat.java @@ -30,19 +30,13 @@ import java.sql.Connection; import java.sql.DriverManager; import java.sql.SQLException; -/** - * OutputFormat to write Rows into a JDBC database. - * - * @see Row - * @see DriverManager - */ -public abstract class AbstractJDBCOutputFormat extends RichOutputFormat { +abstract class AbstractJdbcOutputFormat extends RichOutputFormat { private static final long serialVersionUID = 1L; static final int DEFAULT_FLUSH_MAX_SIZE = 5000; static final long DEFAULT_FLUSH_INTERVAL_MILLS = 0; - private static final Logger LOG = LoggerFactory.getLogger(AbstractJDBCOutputFormat.class); + private static final Logger LOG = LoggerFactory.getLogger(AbstractJdbcOutputFormat.class); private final String username; private final String password; @@ -51,7 +45,7 @@ public abstract class AbstractJDBCOutputFormat extends RichOutputFormat { protected transient Connection connection; - public AbstractJDBCOutputFormat(String username, String password, String drivername, String dbURL) { + public AbstractJdbcOutputFormat(String username, String password, String drivername, String dbURL) { this.username = username; this.password = password; this.drivername = drivername; diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSink.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSink.java index b5bc34cc47..4fa8c56419 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSink.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSink.java @@ -63,7 +63,7 @@ public class JDBCAppendTableSink implements AppendStreamTableSink, BatchTab @Override public DataStreamSink consumeDataStream(DataStream dataStream) { return dataStream - .addSink(new JDBCSinkFunction(outputFormat)) + .addSink(new JdbcSinkFunction(outputFormat)) .setParallelism(dataStream.getParallelism()) .name(TableConnectorUtils.generateRuntimeName(this.getClass(), fieldNames)); } @@ -93,16 +93,16 @@ public class JDBCAppendTableSink implements AppendStreamTableSink, BatchTab int[] types = outputFormat.getTypesArray(); String sinkSchema = - String.join(", ", IntStream.of(types).mapToObj(JDBCTypeUtil::getTypeName).collect(Collectors.toList())); + String.join(", ", IntStream.of(types).mapToObj(JdbcTypeUtil::getTypeName).collect(Collectors.toList())); String tableSchema = - String.join(", ", Stream.of(fieldTypes).map(JDBCTypeUtil::getTypeName).collect(Collectors.toList())); + String.join(", ", Stream.of(fieldTypes).map(JdbcTypeUtil::getTypeName).collect(Collectors.toList())); String msg = String.format("Schema of output table is incompatible with JDBCAppendTableSink schema. " + "Table schema: [%s], sink schema: [%s]", tableSchema, sinkSchema); Preconditions.checkArgument(fieldTypes.length == types.length, msg); for (int i = 0; i < types.length; ++i) { Preconditions.checkArgument( - JDBCTypeUtil.typeInformationToSqlType(fieldTypes[i]) == types[i], + JdbcTypeUtil.typeInformationToSqlType(fieldTypes[i]) == types[i], msg); } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSinkBuilder.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSinkBuilder.java index 023c6240d7..ec966acfb0 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSinkBuilder.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSinkBuilder.java @@ -21,7 +21,7 @@ package org.apache.flink.api.java.io.jdbc; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.util.Preconditions; -import static org.apache.flink.api.java.io.jdbc.AbstractJDBCOutputFormat.DEFAULT_FLUSH_MAX_SIZE; +import static org.apache.flink.api.java.io.jdbc.AbstractJdbcOutputFormat.DEFAULT_FLUSH_MAX_SIZE; /** * A builder to configure and build the JDBCAppendTableSink. @@ -100,7 +100,7 @@ public class JDBCAppendTableSinkBuilder { public JDBCAppendTableSinkBuilder setParameterTypes(TypeInformation... types) { int[] ty = new int[types.length]; for (int i = 0; i < types.length; ++i) { - ty[i] = JDBCTypeUtil.typeInformationToSqlType(types[i]); + ty[i] = JdbcTypeUtil.typeInformationToSqlType(types[i]); } this.parameterTypes = ty; return this; diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCLookupFunction.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCLookupFunction.java index 80d612712b..4cfa4984f3 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCLookupFunction.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCLookupFunction.java @@ -99,8 +99,8 @@ public class JDBCLookupFunction extends TableFunction { this.cacheMaxSize = lookupOptions.getCacheMaxSize(); this.cacheExpireMs = lookupOptions.getCacheExpireMs(); this.maxRetryTimes = lookupOptions.getMaxRetryTimes(); - this.keySqlTypes = Arrays.stream(keyTypes).mapToInt(JDBCTypeUtil::typeInformationToSqlType).toArray(); - this.outputSqlTypes = Arrays.stream(fieldTypes).mapToInt(JDBCTypeUtil::typeInformationToSqlType).toArray(); + this.keySqlTypes = Arrays.stream(keyTypes).mapToInt(JdbcTypeUtil::typeInformationToSqlType).toArray(); + this.outputSqlTypes = Arrays.stream(fieldTypes).mapToInt(JdbcTypeUtil::typeInformationToSqlType).toArray(); this.query = options.getDialect().getSelectFromStatement( options.getTableName(), fieldNames, keyNames); } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCLookupOptions.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCLookupOptions.java index 61373244fc..6a35e8e9a3 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCLookupOptions.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCLookupOptions.java @@ -21,7 +21,7 @@ package org.apache.flink.api.java.io.jdbc; import java.io.Serializable; import java.util.Objects; -import static org.apache.flink.api.java.io.jdbc.JDBCUpsertOutputFormat.DEFAULT_MAX_RETRY_TIMES; +import static org.apache.flink.api.java.io.jdbc.JdbcUpsertOutputFormat.DEFAULT_MAX_RETRY_TIMES; /** * Options for the JDBC lookup. diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java index 0893d7f5b6..4b202aefd3 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java @@ -37,7 +37,7 @@ import static org.apache.flink.api.java.io.jdbc.JDBCUtils.setRecordToStatement; * @see Row * @see DriverManager */ -public class JDBCOutputFormat extends AbstractJDBCOutputFormat { +public class JDBCOutputFormat extends AbstractJdbcOutputFormat { private static final long serialVersionUID = 1L; diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCTableSource.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCTableSource.java index bbd5561465..60339e2276 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCTableSource.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCTableSource.java @@ -37,7 +37,7 @@ import org.apache.flink.types.Row; import java.util.Arrays; import java.util.Objects; -import static org.apache.flink.api.java.io.jdbc.JDBCTypeUtil.normalizeTableSchema; +import static org.apache.flink.api.java.io.jdbc.JdbcTypeUtil.normalizeTableSchema; import static org.apache.flink.table.types.utils.TypeConversions.fromDataTypeToLegacyInfo; import static org.apache.flink.util.Preconditions.checkNotNull; diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSink.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSink.java index 77dac7b400..5ba385d959 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSink.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSink.java @@ -36,10 +36,10 @@ import org.apache.flink.types.Row; import java.util.Arrays; import java.util.Objects; -import static org.apache.flink.api.java.io.jdbc.AbstractJDBCOutputFormat.DEFAULT_FLUSH_INTERVAL_MILLS; -import static org.apache.flink.api.java.io.jdbc.AbstractJDBCOutputFormat.DEFAULT_FLUSH_MAX_SIZE; -import static org.apache.flink.api.java.io.jdbc.JDBCTypeUtil.normalizeTableSchema; -import static org.apache.flink.api.java.io.jdbc.JDBCUpsertOutputFormat.DEFAULT_MAX_RETRY_TIMES; +import static org.apache.flink.api.java.io.jdbc.AbstractJdbcOutputFormat.DEFAULT_FLUSH_INTERVAL_MILLS; +import static org.apache.flink.api.java.io.jdbc.AbstractJdbcOutputFormat.DEFAULT_FLUSH_MAX_SIZE; +import static org.apache.flink.api.java.io.jdbc.JdbcTypeUtil.normalizeTableSchema; +import static org.apache.flink.api.java.io.jdbc.JdbcUpsertOutputFormat.DEFAULT_MAX_RETRY_TIMES; import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -57,11 +57,11 @@ public class JDBCUpsertTableSink implements UpsertStreamTableSink { private boolean isAppendOnly; private JDBCUpsertTableSink( - TableSchema schema, - JDBCOptions options, - int flushMaxSize, - long flushIntervalMills, - int maxRetryTime) { + TableSchema schema, + JDBCOptions options, + int flushMaxSize, + long flushIntervalMills, + int maxRetryTime) { this.schema = TableSchemaUtils.checkNoGeneratedColumns(schema); this.options = options; this.flushMaxSize = flushMaxSize; @@ -69,16 +69,16 @@ public class JDBCUpsertTableSink implements UpsertStreamTableSink { this.maxRetryTime = maxRetryTime; } - private JDBCUpsertOutputFormat newFormat() { + private JdbcUpsertOutputFormat newFormat() { if (!isAppendOnly && (keyFields == null || keyFields.length == 0)) { throw new UnsupportedOperationException("JDBCUpsertTableSink can not support "); } // sql types int[] jdbcSqlTypes = Arrays.stream(schema.getFieldTypes()) - .mapToInt(JDBCTypeUtil::typeInformationToSqlType).toArray(); + .mapToInt(JdbcTypeUtil::typeInformationToSqlType).toArray(); - return JDBCUpsertOutputFormat.builder() + return JdbcUpsertOutputFormat.builder() .setOptions(options) .setFieldNames(schema.getFieldNames()) .setFlushMaxSize(flushMaxSize) @@ -92,7 +92,7 @@ public class JDBCUpsertTableSink implements UpsertStreamTableSink { @Override public DataStreamSink consumeDataStream(DataStream> dataStream) { return dataStream - .addSink(new JDBCUpsertSinkFunction(newFormat())) + .addSink(new JdbcUpsertSinkFunction(newFormat())) .setParallelism(dataStream.getParallelism()) .name(TableConnectorUtils.generateRuntimeName(this.getClass(), schema.getFieldNames())); } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCSinkFunction.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcSinkFunction.java similarity index 94% rename from flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCSinkFunction.java rename to flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcSinkFunction.java index d2fdef6e50..da3f6119a8 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCSinkFunction.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcSinkFunction.java @@ -26,10 +26,10 @@ import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.types.Row; -class JDBCSinkFunction extends RichSinkFunction implements CheckpointedFunction { +class JdbcSinkFunction extends RichSinkFunction implements CheckpointedFunction { final JDBCOutputFormat outputFormat; - JDBCSinkFunction(JDBCOutputFormat outputFormat) { + JdbcSinkFunction(JDBCOutputFormat outputFormat) { this.outputFormat = outputFormat; } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCTypeUtil.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcTypeUtil.java similarity index 99% rename from flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCTypeUtil.java rename to flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcTypeUtil.java index db5b56fa77..d3b2bb9fdc 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCTypeUtil.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcTypeUtil.java @@ -45,7 +45,7 @@ import static org.apache.flink.api.common.typeinfo.BasicTypeInfo.SHORT_TYPE_INFO import static org.apache.flink.api.common.typeinfo.BasicTypeInfo.STRING_TYPE_INFO; import static org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO; -class JDBCTypeUtil { +class JdbcTypeUtil { private static final Map, Integer> TYPE_MAPPING; private static final Map SQL_TYPE_NAMES; @@ -84,7 +84,7 @@ class JDBCTypeUtil { SQL_TYPE_NAMES = Collections.unmodifiableMap(names); } - private JDBCTypeUtil() { + private JdbcTypeUtil() { } static int typeInformationToSqlType(TypeInformation type) { diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertOutputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcUpsertOutputFormat.java similarity index 95% rename from flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertOutputFormat.java rename to flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcUpsertOutputFormat.java index 1736c28f73..48767d4442 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertOutputFormat.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcUpsertOutputFormat.java @@ -41,11 +41,11 @@ import static org.apache.flink.util.Preconditions.checkNotNull; /** * An upsert OutputFormat for JDBC. */ -public class JDBCUpsertOutputFormat extends AbstractJDBCOutputFormat> { +class JdbcUpsertOutputFormat extends AbstractJdbcOutputFormat> { private static final long serialVersionUID = 1L; - private static final Logger LOG = LoggerFactory.getLogger(JDBCUpsertOutputFormat.class); + private static final Logger LOG = LoggerFactory.getLogger(JdbcUpsertOutputFormat.class); static final int DEFAULT_MAX_RETRY_TIMES = 3; @@ -68,7 +68,7 @@ public class JDBCUpsertOutputFormat extends AbstractJDBCOutputFormat { - synchronized (JDBCUpsertOutputFormat.this) { + synchronized (JdbcUpsertOutputFormat.this) { if (closed) { return; } @@ -212,7 +212,7 @@ public class JDBCUpsertOutputFormat extends AbstractJDBCOutputFormat> implements CheckpointedFunction { - private final JDBCUpsertOutputFormat outputFormat; +class JdbcUpsertSinkFunction extends RichSinkFunction> implements CheckpointedFunction { + private final JdbcUpsertOutputFormat outputFormat; - JDBCUpsertSinkFunction(JDBCUpsertOutputFormat outputFormat) { + JdbcUpsertSinkFunction(JdbcUpsertOutputFormat outputFormat) { this.outputFormat = outputFormat; } diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSinkTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSinkTest.java index b3f2b249f5..f76ca62747 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSinkTest.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSinkTest.java @@ -71,9 +71,9 @@ public class JDBCAppendTableSinkTest { .getStreamGraph(StreamExecutionEnvironment.DEFAULT_JOB_NAME, false) .getStreamNode(sinkId) .getOperator(); - assertTrue(planSink.getUserFunction() instanceof JDBCSinkFunction); + assertTrue(planSink.getUserFunction() instanceof JdbcSinkFunction); - JDBCSinkFunction sinkFunction = (JDBCSinkFunction) planSink.getUserFunction(); + JdbcSinkFunction sinkFunction = (JdbcSinkFunction) planSink.getUserFunction(); assertSame(sink.getOutputFormat(), sinkFunction.outputFormat); } diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSinkITCase.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSinkITCase.java index 240c45d4d3..901138e267 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSinkITCase.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSinkITCase.java @@ -42,7 +42,7 @@ import java.util.Collections; import java.util.List; import static org.apache.flink.api.java.io.jdbc.JDBCTestBase.DRIVER_CLASS; -import static org.apache.flink.api.java.io.jdbc.JDBCUpsertOutputFormatTest.check; +import static org.apache.flink.api.java.io.jdbc.JdbcUpsertOutputFormatTest.check; /** * IT case for {@link JDBCUpsertTableSink}. diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTypeUtilTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTypeUtilTest.java similarity index 94% rename from flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTypeUtilTest.java rename to flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTypeUtilTest.java index 790be78aed..6bb17d1c10 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTypeUtilTest.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTypeUtilTest.java @@ -26,14 +26,14 @@ import org.junit.Test; import java.sql.Types; -import static org.apache.flink.api.java.io.jdbc.JDBCTypeUtil.typeInformationToSqlType; +import static org.apache.flink.api.java.io.jdbc.JdbcTypeUtil.typeInformationToSqlType; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; /** * Testing the type conversions from Flink to SQL types. */ -public class JDBCTypeUtilTest { +public class JdbcTypeUtilTest { @Test public void testTypeConversions() { diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertOutputFormatTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcUpsertOutputFormatTest.java similarity index 95% rename from flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertOutputFormatTest.java rename to flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcUpsertOutputFormatTest.java index 225ab1ae15..b0ff204f3a 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertOutputFormatTest.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcUpsertOutputFormatTest.java @@ -43,11 +43,11 @@ import static org.junit.Assert.assertArrayEquals; import static org.mockito.Mockito.doReturn; /** - * Tests for the {@link JDBCUpsertOutputFormat}. + * Tests for the {@link JdbcUpsertOutputFormat}. */ -public class JDBCUpsertOutputFormatTest extends JDBCTestBase { +public class JdbcUpsertOutputFormatTest extends JDBCTestBase { - private JDBCUpsertOutputFormat format; + private JdbcUpsertOutputFormat format; private String[] fieldNames; private String[] keyFields; @@ -59,7 +59,7 @@ public class JDBCUpsertOutputFormatTest extends JDBCTestBase { @Test public void testJDBCOutputFormat() throws Exception { - format = JDBCUpsertOutputFormat.builder() + format = JdbcUpsertOutputFormat.builder() .setOptions(JDBCOptions.builder() .setDBUrl(DB_URL) .setTableName(OUTPUT_TABLE) -- Gitee From b41e0d1a69bec95c6645f1a4825764e2548e8462 Mon Sep 17 00:00:00 2001 From: Roman Khachatryan Date: Tue, 14 Jan 2020 00:43:27 +0100 Subject: [PATCH 111/885] [FLINK-15782][connectors/jdbc] introduce parameter objects Motivated partially by future use in public API --- .../io/jdbc/AbstractJdbcOutputFormat.java | 21 +-- .../api/java/io/jdbc/JDBCAppendTableSink.java | 2 +- .../flink/api/java/io/jdbc/JDBCOptions.java | 31 +--- .../api/java/io/jdbc/JDBCOutputFormat.java | 56 +++---- .../api/java/io/jdbc/JdbcBatchOptions.java | 77 ++++++++++ .../java/io/jdbc/JdbcConnectionOptions.java | 94 ++++++++++++ .../api/java/io/jdbc/JdbcDmlOptions.java | 141 ++++++++++++++++++ .../api/java/io/jdbc/JdbcInsertOptions.java | 54 +++++++ .../java/io/jdbc/JdbcTypedQueryOptions.java | 47 ++++++ .../java/io/jdbc/JdbcUpsertOutputFormat.java | 65 ++++---- .../java/io/jdbc/dialect/JDBCDialects.java | 4 +- .../java/io/jdbc/JDBCOutputFormatTest.java | 2 +- 12 files changed, 494 insertions(+), 100 deletions(-) create mode 100644 flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcBatchOptions.java create mode 100644 flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcConnectionOptions.java create mode 100644 flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcDmlOptions.java create mode 100644 flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcInsertOptions.java create mode 100644 flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcTypedQueryOptions.java diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/AbstractJdbcOutputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/AbstractJdbcOutputFormat.java index 647c968b9c..22282fbe13 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/AbstractJdbcOutputFormat.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/AbstractJdbcOutputFormat.java @@ -20,7 +20,6 @@ package org.apache.flink.api.java.io.jdbc; import org.apache.flink.api.common.io.RichOutputFormat; import org.apache.flink.configuration.Configuration; -import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,18 +37,12 @@ abstract class AbstractJdbcOutputFormat extends RichOutputFormat { private static final Logger LOG = LoggerFactory.getLogger(AbstractJdbcOutputFormat.class); - private final String username; - private final String password; - private final String drivername; - protected final String dbURL; + private final JdbcConnectionOptions options; protected transient Connection connection; - public AbstractJdbcOutputFormat(String username, String password, String drivername, String dbURL) { - this.username = username; - this.password = password; - this.drivername = drivername; - this.dbURL = dbURL; + public AbstractJdbcOutputFormat(JdbcConnectionOptions options) { + this.options = options; } @Override @@ -57,11 +50,11 @@ abstract class AbstractJdbcOutputFormat extends RichOutputFormat { } protected void establishConnection() throws SQLException, ClassNotFoundException { - Class.forName(drivername); - if (username == null) { - connection = DriverManager.getConnection(dbURL); + Class.forName(options.getDriverName()); + if (options.username == null) { + connection = DriverManager.getConnection(options.url); } else { - connection = DriverManager.getConnection(dbURL, username, password); + connection = DriverManager.getConnection(options.url, options.username, options.password); } } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSink.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSink.java index 4fa8c56419..fde2585ef6 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSink.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSink.java @@ -90,7 +90,7 @@ public class JDBCAppendTableSink implements AppendStreamTableSink, BatchTab @Override public TableSink configure(String[] fieldNames, TypeInformation[] fieldTypes) { - int[] types = outputFormat.getTypesArray(); + int[] types = outputFormat.getFieldTypes(); String sinkSchema = String.join(", ", IntStream.of(types).mapToObj(JdbcTypeUtil::getTypeName).collect(Collectors.toList())); diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOptions.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOptions.java index f68b14a9f7..6ccb063b0b 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOptions.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOptions.java @@ -29,45 +29,24 @@ import static org.apache.flink.util.Preconditions.checkNotNull; /** * Common options of {@link JDBCScanOptions} and {@link JDBCLookupOptions} for the JDBC connector. */ -public class JDBCOptions { +public class JDBCOptions extends JdbcConnectionOptions { + + private static final long serialVersionUID = 1L; - private String dbURL; private String tableName; - private String driverName; - private String username; - private String password; private JDBCDialect dialect; private JDBCOptions(String dbURL, String tableName, String driverName, String username, String password, JDBCDialect dialect) { - this.dbURL = dbURL; + super(dbURL, driverName, username, password); this.tableName = tableName; - this.driverName = driverName; - this.username = username; - this.password = password; this.dialect = dialect; } - public String getDbURL() { - return dbURL; - } - public String getTableName() { return tableName; } - public String getDriverName() { - return driverName; - } - - public String getUsername() { - return username; - } - - public String getPassword() { - return password; - } - public JDBCDialect getDialect() { return dialect; } @@ -80,7 +59,7 @@ public class JDBCOptions { public boolean equals(Object o) { if (o instanceof JDBCOptions) { JDBCOptions options = (JDBCOptions) o; - return Objects.equals(dbURL, options.dbURL) && + return Objects.equals(url, options.url) && Objects.equals(tableName, options.tableName) && Objects.equals(driverName, options.driverName) && Objects.equals(username, options.username) && diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java index 4b202aefd3..0a10e24ec2 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java @@ -43,19 +43,26 @@ public class JDBCOutputFormat extends AbstractJdbcOutputFormat { private static final Logger LOG = LoggerFactory.getLogger(JDBCOutputFormat.class); - private final String query; - private final int batchInterval; - private final int[] typesArray; + final JdbcInsertOptions insertOptions; + private final JdbcBatchOptions batchOptions; private PreparedStatement upload; private int batchCount = 0; - public JDBCOutputFormat(String username, String password, String drivername, - String dbURL, String query, int batchInterval, int[] typesArray) { - super(username, password, drivername, dbURL); - this.query = query; - this.batchInterval = batchInterval; - this.typesArray = typesArray; + /** + * @deprecated use {@link #JDBCOutputFormat(JdbcConnectionOptions, JdbcInsertOptions, JdbcBatchOptions)}}. + */ + @Deprecated + public JDBCOutputFormat(String username, String password, String drivername, String dbURL, String query, int batchInterval, int[] typesArray) { + this(new JdbcConnectionOptions(dbURL, drivername, username, password), + new JdbcInsertOptions(query, typesArray), + JdbcBatchOptions.builder().withSize(batchInterval).build()); + } + + public JDBCOutputFormat(JdbcConnectionOptions connectionOptions, JdbcInsertOptions insertOptions, JdbcBatchOptions batchOptions) { + super(connectionOptions); + this.insertOptions = insertOptions; + this.batchOptions = batchOptions; } /** @@ -69,7 +76,7 @@ public class JDBCOutputFormat extends AbstractJdbcOutputFormat { public void open(int taskNumber, int numTasks) throws IOException { try { establishConnection(); - upload = connection.prepareStatement(query); + upload = connection.prepareStatement(insertOptions.getQuery()); } catch (SQLException sqe) { throw new IllegalArgumentException("open() failed.", sqe); } catch (ClassNotFoundException cnfe) { @@ -80,7 +87,7 @@ public class JDBCOutputFormat extends AbstractJdbcOutputFormat { @Override public void writeRecord(Row row) throws IOException { try { - setRecordToStatement(upload, typesArray, row); + setRecordToStatement(upload, insertOptions.getFieldTypes(), row); upload.addBatch(); } catch (SQLException e) { throw new RuntimeException("Preparation of JDBC statement failed.", e); @@ -88,7 +95,7 @@ public class JDBCOutputFormat extends AbstractJdbcOutputFormat { batchCount++; - if (batchCount >= batchInterval) { + if (batchCount >= batchOptions.getSize()) { // execute batch flush(); } @@ -104,7 +111,7 @@ public class JDBCOutputFormat extends AbstractJdbcOutputFormat { } int[] getTypesArray() { - return typesArray; + return insertOptions.getFieldTypes(); } /** @@ -132,6 +139,10 @@ public class JDBCOutputFormat extends AbstractJdbcOutputFormat { return new JDBCOutputFormatBuilder(); } + public int[] getFieldTypes() { + return insertOptions.getFieldTypes(); + } + /** * Builder for a {@link JDBCOutputFormat}. */ @@ -187,25 +198,20 @@ public class JDBCOutputFormat extends AbstractJdbcOutputFormat { * @return Configured JDBCOutputFormat */ public JDBCOutputFormat finish() { + return new JDBCOutputFormat(buildConnectionOptions(), + new JdbcInsertOptions(query, typesArray), + JdbcBatchOptions.builder().withSize(batchInterval).build()); + } + + public JdbcConnectionOptions buildConnectionOptions() { if (this.username == null) { LOG.info("Username was not supplied."); } if (this.password == null) { LOG.info("Password was not supplied."); } - if (this.dbURL == null) { - throw new IllegalArgumentException("No database URL supplied."); - } - if (this.query == null) { - throw new IllegalArgumentException("No query supplied."); - } - if (this.drivername == null) { - throw new IllegalArgumentException("No driver supplied."); - } - return new JDBCOutputFormat( - username, password, drivername, dbURL, - query, batchInterval, typesArray); + return new JdbcConnectionOptions(dbURL, drivername, username, password); } } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcBatchOptions.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcBatchOptions.java new file mode 100644 index 0000000000..69b14de265 --- /dev/null +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcBatchOptions.java @@ -0,0 +1,77 @@ +/* + * 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.api.java.io.jdbc; + +import org.apache.flink.annotation.PublicEvolving; + +import java.io.Serializable; + +/** + * JDBC sink batch options. + */ +@PublicEvolving +public class JdbcBatchOptions implements Serializable { + private static final int DEFAULT_INTERVAL_MILLIS = 0; + private static final int DEFAULT_SIZE = 5000; + + private final long intervalMs; + private final int size; + + private JdbcBatchOptions(long intervalMs, int size) { + this.intervalMs = intervalMs; + this.size = size; + } + + public long getIntervalMs() { + return intervalMs; + } + + public int getSize() { + return size; + } + + public static JdbcBatchOptionsBuilder builder() { + return new JdbcBatchOptionsBuilder(); + } + + public static JdbcBatchOptions defaults() { + return builder().build(); + } + + /** + * JDBCBatchOptionsBuilder. + */ + public static final class JdbcBatchOptionsBuilder { + private long intervalMs = DEFAULT_INTERVAL_MILLIS; + private int size = DEFAULT_SIZE; + + public JdbcBatchOptionsBuilder withSize(int size) { + this.size = size; + return this; + } + + public JdbcBatchOptionsBuilder withIntervalMs(long intervalMs) { + this.intervalMs = intervalMs; + return this; + } + + public JdbcBatchOptions build() { + return new JdbcBatchOptions(intervalMs, size); + } + } +} diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcConnectionOptions.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcConnectionOptions.java new file mode 100644 index 0000000000..7c5d2fbd48 --- /dev/null +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcConnectionOptions.java @@ -0,0 +1,94 @@ +/* + * 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.api.java.io.jdbc; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.util.Preconditions; + +import java.io.Serializable; + +/** + * JDBC connection options. + */ +@PublicEvolving +public class JdbcConnectionOptions implements Serializable { + + private static final long serialVersionUID = 1L; + + protected final String url; + protected final String driverName; + protected final String username; + protected final String password; + + public JdbcConnectionOptions(String url, String driverName, String username, String password) { + this.url = Preconditions.checkNotNull(url, "jdbc url is empty"); + this.driverName = Preconditions.checkNotNull(driverName, "driver name is empty"); + this.username = username; + this.password = password; + } + + public String getDbURL() { + return url; + } + + public String getDriverName() { + return driverName; + } + + public String getUsername() { + return username; + } + + public String getPassword() { + return password; + } + + /** + * JDBCConnectionOptionsBuilder. + */ + public static class JdbcConnectionOptionsBuilder { + private String url; + private String driverName; + private String username; + private String password; + + public JdbcConnectionOptionsBuilder withUrl(String url) { + this.url = url; + return this; + } + + public JdbcConnectionOptionsBuilder withDriverName(String driverName) { + this.driverName = driverName; + return this; + } + + public JdbcConnectionOptionsBuilder withUsername(String username) { + this.username = username; + return this; + } + + public JdbcConnectionOptionsBuilder withPassword(String password) { + this.password = password; + return this; + } + + public JdbcConnectionOptions build() { + return new JdbcConnectionOptions(url, driverName, username, password); + } + } +} diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcDmlOptions.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcDmlOptions.java new file mode 100644 index 0000000000..361d4f4fff --- /dev/null +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcDmlOptions.java @@ -0,0 +1,141 @@ +/* + * 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.api.java.io.jdbc; + +import org.apache.flink.api.java.io.jdbc.dialect.JDBCDialect; +import org.apache.flink.util.Preconditions; + +import javax.annotation.Nullable; + +import java.util.stream.Stream; + +/** + * JDBC sink DML options. + */ +public class JdbcDmlOptions extends JdbcTypedQueryOptions { + + private static final long serialVersionUID = 1L; + + private static final int DEFAULT_MAX_RETRY_TIMES = 3; + + private final String[] fieldNames; + @Nullable + private final String[] keyFields; + private final String tableName; + private final JDBCDialect dialect; + private final int maxRetries; + + public static JDBCUpsertOptionsBuilder builder() { + return new JDBCUpsertOptionsBuilder(); + } + + private JdbcDmlOptions(String tableName, JDBCDialect dialect, String[] fieldNames, int[] fieldTypes, String[] keyFields, int maxRetries) { + super(fieldTypes); + this.tableName = Preconditions.checkNotNull(tableName, "table is empty"); + this.dialect = Preconditions.checkNotNull(dialect, "dialect name is empty"); + this.fieldNames = Preconditions.checkNotNull(fieldNames, "field names is empty"); + this.keyFields = keyFields; + this.maxRetries = maxRetries; + } + + public int getMaxRetries() { + return maxRetries; + } + + public String getTableName() { + return tableName; + } + + public JDBCDialect getDialect() { + Preconditions.checkNotNull(dialect, "dialect not set"); + return dialect; + } + + public String[] getFieldNames() { + return fieldNames; + } + + public String[] getKeyFields() { + return keyFields; + } + + /** + * JDBCUpsertOptionsBuilder. + */ + public static class JDBCUpsertOptionsBuilder extends JDBCUpdateQueryOptionsBuilder { + private String tableName; + private String[] fieldNames; + private String[] keyFields; + private JDBCDialect dialect; + private JDBCDialect customDialect; + private int maxRetries = DEFAULT_MAX_RETRY_TIMES; + + public JDBCUpsertOptionsBuilder withMaxRetries(int maxRetries) { + this.maxRetries = maxRetries; + return this; + } + + @Override + protected JDBCUpsertOptionsBuilder self() { + return this; + } + + public JDBCUpsertOptionsBuilder withFieldNames(String field, String... fieldNames) { + this.fieldNames = concat(field, fieldNames); + return this; + } + + public JDBCUpsertOptionsBuilder withFieldNames(String[] fieldNames) { + this.fieldNames = fieldNames; + return this; + } + + public JDBCUpsertOptionsBuilder withKeyFields(String keyField, String... keyFields) { + this.keyFields = concat(keyField, keyFields); + return this; + } + + public JDBCUpsertOptionsBuilder withKeyFields(String[] keyFields) { + this.keyFields = keyFields; + return this; + } + + public JDBCUpsertOptionsBuilder withTableName(String tableName) { + this.tableName = tableName; + return self(); + } + + public JDBCUpsertOptionsBuilder withDialect(JDBCDialect dialect) { + this.dialect = dialect; + return self(); + } + + public JdbcDmlOptions build() { + return new JdbcDmlOptions(tableName, dialect, fieldNames, fieldTypes, keyFields, maxRetries); + } + + static String[] concat(String first, String... next) { + if (next == null || next.length == 0) { + return new String[]{first}; + } else { + return Stream.concat(Stream.of(new String[]{first}), Stream.of(next)).toArray(String[]::new); + } + } + + } +} diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcInsertOptions.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcInsertOptions.java new file mode 100644 index 0000000000..90e3c4dba1 --- /dev/null +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcInsertOptions.java @@ -0,0 +1,54 @@ +/* + * 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.api.java.io.jdbc; + +import org.apache.flink.util.Preconditions; + +import java.util.stream.IntStream; + +/** + * JDBC sink insert options. + */ +public class JdbcInsertOptions extends JdbcTypedQueryOptions { + + private static final long serialVersionUID = 1L; + + private final String query; + + JdbcInsertOptions(String query, int[] typesArray) { + super(typesArray); + this.query = Preconditions.checkNotNull(query, "query is empty"); + } + + public String getQuery() { + return query; + } + + public static JdbcInsertOptions from(String query, int firstFieldType, int... nextFieldTypes) { + return new JdbcInsertOptions(query, concat(firstFieldType, nextFieldTypes)); + } + + private static int[] concat(int first, int... next) { + if (next == null || next.length == 0) { + return new int[]{first}; + } else { + return IntStream.concat(IntStream.of(new int[]{first}), IntStream.of(next)).toArray(); + } + } + +} diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcTypedQueryOptions.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcTypedQueryOptions.java new file mode 100644 index 0000000000..b4b409a482 --- /dev/null +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcTypedQueryOptions.java @@ -0,0 +1,47 @@ +/* + * 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.api.java.io.jdbc; + +import javax.annotation.Nullable; + +import java.io.Serializable; + +abstract class JdbcTypedQueryOptions implements Serializable { + + @Nullable + private final int[] fieldTypes; + + JdbcTypedQueryOptions(int[] fieldTypes) { + this.fieldTypes = fieldTypes; + } + + int[] getFieldTypes() { + return fieldTypes; + } + + public abstract static class JDBCUpdateQueryOptionsBuilder> { + int[] fieldTypes; + + protected abstract T self(); + + T withFieldTypes(int[] fieldTypes) { + this.fieldTypes = fieldTypes; + return self(); + } + } +} diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcUpsertOutputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcUpsertOutputFormat.java index 48767d4442..dfa2af234c 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcUpsertOutputFormat.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcUpsertOutputFormat.java @@ -49,25 +49,22 @@ class JdbcUpsertOutputFormat extends AbstractJdbcOutputFormat { @@ -127,7 +126,7 @@ class JdbcUpsertOutputFormat extends AbstractJdbcOutputFormat tuple2) throws IOException { + public synchronized void writeRecord(Tuple2 tuple2) { checkFlushException(); try { Tuple2 record = objectReuse ? new Tuple2<>(tuple2.f0, Row.copy(tuple2.f1)) : tuple2; - jdbcWriter.addRecord(record); + jdbcWriter.addRecord(tuple2); batchCount++; - if (batchCount >= flushMaxSize) { + if (batchCount >= batchOptions.getSize()) { flush(); } } catch (Exception e) { @@ -156,14 +155,14 @@ class JdbcUpsertOutputFormat extends AbstractJdbcOutputFormat= maxRetryTimes) { + if (i >= upsertOptions.getMaxRetries()) { throw e; } Thread.sleep(1000 * i); @@ -289,7 +288,11 @@ class JdbcUpsertOutputFormat extends AbstractJdbcOutputFormat Date: Thu, 23 Jan 2020 14:40:12 +0100 Subject: [PATCH 112/885] [FLINK-15782][connectors/jdbc] refactor JDBC connector impl. Motivation: use existing code oriented to Table API in DataStream API Changes: 1. make JDBC writers more general PreparedStatement executors (still operating on Rows) 2. isolate Delete operation specific to Table API 3. generalize JdbcUpsertOutputFormat and rename to JdbcBatchingOutputFormat --- .../io/jdbc/AbstractJdbcOutputFormat.java | 18 +- .../api/java/io/jdbc/JDBCLookupOptions.java | 2 +- .../flink/api/java/io/jdbc/JDBCOptions.java | 2 +- .../api/java/io/jdbc/JDBCOutputFormat.java | 15 +- .../api/java/io/jdbc/JDBCUpsertTableSink.java | 6 +- ...mat.java => JdbcBatchingOutputFormat.java} | 155 +++++------ .../java/io/jdbc/JdbcUpsertSinkFunction.java | 4 +- .../java/io/jdbc/TableJdbcOutputFormat.java | 100 +++++++ .../executor/InsertOrUpdateJdbcExecutor.java | 112 ++++++++ .../JdbcBatchStatementExecutor.java} | 51 +++- .../executor/KeyedBatchStatementExecutor.java | 97 +++++++ .../SimpleBatchStatementExecutor.java} | 57 ++-- .../api/java/io/jdbc/writer/UpsertWriter.java | 257 ------------------ .../java/io/jdbc/JDBCOutputFormatTest.java | 4 +- .../io/jdbc/JDBCUpsertTableSinkITCase.java | 2 +- ...st.java => JdbcTableOutputFormatTest.java} | 20 +- 16 files changed, 495 insertions(+), 407 deletions(-) rename flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/{JdbcUpsertOutputFormat.java => JdbcBatchingOutputFormat.java} (62%) create mode 100644 flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/TableJdbcOutputFormat.java create mode 100644 flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/InsertOrUpdateJdbcExecutor.java rename flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/{writer/JDBCWriter.java => executor/JdbcBatchStatementExecutor.java} (45%) create mode 100644 flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/KeyedBatchStatementExecutor.java rename flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/{writer/AppendOnlyWriter.java => executor/SimpleBatchStatementExecutor.java} (53%) delete mode 100644 flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/writer/UpsertWriter.java rename flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/{JdbcUpsertOutputFormatTest.java => JdbcTableOutputFormatTest.java} (88%) diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/AbstractJdbcOutputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/AbstractJdbcOutputFormat.java index 22282fbe13..6d806bbae2 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/AbstractJdbcOutputFormat.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/AbstractJdbcOutputFormat.java @@ -49,7 +49,21 @@ abstract class AbstractJdbcOutputFormat extends RichOutputFormat { public void configure(Configuration parameters) { } - protected void establishConnection() throws SQLException, ClassNotFoundException { + @Override + public void open(int taskNumber, int numTasks) throws IOException { + try { + establishConnection(); + } catch (SQLException | ClassNotFoundException e) { + throw new IOException("unable to open JDBC writer", e); + } + } + + @Override + public void close() { + closeDbConnection(); + } + + private void establishConnection() throws SQLException, ClassNotFoundException { Class.forName(options.getDriverName()); if (options.username == null) { connection = DriverManager.getConnection(options.url); @@ -58,7 +72,7 @@ abstract class AbstractJdbcOutputFormat extends RichOutputFormat { } } - protected void closeDbConnection() throws IOException { + private void closeDbConnection() { if (connection != null) { try { connection.close(); diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCLookupOptions.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCLookupOptions.java index 6a35e8e9a3..62d9d92e4e 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCLookupOptions.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCLookupOptions.java @@ -21,7 +21,7 @@ package org.apache.flink.api.java.io.jdbc; import java.io.Serializable; import java.util.Objects; -import static org.apache.flink.api.java.io.jdbc.JdbcUpsertOutputFormat.DEFAULT_MAX_RETRY_TIMES; +import static org.apache.flink.api.java.io.jdbc.JdbcBatchingOutputFormat.DEFAULT_MAX_RETRY_TIMES; /** * Options for the JDBC lookup. diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOptions.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOptions.java index 6ccb063b0b..3eb86c536b 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOptions.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOptions.java @@ -27,7 +27,7 @@ import java.util.Optional; import static org.apache.flink.util.Preconditions.checkNotNull; /** - * Common options of {@link JDBCScanOptions} and {@link JDBCLookupOptions} for the JDBC connector. + * Options for the JDBC connector. */ public class JDBCOptions extends JdbcConnectionOptions { diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java index 0a10e24ec2..217dbf562e 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java @@ -46,8 +46,8 @@ public class JDBCOutputFormat extends AbstractJdbcOutputFormat { final JdbcInsertOptions insertOptions; private final JdbcBatchOptions batchOptions; - private PreparedStatement upload; - private int batchCount = 0; + private transient PreparedStatement upload; + private transient int batchCount = 0; /** * @deprecated use {@link #JDBCOutputFormat(JdbcConnectionOptions, JdbcInsertOptions, JdbcBatchOptions)}}. @@ -74,18 +74,16 @@ public class JDBCOutputFormat extends AbstractJdbcOutputFormat { */ @Override public void open(int taskNumber, int numTasks) throws IOException { + super.open(taskNumber, numTasks); try { - establishConnection(); upload = connection.prepareStatement(insertOptions.getQuery()); } catch (SQLException sqe) { throw new IllegalArgumentException("open() failed.", sqe); - } catch (ClassNotFoundException cnfe) { - throw new IllegalArgumentException("JDBC driver class not found.", cnfe); } } @Override - public void writeRecord(Row row) throws IOException { + public void writeRecord(Row row) { try { setRecordToStatement(upload, insertOptions.getFieldTypes(), row); upload.addBatch(); @@ -117,10 +115,9 @@ public class JDBCOutputFormat extends AbstractJdbcOutputFormat { /** * Executes prepared statement and closes all resources of this instance. * - * @throws IOException Thrown, if the input could not be closed properly. */ @Override - public void close() throws IOException { + public void close() { if (upload != null) { flush(); try { @@ -132,7 +129,7 @@ public class JDBCOutputFormat extends AbstractJdbcOutputFormat { } } - closeDbConnection(); + super.close(); } public static JDBCOutputFormatBuilder buildJDBCOutputFormat() { diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSink.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSink.java index 5ba385d959..75854622c9 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSink.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSink.java @@ -38,8 +38,8 @@ import java.util.Objects; import static org.apache.flink.api.java.io.jdbc.AbstractJdbcOutputFormat.DEFAULT_FLUSH_INTERVAL_MILLS; import static org.apache.flink.api.java.io.jdbc.AbstractJdbcOutputFormat.DEFAULT_FLUSH_MAX_SIZE; +import static org.apache.flink.api.java.io.jdbc.JdbcBatchingOutputFormat.DEFAULT_MAX_RETRY_TIMES; import static org.apache.flink.api.java.io.jdbc.JdbcTypeUtil.normalizeTableSchema; -import static org.apache.flink.api.java.io.jdbc.JdbcUpsertOutputFormat.DEFAULT_MAX_RETRY_TIMES; import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -69,7 +69,7 @@ public class JDBCUpsertTableSink implements UpsertStreamTableSink { this.maxRetryTime = maxRetryTime; } - private JdbcUpsertOutputFormat newFormat() { + private JdbcBatchingOutputFormat newFormat() { if (!isAppendOnly && (keyFields == null || keyFields.length == 0)) { throw new UnsupportedOperationException("JDBCUpsertTableSink can not support "); } @@ -78,7 +78,7 @@ public class JDBCUpsertTableSink implements UpsertStreamTableSink { int[] jdbcSqlTypes = Arrays.stream(schema.getFieldTypes()) .mapToInt(JdbcTypeUtil::typeInformationToSqlType).toArray(); - return JdbcUpsertOutputFormat.builder() + return JdbcBatchingOutputFormat.builder() .setOptions(options) .setFieldNames(schema.getFieldNames()) .setFlushMaxSize(flushMaxSize) diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcUpsertOutputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcBatchingOutputFormat.java similarity index 62% rename from flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcUpsertOutputFormat.java rename to flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcBatchingOutputFormat.java index dfa2af234c..213ad3627a 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcUpsertOutputFormat.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcBatchingOutputFormat.java @@ -19,12 +19,8 @@ package org.apache.flink.api.java.io.jdbc; import org.apache.flink.api.java.io.jdbc.dialect.JDBCDialect; -import org.apache.flink.api.java.io.jdbc.writer.AppendOnlyWriter; -import org.apache.flink.api.java.io.jdbc.writer.JDBCWriter; -import org.apache.flink.api.java.io.jdbc.writer.UpsertWriter; -import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.io.jdbc.executor.JdbcBatchStatementExecutor; import org.apache.flink.runtime.util.ExecutorThreadFactory; -import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,34 +34,30 @@ import java.util.concurrent.TimeUnit; import static org.apache.flink.util.Preconditions.checkNotNull; -/** - * An upsert OutputFormat for JDBC. - */ -class JdbcUpsertOutputFormat extends AbstractJdbcOutputFormat> { +abstract class JdbcBatchingOutputFormat> extends AbstractJdbcOutputFormat { private static final long serialVersionUID = 1L; - private static final Logger LOG = LoggerFactory.getLogger(JdbcUpsertOutputFormat.class); + private static final Logger LOG = LoggerFactory.getLogger(JdbcBatchingOutputFormat.class); static final int DEFAULT_MAX_RETRY_TIMES = 3; - private final JdbcDmlOptions upsertOptions; + protected final JdbcDmlOptions dmlOptions; private final JdbcBatchOptions batchOptions; - private transient JDBCWriter jdbcWriter; + protected transient JdbcExec jdbcStatementExecutor; private transient int batchCount = 0; private transient volatile boolean closed = false; private transient ScheduledExecutorService scheduler; private transient ScheduledFuture scheduledFuture; private transient volatile Exception flushException; - private transient boolean objectReuse; /** - * @deprecated use {@link #JdbcUpsertOutputFormat(JdbcConnectionOptions, JdbcDmlOptions, JdbcBatchOptions)} + * @deprecated use {@link #JdbcBatchingOutputFormat(JdbcConnectionOptions, JdbcDmlOptions, JdbcBatchOptions)} */ @Deprecated - public JdbcUpsertOutputFormat( + public JdbcBatchingOutputFormat( JDBCOptions options, String[] fieldNames, String[] keyFields, @@ -79,9 +71,9 @@ class JdbcUpsertOutputFormat extends AbstractJdbcOutputFormat { - synchronized (JdbcUpsertOutputFormat.this) { - if (closed) { - return; - } - try { - flush(); - } catch (Exception e) { - flushException = e; + synchronized (JdbcBatchingOutputFormat.this) { + if (!closed) { + try { + flush(); + } catch (Exception e) { + flushException = e; + } } } }, batchOptions.getIntervalMs(), batchOptions.getIntervalMs(), TimeUnit.MILLISECONDS); @@ -137,12 +114,11 @@ class JdbcUpsertOutputFormat extends AbstractJdbcOutputFormat tuple2) { + public final synchronized void writeRecord(In record) { checkFlushException(); try { - Tuple2 record = objectReuse ? new Tuple2<>(tuple2.f0, Row.copy(tuple2.f1)) : tuple2; - jdbcWriter.addRecord(tuple2); + doWriteRecord(record); batchCount++; if (batchCount >= batchOptions.getSize()) { flush(); @@ -152,17 +128,25 @@ class JdbcUpsertOutputFormat extends AbstractJdbcOutputFormat= upsertOptions.getMaxRetries()) { + if (i >= dmlOptions.getMaxRetries()) { throw e; } Thread.sleep(1000 * i); @@ -170,40 +154,41 @@ class JdbcUpsertOutputFormat extends AbstractJdbcOutputFormat 0) { - try { - flush(); - } catch (Exception e) { - throw new RuntimeException("Writing records to JDBC failed.", e); + if (batchCount > 0) { + try { + flush(); + } catch (Exception e) { + throw new RuntimeException("Writing records to JDBC failed.", e); + } } - } - try { - jdbcWriter.close(); - } catch (SQLException e) { - LOG.warn("Close JDBC writer failed.", e); + try { + jdbcStatementExecutor.close(); + } catch (SQLException e) { + LOG.warn("Close JDBC writer failed.", e); + } } - - closeDbConnection(); + super.close(); } public static Builder builder() { @@ -211,7 +196,7 @@ class JdbcUpsertOutputFormat extends AbstractJdbcOutputFormat> implements CheckpointedFunction { - private final JdbcUpsertOutputFormat outputFormat; + private final JdbcBatchingOutputFormat outputFormat; - JdbcUpsertSinkFunction(JdbcUpsertOutputFormat outputFormat) { + JdbcUpsertSinkFunction(JdbcBatchingOutputFormat outputFormat) { this.outputFormat = outputFormat; } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/TableJdbcOutputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/TableJdbcOutputFormat.java new file mode 100644 index 0000000000..305cf20dfe --- /dev/null +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/TableJdbcOutputFormat.java @@ -0,0 +1,100 @@ +/* + * 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.api.java.io.jdbc; + +import org.apache.flink.api.java.io.jdbc.dialect.JDBCDialect; +import org.apache.flink.api.java.io.jdbc.executor.JdbcBatchStatementExecutor; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.types.Row; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.sql.SQLException; +import java.util.Arrays; + +class TableJdbcOutputFormat extends JdbcBatchingOutputFormat, Row, JdbcBatchStatementExecutor> { + private static final Logger LOG = LoggerFactory.getLogger(TableJdbcOutputFormat.class); + + private JdbcBatchStatementExecutor deleteExecutor; + + TableJdbcOutputFormat(JdbcConnectionOptions connectionOptions, JdbcDmlOptions dmlOptions, JdbcBatchOptions batchOptions) { + super(connectionOptions, dmlOptions, batchOptions); + } + + @Override + public void open(int taskNumber, int numTasks) throws IOException { + super.open(taskNumber, numTasks); + deleteExecutor = createDeleteExecutor(); + try { + deleteExecutor.open(connection); + } catch (SQLException e) { + throw new IOException(e); + } + } + + private JdbcBatchStatementExecutor createDeleteExecutor() { + int[] pkFields = Arrays.stream(dmlOptions.getFieldNames()).mapToInt(Arrays.asList(dmlOptions.getFieldNames())::indexOf).toArray(); + int[] pkTypes = dmlOptions.getFieldTypes() == null ? null : + Arrays.stream(pkFields).map(f -> dmlOptions.getFieldTypes()[f]).toArray(); + String deleteSql = dmlOptions.getDialect().getDeleteStatement(dmlOptions.getTableName(), dmlOptions.getFieldNames()); + boolean objectReuseEnabled = getRuntimeContext().getExecutionConfig().isObjectReuseEnabled(); + return JdbcBatchStatementExecutor.keyed(pkFields, pkTypes, deleteSql, objectReuseEnabled); + } + + @Override + JdbcBatchStatementExecutor createStatementRunner(JDBCDialect dialect) { + return JdbcBatchStatementExecutor.upsert( + dialect, dmlOptions.getTableName(), dmlOptions.getFieldNames(), dmlOptions.getFieldTypes(), dmlOptions.getKeyFields(), + getRuntimeContext().getExecutionConfig().isObjectReuseEnabled()); + } + + @Override + void doWriteRecord(Tuple2 record) throws SQLException { + if (record.f0) { + super.doWriteRecord(record); + } else { + deleteExecutor.process(extractJdbcRecord(record)); + } + } + + @Override + Row extractJdbcRecord(Tuple2 record) { + return record.f1; + } + + @Override + public synchronized void close() { + try { + super.close(); + } finally { + try { + deleteExecutor.close(); + } catch (SQLException e) { + LOG.warn("unable to close delete statement runner", e); + } + } + } + + @Override + void attemptFlush() throws SQLException { + super.attemptFlush(); + deleteExecutor.executeBatch(); + } +} diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/InsertOrUpdateJdbcExecutor.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/InsertOrUpdateJdbcExecutor.java new file mode 100644 index 0000000000..ba37707002 --- /dev/null +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/InsertOrUpdateJdbcExecutor.java @@ -0,0 +1,112 @@ +/* + * 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.api.java.io.jdbc.executor; + +import org.apache.flink.types.Row; + +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +import static org.apache.flink.api.java.io.jdbc.JDBCUtils.setRecordToStatement; + +final class InsertOrUpdateJdbcExecutor implements JdbcBatchStatementExecutor { + + private final String existSQL; + private final String insertSQL; + private final String updateSQL; + private final int[] pkTypes; + private final int[] pkFields; + private final boolean objectReuse; + + private transient PreparedStatement existStatement; + private transient PreparedStatement insertStatement; + private transient PreparedStatement updateStatement; + private transient Map keyToRows = new HashMap<>(); + private final int[] fieldTypes; + + InsertOrUpdateJdbcExecutor( + int[] fieldTypes, + int[] pkFields, int[] pkTypes, + String existSQL, + String insertSQL, + String updateSQL, boolean objectReuse) { + this.pkFields = pkFields; + this.existSQL = existSQL; + this.insertSQL = insertSQL; + this.updateSQL = updateSQL; + this.fieldTypes = fieldTypes; + this.pkTypes = pkTypes; + this.objectReuse = objectReuse; + } + + @Override + public void open(Connection connection) throws SQLException { + keyToRows = new HashMap<>(); + existStatement = connection.prepareStatement(existSQL); + insertStatement = connection.prepareStatement(insertSQL); + updateStatement = connection.prepareStatement(updateSQL); + } + + @Override + public void process(Row record) { + keyToRows.put(KeyedBatchStatementExecutor.getPrimaryKey(record, pkFields), objectReuse ? Row.copy(record) : record); + } + + @Override + public void executeBatch() throws SQLException { + if (keyToRows.size() > 0) { + for (Map.Entry entry : keyToRows.entrySet()) { + processOneRowInBatch(entry.getKey(), entry.getValue()); + } + updateStatement.executeBatch(); + insertStatement.executeBatch(); + keyToRows.clear(); + } + } + + private void processOneRowInBatch(Row pk, Row row) throws SQLException { + if (exist(pk)) { + setRecordToStatement(updateStatement, fieldTypes, row); + updateStatement.addBatch(); + } else { + setRecordToStatement(insertStatement, fieldTypes, row); + insertStatement.addBatch(); + } + } + + private boolean exist(Row pk) throws SQLException { + setRecordToStatement(existStatement, pkTypes, pk); + try (ResultSet resultSet = existStatement.executeQuery()) { + return resultSet.next(); + } + } + + @Override + public void close() throws SQLException { + for (PreparedStatement s : Arrays.asList(existStatement, insertStatement, updateStatement)) { + if (s != null) { + s.close(); + } + } + } +} diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/writer/JDBCWriter.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/JdbcBatchStatementExecutor.java similarity index 45% rename from flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/writer/JDBCWriter.java rename to flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/JdbcBatchStatementExecutor.java index 1a6797a61d..f231a739a2 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/writer/JDBCWriter.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/JdbcBatchStatementExecutor.java @@ -16,29 +16,30 @@ * limitations under the License. */ -package org.apache.flink.api.java.io.jdbc.writer; +package org.apache.flink.api.java.io.jdbc.executor; -import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.java.io.jdbc.dialect.JDBCDialect; import org.apache.flink.types.Row; -import java.io.Serializable; import java.sql.Connection; import java.sql.SQLException; +import java.util.Arrays; + +import static org.apache.flink.util.Preconditions.checkNotNull; /** * JDBCWriter used to execute statements (e.g. INSERT, UPSERT, DELETE). */ -public interface JDBCWriter extends Serializable { +@Internal +public interface JdbcBatchStatementExecutor { /** * Open the writer by JDBC Connection. It can create Statement from Connection. */ void open(Connection connection) throws SQLException; - /** - * Add record to writer, the writer may cache the data. - */ - void addRecord(Tuple2 record); + void process(T record) throws SQLException; /** * Submits a batch of commands to the database for execution. @@ -49,4 +50,38 @@ public interface JDBCWriter extends Serializable { * Close JDBC related statements and other classes. */ void close() throws SQLException; + + static JdbcBatchStatementExecutor upsert( + JDBCDialect dialect, + String tableName, + String[] fieldNames, + int[] fieldTypes, + String[] keyFields, + boolean objectReuse) { + + checkNotNull(keyFields); + + int[] pkFields = Arrays.stream(keyFields).mapToInt(Arrays.asList(fieldNames)::indexOf).toArray(); + int[] pkTypes = fieldTypes == null ? null : Arrays.stream(pkFields).map(f -> fieldTypes[f]).toArray(); + + return dialect + .getUpsertStatement(tableName, fieldNames, keyFields) + .map(sql -> keyed(pkFields, pkTypes, sql, objectReuse)) + .orElseGet(() -> + new InsertOrUpdateJdbcExecutor( + fieldTypes, pkFields, pkTypes, + dialect.getRowExistsStatement(tableName, keyFields), + dialect.getInsertIntoStatement(tableName, fieldNames), + dialect.getUpdateStatement(tableName, fieldNames, keyFields), + objectReuse)); + } + + static JdbcBatchStatementExecutor keyed(int[] pkFields, int[] pkTypes, String sql, boolean objectReuse) { + return new KeyedBatchStatementExecutor(pkFields, pkTypes, sql, objectReuse); + } + + static JdbcBatchStatementExecutor simple(String sql, int[] fieldTypes, boolean objectReuse) { + return new SimpleBatchStatementExecutor(sql, fieldTypes, objectReuse); + } + } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/KeyedBatchStatementExecutor.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/KeyedBatchStatementExecutor.java new file mode 100644 index 0000000000..70ab6793c6 --- /dev/null +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/KeyedBatchStatementExecutor.java @@ -0,0 +1,97 @@ +/* + * 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.api.java.io.jdbc.executor; + +import org.apache.flink.types.Row; + +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.SQLException; +import java.util.HashMap; +import java.util.Map; + +import static org.apache.flink.api.java.io.jdbc.JDBCUtils.setRecordToStatement; + +/** + * Upsert writer to deal with upsert, delete message. + */ +class KeyedBatchStatementExecutor implements JdbcBatchStatementExecutor { + + private final int[] pkTypes; + private final int[] pkFields; + private final String sql; + private final boolean objectReuse; + + private transient Map keyToRows = new HashMap<>(); + private transient PreparedStatement st; + + KeyedBatchStatementExecutor(int[] pkFields, int[] pkTypes, String sql, boolean objectReuse) { + this.pkFields = pkFields; + this.pkTypes = pkTypes; + this.sql = sql; + this.objectReuse = objectReuse; + } + + @Override + public void open(Connection connection) throws SQLException { + keyToRows = new HashMap<>(); + st = connection.prepareStatement(sql); + } + + @Override + public void process(Row record) { + // we don't need perform a deep copy, because jdbc field are immutable object. + Row row = objectReuse ? Row.copy(record) : record; + // add records to buffer + keyToRows.put(getPrimaryKey(row), row); + } + + @Override + public void executeBatch() throws SQLException { + if (keyToRows.size() > 0) { + for (Map.Entry entry : keyToRows.entrySet()) { + setRecordToStatement(st, pkTypes, entry.getKey()); + st.addBatch(); + } + st.executeBatch(); + keyToRows.clear(); + } + } + + @Override + public void close() throws SQLException { + if (st != null) { + st.close(); + st = null; + } + } + + private Row getPrimaryKey(Row row) { + int[] pkFields = this.pkFields; + return getPrimaryKey(row, pkFields); + } + + static Row getPrimaryKey(Row row, int[] pkFields) { + Row pkRow = new Row(pkFields.length); + for (int i = 0; i < pkFields.length; i++) { + pkRow.setField(i, row.getField(pkFields[i])); + } + return pkRow; + } +} diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/writer/AppendOnlyWriter.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/SimpleBatchStatementExecutor.java similarity index 53% rename from flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/writer/AppendOnlyWriter.java rename to flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/SimpleBatchStatementExecutor.java index ff07874a18..16325ecc5b 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/writer/AppendOnlyWriter.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/SimpleBatchStatementExecutor.java @@ -16,9 +16,8 @@ * limitations under the License. */ -package org.apache.flink.api.java.io.jdbc.writer; +package org.apache.flink.api.java.io.jdbc.executor; -import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.types.Row; import java.sql.Connection; @@ -28,55 +27,53 @@ import java.util.ArrayList; import java.util.List; import static org.apache.flink.api.java.io.jdbc.JDBCUtils.setRecordToStatement; -import static org.apache.flink.util.Preconditions.checkArgument; -/** - * Just append record to jdbc, can not receive retract/delete message. - */ -public class AppendOnlyWriter implements JDBCWriter { - - private static final long serialVersionUID = 1L; +class SimpleBatchStatementExecutor implements JdbcBatchStatementExecutor { - private final String insertSQL; - private final int[] fieldTypes; + private final String sql; + private final int[] paramTypes; - private transient List cachedRows; - private transient PreparedStatement statement; + private transient PreparedStatement st; + private List batch; + private final boolean objectReuse; - public AppendOnlyWriter(String insertSQL, int[] fieldTypes) { - this.insertSQL = insertSQL; - this.fieldTypes = fieldTypes; + SimpleBatchStatementExecutor(String sql, int[] paramTypes, boolean objectReuse) { + this.sql = sql; + this.paramTypes = paramTypes; + this.objectReuse = objectReuse; } @Override public void open(Connection connection) throws SQLException { - this.cachedRows = new ArrayList<>(); - this.statement = connection.prepareStatement(insertSQL); + this.batch = new ArrayList<>(); + this.st = connection.prepareStatement(sql); } @Override - public void addRecord(Tuple2 record) { - checkArgument(record.f0, "Append mode can not receive retract/delete message."); - cachedRows.add(record.f1); + public void process(Row record) { + batch.add(objectReuse ? Row.copy(record) : record); } @Override public void executeBatch() throws SQLException { - if (cachedRows.size() > 0) { - for (Row row : cachedRows) { - setRecordToStatement(statement, fieldTypes, row); - statement.addBatch(); + if (!batch.isEmpty()) { + for (Row r : batch) { + setRecordToStatement(st, paramTypes, r); + st.addBatch(); } - statement.executeBatch(); - cachedRows.clear(); + st.executeBatch(); + batch.clear(); } } @Override public void close() throws SQLException { - if (statement != null) { - statement.close(); - statement = null; + if (st != null) { + st.close(); + st = null; + } + if (batch != null) { + batch.clear(); } } } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/writer/UpsertWriter.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/writer/UpsertWriter.java deleted file mode 100644 index 7d07b95d3d..0000000000 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/writer/UpsertWriter.java +++ /dev/null @@ -1,257 +0,0 @@ -/* - * 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.api.java.io.jdbc.writer; - -import org.apache.flink.api.java.io.jdbc.dialect.JDBCDialect; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.types.Row; - -import java.sql.Connection; -import java.sql.PreparedStatement; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.function.Function; - -import static org.apache.flink.api.java.io.jdbc.JDBCUtils.setRecordToStatement; -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * Upsert writer to deal with upsert, delete message. - */ -public abstract class UpsertWriter implements JDBCWriter { - - private static final long serialVersionUID = 1L; - - public static UpsertWriter create( - JDBCDialect dialect, - String tableName, - String[] fieldNames, - int[] fieldTypes, - String[] keyFields) { - - checkNotNull(keyFields); - - List nameList = Arrays.asList(fieldNames); - int[] pkFields = Arrays.stream(keyFields).mapToInt(nameList::indexOf).toArray(); - int[] pkTypes = fieldTypes == null ? null : - Arrays.stream(pkFields).map(f -> fieldTypes[f]).toArray(); - String deleteSQL = dialect.getDeleteStatement(tableName, keyFields); - - Optional upsertSQL = dialect.getUpsertStatement(tableName, fieldNames, keyFields); - return upsertSQL.map((Function) sql -> - new UpsertWriterUsingUpsertStatement( - fieldTypes, pkFields, pkTypes, deleteSQL, sql)) - .orElseGet(() -> - new UpsertWriterUsingInsertUpdateStatement( - fieldTypes, pkFields, pkTypes, deleteSQL, - dialect.getRowExistsStatement(tableName, keyFields), - dialect.getInsertIntoStatement(tableName, fieldNames), - dialect.getUpdateStatement(tableName, fieldNames, keyFields))); - } - - final int[] fieldTypes; - final int[] pkTypes; - private final int[] pkFields; - private final String deleteSQL; - - private transient Map> keyToRows; - private transient PreparedStatement deleteStatement; - - private UpsertWriter(int[] fieldTypes, int[] pkFields, int[] pkTypes, String deleteSQL) { - this.fieldTypes = fieldTypes; - this.pkFields = pkFields; - this.pkTypes = pkTypes; - this.deleteSQL = deleteSQL; - } - - @Override - public void open(Connection connection) throws SQLException { - this.keyToRows = new HashMap<>(); - this.deleteStatement = connection.prepareStatement(deleteSQL); - } - - public void addRecord(Tuple2 record) { - // add records to buffer - keyToRows.put(getPrimaryKey(record.f1), record); - } - - @Override - public void executeBatch() throws SQLException { - if (keyToRows.size() > 0) { - for (Map.Entry> entry : keyToRows.entrySet()) { - Row pk = entry.getKey(); - Tuple2 tuple = entry.getValue(); - if (tuple.f0) { - processOneRowInBatch(pk, tuple.f1); - } else { - setRecordToStatement(deleteStatement, pkTypes, pk); - deleteStatement.addBatch(); - } - } - internalExecuteBatch(); - deleteStatement.executeBatch(); - keyToRows.clear(); - } - } - - abstract void processOneRowInBatch(Row pk, Row row) throws SQLException; - - abstract void internalExecuteBatch() throws SQLException; - - @Override - public void close() throws SQLException { - if (deleteStatement != null) { - deleteStatement.close(); - deleteStatement = null; - } - } - - private Row getPrimaryKey(Row row) { - Row pks = new Row(pkFields.length); - for (int i = 0; i < pkFields.length; i++) { - pks.setField(i, row.getField(pkFields[i])); - } - return pks; - } - - // ---------------------------------------------------------------------------------------- - - private static final class UpsertWriterUsingUpsertStatement extends UpsertWriter { - - private static final long serialVersionUID = 1L; - private final String upsertSQL; - - private transient PreparedStatement upsertStatement; - - private UpsertWriterUsingUpsertStatement( - int[] fieldTypes, - int[] pkFields, - int[] pkTypes, - String deleteSQL, - String upsertSQL) { - super(fieldTypes, pkFields, pkTypes, deleteSQL); - this.upsertSQL = upsertSQL; - } - - @Override - public void open(Connection connection) throws SQLException { - super.open(connection); - upsertStatement = connection.prepareStatement(upsertSQL); - } - - @Override - void processOneRowInBatch(Row pk, Row row) throws SQLException { - setRecordToStatement(upsertStatement, fieldTypes, row); - upsertStatement.addBatch(); - } - - @Override - void internalExecuteBatch() throws SQLException { - upsertStatement.executeBatch(); - } - - @Override - public void close() throws SQLException { - super.close(); - if (upsertStatement != null) { - upsertStatement.close(); - upsertStatement = null; - } - } - } - - private static final class UpsertWriterUsingInsertUpdateStatement extends UpsertWriter { - - private static final long serialVersionUID = 1L; - private final String existSQL; - private final String insertSQL; - private final String updateSQL; - - private transient PreparedStatement existStatement; - private transient PreparedStatement insertStatement; - private transient PreparedStatement updateStatement; - - private UpsertWriterUsingInsertUpdateStatement( - int[] fieldTypes, - int[] pkFields, - int[] pkTypes, - String deleteSQL, - String existSQL, - String insertSQL, - String updateSQL) { - super(fieldTypes, pkFields, pkTypes, deleteSQL); - this.existSQL = existSQL; - this.insertSQL = insertSQL; - this.updateSQL = updateSQL; - } - - @Override - public void open(Connection connection) throws SQLException { - super.open(connection); - existStatement = connection.prepareStatement(existSQL); - insertStatement = connection.prepareStatement(insertSQL); - updateStatement = connection.prepareStatement(updateSQL); - } - - @Override - void processOneRowInBatch(Row pk, Row row) throws SQLException { - setRecordToStatement(existStatement, pkTypes, pk); - ResultSet resultSet = existStatement.executeQuery(); - boolean exist = resultSet.next(); - resultSet.close(); - if (exist) { - // do update - setRecordToStatement(updateStatement, fieldTypes, row); - updateStatement.addBatch(); - } else { - // do insert - setRecordToStatement(insertStatement, fieldTypes, row); - insertStatement.addBatch(); - } - } - - @Override - void internalExecuteBatch() throws SQLException { - updateStatement.executeBatch(); - insertStatement.executeBatch(); - } - - @Override - public void close() throws SQLException { - super.close(); - if (existStatement != null) { - existStatement.close(); - existStatement = null; - } - if (insertStatement != null) { - insertStatement.close(); - insertStatement = null; - } - if (updateStatement != null) { - updateStatement.close(); - updateStatement = null; - } - } - } -} diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormatTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormatTest.java index e3c8af5d9b..22a208fa22 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormatTest.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormatTest.java @@ -50,7 +50,7 @@ public class JDBCOutputFormatTest extends JDBCTestBase { jdbcOutputFormat = null; } - @Test(expected = IllegalArgumentException.class) + @Test(expected = IOException.class) public void testInvalidDriver() throws IOException { jdbcOutputFormat = JDBCOutputFormat.buildJDBCOutputFormat() .setDrivername("org.apache.derby.jdbc.idontexist") @@ -60,7 +60,7 @@ public class JDBCOutputFormatTest extends JDBCTestBase { jdbcOutputFormat.open(0, 1); } - @Test(expected = IllegalArgumentException.class) + @Test(expected = IOException.class) public void testInvalidURL() throws IOException { jdbcOutputFormat = JDBCOutputFormat.buildJDBCOutputFormat() .setDrivername(DRIVER_CLASS) diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSinkITCase.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSinkITCase.java index 901138e267..6f4f810b20 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSinkITCase.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSinkITCase.java @@ -42,7 +42,7 @@ import java.util.Collections; import java.util.List; import static org.apache.flink.api.java.io.jdbc.JDBCTestBase.DRIVER_CLASS; -import static org.apache.flink.api.java.io.jdbc.JdbcUpsertOutputFormatTest.check; +import static org.apache.flink.api.java.io.jdbc.JdbcTableOutputFormatTest.check; /** * IT case for {@link JDBCUpsertTableSink}. diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcUpsertOutputFormatTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTableOutputFormatTest.java similarity index 88% rename from flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcUpsertOutputFormatTest.java rename to flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTableOutputFormatTest.java index b0ff204f3a..2473f7d39d 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcUpsertOutputFormatTest.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTableOutputFormatTest.java @@ -43,11 +43,11 @@ import static org.junit.Assert.assertArrayEquals; import static org.mockito.Mockito.doReturn; /** - * Tests for the {@link JdbcUpsertOutputFormat}. + * Tests for the {@link JdbcBatchingOutputFormat}. */ -public class JdbcUpsertOutputFormatTest extends JDBCTestBase { +public class JdbcTableOutputFormatTest extends JDBCTestBase { - private JdbcUpsertOutputFormat format; + private TableJdbcOutputFormat format; private String[] fieldNames; private String[] keyFields; @@ -59,14 +59,14 @@ public class JdbcUpsertOutputFormatTest extends JDBCTestBase { @Test public void testJDBCOutputFormat() throws Exception { - format = JdbcUpsertOutputFormat.builder() - .setOptions(JDBCOptions.builder() - .setDBUrl(DB_URL) - .setTableName(OUTPUT_TABLE) - .build()) - .setFieldNames(fieldNames) - .setKeyFields(keyFields) + JDBCOptions options = JDBCOptions.builder() + .setDBUrl(DB_URL) + .setTableName(OUTPUT_TABLE) .build(); + JdbcDmlOptions dmlOptions = JdbcDmlOptions.builder() + .withTableName(options.getTableName()).withDialect(options.getDialect().getName()) + .withFieldNames(fieldNames).withKeyFields(keyFields).build(); + format = new TableJdbcOutputFormat(options, dmlOptions, JdbcBatchOptions.defaults()); RuntimeContext context = Mockito.mock(RuntimeContext.class); ExecutionConfig config = Mockito.mock(ExecutionConfig.class); doReturn(config).when(context).getExecutionConfig(); -- Gitee From 427eb55d5048dc40a6dfc7713e6d5d42881da873 Mon Sep 17 00:00:00 2001 From: Roman Khachatryan Date: Thu, 23 Jan 2020 18:22:08 +0100 Subject: [PATCH 113/885] [FLINK-15782][connectors/jdbc] generalize jdbc statement executors (from Row to ) Motivation: use existing code oriented to Table API in DataStream API --- .../flink/api/java/io/jdbc/JDBCUtils.java | 8 +++ .../java/io/jdbc/TableJdbcOutputFormat.java | 5 +- .../executor/InsertOrUpdateJdbcExecutor.java | 65 ++++++++++--------- .../executor/JdbcBatchStatementExecutor.java | 38 ++++++++--- .../executor/KeyedBatchStatementExecutor.java | 60 ++++++----------- .../io/jdbc/executor/ParameterSetter.java | 45 +++++++++++++ .../SimpleBatchStatementExecutor.java | 27 ++++---- 7 files changed, 150 insertions(+), 98 deletions(-) create mode 100644 flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/ParameterSetter.java diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCUtils.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCUtils.java index d50b82ea9f..14902da7a3 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCUtils.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCUtils.java @@ -231,4 +231,12 @@ public class JDBCUtils { return ret; } } + + public static Row getPrimaryKey(Row row, int[] pkFields) { + Row pkRow = new Row(pkFields.length); + for (int i = 0; i < pkFields.length; i++) { + pkRow.setField(i, row.getField(pkFields[i])); + } + return pkRow; + } } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/TableJdbcOutputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/TableJdbcOutputFormat.java index 305cf20dfe..a9a56255a1 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/TableJdbcOutputFormat.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/TableJdbcOutputFormat.java @@ -54,13 +54,12 @@ class TableJdbcOutputFormat extends JdbcBatchingOutputFormat dmlOptions.getFieldTypes()[f]).toArray(); String deleteSql = dmlOptions.getDialect().getDeleteStatement(dmlOptions.getTableName(), dmlOptions.getFieldNames()); - boolean objectReuseEnabled = getRuntimeContext().getExecutionConfig().isObjectReuseEnabled(); - return JdbcBatchStatementExecutor.keyed(pkFields, pkTypes, deleteSql, objectReuseEnabled); + return JdbcBatchStatementExecutor.keyedRow(pkFields, pkTypes, deleteSql); } @Override JdbcBatchStatementExecutor createStatementRunner(JDBCDialect dialect) { - return JdbcBatchStatementExecutor.upsert( + return JdbcBatchStatementExecutor.upsertRow( dialect, dmlOptions.getTableName(), dmlOptions.getFieldNames(), dmlOptions.getFieldTypes(), dmlOptions.getKeyFields(), getRuntimeContext().getExecutionConfig().isObjectReuseEnabled()); } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/InsertOrUpdateJdbcExecutor.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/InsertOrUpdateJdbcExecutor.java index ba37707002..0a5d873db4 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/InsertOrUpdateJdbcExecutor.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/InsertOrUpdateJdbcExecutor.java @@ -17,8 +17,6 @@ package org.apache.flink.api.java.io.jdbc.executor; -import org.apache.flink.types.Row; - import java.sql.Connection; import java.sql.PreparedStatement; import java.sql.ResultSet; @@ -26,76 +24,81 @@ import java.sql.SQLException; import java.util.Arrays; import java.util.HashMap; import java.util.Map; +import java.util.function.Function; -import static org.apache.flink.api.java.io.jdbc.JDBCUtils.setRecordToStatement; - -final class InsertOrUpdateJdbcExecutor implements JdbcBatchStatementExecutor { +final class InsertOrUpdateJdbcExecutor implements JdbcBatchStatementExecutor { private final String existSQL; private final String insertSQL; private final String updateSQL; - private final int[] pkTypes; - private final int[] pkFields; - private final boolean objectReuse; + + private final ParameterSetter existSetter; + private final ParameterSetter insertSetter; + private final ParameterSetter updateSetter; + + private final Function keyExtractor; + private final Function valueMapper; private transient PreparedStatement existStatement; private transient PreparedStatement insertStatement; private transient PreparedStatement updateStatement; - private transient Map keyToRows = new HashMap<>(); - private final int[] fieldTypes; + private transient Map batch = new HashMap<>(); - InsertOrUpdateJdbcExecutor( - int[] fieldTypes, - int[] pkFields, int[] pkTypes, - String existSQL, - String insertSQL, - String updateSQL, boolean objectReuse) { - this.pkFields = pkFields; + InsertOrUpdateJdbcExecutor(String existSQL, + String insertSQL, + String updateSQL, + ParameterSetter existSetter, + ParameterSetter insertSetter, + ParameterSetter updateSetter, + Function keyExtractor, + Function valueExtractor) { this.existSQL = existSQL; this.insertSQL = insertSQL; this.updateSQL = updateSQL; - this.fieldTypes = fieldTypes; - this.pkTypes = pkTypes; - this.objectReuse = objectReuse; + this.existSetter = existSetter; + this.insertSetter = insertSetter; + this.updateSetter = updateSetter; + this.keyExtractor = keyExtractor; + this.valueMapper = valueExtractor; } @Override public void open(Connection connection) throws SQLException { - keyToRows = new HashMap<>(); + batch = new HashMap<>(); existStatement = connection.prepareStatement(existSQL); insertStatement = connection.prepareStatement(insertSQL); updateStatement = connection.prepareStatement(updateSQL); } @Override - public void process(Row record) { - keyToRows.put(KeyedBatchStatementExecutor.getPrimaryKey(record, pkFields), objectReuse ? Row.copy(record) : record); + public void process(R record) { + batch.put(keyExtractor.apply(record), valueMapper.apply(record)); } @Override public void executeBatch() throws SQLException { - if (keyToRows.size() > 0) { - for (Map.Entry entry : keyToRows.entrySet()) { + if (!batch.isEmpty()) { + for (Map.Entry entry : batch.entrySet()) { processOneRowInBatch(entry.getKey(), entry.getValue()); } updateStatement.executeBatch(); insertStatement.executeBatch(); - keyToRows.clear(); + batch.clear(); } } - private void processOneRowInBatch(Row pk, Row row) throws SQLException { + private void processOneRowInBatch(K pk, V row) throws SQLException { if (exist(pk)) { - setRecordToStatement(updateStatement, fieldTypes, row); + updateSetter.accept(updateStatement, row); updateStatement.addBatch(); } else { - setRecordToStatement(insertStatement, fieldTypes, row); + insertSetter.accept(insertStatement, row); insertStatement.addBatch(); } } - private boolean exist(Row pk) throws SQLException { - setRecordToStatement(existStatement, pkTypes, pk); + private boolean exist(K pk) throws SQLException { + existSetter.accept(existStatement, pk); try (ResultSet resultSet = existStatement.executeQuery()) { return resultSet.next(); } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/JdbcBatchStatementExecutor.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/JdbcBatchStatementExecutor.java index f231a739a2..58632f617a 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/JdbcBatchStatementExecutor.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/JdbcBatchStatementExecutor.java @@ -25,7 +25,10 @@ import org.apache.flink.types.Row; import java.sql.Connection; import java.sql.SQLException; import java.util.Arrays; +import java.util.function.Function; +import static org.apache.flink.api.java.io.jdbc.JDBCUtils.getPrimaryKey; +import static org.apache.flink.api.java.io.jdbc.JDBCUtils.setRecordToStatement; import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -51,7 +54,7 @@ public interface JdbcBatchStatementExecutor { */ void close() throws SQLException; - static JdbcBatchStatementExecutor upsert( + static JdbcBatchStatementExecutor upsertRow( JDBCDialect dialect, String tableName, String[] fieldNames, @@ -66,22 +69,39 @@ public interface JdbcBatchStatementExecutor { return dialect .getUpsertStatement(tableName, fieldNames, keyFields) - .map(sql -> keyed(pkFields, pkTypes, sql, objectReuse)) + .map(sql -> keyedRow(pkFields, fieldTypes, sql)) .orElseGet(() -> - new InsertOrUpdateJdbcExecutor( - fieldTypes, pkFields, pkTypes, + new InsertOrUpdateJdbcExecutor<>( dialect.getRowExistsStatement(tableName, keyFields), dialect.getInsertIntoStatement(tableName, fieldNames), dialect.getUpdateStatement(tableName, fieldNames, keyFields), - objectReuse)); + ParameterSetter.forRow(pkTypes), + ParameterSetter.forRow(fieldTypes), + ParameterSetter.forRow(fieldTypes), + rowKeyExtractor(pkFields), + objectReuse ? Row::copy : Function.identity())); } - static JdbcBatchStatementExecutor keyed(int[] pkFields, int[] pkTypes, String sql, boolean objectReuse) { - return new KeyedBatchStatementExecutor(pkFields, pkTypes, sql, objectReuse); + static Function rowKeyExtractor(int[] pkFields) { + return row -> getPrimaryKey(row, pkFields); } - static JdbcBatchStatementExecutor simple(String sql, int[] fieldTypes, boolean objectReuse) { - return new SimpleBatchStatementExecutor(sql, fieldTypes, objectReuse); + static JdbcBatchStatementExecutor keyedRow(int[] pkFields, int[] pkTypes, String sql) { + return keyed(sql, + rowKeyExtractor(pkFields), + (st, record) -> setRecordToStatement(st, pkTypes, rowKeyExtractor(pkFields).apply(record))); + } + + static JdbcBatchStatementExecutor keyed(String sql, Function keyExtractor, ParameterSetter parameterSetter) { + return new KeyedBatchStatementExecutor<>(sql, keyExtractor, parameterSetter); + } + + static JdbcBatchStatementExecutor simpleRow(String sql, int[] fieldTypes, boolean objectReuse) { + return simple(sql, ParameterSetter.forRow(fieldTypes), objectReuse ? Row::copy : Function.identity()); + } + + static JdbcBatchStatementExecutor simple(String sql, ParameterSetter paramSetter, Function valueTransformer) { + return new SimpleBatchStatementExecutor<>(sql, paramSetter, valueTransformer); } } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/KeyedBatchStatementExecutor.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/KeyedBatchStatementExecutor.java index 70ab6793c6..1da9f45dd5 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/KeyedBatchStatementExecutor.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/KeyedBatchStatementExecutor.java @@ -18,59 +18,51 @@ package org.apache.flink.api.java.io.jdbc.executor; -import org.apache.flink.types.Row; - import java.sql.Connection; import java.sql.PreparedStatement; import java.sql.SQLException; -import java.util.HashMap; -import java.util.Map; - -import static org.apache.flink.api.java.io.jdbc.JDBCUtils.setRecordToStatement; +import java.util.HashSet; +import java.util.Set; +import java.util.function.Function; -/** - * Upsert writer to deal with upsert, delete message. - */ -class KeyedBatchStatementExecutor implements JdbcBatchStatementExecutor { +class KeyedBatchStatementExecutor implements JdbcBatchStatementExecutor { - private final int[] pkTypes; - private final int[] pkFields; private final String sql; - private final boolean objectReuse; + private final ParameterSetter parameterSetter; + private final Function keyExtractor; - private transient Map keyToRows = new HashMap<>(); + private transient Set batch = new HashSet<>(); private transient PreparedStatement st; - KeyedBatchStatementExecutor(int[] pkFields, int[] pkTypes, String sql, boolean objectReuse) { - this.pkFields = pkFields; - this.pkTypes = pkTypes; + /** + * Keep in mind object reuse: if it's on then key extractor may be required to return new object. + */ + KeyedBatchStatementExecutor(String sql, Function keyExtractor, ParameterSetter parameterSetter) { + this.parameterSetter = parameterSetter; + this.keyExtractor = keyExtractor; this.sql = sql; - this.objectReuse = objectReuse; } @Override public void open(Connection connection) throws SQLException { - keyToRows = new HashMap<>(); + batch = new HashSet<>(); st = connection.prepareStatement(sql); } @Override - public void process(Row record) { - // we don't need perform a deep copy, because jdbc field are immutable object. - Row row = objectReuse ? Row.copy(record) : record; - // add records to buffer - keyToRows.put(getPrimaryKey(row), row); + public void process(T record) { + batch.add(keyExtractor.apply(record)); } @Override public void executeBatch() throws SQLException { - if (keyToRows.size() > 0) { - for (Map.Entry entry : keyToRows.entrySet()) { - setRecordToStatement(st, pkTypes, entry.getKey()); + if (!batch.isEmpty()) { + for (K entry : batch) { + parameterSetter.accept(st, entry); st.addBatch(); } st.executeBatch(); - keyToRows.clear(); + batch.clear(); } } @@ -82,16 +74,4 @@ class KeyedBatchStatementExecutor implements JdbcBatchStatementExecutor { } } - private Row getPrimaryKey(Row row) { - int[] pkFields = this.pkFields; - return getPrimaryKey(row, pkFields); - } - - static Row getPrimaryKey(Row row, int[] pkFields) { - Row pkRow = new Row(pkFields.length); - for (int i = 0; i < pkFields.length; i++) { - pkRow.setField(i, row.getField(pkFields[i])); - } - return pkRow; - } } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/ParameterSetter.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/ParameterSetter.java new file mode 100644 index 0000000000..bb6b6f5d79 --- /dev/null +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/ParameterSetter.java @@ -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.flink.api.java.io.jdbc.executor; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.types.Row; +import org.apache.flink.util.function.BiConsumerWithException; + +import java.io.Serializable; +import java.sql.PreparedStatement; +import java.sql.SQLException; + +import static org.apache.flink.api.java.io.jdbc.JDBCUtils.setRecordToStatement; + +/** + * Sets {@link PreparedStatement} parameters to use in JDBC Sink based on a specific type of StreamRecord. + * @param type of payload in {@link org.apache.flink.streaming.runtime.streamrecord.StreamRecord StreamRecord} + * @see JdbcBatchStatementExecutor + */ +@Internal +public interface ParameterSetter extends BiConsumerWithException, Serializable { + + /** + * Creates a {@link ParameterSetter} for {@link Row} using the provided SQL types array. + * Uses {@link org.apache.flink.api.java.io.jdbc.JDBCUtils#setRecordToStatement} + */ + static ParameterSetter forRow(int[] types) { + return (st, record) -> setRecordToStatement(st, types, record); + } +} diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/SimpleBatchStatementExecutor.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/SimpleBatchStatementExecutor.java index 16325ecc5b..ff92b9f0ca 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/SimpleBatchStatementExecutor.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/SimpleBatchStatementExecutor.java @@ -18,29 +18,26 @@ package org.apache.flink.api.java.io.jdbc.executor; -import org.apache.flink.types.Row; - import java.sql.Connection; import java.sql.PreparedStatement; import java.sql.SQLException; import java.util.ArrayList; import java.util.List; +import java.util.function.Function; -import static org.apache.flink.api.java.io.jdbc.JDBCUtils.setRecordToStatement; - -class SimpleBatchStatementExecutor implements JdbcBatchStatementExecutor { +class SimpleBatchStatementExecutor implements JdbcBatchStatementExecutor { private final String sql; - private final int[] paramTypes; + private final ParameterSetter parameterSetter; + private final Function valueTransformer; private transient PreparedStatement st; - private List batch; - private final boolean objectReuse; + private transient List batch; - SimpleBatchStatementExecutor(String sql, int[] paramTypes, boolean objectReuse) { + SimpleBatchStatementExecutor(String sql, ParameterSetter parameterSetter, Function valueTransformer) { this.sql = sql; - this.paramTypes = paramTypes; - this.objectReuse = objectReuse; + this.parameterSetter = parameterSetter; + this.valueTransformer = valueTransformer; } @Override @@ -50,15 +47,15 @@ class SimpleBatchStatementExecutor implements JdbcBatchStatementExecutor { } @Override - public void process(Row record) { - batch.add(objectReuse ? Row.copy(record) : record); + public void process(T record) { + batch.add(valueTransformer.apply(record)); } @Override public void executeBatch() throws SQLException { if (!batch.isEmpty()) { - for (Row r : batch) { - setRecordToStatement(st, paramTypes, r); + for (V r : batch) { + parameterSetter.accept(st, r); st.addBatch(); } st.executeBatch(); -- Gitee From 62065c82ffa909f9b5794ed3e2877da279088b68 Mon Sep 17 00:00:00 2001 From: Roman Khachatryan Date: Fri, 24 Jan 2020 15:45:15 +0100 Subject: [PATCH 114/885] [FLINK-15782][connectors/jdbc] generalize output formats and sinks (from Row to ) Motivation: use existing code oriented to Table API in DataStream API --- .../io/jdbc/AbstractJdbcOutputFormat.java | 7 +- ...tion.java => GenericJdbcSinkFunction.java} | 16 ++-- .../api/java/io/jdbc/JDBCOutputFormat.java | 7 +- .../api/java/io/jdbc/JDBCUpsertTableSink.java | 5 +- .../io/jdbc/JdbcBatchingOutputFormat.java | 91 ++++++++++--------- .../api/java/io/jdbc/JdbcSinkFunction.java | 4 + .../java/io/jdbc/JdbcTypedQueryOptions.java | 2 +- ....java => TableJdbcUpsertOutputFormat.java} | 29 +++--- .../executor/JdbcBatchStatementExecutor.java | 34 +++---- .../io/jdbc/JdbcTableOutputFormatTest.java | 4 +- 10 files changed, 103 insertions(+), 96 deletions(-) rename flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/{JdbcUpsertSinkFunction.java => GenericJdbcSinkFunction.java} (80%) rename flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/{TableJdbcOutputFormat.java => TableJdbcUpsertOutputFormat.java} (74%) diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/AbstractJdbcOutputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/AbstractJdbcOutputFormat.java index 6d806bbae2..7bd812fbe8 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/AbstractJdbcOutputFormat.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/AbstractJdbcOutputFormat.java @@ -24,12 +24,13 @@ import org.apache.flink.configuration.Configuration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.Flushable; import java.io.IOException; import java.sql.Connection; import java.sql.DriverManager; import java.sql.SQLException; -abstract class AbstractJdbcOutputFormat extends RichOutputFormat { +abstract class AbstractJdbcOutputFormat extends RichOutputFormat implements Flushable { private static final long serialVersionUID = 1L; static final int DEFAULT_FLUSH_MAX_SIZE = 5000; @@ -83,4 +84,8 @@ abstract class AbstractJdbcOutputFormat extends RichOutputFormat { } } } + + @Override + public void flush() throws IOException { + } } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcUpsertSinkFunction.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/GenericJdbcSinkFunction.java similarity index 80% rename from flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcUpsertSinkFunction.java rename to flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/GenericJdbcSinkFunction.java index 7b6aac7502..0fe21b47a7 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcUpsertSinkFunction.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/GenericJdbcSinkFunction.java @@ -19,18 +19,18 @@ package org.apache.flink.api.java.io.jdbc; import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.FunctionSnapshotContext; import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; -import org.apache.flink.types.Row; -class JdbcUpsertSinkFunction extends RichSinkFunction> implements CheckpointedFunction { - private final JdbcBatchingOutputFormat outputFormat; +import java.io.IOException; - JdbcUpsertSinkFunction(JdbcBatchingOutputFormat outputFormat) { +class GenericJdbcSinkFunction extends RichSinkFunction implements CheckpointedFunction { + private final AbstractJdbcOutputFormat outputFormat; + + GenericJdbcSinkFunction(AbstractJdbcOutputFormat outputFormat) { this.outputFormat = outputFormat; } @@ -43,12 +43,12 @@ class JdbcUpsertSinkFunction extends RichSinkFunction> impl } @Override - public void invoke(Tuple2 value, Context context) throws Exception { + public void invoke(T value, Context context) throws IOException { outputFormat.writeRecord(value); } @Override - public void initializeState(FunctionInitializationContext context) throws Exception { + public void initializeState(FunctionInitializationContext context) { } @Override @@ -57,7 +57,7 @@ class JdbcUpsertSinkFunction extends RichSinkFunction> impl } @Override - public void close() throws Exception { + public void close() { outputFormat.close(); } } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java index 217dbf562e..0a1107fd9c 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java @@ -37,6 +37,10 @@ import static org.apache.flink.api.java.io.jdbc.JDBCUtils.setRecordToStatement; * @see Row * @see DriverManager */ +/** + * @deprecated use {@link JdbcBatchingOutputFormat} + */ +@Deprecated public class JDBCOutputFormat extends AbstractJdbcOutputFormat { private static final long serialVersionUID = 1L; @@ -99,7 +103,8 @@ public class JDBCOutputFormat extends AbstractJdbcOutputFormat { } } - void flush() { + @Override + public void flush() { try { upload.executeBatch(); batchCount = 0; diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSink.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSink.java index 75854622c9..af4076acc3 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSink.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSink.java @@ -20,6 +20,7 @@ package org.apache.flink.api.java.io.jdbc; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.io.jdbc.executor.JdbcBatchStatementExecutor; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.api.java.typeutils.TupleTypeInfo; @@ -69,7 +70,7 @@ public class JDBCUpsertTableSink implements UpsertStreamTableSink { this.maxRetryTime = maxRetryTime; } - private JdbcBatchingOutputFormat newFormat() { + private JdbcBatchingOutputFormat, Row, JdbcBatchStatementExecutor> newFormat() { if (!isAppendOnly && (keyFields == null || keyFields.length == 0)) { throw new UnsupportedOperationException("JDBCUpsertTableSink can not support "); } @@ -92,7 +93,7 @@ public class JDBCUpsertTableSink implements UpsertStreamTableSink { @Override public DataStreamSink consumeDataStream(DataStream> dataStream) { return dataStream - .addSink(new JdbcUpsertSinkFunction(newFormat())) + .addSink(new GenericJdbcSinkFunction<>(newFormat())) .setParallelism(dataStream.getParallelism()) .name(TableConnectorUtils.generateRuntimeName(this.getClass(), schema.getFieldNames())); } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcBatchingOutputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcBatchingOutputFormat.java index 213ad3627a..2d28c7aaea 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcBatchingOutputFormat.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcBatchingOutputFormat.java @@ -18,9 +18,12 @@ package org.apache.flink.api.java.io.jdbc; -import org.apache.flink.api.java.io.jdbc.dialect.JDBCDialect; +import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.java.io.jdbc.executor.JdbcBatchStatementExecutor; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.util.ExecutorThreadFactory; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -31,10 +34,11 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; +import java.util.function.Function; import static org.apache.flink.util.Preconditions.checkNotNull; -abstract class JdbcBatchingOutputFormat> extends AbstractJdbcOutputFormat { +class JdbcBatchingOutputFormat> extends AbstractJdbcOutputFormat { private static final long serialVersionUID = 1L; @@ -42,39 +46,29 @@ abstract class JdbcBatchingOutputFormat scheduledFuture; private transient volatile Exception flushException; - - /** - * @deprecated use {@link #JdbcBatchingOutputFormat(JdbcConnectionOptions, JdbcDmlOptions, JdbcBatchOptions)} - */ - @Deprecated - public JdbcBatchingOutputFormat( - JDBCOptions options, - String[] fieldNames, - String[] keyFields, - int[] fieldTypes, - int flushMaxSize, - long flushIntervalMills, - int maxRetryTimes) { - this( - options, - JdbcDmlOptions.builder().withFieldNames(fieldNames).withKeyFields(keyFields).withFieldTypes(fieldTypes).withMaxRetries(maxRetryTimes).build(), - JdbcBatchOptions.builder().withSize(flushMaxSize).withIntervalMs(flushIntervalMills).build()); - } - - public JdbcBatchingOutputFormat(JdbcConnectionOptions connectionOptions, JdbcDmlOptions dmlOptions, JdbcBatchOptions batchOptions) { + private final Function statementRunnerCreator; + final Function jdbcRecordExtractor; + + JdbcBatchingOutputFormat(JdbcConnectionOptions connectionOptions, + JdbcDmlOptions dmlOptions, + JdbcBatchOptions batchOptions, + Function statementExecutorCreator, + Function recordExtractor) { super(connectionOptions); this.dmlOptions = dmlOptions; this.batchOptions = batchOptions; + this.statementRunnerCreator = statementExecutorCreator; + this.jdbcRecordExtractor = recordExtractor; } /** @@ -85,7 +79,7 @@ abstract class JdbcBatchingOutputFormat= dmlOptions.getMaxRetries()) { - throw e; + throw new IOException(e); + } + try { + Thread.sleep(1000 * i); + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + throw new IOException("unable to flush; interrupted while doing another attempt", e); } - Thread.sleep(1000 * i); } } } @@ -218,7 +214,7 @@ abstract class JdbcBatchingOutputFormat, Row, JdbcBatchStatementExecutor> build() { checkNotNull(options, "No options supplied."); checkNotNull(fieldNames, "No fieldNames supplied."); JdbcDmlOptions dml = JdbcDmlOptions.builder() .withTableName(options.getTableName()).withDialect(options.getDialect()) .withFieldNames(fieldNames).withKeyFields(keyFields).withFieldTypes(fieldTypes).withMaxRetries(maxRetryTimes).build(); if (dml.getKeyFields() == null || dml.getKeyFields().length == 0) { - return new JdbcBatchingOutputFormat( + // warn: don't close over builder fields + String sql = options.getDialect().getInsertIntoStatement(dml.getTableName(), dml.getFieldNames()); + return new JdbcBatchingOutputFormat<>( options, dml, - JdbcBatchOptions.builder().withSize(flushMaxSize).withIntervalMs(flushIntervalMills).build()); + JdbcBatchOptions.builder().withSize(flushMaxSize).withIntervalMs(flushIntervalMills).build(), + unused -> JdbcBatchStatementExecutor.simpleRow(sql, dml.getFieldTypes()), + tuple2 -> { + Preconditions.checkArgument(tuple2.f0); + return tuple2.f1; + }); } else { - return new TableJdbcOutputFormat( + return new TableJdbcUpsertOutputFormat( options, dml, JdbcBatchOptions.builder().withSize(flushMaxSize).withIntervalMs(flushIntervalMills).build()); diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcSinkFunction.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcSinkFunction.java index da3f6119a8..05e069e8b4 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcSinkFunction.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcSinkFunction.java @@ -26,6 +26,10 @@ import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.types.Row; +/** + * @deprecated use {@link GenericJdbcSinkFunction} + */ +@Deprecated class JdbcSinkFunction extends RichSinkFunction implements CheckpointedFunction { final JDBCOutputFormat outputFormat; diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcTypedQueryOptions.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcTypedQueryOptions.java index b4b409a482..22d7eb62c5 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcTypedQueryOptions.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcTypedQueryOptions.java @@ -30,7 +30,7 @@ abstract class JdbcTypedQueryOptions implements Serializable { this.fieldTypes = fieldTypes; } - int[] getFieldTypes() { + public int[] getFieldTypes() { return fieldTypes; } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/TableJdbcOutputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/TableJdbcUpsertOutputFormat.java similarity index 74% rename from flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/TableJdbcOutputFormat.java rename to flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/TableJdbcUpsertOutputFormat.java index a9a56255a1..9935349fd2 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/TableJdbcOutputFormat.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/TableJdbcUpsertOutputFormat.java @@ -17,7 +17,6 @@ package org.apache.flink.api.java.io.jdbc; -import org.apache.flink.api.java.io.jdbc.dialect.JDBCDialect; import org.apache.flink.api.java.io.jdbc.executor.JdbcBatchStatementExecutor; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.types.Row; @@ -28,14 +27,20 @@ import org.slf4j.LoggerFactory; import java.io.IOException; import java.sql.SQLException; import java.util.Arrays; +import java.util.function.Function; -class TableJdbcOutputFormat extends JdbcBatchingOutputFormat, Row, JdbcBatchStatementExecutor> { - private static final Logger LOG = LoggerFactory.getLogger(TableJdbcOutputFormat.class); +class TableJdbcUpsertOutputFormat extends JdbcBatchingOutputFormat, Row, JdbcBatchStatementExecutor> { + private static final Logger LOG = LoggerFactory.getLogger(TableJdbcUpsertOutputFormat.class); private JdbcBatchStatementExecutor deleteExecutor; - TableJdbcOutputFormat(JdbcConnectionOptions connectionOptions, JdbcDmlOptions dmlOptions, JdbcBatchOptions batchOptions) { - super(connectionOptions, dmlOptions, batchOptions); + TableJdbcUpsertOutputFormat(JdbcConnectionOptions connectionOptions, JdbcDmlOptions dmlOptions, JdbcBatchOptions batchOptions) { + super(connectionOptions, dmlOptions, batchOptions, ctx -> JdbcBatchStatementExecutor.upsertRow(dmlOptions, ctx), new Function, Row>() { + @Override + public Row apply(Tuple2 record) { + return record.f1; + } + }); } @Override @@ -57,27 +62,15 @@ class TableJdbcOutputFormat extends JdbcBatchingOutputFormat createStatementRunner(JDBCDialect dialect) { - return JdbcBatchStatementExecutor.upsertRow( - dialect, dmlOptions.getTableName(), dmlOptions.getFieldNames(), dmlOptions.getFieldTypes(), dmlOptions.getKeyFields(), - getRuntimeContext().getExecutionConfig().isObjectReuseEnabled()); - } - @Override void doWriteRecord(Tuple2 record) throws SQLException { if (record.f0) { super.doWriteRecord(record); } else { - deleteExecutor.process(extractJdbcRecord(record)); + deleteExecutor.process(jdbcRecordExtractor.apply(record)); } } - @Override - Row extractJdbcRecord(Tuple2 record) { - return record.f1; - } - @Override public synchronized void close() { try { diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/JdbcBatchStatementExecutor.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/JdbcBatchStatementExecutor.java index 58632f617a..3ca04f54ec 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/JdbcBatchStatementExecutor.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/JdbcBatchStatementExecutor.java @@ -19,6 +19,8 @@ package org.apache.flink.api.java.io.jdbc.executor; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.java.io.jdbc.JdbcDmlOptions; import org.apache.flink.api.java.io.jdbc.dialect.JDBCDialect; import org.apache.flink.types.Row; @@ -54,32 +56,26 @@ public interface JdbcBatchStatementExecutor { */ void close() throws SQLException; - static JdbcBatchStatementExecutor upsertRow( - JDBCDialect dialect, - String tableName, - String[] fieldNames, - int[] fieldTypes, - String[] keyFields, - boolean objectReuse) { + static JdbcBatchStatementExecutor upsertRow(JdbcDmlOptions opt, RuntimeContext ctx) { + checkNotNull(opt.getKeyFields()); + JDBCDialect dialect = opt.getDialectName().getInstance(opt); - checkNotNull(keyFields); - - int[] pkFields = Arrays.stream(keyFields).mapToInt(Arrays.asList(fieldNames)::indexOf).toArray(); - int[] pkTypes = fieldTypes == null ? null : Arrays.stream(pkFields).map(f -> fieldTypes[f]).toArray(); + int[] pkFields = Arrays.stream(opt.getKeyFields()).mapToInt(Arrays.asList(opt.getFieldNames())::indexOf).toArray(); + int[] pkTypes = opt.getFieldTypes() == null ? null : Arrays.stream(pkFields).map(f -> opt.getFieldTypes()[f]).toArray(); return dialect - .getUpsertStatement(tableName, fieldNames, keyFields) - .map(sql -> keyedRow(pkFields, fieldTypes, sql)) + .getUpsertStatement(opt.getTableName(), opt.getFieldNames(), opt.getKeyFields()) + .map(sql -> keyedRow(pkFields, opt.getFieldTypes(), sql)) .orElseGet(() -> new InsertOrUpdateJdbcExecutor<>( - dialect.getRowExistsStatement(tableName, keyFields), - dialect.getInsertIntoStatement(tableName, fieldNames), - dialect.getUpdateStatement(tableName, fieldNames, keyFields), + dialect.getRowExistsStatement(opt.getTableName(), opt.getKeyFields()), + dialect.getInsertIntoStatement(opt.getTableName(), opt.getFieldNames()), + dialect.getUpdateStatement(opt.getTableName(), opt.getFieldNames(), opt.getKeyFields()), ParameterSetter.forRow(pkTypes), - ParameterSetter.forRow(fieldTypes), - ParameterSetter.forRow(fieldTypes), + ParameterSetter.forRow(opt.getFieldTypes()), + ParameterSetter.forRow(opt.getFieldTypes()), rowKeyExtractor(pkFields), - objectReuse ? Row::copy : Function.identity())); + ctx.getExecutionConfig().isObjectReuseEnabled() ? Row::copy : Function.identity())); } static Function rowKeyExtractor(int[] pkFields) { diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTableOutputFormatTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTableOutputFormatTest.java index 2473f7d39d..5a105a9c72 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTableOutputFormatTest.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTableOutputFormatTest.java @@ -47,7 +47,7 @@ import static org.mockito.Mockito.doReturn; */ public class JdbcTableOutputFormatTest extends JDBCTestBase { - private TableJdbcOutputFormat format; + private TableJdbcUpsertOutputFormat format; private String[] fieldNames; private String[] keyFields; @@ -66,7 +66,7 @@ public class JdbcTableOutputFormatTest extends JDBCTestBase { JdbcDmlOptions dmlOptions = JdbcDmlOptions.builder() .withTableName(options.getTableName()).withDialect(options.getDialect().getName()) .withFieldNames(fieldNames).withKeyFields(keyFields).build(); - format = new TableJdbcOutputFormat(options, dmlOptions, JdbcBatchOptions.defaults()); + format = new TableJdbcUpsertOutputFormat(options, dmlOptions, JdbcBatchOptions.defaults()); RuntimeContext context = Mockito.mock(RuntimeContext.class); ExecutionConfig config = Mockito.mock(ExecutionConfig.class); doReturn(config).when(context).getExecutionConfig(); -- Gitee From 3d2e47d6fffed53b4bd8f9dec8a4ca9cf2f86182 Mon Sep 17 00:00:00 2001 From: Roman Khachatryan Date: Wed, 29 Jan 2020 19:25:47 +0100 Subject: [PATCH 115/885] [FLINK-15782][connectors/jdbc] rename Batch to Execution Options --- .../api/java/io/jdbc/JDBCLookupOptions.java | 2 +- .../api/java/io/jdbc/JDBCOutputFormat.java | 13 +++-- .../api/java/io/jdbc/JDBCUpsertTableSink.java | 2 +- .../io/jdbc/JdbcBatchingOutputFormat.java | 52 ++++++++++--------- .../api/java/io/jdbc/JdbcDmlOptions.java | 38 +++++--------- ...Options.java => JdbcExecutionOptions.java} | 45 ++++++++++------ .../io/jdbc/TableJdbcUpsertOutputFormat.java | 12 ++--- .../executor/JdbcBatchStatementExecutor.java | 10 ++-- .../io/jdbc/JdbcTableOutputFormatTest.java | 4 +- 9 files changed, 88 insertions(+), 90 deletions(-) rename flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/{JdbcBatchOptions.java => JdbcExecutionOptions.java} (57%) diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCLookupOptions.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCLookupOptions.java index 62d9d92e4e..1f55cd6483 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCLookupOptions.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCLookupOptions.java @@ -21,7 +21,7 @@ package org.apache.flink.api.java.io.jdbc; import java.io.Serializable; import java.util.Objects; -import static org.apache.flink.api.java.io.jdbc.JdbcBatchingOutputFormat.DEFAULT_MAX_RETRY_TIMES; +import static org.apache.flink.api.java.io.jdbc.JdbcExecutionOptions.DEFAULT_MAX_RETRY_TIMES; /** * Options for the JDBC lookup. diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java index 0a1107fd9c..9bd094bd06 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java @@ -24,7 +24,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; -import java.sql.DriverManager; import java.sql.PreparedStatement; import java.sql.SQLException; @@ -48,22 +47,22 @@ public class JDBCOutputFormat extends AbstractJdbcOutputFormat { private static final Logger LOG = LoggerFactory.getLogger(JDBCOutputFormat.class); final JdbcInsertOptions insertOptions; - private final JdbcBatchOptions batchOptions; + private final JdbcExecutionOptions batchOptions; private transient PreparedStatement upload; private transient int batchCount = 0; /** - * @deprecated use {@link #JDBCOutputFormat(JdbcConnectionOptions, JdbcInsertOptions, JdbcBatchOptions)}}. + * @deprecated use {@link #JDBCOutputFormat(JdbcConnectionOptions, JdbcInsertOptions, JdbcExecutionOptions)}}. */ @Deprecated public JDBCOutputFormat(String username, String password, String drivername, String dbURL, String query, int batchInterval, int[] typesArray) { this(new JdbcConnectionOptions(dbURL, drivername, username, password), new JdbcInsertOptions(query, typesArray), - JdbcBatchOptions.builder().withSize(batchInterval).build()); + JdbcExecutionOptions.builder().withBatchSize(batchInterval).build()); } - public JDBCOutputFormat(JdbcConnectionOptions connectionOptions, JdbcInsertOptions insertOptions, JdbcBatchOptions batchOptions) { + public JDBCOutputFormat(JdbcConnectionOptions connectionOptions, JdbcInsertOptions insertOptions, JdbcExecutionOptions batchOptions) { super(connectionOptions); this.insertOptions = insertOptions; this.batchOptions = batchOptions; @@ -97,7 +96,7 @@ public class JDBCOutputFormat extends AbstractJdbcOutputFormat { batchCount++; - if (batchCount >= batchOptions.getSize()) { + if (batchCount >= batchOptions.getBatchSize()) { // execute batch flush(); } @@ -202,7 +201,7 @@ public class JDBCOutputFormat extends AbstractJdbcOutputFormat { public JDBCOutputFormat finish() { return new JDBCOutputFormat(buildConnectionOptions(), new JdbcInsertOptions(query, typesArray), - JdbcBatchOptions.builder().withSize(batchInterval).build()); + JdbcExecutionOptions.builder().withBatchSize(batchInterval).build()); } public JdbcConnectionOptions buildConnectionOptions() { diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSink.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSink.java index af4076acc3..4acc35db34 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSink.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSink.java @@ -39,7 +39,7 @@ import java.util.Objects; import static org.apache.flink.api.java.io.jdbc.AbstractJdbcOutputFormat.DEFAULT_FLUSH_INTERVAL_MILLS; import static org.apache.flink.api.java.io.jdbc.AbstractJdbcOutputFormat.DEFAULT_FLUSH_MAX_SIZE; -import static org.apache.flink.api.java.io.jdbc.JdbcBatchingOutputFormat.DEFAULT_MAX_RETRY_TIMES; +import static org.apache.flink.api.java.io.jdbc.JdbcExecutionOptions.DEFAULT_MAX_RETRY_TIMES; import static org.apache.flink.api.java.io.jdbc.JdbcTypeUtil.normalizeTableSchema; import static org.apache.flink.util.Preconditions.checkNotNull; diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcBatchingOutputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcBatchingOutputFormat.java index 2d28c7aaea..df94334412 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcBatchingOutputFormat.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcBatchingOutputFormat.java @@ -29,6 +29,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; +import java.io.Serializable; import java.sql.SQLException; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -39,15 +40,22 @@ import java.util.function.Function; import static org.apache.flink.util.Preconditions.checkNotNull; class JdbcBatchingOutputFormat> extends AbstractJdbcOutputFormat { + interface RecordExtractor extends Function, Serializable { + static RecordExtractor identity() { + return x -> x; + } + } + + interface ExecutorCreator> extends Function, Serializable { + } private static final long serialVersionUID = 1L; private static final Logger LOG = LoggerFactory.getLogger(JdbcBatchingOutputFormat.class); - static final int DEFAULT_MAX_RETRY_TIMES = 3; - - final JdbcDmlOptions dmlOptions; - private final JdbcBatchOptions batchOptions; + private final JdbcExecutionOptions executionOptions; + private final ExecutorCreator statementRunnerCreator; + final RecordExtractor jdbcRecordExtractor; private transient JdbcExec jdbcStatementExecutor; private transient int batchCount = 0; @@ -56,17 +64,14 @@ class JdbcBatchingOutputFormat scheduledFuture; private transient volatile Exception flushException; - private final Function statementRunnerCreator; - final Function jdbcRecordExtractor; - - JdbcBatchingOutputFormat(JdbcConnectionOptions connectionOptions, - JdbcDmlOptions dmlOptions, - JdbcBatchOptions batchOptions, - Function statementExecutorCreator, - Function recordExtractor) { + + JdbcBatchingOutputFormat( + JdbcConnectionOptions connectionOptions, + JdbcExecutionOptions executionOptions, + ExecutorCreator statementExecutorCreator, + RecordExtractor recordExtractor) { super(connectionOptions); - this.dmlOptions = dmlOptions; - this.batchOptions = batchOptions; + this.executionOptions = executionOptions; this.statementRunnerCreator = statementExecutorCreator; this.jdbcRecordExtractor = recordExtractor; } @@ -85,7 +90,7 @@ class JdbcBatchingOutputFormat { synchronized (JdbcBatchingOutputFormat.this) { @@ -97,7 +102,7 @@ class JdbcBatchingOutputFormat= batchOptions.getSize()) { + if (batchCount >= executionOptions.getBatchSize()) { flush(); } } catch (Exception e) { @@ -130,14 +135,14 @@ class JdbcBatchingOutputFormat= dmlOptions.getMaxRetries()) { + if (i >= executionOptions.getMaxRetries()) { throw new IOException(e); } try { @@ -201,7 +206,7 @@ class JdbcBatchingOutputFormat( options, - dml, - JdbcBatchOptions.builder().withSize(flushMaxSize).withIntervalMs(flushIntervalMills).build(), + JdbcExecutionOptions.builder().withBatchSize(flushMaxSize).withMaxRetries(maxRetryTimes).withBatchIntervalMs(flushIntervalMills).build(), unused -> JdbcBatchStatementExecutor.simpleRow(sql, dml.getFieldTypes()), tuple2 -> { Preconditions.checkArgument(tuple2.f0); @@ -287,7 +291,7 @@ class JdbcBatchingOutputFormat { + public static class JdbcDmlOptionsBuilder extends JDBCUpdateQueryOptionsBuilder { private String tableName; private String[] fieldNames; private String[] keyFields; private JDBCDialect dialect; private JDBCDialect customDialect; - private int maxRetries = DEFAULT_MAX_RETRY_TIMES; - - public JDBCUpsertOptionsBuilder withMaxRetries(int maxRetries) { - this.maxRetries = maxRetries; - return this; - } @Override - protected JDBCUpsertOptionsBuilder self() { + protected JdbcDmlOptionsBuilder self() { return this; } - public JDBCUpsertOptionsBuilder withFieldNames(String field, String... fieldNames) { + public JdbcDmlOptionsBuilder withFieldNames(String field, String... fieldNames) { this.fieldNames = concat(field, fieldNames); return this; } - public JDBCUpsertOptionsBuilder withFieldNames(String[] fieldNames) { + public JdbcDmlOptionsBuilder withFieldNames(String[] fieldNames) { this.fieldNames = fieldNames; return this; } - public JDBCUpsertOptionsBuilder withKeyFields(String keyField, String... keyFields) { + public JdbcDmlOptionsBuilder withKeyFields(String keyField, String... keyFields) { this.keyFields = concat(keyField, keyFields); return this; } - public JDBCUpsertOptionsBuilder withKeyFields(String[] keyFields) { + public JdbcDmlOptionsBuilder withKeyFields(String[] keyFields) { this.keyFields = keyFields; return this; } - public JDBCUpsertOptionsBuilder withTableName(String tableName) { + public JdbcDmlOptionsBuilder withTableName(String tableName) { this.tableName = tableName; return self(); } - public JDBCUpsertOptionsBuilder withDialect(JDBCDialect dialect) { + public JdbcDmlOptionsBuilder withDialect(JDBCDialect dialect) { this.dialect = dialect; return self(); } public JdbcDmlOptions build() { - return new JdbcDmlOptions(tableName, dialect, fieldNames, fieldTypes, keyFields, maxRetries); + return new JdbcDmlOptions(tableName, dialect, fieldNames, fieldTypes, keyFields); } static String[] concat(String first, String... next) { diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcBatchOptions.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcExecutionOptions.java similarity index 57% rename from flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcBatchOptions.java rename to flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcExecutionOptions.java index 69b14de265..af22d6cfe0 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcBatchOptions.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcExecutionOptions.java @@ -18,6 +18,7 @@ package org.apache.flink.api.java.io.jdbc; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.util.Preconditions; import java.io.Serializable; @@ -25,31 +26,39 @@ import java.io.Serializable; * JDBC sink batch options. */ @PublicEvolving -public class JdbcBatchOptions implements Serializable { +public class JdbcExecutionOptions implements Serializable { + static final int DEFAULT_MAX_RETRY_TIMES = 3; private static final int DEFAULT_INTERVAL_MILLIS = 0; private static final int DEFAULT_SIZE = 5000; - private final long intervalMs; - private final int size; + private final long batchIntervalMs; + private final int batchSize; + private final int maxRetries; - private JdbcBatchOptions(long intervalMs, int size) { - this.intervalMs = intervalMs; - this.size = size; + private JdbcExecutionOptions(long batchIntervalMs, int batchSize, int maxRetries) { + Preconditions.checkArgument(maxRetries >= 1); + this.batchIntervalMs = batchIntervalMs; + this.batchSize = batchSize; + this.maxRetries = maxRetries; } - public long getIntervalMs() { - return intervalMs; + public long getBatchIntervalMs() { + return batchIntervalMs; } - public int getSize() { - return size; + public int getBatchSize() { + return batchSize; + } + + public int getMaxRetries() { + return maxRetries; } public static JdbcBatchOptionsBuilder builder() { return new JdbcBatchOptionsBuilder(); } - public static JdbcBatchOptions defaults() { + public static JdbcExecutionOptions defaults() { return builder().build(); } @@ -59,19 +68,25 @@ public class JdbcBatchOptions implements Serializable { public static final class JdbcBatchOptionsBuilder { private long intervalMs = DEFAULT_INTERVAL_MILLIS; private int size = DEFAULT_SIZE; + private int maxRetries = DEFAULT_MAX_RETRY_TIMES; - public JdbcBatchOptionsBuilder withSize(int size) { + public JdbcBatchOptionsBuilder withBatchSize(int size) { this.size = size; return this; } - public JdbcBatchOptionsBuilder withIntervalMs(long intervalMs) { + public JdbcBatchOptionsBuilder withBatchIntervalMs(long intervalMs) { this.intervalMs = intervalMs; return this; } - public JdbcBatchOptions build() { - return new JdbcBatchOptions(intervalMs, size); + public JdbcBatchOptionsBuilder withMaxRetries(int maxRetries) { + this.maxRetries = maxRetries; + return this; + } + + public JdbcExecutionOptions build() { + return new JdbcExecutionOptions(intervalMs, size, maxRetries); } } } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/TableJdbcUpsertOutputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/TableJdbcUpsertOutputFormat.java index 9935349fd2..1a31d549da 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/TableJdbcUpsertOutputFormat.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/TableJdbcUpsertOutputFormat.java @@ -27,20 +27,16 @@ import org.slf4j.LoggerFactory; import java.io.IOException; import java.sql.SQLException; import java.util.Arrays; -import java.util.function.Function; class TableJdbcUpsertOutputFormat extends JdbcBatchingOutputFormat, Row, JdbcBatchStatementExecutor> { private static final Logger LOG = LoggerFactory.getLogger(TableJdbcUpsertOutputFormat.class); private JdbcBatchStatementExecutor deleteExecutor; + private final JdbcDmlOptions dmlOptions; - TableJdbcUpsertOutputFormat(JdbcConnectionOptions connectionOptions, JdbcDmlOptions dmlOptions, JdbcBatchOptions batchOptions) { - super(connectionOptions, dmlOptions, batchOptions, ctx -> JdbcBatchStatementExecutor.upsertRow(dmlOptions, ctx), new Function, Row>() { - @Override - public Row apply(Tuple2 record) { - return record.f1; - } - }); + TableJdbcUpsertOutputFormat(JdbcConnectionOptions connectionOptions, JdbcDmlOptions dmlOptions, JdbcExecutionOptions batchOptions) { + super(connectionOptions, batchOptions, ctx -> JdbcBatchStatementExecutor.upsertRow(dmlOptions, ctx), tuple2 -> tuple2.f1); + this.dmlOptions = dmlOptions; } @Override diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/JdbcBatchStatementExecutor.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/JdbcBatchStatementExecutor.java index 3ca04f54ec..7e436d06df 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/JdbcBatchStatementExecutor.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/JdbcBatchStatementExecutor.java @@ -21,7 +21,6 @@ package org.apache.flink.api.java.io.jdbc.executor; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.java.io.jdbc.JdbcDmlOptions; -import org.apache.flink.api.java.io.jdbc.dialect.JDBCDialect; import org.apache.flink.types.Row; import java.sql.Connection; @@ -58,19 +57,18 @@ public interface JdbcBatchStatementExecutor { static JdbcBatchStatementExecutor upsertRow(JdbcDmlOptions opt, RuntimeContext ctx) { checkNotNull(opt.getKeyFields()); - JDBCDialect dialect = opt.getDialectName().getInstance(opt); int[] pkFields = Arrays.stream(opt.getKeyFields()).mapToInt(Arrays.asList(opt.getFieldNames())::indexOf).toArray(); int[] pkTypes = opt.getFieldTypes() == null ? null : Arrays.stream(pkFields).map(f -> opt.getFieldTypes()[f]).toArray(); - return dialect + return opt.getDialect() .getUpsertStatement(opt.getTableName(), opt.getFieldNames(), opt.getKeyFields()) .map(sql -> keyedRow(pkFields, opt.getFieldTypes(), sql)) .orElseGet(() -> new InsertOrUpdateJdbcExecutor<>( - dialect.getRowExistsStatement(opt.getTableName(), opt.getKeyFields()), - dialect.getInsertIntoStatement(opt.getTableName(), opt.getFieldNames()), - dialect.getUpdateStatement(opt.getTableName(), opt.getFieldNames(), opt.getKeyFields()), + opt.getDialect().getRowExistsStatement(opt.getTableName(), opt.getKeyFields()), + opt.getDialect().getInsertIntoStatement(opt.getTableName(), opt.getFieldNames()), + opt.getDialect().getUpdateStatement(opt.getTableName(), opt.getFieldNames(), opt.getKeyFields()), ParameterSetter.forRow(pkTypes), ParameterSetter.forRow(opt.getFieldTypes()), ParameterSetter.forRow(opt.getFieldTypes()), diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTableOutputFormatTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTableOutputFormatTest.java index 5a105a9c72..0d545589b8 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTableOutputFormatTest.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTableOutputFormatTest.java @@ -64,9 +64,9 @@ public class JdbcTableOutputFormatTest extends JDBCTestBase { .setTableName(OUTPUT_TABLE) .build(); JdbcDmlOptions dmlOptions = JdbcDmlOptions.builder() - .withTableName(options.getTableName()).withDialect(options.getDialect().getName()) + .withTableName(options.getTableName()).withDialect(options.getDialect()) .withFieldNames(fieldNames).withKeyFields(keyFields).build(); - format = new TableJdbcUpsertOutputFormat(options, dmlOptions, JdbcBatchOptions.defaults()); + format = new TableJdbcUpsertOutputFormat(options, dmlOptions, JdbcExecutionOptions.defaults()); RuntimeContext context = Mockito.mock(RuntimeContext.class); ExecutionConfig config = Mockito.mock(ExecutionConfig.class); doReturn(config).when(context).getExecutionConfig(); -- Gitee From 3ad2f32d7134112ea2a34f86a5f78c012507e61b Mon Sep 17 00:00:00 2001 From: Roman Khachatryan Date: Tue, 14 Jan 2020 00:43:28 +0100 Subject: [PATCH 116/885] [FLINK-15782][connectors/jdbc] inject connection provider instead of options Motivation: manage connections outside of JDBCOutputFormat. In particular, this is required to manage transactions. --- .../io/jdbc/AbstractJdbcOutputFormat.java | 25 +++------ .../api/java/io/jdbc/JDBCOutputFormat.java | 12 ++-- .../io/jdbc/JdbcBatchingOutputFormat.java | 14 ++--- .../java/io/jdbc/JdbcConnectionProvider.java | 30 ++++++++++ .../io/jdbc/SimpleJdbcConnectionProvider.java | 56 +++++++++++++++++++ .../io/jdbc/TableJdbcUpsertOutputFormat.java | 4 +- .../java/io/jdbc/JDBCOutputFormatTest.java | 4 +- .../io/jdbc/JdbcTableOutputFormatTest.java | 2 +- 8 files changed, 113 insertions(+), 34 deletions(-) create mode 100644 flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcConnectionProvider.java create mode 100644 flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/SimpleJdbcConnectionProvider.java diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/AbstractJdbcOutputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/AbstractJdbcOutputFormat.java index 7bd812fbe8..6f4826b2b6 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/AbstractJdbcOutputFormat.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/AbstractJdbcOutputFormat.java @@ -20,6 +20,7 @@ package org.apache.flink.api.java.io.jdbc; import org.apache.flink.api.common.io.RichOutputFormat; import org.apache.flink.configuration.Configuration; +import org.apache.flink.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -27,7 +28,6 @@ import org.slf4j.LoggerFactory; import java.io.Flushable; import java.io.IOException; import java.sql.Connection; -import java.sql.DriverManager; import java.sql.SQLException; abstract class AbstractJdbcOutputFormat extends RichOutputFormat implements Flushable { @@ -37,13 +37,11 @@ abstract class AbstractJdbcOutputFormat extends RichOutputFormat implement static final long DEFAULT_FLUSH_INTERVAL_MILLS = 0; private static final Logger LOG = LoggerFactory.getLogger(AbstractJdbcOutputFormat.class); - - private final JdbcConnectionOptions options; - protected transient Connection connection; + private final JdbcConnectionProvider connectionProvider; - public AbstractJdbcOutputFormat(JdbcConnectionOptions options) { - this.options = options; + public AbstractJdbcOutputFormat(JdbcConnectionProvider connectionProvider) { + this.connectionProvider = Preconditions.checkNotNull(connectionProvider); } @Override @@ -54,25 +52,20 @@ abstract class AbstractJdbcOutputFormat extends RichOutputFormat implement public void open(int taskNumber, int numTasks) throws IOException { try { establishConnection(); - } catch (SQLException | ClassNotFoundException e) { + } catch (Exception e) { throw new IOException("unable to open JDBC writer", e); } } + protected void establishConnection() throws Exception { + connection = connectionProvider.getConnection(); + } + @Override public void close() { closeDbConnection(); } - private void establishConnection() throws SQLException, ClassNotFoundException { - Class.forName(options.getDriverName()); - if (options.username == null) { - connection = DriverManager.getConnection(options.url); - } else { - connection = DriverManager.getConnection(options.url, options.username, options.password); - } - } - private void closeDbConnection() { if (connection != null) { try { diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java index 9bd094bd06..6fd0ffcda7 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java @@ -53,17 +53,17 @@ public class JDBCOutputFormat extends AbstractJdbcOutputFormat { private transient int batchCount = 0; /** - * @deprecated use {@link #JDBCOutputFormat(JdbcConnectionOptions, JdbcInsertOptions, JdbcExecutionOptions)}}. + * @deprecated use {@link #JDBCOutputFormat(JdbcConnectionProvider, JdbcInsertOptions, JdbcExecutionOptions)}}. */ @Deprecated public JDBCOutputFormat(String username, String password, String drivername, String dbURL, String query, int batchInterval, int[] typesArray) { - this(new JdbcConnectionOptions(dbURL, drivername, username, password), + this(new SimpleJdbcConnectionProvider(new JdbcConnectionOptions(dbURL, drivername, username, password)), new JdbcInsertOptions(query, typesArray), JdbcExecutionOptions.builder().withBatchSize(batchInterval).build()); } - public JDBCOutputFormat(JdbcConnectionOptions connectionOptions, JdbcInsertOptions insertOptions, JdbcExecutionOptions batchOptions) { - super(connectionOptions); + public JDBCOutputFormat(JdbcConnectionProvider connectionProvider, JdbcInsertOptions insertOptions, JdbcExecutionOptions batchOptions) { + super(connectionProvider); this.insertOptions = insertOptions; this.batchOptions = batchOptions; } @@ -81,7 +81,7 @@ public class JDBCOutputFormat extends AbstractJdbcOutputFormat { try { upload = connection.prepareStatement(insertOptions.getQuery()); } catch (SQLException sqe) { - throw new IllegalArgumentException("open() failed.", sqe); + throw new IOException("open() failed.", sqe); } } @@ -199,7 +199,7 @@ public class JDBCOutputFormat extends AbstractJdbcOutputFormat { * @return Configured JDBCOutputFormat */ public JDBCOutputFormat finish() { - return new JDBCOutputFormat(buildConnectionOptions(), + return new JDBCOutputFormat(new SimpleJdbcConnectionProvider(buildConnectionOptions()), new JdbcInsertOptions(query, typesArray), JdbcExecutionOptions.builder().withBatchSize(batchInterval).build()); } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcBatchingOutputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcBatchingOutputFormat.java index df94334412..a3d44a588f 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcBatchingOutputFormat.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcBatchingOutputFormat.java @@ -66,11 +66,11 @@ class JdbcBatchingOutputFormat statementExecutorCreator, - RecordExtractor recordExtractor) { - super(connectionOptions); + JdbcConnectionProvider connectionProvider, + JdbcExecutionOptions executionOptions, + ExecutorCreator statementExecutorCreator, + RecordExtractor recordExtractor) { + super(connectionProvider); this.executionOptions = executionOptions; this.statementRunnerCreator = statementExecutorCreator; this.jdbcRecordExtractor = recordExtractor; @@ -280,7 +280,7 @@ class JdbcBatchingOutputFormat( - options, + new SimpleJdbcConnectionProvider(options), JdbcExecutionOptions.builder().withBatchSize(flushMaxSize).withMaxRetries(maxRetryTimes).withBatchIntervalMs(flushIntervalMills).build(), unused -> JdbcBatchStatementExecutor.simpleRow(sql, dml.getFieldTypes()), tuple2 -> { @@ -289,7 +289,7 @@ class JdbcBatchingOutputFormat deleteExecutor; private final JdbcDmlOptions dmlOptions; - TableJdbcUpsertOutputFormat(JdbcConnectionOptions connectionOptions, JdbcDmlOptions dmlOptions, JdbcExecutionOptions batchOptions) { - super(connectionOptions, batchOptions, ctx -> JdbcBatchStatementExecutor.upsertRow(dmlOptions, ctx), tuple2 -> tuple2.f1); + TableJdbcUpsertOutputFormat(JdbcConnectionProvider connectionProvider, JdbcDmlOptions dmlOptions, JdbcExecutionOptions batchOptions) { + super(connectionProvider, batchOptions, ctx -> JdbcBatchStatementExecutor.upsertRow(dmlOptions, ctx), tuple2 -> tuple2.f1); this.dmlOptions = dmlOptions; } diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormatTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormatTest.java index 22a208fa22..104fb875b0 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormatTest.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormatTest.java @@ -70,7 +70,7 @@ public class JDBCOutputFormatTest extends JDBCTestBase { jdbcOutputFormat.open(0, 1); } - @Test(expected = IllegalArgumentException.class) + @Test(expected = IOException.class) public void testInvalidQuery() throws IOException { jdbcOutputFormat = JDBCOutputFormat.buildJDBCOutputFormat() .setDrivername(DRIVER_CLASS) @@ -81,7 +81,7 @@ public class JDBCOutputFormatTest extends JDBCTestBase { } @Test(expected = NullPointerException.class) - public void testIncompleteConfiguration() throws IOException { + public void testIncompleteConfiguration() { jdbcOutputFormat = JDBCOutputFormat.buildJDBCOutputFormat() .setDrivername(DRIVER_CLASS) .setQuery(String.format(INSERT_TEMPLATE, INPUT_TABLE)) diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTableOutputFormatTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTableOutputFormatTest.java index 0d545589b8..234657d04a 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTableOutputFormatTest.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTableOutputFormatTest.java @@ -66,7 +66,7 @@ public class JdbcTableOutputFormatTest extends JDBCTestBase { JdbcDmlOptions dmlOptions = JdbcDmlOptions.builder() .withTableName(options.getTableName()).withDialect(options.getDialect()) .withFieldNames(fieldNames).withKeyFields(keyFields).build(); - format = new TableJdbcUpsertOutputFormat(options, dmlOptions, JdbcExecutionOptions.defaults()); + format = new TableJdbcUpsertOutputFormat(new SimpleJdbcConnectionProvider(options), dmlOptions, JdbcExecutionOptions.defaults()); RuntimeContext context = Mockito.mock(RuntimeContext.class); ExecutionConfig config = Mockito.mock(ExecutionConfig.class); doReturn(config).when(context).getExecutionConfig(); -- Gitee From 40cab49012e2a54db36da2b44b951a237057bd49 Mon Sep 17 00:00:00 2001 From: Roman Khachatryan Date: Tue, 14 Jan 2020 00:43:30 +0100 Subject: [PATCH 117/885] [FLINK-15782][connectors/jdbc] refactor JDBC sink tests Changes: 1. extract DbMetadata to allow to use different databases 2. extract test fixture from the base class 3. use inheritance only for @Before/@After behaviour --- .../flink/api/java/io/jdbc/DbMetadata.java | 36 ++++ .../api/java/io/jdbc/DerbyDbMetadata.java | 64 ++++++ .../api/java/io/jdbc/JDBCDataTestBase.java | 37 ++++ .../flink/api/java/io/jdbc/JDBCFullTest.java | 43 ++-- .../api/java/io/jdbc/JDBCInputFormatTest.java | 78 ++++---- .../io/jdbc/JDBCLookupFunctionITCase.java | 8 +- .../java/io/jdbc/JDBCOutputFormatTest.java | 61 +++--- .../flink/api/java/io/jdbc/JDBCTestBase.java | 164 +++------------- .../api/java/io/jdbc/JDBCTestCheckpoint.java | 35 ++++ .../io/jdbc/JDBCUpsertTableSinkITCase.java | 7 +- .../io/jdbc/JdbcTableOutputFormatTest.java | 15 +- .../api/java/io/jdbc/JdbcTestFixture.java | 184 ++++++++++++++++++ 12 files changed, 493 insertions(+), 239 deletions(-) create mode 100644 flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/DbMetadata.java create mode 100644 flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/DerbyDbMetadata.java create mode 100644 flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCDataTestBase.java create mode 100644 flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTestCheckpoint.java create mode 100644 flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTestFixture.java diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/DbMetadata.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/DbMetadata.java new file mode 100644 index 0000000000..14731e4d33 --- /dev/null +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/DbMetadata.java @@ -0,0 +1,36 @@ +/* + * 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.api.java.io.jdbc; + +import javax.sql.XADataSource; + +import java.io.Serializable; + +/** + * Describes a database: driver, schema and urls. + */ +public interface DbMetadata extends Serializable { + + String getInitUrl(); + + String getUrl(); + + XADataSource buildXaDataSource(); + + String getDriverClass(); +} diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/DerbyDbMetadata.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/DerbyDbMetadata.java new file mode 100644 index 0000000000..79a62bc64d --- /dev/null +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/DerbyDbMetadata.java @@ -0,0 +1,64 @@ +/* + * 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.api.java.io.jdbc; + +import org.apache.derby.jdbc.EmbeddedXADataSource; + +import javax.sql.XADataSource; + +/** + * DerbyDbMetadata. + */ +public class DerbyDbMetadata implements DbMetadata { + private final String dbName; + private final String dbInitUrl; + private final String url; + + public DerbyDbMetadata(String schemaName) { + dbName = "memory:" + schemaName; + url = "jdbc:derby:" + dbName; + dbInitUrl = url + ";create=true"; + } + + public String getDbName() { + return dbName; + } + + @Override + public String getInitUrl() { + return dbInitUrl; + } + + @Override + public XADataSource buildXaDataSource() { + EmbeddedXADataSource ds = new EmbeddedXADataSource(); + ds.setDatabaseName(dbName); + return ds; + } + + @Override + public String getDriverClass() { + return "org.apache.derby.jdbc.EmbeddedDriver"; + } + + @Override + public String getUrl() { + return url; + } + +} diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCDataTestBase.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCDataTestBase.java new file mode 100644 index 0000000000..8fb15399f4 --- /dev/null +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCDataTestBase.java @@ -0,0 +1,37 @@ +/* + * 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.api.java.io.jdbc; + +import org.junit.Before; + +import java.sql.SQLException; + +/** + * Base class for JDBC test using data from {@link JdbcTestFixture}. It uses {@link DerbyDbMetadata} and inserts data before each test. + */ +abstract class JDBCDataTestBase extends JDBCTestBase { + @Before + public void initData() throws SQLException { + JdbcTestFixture.initData(getDbMetadata()); + } + + @Override + protected DbMetadata getDbMetadata() { + return JdbcTestFixture.DERBY_EBOOKSHOP_DB; + } +} diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCFullTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCFullTest.java index 14dc85a3ef..28ac1a1cc4 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCFullTest.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCFullTest.java @@ -26,7 +26,9 @@ import org.apache.flink.types.Row; import org.junit.After; import org.junit.Assert; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; import java.sql.Connection; import java.sql.DriverManager; @@ -35,12 +37,17 @@ import java.sql.ResultSet; import java.sql.Statement; import java.sql.Types; +import static org.apache.flink.api.java.io.jdbc.JdbcTestFixture.OUTPUT_TABLE; +import static org.apache.flink.api.java.io.jdbc.JdbcTestFixture.ROW_TYPE_INFO; import static org.hamcrest.core.StringContains.containsString; /** * Tests using both {@link JDBCInputFormat} and {@link JDBCOutputFormat}. */ -public class JDBCFullTest extends JDBCTestBase { +public class JDBCFullTest extends JDBCDataTestBase { + + @Rule + public ExpectedException exception = ExpectedException.none(); @Test public void testWithoutParallelism() throws Exception { @@ -58,8 +65,8 @@ public class JDBCFullTest extends JDBCTestBase { exception.expectMessage(containsString("field index: 3, field value: 11.11.")); JDBCOutputFormat jdbcOutputFormat = JDBCOutputFormat.buildJDBCOutputFormat() - .setDrivername(JDBCTestBase.DRIVER_CLASS) - .setDBUrl(JDBCTestBase.DB_URL) + .setDrivername(getDbMetadata().getDriverClass()) + .setDBUrl(getDbMetadata().getUrl()) .setQuery("insert into newbooks (id, title, author, price, qty) values (?,?,?,?,?)") .setSqlTypes(new int[]{Types.INTEGER, Types.VARCHAR, Types.VARCHAR, Types.DOUBLE, Types.INTEGER}) .finish(); @@ -73,18 +80,18 @@ public class JDBCFullTest extends JDBCTestBase { private void runTest(boolean exploitParallelism) throws Exception { ExecutionEnvironment environment = ExecutionEnvironment.getExecutionEnvironment(); JDBCInputFormatBuilder inputBuilder = JDBCInputFormat.buildJDBCInputFormat() - .setDrivername(JDBCTestBase.DRIVER_CLASS) - .setDBUrl(JDBCTestBase.DB_URL) - .setQuery(JDBCTestBase.SELECT_ALL_BOOKS) + .setDrivername(getDbMetadata().getDriverClass()) + .setDBUrl(getDbMetadata().getUrl()) + .setQuery(JdbcTestFixture.SELECT_ALL_BOOKS) .setRowTypeInfo(ROW_TYPE_INFO); if (exploitParallelism) { final int fetchSize = 1; - final long min = JDBCTestBase.TEST_DATA[0].id; - final long max = JDBCTestBase.TEST_DATA[JDBCTestBase.TEST_DATA.length - fetchSize].id; + final long min = JdbcTestFixture.TEST_DATA[0].id; + final long max = JdbcTestFixture.TEST_DATA[JdbcTestFixture.TEST_DATA.length - fetchSize].id; //use a "splittable" query to exploit parallelism inputBuilder = inputBuilder - .setQuery(JDBCTestBase.SELECT_ALL_BOOKS_SPLIT_BY_ID) + .setQuery(JdbcTestFixture.SELECT_ALL_BOOKS_SPLIT_BY_ID) .setParametersProvider(new NumericBetweenParametersProvider(min, max).ofBatchSize(fetchSize)); } DataSet source = environment.createInput(inputBuilder.finish()); @@ -93,8 +100,8 @@ public class JDBCFullTest extends JDBCTestBase { //some databases don't null values correctly when no column type was specified //in PreparedStatement.setObject (see its javadoc for more details) source.output(JDBCOutputFormat.buildJDBCOutputFormat() - .setDrivername(JDBCTestBase.DRIVER_CLASS) - .setDBUrl(JDBCTestBase.DB_URL) + .setDrivername(getDbMetadata().getDriverClass()) + .setDBUrl(getDbMetadata().getUrl()) .setQuery("insert into newbooks (id, title, author, price, qty) values (?,?,?,?,?)") .setSqlTypes(new int[]{Types.INTEGER, Types.VARCHAR, Types.VARCHAR, Types.DOUBLE, Types.INTEGER}) .finish()); @@ -102,24 +109,24 @@ public class JDBCFullTest extends JDBCTestBase { environment.execute(); try ( - Connection dbConn = DriverManager.getConnection(JDBCTestBase.DB_URL); - PreparedStatement statement = dbConn.prepareStatement(JDBCTestBase.SELECT_ALL_NEWBOOKS); - ResultSet resultSet = statement.executeQuery() + Connection dbConn = DriverManager.getConnection(getDbMetadata().getUrl()); + PreparedStatement statement = dbConn.prepareStatement(JdbcTestFixture.SELECT_ALL_NEWBOOKS); + ResultSet resultSet = statement.executeQuery() ) { int count = 0; while (resultSet.next()) { count++; } - Assert.assertEquals(JDBCTestBase.TEST_DATA.length, count); + Assert.assertEquals(JdbcTestFixture.TEST_DATA.length, count); } } @After public void clearOutputTable() throws Exception { - Class.forName(DRIVER_CLASS); + Class.forName(getDbMetadata().getDriverClass()); try ( - Connection conn = DriverManager.getConnection(DB_URL); - Statement stat = conn.createStatement()) { + Connection conn = DriverManager.getConnection(getDbMetadata().getUrl()); + Statement stat = conn.createStatement()) { stat.execute("DELETE FROM " + OUTPUT_TABLE); stat.close(); diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java index 559976d06f..ca4f8e388a 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java @@ -18,6 +18,7 @@ package org.apache.flink.api.java.io.jdbc; +import org.apache.flink.api.java.io.jdbc.JdbcTestFixture.TestEntry; import org.apache.flink.api.java.io.jdbc.split.GenericParameterValuesProvider; import org.apache.flink.api.java.io.jdbc.split.NumericBetweenParametersProvider; import org.apache.flink.api.java.io.jdbc.split.ParameterValuesProvider; @@ -34,10 +35,15 @@ import java.sql.DriverManager; import java.sql.ResultSet; import java.sql.SQLException; +import static org.apache.flink.api.java.io.jdbc.JdbcTestFixture.ROW_TYPE_INFO; +import static org.apache.flink.api.java.io.jdbc.JdbcTestFixture.SELECT_ALL_BOOKS; +import static org.apache.flink.api.java.io.jdbc.JdbcTestFixture.SELECT_EMPTY; +import static org.apache.flink.api.java.io.jdbc.JdbcTestFixture.TEST_DATA; + /** * Tests for the {@link JDBCInputFormat}. */ -public class JDBCInputFormatTest extends JDBCTestBase { +public class JDBCInputFormatTest extends JDBCDataTestBase { private JDBCInputFormat jdbcInputFormat; @@ -54,7 +60,7 @@ public class JDBCInputFormatTest extends JDBCTestBase { public void testUntypedRowInfo() throws IOException { jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat() .setDrivername("org.apache.derby.jdbc.idontexist") - .setDBUrl(DB_URL) + .setDBUrl(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()) .setQuery(SELECT_ALL_BOOKS) .finish(); jdbcInputFormat.openInputFormat(); @@ -64,7 +70,7 @@ public class JDBCInputFormatTest extends JDBCTestBase { public void testInvalidDriver() throws IOException { jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat() .setDrivername("org.apache.derby.jdbc.idontexist") - .setDBUrl(DB_URL) + .setDBUrl(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()) .setQuery(SELECT_ALL_BOOKS) .setRowTypeInfo(ROW_TYPE_INFO) .finish(); @@ -74,7 +80,7 @@ public class JDBCInputFormatTest extends JDBCTestBase { @Test(expected = IllegalArgumentException.class) public void testInvalidURL() throws IOException { jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat() - .setDrivername(DRIVER_CLASS) + .setDrivername(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getDriverClass()) .setDBUrl("jdbc:der:iamanerror:mory:ebookshop") .setQuery(SELECT_ALL_BOOKS) .setRowTypeInfo(ROW_TYPE_INFO) @@ -85,8 +91,8 @@ public class JDBCInputFormatTest extends JDBCTestBase { @Test(expected = IllegalArgumentException.class) public void testInvalidQuery() throws IOException { jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat() - .setDrivername(DRIVER_CLASS) - .setDBUrl(DB_URL) + .setDrivername(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getDriverClass()) + .setDBUrl(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()) .setQuery("iamnotsql") .setRowTypeInfo(ROW_TYPE_INFO) .finish(); @@ -96,7 +102,7 @@ public class JDBCInputFormatTest extends JDBCTestBase { @Test(expected = IllegalArgumentException.class) public void testIncompleteConfiguration() throws IOException { jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat() - .setDrivername(DRIVER_CLASS) + .setDrivername(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getDriverClass()) .setQuery(SELECT_ALL_BOOKS) .setRowTypeInfo(ROW_TYPE_INFO) .finish(); @@ -105,8 +111,8 @@ public class JDBCInputFormatTest extends JDBCTestBase { @Test(expected = IllegalArgumentException.class) public void testInvalidFetchSize() { jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat() - .setDrivername(DRIVER_CLASS) - .setDBUrl(DB_URL) + .setDrivername(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getDriverClass()) + .setDBUrl(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()) .setQuery(SELECT_ALL_BOOKS) .setRowTypeInfo(ROW_TYPE_INFO) .setFetchSize(-7) @@ -116,8 +122,8 @@ public class JDBCInputFormatTest extends JDBCTestBase { @Test public void testValidFetchSizeIntegerMin() { jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat() - .setDrivername(DRIVER_CLASS) - .setDBUrl(DB_URL) + .setDrivername(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getDriverClass()) + .setDBUrl(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()) .setQuery(SELECT_ALL_BOOKS) .setRowTypeInfo(ROW_TYPE_INFO) .setFetchSize(Integer.MIN_VALUE) @@ -127,15 +133,15 @@ public class JDBCInputFormatTest extends JDBCTestBase { @Test public void testDefaultFetchSizeIsUsedIfNotConfiguredOtherwise() throws SQLException, ClassNotFoundException { jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat() - .setDrivername(DRIVER_CLASS) - .setDBUrl(DB_URL) + .setDrivername(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getDriverClass()) + .setDBUrl(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()) .setQuery(SELECT_ALL_BOOKS) .setRowTypeInfo(ROW_TYPE_INFO) .finish(); jdbcInputFormat.openInputFormat(); - Class.forName(DRIVER_CLASS); - final int defaultFetchSize = DriverManager.getConnection(DB_URL).createStatement().getFetchSize(); + Class.forName(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getDriverClass()); + final int defaultFetchSize = DriverManager.getConnection(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()).createStatement().getFetchSize(); Assert.assertEquals(defaultFetchSize, jdbcInputFormat.getStatement().getFetchSize()); } @@ -144,8 +150,8 @@ public class JDBCInputFormatTest extends JDBCTestBase { public void testFetchSizeCanBeConfigured() throws SQLException { final int desiredFetchSize = 10_000; jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat() - .setDrivername(DRIVER_CLASS) - .setDBUrl(DB_URL) + .setDrivername(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getDriverClass()) + .setDBUrl(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()) .setQuery(SELECT_ALL_BOOKS) .setRowTypeInfo(ROW_TYPE_INFO) .setFetchSize(desiredFetchSize) @@ -158,15 +164,15 @@ public class JDBCInputFormatTest extends JDBCTestBase { public void testDefaultAutoCommitIsUsedIfNotConfiguredOtherwise() throws SQLException, ClassNotFoundException { jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat() - .setDrivername(DRIVER_CLASS) - .setDBUrl(DB_URL) + .setDrivername(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getDriverClass()) + .setDBUrl(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()) .setQuery(SELECT_ALL_BOOKS) .setRowTypeInfo(ROW_TYPE_INFO) .finish(); jdbcInputFormat.openInputFormat(); - Class.forName(DRIVER_CLASS); - final boolean defaultAutoCommit = DriverManager.getConnection(DB_URL).getAutoCommit(); + Class.forName(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getDriverClass()); + final boolean defaultAutoCommit = DriverManager.getConnection(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()).getAutoCommit(); Assert.assertEquals(defaultAutoCommit, jdbcInputFormat.getDbConn().getAutoCommit()); @@ -177,8 +183,8 @@ public class JDBCInputFormatTest extends JDBCTestBase { final boolean desiredAutoCommit = false; jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat() - .setDrivername(DRIVER_CLASS) - .setDBUrl(DB_URL) + .setDrivername(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getDriverClass()) + .setDBUrl(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()) .setQuery(SELECT_ALL_BOOKS) .setRowTypeInfo(ROW_TYPE_INFO) .setAutoCommit(desiredAutoCommit) @@ -192,8 +198,8 @@ public class JDBCInputFormatTest extends JDBCTestBase { @Test public void testJDBCInputFormatWithoutParallelism() throws IOException { jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat() - .setDrivername(DRIVER_CLASS) - .setDBUrl(DB_URL) + .setDrivername(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getDriverClass()) + .setDBUrl(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()) .setQuery(SELECT_ALL_BOOKS) .setRowTypeInfo(ROW_TYPE_INFO) .setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE) @@ -223,9 +229,9 @@ public class JDBCInputFormatTest extends JDBCTestBase { final long max = TEST_DATA[TEST_DATA.length - fetchSize].id; ParameterValuesProvider pramProvider = new NumericBetweenParametersProvider(min, max).ofBatchSize(fetchSize); jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat() - .setDrivername(DRIVER_CLASS) - .setDBUrl(DB_URL) - .setQuery(JDBCTestBase.SELECT_ALL_BOOKS_SPLIT_BY_ID) + .setDrivername(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getDriverClass()) + .setDBUrl(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()) + .setQuery(JdbcTestFixture.SELECT_ALL_BOOKS_SPLIT_BY_ID) .setRowTypeInfo(ROW_TYPE_INFO) .setParametersProvider(pramProvider) .setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE) @@ -259,9 +265,9 @@ public class JDBCInputFormatTest extends JDBCTestBase { final long fetchSize = max + 1; //generate a single split ParameterValuesProvider pramProvider = new NumericBetweenParametersProvider(min, max).ofBatchSize(fetchSize); jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat() - .setDrivername(DRIVER_CLASS) - .setDBUrl(DB_URL) - .setQuery(JDBCTestBase.SELECT_ALL_BOOKS_SPLIT_BY_ID) + .setDrivername(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getDriverClass()) + .setDBUrl(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()) + .setQuery(JdbcTestFixture.SELECT_ALL_BOOKS_SPLIT_BY_ID) .setRowTypeInfo(ROW_TYPE_INFO) .setParametersProvider(pramProvider) .setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE) @@ -295,9 +301,9 @@ public class JDBCInputFormatTest extends JDBCTestBase { queryParameters[1] = new String[]{TEST_DATA[0].author}; ParameterValuesProvider paramProvider = new GenericParameterValuesProvider(queryParameters); jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat() - .setDrivername(DRIVER_CLASS) - .setDBUrl(DB_URL) - .setQuery(JDBCTestBase.SELECT_ALL_BOOKS_SPLIT_BY_AUTHOR) + .setDrivername(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getDriverClass()) + .setDBUrl(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()) + .setQuery(JdbcTestFixture.SELECT_ALL_BOOKS_SPLIT_BY_AUTHOR) .setRowTypeInfo(ROW_TYPE_INFO) .setParametersProvider(paramProvider) .setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE) @@ -335,8 +341,8 @@ public class JDBCInputFormatTest extends JDBCTestBase { @Test public void testEmptyResults() throws IOException { jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat() - .setDrivername(DRIVER_CLASS) - .setDBUrl(DB_URL) + .setDrivername(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getDriverClass()) + .setDBUrl(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()) .setQuery(SELECT_EMPTY) .setRowTypeInfo(ROW_TYPE_INFO) .setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE) diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCLookupFunctionITCase.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCLookupFunctionITCase.java index 16a0eeba63..dabd88cc43 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCLookupFunctionITCase.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCLookupFunctionITCase.java @@ -45,8 +45,6 @@ import java.util.Arrays; import java.util.Collection; import java.util.List; -import static org.apache.flink.api.java.io.jdbc.JDBCTestBase.DRIVER_CLASS; - /** * IT case for {@link JDBCLookupFunction}. */ @@ -69,9 +67,9 @@ public class JDBCLookupFunctionITCase extends AbstractTestBase { @Before public void before() throws ClassNotFoundException, SQLException { - System.setProperty("derby.stream.error.field", JDBCTestBase.class.getCanonicalName() + ".DEV_NULL"); + System.setProperty("derby.stream.error.field", JdbcTestFixture.class.getCanonicalName() + ".DEV_NULL"); - Class.forName(DRIVER_CLASS); + Class.forName(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getDriverClass()); try ( Connection conn = DriverManager.getConnection(DB_URL + ";create=true"); Statement stat = conn.createStatement()) { @@ -123,7 +121,7 @@ public class JDBCLookupFunctionITCase extends AbstractTestBase { @After public void clearOutputTable() throws Exception { - Class.forName(DRIVER_CLASS); + Class.forName(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getDriverClass()); try ( Connection conn = DriverManager.getConnection(DB_URL); Statement stat = conn.createStatement()) { diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormatTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormatTest.java index 104fb875b0..cc9a92aa79 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormatTest.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormatTest.java @@ -32,13 +32,18 @@ import java.sql.SQLException; import java.sql.Statement; import java.sql.Types; +import static org.apache.flink.api.java.io.jdbc.JdbcTestFixture.INPUT_TABLE; +import static org.apache.flink.api.java.io.jdbc.JdbcTestFixture.INSERT_TEMPLATE; +import static org.apache.flink.api.java.io.jdbc.JdbcTestFixture.OUTPUT_TABLE; +import static org.apache.flink.api.java.io.jdbc.JdbcTestFixture.OUTPUT_TABLE_2; +import static org.apache.flink.api.java.io.jdbc.JdbcTestFixture.TEST_DATA; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; /** * Tests for the {@link JDBCOutputFormat}. */ -public class JDBCOutputFormatTest extends JDBCTestBase { +public class JDBCOutputFormatTest extends JDBCDataTestBase { private JDBCOutputFormat jdbcOutputFormat; @@ -54,7 +59,7 @@ public class JDBCOutputFormatTest extends JDBCTestBase { public void testInvalidDriver() throws IOException { jdbcOutputFormat = JDBCOutputFormat.buildJDBCOutputFormat() .setDrivername("org.apache.derby.jdbc.idontexist") - .setDBUrl(DB_URL) + .setDBUrl(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()) .setQuery(String.format(INSERT_TEMPLATE, INPUT_TABLE)) .finish(); jdbcOutputFormat.open(0, 1); @@ -63,7 +68,7 @@ public class JDBCOutputFormatTest extends JDBCTestBase { @Test(expected = IOException.class) public void testInvalidURL() throws IOException { jdbcOutputFormat = JDBCOutputFormat.buildJDBCOutputFormat() - .setDrivername(DRIVER_CLASS) + .setDrivername(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getDriverClass()) .setDBUrl("jdbc:der:iamanerror:mory:ebookshop") .setQuery(String.format(INSERT_TEMPLATE, INPUT_TABLE)) .finish(); @@ -73,8 +78,8 @@ public class JDBCOutputFormatTest extends JDBCTestBase { @Test(expected = IOException.class) public void testInvalidQuery() throws IOException { jdbcOutputFormat = JDBCOutputFormat.buildJDBCOutputFormat() - .setDrivername(DRIVER_CLASS) - .setDBUrl(DB_URL) + .setDrivername(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getDriverClass()) + .setDBUrl(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()) .setQuery("iamnotsql") .finish(); jdbcOutputFormat.open(0, 1); @@ -83,7 +88,7 @@ public class JDBCOutputFormatTest extends JDBCTestBase { @Test(expected = NullPointerException.class) public void testIncompleteConfiguration() { jdbcOutputFormat = JDBCOutputFormat.buildJDBCOutputFormat() - .setDrivername(DRIVER_CLASS) + .setDrivername(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getDriverClass()) .setQuery(String.format(INSERT_TEMPLATE, INPUT_TABLE)) .finish(); } @@ -91,8 +96,8 @@ public class JDBCOutputFormatTest extends JDBCTestBase { @Test(expected = RuntimeException.class) public void testIncompatibleTypes() throws IOException { jdbcOutputFormat = JDBCOutputFormat.buildJDBCOutputFormat() - .setDrivername(DRIVER_CLASS) - .setDBUrl(DB_URL) + .setDrivername(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getDriverClass()) + .setDBUrl(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()) .setQuery(String.format(INSERT_TEMPLATE, INPUT_TABLE)) .finish(); jdbcOutputFormat.open(0, 1); @@ -111,8 +116,8 @@ public class JDBCOutputFormatTest extends JDBCTestBase { @Test(expected = RuntimeException.class) public void testExceptionOnInvalidType() throws IOException { jdbcOutputFormat = JDBCOutputFormat.buildJDBCOutputFormat() - .setDrivername(DRIVER_CLASS) - .setDBUrl(DB_URL) + .setDrivername(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getDriverClass()) + .setDBUrl(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()) .setQuery(String.format(INSERT_TEMPLATE, OUTPUT_TABLE)) .setSqlTypes(new int[] { Types.INTEGER, @@ -123,7 +128,7 @@ public class JDBCOutputFormatTest extends JDBCTestBase { .finish(); jdbcOutputFormat.open(0, 1); - JDBCTestBase.TestEntry entry = TEST_DATA[0]; + JdbcTestFixture.TestEntry entry = TEST_DATA[0]; Row row = new Row(5); row.setField(0, entry.id); row.setField(1, entry.title); @@ -137,8 +142,8 @@ public class JDBCOutputFormatTest extends JDBCTestBase { public void testExceptionOnClose() throws IOException { jdbcOutputFormat = JDBCOutputFormat.buildJDBCOutputFormat() - .setDrivername(DRIVER_CLASS) - .setDBUrl(DB_URL) + .setDrivername(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getDriverClass()) + .setDBUrl(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()) .setQuery(String.format(INSERT_TEMPLATE, OUTPUT_TABLE)) .setSqlTypes(new int[] { Types.INTEGER, @@ -149,7 +154,7 @@ public class JDBCOutputFormatTest extends JDBCTestBase { .finish(); jdbcOutputFormat.open(0, 1); - JDBCTestBase.TestEntry entry = TEST_DATA[0]; + JdbcTestFixture.TestEntry entry = TEST_DATA[0]; Row row = new Row(5); row.setField(0, entry.id); row.setField(1, entry.title); @@ -165,22 +170,22 @@ public class JDBCOutputFormatTest extends JDBCTestBase { @Test public void testJDBCOutputFormat() throws IOException, SQLException { jdbcOutputFormat = JDBCOutputFormat.buildJDBCOutputFormat() - .setDrivername(DRIVER_CLASS) - .setDBUrl(DB_URL) + .setDrivername(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getDriverClass()) + .setDBUrl(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()) .setQuery(String.format(INSERT_TEMPLATE, OUTPUT_TABLE)) .finish(); jdbcOutputFormat.open(0, 1); - for (JDBCTestBase.TestEntry entry : TEST_DATA) { + for (JdbcTestFixture.TestEntry entry : TEST_DATA) { jdbcOutputFormat.writeRecord(toRow(entry)); } jdbcOutputFormat.close(); try ( - Connection dbConn = DriverManager.getConnection(DB_URL); - PreparedStatement statement = dbConn.prepareStatement(JDBCTestBase.SELECT_ALL_NEWBOOKS); - ResultSet resultSet = statement.executeQuery() + Connection dbConn = DriverManager.getConnection(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()); + PreparedStatement statement = dbConn.prepareStatement(JdbcTestFixture.SELECT_ALL_NEWBOOKS); + ResultSet resultSet = statement.executeQuery() ) { int recordCount = 0; while (resultSet.next()) { @@ -199,14 +204,14 @@ public class JDBCOutputFormatTest extends JDBCTestBase { @Test public void testFlush() throws SQLException, IOException { jdbcOutputFormat = JDBCOutputFormat.buildJDBCOutputFormat() - .setDrivername(DRIVER_CLASS) - .setDBUrl(DB_URL) + .setDrivername(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getDriverClass()) + .setDBUrl(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()) .setQuery(String.format(INSERT_TEMPLATE, OUTPUT_TABLE_2)) .setBatchInterval(3) .finish(); try ( - Connection dbConn = DriverManager.getConnection(DB_URL); - PreparedStatement statement = dbConn.prepareStatement(JDBCTestBase.SELECT_ALL_NEWBOOKS_2) + Connection dbConn = DriverManager.getConnection(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()); + PreparedStatement statement = dbConn.prepareStatement(JdbcTestFixture.SELECT_ALL_NEWBOOKS_2) ) { jdbcOutputFormat.open(0, 1); for (int i = 0; i < 2; ++i) { @@ -235,10 +240,10 @@ public class JDBCOutputFormatTest extends JDBCTestBase { @After public void clearOutputTable() throws Exception { - Class.forName(DRIVER_CLASS); + Class.forName(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getDriverClass()); try ( - Connection conn = DriverManager.getConnection(DB_URL); - Statement stat = conn.createStatement()) { + Connection conn = DriverManager.getConnection(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()); + Statement stat = conn.createStatement()) { stat.execute("DELETE FROM " + OUTPUT_TABLE); stat.close(); @@ -246,7 +251,7 @@ public class JDBCOutputFormatTest extends JDBCTestBase { } } - static Row toRow(TestEntry entry) { + static Row toRow(JdbcTestFixture.TestEntry entry) { Row row = new Row(5); row.setField(0, entry.id); row.setField(1, entry.title); diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTestBase.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTestBase.java index 021d28f608..ffd25e5e65 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTestBase.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTestBase.java @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, @@ -18,148 +17,29 @@ package org.apache.flink.api.java.io.jdbc; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.junit.After; +import org.junit.Before; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.rules.ExpectedException; - -import java.io.OutputStream; -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.SQLException; -import java.sql.Statement; +import static org.apache.flink.api.java.io.jdbc.JdbcTestFixture.cleanUpDatabasesStatic; +import static org.apache.flink.api.java.io.jdbc.JdbcTestFixture.cleanupData; +import static org.apache.flink.api.java.io.jdbc.JdbcTestFixture.initSchema; /** - * Base test class for JDBC Input and Output. + * Base class for JDBC test using DDL from {@link JdbcTestFixture}. It uses create tables before each test and drops afterwards. */ -public class JDBCTestBase { - - @Rule - public ExpectedException exception = ExpectedException.none(); - - public static final String DRIVER_CLASS = "org.apache.derby.jdbc.EmbeddedDriver"; - public static final String DB_URL = "jdbc:derby:memory:ebookshop"; - public static final String INPUT_TABLE = "books"; - public static final String OUTPUT_TABLE = "newbooks"; - public static final String OUTPUT_TABLE_2 = "newbooks2"; - public static final String SELECT_ALL_BOOKS = "select * from " + INPUT_TABLE; - public static final String SELECT_ID_BOOKS = "select id from " + INPUT_TABLE; - public static final String SELECT_ALL_NEWBOOKS = "select * from " + OUTPUT_TABLE; - public static final String SELECT_ALL_NEWBOOKS_2 = "select * from " + OUTPUT_TABLE_2; - public static final String SELECT_EMPTY = "select * from books WHERE QTY < 0"; - public static final String INSERT_TEMPLATE = "insert into %s (id, title, author, price, qty) values (?,?,?,?,?)"; - public static final String SELECT_ALL_BOOKS_SPLIT_BY_ID = SELECT_ALL_BOOKS + " WHERE id BETWEEN ? AND ?"; - public static final String SELECT_ALL_BOOKS_SPLIT_BY_AUTHOR = SELECT_ALL_BOOKS + " WHERE author = ?"; - - public static final TestEntry[] TEST_DATA = { - new TestEntry(1001, ("Java public for dummies"), ("Tan Ah Teck"), 11.11, 11), - new TestEntry(1002, ("More Java for dummies"), ("Tan Ah Teck"), 22.22, 22), - new TestEntry(1003, ("More Java for more dummies"), ("Mohammad Ali"), 33.33, 33), - new TestEntry(1004, ("A Cup of Java"), ("Kumar"), 44.44, 44), - new TestEntry(1005, ("A Teaspoon of Java"), ("Kevin Jones"), 55.55, 55), - new TestEntry(1006, ("A Teaspoon of Java 1.4"), ("Kevin Jones"), 66.66, 66), - new TestEntry(1007, ("A Teaspoon of Java 1.5"), ("Kevin Jones"), 77.77, 77), - new TestEntry(1008, ("A Teaspoon of Java 1.6"), ("Kevin Jones"), 88.88, 88), - new TestEntry(1009, ("A Teaspoon of Java 1.7"), ("Kevin Jones"), 99.99, 99), - new TestEntry(1010, ("A Teaspoon of Java 1.8"), ("Kevin Jones"), null, 1010) - }; - - static class TestEntry { - protected final Integer id; - protected final String title; - protected final String author; - protected final Double price; - protected final Integer qty; +public abstract class JDBCTestBase { - private TestEntry(Integer id, String title, String author, Double price, Integer qty) { - this.id = id; - this.title = title; - this.author = author; - this.price = price; - this.qty = qty; - } + @Before + public final void before() throws Exception { + initSchema(getDbMetadata()); } - public static final RowTypeInfo ROW_TYPE_INFO = new RowTypeInfo( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.DOUBLE_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); - - public static String getCreateQuery(String tableName) { - StringBuilder sqlQueryBuilder = new StringBuilder("CREATE TABLE "); - sqlQueryBuilder.append(tableName).append(" ("); - sqlQueryBuilder.append("id INT NOT NULL DEFAULT 0,"); - sqlQueryBuilder.append("title VARCHAR(50) DEFAULT NULL,"); - sqlQueryBuilder.append("author VARCHAR(50) DEFAULT NULL,"); - sqlQueryBuilder.append("price FLOAT DEFAULT NULL,"); - sqlQueryBuilder.append("qty INT DEFAULT NULL,"); - sqlQueryBuilder.append("PRIMARY KEY (id))"); - return sqlQueryBuilder.toString(); - } - - public static String getInsertQuery() { - StringBuilder sqlQueryBuilder = new StringBuilder("INSERT INTO books (id, title, author, price, qty) VALUES "); - for (int i = 0; i < TEST_DATA.length; i++) { - sqlQueryBuilder.append("(") - .append(TEST_DATA[i].id).append(",'") - .append(TEST_DATA[i].title).append("','") - .append(TEST_DATA[i].author).append("',") - .append(TEST_DATA[i].price).append(",") - .append(TEST_DATA[i].qty).append(")"); - if (i < TEST_DATA.length - 1) { - sqlQueryBuilder.append(","); - } - } - String insertQuery = sqlQueryBuilder.toString(); - return insertQuery; + @After + public final void after() throws Exception { + cleanupData(getDbMetadata().getUrl()); + cleanUpDatabasesStatic(getDbMetadata()); } - public static final OutputStream DEV_NULL = new OutputStream() { - @Override - public void write(int b) { - } - }; - - @BeforeClass - public static void prepareDerbyDatabase() throws Exception { - System.setProperty("derby.stream.error.field", JDBCTestBase.class.getCanonicalName() + ".DEV_NULL"); + protected abstract DbMetadata getDbMetadata(); - Class.forName(DRIVER_CLASS); - try (Connection conn = DriverManager.getConnection(DB_URL + ";create=true")) { - createTable(conn, JDBCTestBase.INPUT_TABLE); - createTable(conn, OUTPUT_TABLE); - createTable(conn, OUTPUT_TABLE_2); - insertDataIntoInputTable(conn); - } - } - - private static void createTable(Connection conn, String tableName) throws SQLException { - Statement stat = conn.createStatement(); - stat.executeUpdate(getCreateQuery(tableName)); - stat.close(); - } - - private static void insertDataIntoInputTable(Connection conn) throws SQLException { - Statement stat = conn.createStatement(); - stat.execute(getInsertQuery()); - stat.close(); - } - - @AfterClass - public static void cleanUpDerbyDatabases() throws Exception { - Class.forName(DRIVER_CLASS); - try ( - Connection conn = DriverManager.getConnection(DB_URL + ";create=true"); - Statement stat = conn.createStatement()) { - - stat.executeUpdate("DROP TABLE " + INPUT_TABLE); - stat.executeUpdate("DROP TABLE " + OUTPUT_TABLE); - stat.executeUpdate("DROP TABLE " + OUTPUT_TABLE_2); - } - } } diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTestCheckpoint.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTestCheckpoint.java new file mode 100644 index 0000000000..6086011bb8 --- /dev/null +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTestCheckpoint.java @@ -0,0 +1,35 @@ +/* + * 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.api.java.io.jdbc; + +/** + * Holds id and indices of items in {@link JdbcTestFixture#TEST_DATA}. + */ +public class JDBCTestCheckpoint { + public final long id; + public final int[] dataItemsIdx; + + JDBCTestCheckpoint(long id, int... dataItemsIdx) { + this.id = id; + this.dataItemsIdx = dataItemsIdx; + } + + public JDBCTestCheckpoint withCheckpointId(long id) { + return new JDBCTestCheckpoint(id, dataItemsIdx); + } +} diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSinkITCase.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSinkITCase.java index 6f4f810b20..76f60bc2fd 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSinkITCase.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSinkITCase.java @@ -41,7 +41,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; -import static org.apache.flink.api.java.io.jdbc.JDBCTestBase.DRIVER_CLASS; import static org.apache.flink.api.java.io.jdbc.JdbcTableOutputFormatTest.check; /** @@ -55,9 +54,9 @@ public class JDBCUpsertTableSinkITCase extends AbstractTestBase { @Before public void before() throws ClassNotFoundException, SQLException { - System.setProperty("derby.stream.error.field", JDBCTestBase.class.getCanonicalName() + ".DEV_NULL"); + System.setProperty("derby.stream.error.field", JdbcTestFixture.class.getCanonicalName() + ".DEV_NULL"); - Class.forName(DRIVER_CLASS); + Class.forName(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getDriverClass()); try ( Connection conn = DriverManager.getConnection(DB_URL + ";create=true"); Statement stat = conn.createStatement()) { @@ -79,7 +78,7 @@ public class JDBCUpsertTableSinkITCase extends AbstractTestBase { @After public void clearOutputTable() throws Exception { - Class.forName(DRIVER_CLASS); + Class.forName(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getDriverClass()); try ( Connection conn = DriverManager.getConnection(DB_URL); Statement stat = conn.createStatement()) { diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTableOutputFormatTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTableOutputFormatTest.java index 234657d04a..aaaa92c9e6 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTableOutputFormatTest.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTableOutputFormatTest.java @@ -20,6 +20,7 @@ package org.apache.flink.api.java.io.jdbc; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.java.io.jdbc.JdbcTestFixture.TestEntry; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.types.Row; @@ -39,13 +40,15 @@ import java.util.Arrays; import java.util.List; import static org.apache.flink.api.java.io.jdbc.JDBCOutputFormatTest.toRow; +import static org.apache.flink.api.java.io.jdbc.JdbcTestFixture.OUTPUT_TABLE; +import static org.apache.flink.api.java.io.jdbc.JdbcTestFixture.TEST_DATA; import static org.junit.Assert.assertArrayEquals; import static org.mockito.Mockito.doReturn; /** * Tests for the {@link JdbcBatchingOutputFormat}. */ -public class JdbcTableOutputFormatTest extends JDBCTestBase { +public class JdbcTableOutputFormatTest extends JDBCDataTestBase { private TableJdbcUpsertOutputFormat format; private String[] fieldNames; @@ -60,7 +63,7 @@ public class JdbcTableOutputFormatTest extends JDBCTestBase { @Test public void testJDBCOutputFormat() throws Exception { JDBCOptions options = JDBCOptions.builder() - .setDBUrl(DB_URL) + .setDBUrl(getDbMetadata().getUrl()) .setTableName(OUTPUT_TABLE) .build(); JdbcDmlOptions dmlOptions = JdbcDmlOptions.builder() @@ -100,7 +103,7 @@ public class JdbcTableOutputFormatTest extends JDBCTestBase { } private void check(Row[] rows) throws SQLException { - check(rows, DB_URL, OUTPUT_TABLE, fieldNames); + check(rows, getDbMetadata().getUrl(), OUTPUT_TABLE, fieldNames); } static void check(Row[] rows, String url, String table, String[] fields) throws SQLException { @@ -131,10 +134,10 @@ public class JdbcTableOutputFormatTest extends JDBCTestBase { format.close(); } format = null; - Class.forName(DRIVER_CLASS); + Class.forName(getDbMetadata().getDriverClass()); try ( - Connection conn = DriverManager.getConnection(DB_URL); - Statement stat = conn.createStatement()) { + Connection conn = DriverManager.getConnection(getDbMetadata().getUrl()); + Statement stat = conn.createStatement()) { stat.execute("DELETE FROM " + OUTPUT_TABLE); stat.close(); diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTestFixture.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTestFixture.java new file mode 100644 index 0000000000..071a2afcf3 --- /dev/null +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTestFixture.java @@ -0,0 +1,184 @@ +/* + * 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.api.java.io.jdbc; + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; + +import java.io.OutputStream; +import java.io.Serializable; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; + +/** + * Test data and helper objects for JDBC tests. + */ +@SuppressWarnings("SpellCheckingInspection") +public class JdbcTestFixture { + public static final JDBCTestCheckpoint CP0 = new JDBCTestCheckpoint(0, 1, 2, 3); + public static final JDBCTestCheckpoint CP1 = new JDBCTestCheckpoint(1, 4, 5, 6); + + public static final String INPUT_TABLE = "books"; + static final String OUTPUT_TABLE = "newbooks"; + static final String OUTPUT_TABLE_2 = "newbooks2"; + static final String SELECT_ALL_BOOKS = "select * from " + INPUT_TABLE; + static final String SELECT_ID_BOOKS = "select id from " + INPUT_TABLE; + static final String SELECT_ALL_NEWBOOKS = "select * from " + OUTPUT_TABLE; + static final String SELECT_ALL_NEWBOOKS_2 = "select * from " + OUTPUT_TABLE_2; + static final String SELECT_EMPTY = "select * from books WHERE QTY < 0"; + public static final String INSERT_TEMPLATE = "insert into %s (id, title, author, price, qty) values (?,?,?,?,?)"; + static final String SELECT_ALL_BOOKS_SPLIT_BY_ID = SELECT_ALL_BOOKS + " WHERE id BETWEEN ? AND ?"; + static final String SELECT_ALL_BOOKS_SPLIT_BY_AUTHOR = SELECT_ALL_BOOKS + " WHERE author = ?"; + + public static final TestEntry[] TEST_DATA = { + new TestEntry(1001, ("Java public for dummies"), ("Tan Ah Teck"), 11.11, 11), + new TestEntry(1002, ("More Java for dummies"), ("Tan Ah Teck"), 22.22, 22), + new TestEntry(1003, ("More Java for more dummies"), ("Mohammad Ali"), 33.33, 33), + new TestEntry(1004, ("A Cup of Java"), ("Kumar"), 44.44, 44), + new TestEntry(1005, ("A Teaspoon of Java"), ("Kevin Jones"), 55.55, 55), + new TestEntry(1006, ("A Teaspoon of Java 1.4"), ("Kevin Jones"), 66.66, 66), + new TestEntry(1007, ("A Teaspoon of Java 1.5"), ("Kevin Jones"), 77.77, 77), + new TestEntry(1008, ("A Teaspoon of Java 1.6"), ("Kevin Jones"), 88.88, 88), + new TestEntry(1009, ("A Teaspoon of Java 1.7"), ("Kevin Jones"), 99.99, 99), + new TestEntry(1010, ("A Teaspoon of Java 1.8"), ("Kevin Jones"), null, 1010) + }; + + private static final String EBOOKSHOP_SCHEMA_NAME = "ebookshop"; + public static final DerbyDbMetadata DERBY_EBOOKSHOP_DB = new DerbyDbMetadata(EBOOKSHOP_SCHEMA_NAME); + + /** + * TestEntry. + */ + public static class TestEntry implements Serializable { + public final Integer id; + public final String title; + public final String author; + public final Double price; + public final Integer qty; + + private TestEntry(Integer id, String title, String author, Double price, Integer qty) { + this.id = id; + this.title = title; + this.author = author; + this.price = price; + this.qty = qty; + } + + @Override + public String toString() { + return "TestEntry{" + + "id=" + id + + ", title='" + title + '\'' + + ", author='" + author + '\'' + + ", price=" + price + + ", qty=" + qty + + '}'; + } + } + + static final RowTypeInfo ROW_TYPE_INFO = new RowTypeInfo( + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.DOUBLE_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO); + + private static String getCreateQuery(String tableName) { + return "CREATE TABLE " + tableName + " (" + + "id INT NOT NULL DEFAULT 0," + + "title VARCHAR(50) DEFAULT NULL," + + "author VARCHAR(50) DEFAULT NULL," + + "price FLOAT DEFAULT NULL," + + "qty INT DEFAULT NULL," + + "PRIMARY KEY (id))"; + } + + public static String getInsertQuery() { + StringBuilder sqlQueryBuilder = new StringBuilder("INSERT INTO books (id, title, author, price, qty) VALUES "); + for (int i = 0; i < TEST_DATA.length; i++) { + sqlQueryBuilder.append("(") + .append(TEST_DATA[i].id).append(",'") + .append(TEST_DATA[i].title).append("','") + .append(TEST_DATA[i].author).append("',") + .append(TEST_DATA[i].price).append(",") + .append(TEST_DATA[i].qty).append(")"); + if (i < TEST_DATA.length - 1) { + sqlQueryBuilder.append(","); + } + } + return sqlQueryBuilder.toString(); + } + + @SuppressWarnings("unused") // used in string constant in prepareDatabase + public static final OutputStream DEV_NULL = new OutputStream() { + @Override + public void write(int b) { + } + }; + + public static void initSchema(DbMetadata dbMetadata) throws ClassNotFoundException, SQLException { + System.setProperty("derby.stream.error.field", JdbcTestFixture.class.getCanonicalName() + ".DEV_NULL"); + Class.forName(dbMetadata.getDriverClass()); + try (Connection conn = DriverManager.getConnection(dbMetadata.getInitUrl())) { + createTable(conn, JdbcTestFixture.INPUT_TABLE); + createTable(conn, OUTPUT_TABLE); + createTable(conn, OUTPUT_TABLE_2); + } + } + + static void initData(DbMetadata dbMetadata) throws SQLException { + try (Connection conn = DriverManager.getConnection(dbMetadata.getUrl())) { + insertDataIntoInputTable(conn); + } + } + + private static void createTable(Connection conn, String tableName) throws SQLException { + Statement stat = conn.createStatement(); + stat.executeUpdate(getCreateQuery(tableName)); + stat.close(); + } + + private static void insertDataIntoInputTable(Connection conn) throws SQLException { + Statement stat = conn.createStatement(); + stat.execute(getInsertQuery()); + stat.close(); + } + + public static void cleanUpDatabasesStatic(DbMetadata dbMetadata) throws ClassNotFoundException, SQLException { + Class.forName(dbMetadata.getDriverClass()); + try ( + Connection conn = DriverManager.getConnection(dbMetadata.getUrl()); + Statement stat = conn.createStatement()) { + + stat.executeUpdate("DROP TABLE " + INPUT_TABLE); + stat.executeUpdate("DROP TABLE " + OUTPUT_TABLE); + stat.executeUpdate("DROP TABLE " + OUTPUT_TABLE_2); + } + } + + static void cleanupData(String url) throws Exception { + try (Connection conn = DriverManager.getConnection(url); + Statement st = conn.createStatement()) { + st.executeUpdate("delete from " + INPUT_TABLE); + } + } + +} -- Gitee From 713bf4c640dae4f3f034c4d506d54f5e9507c140 Mon Sep 17 00:00:00 2001 From: Roman Khachatryan Date: Wed, 29 Jan 2020 19:59:50 +0100 Subject: [PATCH 118/885] [FLINK-15782][connectors/jdbc] add JDBC sink DataStream API ParameterSetter renamed to StatementBuilder: semantically it builds a statement from a user POJO by setting parameters. --- docs/dev/connectors/index.md | 1 + docs/dev/connectors/index.zh.md | 1 + docs/dev/connectors/jdbc.md | 97 +++++++++++++++++ docs/dev/connectors/jdbc.zh.md | 97 +++++++++++++++++ docs/redirects/jdbc.md | 24 ++++ docs/redirects/jdbc.zh.md | 24 ++++ .../io/jdbc/JdbcBatchingOutputFormat.java | 2 +- .../flink/api/java/io/jdbc/JdbcSink.java | 65 +++++++++++ ...rSetter.java => JdbcStatementBuilder.java} | 13 ++- .../executor/InsertOrUpdateJdbcExecutor.java | 14 ++- .../executor/JdbcBatchStatementExecutor.java | 15 +-- .../executor/KeyedBatchStatementExecutor.java | 8 +- .../SimpleBatchStatementExecutor.java | 8 +- .../flink/api/java/io/jdbc/JdbcE2eTest.java | 103 ++++++++++++++++++ .../api/java/io/jdbc/JdbcTestFixture.java | 8 +- 15 files changed, 452 insertions(+), 28 deletions(-) create mode 100644 docs/dev/connectors/jdbc.md create mode 100644 docs/dev/connectors/jdbc.zh.md create mode 100644 docs/redirects/jdbc.md create mode 100644 docs/redirects/jdbc.zh.md create mode 100644 flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcSink.java rename flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/{executor/ParameterSetter.java => JdbcStatementBuilder.java} (76%) create mode 100644 flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcE2eTest.java diff --git a/docs/dev/connectors/index.md b/docs/dev/connectors/index.md index 61320148ba..e83f277db1 100644 --- a/docs/dev/connectors/index.md +++ b/docs/dev/connectors/index.md @@ -48,6 +48,7 @@ Connectors provide code for interfacing with various third-party systems. Curren * [Apache NiFi](nifi.html) (source/sink) * [Twitter Streaming API](twitter.html) (source) * [Google PubSub](pubsub.html) (source/sink) + * [JDBC](jdbc.html) (sink) Keep in mind that to use one of these connectors in an application, additional third party components are usually required, e.g. servers for the data stores or message queues. diff --git a/docs/dev/connectors/index.zh.md b/docs/dev/connectors/index.zh.md index e5896abfbe..e076c85ae9 100644 --- a/docs/dev/connectors/index.zh.md +++ b/docs/dev/connectors/index.zh.md @@ -47,6 +47,7 @@ under the License. * [Apache NiFi](nifi.html) (source/sink) * [Twitter Streaming API](twitter.html) (source) * [Google PubSub](pubsub.html) (source/sink) + * [JDBC](jdbc.html) (sink) 请记住,在使用一种连接器时,通常需要额外的第三方组件,比如:数据存储服务器或者消息队列。 要注意这些列举的连接器是 Flink 工程的一部分,包含在发布的源码中,但是不包含在二进制发行版中。 diff --git a/docs/dev/connectors/jdbc.md b/docs/dev/connectors/jdbc.md new file mode 100644 index 0000000000..8817053fd6 --- /dev/null +++ b/docs/dev/connectors/jdbc.md @@ -0,0 +1,97 @@ +--- +title: "JDBC Connector" +nav-title: JDBC +nav-parent_id: connectors +nav-pos: 9 +--- + + +* This will be replaced by the TOC +{:toc} + + +--- +title: "JDBC Connector" +nav-title: JDBC +nav-parent_id: connectors +nav-pos: 9 +--- + + +* This will be replaced by the TOC +{:toc} + + +This connector provides a sink that writes data to a JDBC database. + +To use this it, add the following dependency to your project (along with your JDBC-driver): + +{% highlight xml %} + + org.apache.flink + flink-jdbc{{ site.scala_version_suffix }} + {{site.version }} + +{% endhighlight %} + +Note that the streaming connectors are currently __NOT__ part of the binary distribution. See how to link with them for cluster execution [here]({{ site.baseurl}}/dev/projectsetup/dependencies.html). + +Created JDBC sink provides at-least-once guarantee. +Effectively exactly-once can be achived using upsert statements or idempotent updates. + +Example usage: +{% highlight java %} +StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); +env + .fromElements(...) + .addSink(JdbcFacade.sink( + "insert into books (id, title, author, price, qty) values (?,?,?,?,?)", + (ps, t) -> { + ps.setInt(1, t.id); + ps.setString(2, t.title); + ps.setString(3, t.author); + ps.setDouble(4, t.price); + ps.setInt(5, t.qty); + }, + new JdbcConnectionOptions.JdbcConnectionOptionsBuilder() + .withUrl(getDbMetadata().getUrl()) + .withDriverName(getDbMetadata().getDriverClass()) + .build())); +env.execute(); +{% endhighlight %} + +Please refer to the [API documentation]({{ site.javadocs_baseurl }}/api/java/org/apache/flink/api/java/io/jdbc/JdbcSink.html) for more details. diff --git a/docs/dev/connectors/jdbc.zh.md b/docs/dev/connectors/jdbc.zh.md new file mode 100644 index 0000000000..8817053fd6 --- /dev/null +++ b/docs/dev/connectors/jdbc.zh.md @@ -0,0 +1,97 @@ +--- +title: "JDBC Connector" +nav-title: JDBC +nav-parent_id: connectors +nav-pos: 9 +--- + + +* This will be replaced by the TOC +{:toc} + + +--- +title: "JDBC Connector" +nav-title: JDBC +nav-parent_id: connectors +nav-pos: 9 +--- + + +* This will be replaced by the TOC +{:toc} + + +This connector provides a sink that writes data to a JDBC database. + +To use this it, add the following dependency to your project (along with your JDBC-driver): + +{% highlight xml %} + + org.apache.flink + flink-jdbc{{ site.scala_version_suffix }} + {{site.version }} + +{% endhighlight %} + +Note that the streaming connectors are currently __NOT__ part of the binary distribution. See how to link with them for cluster execution [here]({{ site.baseurl}}/dev/projectsetup/dependencies.html). + +Created JDBC sink provides at-least-once guarantee. +Effectively exactly-once can be achived using upsert statements or idempotent updates. + +Example usage: +{% highlight java %} +StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); +env + .fromElements(...) + .addSink(JdbcFacade.sink( + "insert into books (id, title, author, price, qty) values (?,?,?,?,?)", + (ps, t) -> { + ps.setInt(1, t.id); + ps.setString(2, t.title); + ps.setString(3, t.author); + ps.setDouble(4, t.price); + ps.setInt(5, t.qty); + }, + new JdbcConnectionOptions.JdbcConnectionOptionsBuilder() + .withUrl(getDbMetadata().getUrl()) + .withDriverName(getDbMetadata().getDriverClass()) + .build())); +env.execute(); +{% endhighlight %} + +Please refer to the [API documentation]({{ site.javadocs_baseurl }}/api/java/org/apache/flink/api/java/io/jdbc/JdbcSink.html) for more details. diff --git a/docs/redirects/jdbc.md b/docs/redirects/jdbc.md new file mode 100644 index 0000000000..988e52901c --- /dev/null +++ b/docs/redirects/jdbc.md @@ -0,0 +1,24 @@ +--- +title: "JDBC Connector" +layout: redirect +redirect: /dev/connectors/jdbc.html +permalink: /apis/streaming/connectors/jdbc.html +--- + diff --git a/docs/redirects/jdbc.zh.md b/docs/redirects/jdbc.zh.md new file mode 100644 index 0000000000..0f65863b87 --- /dev/null +++ b/docs/redirects/jdbc.zh.md @@ -0,0 +1,24 @@ +--- +title: "JDBC Connector" +layout: redirect +redirect: /dev/connectors/jdbc.zh.html +permalink: /apis/streaming/connectors/jdbc.zh.html +--- + diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcBatchingOutputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcBatchingOutputFormat.java index a3d44a588f..292b3d66d5 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcBatchingOutputFormat.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcBatchingOutputFormat.java @@ -282,7 +282,7 @@ class JdbcBatchingOutputFormat( new SimpleJdbcConnectionProvider(options), JdbcExecutionOptions.builder().withBatchSize(flushMaxSize).withMaxRetries(maxRetryTimes).withBatchIntervalMs(flushIntervalMills).build(), - unused -> JdbcBatchStatementExecutor.simpleRow(sql, dml.getFieldTypes()), + ctx -> JdbcBatchStatementExecutor.simpleRow(sql, dml.getFieldTypes(), ctx.getExecutionConfig().isObjectReuseEnabled()), tuple2 -> { Preconditions.checkArgument(tuple2.f0); return tuple2.f1; diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcSink.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcSink.java new file mode 100644 index 0000000000..7c2d64119c --- /dev/null +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcSink.java @@ -0,0 +1,65 @@ +/* + * 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.api.java.io.jdbc; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.java.io.jdbc.executor.JdbcBatchStatementExecutor; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; + +import java.util.function.Function; + +/** + * Facade to create JDBC {@link SinkFunction sinks}. + */ +@PublicEvolving +public class JdbcSink { + + /** + * Create a JDBC sink with the default {@link JdbcExecutionOptions}. + * + * @see #sink(String, JdbcStatementBuilder, JdbcExecutionOptions, JdbcConnectionOptions) + */ + public static SinkFunction sink(String sql, JdbcStatementBuilder statementBuilder, JdbcConnectionOptions connectionOptions) { + return sink(sql, statementBuilder, JdbcExecutionOptions.defaults(), connectionOptions); + } + + /** + * Create a JDBC sink. + * + * @param sql arbitrary DML query (e.g. insert, update, upsert) + * @param statementBuilder sets parameters on {@link java.sql.PreparedStatement} according to the query + * @param type of data in {@link org.apache.flink.streaming.runtime.streamrecord.StreamRecord StreamRecord}. + * @param executionOptions parameters of execution, such as batch size and maximum retries + * @param connectionOptions parameters of connection, such as JDBC URL + */ + public static SinkFunction sink( + String sql, + JdbcStatementBuilder statementBuilder, + JdbcExecutionOptions executionOptions, + JdbcConnectionOptions connectionOptions) { + return new GenericJdbcSinkFunction<>(new JdbcBatchingOutputFormat<>( + new SimpleJdbcConnectionProvider(connectionOptions), + executionOptions, + unused -> JdbcBatchStatementExecutor.simple(sql, statementBuilder), + JdbcBatchingOutputFormat.RecordExtractor.identity() + )); + } + + private JdbcSink() { + } +} diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/ParameterSetter.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcStatementBuilder.java similarity index 76% rename from flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/ParameterSetter.java rename to flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcStatementBuilder.java index bb6b6f5d79..5cdba59863 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/ParameterSetter.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcStatementBuilder.java @@ -15,9 +15,10 @@ * limitations under the License. */ -package org.apache.flink.api.java.io.jdbc.executor; +package org.apache.flink.api.java.io.jdbc; -import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.java.io.jdbc.executor.JdbcBatchStatementExecutor; import org.apache.flink.types.Row; import org.apache.flink.util.function.BiConsumerWithException; @@ -32,14 +33,14 @@ import static org.apache.flink.api.java.io.jdbc.JDBCUtils.setRecordToStatement; * @param type of payload in {@link org.apache.flink.streaming.runtime.streamrecord.StreamRecord StreamRecord} * @see JdbcBatchStatementExecutor */ -@Internal -public interface ParameterSetter extends BiConsumerWithException, Serializable { +@PublicEvolving +public interface JdbcStatementBuilder extends BiConsumerWithException, Serializable { /** - * Creates a {@link ParameterSetter} for {@link Row} using the provided SQL types array. + * Creates a {@link JdbcStatementBuilder} for {@link Row} using the provided SQL types array. * Uses {@link org.apache.flink.api.java.io.jdbc.JDBCUtils#setRecordToStatement} */ - static ParameterSetter forRow(int[] types) { + static JdbcStatementBuilder forRow(int[] types) { return (st, record) -> setRecordToStatement(st, types, record); } } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/InsertOrUpdateJdbcExecutor.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/InsertOrUpdateJdbcExecutor.java index 0a5d873db4..9860a92de2 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/InsertOrUpdateJdbcExecutor.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/InsertOrUpdateJdbcExecutor.java @@ -17,6 +17,8 @@ package org.apache.flink.api.java.io.jdbc.executor; +import org.apache.flink.api.java.io.jdbc.JdbcStatementBuilder; + import java.sql.Connection; import java.sql.PreparedStatement; import java.sql.ResultSet; @@ -32,9 +34,9 @@ final class InsertOrUpdateJdbcExecutor implements JdbcBatchStatementExe private final String insertSQL; private final String updateSQL; - private final ParameterSetter existSetter; - private final ParameterSetter insertSetter; - private final ParameterSetter updateSetter; + private final JdbcStatementBuilder existSetter; + private final JdbcStatementBuilder insertSetter; + private final JdbcStatementBuilder updateSetter; private final Function keyExtractor; private final Function valueMapper; @@ -47,9 +49,9 @@ final class InsertOrUpdateJdbcExecutor implements JdbcBatchStatementExe InsertOrUpdateJdbcExecutor(String existSQL, String insertSQL, String updateSQL, - ParameterSetter existSetter, - ParameterSetter insertSetter, - ParameterSetter updateSetter, + JdbcStatementBuilder existSetter, + JdbcStatementBuilder insertSetter, + JdbcStatementBuilder updateSetter, Function keyExtractor, Function valueExtractor) { this.existSQL = existSQL; diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/JdbcBatchStatementExecutor.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/JdbcBatchStatementExecutor.java index 7e436d06df..97454c4cff 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/JdbcBatchStatementExecutor.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/JdbcBatchStatementExecutor.java @@ -21,6 +21,7 @@ package org.apache.flink.api.java.io.jdbc.executor; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.java.io.jdbc.JdbcDmlOptions; +import org.apache.flink.api.java.io.jdbc.JdbcStatementBuilder; import org.apache.flink.types.Row; import java.sql.Connection; @@ -69,9 +70,9 @@ public interface JdbcBatchStatementExecutor { opt.getDialect().getRowExistsStatement(opt.getTableName(), opt.getKeyFields()), opt.getDialect().getInsertIntoStatement(opt.getTableName(), opt.getFieldNames()), opt.getDialect().getUpdateStatement(opt.getTableName(), opt.getFieldNames(), opt.getKeyFields()), - ParameterSetter.forRow(pkTypes), - ParameterSetter.forRow(opt.getFieldTypes()), - ParameterSetter.forRow(opt.getFieldTypes()), + JdbcStatementBuilder.forRow(pkTypes), + JdbcStatementBuilder.forRow(opt.getFieldTypes()), + JdbcStatementBuilder.forRow(opt.getFieldTypes()), rowKeyExtractor(pkFields), ctx.getExecutionConfig().isObjectReuseEnabled() ? Row::copy : Function.identity())); } @@ -86,15 +87,15 @@ public interface JdbcBatchStatementExecutor { (st, record) -> setRecordToStatement(st, pkTypes, rowKeyExtractor(pkFields).apply(record))); } - static JdbcBatchStatementExecutor keyed(String sql, Function keyExtractor, ParameterSetter parameterSetter) { - return new KeyedBatchStatementExecutor<>(sql, keyExtractor, parameterSetter); + static JdbcBatchStatementExecutor keyed(String sql, Function keyExtractor, JdbcStatementBuilder statementBuilder) { + return new KeyedBatchStatementExecutor<>(sql, keyExtractor, statementBuilder); } static JdbcBatchStatementExecutor simpleRow(String sql, int[] fieldTypes, boolean objectReuse) { - return simple(sql, ParameterSetter.forRow(fieldTypes), objectReuse ? Row::copy : Function.identity()); + return simple(sql, JdbcStatementBuilder.forRow(fieldTypes), objectReuse ? Row::copy : Function.identity()); } - static JdbcBatchStatementExecutor simple(String sql, ParameterSetter paramSetter, Function valueTransformer) { + static JdbcBatchStatementExecutor simple(String sql, JdbcStatementBuilder paramSetter, Function valueTransformer) { return new SimpleBatchStatementExecutor<>(sql, paramSetter, valueTransformer); } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/KeyedBatchStatementExecutor.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/KeyedBatchStatementExecutor.java index 1da9f45dd5..928f73bd3f 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/KeyedBatchStatementExecutor.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/KeyedBatchStatementExecutor.java @@ -18,6 +18,8 @@ package org.apache.flink.api.java.io.jdbc.executor; +import org.apache.flink.api.java.io.jdbc.JdbcStatementBuilder; + import java.sql.Connection; import java.sql.PreparedStatement; import java.sql.SQLException; @@ -28,7 +30,7 @@ import java.util.function.Function; class KeyedBatchStatementExecutor implements JdbcBatchStatementExecutor { private final String sql; - private final ParameterSetter parameterSetter; + private final JdbcStatementBuilder parameterSetter; private final Function keyExtractor; private transient Set batch = new HashSet<>(); @@ -37,8 +39,8 @@ class KeyedBatchStatementExecutor implements JdbcBatchStatementExecutor /** * Keep in mind object reuse: if it's on then key extractor may be required to return new object. */ - KeyedBatchStatementExecutor(String sql, Function keyExtractor, ParameterSetter parameterSetter) { - this.parameterSetter = parameterSetter; + KeyedBatchStatementExecutor(String sql, Function keyExtractor, JdbcStatementBuilder statementBuilder) { + this.parameterSetter = statementBuilder; this.keyExtractor = keyExtractor; this.sql = sql; } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/SimpleBatchStatementExecutor.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/SimpleBatchStatementExecutor.java index ff92b9f0ca..f2588ab4eb 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/SimpleBatchStatementExecutor.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/SimpleBatchStatementExecutor.java @@ -18,6 +18,8 @@ package org.apache.flink.api.java.io.jdbc.executor; +import org.apache.flink.api.java.io.jdbc.JdbcStatementBuilder; + import java.sql.Connection; import java.sql.PreparedStatement; import java.sql.SQLException; @@ -28,15 +30,15 @@ import java.util.function.Function; class SimpleBatchStatementExecutor implements JdbcBatchStatementExecutor { private final String sql; - private final ParameterSetter parameterSetter; + private final JdbcStatementBuilder parameterSetter; private final Function valueTransformer; private transient PreparedStatement st; private transient List batch; - SimpleBatchStatementExecutor(String sql, ParameterSetter parameterSetter, Function valueTransformer) { + SimpleBatchStatementExecutor(String sql, JdbcStatementBuilder statementBuilder, Function valueTransformer) { this.sql = sql; - this.parameterSetter = parameterSetter; + this.parameterSetter = statementBuilder; this.valueTransformer = valueTransformer; } diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcE2eTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcE2eTest.java new file mode 100644 index 0000000000..3f50607756 --- /dev/null +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcE2eTest.java @@ -0,0 +1,103 @@ +/* + * 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.api.java.io.jdbc; + +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.java.io.jdbc.JdbcTestFixture.TestEntry; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + +import org.junit.Ignore; +import org.junit.Test; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.sql.Types; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static org.apache.flink.api.java.io.jdbc.JdbcTestFixture.INPUT_TABLE; +import static org.apache.flink.api.java.io.jdbc.JdbcTestFixture.INSERT_TEMPLATE; +import static org.apache.flink.api.java.io.jdbc.JdbcTestFixture.TEST_DATA; +import static org.junit.Assert.assertEquals; + +/** + * Smoke tests for the {@link JdbcSink} and the underlying classes. + */ +public class JdbcE2eTest extends JDBCTestBase { + + @Test + @Ignore + public void testInsert() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setRestartStrategy(new RestartStrategies.NoRestartStrategyConfiguration()); + env.setParallelism(1); + env + .fromElements(TEST_DATA) + .addSink(JdbcSink.sink( + String.format(INSERT_TEMPLATE, INPUT_TABLE), + (ps, t) -> { + ps.setInt(1, t.id); + ps.setString(2, t.title); + ps.setString(3, t.author); + if (t.price == null) { + ps.setNull(4, Types.DOUBLE); + } else { + ps.setDouble(4, t.price); + } + ps.setInt(5, t.qty); + }, + new JdbcConnectionOptions.JdbcConnectionOptionsBuilder() + .withUrl(getDbMetadata().getUrl()) + .withDriverName(getDbMetadata().getDriverClass()) + .build())); + env.execute(); + + assertEquals(Arrays.asList(TEST_DATA), selectBooks()); + } + + private List selectBooks() throws SQLException { + List result = new ArrayList<>(); + try (Connection connection = DriverManager.getConnection(getDbMetadata().getUrl())) { + connection.setTransactionIsolation(Connection.TRANSACTION_READ_COMMITTED); + connection.setReadOnly(true); + try (Statement st = connection.createStatement()) { + try (ResultSet rs = st.executeQuery("select id, title, author, price, qty from " + INPUT_TABLE)) { + while (rs.next()) { + result.add(new TestEntry( + rs.getInt(1), + rs.getString(2), + rs.getString(3), + rs.getDouble(4), + rs.getInt(5) + )); + } + } + } + } + return result; + } + + @Override + protected DbMetadata getDbMetadata() { + return JdbcTestFixture.DERBY_EBOOKSHOP_DB; + } +} diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTestFixture.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTestFixture.java index 071a2afcf3..a822d31f09 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTestFixture.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTestFixture.java @@ -74,7 +74,7 @@ public class JdbcTestFixture { public final Double price; public final Integer qty; - private TestEntry(Integer id, String title, String author, Double price, Integer qty) { + TestEntry(Integer id, String title, String author, Double price, Integer qty) { this.id = id; this.title = title; this.author = author; @@ -82,6 +82,10 @@ public class JdbcTestFixture { this.qty = qty; } + public TestEntry withTitle(String newTitle) { + return new TestEntry(id, newTitle, author, price, qty); + } + @Override public String toString() { return "TestEntry{" + @@ -176,7 +180,7 @@ public class JdbcTestFixture { static void cleanupData(String url) throws Exception { try (Connection conn = DriverManager.getConnection(url); - Statement st = conn.createStatement()) { + Statement st = conn.createStatement()) { st.executeUpdate("delete from " + INPUT_TABLE); } } -- Gitee From 5afe1b5b3d49b3142fb07246b15991511c1ddc77 Mon Sep 17 00:00:00 2001 From: Roman Khachatryan Date: Thu, 27 Feb 2020 14:33:25 +0100 Subject: [PATCH 119/885] [FLINK-15782][connectors/jdbc][pr-review] use SimpleBatchStatementExecutor for Upsert [pr-review] enforce use of Builder and Optional in JdbcOptions [pr-review] don't expose Row in DataStream API [pr-review] fix typo and remove redirects in docs; add some javadoc [pr-review] rename ExecutorCreator to StatementExecutorCreator and some vars/methods [pr-review] address formatting and some other minor issues This closes #11061. --- docs/dev/connectors/jdbc.md | 2 +- docs/dev/connectors/jdbc.zh.md | 2 +- docs/redirects/jdbc.md | 24 ---- docs/redirects/jdbc.zh.md | 24 ---- .../java/io/jdbc/GenericJdbcSinkFunction.java | 7 +- .../api/java/io/jdbc/JDBCInputFormat.java | 2 +- .../api/java/io/jdbc/JDBCLookupFunction.java | 4 +- .../api/java/io/jdbc/JDBCOutputFormat.java | 23 ++-- .../api/java/io/jdbc/JDBCTableSource.java | 4 +- .../io/jdbc/JdbcBatchingOutputFormat.java | 104 +++++++++++------- .../java/io/jdbc/JdbcConnectionOptions.java | 17 ++- .../api/java/io/jdbc/JdbcDmlOptions.java | 9 +- .../java/io/jdbc/JdbcExecutionOptions.java | 14 +-- .../flink/api/java/io/jdbc/JdbcSink.java | 27 +++-- .../java/io/jdbc/JdbcStatementBuilder.java | 10 -- .../io/jdbc/SimpleJdbcConnectionProvider.java | 6 +- .../io/jdbc/TableJdbcUpsertOutputFormat.java | 52 ++++++++- .../java/io/jdbc/dialect/JDBCDialects.java | 2 +- .../executor/InsertOrUpdateJdbcExecutor.java | 47 +++++--- .../executor/JdbcBatchStatementExecutor.java | 47 +------- .../executor/KeyedBatchStatementExecutor.java | 5 +- .../SimpleBatchStatementExecutor.java | 5 +- .../java/io/jdbc/JDBCAppenOnlyWriterTest.java | 27 +++-- .../api/java/io/jdbc/JDBCTestCheckpoint.java | 35 ------ .../{JdbcE2eTest.java => JdbcITCase.java} | 39 +++---- .../api/java/io/jdbc/JdbcTestFixture.java | 2 - 26 files changed, 255 insertions(+), 285 deletions(-) delete mode 100644 docs/redirects/jdbc.md delete mode 100644 docs/redirects/jdbc.zh.md delete mode 100644 flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTestCheckpoint.java rename flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/{JdbcE2eTest.java => JdbcITCase.java} (81%) diff --git a/docs/dev/connectors/jdbc.md b/docs/dev/connectors/jdbc.md index 8817053fd6..6dc261a4e5 100644 --- a/docs/dev/connectors/jdbc.md +++ b/docs/dev/connectors/jdbc.md @@ -58,7 +58,7 @@ under the License. This connector provides a sink that writes data to a JDBC database. -To use this it, add the following dependency to your project (along with your JDBC-driver): +To use it, add the following dependency to your project (along with your JDBC-driver): {% highlight xml %} diff --git a/docs/dev/connectors/jdbc.zh.md b/docs/dev/connectors/jdbc.zh.md index 8817053fd6..6dc261a4e5 100644 --- a/docs/dev/connectors/jdbc.zh.md +++ b/docs/dev/connectors/jdbc.zh.md @@ -58,7 +58,7 @@ under the License. This connector provides a sink that writes data to a JDBC database. -To use this it, add the following dependency to your project (along with your JDBC-driver): +To use it, add the following dependency to your project (along with your JDBC-driver): {% highlight xml %} diff --git a/docs/redirects/jdbc.md b/docs/redirects/jdbc.md deleted file mode 100644 index 988e52901c..0000000000 --- a/docs/redirects/jdbc.md +++ /dev/null @@ -1,24 +0,0 @@ ---- -title: "JDBC Connector" -layout: redirect -redirect: /dev/connectors/jdbc.html -permalink: /apis/streaming/connectors/jdbc.html ---- - diff --git a/docs/redirects/jdbc.zh.md b/docs/redirects/jdbc.zh.md deleted file mode 100644 index 0f65863b87..0000000000 --- a/docs/redirects/jdbc.zh.md +++ /dev/null @@ -1,24 +0,0 @@ ---- -title: "JDBC Connector" -layout: redirect -redirect: /dev/connectors/jdbc.zh.html -permalink: /apis/streaming/connectors/jdbc.zh.html ---- - diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/GenericJdbcSinkFunction.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/GenericJdbcSinkFunction.java index 0fe21b47a7..40f6341cf6 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/GenericJdbcSinkFunction.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/GenericJdbcSinkFunction.java @@ -24,14 +24,17 @@ import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.FunctionSnapshotContext; import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.util.Preconditions; + +import javax.annotation.Nonnull; import java.io.IOException; class GenericJdbcSinkFunction extends RichSinkFunction implements CheckpointedFunction { private final AbstractJdbcOutputFormat outputFormat; - GenericJdbcSinkFunction(AbstractJdbcOutputFormat outputFormat) { - this.outputFormat = outputFormat; + GenericJdbcSinkFunction(@Nonnull AbstractJdbcOutputFormat outputFormat) { + this.outputFormat = Preconditions.checkNotNull(outputFormat); } @Override diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java index cba125ed85..3f149178f2 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java @@ -346,7 +346,7 @@ public class JDBCInputFormat extends RichInputFormat implements } /** - * Builder for a {@link JDBCInputFormat}. + * Builder for {@link JDBCInputFormat}. */ public static class JDBCInputFormatBuilder { private final JDBCInputFormat format; diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCLookupFunction.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCLookupFunction.java index 4cfa4984f3..dfe996c9be 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCLookupFunction.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCLookupFunction.java @@ -84,8 +84,8 @@ public class JDBCLookupFunction extends TableFunction { String[] fieldNames, TypeInformation[] fieldTypes, String[] keyNames) { this.drivername = options.getDriverName(); this.dbURL = options.getDbURL(); - this.username = options.getUsername(); - this.password = options.getPassword(); + this.username = options.getUsername().orElse(null); + this.password = options.getPassword().orElse(null); this.fieldNames = fieldNames; this.fieldTypes = fieldTypes; List nameList = Arrays.asList(fieldNames); diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java index 6fd0ffcda7..51b29a9b1c 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java @@ -18,6 +18,7 @@ package org.apache.flink.api.java.io.jdbc; +import org.apache.flink.api.java.io.jdbc.JdbcConnectionOptions.JdbcConnectionOptionsBuilder; import org.apache.flink.types.Row; import org.slf4j.Logger; @@ -53,16 +54,16 @@ public class JDBCOutputFormat extends AbstractJdbcOutputFormat { private transient int batchCount = 0; /** - * @deprecated use {@link #JDBCOutputFormat(JdbcConnectionProvider, JdbcInsertOptions, JdbcExecutionOptions)}}. + * @deprecated use {@link JDBCOutputFormatBuilder builder} instead. */ @Deprecated public JDBCOutputFormat(String username, String password, String drivername, String dbURL, String query, int batchInterval, int[] typesArray) { - this(new SimpleJdbcConnectionProvider(new JdbcConnectionOptions(dbURL, drivername, username, password)), + this(new SimpleJdbcConnectionProvider(new JdbcConnectionOptionsBuilder().withUrl(dbURL).withDriverName(drivername).withUsername(username).withPassword(password).build()), new JdbcInsertOptions(query, typesArray), JdbcExecutionOptions.builder().withBatchSize(batchInterval).build()); } - public JDBCOutputFormat(JdbcConnectionProvider connectionProvider, JdbcInsertOptions insertOptions, JdbcExecutionOptions batchOptions) { + private JDBCOutputFormat(JdbcConnectionProvider connectionProvider, JdbcInsertOptions insertOptions, JdbcExecutionOptions batchOptions) { super(connectionProvider); this.insertOptions = insertOptions; this.batchOptions = batchOptions; @@ -145,7 +146,7 @@ public class JDBCOutputFormat extends AbstractJdbcOutputFormat { } /** - * Builder for a {@link JDBCOutputFormat}. + * Builder for {@link JDBCOutputFormat}. */ public static class JDBCOutputFormatBuilder { private String username; @@ -199,9 +200,10 @@ public class JDBCOutputFormat extends AbstractJdbcOutputFormat { * @return Configured JDBCOutputFormat */ public JDBCOutputFormat finish() { - return new JDBCOutputFormat(new SimpleJdbcConnectionProvider(buildConnectionOptions()), - new JdbcInsertOptions(query, typesArray), - JdbcExecutionOptions.builder().withBatchSize(batchInterval).build()); + return new JDBCOutputFormat( + new SimpleJdbcConnectionProvider(buildConnectionOptions()), + new JdbcInsertOptions(query, typesArray), + JdbcExecutionOptions.builder().withBatchSize(batchInterval).build()); } public JdbcConnectionOptions buildConnectionOptions() { @@ -212,7 +214,12 @@ public class JDBCOutputFormat extends AbstractJdbcOutputFormat { LOG.info("Password was not supplied."); } - return new JdbcConnectionOptions(dbURL, drivername, username, password); + return new JdbcConnectionOptionsBuilder() + .withUrl(dbURL) + .withDriverName(drivername) + .withUsername(username) + .withPassword(password) + .build(); } } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCTableSource.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCTableSource.java index 60339e2276..1775be775f 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCTableSource.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCTableSource.java @@ -155,9 +155,9 @@ public class JDBCTableSource implements JDBCInputFormat.JDBCInputFormatBuilder builder = JDBCInputFormat.buildJDBCInputFormat() .setDrivername(options.getDriverName()) .setDBUrl(options.getDbURL()) - .setUsername(options.getUsername()) - .setPassword(options.getPassword()) .setRowTypeInfo(new RowTypeInfo(rowTypeInfo.getFieldTypes(), rowTypeInfo.getFieldNames())); + options.getUsername().ifPresent(builder::setUsername); + options.getPassword().ifPresent(builder::setPassword); if (readOptions.getFetchSize() != 0) { builder.setFetchSize(readOptions.getFetchSize()); diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcBatchingOutputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcBatchingOutputFormat.java index 292b3d66d5..4020119f27 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcBatchingOutputFormat.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcBatchingOutputFormat.java @@ -19,6 +19,7 @@ package org.apache.flink.api.java.io.jdbc; import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.java.io.jdbc.JdbcExecutionOptions.JdbcExecutionOptionsBuilder; import org.apache.flink.api.java.io.jdbc.executor.JdbcBatchStatementExecutor; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.util.ExecutorThreadFactory; @@ -28,6 +29,8 @@ import org.apache.flink.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nonnull; + import java.io.IOException; import java.io.Serializable; import java.sql.SQLException; @@ -37,6 +40,7 @@ import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.function.Function; +import static org.apache.flink.api.java.io.jdbc.JDBCUtils.setRecordToStatement; import static org.apache.flink.util.Preconditions.checkNotNull; class JdbcBatchingOutputFormat> extends AbstractJdbcOutputFormat { @@ -46,7 +50,7 @@ class JdbcBatchingOutputFormat> extends Function, Serializable { + interface StatementExecutorFactory> extends Function, Serializable { } private static final long serialVersionUID = 1L; @@ -54,8 +58,8 @@ class JdbcBatchingOutputFormat statementRunnerCreator; - final RecordExtractor jdbcRecordExtractor; + private final StatementExecutorFactory statementExecutorFactory; + private final RecordExtractor jdbcRecordExtractor; private transient JdbcExec jdbcStatementExecutor; private transient int batchCount = 0; @@ -66,14 +70,14 @@ class JdbcBatchingOutputFormat statementExecutorCreator, - RecordExtractor recordExtractor) { + @Nonnull JdbcConnectionProvider connectionProvider, + @Nonnull JdbcExecutionOptions executionOptions, + @Nonnull StatementExecutorFactory statementExecutorFactory, + @Nonnull RecordExtractor recordExtractor) { super(connectionProvider); - this.executionOptions = executionOptions; - this.statementRunnerCreator = statementExecutorCreator; - this.jdbcRecordExtractor = recordExtractor; + this.executionOptions = checkNotNull(executionOptions); + this.statementExecutorFactory = checkNotNull(statementExecutorFactory); + this.jdbcRecordExtractor = checkNotNull(recordExtractor); } /** @@ -84,12 +88,7 @@ class JdbcBatchingOutputFormat { @@ -106,6 +105,16 @@ class JdbcBatchingOutputFormat statementExecutorFactory) throws IOException { + JdbcExec exec = statementExecutorFactory.apply(getRuntimeContext()); + try { + exec.open(connection); + } catch (SQLException e) { + throw new IOException("unable to open JDBC writer", e); + } + return exec; + } + private void checkFlushException() { if (flushException != null) { throw new RuntimeException("Writing records to JDBC failed.", flushException); @@ -113,22 +122,22 @@ class JdbcBatchingOutputFormat= executionOptions.getBatchSize()) { flush(); } } catch (Exception e) { - throw new RuntimeException("Writing records to JDBC failed.", e); + throw new IOException("Writing records to JDBC failed.", e); } } - void doWriteRecord(In record) throws SQLException { - jdbcStatementExecutor.process(jdbcRecordExtractor.apply(record)); + void addToBatch(In original, JdbcIn extracted) throws SQLException { + jdbcStatementExecutor.addToBatch(extracted); } @Override @@ -204,9 +213,7 @@ class JdbcBatchingOutputFormat 0) { + return new TableJdbcUpsertOutputFormat( + new SimpleJdbcConnectionProvider(options), + dml, + executionOptionsBuilder.build()); + } else { // warn: don't close over builder fields String sql = options.getDialect().getInsertIntoStatement(dml.getTableName(), dml.getFieldNames()); return new JdbcBatchingOutputFormat<>( - new SimpleJdbcConnectionProvider(options), - JdbcExecutionOptions.builder().withBatchSize(flushMaxSize).withMaxRetries(maxRetryTimes).withBatchIntervalMs(flushIntervalMills).build(), - ctx -> JdbcBatchStatementExecutor.simpleRow(sql, dml.getFieldTypes(), ctx.getExecutionConfig().isObjectReuseEnabled()), - tuple2 -> { - Preconditions.checkArgument(tuple2.f0); - return tuple2.f1; - }); - } else { - return new TableJdbcUpsertOutputFormat( - new SimpleJdbcConnectionProvider(options), - dml, - JdbcExecutionOptions.builder().withBatchSize(flushMaxSize).withBatchIntervalMs(flushIntervalMills).build()); + new SimpleJdbcConnectionProvider(options), + executionOptionsBuilder.build(), + ctx -> createSimpleRowExecutor(sql, dml.getFieldTypes(), ctx.getExecutionConfig().isObjectReuseEnabled()), + tuple2 -> { + Preconditions.checkArgument(tuple2.f0); + return tuple2.f1; + }); } } } + + static JdbcBatchStatementExecutor createSimpleRowExecutor(String sql, int[] fieldTypes, boolean objectReuse) { + return JdbcBatchStatementExecutor.simple(sql, createRowJdbcStatementBuilder(fieldTypes), objectReuse ? Row::copy : Function.identity()); + } + + /** + * Creates a {@link JdbcStatementBuilder} for {@link Row} using the provided SQL types array. + * Uses {@link org.apache.flink.api.java.io.jdbc.JDBCUtils#setRecordToStatement} + */ + static JdbcStatementBuilder createRowJdbcStatementBuilder(int[] types) { + return (st, record) -> setRecordToStatement(st, types, record); + } + } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcConnectionOptions.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcConnectionOptions.java index 7c5d2fbd48..3f95a0a045 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcConnectionOptions.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcConnectionOptions.java @@ -20,7 +20,10 @@ package org.apache.flink.api.java.io.jdbc; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.util.Preconditions; +import javax.annotation.Nullable; + import java.io.Serializable; +import java.util.Optional; /** * JDBC connection options. @@ -32,10 +35,12 @@ public class JdbcConnectionOptions implements Serializable { protected final String url; protected final String driverName; + @Nullable protected final String username; + @Nullable protected final String password; - public JdbcConnectionOptions(String url, String driverName, String username, String password) { + JdbcConnectionOptions(String url, String driverName, String username, String password) { this.url = Preconditions.checkNotNull(url, "jdbc url is empty"); this.driverName = Preconditions.checkNotNull(driverName, "driver name is empty"); this.username = username; @@ -50,16 +55,16 @@ public class JdbcConnectionOptions implements Serializable { return driverName; } - public String getUsername() { - return username; + public Optional getUsername() { + return Optional.ofNullable(username); } - public String getPassword() { - return password; + public Optional getPassword() { + return Optional.ofNullable(password); } /** - * JDBCConnectionOptionsBuilder. + * Builder for {@link JdbcConnectionOptions}. */ public static class JdbcConnectionOptionsBuilder { private String url; diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcDmlOptions.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcDmlOptions.java index 5af701fe03..e78f39026a 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcDmlOptions.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcDmlOptions.java @@ -22,6 +22,7 @@ import org.apache.flink.util.Preconditions; import javax.annotation.Nullable; +import java.util.Optional; import java.util.stream.Stream; /** @@ -54,7 +55,6 @@ public class JdbcDmlOptions extends JdbcTypedQueryOptions { } public JDBCDialect getDialect() { - Preconditions.checkNotNull(dialect, "dialect not set"); return dialect; } @@ -62,19 +62,18 @@ public class JdbcDmlOptions extends JdbcTypedQueryOptions { return fieldNames; } - public String[] getKeyFields() { - return keyFields; + public Optional getKeyFields() { + return Optional.ofNullable(keyFields); } /** - * JDBCUpsertOptionsBuilder. + * Builder for {@link JdbcDmlOptions}. */ public static class JdbcDmlOptionsBuilder extends JDBCUpdateQueryOptionsBuilder { private String tableName; private String[] fieldNames; private String[] keyFields; private JDBCDialect dialect; - private JDBCDialect customDialect; @Override protected JdbcDmlOptionsBuilder self() { diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcExecutionOptions.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcExecutionOptions.java index af22d6cfe0..254e71c4df 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcExecutionOptions.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcExecutionOptions.java @@ -54,8 +54,8 @@ public class JdbcExecutionOptions implements Serializable { return maxRetries; } - public static JdbcBatchOptionsBuilder builder() { - return new JdbcBatchOptionsBuilder(); + public static JdbcExecutionOptionsBuilder builder() { + return new JdbcExecutionOptionsBuilder(); } public static JdbcExecutionOptions defaults() { @@ -63,24 +63,24 @@ public class JdbcExecutionOptions implements Serializable { } /** - * JDBCBatchOptionsBuilder. + * Builder for {@link JdbcExecutionOptions}. */ - public static final class JdbcBatchOptionsBuilder { + public static final class JdbcExecutionOptionsBuilder { private long intervalMs = DEFAULT_INTERVAL_MILLIS; private int size = DEFAULT_SIZE; private int maxRetries = DEFAULT_MAX_RETRY_TIMES; - public JdbcBatchOptionsBuilder withBatchSize(int size) { + public JdbcExecutionOptionsBuilder withBatchSize(int size) { this.size = size; return this; } - public JdbcBatchOptionsBuilder withBatchIntervalMs(long intervalMs) { + public JdbcExecutionOptionsBuilder withBatchIntervalMs(long intervalMs) { this.intervalMs = intervalMs; return this; } - public JdbcBatchOptionsBuilder withMaxRetries(int maxRetries) { + public JdbcExecutionOptionsBuilder withMaxRetries(int maxRetries) { this.maxRetries = maxRetries; return this; } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcSink.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcSink.java index 7c2d64119c..579a5c517e 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcSink.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcSink.java @@ -20,6 +20,7 @@ package org.apache.flink.api.java.io.jdbc; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.java.io.jdbc.executor.JdbcBatchStatementExecutor; import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.util.Preconditions; import java.util.function.Function; @@ -40,23 +41,31 @@ public class JdbcSink { /** * Create a JDBC sink. + *

    + * Note: the objects passed to the return sink can be processed in batch and retried. + * Therefore, objects can not be {@link org.apache.flink.api.common.ExecutionConfig#enableObjectReuse() reused}. + *

    * * @param sql arbitrary DML query (e.g. insert, update, upsert) - * @param statementBuilder sets parameters on {@link java.sql.PreparedStatement} according to the query + * @param statementBuilder sets parameters on {@link java.sql.PreparedStatement} according to the query * @param type of data in {@link org.apache.flink.streaming.runtime.streamrecord.StreamRecord StreamRecord}. * @param executionOptions parameters of execution, such as batch size and maximum retries * @param connectionOptions parameters of connection, such as JDBC URL */ public static SinkFunction sink( - String sql, - JdbcStatementBuilder statementBuilder, - JdbcExecutionOptions executionOptions, - JdbcConnectionOptions connectionOptions) { + String sql, + JdbcStatementBuilder statementBuilder, + JdbcExecutionOptions executionOptions, + JdbcConnectionOptions connectionOptions) { return new GenericJdbcSinkFunction<>(new JdbcBatchingOutputFormat<>( - new SimpleJdbcConnectionProvider(connectionOptions), - executionOptions, - unused -> JdbcBatchStatementExecutor.simple(sql, statementBuilder), - JdbcBatchingOutputFormat.RecordExtractor.identity() + new SimpleJdbcConnectionProvider(connectionOptions), + executionOptions, + context -> { + Preconditions.checkState(!context.getExecutionConfig().isObjectReuseEnabled(), + "objects can not be reused with JDBC sink function"); + return JdbcBatchStatementExecutor.simple(sql, statementBuilder, Function.identity()); + }, + JdbcBatchingOutputFormat.RecordExtractor.identity() )); } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcStatementBuilder.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcStatementBuilder.java index 5cdba59863..603144d70a 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcStatementBuilder.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JdbcStatementBuilder.java @@ -19,15 +19,12 @@ package org.apache.flink.api.java.io.jdbc; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.java.io.jdbc.executor.JdbcBatchStatementExecutor; -import org.apache.flink.types.Row; import org.apache.flink.util.function.BiConsumerWithException; import java.io.Serializable; import java.sql.PreparedStatement; import java.sql.SQLException; -import static org.apache.flink.api.java.io.jdbc.JDBCUtils.setRecordToStatement; - /** * Sets {@link PreparedStatement} parameters to use in JDBC Sink based on a specific type of StreamRecord. * @param type of payload in {@link org.apache.flink.streaming.runtime.streamrecord.StreamRecord StreamRecord} @@ -36,11 +33,4 @@ import static org.apache.flink.api.java.io.jdbc.JDBCUtils.setRecordToStatement; @PublicEvolving public interface JdbcStatementBuilder extends BiConsumerWithException, Serializable { - /** - * Creates a {@link JdbcStatementBuilder} for {@link Row} using the provided SQL types array. - * Uses {@link org.apache.flink.api.java.io.jdbc.JDBCUtils#setRecordToStatement} - */ - static JdbcStatementBuilder forRow(int[] types) { - return (st, record) -> setRecordToStatement(st, types, record); - } } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/SimpleJdbcConnectionProvider.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/SimpleJdbcConnectionProvider.java index 3d457de20f..becd756bd1 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/SimpleJdbcConnectionProvider.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/SimpleJdbcConnectionProvider.java @@ -43,10 +43,10 @@ class SimpleJdbcConnectionProvider implements JdbcConnectionProvider, Serializab synchronized (this) { if (connection == null) { Class.forName(jdbcOptions.getDriverName()); - if (jdbcOptions.getUsername() == null) { - connection = DriverManager.getConnection(jdbcOptions.getDbURL()); + if (jdbcOptions.getUsername().isPresent()) { + connection = DriverManager.getConnection(jdbcOptions.getDbURL(), jdbcOptions.getUsername().get(), jdbcOptions.getPassword().orElse(null)); } else { - connection = DriverManager.getConnection(jdbcOptions.getDbURL(), jdbcOptions.getUsername(), jdbcOptions.getPassword()); + connection = DriverManager.getConnection(jdbcOptions.getDbURL()); } } } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/TableJdbcUpsertOutputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/TableJdbcUpsertOutputFormat.java index b5e5136694..4088120ce4 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/TableJdbcUpsertOutputFormat.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/TableJdbcUpsertOutputFormat.java @@ -17,6 +17,8 @@ package org.apache.flink.api.java.io.jdbc; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.java.io.jdbc.executor.InsertOrUpdateJdbcExecutor; import org.apache.flink.api.java.io.jdbc.executor.JdbcBatchStatementExecutor; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.types.Row; @@ -27,6 +29,11 @@ import org.slf4j.LoggerFactory; import java.io.IOException; import java.sql.SQLException; import java.util.Arrays; +import java.util.function.Function; + +import static org.apache.flink.api.java.io.jdbc.JDBCUtils.getPrimaryKey; +import static org.apache.flink.api.java.io.jdbc.JDBCUtils.setRecordToStatement; +import static org.apache.flink.util.Preconditions.checkArgument; class TableJdbcUpsertOutputFormat extends JdbcBatchingOutputFormat, Row, JdbcBatchStatementExecutor> { private static final Logger LOG = LoggerFactory.getLogger(TableJdbcUpsertOutputFormat.class); @@ -35,7 +42,7 @@ class TableJdbcUpsertOutputFormat extends JdbcBatchingOutputFormat JdbcBatchStatementExecutor.upsertRow(dmlOptions, ctx), tuple2 -> tuple2.f1); + super(connectionProvider, batchOptions, ctx -> createUpsertRowExecutor(dmlOptions, ctx), tuple2 -> tuple2.f1); this.dmlOptions = dmlOptions; } @@ -55,15 +62,15 @@ class TableJdbcUpsertOutputFormat extends JdbcBatchingOutputFormat dmlOptions.getFieldTypes()[f]).toArray(); String deleteSql = dmlOptions.getDialect().getDeleteStatement(dmlOptions.getTableName(), dmlOptions.getFieldNames()); - return JdbcBatchStatementExecutor.keyedRow(pkFields, pkTypes, deleteSql); + return createKeyedRowExecutor(pkFields, pkTypes, deleteSql); } @Override - void doWriteRecord(Tuple2 record) throws SQLException { - if (record.f0) { - super.doWriteRecord(record); + void addToBatch(Tuple2 original, Row extracted) throws SQLException { + if (original.f0) { + super.addToBatch(original, extracted); } else { - deleteExecutor.process(jdbcRecordExtractor.apply(record)); + deleteExecutor.addToBatch(extracted); } } @@ -85,4 +92,37 @@ class TableJdbcUpsertOutputFormat extends JdbcBatchingOutputFormat createKeyedRowExecutor(int[] pkFields, int[] pkTypes, String sql) { + return JdbcBatchStatementExecutor.keyed( + sql, + createRowKeyExtractor(pkFields), + (st, record) -> setRecordToStatement(st, pkTypes, createRowKeyExtractor(pkFields).apply(record))); + } + + private static JdbcBatchStatementExecutor createUpsertRowExecutor(JdbcDmlOptions opt, RuntimeContext ctx) { + checkArgument(opt.getKeyFields().isPresent()); + + int[] pkFields = Arrays.stream(opt.getKeyFields().get()).mapToInt(Arrays.asList(opt.getFieldNames())::indexOf).toArray(); + int[] pkTypes = opt.getFieldTypes() == null ? null : Arrays.stream(pkFields).map(f -> opt.getFieldTypes()[f]).toArray(); + + return opt.getDialect() + .getUpsertStatement(opt.getTableName(), opt.getFieldNames(), opt.getKeyFields().get()) + .map(sql -> createSimpleRowExecutor(sql, opt.getFieldTypes(), ctx.getExecutionConfig().isObjectReuseEnabled())) + .orElseGet(() -> + new InsertOrUpdateJdbcExecutor<>( + opt.getDialect().getRowExistsStatement(opt.getTableName(), opt.getKeyFields().get()), + opt.getDialect().getInsertIntoStatement(opt.getTableName(), opt.getFieldNames()), + opt.getDialect().getUpdateStatement(opt.getTableName(), opt.getFieldNames(), opt.getKeyFields().get()), + createRowJdbcStatementBuilder(pkTypes), + createRowJdbcStatementBuilder(opt.getFieldTypes()), + createRowJdbcStatementBuilder(opt.getFieldTypes()), + createRowKeyExtractor(pkFields), + ctx.getExecutionConfig().isObjectReuseEnabled() ? Row::copy : Function.identity())); + } + + private static Function createRowKeyExtractor(int[] pkFields) { + return row -> getPrimaryKey(row, pkFields); + } + } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialects.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialects.java index c0ef09bdc0..fa7192efe5 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialects.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialects.java @@ -152,7 +152,7 @@ public final class JDBCDialects { return identifier; } -@Override + @Override public String dialectName() { return "derby"; } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/InsertOrUpdateJdbcExecutor.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/InsertOrUpdateJdbcExecutor.java index 9860a92de2..ad9bbbed2c 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/InsertOrUpdateJdbcExecutor.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/InsertOrUpdateJdbcExecutor.java @@ -17,8 +17,11 @@ package org.apache.flink.api.java.io.jdbc.executor; +import org.apache.flink.annotation.Internal; import org.apache.flink.api.java.io.jdbc.JdbcStatementBuilder; +import javax.annotation.Nonnull; + import java.sql.Connection; import java.sql.PreparedStatement; import java.sql.ResultSet; @@ -28,7 +31,14 @@ import java.util.HashMap; import java.util.Map; import java.util.function.Function; -final class InsertOrUpdateJdbcExecutor implements JdbcBatchStatementExecutor { +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * {@link JdbcBatchStatementExecutor} that provides upsert semantics by updating row if it exists and inserting otherwise. + * Used in Table API. + */ +@Internal +public final class InsertOrUpdateJdbcExecutor implements JdbcBatchStatementExecutor { private final String existSQL; private final String insertSQL; @@ -46,22 +56,23 @@ final class InsertOrUpdateJdbcExecutor implements JdbcBatchStatementExe private transient PreparedStatement updateStatement; private transient Map batch = new HashMap<>(); - InsertOrUpdateJdbcExecutor(String existSQL, - String insertSQL, - String updateSQL, - JdbcStatementBuilder existSetter, - JdbcStatementBuilder insertSetter, - JdbcStatementBuilder updateSetter, - Function keyExtractor, - Function valueExtractor) { - this.existSQL = existSQL; - this.insertSQL = insertSQL; - this.updateSQL = updateSQL; - this.existSetter = existSetter; - this.insertSetter = insertSetter; - this.updateSetter = updateSetter; - this.keyExtractor = keyExtractor; - this.valueMapper = valueExtractor; + public InsertOrUpdateJdbcExecutor( + @Nonnull String existSQL, + @Nonnull String insertSQL, + @Nonnull String updateSQL, + @Nonnull JdbcStatementBuilder existSetter, + @Nonnull JdbcStatementBuilder insertSetter, + @Nonnull JdbcStatementBuilder updateSetter, + @Nonnull Function keyExtractor, + @Nonnull Function valueExtractor) { + this.existSQL = checkNotNull(existSQL); + this.insertSQL = checkNotNull(insertSQL); + this.updateSQL = checkNotNull(updateSQL); + this.existSetter = checkNotNull(existSetter); + this.insertSetter = checkNotNull(insertSetter); + this.updateSetter = checkNotNull(updateSetter); + this.keyExtractor = checkNotNull(keyExtractor); + this.valueMapper = checkNotNull(valueExtractor); } @Override @@ -73,7 +84,7 @@ final class InsertOrUpdateJdbcExecutor implements JdbcBatchStatementExe } @Override - public void process(R record) { + public void addToBatch(R record) { batch.put(keyExtractor.apply(record), valueMapper.apply(record)); } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/JdbcBatchStatementExecutor.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/JdbcBatchStatementExecutor.java index 97454c4cff..5be7ba911b 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/JdbcBatchStatementExecutor.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/JdbcBatchStatementExecutor.java @@ -19,22 +19,14 @@ package org.apache.flink.api.java.io.jdbc.executor; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.api.java.io.jdbc.JdbcDmlOptions; import org.apache.flink.api.java.io.jdbc.JdbcStatementBuilder; -import org.apache.flink.types.Row; import java.sql.Connection; import java.sql.SQLException; -import java.util.Arrays; import java.util.function.Function; -import static org.apache.flink.api.java.io.jdbc.JDBCUtils.getPrimaryKey; -import static org.apache.flink.api.java.io.jdbc.JDBCUtils.setRecordToStatement; -import static org.apache.flink.util.Preconditions.checkNotNull; - /** - * JDBCWriter used to execute statements (e.g. INSERT, UPSERT, DELETE). + * Executes the given JDBC statement in batch for the accumulated records. */ @Internal public interface JdbcBatchStatementExecutor { @@ -44,7 +36,7 @@ public interface JdbcBatchStatementExecutor { */ void open(Connection connection) throws SQLException; - void process(T record) throws SQLException; + void addToBatch(T record) throws SQLException; /** * Submits a batch of commands to the database for execution. @@ -56,45 +48,10 @@ public interface JdbcBatchStatementExecutor { */ void close() throws SQLException; - static JdbcBatchStatementExecutor upsertRow(JdbcDmlOptions opt, RuntimeContext ctx) { - checkNotNull(opt.getKeyFields()); - - int[] pkFields = Arrays.stream(opt.getKeyFields()).mapToInt(Arrays.asList(opt.getFieldNames())::indexOf).toArray(); - int[] pkTypes = opt.getFieldTypes() == null ? null : Arrays.stream(pkFields).map(f -> opt.getFieldTypes()[f]).toArray(); - - return opt.getDialect() - .getUpsertStatement(opt.getTableName(), opt.getFieldNames(), opt.getKeyFields()) - .map(sql -> keyedRow(pkFields, opt.getFieldTypes(), sql)) - .orElseGet(() -> - new InsertOrUpdateJdbcExecutor<>( - opt.getDialect().getRowExistsStatement(opt.getTableName(), opt.getKeyFields()), - opt.getDialect().getInsertIntoStatement(opt.getTableName(), opt.getFieldNames()), - opt.getDialect().getUpdateStatement(opt.getTableName(), opt.getFieldNames(), opt.getKeyFields()), - JdbcStatementBuilder.forRow(pkTypes), - JdbcStatementBuilder.forRow(opt.getFieldTypes()), - JdbcStatementBuilder.forRow(opt.getFieldTypes()), - rowKeyExtractor(pkFields), - ctx.getExecutionConfig().isObjectReuseEnabled() ? Row::copy : Function.identity())); - } - - static Function rowKeyExtractor(int[] pkFields) { - return row -> getPrimaryKey(row, pkFields); - } - - static JdbcBatchStatementExecutor keyedRow(int[] pkFields, int[] pkTypes, String sql) { - return keyed(sql, - rowKeyExtractor(pkFields), - (st, record) -> setRecordToStatement(st, pkTypes, rowKeyExtractor(pkFields).apply(record))); - } - static JdbcBatchStatementExecutor keyed(String sql, Function keyExtractor, JdbcStatementBuilder statementBuilder) { return new KeyedBatchStatementExecutor<>(sql, keyExtractor, statementBuilder); } - static JdbcBatchStatementExecutor simpleRow(String sql, int[] fieldTypes, boolean objectReuse) { - return simple(sql, JdbcStatementBuilder.forRow(fieldTypes), objectReuse ? Row::copy : Function.identity()); - } - static JdbcBatchStatementExecutor simple(String sql, JdbcStatementBuilder paramSetter, Function valueTransformer) { return new SimpleBatchStatementExecutor<>(sql, paramSetter, valueTransformer); } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/KeyedBatchStatementExecutor.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/KeyedBatchStatementExecutor.java index 928f73bd3f..5306410bc0 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/KeyedBatchStatementExecutor.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/KeyedBatchStatementExecutor.java @@ -27,6 +27,9 @@ import java.util.HashSet; import java.util.Set; import java.util.function.Function; +/** + * A {@link JdbcBatchStatementExecutor} that extracts SQL keys from the supplied stream elements and executes a SQL query for them. + */ class KeyedBatchStatementExecutor implements JdbcBatchStatementExecutor { private final String sql; @@ -52,7 +55,7 @@ class KeyedBatchStatementExecutor implements JdbcBatchStatementExecutor } @Override - public void process(T record) { + public void addToBatch(T record) { batch.add(keyExtractor.apply(record)); } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/SimpleBatchStatementExecutor.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/SimpleBatchStatementExecutor.java index f2588ab4eb..c69e22ea36 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/SimpleBatchStatementExecutor.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/executor/SimpleBatchStatementExecutor.java @@ -27,6 +27,9 @@ import java.util.ArrayList; import java.util.List; import java.util.function.Function; +/** + * A {@link JdbcBatchStatementExecutor} that executes supplied statement for given the records (without any pre-processing). + */ class SimpleBatchStatementExecutor implements JdbcBatchStatementExecutor { private final String sql; @@ -49,7 +52,7 @@ class SimpleBatchStatementExecutor implements JdbcBatchStatementExecutor { - ps.setInt(1, t.id); - ps.setString(2, t.title); - ps.setString(3, t.author); - if (t.price == null) { - ps.setNull(4, Types.DOUBLE); - } else { - ps.setDouble(4, t.price); - } - ps.setInt(5, t.qty); - }, - new JdbcConnectionOptions.JdbcConnectionOptionsBuilder() - .withUrl(getDbMetadata().getUrl()) - .withDriverName(getDbMetadata().getDriverClass()) - .build())); + .fromElements(TEST_DATA) + .addSink(JdbcSink.sink( + String.format(INSERT_TEMPLATE, INPUT_TABLE), + (ps, t) -> { + ps.setInt(1, t.id); + ps.setString(2, t.title); + ps.setString(3, t.author); + if (t.price == null) { + ps.setNull(4, Types.DOUBLE); + } else { + ps.setDouble(4, t.price); + } + ps.setInt(5, t.qty); + }, + new JdbcConnectionOptionsBuilder() + .withUrl(getDbMetadata().getUrl()) + .withDriverName(getDbMetadata().getDriverClass()) + .build())); env.execute(); assertEquals(Arrays.asList(TEST_DATA), selectBooks()); diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTestFixture.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTestFixture.java index a822d31f09..8dd5402153 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTestFixture.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTestFixture.java @@ -33,8 +33,6 @@ import java.sql.Statement; */ @SuppressWarnings("SpellCheckingInspection") public class JdbcTestFixture { - public static final JDBCTestCheckpoint CP0 = new JDBCTestCheckpoint(0, 1, 2, 3); - public static final JDBCTestCheckpoint CP1 = new JDBCTestCheckpoint(1, 4, 5, 6); public static final String INPUT_TABLE = "books"; static final String OUTPUT_TABLE = "newbooks"; -- Gitee From bd18b8715466bfe49d8b31019075f8081a618875 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Wed, 11 Mar 2020 09:46:38 +0800 Subject: [PATCH 120/885] [FLINK-16455][hive] Introduce flink-sql-connector-hive modules to provide hive uber jars This closes #11328 --- docs/dev/table/hive/index.md | 80 ++++------ docs/dev/table/hive/index.zh.md | 79 ++++------ .../flink-sql-connector-hive-1.2.2/pom.xml | 142 ++++++++++++++++++ .../src/main/resources/META-INF/NOTICE | 13 ++ .../flink-sql-connector-hive-2.2.0/pom.xml | 115 ++++++++++++++ .../src/main/resources/META-INF/NOTICE | 11 ++ .../flink-sql-connector-hive-2.3.6/pom.xml | 89 +++++++++++ .../src/main/resources/META-INF/NOTICE | 9 ++ .../flink-sql-connector-hive-3.1.2/pom.xml | 102 +++++++++++++ .../src/main/resources/META-INF/NOTICE | 10 ++ flink-connectors/pom.xml | 4 + 11 files changed, 557 insertions(+), 97 deletions(-) create mode 100644 flink-connectors/flink-sql-connector-hive-1.2.2/pom.xml create mode 100644 flink-connectors/flink-sql-connector-hive-1.2.2/src/main/resources/META-INF/NOTICE create mode 100644 flink-connectors/flink-sql-connector-hive-2.2.0/pom.xml create mode 100644 flink-connectors/flink-sql-connector-hive-2.2.0/src/main/resources/META-INF/NOTICE create mode 100644 flink-connectors/flink-sql-connector-hive-2.3.6/pom.xml create mode 100644 flink-connectors/flink-sql-connector-hive-2.3.6/src/main/resources/META-INF/NOTICE create mode 100644 flink-connectors/flink-sql-connector-hive-3.1.2/pom.xml create mode 100644 flink-connectors/flink-sql-connector-hive-3.1.2/src/main/resources/META-INF/NOTICE diff --git a/docs/dev/table/hive/index.md b/docs/dev/table/hive/index.md index f1e8eed108..3698df2571 100644 --- a/docs/dev/table/hive/index.md +++ b/docs/dev/table/hive/index.md @@ -92,6 +92,32 @@ to make the integration work in Table API program or SQL in SQL Client. Alternatively, you can put these dependencies in a dedicated folder, and add them to classpath with the `-C` or `-l` option for Table API program or SQL Client respectively. +Apache Hive is built on Hadoop, so you need Hadoop dependency first, please refer to +[Providing Hadoop classes]({{ site.baseurl }}/ops/deployment/hadoop.html#providing-hadoop-classes). + +There are two ways to add Hive dependencies. First is to use Flink's bundled Hive jars. You can choose a bundled Hive jar according to the version of the metastore you use. Second is to add each of the required jars separately. The second way can be useful if the Hive version you're using is not listed here. + +#### Using bundled hive jar + +The following tables list all available bundled hive jars. You can pick one to the `/lib/` directory in Flink distribution. + +{% if site.is_stable %} + +| Metastore version | Maven dependency | SQL Client JAR | +| :---------------- | :--------------------------- | :----------------------| +| 1.0.0 - 1.2.2 | `flink-connector-hive-1.2.2` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-sql-connector-hive-1.2.2{{site.scala_version_suffix}}/{{site.version}}/flink-sql-connector-hive-1.2.2{{site.scala_version_suffix}}-{{site.version}}.jar) | +| 2.0.0 - 2.2.0 | `flink-connector-hive-2.2.0` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-sql-connector-hive-2.2.0{{site.scala_version_suffix}}/{{site.version}}/flink-sql-connector-hive-2.2.0{{site.scala_version_suffix}}-{{site.version}}.jar) | +| 2.3.0 - 2.3.6 | `flink-connector-hive-2.3.6` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-sql-connector-hive-2.3.6{{site.scala_version_suffix}}/{{site.version}}/flink-sql-connector-hive-2.3.6{{site.scala_version_suffix}}-{{site.version}}.jar) | +| 3.0.0 - 3.1.2 | `flink-connector-hive-3.1.2` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-sql-connector-hive-3.1.2{{site.scala_version_suffix}}/{{site.version}}/flink-sql-connector-hive-3.1.2{{site.scala_version_suffix}}-{{site.version}}.jar) | + +{% else %} + +These tables are only available for stable releases. + +{% endif %} + +#### User defined dependencies + Please find the required dependencies for different Hive major versions below. @@ -105,12 +131,6 @@ Please find the required dependencies for different Hive major versions below. // Flink's Hive connector.Contains flink-hadoop-compatibility and flink-orc jars flink-connector-hive{{ site.scala_version_suffix }}-{{ site.version }}.jar - // Hadoop dependencies - // You can pick a pre-built Hadoop uber jar provided by Flink, alternatively - // you can use your own hadoop jars. Either way, make sure it's compatible with your Hadoop - // cluster and the Hive version you're using. - flink-shaded-hadoop-2-uber-2.7.5-{{ site.shaded_version }}.jar - // Hive dependencies hive-exec-2.3.4.jar @@ -125,12 +145,6 @@ Please find the required dependencies for different Hive major versions below. // Flink's Hive connector flink-connector-hive{{ site.scala_version_suffix }}-{{ site.version }}.jar - // Hadoop dependencies - // You can pick a pre-built Hadoop uber jar provided by Flink, alternatively - // you can use your own hadoop jars. Either way, make sure it's compatible with your Hadoop - // cluster and the Hive version you're using. - flink-shaded-hadoop-2-uber-2.6.5-{{ site.shaded_version }}.jar - // Hive dependencies hive-metastore-1.0.0.jar hive-exec-1.0.0.jar @@ -151,12 +165,6 @@ Please find the required dependencies for different Hive major versions below. // Flink's Hive connector flink-connector-hive{{ site.scala_version_suffix }}-{{ site.version }}.jar - // Hadoop dependencies - // You can pick a pre-built Hadoop uber jar provided by Flink, alternatively - // you can use your own hadoop jars. Either way, make sure it's compatible with your Hadoop - // cluster and the Hive version you're using. - flink-shaded-hadoop-2-uber-2.6.5-{{ site.shaded_version }}.jar - // Hive dependencies hive-metastore-1.1.0.jar hive-exec-1.1.0.jar @@ -177,12 +185,6 @@ Please find the required dependencies for different Hive major versions below. // Flink's Hive connector flink-connector-hive{{ site.scala_version_suffix }}-{{ site.version }}.jar - // Hadoop dependencies - // You can pick a pre-built Hadoop uber jar provided by Flink, alternatively - // you can use your own hadoop jars. Either way, make sure it's compatible with your Hadoop - // cluster and the Hive version you're using. - flink-shaded-hadoop-2-uber-2.6.5-{{ site.shaded_version }}.jar - // Hive dependencies hive-metastore-1.2.1.jar hive-exec-1.2.1.jar @@ -203,12 +205,6 @@ Please find the required dependencies for different Hive major versions below. // Flink's Hive connector flink-connector-hive{{ site.scala_version_suffix }}-{{ site.version }}.jar - // Hadoop dependencies - // You can pick a pre-built Hadoop uber jar provided by Flink, alternatively - // you can use your own hadoop jars. Either way, make sure it's compatible with your Hadoop - // cluster and the Hive version you're using. - flink-shaded-hadoop-2-uber-2.7.5-{{ site.shaded_version }}.jar - // Hive dependencies hive-exec-2.0.0.jar @@ -223,12 +219,6 @@ Please find the required dependencies for different Hive major versions below. // Flink's Hive connector flink-connector-hive{{ site.scala_version_suffix }}-{{ site.version }}.jar - // Hadoop dependencies - // You can pick a pre-built Hadoop uber jar provided by Flink, alternatively - // you can use your own hadoop jars. Either way, make sure it's compatible with your Hadoop - // cluster and the Hive version you're using. - flink-shaded-hadoop-2-uber-2.7.5-{{ site.shaded_version }}.jar - // Hive dependencies hive-exec-2.1.0.jar @@ -243,12 +233,6 @@ Please find the required dependencies for different Hive major versions below. // Flink's Hive connector flink-connector-hive{{ site.scala_version_suffix }}-{{ site.version }}.jar - // Hadoop dependencies - // You can pick a pre-built Hadoop uber jar provided by Flink, alternatively - // you can use your own hadoop jars. Either way, make sure it's compatible with your Hadoop - // cluster and the Hive version you're using. - flink-shaded-hadoop-2-uber-2.7.5-{{ site.shaded_version }}.jar - // Hive dependencies hive-exec-2.2.0.jar @@ -267,12 +251,6 @@ Please find the required dependencies for different Hive major versions below. // Flink's Hive connector flink-connector-hive{{ site.scala_version_suffix }}-{{ site.version }}.jar - // Hadoop dependencies - // You can pick a pre-built Hadoop uber jar provided by Flink, alternatively - // you can use your own hadoop jars. Either way, make sure it's compatible with your Hadoop - // cluster and the Hive version you're using. - flink-shaded-hadoop-2-uber-2.8.3-{{ site.shaded_version }}.jar - // Hive dependencies hive-exec-3.1.0.jar libfb303-0.9.3.jar // libfb303 is not packed into hive-exec in some versions, need to add it separately @@ -281,6 +259,11 @@ Please find the required dependencies for different Hive major versions below. +If you use the hive version of HDP or CDH, you need to refer to the dependency in the previous section and select a similar version. + +And you need to specify selected and supported "hive-version" in yaml, HiveCatalog and HiveModule. + +### Program maven If you are building your own program, you need the following dependencies in your mvn file. It's recommended not to include these dependencies in the resulting jar file. @@ -386,4 +369,3 @@ DDL to create Hive tables, views, partitions, functions within Flink will be sup ## DML Flink supports DML writing to Hive tables. Please refer to details in [Reading & Writing Hive Tables]({{ site.baseurl }}/dev/table/hive/read_write_hive.html) - diff --git a/docs/dev/table/hive/index.zh.md b/docs/dev/table/hive/index.zh.md index 2da1559a23..c1cb9d6e02 100644 --- a/docs/dev/table/hive/index.zh.md +++ b/docs/dev/table/hive/index.zh.md @@ -92,6 +92,32 @@ to make the integration work in Table API program or SQL in SQL Client. Alternatively, you can put these dependencies in a dedicated folder, and add them to classpath with the `-C` or `-l` option for Table API program or SQL Client respectively. +Apache Hive is built on Hadoop, so you need Hadoop dependency first, please refer to +[Providing Hadoop classes]({{ site.baseurl }}/ops/deployment/hadoop.html#providing-hadoop-classes). + +There are two ways to add Hive dependencies. First is to use Flink's bundled Hive jars. You can choose a bundled Hive jar according to the version of the metastore you use. Second is to add each of the required jars separately. The second way can be useful if the Hive version you're using is not listed here. + +#### Using bundled hive jar + +The following tables list all available bundled hive jars. You can pick one to the `/lib/` directory in Flink distribution. + +{% if site.is_stable %} + +| Metastore version | Maven dependency | SQL Client JAR | +| :---------------- | :--------------------------- | :----------------------| +| 1.0.0 - 1.2.2 | `flink-connector-hive-1.2.2` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-sql-connector-hive-1.2.2{{site.scala_version_suffix}}/{{site.version}}/flink-sql-connector-hive-1.2.2{{site.scala_version_suffix}}-{{site.version}}.jar) | +| 2.0.0 - 2.2.0 | `flink-connector-hive-2.2.0` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-sql-connector-hive-2.2.0{{site.scala_version_suffix}}/{{site.version}}/flink-sql-connector-hive-2.2.0{{site.scala_version_suffix}}-{{site.version}}.jar) | +| 2.3.0 - 2.3.6 | `flink-connector-hive-2.3.6` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-sql-connector-hive-2.3.6{{site.scala_version_suffix}}/{{site.version}}/flink-sql-connector-hive-2.3.6{{site.scala_version_suffix}}-{{site.version}}.jar) | +| 3.0.0 - 3.1.2 | `flink-connector-hive-3.1.2` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-sql-connector-hive-3.1.2{{site.scala_version_suffix}}/{{site.version}}/flink-sql-connector-hive-3.1.2{{site.scala_version_suffix}}-{{site.version}}.jar) | + +{% else %} + +These tables are only available for stable releases. + +{% endif %} + +#### User defined dependencies + Please find the required dependencies for different Hive major versions below. @@ -105,12 +131,6 @@ Please find the required dependencies for different Hive major versions below. // Flink's Hive connector.Contains flink-hadoop-compatibility and flink-orc jars flink-connector-hive{{ site.scala_version_suffix }}-{{ site.version }}.jar - // Hadoop dependencies - // You can pick a pre-built Hadoop uber jar provided by Flink, alternatively - // you can use your own hadoop jars. Either way, make sure it's compatible with your Hadoop - // cluster and the Hive version you're using. - flink-shaded-hadoop-2-uber-2.7.5-{{ site.shaded_version }}.jar - // Hive dependencies hive-exec-2.3.4.jar @@ -125,12 +145,6 @@ Please find the required dependencies for different Hive major versions below. // Flink's Hive connector flink-connector-hive{{ site.scala_version_suffix }}-{{ site.version }}.jar - // Hadoop dependencies - // You can pick a pre-built Hadoop uber jar provided by Flink, alternatively - // you can use your own hadoop jars. Either way, make sure it's compatible with your Hadoop - // cluster and the Hive version you're using. - flink-shaded-hadoop-2-uber-2.6.5-{{ site.shaded_version }}.jar - // Hive dependencies hive-metastore-1.0.0.jar hive-exec-1.0.0.jar @@ -151,12 +165,6 @@ Please find the required dependencies for different Hive major versions below. // Flink's Hive connector flink-connector-hive{{ site.scala_version_suffix }}-{{ site.version }}.jar - // Hadoop dependencies - // You can pick a pre-built Hadoop uber jar provided by Flink, alternatively - // you can use your own hadoop jars. Either way, make sure it's compatible with your Hadoop - // cluster and the Hive version you're using. - flink-shaded-hadoop-2-uber-2.6.5-{{ site.shaded_version }}.jar - // Hive dependencies hive-metastore-1.1.0.jar hive-exec-1.1.0.jar @@ -177,12 +185,6 @@ Please find the required dependencies for different Hive major versions below. // Flink's Hive connector flink-connector-hive{{ site.scala_version_suffix }}-{{ site.version }}.jar - // Hadoop dependencies - // You can pick a pre-built Hadoop uber jar provided by Flink, alternatively - // you can use your own hadoop jars. Either way, make sure it's compatible with your Hadoop - // cluster and the Hive version you're using. - flink-shaded-hadoop-2-uber-2.6.5-{{ site.shaded_version }}.jar - // Hive dependencies hive-metastore-1.2.1.jar hive-exec-1.2.1.jar @@ -203,12 +205,6 @@ Please find the required dependencies for different Hive major versions below. // Flink's Hive connector flink-connector-hive{{ site.scala_version_suffix }}-{{ site.version }}.jar - // Hadoop dependencies - // You can pick a pre-built Hadoop uber jar provided by Flink, alternatively - // you can use your own hadoop jars. Either way, make sure it's compatible with your Hadoop - // cluster and the Hive version you're using. - flink-shaded-hadoop-2-uber-2.7.5-{{ site.shaded_version }}.jar - // Hive dependencies hive-exec-2.0.0.jar @@ -223,12 +219,6 @@ Please find the required dependencies for different Hive major versions below. // Flink's Hive connector flink-connector-hive{{ site.scala_version_suffix }}-{{ site.version }}.jar - // Hadoop dependencies - // You can pick a pre-built Hadoop uber jar provided by Flink, alternatively - // you can use your own hadoop jars. Either way, make sure it's compatible with your Hadoop - // cluster and the Hive version you're using. - flink-shaded-hadoop-2-uber-2.7.5-{{ site.shaded_version }}.jar - // Hive dependencies hive-exec-2.1.0.jar @@ -243,12 +233,6 @@ Please find the required dependencies for different Hive major versions below. // Flink's Hive connector flink-connector-hive{{ site.scala_version_suffix }}-{{ site.version }}.jar - // Hadoop dependencies - // You can pick a pre-built Hadoop uber jar provided by Flink, alternatively - // you can use your own hadoop jars. Either way, make sure it's compatible with your Hadoop - // cluster and the Hive version you're using. - flink-shaded-hadoop-2-uber-2.7.5-{{ site.shaded_version }}.jar - // Hive dependencies hive-exec-2.2.0.jar @@ -267,12 +251,6 @@ Please find the required dependencies for different Hive major versions below. // Flink's Hive connector flink-connector-hive{{ site.scala_version_suffix }}-{{ site.version }}.jar - // Hadoop dependencies - // You can pick a pre-built Hadoop uber jar provided by Flink, alternatively - // you can use your own hadoop jars. Either way, make sure it's compatible with your Hadoop - // cluster and the Hive version you're using. - flink-shaded-hadoop-2-uber-2.8.3-{{ site.shaded_version }}.jar - // Hive dependencies hive-exec-3.1.0.jar libfb303-0.9.3.jar // libfb303 is not packed into hive-exec in some versions, need to add it separately @@ -281,6 +259,11 @@ Please find the required dependencies for different Hive major versions below. +If you use the hive version of HDP or CDH, you need to refer to the dependency in the previous section and select a similar version. + +And you need to specify selected and supported "hive-version" in yaml, HiveCatalog and HiveModule. + +#### Program maven If you are building your own program, you need the following dependencies in your mvn file. It's recommended not to include these dependencies in the resulting jar file. diff --git a/flink-connectors/flink-sql-connector-hive-1.2.2/pom.xml b/flink-connectors/flink-sql-connector-hive-1.2.2/pom.xml new file mode 100644 index 0000000000..66dec680b8 --- /dev/null +++ b/flink-connectors/flink-sql-connector-hive-1.2.2/pom.xml @@ -0,0 +1,142 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-connectors + 1.11-SNAPSHOT + .. + + + flink-sql-connector-hive-1.2.2_${scala.binary.version} + flink-sql-connector-hive-1.2.2 + + jar + + + + org.apache.flink + flink-connector-hive_${scala.binary.version} + ${project.version} + + + * + * + + + + + + org.apache.hive + hive-exec + 1.2.2 + + + * + * + + + + + + org.apache.hive + hive-metastore + 1.2.2 + + + * + * + + + + + + org.apache.thrift + libfb303 + 0.9.2 + + + * + * + + + + + + org.apache.orc + orc-core + 1.4.3 + nohive + + + * + * + + + + + + io.airlift + aircompressor + 0.8 + + + * + * + + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + + + org.apache.flink:flink-connector-hive_${scala.binary.version} + org.apache.hive:hive-exec + org.apache.hive:hive-metastore + org.apache.thrift:libfb303 + org.apache.orc:orc-core + io.airlift:aircompressor + + + + + + + + + diff --git a/flink-connectors/flink-sql-connector-hive-1.2.2/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-hive-1.2.2/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000..ae2657902d --- /dev/null +++ b/flink-connectors/flink-sql-connector-hive-1.2.2/src/main/resources/META-INF/NOTICE @@ -0,0 +1,13 @@ +flink-sql-connector-hive-1.2.2 +Copyright 2014-2019 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- org.apache.hive:hive-exec:1.2.2 +- org.apache.hive:hive-metastore:1.2.2 +- org.apache.thrift:libfb303:0.9.2 +- org.apache.orc:orc-core:1.4.3 +- io.airlift:aircompressor:0.8 diff --git a/flink-connectors/flink-sql-connector-hive-2.2.0/pom.xml b/flink-connectors/flink-sql-connector-hive-2.2.0/pom.xml new file mode 100644 index 0000000000..d3ef7c57b6 --- /dev/null +++ b/flink-connectors/flink-sql-connector-hive-2.2.0/pom.xml @@ -0,0 +1,115 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-connectors + 1.11-SNAPSHOT + .. + + + flink-sql-connector-hive-2.2.0_${scala.binary.version} + flink-sql-connector-hive-2.2.0 + + jar + + + + org.apache.flink + flink-connector-hive_${scala.binary.version} + ${project.version} + + + * + * + + + + + + org.apache.hive + hive-exec + 2.2.0 + + + * + * + + + + + + org.apache.orc + orc-core + 1.4.3 + + + * + * + + + + + + io.airlift + aircompressor + 0.8 + + + * + * + + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + + + org.apache.flink:flink-connector-hive_${scala.binary.version} + org.apache.hive:hive-exec + org.apache.orc:orc-core + io.airlift:aircompressor + + + + + + + + + diff --git a/flink-connectors/flink-sql-connector-hive-2.2.0/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-hive-2.2.0/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000..d7664ccaf1 --- /dev/null +++ b/flink-connectors/flink-sql-connector-hive-2.2.0/src/main/resources/META-INF/NOTICE @@ -0,0 +1,11 @@ +flink-sql-connector-hive-2.2.0 +Copyright 2014-2019 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- org.apache.hive:hive-exec:2.2.0 +- org.apache.orc:orc-core:1.4.3 +- io.airlift:aircompressor:0.8 diff --git a/flink-connectors/flink-sql-connector-hive-2.3.6/pom.xml b/flink-connectors/flink-sql-connector-hive-2.3.6/pom.xml new file mode 100644 index 0000000000..81c562ebb8 --- /dev/null +++ b/flink-connectors/flink-sql-connector-hive-2.3.6/pom.xml @@ -0,0 +1,89 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-connectors + 1.11-SNAPSHOT + .. + + + flink-sql-connector-hive-2.3.6_${scala.binary.version} + flink-sql-connector-hive-2.3.6 + + jar + + + + org.apache.flink + flink-connector-hive_${scala.binary.version} + ${project.version} + + + * + * + + + + + + org.apache.hive + hive-exec + 2.3.6 + + + * + * + + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + + + org.apache.flink:flink-connector-hive_${scala.binary.version} + org.apache.hive:hive-exec + + + + + + + + + diff --git a/flink-connectors/flink-sql-connector-hive-2.3.6/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-hive-2.3.6/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000..a2b038f00b --- /dev/null +++ b/flink-connectors/flink-sql-connector-hive-2.3.6/src/main/resources/META-INF/NOTICE @@ -0,0 +1,9 @@ +flink-sql-connector-hive-2.3.6 +Copyright 2014-2019 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- org.apache.hive:hive-exec:2.3.6 diff --git a/flink-connectors/flink-sql-connector-hive-3.1.2/pom.xml b/flink-connectors/flink-sql-connector-hive-3.1.2/pom.xml new file mode 100644 index 0000000000..8225a4a5cb --- /dev/null +++ b/flink-connectors/flink-sql-connector-hive-3.1.2/pom.xml @@ -0,0 +1,102 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-connectors + 1.11-SNAPSHOT + .. + + + flink-sql-connector-hive-3.1.2_${scala.binary.version} + flink-sql-connector-hive-3.1.2 + + jar + + + + org.apache.flink + flink-connector-hive_${scala.binary.version} + ${project.version} + + + * + * + + + + + + org.apache.hive + hive-exec + 3.1.2 + + + * + * + + + + + + org.apache.thrift + libfb303 + 0.9.3 + + + * + * + + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + + + org.apache.flink:flink-connector-hive_${scala.binary.version} + org.apache.hive:hive-exec + org.apache.thrift:libfb303 + + + + + + + + + diff --git a/flink-connectors/flink-sql-connector-hive-3.1.2/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-hive-3.1.2/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000..71df3aae2b --- /dev/null +++ b/flink-connectors/flink-sql-connector-hive-3.1.2/src/main/resources/META-INF/NOTICE @@ -0,0 +1,10 @@ +flink-sql-connector-hive-3.1.2 +Copyright 2014-2019 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- org.apache.hive:hive-exec:3.1.2 +- org.apache.thrift:libfb303:0.9.3 diff --git a/flink-connectors/pom.xml b/flink-connectors/pom.xml index bad4cf4da6..1e92a10541 100644 --- a/flink-connectors/pom.xml +++ b/flink-connectors/pom.xml @@ -97,6 +97,10 @@ under the License. flink-sql-connector-kafka-0.10 flink-sql-connector-kafka-0.11 flink-sql-connector-kafka + flink-sql-connector-hive-1.2.2 + flink-sql-connector-hive-2.2.0 + flink-sql-connector-hive-2.3.6 + flink-sql-connector-hive-3.1.2 -- Gitee From 87ebe5533d3cd675b416146189abb0f6de61559d Mon Sep 17 00:00:00 2001 From: zoudan Date: Tue, 18 Feb 2020 16:32:28 +0800 Subject: [PATCH 121/885] [FLINK-15396][json] Support to ignore parse errors for JSON format This closes #11119 --- docs/dev/table/connect.md | 8 +- docs/dev/table/connect.zh.md | 7 +- .../json/JsonRowDeserializationSchema.java | 134 +++++++++++-- .../formats/json/JsonRowFormatFactory.java | 7 +- .../apache/flink/table/descriptors/Json.java | 17 ++ .../table/descriptors/JsonValidator.java | 7 + .../JsonRowDeserializationSchemaTest.java | 177 ++++++++++++++++++ .../flink/table/descriptors/JsonTest.java | 24 ++- 8 files changed, 353 insertions(+), 28 deletions(-) diff --git a/docs/dev/table/connect.md b/docs/dev/table/connect.md index d8c3274e07..204c4cb882 100644 --- a/docs/dev/table/connect.md +++ b/docs/dev/table/connect.md @@ -1586,9 +1586,10 @@ CREATE TABLE MyUserTable ( ... ) WITH ( 'format.type' = 'json', -- required: specify the format type - 'format.fail-on-missing-field' = 'true' -- optional: flag whether to fail if a field is missing or not, + 'format.fail-on-missing-field' = 'true', -- optional: flag whether to fail if a field is missing or not, -- 'false' by default - + 'format.ignore-parse-errors' = 'true', -- optional: skip fields and rows with parse errors instead of failing; + -- fields are set to null in case of errors -- deprecated: define the schema explicitly using JSON schema which parses to DECIMAL and TIMESTAMP. 'format.json-schema' = '{ @@ -1612,7 +1613,8 @@ CREATE TABLE MyUserTable ( .withFormat( new Json() .failOnMissingField(true) // optional: flag whether to fail if a field is missing or not, false by default - + .ignoreParseErrors(true) // optional: skip fields and rows with parse errors instead of failing; + // fields are set to null in case of errors // deprecated: define the schema explicitly using JSON schema which parses to DECIMAL and TIMESTAMP. .jsonSchema( "{" + diff --git a/docs/dev/table/connect.zh.md b/docs/dev/table/connect.zh.md index 504bb01b30..66b8d9a74f 100644 --- a/docs/dev/table/connect.zh.md +++ b/docs/dev/table/connect.zh.md @@ -1586,8 +1586,10 @@ CREATE TABLE MyUserTable ( ... ) WITH ( 'format.type' = 'json', -- required: specify the format type - 'format.fail-on-missing-field' = 'true' -- optional: flag whether to fail if a field is missing or not, + 'format.fail-on-missing-field' = 'true', -- optional: flag whether to fail if a field is missing or not, -- 'false' by default + 'format.ignore-parse-errors' = 'true' -- optional: skip fields and rows with parse errors instead of failing; + -- fields are set to null in case of errors -- deprecated: define the schema explicitly using JSON schema which parses to DECIMAL and TIMESTAMP. 'format.json-schema' = @@ -1612,7 +1614,8 @@ CREATE TABLE MyUserTable ( .withFormat( new Json() .failOnMissingField(true) // optional: flag whether to fail if a field is missing or not, false by default - + .ignoreParseErrors(true) // optional: skip fields and rows with parse errors instead of failing; + // fields are set to null in case of errors // deprecated: define the schema explicitly using JSON schema which parses to DECIMAL and TIMESTAMP. .jsonSchema( "{" + diff --git a/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java b/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java index 4cdd53807b..cc9b55d589 100644 --- a/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java +++ b/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java @@ -28,7 +28,6 @@ import org.apache.flink.api.java.typeutils.MapTypeInfo; import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.types.Row; -import org.apache.flink.util.WrappingRuntimeException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; @@ -40,6 +39,8 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.Text import java.io.IOException; import java.io.Serializable; import java.lang.reflect.Array; +import java.math.BigDecimal; +import java.math.BigInteger; import java.sql.Date; import java.sql.Time; import java.sql.Timestamp; @@ -88,14 +89,23 @@ public class JsonRowDeserializationSchema implements DeserializationSchema private DeserializationRuntimeConverter runtimeConverter; + /** Flag indicating whether to ignore invalid fields/rows (default: throw an exception). */ + private final boolean ignoreParseErrors; + private JsonRowDeserializationSchema( TypeInformation typeInfo, - boolean failOnMissingField) { + boolean failOnMissingField, + boolean ignoreParseErrors) { checkNotNull(typeInfo, "Type information"); checkArgument(typeInfo instanceof RowTypeInfo, "Only RowTypeInfo is supported"); + if (ignoreParseErrors && failOnMissingField) { + throw new IllegalArgumentException( + "JSON format doesn't support failOnMissingField and ignoreParseErrors are both true."); + } this.typeInfo = (RowTypeInfo) typeInfo; this.failOnMissingField = failOnMissingField; this.runtimeConverter = createConverter(this.typeInfo); + this.ignoreParseErrors = ignoreParseErrors; } /** @@ -103,7 +113,7 @@ public class JsonRowDeserializationSchema implements DeserializationSchema */ @Deprecated public JsonRowDeserializationSchema(TypeInformation typeInfo) { - this(typeInfo, false); + this(typeInfo, false, false); } /** @@ -111,7 +121,7 @@ public class JsonRowDeserializationSchema implements DeserializationSchema */ @Deprecated public JsonRowDeserializationSchema(String jsonSchema) { - this(JsonRowSchemaConverter.convert(checkNotNull(jsonSchema)), false); + this(JsonRowSchemaConverter.convert(checkNotNull(jsonSchema)), false, false); } /** @@ -130,7 +140,10 @@ public class JsonRowDeserializationSchema implements DeserializationSchema final JsonNode root = objectMapper.readTree(message); return (Row) runtimeConverter.convert(objectMapper, root); } catch (Throwable t) { - throw new IOException("Failed to deserialize JSON object.", t); + if (ignoreParseErrors) { + return null; + } + throw new IOException(format("Failed to deserialize JSON '%s'.", new String(message)), t); } } @@ -151,6 +164,7 @@ public class JsonRowDeserializationSchema implements DeserializationSchema private final RowTypeInfo typeInfo; private boolean failOnMissingField = false; + private boolean ignoreParseErrors = false; /** * Creates a JSON deserialization schema for the given type information. @@ -184,8 +198,18 @@ public class JsonRowDeserializationSchema implements DeserializationSchema return this; } + /** + * Configures schema to fail when parsing json failed. + * + *

    By default, an exception will be thrown when parsing json fails. + */ + public Builder ignoreParseErrors() { + this.ignoreParseErrors = true; + return this; + } + public JsonRowDeserializationSchema build() { - return new JsonRowDeserializationSchema(typeInfo, failOnMissingField); + return new JsonRowDeserializationSchema(typeInfo, failOnMissingField, ignoreParseErrors); } } @@ -199,12 +223,13 @@ public class JsonRowDeserializationSchema implements DeserializationSchema } final JsonRowDeserializationSchema that = (JsonRowDeserializationSchema) o; return Objects.equals(typeInfo, that.typeInfo) && - Objects.equals(failOnMissingField, that.failOnMissingField); + Objects.equals(failOnMissingField, that.failOnMissingField) && + Objects.equals(ignoreParseErrors, that.ignoreParseErrors); } @Override public int hashCode() { - return Objects.hash(typeInfo, failOnMissingField); + return Objects.hash(typeInfo, failOnMissingField, ignoreParseErrors); } /* @@ -232,8 +257,14 @@ public class JsonRowDeserializationSchema implements DeserializationSchema if (jsonNode.isNull()) { return null; } - - return converter.convert(mapper, jsonNode); + try { + return converter.convert(mapper, jsonNode); + } catch (Throwable t) { + if (!ignoreParseErrors) { + throw t; + } + return null; + } }; } @@ -257,7 +288,6 @@ public class JsonRowDeserializationSchema implements DeserializationSchema private DeserializationRuntimeConverter createMapConverter(TypeInformation keyType, TypeInformation valueType) { DeserializationRuntimeConverter valueConverter = createConverter(valueType); DeserializationRuntimeConverter keyConverter = createConverter(keyType); - return (mapper, jsonNode) -> { Iterator> fields = jsonNode.fields(); Map result = new HashMap<>(); @@ -276,7 +306,7 @@ public class JsonRowDeserializationSchema implements DeserializationSchema try { return jsonNode.binaryValue(); } catch (IOException e) { - throw new WrappingRuntimeException("Unable to deserialize byte array.", e); + throw new JsonParseException("Unable to deserialize byte array.", e); } }; } @@ -306,7 +336,7 @@ public class JsonRowDeserializationSchema implements DeserializationSchema try { return mapper.treeToValue(jsonNode, valueType); } catch (JsonProcessingException e) { - throw new WrappingRuntimeException(format("Could not convert node: %s", jsonNode), e); + throw new JsonParseException(format("Could not convert node: %s", jsonNode), e); } }; } @@ -315,15 +345,15 @@ public class JsonRowDeserializationSchema implements DeserializationSchema if (simpleTypeInfo == Types.VOID) { return Optional.of((mapper, jsonNode) -> null); } else if (simpleTypeInfo == Types.BOOLEAN) { - return Optional.of((mapper, jsonNode) -> jsonNode.asBoolean()); + return Optional.of(this::convertToBoolean); } else if (simpleTypeInfo == Types.STRING) { return Optional.of((mapper, jsonNode) -> jsonNode.asText()); } else if (simpleTypeInfo == Types.INT) { - return Optional.of((mapper, jsonNode) -> jsonNode.asInt()); + return Optional.of(this::convertToInt); } else if (simpleTypeInfo == Types.LONG) { - return Optional.of((mapper, jsonNode) -> jsonNode.asLong()); + return Optional.of(this::convertToLong); } else if (simpleTypeInfo == Types.DOUBLE) { - return Optional.of((mapper, jsonNode) -> jsonNode.asDouble()); + return Optional.of(this::convertToDouble); } else if (simpleTypeInfo == Types.FLOAT) { return Optional.of((mapper, jsonNode) -> Float.parseFloat(jsonNode.asText().trim())); } else if (simpleTypeInfo == Types.SHORT) { @@ -331,9 +361,9 @@ public class JsonRowDeserializationSchema implements DeserializationSchema } else if (simpleTypeInfo == Types.BYTE) { return Optional.of((mapper, jsonNode) -> Byte.parseByte(jsonNode.asText().trim())); } else if (simpleTypeInfo == Types.BIG_DEC) { - return Optional.of((mapper, jsonNode) -> jsonNode.decimalValue()); + return Optional.of(this::convertToBigDecimal); } else if (simpleTypeInfo == Types.BIG_INT) { - return Optional.of((mapper, jsonNode) -> jsonNode.bigIntegerValue()); + return Optional.of(this::convertToBigInteger); } else if (simpleTypeInfo == Types.SQL_DATE) { return Optional.of(this::convertToDate); } else if (simpleTypeInfo == Types.SQL_TIME) { @@ -351,6 +381,60 @@ public class JsonRowDeserializationSchema implements DeserializationSchema } } + private boolean convertToBoolean(ObjectMapper mapper, JsonNode jsonNode) { + if (jsonNode.isBoolean()) { + // avoid redundant toString and parseBoolean, for better performance + return jsonNode.asBoolean(); + } else { + return Boolean.parseBoolean(jsonNode.asText().trim()); + } + } + + private int convertToInt(ObjectMapper mapper, JsonNode jsonNode) { + if (jsonNode.canConvertToInt()) { + // avoid redundant toString and parseInt, for better performance + return jsonNode.asInt(); + } else { + return Integer.parseInt(jsonNode.asText().trim()); + } + } + + private long convertToLong(ObjectMapper mapper, JsonNode jsonNode) { + if (jsonNode.canConvertToLong()) { + // avoid redundant toString and parseLong, for better performance + return jsonNode.asLong(); + } else { + return Long.parseLong(jsonNode.asText().trim()); + } + } + + private double convertToDouble(ObjectMapper mapper, JsonNode jsonNode) { + if (jsonNode.isDouble()) { + // avoid redundant toString and parseDouble, for better performance + return jsonNode.asDouble(); + } else { + return Double.parseDouble(jsonNode.asText().trim()); + } + } + + private BigDecimal convertToBigDecimal(ObjectMapper mapper, JsonNode jsonNode) { + if (jsonNode.isBigDecimal()) { + // avoid redundant toString and toDecimal, for better performance + return jsonNode.decimalValue(); + } else { + return new BigDecimal(jsonNode.asText().trim()); + } + } + + private BigInteger convertToBigInteger(ObjectMapper mapper, JsonNode jsonNode) { + if (jsonNode.isBigInteger()) { + // avoid redundant toString and toBigInteger, for better performance + return jsonNode.bigIntegerValue(); + } else { + return new BigInteger(jsonNode.asText().trim()); + } + } + private LocalDate convertToLocalDate(ObjectMapper mapper, JsonNode jsonNode) { return ISO_LOCAL_DATE.parse(jsonNode.asText()).query(TemporalQueries.localDate()); } @@ -387,6 +471,7 @@ public class JsonRowDeserializationSchema implements DeserializationSchema // according to RFC 3339 every full-time must have a timezone; // until we have full timezone support, we only support UTC; // users can parse their time as string as a workaround + TemporalAccessor parsedTime = RFC3339_TIME_FORMAT.parse(jsonNode.asText()); ZoneOffset zoneOffset = parsedTime.query(TemporalQueries.offset()); @@ -409,7 +494,6 @@ public class JsonRowDeserializationSchema implements DeserializationSchema List fieldConverters) { return (mapper, jsonNode) -> { ObjectNode node = (ObjectNode) jsonNode; - int arity = fieldNames.length; Row row = new Row(arity); for (int i = 0; i < arity; i++) { @@ -457,4 +541,14 @@ public class JsonRowDeserializationSchema implements DeserializationSchema return array; }; } + + /** + * Exception which refers to parse errors in converters. + * */ + private static final class JsonParseException extends RuntimeException { + private static final long serialVersionUID = 1L; + public JsonParseException(String message, Throwable cause) { + super(message, cause); + } + } } diff --git a/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java b/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java index af758b63c8..b855d94e76 100644 --- a/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java +++ b/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java @@ -67,7 +67,12 @@ public class JsonRowFormatFactory extends TableFormatFactoryBase schema.failOnMissingField(); } }); - + descriptorProperties.getOptionalBoolean(JsonValidator.FORMAT_IGNORE_PARSE_ERRORS) + .ifPresent(flag -> { + if (flag) { + schema.ignoreParseErrors(); + } + }); return schema.build(); } diff --git a/flink-formats/flink-json/src/main/java/org/apache/flink/table/descriptors/Json.java b/flink-formats/flink-json/src/main/java/org/apache/flink/table/descriptors/Json.java index 0f4df09edf..564c88b01d 100644 --- a/flink-formats/flink-json/src/main/java/org/apache/flink/table/descriptors/Json.java +++ b/flink-formats/flink-json/src/main/java/org/apache/flink/table/descriptors/Json.java @@ -27,6 +27,7 @@ import java.util.Map; import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT_DERIVE_SCHEMA; import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_FAIL_ON_MISSING_FIELD; +import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_IGNORE_PARSE_ERRORS; import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_JSON_SCHEMA; import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_SCHEMA; import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_TYPE_VALUE; @@ -38,6 +39,7 @@ public class Json extends FormatDescriptor { private Boolean failOnMissingField; private Boolean deriveSchema; + private Boolean ignoreParseErrors; private String jsonSchema; private String schema; @@ -59,6 +61,17 @@ public class Json extends FormatDescriptor { return this; } + /** + * Sets flag whether to fail when parsing json fails. + * + * @param ignoreParseErrors If set to true, the operation will ignore parse errors. + * If set to false, the operation fails when parsing json fails. + */ + public Json ignoreParseErrors(boolean ignoreParseErrors) { + this.ignoreParseErrors = ignoreParseErrors; + return this; + } + /** * Sets the JSON schema string with field names and the types according to the JSON schema * specification [[http://json-schema.org/specification.html]]. @@ -140,6 +153,10 @@ public class Json extends FormatDescriptor { properties.putBoolean(FORMAT_FAIL_ON_MISSING_FIELD, failOnMissingField); } + if (ignoreParseErrors != null) { + properties.putBoolean(FORMAT_IGNORE_PARSE_ERRORS, ignoreParseErrors); + } + return properties.asMap(); } } diff --git a/flink-formats/flink-json/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java b/flink-formats/flink-json/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java index 061d7fd165..2d0b04e113 100644 --- a/flink-formats/flink-json/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java +++ b/flink-formats/flink-json/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java @@ -31,6 +31,7 @@ public class JsonValidator extends FormatDescriptorValidator { public static final String FORMAT_SCHEMA = "format.schema"; public static final String FORMAT_JSON_SCHEMA = "format.json-schema"; public static final String FORMAT_FAIL_ON_MISSING_FIELD = "format.fail-on-missing-field"; + public static final String FORMAT_IGNORE_PARSE_ERRORS = "format.ignore-parse-errors"; @Override public void validate(DescriptorProperties properties) { @@ -52,5 +53,11 @@ public class JsonValidator extends FormatDescriptorValidator { } properties.validateBoolean(FORMAT_FAIL_ON_MISSING_FIELD, true); + properties.validateBoolean(FORMAT_IGNORE_PARSE_ERRORS, true); + boolean failOnMissingField = properties.getOptionalBoolean(FORMAT_FAIL_ON_MISSING_FIELD).orElse(false); + boolean ignoreParseErrors = properties.getOptionalBoolean(FORMAT_IGNORE_PARSE_ERRORS).orElse(false); + if (ignoreParseErrors && failOnMissingField) { + throw new ValidationException(FORMAT_FAIL_ON_MISSING_FIELD + " and " + FORMAT_IGNORE_PARSE_ERRORS + " shouldn't both be true."); + } } } diff --git a/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java b/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java index 7178987709..cba7fcefd2 100644 --- a/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java +++ b/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java @@ -26,26 +26,37 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMap import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; import org.junit.Assert; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; + +import javax.annotation.Nullable; import java.math.BigDecimal; import java.sql.Date; import java.sql.Time; import java.sql.Timestamp; +import java.util.Arrays; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.concurrent.ThreadLocalRandom; import static org.apache.flink.formats.utils.DeserializationSchemaMatcher.whenDeserializedWith; +import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.instanceOf; import static org.junit.Assert.assertThat; import static org.junit.internal.matchers.ThrowableCauseMatcher.hasCause; +import static org.junit.internal.matchers.ThrowableMessageMatcher.hasMessage; /** * Tests for the {@link JsonRowDeserializationSchema}. */ public class JsonRowDeserializationSchemaTest { + @Rule + public ExpectedException thrown = ExpectedException.none(); + /** * Tests simple deserialization using type information. */ @@ -211,6 +222,20 @@ public class JsonRowDeserializationSchemaTest { assertThat(serializedJson, whenDeserializedWith(deserializationSchema) .failsWithException(hasCause(instanceOf(IllegalStateException.class)))); + + // ignore-parse-errors ignores missing field exception too + deserializationSchema = new JsonRowDeserializationSchema.Builder(rowTypeInformation) + .ignoreParseErrors() + .build(); + assertThat(serializedJson, + whenDeserializedWith(deserializationSchema).equalsTo(row)); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("JSON format doesn't support failOnMissingField and ignoreParseErrors are both true"); + new JsonRowDeserializationSchema.Builder(rowTypeInformation) + .failOnMissingField() + .ignoreParseErrors() + .build(); } /** @@ -228,4 +253,156 @@ public class JsonRowDeserializationSchemaTest { // Expected } } + + @Test + public void testJsonParse() { + for (TestSpec spec : testData) { + testIgnoreParseErrors(spec); + if (spec.errorMessage != null) { + testParseErrors(spec); + } + } + } + + private void testIgnoreParseErrors(TestSpec spec) { + // the parsing field should be null and no exception is thrown + JsonRowDeserializationSchema ignoreErrorsSchema = + new JsonRowDeserializationSchema.Builder(spec.rowTypeInformation) + .ignoreParseErrors() + .build(); + Row expected; + if (spec.expected != null) { + expected = spec.expected; + } else { + expected = new Row(1); + } + assertThat("Test Ignore Parse Error: " + spec.json, + spec.json.getBytes(), + whenDeserializedWith(ignoreErrorsSchema).equalsTo(expected)); + } + + private void testParseErrors(TestSpec spec) { + // expect exception if parse error is not ignored + JsonRowDeserializationSchema failingSchema = + new JsonRowDeserializationSchema.Builder(spec.rowTypeInformation) + .build(); + assertThat("Test Parse Error: " + spec.json, + spec.json.getBytes(), + whenDeserializedWith(failingSchema) + .failsWithException(hasMessage(containsString(spec.errorMessage)))); + } + + private static List testData = Arrays.asList( + TestSpec + .json("{\"id\": \"trueA\"}") + .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.BOOLEAN)) + .expect(Row.of(false)), + + TestSpec + .json("{\"id\": true}") + .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.BOOLEAN)) + .expect(Row.of(true)), + + TestSpec + .json("{\"id\":\"abc\"}") + .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.INT)) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"abc\"}'"), + + TestSpec + .json("{\"id\":112.013}") + .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.LONG)) + .expect(Row.of(112L)), + + TestSpec + .json("{\"id\":\"long\"}") + .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.LONG)) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"long\"}'"), + + TestSpec + .json("{\"id\":\"112.013.123\"}") + .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.FLOAT)) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"112.013.123\"}'"), + + TestSpec + .json("{\"id\":\"112.013.123\"}") + .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.DOUBLE)) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"112.013.123\"}'"), + + TestSpec + .json("{\"id\":\"18:00:243\"}") + .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.SQL_TIME)) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"18:00:243\"}'"), + + TestSpec + .json("{\"id\":\"20191112\"}") + .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.SQL_DATE)) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"20191112\"}'"), + + TestSpec + .json("{\"id\":\"2019-11-12 18:00:12\"}") + .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.SQL_TIMESTAMP)) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"2019-11-12 18:00:12\"}'"), + + TestSpec + .json("{\"id\":\"abc\"}") + .typeInfo(Types.ROW_NAMED(new String[]{"id"}, Types.BIG_DEC)) + .expectErrorMessage("Failed to deserialize JSON '{\"id\":\"abc\"}'"), + + TestSpec + .json("{\"row\":{\"id\":\"abc\"}}") + .typeInfo(Types.ROW_NAMED(new String[]{"row"}, Types.ROW_NAMED(new String[]{"id"}, Types.INT))) + .expect(Row.of(new Row(1))) + .expectErrorMessage("Failed to deserialize JSON '{\"row\":{\"id\":\"abc\"}}'"), + + TestSpec + .json("{\"array\":[123, \"abc\"]}") + .typeInfo(Types.ROW_NAMED(new String[]{"array"}, Types.OBJECT_ARRAY(Types.INT))) + .expect(Row.of((Object) new Integer[]{123, null})) + .expectErrorMessage("Failed to deserialize JSON '{\"array\":[123, \"abc\"]}'"), + + TestSpec + .json("{\"map\":{\"key1\":\"123\", \"key2\":\"abc\"}}") + .typeInfo(Types.ROW_NAMED(new String[]{"map"}, Types.MAP(Types.STRING, Types.INT))) + .expect(Row.of(createHashMap("key1", 123, "key2", null))) + .expectErrorMessage("Failed to deserialize JSON '{\"map\":{\"key1\":\"123\", \"key2\":\"abc\"}}'") + + + ); + + private static Map createHashMap(String k1, Integer v1, String k2, Integer v2) { + Map map = new HashMap<>(); + map.put(k1, v1); + map.put(k2, v2); + return map; + } + + private static class TestSpec { + private final String json; + private @Nullable TypeInformation rowTypeInformation; + private @Nullable Row expected; + private @Nullable String errorMessage; + + private TestSpec(String json) { + this.json = json; + } + + public static TestSpec json(String json) { + return new TestSpec(json); + } + + TestSpec expect(Row row) { + this.expected = row; + return this; + } + + TestSpec typeInfo(TypeInformation rowTypeInformation) { + this.rowTypeInformation = rowTypeInformation; + return this; + } + + TestSpec expectErrorMessage(String errorMessage) { + this.errorMessage = errorMessage; + return this; + } + } } diff --git a/flink-formats/flink-json/src/test/java/org/apache/flink/table/descriptors/JsonTest.java b/flink-formats/flink-json/src/test/java/org/apache/flink/table/descriptors/JsonTest.java index 2bc58987dc..5436b2d2d6 100644 --- a/flink-formats/flink-json/src/test/java/org/apache/flink/table/descriptors/JsonTest.java +++ b/flink-formats/flink-json/src/test/java/org/apache/flink/table/descriptors/JsonTest.java @@ -65,6 +65,11 @@ public class JsonTest extends DescriptorTestBase { addPropertyAndVerify(descriptors().get(0), "format.schema", "DDD"); } + @Test(expected = ValidationException.class) + public void testInvalidIgnoreParseErrors() { + addPropertyAndVerify(descriptors().get(0), "format.ignore-parse-errors", "DDD"); + } + // -------------------------------------------------------------------------------------------- @Override @@ -84,7 +89,11 @@ public class JsonTest extends DescriptorTestBase { final Descriptor desc5 = new Json().failOnMissingField(false); - return Arrays.asList(desc1, desc2, desc3, desc4, desc5); + final Descriptor desc6 = new Json().jsonSchema(JSON_SCHEMA).ignoreParseErrors(false); + + final Descriptor desc7 = new Json().ignoreParseErrors(true); + + return Arrays.asList(desc1, desc2, desc3, desc4, desc5, desc6, desc7); } @Override @@ -116,7 +125,18 @@ public class JsonTest extends DescriptorTestBase { props5.put("format.property-version", "1"); props5.put("format.fail-on-missing-field", "false"); - return Arrays.asList(props1, props2, props3, props4, props5); + final Map props6 = new HashMap<>(); + props6.put("format.type", "json"); + props6.put("format.property-version", "1"); + props6.put("format.json-schema", JSON_SCHEMA); + props6.put("format.ignore-parse-errors", "false"); + + final Map props7 = new HashMap<>(); + props7.put("format.type", "json"); + props7.put("format.property-version", "1"); + props7.put("format.ignore-parse-errors", "true"); + + return Arrays.asList(props1, props2, props3, props4, props5, props6, props7); } @Override -- Gitee From 1ad361b787ad2929b334a72e7f5be944f4fe27af Mon Sep 17 00:00:00 2001 From: Roman Khachatryan Date: Thu, 5 Mar 2020 15:19:23 +0100 Subject: [PATCH 122/885] [FLINK-16512][task] Unaligned checkpoints: API for persistence --- .../channel/ChannelStateReader.java | 66 ++++++++++ .../channel/ChannelStateWriter.java | 117 ++++++++++++++++++ .../checkpoint/channel/InputChannelInfo.java | 48 +++++++ .../channel/ResultSubpartitionInfo.java | 49 ++++++++ .../state/InputChannelStateHandle.java | 26 ++++ .../state/ResultSubpartitionStateHandle.java | 26 ++++ 6 files changed, 332 insertions(+) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateReader.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriter.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/InputChannelInfo.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ResultSubpartitionInfo.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/InputChannelStateHandle.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/ResultSubpartitionStateHandle.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateReader.java new file mode 100644 index 0000000000..49321cc1d4 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateReader.java @@ -0,0 +1,66 @@ +package org.apache.flink.runtime.checkpoint.channel; +/* + * 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. + */ + +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.io.network.buffer.Buffer; + +import java.io.IOException; + +/** + * Reads channel state saved during checkpoint/savepoint. + */ +@Internal +public interface ChannelStateReader extends AutoCloseable { + + /** + * Status of reading result. + */ + enum ReadResult { HAS_MORE_DATA, NO_MORE_DATA } + + /** + * Put data into the supplied buffer to be injected into + * {@link org.apache.flink.runtime.io.network.partition.consumer.InputChannel InputChannel}. + */ + ReadResult readInputData(InputChannelInfo info, Buffer buffer) throws IOException; + + /** + * Put data into the supplied buffer to be injected into + * {@link org.apache.flink.runtime.io.network.partition.ResultSubpartition ResultSubpartition}. + */ + ReadResult readOutputData(ResultSubpartitionInfo info, Buffer buffer) throws IOException; + + @Override + void close() throws Exception; + + ChannelStateReader NO_OP = new ChannelStateReader() { + + @Override + public ReadResult readInputData(InputChannelInfo info, Buffer buffer) { + return ReadResult.NO_MORE_DATA; + } + + @Override + public ReadResult readOutputData(ResultSubpartitionInfo info, Buffer buffer) { + return ReadResult.NO_MORE_DATA; + } + + @Override + public void close() { + } + }; +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriter.java new file mode 100644 index 0000000000..24a3a4edc6 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriter.java @@ -0,0 +1,117 @@ +/* + * 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.runtime.checkpoint.channel; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.state.StateObject; + +import java.util.Collection; +import java.util.Collections; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Future; + +/** + * Writes channel state during checkpoint/savepoint. + */ +@Internal +public interface ChannelStateWriter extends AutoCloseable { + + /** + * Sequence number for the buffers that were saved during the previous execution attempt; then restored; and now are + * to be saved again (as opposed to the buffers received from the upstream or from the operator). + */ + int SEQUENCE_NUMBER_RESTORED = -1; + + /** + * Signifies that buffer sequence number is unknown (e.g. if passing sequence numbers is not implemented). + */ + int SEQUENCE_NUMBER_UNKNOWN = -2; + + /** + * Initiate write of channel state for the given checkpoint id. + */ + void start(long checkpointId); + + /** + * Add in-flight buffers from the {@link org.apache.flink.runtime.io.network.partition.consumer.InputChannel InputChannel}. + * Must be called after {@link #start(long)} and before {@link #finish(long)}. + * @param startSeqNum sequence number of the 1st passed buffer. + * It is intended to use for incremental snapshots. + * If no data is passed it is ignored. + * @param data zero or more buffers ordered by their sequence numbers + * @see org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter#SEQUENCE_NUMBER_RESTORED + * @see org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter#SEQUENCE_NUMBER_UNKNOWN + */ + void addInputData(long checkpointId, InputChannelInfo info, int startSeqNum, Buffer... data); + + /** + * Add in-flight buffers from the {@link org.apache.flink.runtime.io.network.partition.ResultSubpartition ResultSubpartition}. + * Must be called after {@link #start(long)} and before {@link #finish(long)}. + * @param startSeqNum sequence number of the 1st passed buffer. + * It is intended to use for incremental snapshots. + * If no data is passed it is ignored. + * @param data zero or more buffers ordered by their sequence numbers + * @see org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter#SEQUENCE_NUMBER_RESTORED + * @see org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter#SEQUENCE_NUMBER_UNKNOWN + */ + void addOutputData(long checkpointId, ResultSubpartitionInfo info, int startSeqNum, Buffer... data); + + /** + * Finalize write of channel state for the given checkpoint id. + * Must be called after {@link #start(long)} and all of the data of the given checkpoint added. + */ + void finish(long checkpointId); + + /** + * Must be called after {@link #start(long)}. + */ + Future> getWriteCompletionFuture(long checkpointId); + + @Override + void close() throws Exception; + + ChannelStateWriter NO_OP = new ChannelStateWriter() { + + @Override + public void start(long checkpointId) { + } + + @Override + public void addInputData(long checkpointId, InputChannelInfo info, int startSeqNum, Buffer... data) { + } + + @Override + public void addOutputData(long checkpointId, ResultSubpartitionInfo info, int startSeqNum, Buffer... data) { + } + + @Override + public void finish(long checkpointId) { + } + + @Override + public Future> getWriteCompletionFuture(long checkpointId) { + return CompletableFuture.completedFuture(Collections.emptyList()); + } + + @Override + public void close() { + } + }; + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/InputChannelInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/InputChannelInfo.java new file mode 100644 index 0000000000..e8339b8356 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/InputChannelInfo.java @@ -0,0 +1,48 @@ +/* + * 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.runtime.checkpoint.channel; + +import org.apache.flink.annotation.Internal; + +import java.io.Serializable; + +/** + * Identifies {@link org.apache.flink.runtime.io.network.partition.consumer.InputChannel} in a given subtask. + * Note that {@link org.apache.flink.runtime.io.network.partition.consumer.InputChannelID InputChannelID} + * can not be used because it is generated randomly. + */ +@Internal +public class InputChannelInfo implements Serializable { + private static final long serialVersionUID = 1L; + + private final int gateIdx; + private final int inputChannelIdx; + + public InputChannelInfo(int gateIdx, int inputChannelIdx) { + this.gateIdx = gateIdx; + this.inputChannelIdx = inputChannelIdx; + } + + public int getGateIdx() { + return gateIdx; + } + + public int getInputChannelIdx() { + return inputChannelIdx; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ResultSubpartitionInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ResultSubpartitionInfo.java new file mode 100644 index 0000000000..513f1560c8 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ResultSubpartitionInfo.java @@ -0,0 +1,49 @@ +/* + * 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.runtime.checkpoint.channel; + +import org.apache.flink.annotation.Internal; + +import java.io.Serializable; + +/** + * Identifies {@link org.apache.flink.runtime.io.network.partition.ResultSubpartition ResultSubpartition} in a given subtask. + * Note that {@link org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID IntermediateResultPartitionID} + * can not be used because it: a) identifies the whole + * {@link org.apache.flink.runtime.io.network.partition.ResultPartition ResultPartition} b) is generated randomly. + */ +@Internal +public class ResultSubpartitionInfo implements Serializable { + private static final long serialVersionUID = 1L; + + private final int partitionIdx; + private final int subPartitionIdx; + + public ResultSubpartitionInfo(int partitionIdx, int subPartitionIdx) { + this.partitionIdx = partitionIdx; + this.subPartitionIdx = subPartitionIdx; + } + + public int getPartitionIdx() { + return partitionIdx; + } + + public int getSubPartitionIdx() { + return subPartitionIdx; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/InputChannelStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/InputChannelStateHandle.java new file mode 100644 index 0000000000..61b902f2f9 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/InputChannelStateHandle.java @@ -0,0 +1,26 @@ +package org.apache.flink.runtime.state; +/* + * 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. + */ + +import org.apache.flink.runtime.checkpoint.channel.InputChannelInfo; + +/** + * {@link StateObject Handle} to an {@link org.apache.flink.runtime.io.network.partition.consumer.InputChannel InputChannel} state. + */ +public interface InputChannelStateHandle extends StateObject { + InputChannelInfo getInputChannelInfo(); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/ResultSubpartitionStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ResultSubpartitionStateHandle.java new file mode 100644 index 0000000000..7f6c90e37e --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ResultSubpartitionStateHandle.java @@ -0,0 +1,26 @@ +package org.apache.flink.runtime.state; +/* + * 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. + */ + +import org.apache.flink.runtime.checkpoint.channel.ResultSubpartitionInfo; + +/** + * {@link StateObject Handle} to a {@link org.apache.flink.runtime.io.network.partition.ResultSubpartition ResultSubpartition} state. + */ +public interface ResultSubpartitionStateHandle extends StateObject { + ResultSubpartitionInfo getResultSubpartitionInfo(); +} -- Gitee From 642f096163798962d1e9412c195985aee13a82fc Mon Sep 17 00:00:00 2001 From: Fabian Paul Date: Tue, 10 Mar 2020 16:51:09 +0100 Subject: [PATCH 123/885] [FLINK-16525][task] Increment subtask id by 1 to display subtask name --- .../api/functions/sink/TwoPhaseCommitSinkFunction.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java index 25e1d4a26d..6a42fb964d 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java @@ -484,7 +484,7 @@ public abstract class TwoPhaseCommitSinkFunction return String.format( "%s %s/%s", this.getClass().getSimpleName(), - getRuntimeContext().getIndexOfThisSubtask(), + getRuntimeContext().getIndexOfThisSubtask() + 1, getRuntimeContext().getNumberOfParallelSubtasks()); } -- Gitee From ffc8e42d6c891c0959ee3b4c0347903eed369e8e Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Wed, 11 Mar 2020 09:31:10 +0100 Subject: [PATCH 124/885] [FLINK-16014][s3] Force usage of SAXParserFactory over XMLReaderFactory This avoids a ClassNotFound exception of the underlying XML reader class in plugin classloaders. Root cause is a bug in JDK8 (JDK-8015099). XMLReaderFactory caches the class name independent of the classloader. On EMR, xercesImpl is on classpath (because of HDFS) and will be loaded at some point in time. --- .../transform/XmlResponsesSaxParser.java | 3767 +++++++++++++++++ flink-filesystems/flink-s3-fs-hadoop/pom.xml | 9 + flink-filesystems/flink-s3-fs-presto/pom.xml | 9 + tools/maven/suppressions.xml | 4 + 4 files changed, 3789 insertions(+) create mode 100644 flink-filesystems/flink-s3-fs-base/src/main/java/com/amazonaws/services/s3/model/transform/XmlResponsesSaxParser.java diff --git a/flink-filesystems/flink-s3-fs-base/src/main/java/com/amazonaws/services/s3/model/transform/XmlResponsesSaxParser.java b/flink-filesystems/flink-s3-fs-base/src/main/java/com/amazonaws/services/s3/model/transform/XmlResponsesSaxParser.java new file mode 100644 index 0000000000..bc87b234df --- /dev/null +++ b/flink-filesystems/flink-s3-fs-base/src/main/java/com/amazonaws/services/s3/model/transform/XmlResponsesSaxParser.java @@ -0,0 +1,3767 @@ +/* + * Copyright 2010-2018 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Portions copyright 2006-2009 James Murty. Please see LICENSE.txt + * for applicable license terms and NOTICE.txt for applicable notices. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://aws.amazon.com/apache2.0 + * + * or in the "license" file accompanying this file. This file 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 com.amazonaws.services.s3.model.transform; + +import com.amazonaws.services.s3.model.*; + +import static com.amazonaws.util.StringUtils.UTF8; + +import com.amazonaws.services.s3.model.inventory.ServerSideEncryptionKMS; +import com.amazonaws.services.s3.model.inventory.ServerSideEncryptionS3; +import java.io.BufferedReader; +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.util.ArrayList; +import java.util.Date; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +import com.amazonaws.services.s3.model.lifecycle.LifecycleAndOperator; +import com.amazonaws.services.s3.model.lifecycle.LifecycleFilter; +import com.amazonaws.services.s3.model.lifecycle.LifecycleFilterPredicate; +import com.amazonaws.services.s3.model.lifecycle.LifecyclePrefixPredicate; +import com.amazonaws.services.s3.model.lifecycle.LifecycleTagPredicate; +import com.amazonaws.services.s3.model.metrics.MetricsAndOperator; +import com.amazonaws.services.s3.model.metrics.MetricsConfiguration; +import com.amazonaws.services.s3.model.metrics.MetricsFilter; +import com.amazonaws.services.s3.model.metrics.MetricsFilterPredicate; +import com.amazonaws.services.s3.model.metrics.MetricsPrefixPredicate; +import com.amazonaws.services.s3.model.metrics.MetricsTagPredicate; +import com.amazonaws.services.s3.model.analytics.AnalyticsAndOperator; +import com.amazonaws.services.s3.model.analytics.AnalyticsConfiguration; +import com.amazonaws.services.s3.model.analytics.AnalyticsExportDestination; +import com.amazonaws.services.s3.model.analytics.AnalyticsFilter; +import com.amazonaws.services.s3.model.analytics.AnalyticsFilterPredicate; +import com.amazonaws.services.s3.model.analytics.AnalyticsPrefixPredicate; +import com.amazonaws.services.s3.model.analytics.AnalyticsS3BucketDestination; +import com.amazonaws.services.s3.model.analytics.AnalyticsTagPredicate; +import com.amazonaws.services.s3.model.analytics.StorageClassAnalysis; +import com.amazonaws.services.s3.model.analytics.StorageClassAnalysisDataExport; +import com.amazonaws.services.s3.model.inventory.InventoryConfiguration; +import com.amazonaws.services.s3.model.inventory.InventoryDestination; +import com.amazonaws.services.s3.model.inventory.InventoryFilter; +import com.amazonaws.services.s3.model.inventory.InventoryPrefixPredicate; +import com.amazonaws.services.s3.model.inventory.InventoryS3BucketDestination; +import com.amazonaws.services.s3.model.inventory.InventorySchedule; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.xml.sax.Attributes; +import org.xml.sax.InputSource; +import org.xml.sax.SAXException; +import org.xml.sax.SAXNotRecognizedException; +import org.xml.sax.SAXNotSupportedException; +import org.xml.sax.XMLReader; +import org.xml.sax.helpers.DefaultHandler; +import org.xml.sax.helpers.XMLReaderFactory; + +import com.amazonaws.SdkClientException; +import com.amazonaws.services.s3.internal.Constants; +import com.amazonaws.services.s3.internal.DeleteObjectsResponse; +import com.amazonaws.services.s3.internal.ObjectExpirationResult; +import com.amazonaws.services.s3.internal.S3RequesterChargedResult; +import com.amazonaws.services.s3.internal.S3VersionResult; +import com.amazonaws.services.s3.internal.ServerSideEncryptionResult; +import com.amazonaws.services.s3.internal.ServiceUtils; +import com.amazonaws.services.s3.model.BucketLifecycleConfiguration.NoncurrentVersionTransition; +import com.amazonaws.services.s3.model.BucketLifecycleConfiguration.Rule; +import com.amazonaws.services.s3.model.BucketLifecycleConfiguration.Transition; +import com.amazonaws.services.s3.model.CORSRule.AllowedMethods; +import com.amazonaws.services.s3.model.DeleteObjectsResult.DeletedObject; +import com.amazonaws.services.s3.model.MultiObjectDeleteException.DeleteError; +import com.amazonaws.services.s3.model.RequestPaymentConfiguration.Payer; +import com.amazonaws.util.DateUtils; +import com.amazonaws.util.SdkHttpUtils; +import com.amazonaws.util.StringUtils; + +import javax.xml.parsers.ParserConfigurationException; +import javax.xml.parsers.SAXParserFactory; + +// ---------------------------------------------------------------------------- +// ---------------------------------------------------------------------------- +// +// This class is copied from aws-java-sdk-s3 (Apache License 2.0) +// in order to replaced {@code XMLReaderFactory.createXMLReader} by +// {@code SAX_PARSER_FACTORY.newSAXParser().getXMLReader()} to +// avoid JDK-8015099. +// +// ---------------------------------------------------------------------------- +// ---------------------------------------------------------------------------- + +/** + * XML Sax parser to read XML documents returned by S3 via the REST interface, + * converting these documents into objects. + */ +public class XmlResponsesSaxParser { + private static final Log log = LogFactory.getLog(XmlResponsesSaxParser.class); + + private static final SAXParserFactory SAX_PARSER_FACTORY = SAXParserFactory.newInstance(); + + private XMLReader xr = null; + + private boolean sanitizeXmlDocument = true; + + /** + * Constructs the XML SAX parser. + * + * @throws SdkClientException + */ + public XmlResponsesSaxParser() throws SdkClientException { + // Ensure we can load the XML Reader. + try { + xr = SAX_PARSER_FACTORY.newSAXParser().getXMLReader(); + disableExternalResourceFetching(xr); + } catch (SAXException | ParserConfigurationException e) { + throw new SdkClientException("Couldn't initialize a SAX driver to create an XMLReader", e); + } + } + + /** + * Parses an XML document from an input stream using a document handler. + * + * @param handler + * the handler for the XML document + * @param inputStream + * an input stream containing the XML document to parse + * + * @throws IOException + * on error reading from the input stream (ie connection reset) + * @throws SdkClientException + * on error with malformed XML, etc + */ + protected void parseXmlInputStream(DefaultHandler handler, InputStream inputStream) + throws IOException { + try { + + if (log.isDebugEnabled()) { + log.debug("Parsing XML response document with handler: " + handler.getClass()); + } + + BufferedReader breader = new BufferedReader(new InputStreamReader(inputStream, + Constants.DEFAULT_ENCODING)); + xr.setContentHandler(handler); + xr.setErrorHandler(handler); + xr.parse(new InputSource(breader)); + + } catch (IOException e) { + throw e; + + } catch (Throwable t) { + try { + inputStream.close(); + } catch (IOException e) { + if (log.isErrorEnabled()) { + log.error("Unable to close response InputStream up after XML parse failure", e); + } + } + throw new SdkClientException("Failed to parse XML document with handler " + + handler.getClass(), t); + } + } + + protected InputStream sanitizeXmlDocument(DefaultHandler handler, InputStream inputStream) + throws IOException { + + if (!sanitizeXmlDocument) { + // No sanitizing will be performed, return the original input stream unchanged. + return inputStream; + } else { + if (log.isDebugEnabled()) { + log.debug("Sanitizing XML document destined for handler " + handler.getClass()); + } + + InputStream sanitizedInputStream = null; + + try { + + /* + * Read object listing XML document from input stream provided into a + * string buffer, so we can replace troublesome characters before + * sending the document to the XML parser. + */ + StringBuilder listingDocBuffer = new StringBuilder(); + BufferedReader br = new BufferedReader( + new InputStreamReader(inputStream, Constants.DEFAULT_ENCODING)); + + char[] buf = new char[8192]; + int read = -1; + while ((read = br.read(buf)) != -1) { + listingDocBuffer.append(buf, 0, read); + } + br.close(); + + /* + * Replace any carriage return (\r) characters with explicit XML + * character entities, to prevent the SAX parser from + * misinterpreting 0x0D characters as 0x0A and being unable to + * parse the XML. + */ + String listingDoc = listingDocBuffer.toString().replaceAll("\r", " "); + + sanitizedInputStream = new ByteArrayInputStream( + listingDoc.getBytes(UTF8)); + + } catch (IOException e) { + throw e; + + } catch (Throwable t) { + try { + inputStream.close(); + } catch (IOException e) { + if (log.isErrorEnabled()) { + log.error("Unable to close response InputStream after failure sanitizing XML document", e); + } + } + throw new SdkClientException("Failed to sanitize XML document destined for handler " + + handler.getClass(), t); + } + return sanitizedInputStream; + } + } + + /** + * Disables certain dangerous features that attempt to automatically fetch DTDs + * + * See OWASP XXE Cheat Sheet + * @param reader the reader to disable the features on + * @throws SAXNotRecognizedException + * @throws SAXNotSupportedException + */ + private void disableExternalResourceFetching(XMLReader reader) throws SAXNotRecognizedException, SAXNotSupportedException { + reader.setFeature("http://xml.org/sax/features/external-general-entities", false); + reader.setFeature("http://xml.org/sax/features/external-parameter-entities", false); + reader.setFeature("http://apache.org/xml/features/nonvalidating/load-external-dtd",false); + } + + /** + * Checks if the specified string is empty or null and if so, returns null. + * Otherwise simply returns the string. + * + * @param s + * The string to check. + * @return Null if the specified string was null, or empty, otherwise + * returns the string the caller passed in. + */ + private static String checkForEmptyString(String s) { + if (s == null) return null; + if (s.length() == 0) return null; + + return s; + } + + /** + * Safely parses the specified string as an integer and returns the value. + * If a NumberFormatException occurs while parsing the integer, an error is + * logged and -1 is returned. + * + * @param s + * The string to parse and return as an integer. + * + * @return The integer value of the specified string, otherwise -1 if there + * were any problems parsing the string as an integer. + */ + private static int parseInt(String s) { + try { + return Integer.parseInt(s); + } catch (NumberFormatException nfe) { + log.error("Unable to parse integer value '" + s + "'", nfe); + } + + return -1; + } + + /** + * Safely parses the specified string as a long and returns the value. If a + * NumberFormatException occurs while parsing the long, an error is logged + * and -1 is returned. + * + * @param s + * The string to parse and return as a long. + * + * @return The long value of the specified string, otherwise -1 if there + * were any problems parsing the string as a long. + */ + private static long parseLong(String s) { + try { + return Long.parseLong(s); + } catch (NumberFormatException nfe) { + log.error("Unable to parse long value '" + s + "'", nfe); + } + + return -1; + } + + /** + * Perform a url decode on the given value if specified. + * Return value by default; + */ + private static String decodeIfSpecified(String value, boolean decode) { + return decode ? SdkHttpUtils.urlDecode(value) : value; + } + + /** + * Parses a ListBucket response XML document from an input stream. + * + * @param inputStream + * XML data input stream. + * @return the XML handler object populated with data parsed from the XML + * stream. + * @throws SdkClientException + */ + public ListBucketHandler parseListBucketObjectsResponse(InputStream inputStream, final boolean shouldSDKDecodeResponse) + throws IOException { + ListBucketHandler handler = new ListBucketHandler(shouldSDKDecodeResponse); + parseXmlInputStream(handler, sanitizeXmlDocument(handler, inputStream)); + + return handler; + } + + /** + * Parses a ListBucketV2 response XML document from an input stream. + * + * @param inputStream + * XML data input stream. + * @return the XML handler object populated with data parsed from the XML + * stream. + * @throws SdkClientException + */ + public ListObjectsV2Handler parseListObjectsV2Response(InputStream inputStream, final boolean shouldSDKDecodeResponse) + throws IOException { + ListObjectsV2Handler handler = new ListObjectsV2Handler(shouldSDKDecodeResponse); + parseXmlInputStream(handler, sanitizeXmlDocument(handler, inputStream)); + + return handler; + } + + /** + * Parses a ListVersions response XML document from an input stream. + * + * @param inputStream + * XML data input stream. + * @return the XML handler object populated with data parsed from the XML + * stream. + * @throws SdkClientException + */ + public ListVersionsHandler parseListVersionsResponse(InputStream inputStream, final boolean shouldSDKDecodeResponse) + throws IOException { + ListVersionsHandler handler = new ListVersionsHandler(shouldSDKDecodeResponse); + parseXmlInputStream(handler, sanitizeXmlDocument(handler, inputStream)); + return handler; + } + + /** + * Parses a ListAllMyBuckets response XML document from an input stream. + * + * @param inputStream + * XML data input stream. + * @return the XML handler object populated with data parsed from the XML + * stream. + * @throws SdkClientException + */ + public ListAllMyBucketsHandler parseListMyBucketsResponse(InputStream inputStream) + throws IOException { + ListAllMyBucketsHandler handler = new ListAllMyBucketsHandler(); + parseXmlInputStream(handler, sanitizeXmlDocument(handler, inputStream)); + return handler; + } + + /** + * Parses an AccessControlListHandler response XML document from an input + * stream. + * + * @param inputStream + * XML data input stream. + * @return the XML handler object populated with data parsed from the XML + * stream. + * + * @throws SdkClientException + */ + public AccessControlListHandler parseAccessControlListResponse(InputStream inputStream) + throws IOException { + AccessControlListHandler handler = new AccessControlListHandler(); + parseXmlInputStream(handler, inputStream); + return handler; + } + + /** + * Parses a LoggingStatus response XML document for a bucket from an input + * stream. + * + * @param inputStream + * XML data input stream. + * @return the XML handler object populated with data parsed from the XML + * stream. + * + * @throws SdkClientException + */ + public BucketLoggingConfigurationHandler parseLoggingStatusResponse(InputStream inputStream) + throws IOException { + BucketLoggingConfigurationHandler handler = new BucketLoggingConfigurationHandler(); + parseXmlInputStream(handler, inputStream); + return handler; + } + + public BucketLifecycleConfigurationHandler parseBucketLifecycleConfigurationResponse(InputStream inputStream) + throws IOException { + BucketLifecycleConfigurationHandler handler = new BucketLifecycleConfigurationHandler(); + parseXmlInputStream(handler, inputStream); + return handler; + } + + public BucketCrossOriginConfigurationHandler parseBucketCrossOriginConfigurationResponse(InputStream inputStream) + throws IOException { + BucketCrossOriginConfigurationHandler handler = new BucketCrossOriginConfigurationHandler(); + parseXmlInputStream(handler, inputStream); + return handler; + } + + public String parseBucketLocationResponse(InputStream inputStream) + throws IOException { + BucketLocationHandler handler = new BucketLocationHandler(); + parseXmlInputStream(handler, inputStream); + return handler.getLocation(); + } + + public BucketVersioningConfigurationHandler parseVersioningConfigurationResponse(InputStream inputStream) + throws IOException { + BucketVersioningConfigurationHandler handler = new BucketVersioningConfigurationHandler(); + parseXmlInputStream(handler, inputStream); + return handler; + } + + public BucketWebsiteConfigurationHandler parseWebsiteConfigurationResponse(InputStream inputStream) + throws IOException { + BucketWebsiteConfigurationHandler handler = new BucketWebsiteConfigurationHandler(); + parseXmlInputStream(handler, inputStream); + return handler; + } + + + public BucketReplicationConfigurationHandler parseReplicationConfigurationResponse(InputStream inputStream) + throws IOException { + BucketReplicationConfigurationHandler handler = new BucketReplicationConfigurationHandler(); + parseXmlInputStream(handler, inputStream); + return handler; + } + + public BucketTaggingConfigurationHandler parseTaggingConfigurationResponse(InputStream inputStream) + throws IOException { + BucketTaggingConfigurationHandler handler = new BucketTaggingConfigurationHandler(); + parseXmlInputStream(handler, inputStream); + return handler; + } + + public BucketAccelerateConfigurationHandler parseAccelerateConfigurationResponse(InputStream inputStream) + throws IOException { + BucketAccelerateConfigurationHandler handler = new BucketAccelerateConfigurationHandler(); + parseXmlInputStream(handler, inputStream); + return handler; + } + + public DeleteObjectsHandler parseDeletedObjectsResult(InputStream inputStream) + throws IOException { + DeleteObjectsHandler handler = new DeleteObjectsHandler(); + parseXmlInputStream(handler, inputStream); + return handler; + } + + public CopyObjectResultHandler parseCopyObjectResponse(InputStream inputStream) + throws IOException { + CopyObjectResultHandler handler = new CopyObjectResultHandler(); + parseXmlInputStream(handler, inputStream); + return handler; + } + + public CompleteMultipartUploadHandler parseCompleteMultipartUploadResponse(InputStream inputStream) + throws IOException { + CompleteMultipartUploadHandler handler = new CompleteMultipartUploadHandler(); + parseXmlInputStream(handler, inputStream); + return handler; + } + + public InitiateMultipartUploadHandler parseInitiateMultipartUploadResponse(InputStream inputStream) + throws IOException { + InitiateMultipartUploadHandler handler = new InitiateMultipartUploadHandler(); + parseXmlInputStream(handler, inputStream); + return handler; + } + + public ListMultipartUploadsHandler parseListMultipartUploadsResponse(InputStream inputStream) + throws IOException { + ListMultipartUploadsHandler handler = new ListMultipartUploadsHandler(); + parseXmlInputStream(handler, inputStream); + return handler; + } + + public ListPartsHandler parseListPartsResponse(InputStream inputStream) + throws IOException { + ListPartsHandler handler = new ListPartsHandler(); + parseXmlInputStream(handler, inputStream); + return handler; + } + + public GetObjectTaggingHandler parseObjectTaggingResponse(InputStream inputStream) throws IOException { + GetObjectTaggingHandler handler = new GetObjectTaggingHandler(); + parseXmlInputStream(handler, inputStream); + return handler; + } + + public GetBucketMetricsConfigurationHandler parseGetBucketMetricsConfigurationResponse(InputStream inputStream) + throws IOException { + GetBucketMetricsConfigurationHandler handler = new GetBucketMetricsConfigurationHandler(); + parseXmlInputStream(handler, inputStream); + return handler; + } + + public ListBucketMetricsConfigurationsHandler parseListBucketMetricsConfigurationsResponse(InputStream inputStream) + throws IOException { + ListBucketMetricsConfigurationsHandler handler = new ListBucketMetricsConfigurationsHandler(); + parseXmlInputStream(handler, inputStream); + return handler; + } + + public GetBucketAnalyticsConfigurationHandler parseGetBucketAnalyticsConfigurationResponse(InputStream inputStream) + throws IOException { + GetBucketAnalyticsConfigurationHandler handler = new GetBucketAnalyticsConfigurationHandler(); + parseXmlInputStream(handler, inputStream); + return handler; + } + + public ListBucketAnalyticsConfigurationHandler parseListBucketAnalyticsConfigurationResponse(InputStream inputStream) + throws IOException { + ListBucketAnalyticsConfigurationHandler handler = new ListBucketAnalyticsConfigurationHandler(); + parseXmlInputStream(handler, inputStream); + return handler; + } + + public GetBucketInventoryConfigurationHandler parseGetBucketInventoryConfigurationResponse(InputStream inputStream) + throws IOException { + GetBucketInventoryConfigurationHandler handler = new GetBucketInventoryConfigurationHandler(); + parseXmlInputStream(handler, inputStream); + return handler; + } + + public ListBucketInventoryConfigurationsHandler parseBucketListInventoryConfigurationsResponse(InputStream inputStream) + throws IOException { + ListBucketInventoryConfigurationsHandler handler = new ListBucketInventoryConfigurationsHandler(); + parseXmlInputStream(handler, inputStream); + return handler; + } + + /** + * @param inputStream + * + * @return true if the bucket's is configured as Requester Pays, false if it + * is configured as Owner pays. + * + * @throws SdkClientException + */ + public RequestPaymentConfigurationHandler parseRequestPaymentConfigurationResponse(InputStream inputStream) + throws IOException { + RequestPaymentConfigurationHandler handler = new RequestPaymentConfigurationHandler(); + parseXmlInputStream(handler, inputStream); + return handler; + } + + // //////////// + // Handlers // + // //////////// + + /** + * Handler for ListBucket response XML documents. + */ + public static class ListBucketHandler extends AbstractHandler { + private final ObjectListing objectListing = new ObjectListing(); + private final boolean shouldSDKDecodeResponse; + + private S3ObjectSummary currentObject = null; + private Owner currentOwner = null; + private String lastKey = null; + + public ListBucketHandler(final boolean shouldSDKDecodeResponse) { + this.shouldSDKDecodeResponse = shouldSDKDecodeResponse; + } + + public ObjectListing getObjectListing() { + return objectListing; + } + + @Override + protected void doStartElement( + String uri, + String name, + String qName, + Attributes attrs) { + + if (in("ListBucketResult")) { + if (name.equals("Contents")) { + currentObject = new S3ObjectSummary(); + currentObject.setBucketName(objectListing.getBucketName()); + } + } + + else if (in("ListBucketResult", "Contents")) { + if (name.equals("Owner")) { + currentOwner = new Owner(); + } + } + } + + @Override + protected void doEndElement(String uri, String name, String qName) { + if (atTopLevel()) { + if (name.equals("ListBucketResult")) { + /* + * S3 only includes the NextMarker XML element if the + * request specified a delimiter, but for consistency we'd + * like to always give easy access to the next marker if + * we're returning a list of results that's truncated. + */ + if (objectListing.isTruncated() + && objectListing.getNextMarker() == null) { + + String nextMarker = null; + if (!objectListing.getObjectSummaries().isEmpty()) { + nextMarker = objectListing.getObjectSummaries() + .get(objectListing.getObjectSummaries().size() - 1) + .getKey(); + + } else if (!objectListing.getCommonPrefixes().isEmpty()) { + nextMarker = objectListing.getCommonPrefixes() + .get(objectListing.getCommonPrefixes().size() - 1); + } else { + log.error("S3 response indicates truncated results, " + + "but contains no object summaries or " + + "common prefixes."); + } + + objectListing.setNextMarker(nextMarker); + } + } + } + + else if (in("ListBucketResult")) { + if (name.equals("Name")) { + objectListing.setBucketName(getText()); + if (log.isDebugEnabled()) { + log.debug("Examining listing for bucket: " + + objectListing.getBucketName()); + } + + } else if (name.equals("Prefix")) { + objectListing.setPrefix(decodeIfSpecified + (checkForEmptyString(getText()), shouldSDKDecodeResponse)); + + } else if (name.equals("Marker")) { + objectListing.setMarker(decodeIfSpecified + (checkForEmptyString(getText()), shouldSDKDecodeResponse)); + + } else if (name.equals("NextMarker")) { + objectListing.setNextMarker(decodeIfSpecified + (getText(), shouldSDKDecodeResponse)); + + } else if (name.equals("MaxKeys")) { + objectListing.setMaxKeys(parseInt(getText())); + + } else if (name.equals("Delimiter")) { + objectListing.setDelimiter(decodeIfSpecified + (checkForEmptyString(getText()), shouldSDKDecodeResponse)); + + } else if (name.equals("EncodingType")) { + objectListing.setEncodingType(shouldSDKDecodeResponse ? + null : checkForEmptyString(getText())); + } else if (name.equals("IsTruncated")) { + String isTruncatedStr = + StringUtils.lowerCase(getText()); + + if (isTruncatedStr.startsWith("false")) { + objectListing.setTruncated(false); + } else if (isTruncatedStr.startsWith("true")) { + objectListing.setTruncated(true); + } else { + throw new IllegalStateException( + "Invalid value for IsTruncated field: " + + isTruncatedStr); + } + + } else if (name.equals("Contents")) { + objectListing.getObjectSummaries().add(currentObject); + currentObject = null; + } + } + + else if (in("ListBucketResult", "Contents")) { + if (name.equals("Key")) { + lastKey = getText(); + currentObject.setKey(decodeIfSpecified + (lastKey, shouldSDKDecodeResponse)); + } else if (name.equals("LastModified")) { + currentObject.setLastModified( + ServiceUtils.parseIso8601Date(getText())); + + } else if (name.equals("ETag")) { + currentObject.setETag( + ServiceUtils.removeQuotes(getText())); + + } else if (name.equals("Size")) { + currentObject.setSize(parseLong(getText())); + + } else if (name.equals("StorageClass")) { + currentObject.setStorageClass(getText()); + + } else if (name.equals("Owner")) { + currentObject.setOwner(currentOwner); + currentOwner = null; + } + } + + else if (in("ListBucketResult", "Contents", "Owner")) { + if (name.equals("ID")) { + currentOwner.setId(getText()); + + } else if (name.equals("DisplayName")) { + currentOwner.setDisplayName(getText()); + } + } + + else if (in("ListBucketResult", "CommonPrefixes")) { + if (name.equals("Prefix")) { + objectListing.getCommonPrefixes().add + (decodeIfSpecified(getText(), shouldSDKDecodeResponse)); + } + } + } + } + + /** + * Handler for ListObjectsV2 response XML documents. + */ + public static class ListObjectsV2Handler extends AbstractHandler { + private final ListObjectsV2Result result = new ListObjectsV2Result(); + private final boolean shouldSDKDecodeResponse; + + private S3ObjectSummary currentObject = null; + private Owner currentOwner = null; + private String lastKey = null; + + public ListObjectsV2Handler(final boolean shouldSDKDecodeResponse) { + this.shouldSDKDecodeResponse = shouldSDKDecodeResponse; + } + + public ListObjectsV2Result getResult() { + return result; + } + + @Override + protected void doStartElement( + String uri, + String name, + String qName, + Attributes attrs) { + + if (in("ListBucketResult")) { + if (name.equals("Contents")) { + currentObject = new S3ObjectSummary(); + currentObject.setBucketName(result.getBucketName()); + } + } + + else if (in("ListBucketResult", "Contents")) { + if (name.equals("Owner")) { + currentOwner = new Owner(); + } + } + } + + @Override + protected void doEndElement(String uri, String name, String qName) { + if (atTopLevel()) { + if (name.equals("ListBucketResult")) { + /* + * S3 only includes the NextContinuationToken XML element if the + * request specified a delimiter, but for consistency we'd + * like to always give easy access to the next token if + * we're returning a list of results that's truncated. + */ + if (result.isTruncated() + && result.getNextContinuationToken() == null) { + + String nextContinuationToken = null; + if (!result.getObjectSummaries().isEmpty()) { + nextContinuationToken = result.getObjectSummaries() + .get(result.getObjectSummaries().size() - 1) + .getKey(); + + } else { + log.error("S3 response indicates truncated results, " + + "but contains no object summaries."); + } + + result.setNextContinuationToken(nextContinuationToken); + } + } + } + + else if (in("ListBucketResult")) { + if (name.equals("Name")) { + result.setBucketName(getText()); + if (log.isDebugEnabled()) { + log.debug("Examining listing for bucket: " + + result.getBucketName()); + } + + } else if (name.equals("Prefix")) { + result.setPrefix(decodeIfSpecified + (checkForEmptyString(getText()), shouldSDKDecodeResponse)); + + } else if (name.equals("MaxKeys")) { + result.setMaxKeys(parseInt(getText())); + + } else if (name.equals("NextContinuationToken")) { + result.setNextContinuationToken(getText()); + + } else if (name.equals("ContinuationToken")) { + result.setContinuationToken(getText()); + + } else if (name.equals("StartAfter")) { + result.setStartAfter(decodeIfSpecified + (getText(), shouldSDKDecodeResponse)); + + } else if (name.equals("KeyCount")) { + result.setKeyCount(parseInt(getText())); + + } else if (name.equals("Delimiter")) { + result.setDelimiter(decodeIfSpecified + (checkForEmptyString(getText()), shouldSDKDecodeResponse)); + + } else if (name.equals("EncodingType")) { + result.setEncodingType(checkForEmptyString(getText())); + } else if (name.equals("IsTruncated")) { + String isTruncatedStr = + StringUtils.lowerCase(getText()); + + if (isTruncatedStr.startsWith("false")) { + result.setTruncated(false); + } else if (isTruncatedStr.startsWith("true")) { + result.setTruncated(true); + } else { + throw new IllegalStateException( + "Invalid value for IsTruncated field: " + + isTruncatedStr); + } + + } else if (name.equals("Contents")) { + result.getObjectSummaries().add(currentObject); + currentObject = null; + } + } + + else if (in("ListBucketResult", "Contents")) { + if (name.equals("Key")) { + lastKey = getText(); + currentObject.setKey(decodeIfSpecified + (lastKey, shouldSDKDecodeResponse)); + } else if (name.equals("LastModified")) { + currentObject.setLastModified( + ServiceUtils.parseIso8601Date(getText())); + + } else if (name.equals("ETag")) { + currentObject.setETag( + ServiceUtils.removeQuotes(getText())); + + } else if (name.equals("Size")) { + currentObject.setSize(parseLong(getText())); + + } else if (name.equals("StorageClass")) { + currentObject.setStorageClass(getText()); + + } else if (name.equals("Owner")) { + currentObject.setOwner(currentOwner); + currentOwner = null; + } + } + + else if (in("ListBucketResult", "Contents", "Owner")) { + if (name.equals("ID")) { + currentOwner.setId(getText()); + + } else if (name.equals("DisplayName")) { + currentOwner.setDisplayName(getText()); + } + } + + else if (in("ListBucketResult", "CommonPrefixes")) { + if (name.equals("Prefix")) { + result.getCommonPrefixes().add + (decodeIfSpecified(getText(), shouldSDKDecodeResponse)); + } + } + } + } + + /** + * Handler for ListAllMyBuckets response XML documents. The document is + * parsed into {@link Bucket}s available via the {@link #getBuckets()} + * method. + */ + public static class ListAllMyBucketsHandler extends AbstractHandler { + + private final List buckets = new ArrayList(); + private Owner bucketsOwner = null; + + private Bucket currentBucket = null; + + /** + * @return the buckets listed in the document. + */ + public List getBuckets() { + return buckets; + } + + /** + * @return the owner of the buckets. + */ + public Owner getOwner() { + return bucketsOwner; + } + + @Override + protected void doStartElement( + String uri, + String name, + String qName, + Attributes attrs) { + + if (in("ListAllMyBucketsResult")) { + if (name.equals("Owner")) { + bucketsOwner = new Owner(); + } + } else if (in("ListAllMyBucketsResult", "Buckets")) { + if (name.equals("Bucket")) { + currentBucket = new Bucket(); + currentBucket.setOwner(bucketsOwner); + } + } + } + + @Override + protected void doEndElement(String uri, String name, String qName) { + if (in("ListAllMyBucketsResult", "Owner")) { + if (name.equals("ID")) { + bucketsOwner.setId(getText()); + + } else if (name.equals("DisplayName")) { + bucketsOwner.setDisplayName(getText()); + } + } + + else if (in("ListAllMyBucketsResult", "Buckets")) { + if (name.equals("Bucket")) { + buckets.add(currentBucket); + currentBucket = null; + } + } + + else if (in("ListAllMyBucketsResult", "Buckets", "Bucket")) { + if (name.equals("Name")) { + currentBucket.setName(getText()); + + } else if (name.equals("CreationDate")) { + Date creationDate = DateUtils.parseISO8601Date(getText()); + currentBucket.setCreationDate(creationDate); + } + } + } + } + + /** + * Handler for AccessControlList response XML documents. The document is + * parsed into an {@link AccessControlList} object available via the + * {@link #getAccessControlList()} method. + */ + public static class AccessControlListHandler extends AbstractHandler { + + private final AccessControlList accessControlList = + new AccessControlList(); + + private Grantee currentGrantee = null; + private Permission currentPermission = null; + + /** + * @return an object representing the ACL document. + */ + public AccessControlList getAccessControlList() { + return accessControlList; + } + + @Override + protected void doStartElement( + String uri, + String name, + String qName, + Attributes attrs) { + + if (in("AccessControlPolicy")) { + if (name.equals("Owner")) { + accessControlList.setOwner(new Owner()); + + } + } + + else if (in("AccessControlPolicy", "AccessControlList", "Grant")) { + if (name.equals("Grantee")) { + String type = XmlResponsesSaxParser + .findAttributeValue( "xsi:type", attrs ); + + if ("AmazonCustomerByEmail".equals(type)) { + currentGrantee = new EmailAddressGrantee(null); + } else if ("CanonicalUser".equals(type)) { + currentGrantee = new CanonicalGrantee(null); + } else if ("Group".equals(type)) { + /* + * Nothing to do for GroupGrantees here since we + * can't construct an empty enum value early. + */ + } + } + } + } + + @Override + protected void doEndElement(String uri, String name, String qName) { + if (in("AccessControlPolicy", "Owner")) { + if (name.equals("ID")) { + accessControlList.getOwner().setId(getText()); + } else if (name.equals("DisplayName")) { + accessControlList.getOwner().setDisplayName(getText()); + } + } + + else if (in("AccessControlPolicy", "AccessControlList")) { + if (name.equals("Grant")) { + accessControlList.grantPermission( + currentGrantee, currentPermission); + + currentGrantee = null; + currentPermission = null; + } + } + + else if (in("AccessControlPolicy", "AccessControlList", "Grant")) { + if (name.equals("Permission")) { + currentPermission = Permission.parsePermission(getText()); + } + } + + else if (in("AccessControlPolicy", "AccessControlList", "Grant", "Grantee")) { + if (name.equals("ID")) { + currentGrantee.setIdentifier(getText()); + + } else if (name.equals("EmailAddress")) { + currentGrantee.setIdentifier(getText()); + + } else if (name.equals("URI")) { + /* + * Only GroupGrantees contain an URI element in them, and we + * can't construct currentGrantee during startElement for a + * GroupGrantee since it's an enum. + */ + currentGrantee = GroupGrantee.parseGroupGrantee(getText()); + + } else if (name.equals("DisplayName")) { + ((CanonicalGrantee) currentGrantee) + .setDisplayName(getText()); + } + } + } + } + + /** + * Handler for LoggingStatus response XML documents for a bucket. The + * document is parsed into an {@link BucketLoggingConfiguration} object available + * via the {@link #getBucketLoggingConfiguration()} method. + */ + public static class BucketLoggingConfigurationHandler extends AbstractHandler { + + private final BucketLoggingConfiguration bucketLoggingConfiguration = + new BucketLoggingConfiguration(); + + /** + * @return + * an object representing the bucket's LoggingStatus document. + */ + public BucketLoggingConfiguration getBucketLoggingConfiguration() { + return bucketLoggingConfiguration; + } + + @Override + protected void doStartElement( + String uri, + String name, + String qName, + Attributes attrs) { + + } + + @Override + protected void doEndElement(String uri, String name, String qName) { + if (in("BucketLoggingStatus", "LoggingEnabled")) { + if (name.equals("TargetBucket")) { + bucketLoggingConfiguration + .setDestinationBucketName(getText()); + + } else if (name.equals("TargetPrefix")) { + bucketLoggingConfiguration + .setLogFilePrefix(getText()); + } + } + } + } + + /** + * Handler for CreateBucketConfiguration response XML documents for a + * bucket. The document is parsed into a String representing the bucket's + * location, available via the {@link #getLocation()} method. + */ + public static class BucketLocationHandler extends AbstractHandler { + + private String location = null; + + /** + * @return + * the bucket's location. + */ + public String getLocation() { + return location; + } + + @Override + protected void doStartElement( + String uri, + String name, + String qName, + Attributes attrs) { + + } + + @Override + protected void doEndElement(String uri, String name, String qName) { + if (atTopLevel()) { + if (name.equals("LocationConstraint")) { + String elementText = getText(); + if (elementText.length() == 0) { + location = null; + } else { + location = elementText; + } + } + } + } + } + + public static class CopyObjectResultHandler extends AbstractSSEHandler + implements ObjectExpirationResult, S3RequesterChargedResult, S3VersionResult { + + // Data items for successful copy + private final CopyObjectResult result = new CopyObjectResult(); + + // Data items for failed copy + private String errorCode = null; + private String errorMessage = null; + private String errorRequestId = null; + private String errorHostId = null; + private boolean receivedErrorResponse = false; + + @Override + protected ServerSideEncryptionResult sseResult() { + return result; + } + + public Date getLastModified() { + return result.getLastModifiedDate(); + } + + @Override + public String getVersionId() { + return result.getVersionId(); + } + + @Override + public void setVersionId(String versionId) { + result.setVersionId(versionId); + } + + @Override + public Date getExpirationTime() { + return result.getExpirationTime(); + } + + @Override + public void setExpirationTime(Date expirationTime) { + result.setExpirationTime(expirationTime); + } + + @Override + public String getExpirationTimeRuleId() { + return result.getExpirationTimeRuleId(); + } + + @Override + public void setExpirationTimeRuleId(String expirationTimeRuleId) { + result.setExpirationTimeRuleId(expirationTimeRuleId); + } + + public String getETag() { + return result.getETag(); + } + + public String getErrorCode() { + return errorCode; + } + + public String getErrorHostId() { + return errorHostId; + } + + public String getErrorMessage() { + return errorMessage; + } + + public String getErrorRequestId() { + return errorRequestId; + } + + public boolean isErrorResponse() { + return receivedErrorResponse; + } + + public boolean isRequesterCharged() { + return result.isRequesterCharged(); + } + + public void setRequesterCharged(boolean isRequesterCharged) { + result.setRequesterCharged(isRequesterCharged); + } + + + @Override + protected void doStartElement( + String uri, + String name, + String qName, + Attributes attrs) { + + if (atTopLevel()) { + if (name.equals("CopyObjectResult") || name.equals("CopyPartResult")) { + receivedErrorResponse = false; + } else if (name.equals("Error")) { + receivedErrorResponse = true; + } + } + } + + @Override + protected void doEndElement(String uri, String name, String qName) { + if (in("CopyObjectResult") || in ("CopyPartResult")) { + if (name.equals("LastModified")) { + result.setLastModifiedDate(ServiceUtils.parseIso8601Date(getText())); + } else if (name.equals("ETag")) { + result.setETag(ServiceUtils.removeQuotes(getText())); + } + } + + else if (in("Error")) { + if (name.equals("Code")) { + errorCode = getText(); + } else if (name.equals("Message")) { + errorMessage = getText(); + } else if (name.equals("RequestId")) { + errorRequestId = getText(); + } else if (name.equals("HostId")) { + errorHostId = getText(); + } + } + } + } + + /** + * Handler for parsing RequestPaymentConfiguration XML response associated + * with an Amazon S3 bucket. The XML response is parsed into a + * RequestPaymentConfiguration object. + */ + public static class RequestPaymentConfigurationHandler extends AbstractHandler { + + private String payer = null; + + public RequestPaymentConfiguration getConfiguration(){ + return new RequestPaymentConfiguration(Payer.valueOf(payer)); + } + + @Override + protected void doStartElement( + String uri, + String name, + String qName, + Attributes attrs) { + + } + + @Override + protected void doEndElement(String uri, String name, String qName) { + if (in("RequestPaymentConfiguration")) { + if (name.equals("Payer")) { + payer = getText(); + } + } + } + } + + /** + * Handler for ListVersionsResult XML document. + */ + public static class ListVersionsHandler extends AbstractHandler { + + private final VersionListing versionListing = new VersionListing(); + private final boolean shouldSDKDecodeResponse; + + private S3VersionSummary currentVersionSummary; + private Owner currentOwner; + + public ListVersionsHandler(final boolean shouldSDKDecodeResponse) { + this.shouldSDKDecodeResponse = shouldSDKDecodeResponse; + } + + public VersionListing getListing() { + return versionListing; + } + + @Override + protected void doStartElement( + String uri, + String name, + String qName, + Attributes attrs) { + + if (in("ListVersionsResult")) { + if (name.equals("Version")) { + currentVersionSummary = new S3VersionSummary(); + currentVersionSummary.setBucketName( + versionListing.getBucketName()); + + } else if (name.equals("DeleteMarker")) { + currentVersionSummary = new S3VersionSummary(); + currentVersionSummary.setBucketName( + versionListing.getBucketName()); + currentVersionSummary.setIsDeleteMarker(true); + } + } + + else if (in("ListVersionsResult", "Version") + || in("ListVersionsResult", "DeleteMarker")) { + if (name.equals("Owner")) { + currentOwner = new Owner(); + } + } + } + + @Override + protected void doEndElement( + String uri, + String name, + String qName) { + + if (in("ListVersionsResult")) { + if (name.equals("Name")) { + versionListing.setBucketName(getText()); + + } else if (name.equals("Prefix")) { + versionListing.setPrefix(decodeIfSpecified + (checkForEmptyString(getText()), shouldSDKDecodeResponse)); + } else if (name.equals("KeyMarker")) { + versionListing.setKeyMarker(decodeIfSpecified + (checkForEmptyString(getText()), shouldSDKDecodeResponse)); + } else if (name.equals("VersionIdMarker")) { + versionListing.setVersionIdMarker(checkForEmptyString( + getText())); + + } else if (name.equals("MaxKeys")) { + versionListing.setMaxKeys(Integer.parseInt(getText())); + + } else if (name.equals("Delimiter")) { + versionListing.setDelimiter(decodeIfSpecified + (checkForEmptyString(getText()), shouldSDKDecodeResponse)); + + } else if (name.equals("EncodingType")) { + versionListing.setEncodingType(shouldSDKDecodeResponse ? + null : checkForEmptyString(getText())); + } else if (name.equals("NextKeyMarker")) { + versionListing.setNextKeyMarker(decodeIfSpecified + (checkForEmptyString(getText()), shouldSDKDecodeResponse)); + + } else if (name.equals("NextVersionIdMarker")) { + versionListing.setNextVersionIdMarker(getText()); + + } else if (name.equals("IsTruncated")) { + versionListing.setTruncated("true".equals(getText())); + + } else if (name.equals("Version") + || name.equals("DeleteMarker")) { + + versionListing.getVersionSummaries() + .add(currentVersionSummary); + + currentVersionSummary = null; + } + } + + else if (in("ListVersionsResult", "CommonPrefixes")) { + if (name.equals("Prefix")) { + final String commonPrefix = checkForEmptyString(getText()); + versionListing.getCommonPrefixes() + .add(shouldSDKDecodeResponse ? + SdkHttpUtils.urlDecode(commonPrefix) : commonPrefix); + } + } + + else if (in("ListVersionsResult", "Version") + || in("ListVersionsResult", "DeleteMarker")) { + + if (name.equals("Key")) { + currentVersionSummary.setKey(decodeIfSpecified(getText(), shouldSDKDecodeResponse)); + + } else if (name.equals("VersionId")) { + currentVersionSummary.setVersionId(getText()); + + } else if (name.equals("IsLatest")) { + currentVersionSummary.setIsLatest("true".equals(getText())); + + } else if (name.equals("LastModified")) { + currentVersionSummary.setLastModified( + ServiceUtils.parseIso8601Date(getText())); + + } else if (name.equals("ETag")) { + currentVersionSummary.setETag( + ServiceUtils.removeQuotes(getText())); + + } else if (name.equals("Size")) { + currentVersionSummary.setSize(Long.parseLong(getText())); + + } else if (name.equals("Owner")) { + currentVersionSummary.setOwner(currentOwner); + currentOwner = null; + + } else if (name.equals("StorageClass")) { + currentVersionSummary.setStorageClass(getText()); + } + } + + else if (in("ListVersionsResult", "Version", "Owner") + || in("ListVersionsResult", "DeleteMarker", "Owner")) { + + if (name.equals("ID")) { + currentOwner.setId(getText()); + } else if (name.equals("DisplayName")) { + currentOwner.setDisplayName(getText()); + } + } + } + } + + public static class BucketWebsiteConfigurationHandler extends AbstractHandler { + + private final BucketWebsiteConfiguration configuration = + new BucketWebsiteConfiguration(null); + + private RoutingRuleCondition currentCondition = null; + private RedirectRule currentRedirectRule = null; + private RoutingRule currentRoutingRule = null; + + public BucketWebsiteConfiguration getConfiguration() { + return configuration; + } + + @Override + protected void doStartElement( + String uri, + String name, + String qName, + Attributes attrs) { + + if (in("WebsiteConfiguration")) { + if (name.equals("RedirectAllRequestsTo")) { + currentRedirectRule = new RedirectRule(); + } + } + + else if (in("WebsiteConfiguration", "RoutingRules")) { + if (name.equals("RoutingRule")) { + currentRoutingRule = new RoutingRule(); + } + } + + else if (in("WebsiteConfiguration", "RoutingRules", "RoutingRule")) { + if (name.equals("Condition")) { + currentCondition = new RoutingRuleCondition(); + } else if (name.equals("Redirect")) { + currentRedirectRule = new RedirectRule(); + } + } + } + + @Override + protected void doEndElement(String uri, String name, String qName) { + if (in("WebsiteConfiguration")) { + if (name.equals("RedirectAllRequestsTo")) { + configuration.setRedirectAllRequestsTo(currentRedirectRule); + currentRedirectRule = null; + } + } + + else if (in("WebsiteConfiguration", "IndexDocument")) { + if (name.equals("Suffix")) { + configuration.setIndexDocumentSuffix(getText()); + } + } + + else if (in("WebsiteConfiguration", "ErrorDocument")) { + if (name.equals("Key")) { + configuration.setErrorDocument(getText()); + } + } + + else if (in("WebsiteConfiguration", "RoutingRules")) { + if (name.equals("RoutingRule")) { + configuration.getRoutingRules().add(currentRoutingRule); + currentRoutingRule = null; + } + } + + else if (in("WebsiteConfiguration", "RoutingRules", "RoutingRule")) { + if (name.equals("Condition")) { + currentRoutingRule.setCondition(currentCondition); + currentCondition = null; + } else if (name.equals("Redirect")) { + currentRoutingRule.setRedirect(currentRedirectRule); + currentRedirectRule = null; + } + } + + else if (in("WebsiteConfiguration", "RoutingRules", "RoutingRule", "Condition")) { + if (name.equals("KeyPrefixEquals")) { + currentCondition.setKeyPrefixEquals(getText()); + } else if (name.equals("HttpErrorCodeReturnedEquals")) { + currentCondition.setHttpErrorCodeReturnedEquals(getText()); + } + } + + else if (in("WebsiteConfiguration", "RedirectAllRequestsTo") + || in("WebsiteConfiguration", "RoutingRules", "RoutingRule", "Redirect")) { + + if (name.equals("Protocol")) { + currentRedirectRule.setProtocol(getText()); + + } else if (name.equals("HostName")) { + currentRedirectRule.setHostName(getText()); + + } else if (name.equals("ReplaceKeyPrefixWith")) { + currentRedirectRule.setReplaceKeyPrefixWith(getText()); + + } else if (name.equals("ReplaceKeyWith")) { + currentRedirectRule.setReplaceKeyWith(getText()); + + } else if (name.equals("HttpRedirectCode")) { + currentRedirectRule.setHttpRedirectCode(getText()); + } + } + } + } + + public static class BucketVersioningConfigurationHandler extends AbstractHandler { + + private final BucketVersioningConfiguration configuration = + new BucketVersioningConfiguration(); + + public BucketVersioningConfiguration getConfiguration() { return configuration; } + + @Override + protected void doStartElement( + String uri, + String name, + String qName, + Attributes attrs) { + + } + + @Override + protected void doEndElement(String uri, String name, String qName) { + if (in("VersioningConfiguration")) { + if (name.equals("Status")) { + configuration.setStatus(getText()); + + } else if (name.equals("MfaDelete")) { + String mfaDeleteStatus = getText(); + + if (mfaDeleteStatus.equals("Disabled")) { + configuration.setMfaDeleteEnabled(false); + } else if (mfaDeleteStatus.equals("Enabled")) { + configuration.setMfaDeleteEnabled(true); + } else { + configuration.setMfaDeleteEnabled(null); + } + } + } + } + } + + public static class BucketAccelerateConfigurationHandler extends AbstractHandler { + + private final BucketAccelerateConfiguration configuration = new BucketAccelerateConfiguration((String) null); + + public BucketAccelerateConfiguration getConfiguration() { return configuration; } + + @Override + protected void doStartElement( + String uri, + String name, + String qName, + Attributes attrs) { + + } + + @Override + protected void doEndElement(String uri, String name, String qName) { + if (in("AccelerateConfiguration")) { + if (name.equals("Status")) { + configuration.setStatus(getText()); + } + } + } + } + + /* + * + * + * http://Example-Bucket.s3.amazonaws.com/Example-Object + * Example-Bucket + * Example-Object + * "3858f62230ac3c915f300c664312c11f-9" + * + * + * Or if an error occurred while completing: + * + * + * + * InternalError + * We encountered an internal error. Please try again. + * 656c76696e6727732072657175657374 + * Uuag1LuByRx9e6j5Onimru9pO4ZVKnJ2Qz7/C1NPcfTWAtRPfTaOFg== + * + */ + public static class CompleteMultipartUploadHandler extends AbstractSSEHandler + implements ObjectExpirationResult, S3VersionResult, S3RequesterChargedResult { + // Successful completion + private CompleteMultipartUploadResult result; + + // Error during completion + private AmazonS3Exception ase; + private String hostId; + private String requestId; + private String errorCode; + + @Override + protected ServerSideEncryptionResult sseResult() { + return result; + } + /** + * @see com.amazonaws.services.s3.model.CompleteMultipartUploadResult#getExpirationTime() + */ + @Override + public Date getExpirationTime() { + return result == null ? null : result.getExpirationTime(); + } + + /** + * @see com.amazonaws.services.s3.model.CompleteMultipartUploadResult#setExpirationTime(java.util.Date) + */ + @Override + public void setExpirationTime(Date expirationTime) { + if (result != null) { + result.setExpirationTime(expirationTime); + } + } + + /** + * @see com.amazonaws.services.s3.model.CompleteMultipartUploadResult#getExpirationTimeRuleId() + */ + @Override + public String getExpirationTimeRuleId() { + return result == null ? null : result.getExpirationTimeRuleId(); + } + + /** + * @see com.amazonaws.services.s3.model.CompleteMultipartUploadResult#setExpirationTimeRuleId(java.lang.String) + */ + @Override + public void setExpirationTimeRuleId(String expirationTimeRuleId) { + if (result != null) { + result.setExpirationTimeRuleId(expirationTimeRuleId); + } + } + + @Override + public void setVersionId(String versionId) { + if (result != null) { + result.setVersionId(versionId); + } + } + + @Override + public String getVersionId() { + return result == null ? null : result.getVersionId(); + } + + /** + * @see com.amazonaws.services.s3.model.CompleteMultipartUploadResult#isRequesterCharged() + */ + public boolean isRequesterCharged() { + return result == null ? false : result.isRequesterCharged(); + } + + /** + * @see com.amazonaws.services.s3.model.CompleteMultipartUploadResult#setRequesterCharged(boolean) + */ + public void setRequesterCharged(boolean isRequesterCharged) { + if (result != null) { + result.setRequesterCharged(isRequesterCharged); + } + } + + public CompleteMultipartUploadResult getCompleteMultipartUploadResult() { + return result; + } + + public AmazonS3Exception getAmazonS3Exception() { + return ase; + } + + @Override + protected void doStartElement( + String uri, + String name, + String qName, + Attributes attrs) { + + if (atTopLevel()) { + if (name.equals("CompleteMultipartUploadResult")) { + result = new CompleteMultipartUploadResult(); + } + } + } + + @Override + protected void doEndElement(String uri, String name, String qName) { + if (atTopLevel()) { + if (name.equals("Error")) { + if (ase != null) { + ase.setErrorCode(errorCode); + ase.setRequestId(requestId); + ase.setExtendedRequestId(hostId); + } + } + } + + else if (in("CompleteMultipartUploadResult")) { + if (name.equals("Location")) { + result.setLocation(getText()); + } else if (name.equals("Bucket")) { + result.setBucketName(getText()); + } else if (name.equals("Key")) { + result.setKey(getText()); + } else if (name.equals("ETag")) { + result.setETag(ServiceUtils.removeQuotes(getText())); + } + } + + else if (in("Error")) { + if (name.equals("Code")) { + errorCode = getText(); + } else if (name.equals("Message")) { + ase = new AmazonS3Exception(getText()); + } else if (name.equals("RequestId")) { + requestId = getText(); + } else if (name.equals("HostId")) { + hostId = getText(); + } + } + } + } + + /* + * + * + * example-bucket + * example-object + * VXBsb2FkIElEIGZvciA2aWWpbmcncyBteS1tb3ZpZS5tMnRzIHVwbG9hZA + * + */ + public static class InitiateMultipartUploadHandler extends AbstractHandler { + + private final InitiateMultipartUploadResult result = + new InitiateMultipartUploadResult(); + + public InitiateMultipartUploadResult getInitiateMultipartUploadResult() { + return result; + } + + @Override + protected void doStartElement( + String uri, + String name, + String qName, + Attributes attrs) { + + } + + @Override + protected void doEndElement(String uri, String name, String qName) { + if (in("InitiateMultipartUploadResult")) { + if (name.equals("Bucket")) { + result.setBucketName(getText()); + + } else if (name.equals("Key")) { + result.setKey(getText()); + + } else if (name.equals("UploadId")) { + result.setUploadId(getText()); + } + } + } + } + + /* + * HTTP/1.1 200 OK + * x-amz-id-2: Uuag1LuByRx9e6j5Onimru9pO4ZVKnJ2Qz7/C1NPcfTWAtRPfTaOFg== + * x-amz-request-id: 656c76696e6727732072657175657374 + * Date: Tue, 16 Feb 2010 20:34:56 GMT + * Content-Length: 1330 + * Connection: keep-alive + * Server: AmazonS3 + * + * + * + * bucket + * + * / + * + * + * my-movie.m2ts + * YW55IGlkZWEgd2h5IGVsdmluZydzIHVwbG9hZCBmYWlsZWQ + * 3 + * true + * + * my-divisor + * XMgbGlrZSBlbHZpbmcncyBub3QgaGF2aW5nIG11Y2ggbHVjaw + * + * b1d16700c70b0b05597d7acd6a3f92be + * delving + * + * STANDARD + * Tue, 26 Jan 2010 19:42:19 GMT + * + * + * my-movie.m2ts + * VXBsb2FkIElEIGZvciBlbHZpbmcncyBteS1tb3ZpZS5tMnRzIHVwbG9hZA + * + * b1d16700c70b0b05597d7acd6a3f92be + * delving + * + * STANDARD + * Tue, 16 Feb 2010 20:34:56 GMT + * + * + * my-movie.m2ts + * YW55IGlkZWEgd2h5IGVsdmluZydzIHVwbG9hZCBmYWlsZWQ + * + * b1d16700c70b0b05597d7acd6a3f92be + * delving + * + * STANDARD + * Wed, 27 Jan 2010 03:02:01 GMT + * + * + * photos/ + * + * + * videos/ + * + * + */ + public static class ListMultipartUploadsHandler extends AbstractHandler { + + private final MultipartUploadListing result = + new MultipartUploadListing(); + + private MultipartUpload currentMultipartUpload; + private Owner currentOwner; + + public MultipartUploadListing getListMultipartUploadsResult() { + return result; + } + + @Override + protected void doStartElement( + String uri, + String name, + String qName, + Attributes attrs) { + + if (in("ListMultipartUploadsResult")) { + if (name.equals("Upload")) { + currentMultipartUpload = new MultipartUpload(); + } + } else if (in("ListMultipartUploadsResult", "Upload")) { + if (name.equals("Owner") || name.equals("Initiator")) { + currentOwner = new Owner(); + } + } + } + + @Override + protected void doEndElement(String uri, String name, String qName) { + if (in("ListMultipartUploadsResult")) { + if (name.equals("Bucket")) { + result.setBucketName(getText()); + } else if (name.equals("KeyMarker")) { + result.setKeyMarker(checkForEmptyString(getText())); + } else if (name.equals("Delimiter")) { + result.setDelimiter(checkForEmptyString(getText())); + } else if (name.equals("Prefix")) { + result.setPrefix(checkForEmptyString(getText())); + } else if (name.equals("UploadIdMarker")) { + result.setUploadIdMarker(checkForEmptyString(getText())); + } else if (name.equals("NextKeyMarker")) { + result.setNextKeyMarker(checkForEmptyString(getText())); + } else if (name.equals("NextUploadIdMarker")) { + result.setNextUploadIdMarker(checkForEmptyString(getText())); + } else if (name.equals("MaxUploads")) { + result.setMaxUploads(Integer.parseInt(getText())); + } else if (name.equals("EncodingType")) { + result.setEncodingType(checkForEmptyString(getText())); + } else if (name.equals("IsTruncated")) { + result.setTruncated(Boolean.parseBoolean(getText())); + } else if (name.equals("Upload")) { + result.getMultipartUploads().add(currentMultipartUpload); + currentMultipartUpload = null; + } + } + + else if (in("ListMultipartUploadsResult", "CommonPrefixes")) { + if (name.equals("Prefix")) { + result.getCommonPrefixes().add(getText()); + } + } + + else if (in("ListMultipartUploadsResult", "Upload")) { + if (name.equals("Key")) { + currentMultipartUpload.setKey(getText()); + } else if (name.equals("UploadId")) { + currentMultipartUpload.setUploadId(getText()); + } else if (name.equals("Owner")) { + currentMultipartUpload.setOwner(currentOwner); + currentOwner = null; + } else if (name.equals("Initiator")) { + currentMultipartUpload.setInitiator(currentOwner); + currentOwner = null; + } else if (name.equals("StorageClass")) { + currentMultipartUpload.setStorageClass(getText()); + } else if (name.equals("Initiated")) { + currentMultipartUpload.setInitiated( + ServiceUtils.parseIso8601Date(getText())); + } + } + + else if (in("ListMultipartUploadsResult", "Upload", "Owner") + || in("ListMultipartUploadsResult", "Upload", "Initiator")) { + + if (name.equals("ID")) { + currentOwner.setId(checkForEmptyString(getText())); + } else if (name.equals("DisplayName")) { + currentOwner.setDisplayName(checkForEmptyString(getText())); + } + } + } + } + + /* + * HTTP/1.1 200 OK + * x-amz-id-2: Uuag1LuByRx9e6j5Onimru9pO4ZVKnJ2Qz7/C1NPcfTWAtRPfTaOFg== + * x-amz-request-id: 656c76696e6727732072657175657374 + * Date: Tue, 16 Feb 2010 20:34:56 GMT + * Content-Length: 985 + * Connection: keep-alive + * Server: AmazonS3 + * + * + * + * example-bucket + * example-object + * XXBsb2FkIElEIGZvciBlbHZpbmcncyVcdS1tb3ZpZS5tMnRzEEEwbG9hZA + * + * x1x16700c70b0b05597d7ecd6a3f92be + * username + * + * + * x1x16700c70b0b05597d7ecd6a3f92be + * username + * + * STANDARD + * 1 + * 3 + * 2 + * true + * + * 2 + * Wed, 27 Jan 2010 03:02:03 GMT + * "7778aef83f66abc1fa1e8477f296d394" + * 10485760 + * + * + * 3 + * Wed, 27 Jan 2010 03:02:02 GMT + * "aaaa18db4cc2f85cedef654fccc4a4x8" + * 10485760 + * + * + */ + public static class ListPartsHandler extends AbstractHandler { + + private final PartListing result = new PartListing(); + + private PartSummary currentPart; + private Owner currentOwner; + + public PartListing getListPartsResult() { + return result; + } + + @Override + protected void doStartElement( + String uri, + String name, + String qName, + Attributes attrs) { + + if (in("ListPartsResult")) { + if (name.equals("Part")) { + currentPart = new PartSummary(); + } else if (name.equals("Owner") || name.equals("Initiator")) { + currentOwner = new Owner(); + } + } + } + + @Override + protected void doEndElement(String uri, String name, String qName) { + if (in("ListPartsResult")) { + if (name.equals("Bucket")) { + result.setBucketName(getText()); + } else if (name.equals("Key")) { + result.setKey(getText()); + } else if (name.equals("UploadId")) { + result.setUploadId(getText()); + } else if (name.equals("Owner")) { + result.setOwner(currentOwner); + currentOwner = null; + } else if (name.equals("Initiator")) { + result.setInitiator(currentOwner); + currentOwner = null; + } else if (name.equals("StorageClass")) { + result.setStorageClass(getText()); + } else if (name.equals("PartNumberMarker")) { + result.setPartNumberMarker(parseInteger(getText())); + } else if (name.equals("NextPartNumberMarker")) { + result.setNextPartNumberMarker(parseInteger(getText())); + } else if (name.equals("MaxParts")) { + result.setMaxParts(parseInteger(getText())); + } else if (name.equals("EncodingType")) { + result.setEncodingType(checkForEmptyString(getText())); + } else if (name.equals("IsTruncated")) { + result.setTruncated(Boolean.parseBoolean(getText())); + } else if (name.equals("Part")) { + result.getParts().add(currentPart); + currentPart = null; + } + } + + else if (in("ListPartsResult", "Part")) { + if (name.equals("PartNumber")) { + currentPart.setPartNumber(Integer.parseInt(getText())); + } else if (name.equals("LastModified")) { + currentPart.setLastModified( + ServiceUtils.parseIso8601Date(getText())); + } else if (name.equals("ETag")) { + currentPart.setETag(ServiceUtils.removeQuotes(getText())); + } else if (name.equals("Size")) { + currentPart.setSize(Long.parseLong(getText())); + } + } + + else if (in("ListPartsResult", "Owner") + || in("ListPartsResult", "Initiator")) { + + if (name.equals("ID")) { + currentOwner.setId(checkForEmptyString(getText())); + } else if (name.equals("DisplayName")) { + currentOwner.setDisplayName(checkForEmptyString(getText())); + } + } + } + + private Integer parseInteger(String text) { + text = checkForEmptyString(getText()); + if (text == null) return null; + return Integer.parseInt(text); + } + } + + /** + * Handler for parsing the get replication configuration response from + * Amazon S3. Sample HTTP response is given below. + * + *

    +     * 
    +     * 	
    +     *   	replication-rule-1-1421862858808
    +     *   	testPrefix1
    +     *   	Enabled
    +     *   	
    +     *       	bucketARN
    +     *   	
    +     *	
    +     *	
    +     *   	replication-rule-2-1421862858808
    +     *   	testPrefix2
    +     *   	Disabled
    +     *   	
    +     *       	arn:aws:s3:::bucket-dest-replication-integ-test-1421862858808
    +     *   	
    +     *	
    +     * 
    +     * 
    + */ + public static class BucketReplicationConfigurationHandler extends + AbstractHandler { + + private final BucketReplicationConfiguration bucketReplicationConfiguration = new BucketReplicationConfiguration(); + private String currentRuleId; + private ReplicationRule currentRule; + private ReplicationDestinationConfig destinationConfig; + private AccessControlTranslation accessControlTranslation; + private EncryptionConfiguration encryptionConfiguration; + private SourceSelectionCriteria sourceSelectionCriteria; + private SseKmsEncryptedObjects sseKmsEncryptedObjects; + private static final String REPLICATION_CONFIG = "ReplicationConfiguration"; + private static final String ROLE = "Role"; + private static final String RULE = "Rule"; + private static final String DESTINATION = "Destination"; + private static final String ID = "ID"; + private static final String PREFIX = "Prefix"; + private static final String STATUS = "Status"; + private static final String BUCKET = "Bucket"; + private static final String STORAGECLASS = "StorageClass"; + private static final String ACCOUNT = "Account"; + private static final String ACCESS_CONTROL_TRANSLATION = "AccessControlTranslation"; + private static final String OWNER = "Owner"; + private static final String ENCRYPTION_CONFIGURATION = "EncryptionConfiguration"; + private static final String REPLICA_KMS_KEY_ID = "ReplicaKmsKeyID"; + private static final String SOURCE_SELECTION_CRITERIA = "SourceSelectionCriteria"; + private static final String SSE_KMS_ENCRYPTED_OBJECTS = "SseKmsEncryptedObjects"; + + public BucketReplicationConfiguration getConfiguration() { + return bucketReplicationConfiguration; + } + + @Override + protected void doStartElement(String uri, String name, String qName, + Attributes attrs) { + + if (in(REPLICATION_CONFIG)) { + if (name.equals(RULE)) { + currentRule = new ReplicationRule(); + } + } else if (in(REPLICATION_CONFIG, RULE)) { + if (name.equals(DESTINATION)) { + destinationConfig = new ReplicationDestinationConfig(); + } else if (name.equals(SOURCE_SELECTION_CRITERIA)) { + sourceSelectionCriteria = new SourceSelectionCriteria(); + } + } else if (in(REPLICATION_CONFIG, RULE, DESTINATION)) { + if (name.equals(ACCESS_CONTROL_TRANSLATION)) { + accessControlTranslation = new AccessControlTranslation(); + } else if (name.equals(ENCRYPTION_CONFIGURATION)) { + encryptionConfiguration = new EncryptionConfiguration(); + } + } else if (in(REPLICATION_CONFIG, RULE, SOURCE_SELECTION_CRITERIA)) { + if (name.equals(SSE_KMS_ENCRYPTED_OBJECTS)) { + sseKmsEncryptedObjects = new SseKmsEncryptedObjects(); + } + } + } + + @Override + protected void doEndElement(String uri, String name, String qName) { + if (in(REPLICATION_CONFIG)) { + if (name.equals(RULE)) { + bucketReplicationConfiguration.addRule(currentRuleId, + currentRule); + currentRule = null; + currentRuleId = null; + destinationConfig = null; + sseKmsEncryptedObjects = null; + accessControlTranslation = null; + encryptionConfiguration = null; + } else if (name.equals(ROLE)) { + bucketReplicationConfiguration.setRoleARN(getText()); + } + } else if (in(REPLICATION_CONFIG, RULE)) { + if (name.equals(ID)) { + currentRuleId = getText(); + } else if (name.equals(PREFIX)) { + currentRule.setPrefix(getText()); + } else if (name.equals(SOURCE_SELECTION_CRITERIA)) { + currentRule.setSourceSelectionCriteria(sourceSelectionCriteria); + } else { + if (name.equals(STATUS)) { + currentRule.setStatus(getText()); + + } else if (name.equals(DESTINATION)) { + currentRule.setDestinationConfig(destinationConfig); + } + } + } else if (in(REPLICATION_CONFIG, RULE, SOURCE_SELECTION_CRITERIA)) { + if (name.equals(SSE_KMS_ENCRYPTED_OBJECTS)) { + sourceSelectionCriteria.setSseKmsEncryptedObjects(sseKmsEncryptedObjects); + } + } else if (in(REPLICATION_CONFIG, RULE, SOURCE_SELECTION_CRITERIA, SSE_KMS_ENCRYPTED_OBJECTS)) { + if (name.equals(STATUS)) { + sseKmsEncryptedObjects.setStatus(getText()); + } + } else if (in(REPLICATION_CONFIG, RULE, DESTINATION)) { + if (name.equals(BUCKET)) { + destinationConfig.setBucketARN(getText()); + } else if (name.equals(STORAGECLASS)) { + destinationConfig.setStorageClass(getText()); + } else if (name.equals(ACCOUNT)) { + destinationConfig.setAccount(getText()); + } else if (name.equals(ACCESS_CONTROL_TRANSLATION)) { + destinationConfig.setAccessControlTranslation(accessControlTranslation); + } else if (name.equals(ENCRYPTION_CONFIGURATION)) { + destinationConfig.setEncryptionConfiguration(encryptionConfiguration); + } + } else if (in(REPLICATION_CONFIG, RULE, DESTINATION, ACCESS_CONTROL_TRANSLATION)) { + if (name.equals(OWNER)) { + accessControlTranslation.setOwner(getText()); + } + } else if (in(REPLICATION_CONFIG, RULE, DESTINATION, ENCRYPTION_CONFIGURATION)) { + if (name.equals(REPLICA_KMS_KEY_ID)) { + encryptionConfiguration.setReplicaKmsKeyID(getText()); + } + } + } + } + + public static class BucketTaggingConfigurationHandler extends AbstractHandler { + + private final BucketTaggingConfiguration configuration = + new BucketTaggingConfiguration(); + + private Map currentTagSet; + private String currentTagKey; + private String currentTagValue; + + public BucketTaggingConfiguration getConfiguration() { + return configuration; + } + + @Override + protected void doStartElement( + String uri, + String name, + String qName, + Attributes attrs) { + + if (in("Tagging")) { + if (name.equals("TagSet")) { + currentTagSet = new HashMap(); + } + } + } + + @Override + protected void doEndElement(String uri, String name, String qName) { + if (in("Tagging")) { + if (name.equals("TagSet")) { + configuration.getAllTagSets() + .add(new TagSet(currentTagSet)); + currentTagSet = null; + } + } + + else if (in("Tagging", "TagSet")) { + if (name.equals("Tag")) { + if (currentTagKey != null && currentTagValue != null) { + currentTagSet.put(currentTagKey, currentTagValue); + } + currentTagKey = null; + currentTagValue = null; + } + } + + else if (in("Tagging", "TagSet", "Tag")) { + if (name.equals("Key")) { + currentTagKey = getText(); + } else if (name.equals("Value")) { + currentTagValue = getText(); + } + } + } + } + + /** + * Handler for unmarshalling the response from GET Object Tagging. + * + * + * + * + * Foo + * 1 + * + * + * Bar + * 2 + * + * + * Baz + * 3 + * + * + * + */ + public static class GetObjectTaggingHandler extends AbstractHandler { + private GetObjectTaggingResult getObjectTaggingResult; + private List tagSet; + private String currentTagValue; + private String currentTagKey; + + public GetObjectTaggingResult getResult() { + return getObjectTaggingResult; + } + + + @Override + protected void doStartElement(String uri, String name, String qName, Attributes attrs) { + if (in("Tagging")) { + if (name.equals("TagSet")) { + tagSet = new ArrayList(); + } + } + } + + @Override + protected void doEndElement(String uri, String name, String qName) { + if (in("Tagging")) { + if (name.equals("TagSet")) { + getObjectTaggingResult = new GetObjectTaggingResult(tagSet); + tagSet = null; + } + } + if (in("Tagging", "TagSet")) { + if (name.equals("Tag")) { + tagSet.add(new Tag(currentTagKey, currentTagValue)); + currentTagKey = null; + currentTagValue = null; + } + } else if (in("Tagging", "TagSet", "Tag")) { + if (name.equals("Key")) { + currentTagKey = getText(); + } else if (name.equals("Value")) { + currentTagValue = getText(); + } + } + } + } + + /* + HTTP/1.1 200 OK + x-amz-id-2: Uuag1LuByRx9e6j5Onimru9pO4ZVKnJ2Qz7/C1NPcfTWAtRPfTaOFg== + x-amz-request-id: 656c76696e6727732072657175657374 + Date: Tue, 20 Sep 2012 20:34:56 GMT + Content-Type: application/xml + Transfer-Encoding: chunked + Connection: keep-alive + Server: AmazonS3 + + + + + Key + Version + + + Key + Version + Code + Message + + + Key + true + Version + + + */ + public static class DeleteObjectsHandler extends AbstractHandler { + + private final DeleteObjectsResponse response = + new DeleteObjectsResponse(); + + private DeletedObject currentDeletedObject = null; + private DeleteError currentError = null; + + public DeleteObjectsResponse getDeleteObjectResult() { + return response; + } + + @Override + protected void doStartElement( + String uri, + String name, + String qName, + Attributes attrs) { + + if (in("DeleteResult")) { + if (name.equals("Deleted")) { + currentDeletedObject = new DeletedObject(); + } else if (name.equals("Error")) { + currentError = new DeleteError(); + } + } + } + + @Override + protected void doEndElement(String uri, String name, String qName) { + if (in("DeleteResult")) { + if (name.equals("Deleted")) { + response.getDeletedObjects().add(currentDeletedObject); + currentDeletedObject = null; + } else if (name.equals("Error")) { + response.getErrors().add(currentError); + currentError = null; + } + } + + else if (in("DeleteResult", "Deleted")) { + if (name.equals("Key")) { + currentDeletedObject.setKey(getText()); + + } else if (name.equals("VersionId")) { + currentDeletedObject.setVersionId(getText()); + + } else if (name.equals("DeleteMarker")) { + currentDeletedObject.setDeleteMarker( + getText().equals("true")); + + } else if (name.equals("DeleteMarkerVersionId")) { + currentDeletedObject.setDeleteMarkerVersionId(getText()); + } + } + + else if (in("DeleteResult", "Error")) { + if (name.equals("Key")) { + currentError.setKey(getText()); + + } else if (name.equals("VersionId")) { + currentError.setVersionId(getText()); + + } else if (name.equals("Code")) { + currentError.setCode(getText()); + + } else if (name.equals("Message")) { + currentError.setMessage(getText()); + } + } + } + } + + /** + * HTTP/1.1 200 OK + x-amz-id-2: Uuag1LuByRx9e6j5Onimru9pO4ZVKnJ2Qz7/C1NPcfTWAtRPfTaOFg== + x-amz-request-id: 656c76696e6727732072657175657374 + Date: Tue, 20 Sep 2012 20:34:56 GMT + Content-Length: xxx + Connection: keep-alive + Server: AmazonS3 + + + + logs-rule + logs/ + Enabled + + logs/ + + key1 + value1 + + + logs/ + + key1 + value1 + + + key1 + value1 + + + + + 30 + STANDARD_IA + + + 90 + GLACIER + + + 365 + + + 7 + STANDARD_IA + + + 14 + GLACIER + + + 365 + + + + image-rule + image/ + Enabled + + 2012-12-31T00:00:00.000Z + GLACIER + + + 2020-12-31T00:00:00.000Z + + + 10 + + + + */ + public static class BucketLifecycleConfigurationHandler extends AbstractHandler { + + private final BucketLifecycleConfiguration configuration = + new BucketLifecycleConfiguration(new ArrayList()); + + private Rule currentRule; + private Transition currentTransition; + private NoncurrentVersionTransition currentNcvTransition; + private AbortIncompleteMultipartUpload abortIncompleteMultipartUpload; + private LifecycleFilter currentFilter; + private List andOperandsList; + private String currentTagKey; + private String currentTagValue; + + public BucketLifecycleConfiguration getConfiguration() { + return configuration; + } + + @Override + protected void doStartElement( + String uri, + String name, + String qName, + Attributes attrs) { + + if (in("LifecycleConfiguration")) { + if (name.equals("Rule")) { + currentRule = new Rule(); + } + } else if (in("LifecycleConfiguration", "Rule")) { + if (name.equals("Transition")) { + currentTransition = new Transition(); + } else if (name.equals("NoncurrentVersionTransition")) { + currentNcvTransition = new NoncurrentVersionTransition(); + } else if (name.equals("AbortIncompleteMultipartUpload")) { + abortIncompleteMultipartUpload = new + AbortIncompleteMultipartUpload(); + } else if (name.equals("Filter")) { + currentFilter = new LifecycleFilter(); + } + } else if (in("LifecycleConfiguration", "Rule", "Filter")) { + if (name.equals("And")) { + andOperandsList = new ArrayList(); + } + } + } + + @Override + protected void doEndElement(String uri, String name, String qName) { + if (in("LifecycleConfiguration")) { + if (name.equals("Rule")) { + configuration.getRules().add(currentRule); + currentRule = null; + } + } + + else if (in("LifecycleConfiguration", "Rule")) { + if ( name.equals("ID") ) { + currentRule.setId(getText()); + + } else if ( name.equals("Prefix") ) { + currentRule.setPrefix(getText()); + + } else if ( name.equals("Status") ) { + currentRule.setStatus(getText()); + + } else if (name.equals("Transition")) { + currentRule.addTransition(currentTransition); + currentTransition = null; + + } else if (name.equals("NoncurrentVersionTransition")) { + currentRule.addNoncurrentVersionTransition( + currentNcvTransition); + currentNcvTransition = null; + } else if (name.equals("AbortIncompleteMultipartUpload")) { + currentRule.setAbortIncompleteMultipartUpload(abortIncompleteMultipartUpload); + abortIncompleteMultipartUpload = null; + } else if (name.equals("Filter")) { + currentRule.setFilter(currentFilter); + currentFilter = null; + } + } + + else if (in("LifecycleConfiguration", "Rule", "Expiration")) { + if (name.equals("Date")) { + currentRule.setExpirationDate(ServiceUtils.parseIso8601Date(getText())); + } else if (name.equals("Days")) { + currentRule.setExpirationInDays(Integer.parseInt(getText())); + } else if (name.equals("ExpiredObjectDeleteMarker")) { + if ("true".equals(getText())) { + currentRule.setExpiredObjectDeleteMarker(true); + } + } + } + + else if (in("LifecycleConfiguration", "Rule", "Transition")) { + if (name.equals("StorageClass")) { + currentTransition.setStorageClass(getText()); + } else if (name.equals("Date")) { + currentTransition.setDate( + ServiceUtils.parseIso8601Date(getText())); + + } else if (name.equals("Days")) { + currentTransition.setDays(Integer.parseInt(getText())); + } + } + + else if (in("LifecycleConfiguration", "Rule", "NoncurrentVersionExpiration")) { + if (name.equals("NoncurrentDays")) { + currentRule.setNoncurrentVersionExpirationInDays( + Integer.parseInt(getText())); + } + } + + else if (in("LifecycleConfiguration", "Rule", "NoncurrentVersionTransition")) { + if (name.equals("StorageClass")) { + currentNcvTransition.setStorageClass(getText()); + } else if (name.equals("NoncurrentDays")) { + currentNcvTransition.setDays(Integer.parseInt(getText())); + } + } + + else if (in("LifecycleConfiguration", "Rule", "AbortIncompleteMultipartUpload")) { + if (name.equals("DaysAfterInitiation")) { + abortIncompleteMultipartUpload.setDaysAfterInitiation + (Integer.parseInt(getText())); + } + } + + else if (in("LifecycleConfiguration", "Rule", "Filter")) { + if (name.equals("Prefix")) { + currentFilter.setPredicate(new LifecyclePrefixPredicate(getText())); + } else if (name.equals("Tag")) { + currentFilter.setPredicate(new LifecycleTagPredicate(new Tag(currentTagKey, currentTagValue))); + currentTagKey = null; + currentTagValue = null; + } else if (name.equals("And")) { + currentFilter.setPredicate(new LifecycleAndOperator(andOperandsList)); + andOperandsList = null; + } + } + + else if (in("LifecycleConfiguration", "Rule", "Filter", "Tag")) { + if (name.equals("Key")) { + currentTagKey = getText(); + } else if (name.equals("Value")) { + currentTagValue = getText(); + } + } + + else if (in("LifecycleConfiguration", "Rule", "Filter", "And")) { + if (name.equals("Prefix")) { + andOperandsList.add(new LifecyclePrefixPredicate(getText())); + } else if (name.equals("Tag")) { + andOperandsList.add(new LifecycleTagPredicate(new Tag(currentTagKey, currentTagValue))); + currentTagKey = null; + currentTagValue = null; + } + } + + else if (in("LifecycleConfiguration", "Rule", "Filter", "And", "Tag")) { + if (name.equals("Key")) { + currentTagKey = getText(); + } else if (name.equals("Value")) { + currentTagValue = getText(); + } + } + + } + } + + /* + HTTP/1.1 200 OK + x-amz-id-2: Uuag1LuByRx9e6j5Onimru9pO4ZVKnJ2Qz7/C1NPcfTWAtRPfTaOFg== + x-amz-request-id: 656c76696e6727732072657175657374 + Date: Tue, 20 Sep 2011 20:34:56 GMT + Content-Length: Some Length + Connection: keep-alive + Server: AmazonS3 + + + http://www.foobar.com + GET + 3000 + x-amz-server-side-encryption + + + */ + public static class BucketCrossOriginConfigurationHandler extends AbstractHandler { + + private final BucketCrossOriginConfiguration configuration = + new BucketCrossOriginConfiguration(new ArrayList()); + + private CORSRule currentRule; + private List allowedMethods = null; + private List allowedOrigins = null; + private List exposedHeaders = null; + private List allowedHeaders = null; + + public BucketCrossOriginConfiguration getConfiguration() { + return configuration; + } + + @Override + protected void doStartElement( + String uri, + String name, + String qName, + Attributes attrs) { + + if (in("CORSConfiguration")) { + if (name.equals("CORSRule")) { + currentRule = new CORSRule(); + } + } else if (in("CORSConfiguration", "CORSRule")) { + if (name.equals("AllowedOrigin")) { + if (allowedOrigins == null) { + allowedOrigins = new ArrayList(); + } + } else if (name.equals("AllowedMethod")) { + if (allowedMethods == null) { + allowedMethods = new ArrayList(); + } + } else if (name.equals("ExposeHeader")) { + if (exposedHeaders == null) { + exposedHeaders = new ArrayList(); + } + } else if (name.equals("AllowedHeader")) { + if (allowedHeaders == null) { + allowedHeaders = new LinkedList(); + } + } + } + } + + @Override + protected void doEndElement(String uri, String name, String qName) { + if (in("CORSConfiguration")) { + if (name.equals("CORSRule")) { + currentRule.setAllowedHeaders(allowedHeaders); + currentRule.setAllowedMethods(allowedMethods); + currentRule.setAllowedOrigins(allowedOrigins); + currentRule.setExposedHeaders(exposedHeaders); + allowedHeaders = null; + allowedMethods = null; + allowedOrigins = null; + exposedHeaders = null; + + configuration.getRules().add(currentRule); + currentRule = null; + } + } else if (in("CORSConfiguration", "CORSRule")) { + if (name.equals("ID")) { + currentRule.setId(getText()); + + } else if (name.equals("AllowedOrigin")) { + allowedOrigins.add(getText()); + + } else if (name.equals("AllowedMethod")) { + allowedMethods.add(AllowedMethods.fromValue(getText())); + + } else if (name.equals("MaxAgeSeconds")) { + currentRule.setMaxAgeSeconds(Integer.parseInt(getText())); + + } else if (name.equals("ExposeHeader")) { + exposedHeaders.add(getText()); + + } else if (name.equals("AllowedHeader")) { + allowedHeaders.add(getText()); + } + } + } + } + + /* + HTTP/1.1 200 OK + x-amz-id-2: ITnGT1y4RyTmXa3rPi4hklTXouTf0hccUjo0iCPjz6FnfIutBj3M7fPGlWO2SEWp + x-amz-request-id: 51991C342C575321 + Date: Wed, 14 May 2014 02:11:22 GMT + Server: AmazonS3 + Content-Length: ... + + + + metrics-id + + + + com/amazonaws/services/s3/model/transform/XmlResponsesSaxParser** + + diff --git a/flink-filesystems/flink-s3-fs-presto/pom.xml b/flink-filesystems/flink-s3-fs-presto/pom.xml index fd5e227011..037d1cf516 100644 --- a/flink-filesystems/flink-s3-fs-presto/pom.xml +++ b/flink-filesystems/flink-s3-fs-presto/pom.xml @@ -376,6 +376,15 @@ under the License. com/facebook/presto/hadoop/** + + com.amazonaws:aws-java-sdk-s3 + + + com/amazonaws/services/s3/model/transform/XmlResponsesSaxParser** + + diff --git a/tools/maven/suppressions.xml b/tools/maven/suppressions.xml index a6cc55353e..b21c26eb22 100644 --- a/tools/maven/suppressions.xml +++ b/tools/maven/suppressions.xml @@ -53,6 +53,10 @@ under the License. + + Date: Tue, 3 Mar 2020 16:08:21 +0800 Subject: [PATCH 125/885] [FLINK-16363][table] Correct the execution behavior of TableEnvironment and StreamTableEnvironment Note: In previous versions, TableEnvironment.execute() and StreamExecutionEnvironment.execute() can both trigger table and DataStream programs. Since 1.11.0, table programs can only be triggered by TableEnvironment.execute(). Once table program is convereted into DataStream program (through toAppendStream() or toRetractStream() method), it can only be triggered by StreamExecutionEnvironment.execute(). This closes #11296 --- docs/dev/table/common.md | 23 +- docs/dev/table/common.zh.md | 24 +- .../cassandra/CassandraConnectorITCase.java | 2 +- .../hive/HiveCatalogUseBlinkITCase.java | 7 +- .../connectors/kafka/KafkaTableTestBase.java | 2 +- .../io/jdbc/JDBCUpsertTableSinkITCase.java | 6 +- .../table/examples/java/StreamSQLExample.java | 2 + .../examples/java/StreamWindowSQLExample.java | 5 +- .../test_stream_execution_environment.py | 3 + .../table/tests/test_table_environment_api.py | 4 +- .../gateway/local/ExecutionContext.java | 18 +- .../internal/StreamTableEnvironmentImpl.java | 19 +- .../api/internal/TableEnvironmentImpl.java | 56 ++- .../flink/table/delegation/Executor.java | 21 +- .../flink/table/utils/ExecutorMock.java | 8 +- .../internal/StreamTableEnvironmentImpl.scala | 9 +- .../planner/delegation/BatchExecutor.java | 17 +- .../planner/delegation/ExecutorBase.java | 18 +- .../planner/delegation/StreamExecutor.java | 16 +- .../DummyStreamExecutionEnvironment.java | 28 +- .../planner/delegation/BatchPlanner.scala | 15 +- .../planner/delegation/PlannerBase.scala | 2 - .../planner/delegation/StreamPlanner.scala | 11 +- .../planner/delegation/BatchExecutorTest.java | 7 +- .../runtime/stream/sql/FunctionITCase.java | 6 + .../testFromToDataStreamAndSqlUpdate.out | 34 ++ .../explain/testSqlUpdateAndToDataStream.out | 33 ++ .../testStreamTableEnvironmentExplain.out | 33 ++ .../table/api/TableEnvironmentITCase.scala | 201 ++++++++-- .../table/api/TableEnvironmentTest.scala | 8 +- .../stream/sql/TimeAttributeITCase.scala | 6 +- .../stream/table/TableSinkITCase.scala | 24 +- .../runtime/utils/StreamTestSink.scala | 2 + .../table/planner/utils/TableTestBase.scala | 30 +- .../flink/table/executor/StreamExecutor.java | 23 +- .../util/DummyStreamExecutionEnvironment.java | 298 ++++++++++++++ .../flink/table/planner/StreamPlanner.scala | 13 +- .../table/api/TableEnvironmentITCase.scala | 362 ++++++++++++++++++ .../api/stream/sql/TableFactoryTest.scala | 2 + .../validation/InsertIntoValidationTest.scala | 4 + .../validation/InsertIntoValidationTest.scala | 6 + .../table/catalog/CatalogTableITCase.scala | 2 +- .../runtime/stream/TimeAttributesITCase.scala | 2 +- .../runtime/stream/sql/InsertIntoITCase.scala | 16 +- .../table/runtime/stream/sql/SortITCase.scala | 2 +- .../table/runtime/stream/sql/SqlITCase.scala | 1 + .../stream/table/AggregateITCase.scala | 2 +- .../runtime/stream/table/JoinITCase.scala | 4 +- .../stream/table/TableSinkITCase.scala | 22 +- .../utils/MemoryTableSourceSinkUtil.scala | 3 +- .../testFromToDataStreamAndSqlUpdate.out | 36 ++ .../testSqlUpdateAndToDataStream.out | 19 + 52 files changed, 1234 insertions(+), 283 deletions(-) create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/explain/testFromToDataStreamAndSqlUpdate.out create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/explain/testSqlUpdateAndToDataStream.out create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/explain/testStreamTableEnvironmentExplain.out create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/util/DummyStreamExecutionEnvironment.java create mode 100644 flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableEnvironmentITCase.scala create mode 100644 flink-table/flink-table-planner/src/test/scala/resources/testFromToDataStreamAndSqlUpdate.out create mode 100644 flink-table/flink-table-planner/src/test/scala/resources/testSqlUpdateAndToDataStream.out diff --git a/docs/dev/table/common.md b/docs/dev/table/common.md index bd015e3285..0d926ff12a 100644 --- a/docs/dev/table/common.md +++ b/docs/dev/table/common.md @@ -835,13 +835,18 @@ Table API and SQL queries are translated into [DataStream]({{ site.baseurl }}/de 1. Optimization of the logical plan 2. Translation into a DataStream or DataSet program -A Table API or SQL query is translated when: +For streaming, a Table API or SQL query is translated when: + +* `TableEnvironment.execute()` is called. A `Table` (emitted to a `TableSink` through `Table.insertInto()`) or a SQL update query (specified through `TableEnvironment.sqlUpdate()`) will be buffered in `TableEnvironment` first. Each sink will be optimized independently. The execution graph contains multiple independent sub-DAGs. +* A `Table` is translated when it is converted into a `DataStream` (see [Integration with DataStream and DataSet API](#integration-with-datastream-and-dataset-api)). Once translated, it's a regular DataStream program and is executed when `StreamExecutionEnvironment.execute()` is called. + +For batch, a Table API or SQL query is translated when: * a `Table` is emitted to a `TableSink`, i.e., when `Table.insertInto()` is called. * a SQL update query is specified, i.e., when `TableEnvironment.sqlUpdate()` is called. -* a `Table` is converted into a `DataStream` or `DataSet` (see [Integration with DataStream and DataSet API](#integration-with-datastream-and-dataset-api)). +* a `Table` is converted into a `DataSet` (see [Integration with DataStream and DataSet API](#integration-with-datastream-and-dataset-api)). -Once translated, a Table API or SQL query is handled like a regular DataStream or DataSet program and is executed when `StreamExecutionEnvironment.execute()` or `ExecutionEnvironment.execute()` is called. +Once translated, a Table API or SQL query is handled like a regular DataSet program and is executed when `ExecutionEnvironment.execute()` is called. @@ -851,17 +856,11 @@ Table API and SQL queries are translated into [DataStream]({{ site.baseurl }}/de 1. Optimization of the logical plan, 2. Translation into a DataStream program. -The behavior of translating a query is different for `TableEnvironment` and `StreamTableEnvironment`. - -For `TableEnvironment`, a Table API or SQL query is translated when `TableEnvironment.execute()` is called, because `TableEnvironment` will optimize multiple-sinks into one DAG. +a Table API or SQL query is translated when: -While for `StreamTableEnvironment`, a Table API or SQL query is translated when: - -* a `Table` is emitted to a `TableSink`, i.e., when `Table.insertInto()` is called. -* a SQL update query is specified, i.e., when `TableEnvironment.sqlUpdate()` is called. -* a `Table` is converted into a `DataStream`. +* `TableEnvironment.execute()` is called. A `Table` (emitted to a `TableSink` through `Table.insertInto()`) or a SQL update query (specified through `TableEnvironment.sqlUpdate()`) will be buffered in `TableEnvironment` first. All sinks will be optimized into one DAG. +* A `Table` is translated when it is converted into a `DataStream` (see [Integration with DataStream and DataSet API](#integration-with-datastream-and-dataset-api)). Once translated, it's a regular DataStream program and is executed when `StreamExecutionEnvironment.execute()` is called. -Once translated, a Table API or SQL query is handled like a regular DataStream program and is executed when `TableEnvironment.execute()` or `StreamExecutionEnvironment.execute()` is called. diff --git a/docs/dev/table/common.zh.md b/docs/dev/table/common.zh.md index ff6b95a19d..adbc695804 100644 --- a/docs/dev/table/common.zh.md +++ b/docs/dev/table/common.zh.md @@ -822,13 +822,18 @@ Table API 和 SQL 查询会被翻译成 [DataStream]({{ site.baseurl }}/zh/dev/d 1. 优化逻辑执行计划 2. 翻译成 DataStream 或 DataSet 程序 -Table API 或者 SQL 查询在下列情况下会被翻译: +对于 Streaming 而言,Table API 或者 SQL 查询在下列情况下会被翻译: + +* 当 `TableEnvironment.execute()` 被调用时。`Table` (通过 `Table.insertInto()` 输出给 `TableSink`)和 SQL (通过调用 `TableEnvironment.sqlUpdate()`)会先被缓存到 `TableEnvironment` 中,每个 sink 会被单独优化。执行计划将包括多个独立的有向无环子图。 +* `Table` 被转换成 `DataStream` 时(参阅[与 DataStream 和 DataSet API 结合](#integration-with-datastream-and-dataset-api))。转换完成后,它就成为一个普通的 DataStream 程序,并且会在调用 `StreamExecutionEnvironment.execute()` 的时候被执行。 + +对于 Batch 而言,Table API 或者 SQL 查询在下列情况下会被翻译: * `Table` 被输出给 `TableSink`,即当调用 `Table.insertInto()` 时。 * SQL 更新语句执行时,即,当调用 `TableEnvironment.sqlUpdate()` 时。 -* `Table` 被转换成 `DataStream` 或者 `DataSet` 时(参阅[与 DataStream 和 DataSet API 结合](#integration-with-datastream-and-dataset-api))。 +* `Table` 被转换成 `DataSet` 时(参阅[与 DataStream 和 DataSet API 结合](#integration-with-datastream-and-dataset-api))。 -翻译完成后,Table API 或者 SQL 查询会被当做普通的 DataStream 或 DataSet 程序对待并且会在调用 `StreamExecutionEnvironment.execute()` 或 `ExecutionEnvironment.execute()` 的时候被执行。 +翻译完成后,Table API 或者 SQL 查询会被当做普通的 DataSet 程序对待并且会在调用 `ExecutionEnvironment.execute()` 的时候被执行。 @@ -838,17 +843,10 @@ Table API 或者 SQL 查询在下列情况下会被翻译: 1. 优化逻辑执行计划 2. 翻译成 DataStream 程序 -TableEnvironment 和 StreamTableEnvironment 翻译查询的方式不同。 - -对于 `TableEnvironment`,Table API 和 SQL 查询会在调用 `TableEnvironment.execute()` 时被翻译,因为 `TableEnvironment` 会将多 sink 优化成一张有向无环图。 - -而对于 `StreamTableEnvironment`,当下列情况发生时,Table API 和 SQL 查询才会被翻译: - -* `Table 被发送至`TableSink`,即,当 `Table.insertInto()` 被调用时。 -* SQL 更新语句执行时,即,当调用 `TableEnvironment.sqlUpdate()` 时。 -* `Table` 被转换成 `DataStream` 时。 +Table API 或者 SQL 查询在下列情况下会被翻译: -翻译完成后,Table API 或者 SQL 查询会被当做普通的 DataStream 程序对待并且会在调用 `TableEnvironment.execute()` 或者 `StreamExecutionEnvironment.execute()` 的时候被执行。 +* 当 `TableEnvironment.execute()` 被调用时。`Table` (通过 `Table.insertInto()` 输出给 `TableSink`)和 SQL (通过调用 `TableEnvironment.sqlUpdate()`)会先被缓存到 `TableEnvironment` 中,所有的 sink 会被优化成一张有向无环图。 +* `Table` 被转换成 `DataStream` 时(参阅[与 DataStream 和 DataSet API 结合](#integration-with-datastream-and-dataset-api))。转换完成后,它就成为一个普通的 DataStream 程序,并且会在调用 `StreamExecutionEnvironment.execute()` 的时候被执行。 diff --git a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java index 5aa282ef29..5144945e74 100644 --- a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java +++ b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java @@ -471,7 +471,7 @@ public class CassandraConnectorITCase extends WriteAheadSinkTestBase 100" % source) t_env.sql_update("insert into sink2 select * from %s where a < 100" % source) - with self.assertRaises(TableException): - t_env.explain(extended=True) + actual = t_env.explain(extended=True) + assert isinstance(actual, str) def test_sql_query(self): t_env = self.t_env diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java index 8bad114024..51de2c0e92 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java @@ -84,7 +84,6 @@ import org.apache.flink.table.functions.TableFunction; import org.apache.flink.table.functions.UserDefinedFunction; import org.apache.flink.table.module.Module; import org.apache.flink.table.module.ModuleManager; -import org.apache.flink.table.planner.delegation.ExecutorBase; import org.apache.flink.table.sinks.TableSink; import org.apache.flink.table.sources.TableSource; import org.apache.flink.util.FlinkException; @@ -273,26 +272,13 @@ public class ExecutionContext { public Pipeline createPipeline(String name) { if (streamExecEnv != null) { - // special case for Blink planner to apply batch optimizations - // note: it also modifies the ExecutionConfig! - if (isBlinkPlanner(executor.getClass())) { - return ((ExecutorBase) executor).getStreamGraph(name); - } - return streamExecEnv.getStreamGraph(name); + StreamTableEnvironmentImpl streamTableEnv = (StreamTableEnvironmentImpl) tableEnv; + return streamTableEnv.getPipeline(name); } else { return execEnv.createProgramPlan(name); } } - private boolean isBlinkPlanner(Class executorClass) { - try { - return ExecutorBase.class.isAssignableFrom(executorClass); - } catch (NoClassDefFoundError ignore) { - // blink planner might not be on the class path - return false; - } - } - /** Returns a builder for this {@link ExecutionContext}. */ public static Builder builder( Environment defaultEnv, diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImpl.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImpl.java index d95e8df5d6..c88d5edec4 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImpl.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImpl.java @@ -21,6 +21,7 @@ package org.apache.flink.table.api.java.internal; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.time.Time; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.dag.Pipeline; import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TupleTypeInfo; @@ -357,6 +358,13 @@ public final class StreamTableEnvironmentImpl extends TableEnvironmentImpl imple return executionEnvironment; } + /** + * This method is used for sql client to submit job. + */ + public Pipeline getPipeline(String jobName) { + return execEnv.createPipeline(translateAndClearBuffer(), tableConfig, jobName); + } + private DataStream toDataStream(Table table, OutputConversionModifyOperation modifyOperation) { List> transformations = planner.translate(Collections.singletonList(modifyOperation)); @@ -372,17 +380,6 @@ public final class StreamTableEnvironmentImpl extends TableEnvironmentImpl imple validateTimeCharacteristic(TableSourceValidation.hasRowtimeAttribute(tableSource)); } - @Override - protected boolean isEagerOperationTranslation() { - return true; - } - - @Override - public String explain(boolean extended) { - // throw exception directly, because the operations to explain are always empty - throw new TableException("'explain' method without any tables is unsupported in StreamTableEnvironment."); - } - private TypeInformation extractTypeInformation(Table table, Class clazz) { try { return TypeExtractor.createTypeInfo(clazz); diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java index 9e0b7ac666..16c8d1f7be 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java @@ -22,6 +22,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.dag.Pipeline; import org.apache.flink.api.dag.Transformation; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.SqlParserException; @@ -427,11 +428,7 @@ public class TableEnvironmentImpl implements TableEnvironment { objectIdentifier, table.getQueryOperation())); - if (isEagerOperationTranslation()) { - translate(modifyOperations); - } else { - buffer(modifyOperations); - } + buffer(modifyOperations); } private Optional scanInternal(UnresolvedIdentifier identifier) { @@ -566,12 +563,7 @@ public class TableEnvironmentImpl implements TableEnvironment { Operation operation = operations.get(0); if (operation instanceof ModifyOperation) { - List modifyOperations = Collections.singletonList((ModifyOperation) operation); - if (isEagerOperationTranslation()) { - translate(modifyOperations); - } else { - buffer(modifyOperations); - } + buffer(Collections.singletonList((ModifyOperation) operation)); } else if (operation instanceof CreateTableOperation) { CreateTableOperation createTableOperation = (CreateTableOperation) operation; catalogManager.createTable( @@ -723,9 +715,8 @@ public class TableEnvironmentImpl implements TableEnvironment { @Override public JobExecutionResult execute(String jobName) throws Exception { - translate(bufferedModifyOperations); - bufferedModifyOperations.clear(); - return execEnv.execute(jobName); + Pipeline pipeline = execEnv.createPipeline(translateAndClearBuffer(), tableConfig, jobName); + return execEnv.execute(pipeline); } /** @@ -738,22 +729,6 @@ public class TableEnvironmentImpl implements TableEnvironment { return queryOperation; } - /** - * Defines the behavior of this {@link TableEnvironment}. If true the queries will - * be translated immediately. If false the {@link ModifyOperation}s will be buffered - * and translated only when {@link #execute(String)} is called. - * - *

    If the {@link TableEnvironment} works in a lazy manner it is undefined what - * configurations values will be used. It depends on the characteristic of the particular - * parameter. Some might used values current to the time of query construction (e.g. the currentCatalog) - * and some use values from the time when {@link #execute(String)} is called (e.g. timeZone). - * - * @return true if the queries should be translated immediately. - */ - protected boolean isEagerOperationTranslation() { - return false; - } - /** * Subclasses can override this method to add additional checks. * @@ -763,10 +738,25 @@ public class TableEnvironmentImpl implements TableEnvironment { TableSourceValidation.validateTableSource(tableSource, tableSource.getTableSchema()); } - private void translate(List modifyOperations) { - List> transformations = planner.translate(modifyOperations); + /** + * Translate the buffered operations to Transformations, and clear the buffer. + * + *

    The buffer will be clear even if the `translate` fails. In most cases, + * the failure is not retryable (e.g. type mismatch, can't generate physical plan). + * If the buffer is not clear after failure, the following `translate` will also fail. + */ + protected List> translateAndClearBuffer() { + List> transformations; + try { + transformations = translate(bufferedModifyOperations); + } finally { + bufferedModifyOperations.clear(); + } + return transformations; + } - execEnv.apply(transformations); + private List> translate(List modifyOperations) { + return planner.translate(modifyOperations); } private void buffer(List modifyOperations) { diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/delegation/Executor.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/delegation/Executor.java index 8511a6e305..70e115a0b4 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/delegation/Executor.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/delegation/Executor.java @@ -20,7 +20,9 @@ package org.apache.flink.table.delegation; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.dag.Pipeline; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.TableEnvironment; import java.util.List; @@ -35,19 +37,24 @@ import java.util.List; public interface Executor { /** - * Applies all given transformations. This should not run the transformations already, but just apply for - * future execution via {@link #execute(String)} + * Translates the given transformations to a Pipeline. * - * @param transformations list of transformations to apply + * @param transformations list of transformations + * @param jobName what should be the name of the job + * @return The pipeline representing the transformations. */ - void apply(List> transformations); + Pipeline createPipeline( + List> transformations, + TableConfig tableConfig, + String jobName); /** - * Executes all the previously applied transformations via {@link #apply(List)}. + * Executes the given pipeline. * - * @param jobName what should be the name of the job + * @param pipeline the pipeline to execute * @return The result of the job execution, containing elapsed time and accumulators. * @throws Exception which occurs during job execution. */ - JobExecutionResult execute(String jobName) throws Exception; + JobExecutionResult execute(Pipeline pipeline) throws Exception; + } diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/ExecutorMock.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/ExecutorMock.java index bcd7b7a0ad..eb4b177a6b 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/ExecutorMock.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/ExecutorMock.java @@ -19,7 +19,9 @@ package org.apache.flink.table.utils; import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.dag.Pipeline; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.delegation.Executor; import java.util.List; @@ -30,12 +32,12 @@ import java.util.List; public class ExecutorMock implements Executor { @Override - public void apply(List> transformations) { - // nothing to do + public Pipeline createPipeline(List> transformations, TableConfig tableConfig, String jobName) { + return null; } @Override - public JobExecutionResult execute(String jobName) throws Exception { + public JobExecutionResult execute(Pipeline pipeline) throws Exception { return null; } } diff --git a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/internal/StreamTableEnvironmentImpl.scala b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/internal/StreamTableEnvironmentImpl.scala index 7ac1ffb9d3..97e6c4e53f 100644 --- a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/internal/StreamTableEnvironmentImpl.scala +++ b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/internal/StreamTableEnvironmentImpl.scala @@ -40,6 +40,7 @@ import org.apache.flink.table.operations.{OutputConversionModifyOperation, Query import org.apache.flink.table.sources.{TableSource, TableSourceValidation} import org.apache.flink.table.types.utils.TypeConversions import org.apache.flink.table.typeutils.FieldInfoUtils + import java.util import java.util.{Collections, List => JList, Map => JMap} @@ -188,14 +189,6 @@ class StreamTableEnvironmentImpl ( } } - override protected def isEagerOperationTranslation(): Boolean = true - - override def explain(extended: Boolean): String = { - // throw exception directly, because the operations to explain are always empty - throw new TableException( - "'explain' method without any tables is unsupported in StreamTableEnvironment.") - } - private def toDataStream[T]( table: Table, modifyOperation: OutputConversionModifyOperation) diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/BatchExecutor.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/BatchExecutor.java index 2334e930a3..6ada54a494 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/BatchExecutor.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/BatchExecutor.java @@ -20,12 +20,16 @@ package org.apache.flink.table.planner.delegation; import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.dag.Pipeline; +import org.apache.flink.api.dag.Transformation; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.graph.StreamGraph; +import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.delegation.Executor; import org.apache.flink.table.planner.utils.ExecutorUtils; +import java.util.List; + /** * An implementation of {@link Executor} that is backed by a {@link StreamExecutionEnvironment}. * This is the only executor that {@link org.apache.flink.table.planner.delegation.BatchPlanner} supports. @@ -39,19 +43,12 @@ public class BatchExecutor extends ExecutorBase { } @Override - public JobExecutionResult execute(String jobName) throws Exception { - StreamGraph streamGraph = getStreamGraph(jobName); - return getExecutionEnvironment().execute(streamGraph); - } - - @Override - public StreamGraph getStreamGraph(String jobName) { + public Pipeline createPipeline(List> transformations, TableConfig tableConfig, String jobName) { StreamExecutionEnvironment execEnv = getExecutionEnvironment(); ExecutorUtils.setBatchProperties(execEnv, tableConfig); - StreamGraph streamGraph = execEnv.getStreamGraph(); + StreamGraph streamGraph = ExecutorUtils.generateStreamGraph(execEnv, transformations); streamGraph.setJobName(getNonEmptyJobName(jobName)); ExecutorUtils.setBatchProperties(streamGraph, tableConfig); return streamGraph; } - } diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/ExecutorBase.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/ExecutorBase.java index af12c2258e..44165c905a 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/ExecutorBase.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/ExecutorBase.java @@ -19,15 +19,14 @@ package org.apache.flink.table.planner.delegation; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.dag.Transformation; +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.dag.Pipeline; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.graph.StreamGraph; import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.delegation.Executor; import org.apache.flink.util.StringUtils; -import java.util.List; - /** * An implementation of {@link Executor} that is backed by a {@link StreamExecutionEnvironment}. */ @@ -43,24 +42,15 @@ public abstract class ExecutorBase implements Executor { this.executionEnvironment = executionEnvironment; } - public void setTableConfig(TableConfig tableConfig) { - this.tableConfig = tableConfig; - } - public StreamExecutionEnvironment getExecutionEnvironment() { return executionEnvironment; } @Override - public void apply(List> transformations) { - transformations.forEach(getExecutionEnvironment()::addOperator); + public JobExecutionResult execute(Pipeline pipeline) throws Exception { + return executionEnvironment.execute((StreamGraph) pipeline); } - /** - * Translates the transformations applied into this executor to a stream graph. - */ - public abstract StreamGraph getStreamGraph(String jobName); - protected String getNonEmptyJobName(String jobName) { if (StringUtils.isNullOrWhitespaceOnly(jobName)) { return DEFAULT_JOB_NAME; diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/StreamExecutor.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/StreamExecutor.java index 034da4d0c7..b89629e6ed 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/StreamExecutor.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/StreamExecutor.java @@ -20,10 +20,15 @@ package org.apache.flink.table.planner.delegation; import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.dag.Pipeline; +import org.apache.flink.api.dag.Transformation; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.graph.StreamGraph; +import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.delegation.Executor; +import org.apache.flink.table.planner.utils.ExecutorUtils; + +import java.util.List; /** * An implementation of {@link Executor} that is backed by a {@link StreamExecutionEnvironment}. @@ -38,13 +43,8 @@ public class StreamExecutor extends ExecutorBase { } @Override - public JobExecutionResult execute(String jobName) throws Exception { - return getExecutionEnvironment().execute(getNonEmptyJobName(jobName)); - } - - @Override - public StreamGraph getStreamGraph(String jobName) { - StreamGraph streamGraph = getExecutionEnvironment().getStreamGraph(); + public Pipeline createPipeline(List> transformations, TableConfig tableConfig, String jobName) { + StreamGraph streamGraph = ExecutorUtils.generateStreamGraph(getExecutionEnvironment(), transformations); streamGraph.setJobName(getNonEmptyJobName(jobName)); return streamGraph; } diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/utils/DummyStreamExecutionEnvironment.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/utils/DummyStreamExecutionEnvironment.java index 309b3d9dd3..706bd3a536 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/utils/DummyStreamExecutionEnvironment.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/utils/DummyStreamExecutionEnvironment.java @@ -31,9 +31,10 @@ import org.apache.flink.streaming.api.TimeCharacteristic; import org.apache.flink.streaming.api.environment.CheckpointConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.graph.StreamGraph; -import org.apache.flink.table.planner.delegation.PlannerBase; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.delegation.Planner; import org.apache.flink.table.sinks.StreamTableSink; -import org.apache.flink.table.sources.TableSource; +import org.apache.flink.table.sources.StreamTableSource; import com.esotericsoftware.kryo.Serializer; @@ -41,21 +42,26 @@ import java.io.Serializable; import java.util.List; /** - * This is dummy {@link StreamExecutionEnvironment}, only used for {@link PlannerBase#explain(List, boolean)} method. + * This is dummy {@link StreamExecutionEnvironment}, which holds a real {@link StreamExecutionEnvironment}, + * shares all configurations of the real environment, and disables all configuration setting methods. * - *

    {@link Transformation}s will be added into a {@link StreamExecutionEnvironment} when translating ExecNode to plan, - * and they will be cleared only when calling {@link StreamExecutionEnvironment#execute()} method. + *

    When translating relational plan to execution plan in the {@link Planner}, the generated {@link Transformation}s + * will be added into StreamExecutionEnvironment's buffer, and they will be cleared only when + * {@link StreamExecutionEnvironment#execute()} method is called. Each {@link TableEnvironment} instance holds + * an immutable StreamExecutionEnvironment instance. If there are multiple translations (not all for `execute`, + * e.g. `explain` and then `execute`) in one TableEnvironment instance, the transformation buffer is dirty, + * and execution result may be incorrect. * - *

    {@link PlannerBase#explain(List, boolean)} method will not only print logical plan but also execution plan, - * translating will happen in explain method. If calling explain method before execute method, the transformations in - * StreamExecutionEnvironment is dirty, and execution result may be incorrect. + *

    This dummy StreamExecutionEnvironment is only used for buffering the transformations generated in the planner. + * A new dummy StreamExecutionEnvironment instance should be created for each translation, and this could avoid + * dirty the transformation buffer of the real StreamExecutionEnvironment instance. * *

    All set methods (e.g. `setXX`, `enableXX`, `disableXX`, etc) are disabled to prohibit changing configuration, - * all get methods (e.g. `getXX`, `isXX`, etc) will be delegated to real StreamExecutionEnvironment. + * all get methods (e.g. `getXX`, `isXX`, etc) will be delegated to the real StreamExecutionEnvironment. * `execute`, `getStreamGraph`, `getExecutionPlan` methods are also disabled, while `addOperator` method is enabled to - * let `explain` method add Transformations to this StreamExecutionEnvironment. + * allow the planner to add the generated transformations to the dummy StreamExecutionEnvironment. * - *

    This class could be removed once the {@link TableSource} interface and {@link StreamTableSink} interface + *

    This class could be removed once the {@link StreamTableSource} interface and {@link StreamTableSink} interface * are reworked. * *

    NOTE: Please remove {@code com.esotericsoftware.kryo} item in the whitelist of checkCodeDependencies() diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/BatchPlanner.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/BatchPlanner.scala index a43d778738..2626809fea 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/BatchPlanner.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/BatchPlanner.scala @@ -65,8 +65,11 @@ class BatchPlanner( override protected def translateToPlan( execNodes: util.List[ExecNode[_, _]]): util.List[Transformation[_]] = { + val planner = createDummyPlanner() + planner.overrideEnvParallelism() + execNodes.map { - case node: BatchExecNode[_] => node.translateToPlan(this) + case node: BatchExecNode[_] => node.translateToPlan(planner) case _ => throw new TableException("Cannot generate BoundedStream due to an invalid logical plan. " + "This is a bug and should not happen. Please file an issue.") @@ -85,9 +88,7 @@ class BatchPlanner( val optimizedRelNodes = optimize(sinkRelNodes) val execNodes = translateToExecNodePlan(optimizedRelNodes) - val plannerForExplain = createDummyPlannerForExplain() - plannerForExplain.overrideEnvParallelism() - val transformations = plannerForExplain.translateToPlan(execNodes) + val transformations = translateToPlan(execNodes) val execEnv = getExecEnv ExecutorUtils.setBatchProperties(execEnv, getTableConfig) @@ -119,10 +120,10 @@ class BatchPlanner( sb.toString() } - private def createDummyPlannerForExplain(): BatchPlanner = { + private def createDummyPlanner(): BatchPlanner = { val dummyExecEnv = new DummyStreamExecutionEnvironment(getExecEnv) - val executorForExplain = new BatchExecutor(dummyExecEnv) - new BatchPlanner(executorForExplain, config, functionCatalog, catalogManager) + val executor = new BatchExecutor(dummyExecEnv) + new BatchPlanner(executor, config, functionCatalog, catalogManager) } } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala index 47892a13fb..5a32481daa 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala @@ -79,8 +79,6 @@ abstract class PlannerBase( // temporary utility until we don't use planner expressions anymore functionCatalog.setPlannerTypeInferenceUtil(PlannerTypeInferenceUtilImpl.INSTANCE) - executor.asInstanceOf[ExecutorBase].setTableConfig(config) - @VisibleForTesting private[flink] val plannerContext: PlannerContext = new PlannerContext( diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/StreamPlanner.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/StreamPlanner.scala index a690f29c19..886d433e22 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/StreamPlanner.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/StreamPlanner.scala @@ -56,8 +56,11 @@ class StreamPlanner( override protected def translateToPlan( execNodes: util.List[ExecNode[_, _]]): util.List[Transformation[_]] = { + val planner = createDummyPlanner() + planner.overrideEnvParallelism() + execNodes.map { - case node: StreamExecNode[_] => node.translateToPlan(this) + case node: StreamExecNode[_] => node.translateToPlan(planner) case _ => throw new TableException("Cannot generate DataStream due to an invalid logical plan. " + "This is a bug and should not happen. Please file an issue.") @@ -76,9 +79,7 @@ class StreamPlanner( val optimizedRelNodes = optimize(sinkRelNodes) val execNodes = translateToExecNodePlan(optimizedRelNodes) - val plannerForExplain = createDummyPlannerForExplain() - plannerForExplain.overrideEnvParallelism() - val transformations = plannerForExplain.translateToPlan(execNodes) + val transformations = translateToPlan(execNodes) val streamGraph = ExecutorUtils.generateStreamGraph(getExecEnv, transformations) val executionPlan = PlanUtil.explainStreamGraph(streamGraph) @@ -109,7 +110,7 @@ class StreamPlanner( sb.toString() } - private def createDummyPlannerForExplain(): StreamPlanner = { + private def createDummyPlanner(): StreamPlanner = { val dummyExecEnv = new DummyStreamExecutionEnvironment(getExecEnv) val executor = new StreamExecutor(dummyExecEnv) new StreamPlanner(executor, config, functionCatalog, catalogManager) diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/delegation/BatchExecutorTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/delegation/BatchExecutorTest.java index a4b9adb140..fe6b5c62df 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/delegation/BatchExecutorTest.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/delegation/BatchExecutorTest.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.delegation; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.dag.Pipeline; import org.apache.flink.api.dag.Transformation; import org.apache.flink.streaming.api.environment.LocalStreamEnvironment; import org.apache.flink.streaming.api.functions.source.SourceFunction; @@ -45,7 +46,6 @@ public class BatchExecutorTest extends TestLogger { public BatchExecutorTest() { batchExecutor = new BatchExecutor(LocalStreamEnvironment.getExecutionEnvironment()); - batchExecutor.setTableConfig(new TableConfig()); final Transformation testTransform = new SourceTransformation<>( "MockTransform", @@ -60,8 +60,9 @@ public class BatchExecutorTest extends TestLogger { }), BasicTypeInfo.STRING_TYPE_INFO, 1); - batchExecutor.apply(Collections.singletonList(testTransform)); - streamGraph = batchExecutor.getStreamGraph("Test Job"); + Pipeline pipeline = batchExecutor.createPipeline( + Collections.singletonList(testTransform), new TableConfig(), "Test Job"); + streamGraph = (StreamGraph) pipeline; } @Test diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/FunctionITCase.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/FunctionITCase.java index 2c67306c6d..0491326e45 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/FunctionITCase.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/FunctionITCase.java @@ -547,6 +547,8 @@ public class FunctionITCase extends StreamingTestBase { tEnv().sqlUpdate( "INSERT INTO SinkTable " + "SELECT CustomScalarFunction('test')"); + // trigger translation + tEnv().explain(false); fail(); } catch (CodeGenException e) { assertThat( @@ -621,6 +623,8 @@ public class FunctionITCase extends StreamingTestBase { tEnv().sqlUpdate( "INSERT INTO SinkTable " + "SELECT * FROM TABLE(PrimitiveScalarFunction(1, 2, '3'))"); + // trigger translation + tEnv().explain(false); fail(); } catch (ValidationException e) { assertThat( @@ -639,6 +643,8 @@ public class FunctionITCase extends StreamingTestBase { tEnv().sqlUpdate( "INSERT INTO SinkTable " + "SELECT * FROM TABLE(MD5('3'))"); + // trigger translation + tEnv().explain(false); fail(); } catch (ValidationException e) { assertThat( diff --git a/flink-table/flink-table-planner-blink/src/test/resources/explain/testFromToDataStreamAndSqlUpdate.out b/flink-table/flink-table-planner-blink/src/test/resources/explain/testFromToDataStreamAndSqlUpdate.out new file mode 100644 index 0000000000..f3bc0cb93f --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/resources/explain/testFromToDataStreamAndSqlUpdate.out @@ -0,0 +1,34 @@ +== Abstract Syntax Tree == +LogicalSink(name=[`default_catalog`.`default_database`.`MySink1`], fields=[first]) ++- LogicalProject(first=[$0]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + +== Optimized Logical Plan == +Sink(name=[`default_catalog`.`default_database`.`MySink1`], fields=[first]) ++- Calc(select=[first]) + +- DataStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[first, id, score, last]) + +== Physical Execution Plan == + : Data Source + content : Source: Collection Source + + : Operator + content : SourceConversion(table=[default_catalog.default_database.MyTable], fields=[first, id, score, last]) + ship_strategy : FORWARD + + : Operator + content : Calc(select=[first]) + ship_strategy : FORWARD + + : Operator + content : SinkConversionToRow + ship_strategy : FORWARD + + : Operator + content : Map + ship_strategy : FORWARD + + : Data Sink + content : Sink: CsvTableSink(first) + ship_strategy : FORWARD + diff --git a/flink-table/flink-table-planner-blink/src/test/resources/explain/testSqlUpdateAndToDataStream.out b/flink-table/flink-table-planner-blink/src/test/resources/explain/testSqlUpdateAndToDataStream.out new file mode 100644 index 0000000000..29017e000d --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/resources/explain/testSqlUpdateAndToDataStream.out @@ -0,0 +1,33 @@ +== Abstract Syntax Tree == +LogicalSink(name=[`default_catalog`.`default_database`.`MySink1`], fields=[first]) ++- LogicalProject(first=[$0]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [CsvTableSource(read fields: first, id, score, last)]]]) + +== Optimized Logical Plan == +Sink(name=[`default_catalog`.`default_database`.`MySink1`], fields=[first]) ++- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [CsvTableSource(read fields: first)]]], fields=[first]) + +== Physical Execution Plan == + : Data Source + content : Source: Custom File source + + : Operator + content : CsvTableSource(read fields: first) + ship_strategy : REBALANCE + + : Operator + content : SourceConversion(table=[default_catalog.default_database.MyTable, source: [CsvTableSource(read fields: first)]], fields=[first]) + ship_strategy : FORWARD + + : Operator + content : SinkConversionToRow + ship_strategy : FORWARD + + : Operator + content : Map + ship_strategy : REBALANCE + + : Data Sink + content : Sink: CsvTableSink(first) + ship_strategy : FORWARD + diff --git a/flink-table/flink-table-planner-blink/src/test/resources/explain/testStreamTableEnvironmentExplain.out b/flink-table/flink-table-planner-blink/src/test/resources/explain/testStreamTableEnvironmentExplain.out new file mode 100644 index 0000000000..b0758cb2b1 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/resources/explain/testStreamTableEnvironmentExplain.out @@ -0,0 +1,33 @@ +== Abstract Syntax Tree == +LogicalSink(name=[`default_catalog`.`default_database`.`MySink`], fields=[first]) ++- LogicalProject(first=[$0]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [CsvTableSource(read fields: first, id, score, last)]]]) + +== Optimized Logical Plan == +Sink(name=[`default_catalog`.`default_database`.`MySink`], fields=[first]) ++- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [CsvTableSource(read fields: first)]]], fields=[first]) + +== Physical Execution Plan == + : Data Source + content : Source: Custom File source + + : Operator + content : CsvTableSource(read fields: first) + ship_strategy : REBALANCE + + : Operator + content : SourceConversion(table=[default_catalog.default_database.MyTable, source: [CsvTableSource(read fields: first)]], fields=[first]) + ship_strategy : FORWARD + + : Operator + content : SinkConversionToRow + ship_strategy : FORWARD + + : Operator + content : Map + ship_strategy : FORWARD + + : Data Sink + content : Sink: CsvTableSink(first) + ship_strategy : FORWARD + diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentITCase.scala index a4155df372..24ee0c203e 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentITCase.scala @@ -20,50 +20,63 @@ package org.apache.flink.table.api import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.common.typeinfo.Types.STRING +import org.apache.flink.api.scala._ import org.apache.flink.core.fs.FileSystem.WriteMode -import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment +import org.apache.flink.streaming.api.scala.{StreamExecutionEnvironment => ScalaStreamExecutionEnvironment} import org.apache.flink.table.api.internal.TableEnvironmentImpl -import org.apache.flink.table.api.scala.StreamTableEnvironment -import org.apache.flink.table.planner.utils.{TableTestUtil, TestTableSources} +import org.apache.flink.table.api.java.StreamTableEnvironment +import org.apache.flink.table.api.scala.{StreamTableEnvironment => ScalaStreamTableEnvironment, _} +import org.apache.flink.table.planner.runtime.utils.TestingAppendSink +import org.apache.flink.table.planner.utils.TableTestUtil.{readFromResource, replaceStageId} +import org.apache.flink.table.planner.utils.TestTableSources.getPersonCsvTableSource import org.apache.flink.table.sinks.CsvTableSink +import org.apache.flink.types.Row import org.apache.flink.util.FileUtils import org.junit.Assert.{assertEquals, assertFalse, assertTrue} import org.junit.rules.TemporaryFolder import org.junit.runner.RunWith import org.junit.runners.Parameterized -import org.junit.{Rule, Test} +import org.junit.{Before, Rule, Test} import _root_.java.io.File import _root_.java.util +import _root_.scala.collection.mutable + @RunWith(classOf[Parameterized]) -class TableEnvironmentITCase(settings: EnvironmentSettings, mode: String) { +class TableEnvironmentITCase(tableEnvName: String, isStreaming: Boolean) { private val _tempFolder = new TemporaryFolder() @Rule def tempFolder: TemporaryFolder = _tempFolder - @Test - def testExecuteTwiceUsingSameTableEnv(): Unit = { - testExecuteTwiceUsingSameTableEnv(TableEnvironmentImpl.create(settings)) + var tEnv: TableEnvironment = _ + + private val settings = if (isStreaming) { + EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build() + } else { + EnvironmentSettings.newInstance().useBlinkPlanner().inBatchMode().build() } - @Test - def testExecuteTwiceUsingSameStreamTableEnv(): Unit = { - if (settings.isStreamingMode) { - testExecuteTwiceUsingSameTableEnv(StreamTableEnvironment.create( - StreamExecutionEnvironment.getExecutionEnvironment, settings)) - } else { - // batch planner is not supported on StreamTableEnvironment + @Before + def setup(): Unit = { + tableEnvName match { + case "TableEnvironment" => + tEnv = TableEnvironmentImpl.create(settings) + case "StreamTableEnvironment" => + tEnv = StreamTableEnvironment.create( + StreamExecutionEnvironment.getExecutionEnvironment, settings) + case _ => throw new UnsupportedOperationException("unsupported tableEnvName: " + tableEnvName) } + tEnv.registerTableSource("MyTable", getPersonCsvTableSource) } - private def testExecuteTwiceUsingSameTableEnv(tEnv: TableEnvironment): Unit = { - val tEnv = TableEnvironmentImpl.create(settings) - tEnv.registerTableSource("MyTable", TestTableSources.getPersonCsvTableSource) + @Test + def testExecuteTwiceUsingSameTableEnv(): Unit = { val sink1Path = registerCsvTableSink(tEnv, Array("first"), Array(STRING), "MySink1") val sink2Path = registerCsvTableSink(tEnv, Array("last"), Array(STRING), "MySink2") checkEmptyFile(sink1Path) @@ -88,8 +101,6 @@ class TableEnvironmentITCase(settings: EnvironmentSettings, mode: String) { @Test def testExplainAndExecuteSingleSink(): Unit = { - val tEnv = TableEnvironmentImpl.create(settings) - tEnv.registerTableSource("MyTable", TestTableSources.getPersonCsvTableSource) val sinkPath = registerCsvTableSink(tEnv, Array("first"), Array(STRING), "MySink1") val table1 = tEnv.sqlQuery("select first from MyTable") @@ -102,8 +113,6 @@ class TableEnvironmentITCase(settings: EnvironmentSettings, mode: String) { @Test def testExplainAndExecuteMultipleSink(): Unit = { - val tEnv = TableEnvironmentImpl.create(settings) - tEnv.registerTableSource("MyTable", TestTableSources.getPersonCsvTableSource) val sink1Path = registerCsvTableSink(tEnv, Array("first"), Array(STRING), "MySink1") val sink2Path = registerCsvTableSink(tEnv, Array("first"), Array(STRING), "MySink2") @@ -120,8 +129,6 @@ class TableEnvironmentITCase(settings: EnvironmentSettings, mode: String) { @Test def testExplainTwice(): Unit = { - val tEnv = TableEnvironmentImpl.create(settings) - tEnv.registerTableSource("MyTable", TestTableSources.getPersonCsvTableSource) registerCsvTableSink(tEnv, Array("first"), Array(STRING), "MySink1") registerCsvTableSink(tEnv, Array("first"), Array(STRING), "MySink2") @@ -132,7 +139,114 @@ class TableEnvironmentITCase(settings: EnvironmentSettings, mode: String) { val result1 = tEnv.explain(false) val result2 = tEnv.explain(false) - assertEquals(TableTestUtil.replaceStageId(result1), TableTestUtil.replaceStageId(result2)) + assertEquals(replaceStageId(result1), replaceStageId(result2)) + } + + @Test + def testSqlUpdateAndToDataStream(): Unit = { + if (!tableEnvName.equals("StreamTableEnvironment")) { + return + } + val streamEnv = StreamExecutionEnvironment.getExecutionEnvironment + val streamTableEnv = StreamTableEnvironment.create(streamEnv, settings) + streamTableEnv.registerTableSource("MyTable", getPersonCsvTableSource) + val sink1Path = registerCsvTableSink(streamTableEnv, Array("first"), Array(STRING), "MySink1") + checkEmptyFile(sink1Path) + + streamTableEnv.sqlUpdate("insert into MySink1 select first from MyTable") + + val table = streamTableEnv.sqlQuery("select last from MyTable where id > 0") + val resultSet = streamTableEnv.toAppendStream(table, classOf[Row]) + val sink = new TestingAppendSink + resultSet.addSink(sink) + + val explain = streamTableEnv.explain(false) + assertEquals( + replaceStageId(readFromResource("/explain/testSqlUpdateAndToDataStream.out")), + replaceStageId(explain)) + + streamTableEnv.execute("test1") + assertFirstValues(sink1Path) + + // the DataStream program is not executed + assertFalse(sink.isInitialized) + + deleteFile(sink1Path) + + streamEnv.execute("test2") + assertEquals(getExpectedLastValues.sorted, sink.getAppendResults.sorted) + // the table program is not executed again + assertFileNotExist(sink1Path) + } + + @Test + def testToDataStreamAndSqlUpdate(): Unit = { + if (!tableEnvName.equals("StreamTableEnvironment")) { + return + } + val streamEnv = StreamExecutionEnvironment.getExecutionEnvironment + val streamTableEnv = StreamTableEnvironment.create(streamEnv, settings) + streamTableEnv.registerTableSource("MyTable", getPersonCsvTableSource) + val sink1Path = registerCsvTableSink(streamTableEnv, Array("first"), Array(STRING), "MySink1") + checkEmptyFile(sink1Path) + + val table = streamTableEnv.sqlQuery("select last from MyTable where id > 0") + val resultSet = streamTableEnv.toAppendStream(table, classOf[Row]) + val sink = new TestingAppendSink + resultSet.addSink(sink) + + streamTableEnv.sqlUpdate("insert into MySink1 select first from MyTable") + + val explain = streamTableEnv.explain(false) + assertEquals( + replaceStageId(readFromResource("/explain/testSqlUpdateAndToDataStream.out")), + replaceStageId(explain)) + + streamEnv.execute("test2") + // the table program is not executed + checkEmptyFile(sink1Path) + assertEquals(getExpectedLastValues.sorted, sink.getAppendResults.sorted) + + streamTableEnv.execute("test1") + assertFirstValues(sink1Path) + // the DataStream program is not executed again because the result in sink is not changed + assertEquals(getExpectedLastValues.sorted, sink.getAppendResults.sorted) + } + + @Test + def testFromToDataStreamAndSqlUpdate(): Unit = { + if (!tableEnvName.equals("StreamTableEnvironment")) { + return + } + val streamEnv = ScalaStreamExecutionEnvironment.getExecutionEnvironment + val streamTableEnv = ScalaStreamTableEnvironment.create(streamEnv, settings) + val t = streamEnv.fromCollection(getPersonData) + .toTable(streamTableEnv, 'first, 'id, 'score, 'last) + streamTableEnv.registerTable("MyTable", t) + val sink1Path = registerCsvTableSink(streamTableEnv, Array("first"), Array(STRING), "MySink1") + checkEmptyFile(sink1Path) + + val table = streamTableEnv.sqlQuery("select last from MyTable where id > 0") + val resultSet = streamTableEnv.toAppendStream[Row](table) + val sink = new TestingAppendSink + resultSet.addSink(sink) + + streamTableEnv.sqlUpdate("insert into MySink1 select first from MyTable") + + val explain = streamTableEnv.explain(false) + assertEquals( + replaceStageId(readFromResource("/explain/testFromToDataStreamAndSqlUpdate.out")), + replaceStageId(explain)) + + streamEnv.execute("test2") + // the table program is not executed + checkEmptyFile(sink1Path) + assertEquals(getExpectedLastValues.sorted, sink.getAppendResults.sorted) + + streamTableEnv.execute("test1") + assertFirstValues(sink1Path) + // the DataStream program is not executed again because the result in sink is not changed + assertEquals(getExpectedLastValues.sorted, sink.getAppendResults.sorted) } private def registerCsvTableSink( @@ -150,6 +264,19 @@ class TableEnvironmentITCase(settings: EnvironmentSettings, mode: String) { path } + def getPersonData: List[(String, Int, Double, String)] = { + val data = new mutable.MutableList[(String, Int, Double, String)] + data.+=(("Mike", 1, 12.3, "Smith")) + data.+=(("Bob", 2, 45.6, "Taylor")) + data.+=(("Sam", 3, 7.89, "Miller")) + data.+=(("Peter", 4, 0.12, "Smith")) + data.+=(("Liz", 5, 34.5, "Williams")) + data.+=(("Sally", 6, 6.78, "Miller")) + data.+=(("Alice", 7, 90.1, "Smith")) + data.+=(("Kelly", 8, 2.34, "Williams")) + data.toList + } + private def assertFirstValues(csvFilePath: String): Unit = { val expected = List("Mike", "Bob", "Sam", "Peter", "Liz", "Sally", "Alice", "Kelly") val actual = FileUtils.readFileUtf8(new File(csvFilePath)).split("\n").toList @@ -157,23 +284,35 @@ class TableEnvironmentITCase(settings: EnvironmentSettings, mode: String) { } private def assertLastValues(csvFilePath: String): Unit = { - val expected = List( - "Smith", "Taylor", "Miller", "Smith", "Williams", "Miller", "Smith", "Williams") val actual = FileUtils.readFileUtf8(new File(csvFilePath)).split("\n").toList - assertEquals(expected.sorted, actual.sorted) + assertEquals(getExpectedLastValues.sorted, actual.sorted) + } + + private def getExpectedLastValues: List[String] = { + List("Smith", "Taylor", "Miller", "Smith", "Williams", "Miller", "Smith", "Williams") } private def checkEmptyFile(csvFilePath: String): Unit = { assertTrue(FileUtils.readFileUtf8(new File(csvFilePath)).isEmpty) } + + private def deleteFile(path: String): Unit = { + new File(path).delete() + assertFalse(new File(path).exists()) + } + + private def assertFileNotExist(path: String): Unit = { + assertFalse(new File(path).exists()) + } } object TableEnvironmentITCase { - @Parameterized.Parameters(name = "{1}") + @Parameterized.Parameters(name = "{0}:isStream={1}") def parameters(): util.Collection[Array[_]] = { util.Arrays.asList( - Array(EnvironmentSettings.newInstance().useBlinkPlanner().inBatchMode().build(), "batch"), - Array(EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build(), "stream") + Array("TableEnvironment", true), + Array("TableEnvironment", false), + Array("StreamTableEnvironment", true) ) } } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala index 8ba4236893..787f2d673d 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala @@ -81,10 +81,6 @@ class TableEnvironmentTest { @Test def testStreamTableEnvironmentExplain(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage( - "'explain' method without any tables is unsupported in StreamTableEnvironment.") - val execEnv = StreamExecutionEnvironment.getExecutionEnvironment val settings = EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build() val tEnv = StreamTableEnvironment.create(execEnv, settings) @@ -96,7 +92,9 @@ class TableEnvironmentTest { val table1 = tEnv.sqlQuery("select first from MyTable") tEnv.insertInto(table1, "MySink") - tEnv.explain(false) + val expected = TableTestUtil.readFromResource("/explain/testStreamTableEnvironmentExplain.out") + val actual = tEnv.explain(false) + assertEquals(TableTestUtil.replaceStageId(expected), TableTestUtil.replaceStageId(actual)) } } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TimeAttributeITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TimeAttributeITCase.scala index c1be4c945a..b2dd257dae 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TimeAttributeITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TimeAttributeITCase.scala @@ -75,7 +75,7 @@ class TimeAttributeITCase extends StreamingTestBase { tEnv.sqlUpdate(ddl) val sink = new TestingAppendSink() tEnv.sqlQuery(query).toAppendStream[Row].addSink(sink) - tEnv.execute("SQL JOB") + env.execute("SQL JOB") val expected = Seq( "1970-01-01T00:00:00.003,2,3.0", @@ -112,7 +112,7 @@ class TimeAttributeITCase extends StreamingTestBase { tEnv.sqlUpdate(ddl) val sink = new TestingAppendSink() tEnv.sqlQuery(query).toAppendStream[Row].addSink(sink) - tEnv.execute("SQL JOB") + env.execute("SQL JOB") val expected = Seq( "1970-01-01T00:00:00.003,2,3.0", @@ -149,7 +149,7 @@ class TimeAttributeITCase extends StreamingTestBase { tEnv.sqlUpdate(ddl) val sink = new TestingAppendSink() tEnv.sqlQuery(query).toAppendStream[Row].addSink(sink) - tEnv.execute("SQL JOB") + env.execute("SQL JOB") val expected = Seq( "1970-01-01T00:00:00.003,2,3.0", diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/TableSinkITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/TableSinkITCase.scala index 23f69f0bd8..320cf4afed 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/TableSinkITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/TableSinkITCase.scala @@ -63,7 +63,7 @@ class TableSinkITCase extends AbstractTestBase { .where('a < 3 || 'a > 19) .select('c, 't, 'b) .insertInto("targetTable") - env.execute() + tEnv.execute("job name") val expected = Seq( "Hi,1970-01-01 00:00:00.001,1.000000000000000000", @@ -103,7 +103,7 @@ class TableSinkITCase extends AbstractTestBase { .select(ifThenElse('a < 4, nullOf(Types.INT()), 'a), 'c, 'b) .insertInto("csvSink") - env.execute() + tEnv.execute("job name") val expected = Seq( ",Hello world,1970-01-01 00:00:00.002", @@ -141,7 +141,7 @@ class TableSinkITCase extends AbstractTestBase { .select('w.end as 't, 'id.count as 'icnt, 'num.sum as 'nsum) .insertInto("appendSink") - env.execute() + tEnv.execute("job name") val result = sink.getAppendResults.sorted val expected = List( @@ -174,7 +174,7 @@ class TableSinkITCase extends AbstractTestBase { tEnv.sqlUpdate("INSERT INTO appendSink SELECT id, ROW(num, text) FROM src") - env.execute() + tEnv.execute("job name") val result = sink.getAppendResults.sorted val expected = List( @@ -205,7 +205,7 @@ class TableSinkITCase extends AbstractTestBase { .select('c, 'g) .insertInto("appendSink") - env.execute() + tEnv.execute("job name") val result = sink.getAppendResults.sorted val expected = List("Hi,Hallo", "Hello,Hallo Welt", "Hello world,Hallo Welt").sorted @@ -235,7 +235,7 @@ class TableSinkITCase extends AbstractTestBase { .select('len, 'id.count as 'icnt, 'num.sum as 'nsum) .insertInto("retractSink") - env.execute() + tEnv.execute("job name") val retracted = sink.getRetractResults.sorted val expected = List( @@ -273,7 +273,7 @@ class TableSinkITCase extends AbstractTestBase { .select('w.end as 't, 'id.count as 'icnt, 'num.sum as 'nsum) .insertInto("retractSink") - env.execute() + tEnv.execute("job name") assertFalse( "Received retraction messages for append only table", @@ -317,7 +317,7 @@ class TableSinkITCase extends AbstractTestBase { .select('count, 'len.count as 'lencnt, 'cTrue) .insertInto("upsertSink") - env.execute() + tEnv.execute("job name") assertTrue( "Results must include delete messages", @@ -356,7 +356,7 @@ class TableSinkITCase extends AbstractTestBase { .select('num, 'w.end as 'window_end, 'id.count as 'icnt) .insertInto("upsertSink") - env.execute() + tEnv.execute("job name") assertFalse( "Received retraction messages for append only table", @@ -403,7 +403,7 @@ class TableSinkITCase extends AbstractTestBase { .select('w.start as 'wstart, 'w.end as 'wend, 'num, 'id.count as 'icnt) .insertInto("upsertSink") - env.execute() + tEnv.execute("job name") assertFalse( "Received retraction messages for append only table", @@ -448,7 +448,7 @@ class TableSinkITCase extends AbstractTestBase { .select('w.end as 'wend, 'id.count as 'cnt) .insertInto("upsertSink") - env.execute() + tEnv.execute("job name") assertFalse( "Received retraction messages for append only table", @@ -493,7 +493,7 @@ class TableSinkITCase extends AbstractTestBase { .select('num, 'id.count as 'cnt) .insertInto("upsertSink") - env.execute() + tEnv.execute("job name") assertFalse( "Received retraction messages for append only table", diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamTestSink.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamTestSink.scala index 1ddf312684..f36562377c 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamTestSink.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamTestSink.scala @@ -82,6 +82,8 @@ abstract class AbstractExactlyOnceSink[T] extends RichSinkFunction[T] with Check protected var globalRetractResults: mutable.Map[Int, ArrayBuffer[String]] = _ protected var globalUpsertResults: mutable.Map[Int, mutable.Map[String, String]] = _ + def isInitialized: Boolean = globalResults != null + override def initializeState(context: FunctionInitializationContext): Unit = { resultsState = context.getOperatorStateStore .getListState(new ListStateDescriptor[String]("sink-results", Types.STRING)) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala index b89088c730..5e45312fc1 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala @@ -1012,13 +1012,7 @@ class TestingTableEnvironment private( override def insertInto(path: String, table: Table): Unit = { val unresolvedIdentifier = parser.parseIdentifier(path) val identifier = catalogManager.qualifyIdentifier(unresolvedIdentifier) - - val modifyOperations = List(new CatalogSinkModifyOperation(identifier, table.getQueryOperation)) - if (isEagerOperationTranslation) { - translate(modifyOperations) - } else { - buffer(modifyOperations) - } + buffer(List(new CatalogSinkModifyOperation(identifier, table.getQueryOperation))) } override def sqlUpdate(stmt: String): Unit = { @@ -1030,12 +1024,7 @@ class TestingTableEnvironment private( val operation = operations.get(0) operation match { case modifyOperation: ModifyOperation => - val modifyOperations = List(modifyOperation) - if (isEagerOperationTranslation) { - translate(modifyOperations) - } else { - buffer(modifyOperations) - } + buffer(List(modifyOperation)) case createOperation: CreateTableOperation => catalogManager.createTable( createOperation.getCatalogTable, @@ -1052,25 +1041,16 @@ class TestingTableEnvironment private( @throws[Exception] override def execute(jobName: String): JobExecutionResult = { - translate(bufferedOperations.toList) + val transformations = planner.translate(bufferedOperations) bufferedOperations.clear() - execEnv.execute(jobName) - } - - // for test - def translate(): Unit = { - translate(bufferedOperations.toList) + val pipeline = executor.createPipeline(transformations, tableConfig, jobName) + execEnv.execute(pipeline) } override def createTable(tableOperation: QueryOperation): TableImpl = { super.createTable(tableOperation) } - private def translate(modifyOperations: List[ModifyOperation]): Unit = { - val transformations = planner.translate(modifyOperations) - execEnv.apply(transformations) - } - private def buffer(modifyOperations: List[ModifyOperation]): Unit = { bufferedOperations.addAll(modifyOperations) } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/executor/StreamExecutor.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/executor/StreamExecutor.java index 426e09e558..502d58768d 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/executor/StreamExecutor.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/executor/StreamExecutor.java @@ -21,8 +21,12 @@ package org.apache.flink.table.executor; import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.dag.Pipeline; import org.apache.flink.api.dag.Transformation; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.graph.StreamGraph; +import org.apache.flink.streaming.api.graph.StreamGraphGenerator; +import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.delegation.Executor; import java.util.List; @@ -41,13 +45,24 @@ public class StreamExecutor implements Executor { } @Override - public void apply(List> transformations) { - transformations.forEach(executionEnvironment::addOperator); + public Pipeline createPipeline(List> transformations, TableConfig tableConfig, String jobName) { + if (transformations.size() <= 0) { + throw new IllegalStateException("No operators defined in streaming topology. Cannot generate StreamGraph."); + } + return new StreamGraphGenerator( + transformations, executionEnvironment.getConfig(), executionEnvironment.getCheckpointConfig()) + .setStateBackend(executionEnvironment.getStateBackend()) + .setChaining(executionEnvironment.isChainingEnabled()) + .setUserArtifacts(executionEnvironment.getCachedFiles()) + .setTimeCharacteristic(executionEnvironment.getStreamTimeCharacteristic()) + .setDefaultBufferTimeout(executionEnvironment.getBufferTimeout()) + .setJobName(jobName) + .generate(); } @Override - public JobExecutionResult execute(String jobName) throws Exception { - return executionEnvironment.execute(jobName); + public JobExecutionResult execute(Pipeline pipeline) throws Exception { + return executionEnvironment.execute((StreamGraph) pipeline); } public StreamExecutionEnvironment getExecutionEnvironment() { diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/util/DummyStreamExecutionEnvironment.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/util/DummyStreamExecutionEnvironment.java new file mode 100644 index 0000000000..0f3a9e85db --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/util/DummyStreamExecutionEnvironment.java @@ -0,0 +1,298 @@ +/* + * 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.table.util; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.cache.DistributedCache; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.state.AbstractStateBackend; +import org.apache.flink.runtime.state.StateBackend; +import org.apache.flink.streaming.api.CheckpointingMode; +import org.apache.flink.streaming.api.TimeCharacteristic; +import org.apache.flink.streaming.api.environment.CheckpointConfig; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.graph.StreamGraph; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.delegation.Planner; +import org.apache.flink.table.sinks.StreamTableSink; +import org.apache.flink.table.sources.StreamTableSource; + +import com.esotericsoftware.kryo.Serializer; + +import java.io.Serializable; +import java.util.List; + +/** + * This is dummy {@link StreamExecutionEnvironment}, which holds a real {@link StreamExecutionEnvironment}, + * shares all configurations of the real environment, and disables all configuration setting methods. + * + *

    When translating relational plan to execution plan in the {@link Planner}, the generated {@link Transformation}s + * will be added into StreamExecutionEnvironment's buffer, and they will be cleared only when + * {@link StreamExecutionEnvironment#execute()} method is called. Each {@link TableEnvironment} instance holds + * an immutable StreamExecutionEnvironment instance. If there are multiple translations (not all for `execute`, + * e.g. `explain` and then `execute`) in one TableEnvironment instance, the transformation buffer is dirty, + * and execution result may be incorrect. + * + *

    This dummy StreamExecutionEnvironment is only used for buffering the transformations generated in the planner. + * A new dummy StreamExecutionEnvironment instance should be created for each translation, and this could avoid + * dirty the buffer of the real StreamExecutionEnvironment instance. + * + *

    All set methods (e.g. `setXX`, `enableXX`, `disableXX`, etc) are disabled to prohibit changing configuration, + * all get methods (e.g. `getXX`, `isXX`, etc) will be delegated to the real StreamExecutionEnvironment. + * `execute`, `getStreamGraph` methods are also disabled, while `addOperator` method is enabled to + * allow the planner to add the generated transformations to the dummy StreamExecutionEnvironment. + * + *

    This class could be removed once the {@link StreamTableSource} interface and {@link StreamTableSink} interface + * are reworked. + * + *

    NOTE: Please remove {@code com.esotericsoftware.kryo} item in the whitelist of checkCodeDependencies() + * method in {@code test_table_shaded_dependencies.sh} end-to-end test when this class is removed. + */ +public class DummyStreamExecutionEnvironment extends StreamExecutionEnvironment { + + private final StreamExecutionEnvironment realExecEnv; + + public DummyStreamExecutionEnvironment(StreamExecutionEnvironment realExecEnv) { + this.realExecEnv = realExecEnv; + } + + @Override + public ExecutionConfig getConfig() { + return realExecEnv.getConfig(); + } + + @Override + public List> getCachedFiles() { + return realExecEnv.getCachedFiles(); + } + + @Override + public StreamExecutionEnvironment setParallelism(int parallelism) { + throw new UnsupportedOperationException( + "This is a dummy StreamExecutionEnvironment, setParallelism method is unsupported."); + } + + @Override + public StreamExecutionEnvironment setMaxParallelism(int maxParallelism) { + throw new UnsupportedOperationException( + "This is a dummy StreamExecutionEnvironment, setMaxParallelism method is unsupported."); + } + + @Override + public int getParallelism() { + return realExecEnv.getParallelism(); + } + + @Override + public int getMaxParallelism() { + return realExecEnv.getMaxParallelism(); + } + + @Override + public StreamExecutionEnvironment setBufferTimeout(long timeoutMillis) { + throw new UnsupportedOperationException( + "This is a dummy StreamExecutionEnvironment, setBufferTimeout method is unsupported."); + } + + @Override + public long getBufferTimeout() { + return realExecEnv.getBufferTimeout(); + } + + @Override + public StreamExecutionEnvironment disableOperatorChaining() { + throw new UnsupportedOperationException( + "This is a dummy StreamExecutionEnvironment, disableOperatorChaining method is unsupported."); + } + + @Override + public boolean isChainingEnabled() { + return realExecEnv.isChainingEnabled(); + } + + @Override + public CheckpointConfig getCheckpointConfig() { + return realExecEnv.getCheckpointConfig(); + } + + @Override + public StreamExecutionEnvironment enableCheckpointing(long interval) { + throw new UnsupportedOperationException( + "This is a dummy StreamExecutionEnvironment, enableCheckpointing method is unsupported."); + } + + @Override + public StreamExecutionEnvironment enableCheckpointing(long interval, CheckpointingMode mode) { + throw new UnsupportedOperationException( + "This is a dummy StreamExecutionEnvironment, enableCheckpointing method is unsupported."); + } + + @Override + public StreamExecutionEnvironment enableCheckpointing(long interval, CheckpointingMode mode, boolean force) { + throw new UnsupportedOperationException( + "This is a dummy StreamExecutionEnvironment, enableCheckpointing method is unsupported."); + } + + @Override + public StreamExecutionEnvironment enableCheckpointing() { + throw new UnsupportedOperationException( + "This is a dummy StreamExecutionEnvironment, enableCheckpointing method is unsupported."); + } + + @Override + public long getCheckpointInterval() { + return realExecEnv.getCheckpointInterval(); + } + + @Override + public boolean isForceCheckpointing() { + return realExecEnv.isForceCheckpointing(); + } + + @Override + public CheckpointingMode getCheckpointingMode() { + return realExecEnv.getCheckpointingMode(); + } + + @Override + public StreamExecutionEnvironment setStateBackend(StateBackend backend) { + throw new UnsupportedOperationException( + "This is a dummy StreamExecutionEnvironment, setStateBackend method is unsupported."); + } + + @Override + public StreamExecutionEnvironment setStateBackend(AbstractStateBackend backend) { + throw new UnsupportedOperationException( + "This is a dummy StreamExecutionEnvironment, setStateBackend method is unsupported."); + } + + @Override + public StateBackend getStateBackend() { + return realExecEnv.getStateBackend(); + } + + @Override + public void setRestartStrategy(RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration) { + throw new UnsupportedOperationException( + "This is a dummy StreamExecutionEnvironment, setRestartStrategy method is unsupported."); + } + + @Override + public RestartStrategies.RestartStrategyConfiguration getRestartStrategy() { + return realExecEnv.getRestartStrategy(); + } + + @Override + public void setNumberOfExecutionRetries(int numberOfExecutionRetries) { + throw new UnsupportedOperationException( + "This is a dummy StreamExecutionEnvironment, setNumberOfExecutionRetries method is unsupported."); + } + + @Override + public int getNumberOfExecutionRetries() { + return realExecEnv.getNumberOfExecutionRetries(); + } + + @Override + public & Serializable> void addDefaultKryoSerializer(Class type, T serializer) { + throw new UnsupportedOperationException( + "This is a dummy StreamExecutionEnvironment, addDefaultKryoSerializer method is unsupported."); + } + + @Override + public void addDefaultKryoSerializer(Class type, Class> serializerClass) { + throw new UnsupportedOperationException( + "This is a dummy StreamExecutionEnvironment, addDefaultKryoSerializer method is unsupported."); + } + + @Override + public & Serializable> void registerTypeWithKryoSerializer(Class type, T serializer) { + throw new UnsupportedOperationException( + "This is a dummy StreamExecutionEnvironment, registerTypeWithKryoSerializer method is unsupported."); + } + + @Override + public void registerTypeWithKryoSerializer(Class type, Class serializerClass) { + throw new UnsupportedOperationException( + "This is a dummy StreamExecutionEnvironment, registerTypeWithKryoSerializer method is unsupported."); + } + + @Override + public void registerType(Class type) { + throw new UnsupportedOperationException( + "This is a dummy StreamExecutionEnvironment, registerType method is unsupported."); + } + + @Override + public void setStreamTimeCharacteristic(TimeCharacteristic characteristic) { + throw new UnsupportedOperationException( + "This is a dummy StreamExecutionEnvironment, setStreamTimeCharacteristic method is unsupported."); + } + + @Override + public TimeCharacteristic getStreamTimeCharacteristic() { + return realExecEnv.getStreamTimeCharacteristic(); + } + + @Override + public JobExecutionResult execute() throws Exception { + throw new UnsupportedOperationException( + "This is a dummy StreamExecutionEnvironment, execute method is unsupported."); + } + + @Override + public JobExecutionResult execute(String jobName) throws Exception { + throw new UnsupportedOperationException( + "This is a dummy StreamExecutionEnvironment, execute method is unsupported."); + } + + @Override + public JobExecutionResult execute(StreamGraph streamGraph) throws Exception { + throw new UnsupportedOperationException( + "This is a dummy StreamExecutionEnvironment, execute method is unsupported."); + } + + @Override + public void registerCachedFile(String filePath, String name) { + throw new UnsupportedOperationException( + "This is a dummy StreamExecutionEnvironment, registerCachedFile method is unsupported."); + } + + @Override + public void registerCachedFile(String filePath, String name, boolean executable) { + throw new UnsupportedOperationException( + "This is a dummy StreamExecutionEnvironment, registerCachedFile method is unsupported."); + } + + @Override + public StreamGraph getStreamGraph() { + throw new UnsupportedOperationException( + "This is a dummy StreamExecutionEnvironment, getStreamGraph method is unsupported."); + } + + @Override + public StreamGraph getStreamGraph(String jobName) { + throw new UnsupportedOperationException( + "This is a dummy StreamExecutionEnvironment, getStreamGraph method is unsupported."); + } + +} diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/StreamPlanner.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/StreamPlanner.scala index 0e53e79025..7d4dc2d6ba 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/StreamPlanner.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/StreamPlanner.scala @@ -39,7 +39,7 @@ import org.apache.flink.table.plan.util.UpdatingPlanChecker import org.apache.flink.table.runtime.types.CRow import org.apache.flink.table.sinks._ import org.apache.flink.table.types.utils.TypeConversions -import org.apache.flink.table.util.JavaScalaConversionUtil +import org.apache.flink.table.util.{DummyStreamExecutionEnvironment, JavaScalaConversionUtil} import org.apache.calcite.jdbc.CalciteSchema import org.apache.calcite.jdbc.CalciteSchemaBuilder.asRootSchema @@ -120,6 +120,8 @@ class StreamPlanner( operations.asScala.map { case queryOperation: QueryOperation => explain(queryOperation, unwrapQueryConfig) + case modifyOperation: ModifyOperation => + explain(modifyOperation.getChild, unwrapQueryConfig) case operation => throw new TableException(s"${operation.getClass.getCanonicalName} is not supported") }.mkString(s"${System.lineSeparator}${System.lineSeparator}") @@ -241,13 +243,14 @@ class StreamPlanner( private def translateToCRow( logicalPlan: RelNode, queryConfig: StreamQueryConfig): DataStream[CRow] = { + val planner = createDummyPlanner() logicalPlan match { case node: DataStreamRel => getExecutionEnvironment.configure( config.getConfiguration, Thread.currentThread().getContextClassLoader) - node.translateToPlan(this, queryConfig) + node.translateToPlan(planner, queryConfig) case _ => throw new TableException("Cannot generate DataStream due to an invalid logical plan. " + "This is a bug and should not happen. Please file an issue.") @@ -446,4 +449,10 @@ class StreamPlanner( case _ => None } } + + private def createDummyPlanner(): StreamPlanner = { + val dummyExecEnv = new DummyStreamExecutionEnvironment(getExecutionEnvironment) + val executor = new StreamExecutor(dummyExecEnv) + new StreamPlanner(executor, config, functionCatalog, catalogManager) + } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableEnvironmentITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableEnvironmentITCase.scala new file mode 100644 index 0000000000..ef604b35dd --- /dev/null +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableEnvironmentITCase.scala @@ -0,0 +1,362 @@ +/* + * 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.table.api + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.common.typeinfo.Types.STRING +import org.apache.flink.api.scala._ +import org.apache.flink.core.fs.FileSystem.WriteMode +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment +import org.apache.flink.streaming.api.scala.{StreamExecutionEnvironment => ScalaStreamExecutionEnvironment} +import org.apache.flink.table.api.TableEnvironmentITCase.{getPersonCsvTableSource, getPersonData, readFromResource, replaceStageId} +import org.apache.flink.table.api.internal.TableEnvironmentImpl +import org.apache.flink.table.api.java.StreamTableEnvironment +import org.apache.flink.table.api.scala.{StreamTableEnvironment => ScalaStreamTableEnvironment, _} +import org.apache.flink.table.runtime.utils.StreamITCase +import org.apache.flink.table.sinks.CsvTableSink +import org.apache.flink.table.sources.CsvTableSource +import org.apache.flink.types.Row +import org.apache.flink.util.FileUtils + +import org.junit.Assert.{assertEquals, assertFalse, assertTrue} +import org.junit.rules.TemporaryFolder +import org.junit.runner.RunWith +import org.junit.runners.Parameterized +import org.junit.{Before, Rule, Test} + +import _root_.java.io.{File, FileOutputStream, OutputStreamWriter} +import _root_.java.util + +import _root_.scala.collection.mutable +import _root_.scala.io.Source + + +@RunWith(classOf[Parameterized]) +class TableEnvironmentITCase(tableEnvName: String) { + + private val _tempFolder = new TemporaryFolder() + + @Rule + def tempFolder: TemporaryFolder = _tempFolder + + var tEnv: TableEnvironment = _ + + private val settings = + EnvironmentSettings.newInstance().useOldPlanner().inStreamingMode().build() + + @Before + def setup(): Unit = { + tableEnvName match { + case "TableEnvironment" => + tEnv = TableEnvironmentImpl.create(settings) + case "StreamTableEnvironment" => + tEnv = StreamTableEnvironment.create( + StreamExecutionEnvironment.getExecutionEnvironment, settings) + case _ => throw new UnsupportedOperationException("unsupported tableEnvName: " + tableEnvName) + } + tEnv.registerTableSource("MyTable", getPersonCsvTableSource) + } + + @Test + def testExecuteTwiceUsingSameTableEnv(): Unit = { + val sink1Path = registerCsvTableSink(tEnv, Array("first"), Array(STRING), "MySink1") + val sink2Path = registerCsvTableSink(tEnv, Array("last"), Array(STRING), "MySink2") + checkEmptyFile(sink1Path) + checkEmptyFile(sink2Path) + + val table1 = tEnv.sqlQuery("select first from MyTable") + tEnv.insertInto(table1, "MySink1") + tEnv.execute("test1") + assertFirstValues(sink1Path) + checkEmptyFile(sink2Path) + + // delete first csv file + new File(sink1Path).delete() + assertFalse(new File(sink1Path).exists()) + + val table2 = tEnv.sqlQuery("select last from MyTable") + tEnv.insertInto(table2, "MySink2") + tEnv.execute("test2") + assertFalse(new File(sink1Path).exists()) + assertLastValues(sink2Path) + } + + @Test + def testExplainAndExecuteSingleSink(): Unit = { + val sinkPath = registerCsvTableSink(tEnv, Array("first"), Array(STRING), "MySink1") + + val table1 = tEnv.sqlQuery("select first from MyTable") + tEnv.insertInto(table1, "MySink1") + + tEnv.explain(false) + tEnv.execute("test1") + assertFirstValues(sinkPath) + } + + @Test + def testExplainAndExecuteMultipleSink(): Unit = { + val sink1Path = registerCsvTableSink(tEnv, Array("first"), Array(STRING), "MySink1") + val sink2Path = registerCsvTableSink(tEnv, Array("first"), Array(STRING), "MySink2") + + val table1 = tEnv.sqlQuery("select first from MyTable") + tEnv.insertInto(table1, "MySink1") + val table2 = tEnv.sqlQuery("select last from MyTable") + tEnv.insertInto(table2, "MySink2") + + tEnv.explain(false) + tEnv.execute("test1") + assertFirstValues(sink1Path) + assertLastValues(sink2Path) + } + + @Test + def testExplainTwice(): Unit = { + registerCsvTableSink(tEnv, Array("first"), Array(STRING), "MySink1") + registerCsvTableSink(tEnv, Array("first"), Array(STRING), "MySink2") + + val table1 = tEnv.sqlQuery("select first from MyTable") + tEnv.insertInto(table1, "MySink1") + val table2 = tEnv.sqlQuery("select last from MyTable") + tEnv.insertInto(table2, "MySink2") + + val result1 = tEnv.explain(false) + val result2 = tEnv.explain(false) + assertEquals(replaceStageId(result1), replaceStageId(result2)) + } + + @Test + def testSqlUpdateAndToDataStream(): Unit = { + if (!tableEnvName.equals("StreamTableEnvironment")) { + return + } + val streamEnv = StreamExecutionEnvironment.getExecutionEnvironment + val streamTableEnv = StreamTableEnvironment.create(streamEnv, settings) + streamTableEnv.registerTableSource("MyTable", getPersonCsvTableSource) + val sink1Path = registerCsvTableSink(streamTableEnv, Array("first"), Array(STRING), "MySink1") + checkEmptyFile(sink1Path) + StreamITCase.clear + + streamTableEnv.sqlUpdate("insert into MySink1 select first from MyTable") + + val table = streamTableEnv.sqlQuery("select last from MyTable where id > 0") + val resultSet = streamTableEnv.toAppendStream(table, classOf[Row]) + resultSet.addSink(new StreamITCase.StringSink[Row]) + + val explain = streamTableEnv.explain(false) + assertEquals( + replaceStageId(readFromResource("testSqlUpdateAndToDataStream.out")), + replaceStageId(explain)) + + streamTableEnv.execute("test1") + assertFirstValues(sink1Path) + + // the DataStream program is not executed + assertTrue(StreamITCase.testResults.isEmpty) + + deleteFile(sink1Path) + + streamEnv.execute("test2") + assertEquals(getExpectedLastValues.sorted, StreamITCase.testResults.sorted) + // the table program is not executed again + assertFileNotExist(sink1Path) + } + + @Test + def testToDataStreamAndSqlUpdate(): Unit = { + if (!tableEnvName.equals("StreamTableEnvironment")) { + return + } + val streamEnv = StreamExecutionEnvironment.getExecutionEnvironment + val streamTableEnv = StreamTableEnvironment.create(streamEnv, settings) + streamTableEnv.registerTableSource("MyTable", getPersonCsvTableSource) + val sink1Path = registerCsvTableSink(streamTableEnv, Array("first"), Array(STRING), "MySink1") + checkEmptyFile(sink1Path) + StreamITCase.clear + + val table = streamTableEnv.sqlQuery("select last from MyTable where id > 0") + val resultSet = streamTableEnv.toAppendStream(table, classOf[Row]) + resultSet.addSink(new StreamITCase.StringSink[Row]) + + streamTableEnv.sqlUpdate("insert into MySink1 select first from MyTable") + + val explain = streamTableEnv.explain(false) + assertEquals( + replaceStageId(readFromResource("testSqlUpdateAndToDataStream.out")), + replaceStageId(explain)) + + streamEnv.execute("test2") + // the table program is not executed + checkEmptyFile(sink1Path) + assertEquals(getExpectedLastValues.sorted, StreamITCase.testResults.sorted) + StreamITCase.testResults.clear() + + streamTableEnv.execute("test1") + assertFirstValues(sink1Path) + // the DataStream program is not executed again + assertTrue(StreamITCase.testResults.isEmpty) + } + + @Test + def testFromToDataStreamAndSqlUpdate(): Unit = { + if (!tableEnvName.equals("StreamTableEnvironment")) { + return + } + val streamEnv = ScalaStreamExecutionEnvironment.getExecutionEnvironment + val streamTableEnv = ScalaStreamTableEnvironment.create(streamEnv, settings) + val t = streamEnv.fromCollection(getPersonData) + .toTable(streamTableEnv, 'first, 'id, 'score, 'last) + streamTableEnv.registerTable("MyTable", t) + val sink1Path = registerCsvTableSink(streamTableEnv, Array("first"), Array(STRING), "MySink1") + checkEmptyFile(sink1Path) + StreamITCase.clear + + val table = streamTableEnv.sqlQuery("select last from MyTable where id > 0") + val resultSet = streamTableEnv.toAppendStream[Row](table) + resultSet.addSink(new StreamITCase.StringSink[Row]) + + streamTableEnv.sqlUpdate("insert into MySink1 select first from MyTable") + + val explain = streamTableEnv.explain(false) + assertEquals( + replaceStageId(readFromResource("testFromToDataStreamAndSqlUpdate.out")), + replaceStageId(explain).replaceAll("Scan\\(id=\\[\\d+\\], ", "Scan(")) + + streamEnv.execute("test2") + // the table program is not executed + checkEmptyFile(sink1Path) + assertEquals(getExpectedLastValues.sorted, StreamITCase.testResults.sorted) + StreamITCase.testResults.clear() + + streamTableEnv.execute("test1") + assertFirstValues(sink1Path) + // the DataStream program is not executed again + assertTrue(StreamITCase.testResults.isEmpty) + } + + private def registerCsvTableSink( + tEnv: TableEnvironment, + fieldNames: Array[String], + fieldTypes: Array[TypeInformation[_]], + tableName: String): String = { + val resultFile = _tempFolder.newFile() + val path = resultFile.getAbsolutePath + + val configuredSink = new CsvTableSink(path, ",", 1, WriteMode.OVERWRITE) + .configure(fieldNames, fieldTypes) + tEnv.registerTableSink(tableName, configuredSink) + + path + } + + private def assertFirstValues(csvFilePath: String): Unit = { + val expected = List("Mike", "Bob", "Sam", "Peter", "Liz", "Sally", "Alice", "Kelly") + val actual = FileUtils.readFileUtf8(new File(csvFilePath)).split("\n").toList + assertEquals(expected.sorted, actual.sorted) + } + + private def assertLastValues(csvFilePath: String): Unit = { + val actual = FileUtils.readFileUtf8(new File(csvFilePath)).split("\n").toList + assertEquals(getExpectedLastValues.sorted, actual.sorted) + } + + private def getExpectedLastValues: List[String] = { + List("Smith", "Taylor", "Miller", "Smith", "Williams", "Miller", "Smith", "Williams") + } + + private def checkEmptyFile(csvFilePath: String): Unit = { + assertTrue(FileUtils.readFileUtf8(new File(csvFilePath)).isEmpty) + } + + private def deleteFile(path: String): Unit = { + new File(path).delete() + assertFalse(new File(path).exists()) + } + + private def assertFileNotExist(path: String): Unit = { + assertFalse(new File(path).exists()) + } +} + +object TableEnvironmentITCase { + @Parameterized.Parameters(name = "{0}") + def parameters(): util.Collection[Array[_]] = { + util.Arrays.asList( + Array("TableEnvironment"), + Array("StreamTableEnvironment") + ) + } + + def readFromResource(file: String): String = { + val source = s"${getClass.getResource("/").getFile}../../src/test/scala/resources/$file" + Source.fromFile(source).mkString + } + + def replaceStageId(s: String): String = { + s.replaceAll("\\r\\n", "\n").replaceAll("Stage \\d+", "") + } + + def getPersonCsvTableSource: CsvTableSource = { + val header = "First#Id#Score#Last" + val csvRecords = getPersonData.map { + case (first, id, score, last) => s"$first#$id#$score#$last" + } + + val tempFilePath = writeToTempFile( + header + "$" + csvRecords.mkString("$"), + "csv-test", + "tmp") + CsvTableSource.builder() + .path(tempFilePath) + .field("first", Types.STRING) + .field("id", Types.INT) + .field("score", Types.DOUBLE) + .field("last", Types.STRING) + .fieldDelimiter("#") + .lineDelimiter("$") + .ignoreFirstLine() + .commentPrefix("%") + .build() + } + + def getPersonData: List[(String, Int, Double, String)] = { + val data = new mutable.MutableList[(String, Int, Double, String)] + data.+=(("Mike", 1, 12.3, "Smith")) + data.+=(("Bob", 2, 45.6, "Taylor")) + data.+=(("Sam", 3, 7.89, "Miller")) + data.+=(("Peter", 4, 0.12, "Smith")) + data.+=(("Liz", 5, 34.5, "Williams")) + data.+=(("Sally", 6, 6.78, "Miller")) + data.+=(("Alice", 7, 90.1, "Smith")) + data.+=(("Kelly", 8, 2.34, "Williams")) + data.toList + } + + private def writeToTempFile( + contents: String, + filePrefix: String, + fileSuffix: String, + charset: String = "UTF-8"): String = { + val tempFile = File.createTempFile(filePrefix, fileSuffix) + tempFile.deleteOnExit() + val tmpWriter = new OutputStreamWriter(new FileOutputStream(tempFile), charset) + tmpWriter.write(contents) + tmpWriter.close() + tempFile.getAbsolutePath + } +} diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/TableFactoryTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/TableFactoryTest.scala index 9bdbfd1444..419470bced 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/TableFactoryTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/TableFactoryTest.scala @@ -68,6 +68,8 @@ class TableFactoryTest extends TableTestBase { util.tableEnv.sqlUpdate(sourceDDL) util.tableEnv.sqlUpdate(sinkDDL) util.tableEnv.sqlUpdate(query) + // trigger translating + util.tableEnv.execute("job name") Assert.assertTrue(factory.hasInvokedSource) Assert.assertTrue(factory.hasInvokedSink) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/validation/InsertIntoValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/validation/InsertIntoValidationTest.scala index 4cf907daa0..178cc244ee 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/validation/InsertIntoValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/validation/InsertIntoValidationTest.scala @@ -45,6 +45,8 @@ class InsertIntoValidationTest { // must fail because table sink has too few fields. tEnv.sqlUpdate(sql) + // trigger translation + tEnv.execute("job name") } @Test(expected = classOf[ValidationException]) @@ -64,6 +66,8 @@ class InsertIntoValidationTest { // must fail because field types of table sink are incompatible. tEnv.sqlUpdate(sql) + // trigger translation + tEnv.execute("job name") } @Test(expected = classOf[ValidationException]) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/table/validation/InsertIntoValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/table/validation/InsertIntoValidationTest.scala index e3ffcf9efa..3e2efc772e 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/table/validation/InsertIntoValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/table/validation/InsertIntoValidationTest.scala @@ -45,6 +45,9 @@ class InsertIntoValidationTest { tEnv.scan("sourceTable") .select('a, 'b, 'c) .insertInto("targetTable") + + // trigger translation + tEnv.execute("job name") } @Test(expected = classOf[ValidationException]) @@ -64,5 +67,8 @@ class InsertIntoValidationTest { tEnv.scan("sourceTable") .select('a, 'b, 'c) .insertInto("targetTable") + + // trigger translation + tEnv.execute("job name") } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/catalog/CatalogTableITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/catalog/CatalogTableITCase.scala index e406afd3a3..55dc36c838 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/catalog/CatalogTableITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/catalog/CatalogTableITCase.scala @@ -90,7 +90,7 @@ class CatalogTableITCase(isStreaming: Boolean) extends AbstractTestBase { def execJob(name: String) = { if (isStreaming) { - streamExec.execute(name) + tableEnv.execute(name) } else { batchExec.execute(name) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/TimeAttributesITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/TimeAttributesITCase.scala index b3328181b0..6960221f94 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/TimeAttributesITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/TimeAttributesITCase.scala @@ -206,7 +206,7 @@ class TimeAttributesITCase extends AbstractTestBase { 'rowtime.ceil(TimeIntervalUnit.DAY).as('ceilDay)) .insertInto("testSink") - env.execute() + tEnv.execute("job name") val expected = Seq( "1970-01-01 00:00:00.007,1970-01-01 00:00:00.0,1970-01-02 00:00:00.0", diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/sql/InsertIntoITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/sql/InsertIntoITCase.scala index ef7fa62a15..87795447ba 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/sql/InsertIntoITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/sql/InsertIntoITCase.scala @@ -61,7 +61,7 @@ class InsertIntoITCase extends StreamingWithStateTestBase { |WHERE a < 3 OR a > 19 """.stripMargin) - env.execute() + tEnv.execute("job name") val expected = Seq( "Hi,1970-01-01 00:00:00.001,1", @@ -96,7 +96,7 @@ class InsertIntoITCase extends StreamingWithStateTestBase { |GROUP BY len """.stripMargin) - env.execute() + tEnv.execute("job name") val results = RowCollector.getAndClearValues val retracted = RowCollector.retractResults(results).sorted @@ -141,7 +141,7 @@ class InsertIntoITCase extends StreamingWithStateTestBase { |GROUP BY TUMBLE(rowtime, INTERVAL '0.005' SECOND) """.stripMargin) - env.execute() + tEnv.execute("job name") val results = RowCollector.getAndClearValues assertFalse( @@ -190,7 +190,7 @@ class InsertIntoITCase extends StreamingWithStateTestBase { |GROUP BY cnt, cTrue """.stripMargin) - env.execute() + tEnv.execute("job name") val results = RowCollector.getAndClearValues assertTrue( @@ -236,7 +236,7 @@ class InsertIntoITCase extends StreamingWithStateTestBase { |GROUP BY TUMBLE(rowtime, INTERVAL '0.005' SECOND), num """.stripMargin) - env.execute() + tEnv.execute("job name") val results = RowCollector.getAndClearValues assertFalse( @@ -288,7 +288,7 @@ class InsertIntoITCase extends StreamingWithStateTestBase { |GROUP BY TUMBLE(rowtime, INTERVAL '0.005' SECOND), num """.stripMargin) - env.execute() + tEnv.execute("job name") val results = RowCollector.getAndClearValues assertFalse( @@ -338,7 +338,7 @@ class InsertIntoITCase extends StreamingWithStateTestBase { |GROUP BY TUMBLE(rowtime, INTERVAL '0.005' SECOND), num """.stripMargin) - env.execute() + tEnv.execute("job name") val results = RowCollector.getAndClearValues assertFalse( @@ -388,7 +388,7 @@ class InsertIntoITCase extends StreamingWithStateTestBase { |GROUP BY TUMBLE(rowtime, INTERVAL '0.005' SECOND), num """.stripMargin) - env.execute() + tEnv.execute("job name") val results = RowCollector.getAndClearValues assertFalse( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/sql/SortITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/sql/SortITCase.scala index aa80a76521..f275187a90 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/sql/SortITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/sql/SortITCase.scala @@ -129,7 +129,7 @@ class SortITCase extends StreamingWithStateTestBase { val sql = "INSERT INTO targetTable SELECT a, b, c, rowtime " + "FROM sourceTable ORDER BY rowtime, a desc" tEnv.sqlUpdate(sql) - env.execute() + tEnv.execute("job name") val expected = List( "1,1,Hi,1970-01-01 00:00:00.001", diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala index ff25a0afbe..c18fff18b2 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala @@ -783,6 +783,7 @@ class SqlITCase extends StreamingWithStateTestBase { new InMemoryTableFactory(3).createStreamTableSink(properties)) tEnv.sqlUpdate("INSERT INTO targetTable SELECT a, b, c, rowtime FROM sourceTable") + tEnv.execute("job name") tEnv.sqlQuery("SELECT a, e, f, t from targetTable") .addSink(new StreamITCase.StringSink[Row]) env.execute() diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/table/AggregateITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/table/AggregateITCase.scala index efb6c1ba1e..1176a401a1 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/table/AggregateITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/table/AggregateITCase.scala @@ -333,7 +333,7 @@ class AggregateITCase extends StreamingWithStateTestBase { .select('c, 'b.max) t.insertInto("testSink") - env.execute() + tEnv.execute("test") val expected = List("(true,A,1)", "(true,B,2)", "(true,C,3)") assertEquals(expected.sorted, RowCollector.getAndClearValues.map(_.toString).sorted) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/table/JoinITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/table/JoinITCase.scala index b8fef219a0..2cd5d45f45 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/table/JoinITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/table/JoinITCase.scala @@ -98,7 +98,7 @@ class JoinITCase extends StreamingWithStateTestBase { .select('a, 'b, 'c) .insertInto("upsertSink", queryConfig) - env.execute() + tEnv.execute("job name") val results = RowCollector.getAndClearValues val retracted = RowCollector.upsertResults(results, Array(0, 1)) @@ -158,7 +158,7 @@ class JoinITCase extends StreamingWithStateTestBase { .select('a, 'b, 'c, 'd) .insertInto("retractSink", queryConfig) - env.execute() + tEnv.execute("job name") val results = RowCollector.getAndClearValues val retracted = RowCollector.retractResults(results) val expected = Seq("1,1,1,1", "1,1,1,1", "1,1,1,1", "1,1,1,1", "2,2,2,2", "3,3,3,3", diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSinkITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSinkITCase.scala index 3e6b6375f5..121027b05b 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSinkITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSinkITCase.scala @@ -68,7 +68,7 @@ class TableSinkITCase extends AbstractTestBase { .where('a < 3 || 'a > 19) .select('c, 't, 'b) .insertInto("targetTable") - env.execute() + tEnv.execute("job name") val expected = Seq( "Hi,1970-01-01 00:00:00.001,1", @@ -108,7 +108,7 @@ class TableSinkITCase extends AbstractTestBase { .select(ifThenElse('a < 4, nullOf(Types.INT()), 'a), 'c, 'b) .insertInto("csvSink") - env.execute() + tEnv.execute("job name") val expected = Seq( ",Hello world,1970-01-01 00:00:00.002", @@ -146,7 +146,7 @@ class TableSinkITCase extends AbstractTestBase { .select('w.end as 't, 'id.count as 'icnt, 'num.sum as 'nsum) .insertInto("appendSink") - env.execute() + tEnv.execute("job name") val result = RowCollector.getAndClearValues.map(_.f1.toString).sorted val expected = List( @@ -179,7 +179,7 @@ class TableSinkITCase extends AbstractTestBase { .select('c, 'g) .insertInto("appendSink") - env.execute() + tEnv.execute("job name") val result = RowCollector.getAndClearValues.map(_.f1.toString).sorted val expected = List("Hi,Hallo", "Hello,Hallo Welt", "Hello world,Hallo Welt").sorted @@ -208,7 +208,7 @@ class TableSinkITCase extends AbstractTestBase { .select('len, 'id.count as 'icnt, 'num.sum as 'nsum) .insertInto("retractSink") - env.execute() + tEnv.execute("job name") val results = RowCollector.getAndClearValues val retracted = RowCollector.retractResults(results).sorted @@ -246,7 +246,7 @@ class TableSinkITCase extends AbstractTestBase { .select('w.end as 't, 'id.count as 'icnt, 'num.sum as 'nsum) .insertInto("retractSink") - env.execute() + tEnv.execute("job name") val results = RowCollector.getAndClearValues assertFalse( @@ -290,7 +290,7 @@ class TableSinkITCase extends AbstractTestBase { .select('count, 'len.count as 'lencnt, 'cTrue) .insertInto("upsertSink") - env.execute() + tEnv.execute("job name") val results = RowCollector.getAndClearValues assertTrue( @@ -330,7 +330,7 @@ class TableSinkITCase extends AbstractTestBase { .select('num, 'w.end as 'window_end, 'id.count as 'icnt) .insertInto("upsertSink") - env.execute() + tEnv.execute("job name") val results = RowCollector.getAndClearValues assertFalse( @@ -375,7 +375,7 @@ class TableSinkITCase extends AbstractTestBase { .select('w.start as 'wstart, 'w.end as 'wend, 'num, 'id.count as 'icnt) .insertInto("upsertSink") - env.execute() + tEnv.execute("job name") val results = RowCollector.getAndClearValues assertFalse( @@ -419,7 +419,7 @@ class TableSinkITCase extends AbstractTestBase { .select('w.end as 'wend, 'id.count as 'cnt) .insertInto("upsertSink") - env.execute() + tEnv.execute("job name") val results = RowCollector.getAndClearValues assertFalse( @@ -463,7 +463,7 @@ class TableSinkITCase extends AbstractTestBase { .select('num, 'id.count as 'cnt) .insertInto("upsertSink") - env.execute() + tEnv.execute("job name") val results = RowCollector.getAndClearValues assertFalse( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MemoryTableSourceSinkUtil.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MemoryTableSourceSinkUtil.scala index 5b2e9d2e36..a835bba752 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MemoryTableSourceSinkUtil.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MemoryTableSourceSinkUtil.scala @@ -18,7 +18,6 @@ package org.apache.flink.table.utils -import java.util import org.apache.flink.api.common.io.{OutputFormat, RichOutputFormat} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.RowTypeInfo @@ -34,6 +33,8 @@ import org.apache.flink.table.sinks._ import org.apache.flink.table.sources._ import org.apache.flink.types.Row +import java.util + import scala.collection.JavaConverters._ import scala.collection.mutable diff --git a/flink-table/flink-table-planner/src/test/scala/resources/testFromToDataStreamAndSqlUpdate.out b/flink-table/flink-table-planner/src/test/scala/resources/testFromToDataStreamAndSqlUpdate.out new file mode 100644 index 0000000000..9cc6251710 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/scala/resources/testFromToDataStreamAndSqlUpdate.out @@ -0,0 +1,36 @@ +== Abstract Syntax Tree == +LogicalProject(first=[$0]) + FlinkLogicalDataStreamScan(fields=[first, id, score, last]) + +== Optimized Logical Plan == +DataStreamCalc(select=[first]) + DataStreamScan(fields=[first, id, score, last]) + +== Physical Execution Plan == + : Data Source + content : collect elements with CollectionInputFormat + + : Operator + content : from: (first, id, score, last) + ship_strategy : REBALANCE + + : Operator + content : where: (>(id, 0)), select: (last) + ship_strategy : FORWARD + + : Operator + content : to: Row + ship_strategy : FORWARD + + : Operator + content : from: (first, id, score, last) + ship_strategy : REBALANCE + + : Operator + content : select: (first) + ship_strategy : FORWARD + + : Data Sink + content : Sink: Unnamed + ship_strategy : FORWARD + diff --git a/flink-table/flink-table-planner/src/test/scala/resources/testSqlUpdateAndToDataStream.out b/flink-table/flink-table-planner/src/test/scala/resources/testSqlUpdateAndToDataStream.out new file mode 100644 index 0000000000..1ef95b0d2a --- /dev/null +++ b/flink-table/flink-table-planner/src/test/scala/resources/testSqlUpdateAndToDataStream.out @@ -0,0 +1,19 @@ +== Abstract Syntax Tree == +LogicalProject(first=[$0]) + EnumerableTableScan(table=[[default_catalog, default_database, MyTable]]) + +== Optimized Logical Plan == +StreamTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[first], source=[CsvTableSource(read fields: first)]) + +== Physical Execution Plan == + : Data Source + content : collect elements with CollectionInputFormat + + : Operator + content : CsvTableSource(read fields: first) + ship_strategy : REBALANCE + + : Operator + content : Map + ship_strategy : FORWARD + -- Gitee From 610c9fb405615cf3eae4b082cbbc6c81f06d0028 Mon Sep 17 00:00:00 2001 From: godfrey he Date: Wed, 11 Mar 2020 18:42:22 +0800 Subject: [PATCH 126/885] [FLINK-16519][checkpointing][tests] Remove PowerMock --- .../runtime/checkpoint/CheckpointCoordinatorFailureTest.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java index 754daf9b26..e3828224c0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java @@ -32,9 +32,6 @@ import org.apache.flink.runtime.state.OperatorStreamStateHandle; import org.apache.flink.util.TestLogger; import org.junit.Test; -import org.junit.runner.RunWith; -import org.powermock.core.classloader.annotations.PrepareForTest; -import org.powermock.modules.junit4.PowerMockRunner; import java.util.List; @@ -50,8 +47,6 @@ import static org.mockito.Mockito.when; /** * Tests for failure of checkpoint coordinator. */ -@RunWith(PowerMockRunner.class) -@PrepareForTest(PendingCheckpoint.class) public class CheckpointCoordinatorFailureTest extends TestLogger { /** -- Gitee From 2a4a2e22c65756d95d3c378d42be11bb583bc01d Mon Sep 17 00:00:00 2001 From: tison Date: Wed, 11 Mar 2020 19:00:24 +0800 Subject: [PATCH 127/885] [FLINK-14041][tests] Refactor LeaderRetrievalServiceHostnameResolutionTest - rename class and test cases for clarity - inline StandaloneUtils as it is just unnecessary --- .../flink/runtime/util/StandaloneUtils.java | 90 ------------------- .../runtime/util/AddressResolutionTest.java | 88 ++++++++---------- 2 files changed, 37 insertions(+), 141 deletions(-) delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/util/StandaloneUtils.java rename flink-clients/src/test/java/org/apache/flink/client/program/LeaderRetrievalServiceHostnameResolutionTest.java => flink-runtime/src/test/java/org/apache/flink/runtime/util/AddressResolutionTest.java (51%) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/StandaloneUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/StandaloneUtils.java deleted file mode 100644 index 1719b38c3b..0000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/StandaloneUtils.java +++ /dev/null @@ -1,90 +0,0 @@ -/* - * 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.runtime.util; - -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; -import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils.AddressResolution; -import org.apache.flink.runtime.jobmaster.JobMaster; -import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService; -import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils; -import org.apache.flink.util.ConfigurationException; - -import java.net.UnknownHostException; - -/** - * Utility class to work with Flink standalone mode. - */ -public final class StandaloneUtils { - - /** - * Creates a {@link StandaloneLeaderRetrievalService} from the given configuration. The - * host and port for the remote Akka URL are retrieved from the provided configuration. - * - * @param configuration Configuration instance containing the host and port information - * @return StandaloneLeaderRetrievalService - * @throws ConfigurationException - * @throws UnknownHostException - */ - public static StandaloneLeaderRetrievalService createLeaderRetrievalService(Configuration configuration) - throws ConfigurationException, UnknownHostException { - return createLeaderRetrievalService( - configuration, - false, - null); - } - - /** - * Creates a {@link StandaloneLeaderRetrievalService} form the given configuration and the - * JobManager name. The host and port for the remote Akka URL are retrieved from the provided - * configuration. Instead of using the standard JobManager Akka name, the provided one is used - * for the remote Akka URL. - * - * @param configuration Configuration instance containing hte host and port information - * @param resolveInitialHostName If true, resolves the hostname of the StandaloneLeaderRetrievalService - * @param jobManagerName Name of the JobManager actor - * @return StandaloneLeaderRetrievalService - * @throws ConfigurationException if the job manager address cannot be retrieved from the configuration - * @throws UnknownHostException if the job manager address cannot be resolved - */ - public static StandaloneLeaderRetrievalService createLeaderRetrievalService( - Configuration configuration, - boolean resolveInitialHostName, - String jobManagerName) - throws ConfigurationException, UnknownHostException { - Tuple2 hostnamePort = HighAvailabilityServicesUtils.getJobManagerAddress(configuration); - - String jobManagerAkkaUrl = AkkaRpcServiceUtils.getRpcUrl( - hostnamePort.f0, - hostnamePort.f1, - jobManagerName != null ? jobManagerName : JobMaster.JOB_MANAGER_NAME, - resolveInitialHostName ? AddressResolution.TRY_ADDRESS_RESOLUTION : AddressResolution.NO_ADDRESS_RESOLUTION, - configuration); - - return new StandaloneLeaderRetrievalService(jobManagerAkkaUrl); - } - - /** - * Private constructor to prevent instantiation. - */ - private StandaloneUtils() { - throw new RuntimeException(); - } -} diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/LeaderRetrievalServiceHostnameResolutionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/AddressResolutionTest.java similarity index 51% rename from flink-clients/src/test/java/org/apache/flink/client/program/LeaderRetrievalServiceHostnameResolutionTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/util/AddressResolutionTest.java index a62f9d8743..bec28a7863 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/LeaderRetrievalServiceHostnameResolutionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/AddressResolutionTest.java @@ -16,13 +16,11 @@ * limitations under the License. */ -package org.apache.flink.client.program; +package org.apache.flink.runtime.util; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.JobManagerOptions; -import org.apache.flink.runtime.jobmaster.JobMaster; -import org.apache.flink.runtime.util.StandaloneUtils; -import org.apache.flink.util.ConfigurationException; +import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; +import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils; import org.apache.flink.util.TestLogger; import org.junit.BeforeClass; @@ -35,70 +33,58 @@ import static org.junit.Assert.fail; import static org.junit.Assume.assumeTrue; /** - * Tests that verify that the LeaderRetrievalService correctly handles non-resolvable host names - * and does not fail with another exception. + * Unit tests for respecting {@link HighAvailabilityServicesUtils.AddressResolution}. */ -public class LeaderRetrievalServiceHostnameResolutionTest extends TestLogger { +public class AddressResolutionTest extends TestLogger { - private static final String nonExistingHostname = "foo.bar.com.invalid"; + private static final String ENDPOINT_NAME = "endpoint"; + private static final String NON_EXISTING_HOSTNAME = "foo.bar.com.invalid"; + private static final int PORT = 17234; @BeforeClass public static void check() { checkPreconditions(); } - /* - * Tests that the StandaloneLeaderRetrievalService resolves host names if specified. - */ - @Test - public void testUnresolvableHostname1() throws UnknownHostException, ConfigurationException { - Configuration config = new Configuration(); - - config.setString(JobManagerOptions.ADDRESS, nonExistingHostname); - config.setInteger(JobManagerOptions.PORT, 17234); - - StandaloneUtils.createLeaderRetrievalService( - config, - false, - JobMaster.JOB_MANAGER_NAME); - } - - /* - * Tests that the StandaloneLeaderRetrievalService does not resolve host names by default. - */ - @Test - public void testUnresolvableHostname2() throws Exception { - - try { - Configuration config = new Configuration(); - - config.setString(JobManagerOptions.ADDRESS, nonExistingHostname); - config.setInteger(JobManagerOptions.PORT, 17234); - - StandaloneUtils.createLeaderRetrievalService( - config, - true, - JobMaster.JOB_MANAGER_NAME); - fail("This should fail with an UnknownHostException"); - } - catch (UnknownHostException e) { - // that is what we want! - } - } - private static void checkPreconditions() { // the test can only work if the invalid URL cannot be resolves // some internet providers resolve unresolvable URLs to navigational aid servers, // voiding this test. boolean throwsException; + try { //noinspection ResultOfMethodCallIgnored - InetAddress.getByName(nonExistingHostname); + InetAddress.getByName(NON_EXISTING_HOSTNAME); throwsException = false; - } - catch (UnknownHostException e) { + } catch (UnknownHostException e) { throwsException = true; } + assumeTrue(throwsException); } + + @Test + public void testNoAddressResolution() throws UnknownHostException { + AkkaRpcServiceUtils.getRpcUrl( + NON_EXISTING_HOSTNAME, + PORT, + ENDPOINT_NAME, + HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION, + new Configuration()); + } + + @Test + public void testTryAddressResolution() { + try { + AkkaRpcServiceUtils.getRpcUrl( + NON_EXISTING_HOSTNAME, + PORT, + ENDPOINT_NAME, + HighAvailabilityServicesUtils.AddressResolution.TRY_ADDRESS_RESOLUTION, + new Configuration()); + fail("This should fail with an UnknownHostException"); + } catch (UnknownHostException ignore) { + // expected + } + } } -- Gitee From 5fa42593b79b57874237a784e8bce1d3ad632b60 Mon Sep 17 00:00:00 2001 From: hequn8128 Date: Wed, 11 Mar 2020 19:52:47 +0800 Subject: [PATCH 128/885] [FLINK-16477][hotfix][test] Add SerialVersionUID to the test Class to get stable functionIdentifier --- .../flink/table/planner/plan/batch/table/CorrelateTest.xml | 4 ++-- .../flink/table/planner/plan/stream/table/CorrelateTest.xml | 4 ++-- .../plan/rules/logical/ExpressionReductionRulesTest.scala | 1 + .../flink/table/planner/utils/UserDefinedTableFunctions.scala | 1 + .../flink/table/plan/ExpressionReductionRulesTest.scala | 1 + .../apache/flink/table/utils/UserDefinedTableFunctions.scala | 1 + 6 files changed, 8 insertions(+), 4 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/table/CorrelateTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/table/CorrelateTest.xml index 278a0fb72b..a01dce418e 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/table/CorrelateTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/table/CorrelateTest.xml @@ -132,12 +132,12 @@ Calc(select=[c, s]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/CorrelateTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/CorrelateTest.xml index efcaec1ac9..5e1aa3621b 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/CorrelateTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/CorrelateTest.xml @@ -211,12 +211,12 @@ Correlate(invocation=[org$apache$flink$table$planner$utils$PojoTableFunc$eb4ab6b diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ExpressionReductionRulesTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ExpressionReductionRulesTest.scala index 9a76699cae..13a42775b9 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ExpressionReductionRulesTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ExpressionReductionRulesTest.scala @@ -55,6 +55,7 @@ class ExpressionReductionRulesTest extends TableTestBase { } } +@SerialVersionUID(1L) object DeterministicPythonFunc extends ScalarFunction with PythonFunction { def eval(): Long = 1 diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/UserDefinedTableFunctions.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/UserDefinedTableFunctions.scala index a29e78145c..3293828589 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/UserDefinedTableFunctions.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/UserDefinedTableFunctions.scala @@ -114,6 +114,7 @@ class TableFunc3(data: String, conf: Map[String, String]) extends TableFunction[ } } +@SerialVersionUID(1L) class MockPythonTableFunction extends TableFunction[Row] with PythonFunction { def eval(x: Int, y: Int) = ??? diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/ExpressionReductionRulesTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/ExpressionReductionRulesTest.scala index 5e0cdba500..fa44b7814e 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/ExpressionReductionRulesTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/ExpressionReductionRulesTest.scala @@ -574,6 +574,7 @@ object DeterministicNullFunc extends ScalarFunction { override def isDeterministic = true } +@SerialVersionUID(1L) object DeterministicPythonFunc extends ScalarFunction with PythonFunction { def eval(): Long = 1L diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/UserDefinedTableFunctions.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/UserDefinedTableFunctions.scala index 3c82c36562..990d04dfd1 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/UserDefinedTableFunctions.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/UserDefinedTableFunctions.scala @@ -173,6 +173,7 @@ class PojoUser() { } } +@SerialVersionUID(1L) class MockPythonTableFunction extends TableFunction[Row] with PythonFunction { def eval(x: Int, y: Int) = ??? -- Gitee From b2c20e9cf5a58096d033bfe14cce277c938a53c6 Mon Sep 17 00:00:00 2001 From: HuangXingBo Date: Wed, 11 Mar 2020 22:06:32 +0800 Subject: [PATCH 129/885] [FLINK-16516][python] Avoid codegen user-defined function for Python UDF (#11358) --- .../pyflink/table/table_environment.py | 3 +- .../pyflink/table/tests/test_catalog.py | 2 +- flink-python/pyflink/table/udf.py | 80 +++--- .../python/PythonScalarFunction.java | 97 ++++++++ ...Function.java => PythonTableFunction.java} | 70 ++++-- .../codegen/PythonFunctionCodeGenerator.scala | 230 ------------------ .../plan/nodes/common/CommonPythonBase.scala | 10 +- .../utils/python/PythonTableUtils.scala | 64 +---- .../codegen/PythonFunctionCodeGenerator.scala | 230 ------------------ .../table/plan/nodes/CommonPythonBase.scala | 10 +- .../table/util/python/PythonTableUtils.scala | 58 ----- 11 files changed, 184 insertions(+), 670 deletions(-) create mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/python/PythonScalarFunction.java rename flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/python/{SimplePythonFunction.java => PythonTableFunction.java} (47%) delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/PythonFunctionCodeGenerator.scala delete mode 100644 flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/PythonFunctionCodeGenerator.scala diff --git a/flink-python/pyflink/table/table_environment.py b/flink-python/pyflink/table/table_environment.py index 6035a7752d..46aa70f4ea 100644 --- a/flink-python/pyflink/table/table_environment.py +++ b/flink-python/pyflink/table/table_environment.py @@ -770,8 +770,7 @@ class TableEnvironment(object): .. versionadded:: 1.10.0 """ - self._j_tenv.registerFunction(name, function.java_user_defined_function( - self._is_blink_planner, self.get_config()._j_table_config)) + self._j_tenv.registerFunction(name, function.java_user_defined_function()) def create_temporary_view(self, view_path, table): """ diff --git a/flink-python/pyflink/table/tests/test_catalog.py b/flink-python/pyflink/table/tests/test_catalog.py index ed0164b981..ebd0741ea3 100644 --- a/flink-python/pyflink/table/tests/test_catalog.py +++ b/flink-python/pyflink/table/tests/test_catalog.py @@ -180,7 +180,7 @@ class CatalogTestBase(PyFlinkTestCase): def create_function(): gateway = get_gateway() j_function = gateway.jvm.CatalogFunctionImpl( - "org.apache.flink.table.functions.python.SimplePythonFunction") + "org.apache.flink.table.functions.python.PythonScalarFunction") return CatalogFunction(j_function) @staticmethod diff --git a/flink-python/pyflink/table/udf.py b/flink-python/pyflink/table/udf.py index 091c495336..d150fa1ae4 100644 --- a/flink-python/pyflink/table/udf.py +++ b/flink-python/pyflink/table/udf.py @@ -163,7 +163,7 @@ class UserDefinedFunctionWrapper(object): self._deterministic = deterministic if deterministic is not None else ( func.is_deterministic() if isinstance(func, UserDefinedFunction) else True) - def java_user_defined_function(self, is_blink_planner, table_config): + def java_user_defined_function(self): pass @@ -183,12 +183,12 @@ class UserDefinedScalarFunctionWrapper(UserDefinedFunctionWrapper): self._udf_type = udf_type self._judf_placeholder = None - def java_user_defined_function(self, is_blink_planner, table_config): + def java_user_defined_function(self): if self._judf_placeholder is None: - self._judf_placeholder = self._create_judf(is_blink_planner, table_config) + self._judf_placeholder = self._create_judf() return self._judf_placeholder - def _create_judf(self, is_blink_planner, table_config): + def _create_judf(self): gateway = get_gateway() def get_python_function_kind(udf_type): @@ -212,29 +212,16 @@ class UserDefinedScalarFunctionWrapper(UserDefinedFunctionWrapper): [_to_java_type(i) for i in self._input_types]) j_result_type = _to_java_type(self._result_type) j_function_kind = get_python_function_kind(self._udf_type) - if is_blink_planner: - PythonTableUtils = gateway.jvm\ - .org.apache.flink.table.planner.utils.python.PythonTableUtils - j_scalar_function = PythonTableUtils \ - .createPythonScalarFunction(table_config, - self._name, - bytearray(serialized_func), - j_input_types, - j_result_type, - j_function_kind, - self._deterministic, - _get_python_env()) - else: - PythonTableUtils = gateway.jvm.PythonTableUtils - j_scalar_function = PythonTableUtils \ - .createPythonScalarFunction(self._name, - bytearray(serialized_func), - j_input_types, - j_result_type, - j_function_kind, - self._deterministic, - _get_python_env()) - + PythonScalarFunction = gateway.jvm \ + .org.apache.flink.table.functions.python.PythonScalarFunction + j_scalar_function = PythonScalarFunction( + self._name, + bytearray(serialized_func), + j_input_types, + j_result_type, + j_function_kind, + self._deterministic, + _get_python_env()) return j_scalar_function @@ -259,12 +246,12 @@ class UserDefinedTableFunctionWrapper(UserDefinedFunctionWrapper): self._result_types = result_types self._judtf_placeholder = None - def java_user_defined_function(self, is_blink_planner, table_config): + def java_user_defined_function(self): if self._judtf_placeholder is None: - self._judtf_placeholder = self._create_judtf(is_blink_planner, table_config) + self._judtf_placeholder = self._create_judtf() return self._judtf_placeholder - def _create_judtf(self, is_blink_planner, table_config): + def _create_judtf(self): func = self._func if not isinstance(self._func, UserDefinedFunction): func = DelegationTableFunction(self._func) @@ -278,26 +265,19 @@ class UserDefinedTableFunctionWrapper(UserDefinedFunctionWrapper): j_result_types = utils.to_jarray(gateway.jvm.TypeInformation, [_to_java_type(i) for i in self._result_types]) - if is_blink_planner: - PythonTableUtils = gateway.jvm \ - .org.apache.flink.table.planner.utils.python.PythonTableUtils - j_table_function = PythonTableUtils \ - .createPythonTableFunction(table_config, - self._name, - bytearray(serialized_func), - j_input_types, - j_result_types, - self._deterministic, - _get_python_env()) - else: - PythonTableUtils = gateway.jvm.PythonTableUtils - j_table_function = PythonTableUtils \ - .createPythonTableFunction(self._name, - bytearray(serialized_func), - j_input_types, - j_result_types, - self._deterministic, - _get_python_env()) + j_result_type = gateway.jvm.org.apache.flink.api.java.typeutils.RowTypeInfo(j_result_types) + j_function_kind = gateway.jvm.org.apache.flink.table.functions.python. \ + PythonFunctionKind.GENERAL + PythonTableFunction = gateway.jvm \ + .org.apache.flink.table.functions.python.PythonTableFunction + j_table_function = PythonTableFunction( + self._name, + bytearray(serialized_func), + j_input_types, + j_result_type, + j_function_kind, + self._deterministic, + _get_python_env()) return j_table_function diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/python/PythonScalarFunction.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/python/PythonScalarFunction.java new file mode 100644 index 0000000000..db2480c374 --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/python/PythonScalarFunction.java @@ -0,0 +1,97 @@ +/* + * 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.table.functions.python; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.table.functions.ScalarFunction; + +/** + * The wrapper of user defined python scalar function. + */ +@Internal +public class PythonScalarFunction extends ScalarFunction implements PythonFunction { + + private static final long serialVersionUID = 1L; + + private final String name; + private final byte[] serializedScalarFunction; + private final TypeInformation[] inputTypes; + private final TypeInformation resultType; + private final PythonFunctionKind pythonFunctionKind; + private final boolean deterministic; + private final PythonEnv pythonEnv; + + public PythonScalarFunction( + String name, + byte[] serializedScalarFunction, + TypeInformation[] inputTypes, + TypeInformation resultType, + PythonFunctionKind pythonFunctionKind, + boolean deterministic, + PythonEnv pythonEnv) { + this.name = name; + this.serializedScalarFunction = serializedScalarFunction; + this.inputTypes = inputTypes; + this.resultType = resultType; + this.pythonFunctionKind = pythonFunctionKind; + this.deterministic = deterministic; + this.pythonEnv = pythonEnv; + } + + public Object eval(Object... args) { + throw new UnsupportedOperationException( + "This method is a placeholder and should not be called."); + } + + @Override + public byte[] getSerializedPythonFunction() { + return serializedScalarFunction; + } + + @Override + public PythonEnv getPythonEnv() { + return pythonEnv; + } + + @Override + public PythonFunctionKind getPythonFunctionKind() { + return pythonFunctionKind; + } + + @Override + public boolean isDeterministic() { + return deterministic; + } + + @Override + public TypeInformation[] getParameterTypes(Class[] signature) { + return inputTypes; + } + + @Override + public TypeInformation getResultType(Class[] signature) { + return resultType; + } + + @Override + public String toString() { + return name; + } +} diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/python/SimplePythonFunction.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/python/PythonTableFunction.java similarity index 47% rename from flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/python/SimplePythonFunction.java rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/python/PythonTableFunction.java index 1cb70f524f..d9bdc6ed7a 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/python/SimplePythonFunction.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/python/PythonTableFunction.java @@ -19,40 +19,50 @@ package org.apache.flink.table.functions.python; import org.apache.flink.annotation.Internal; -import org.apache.flink.util.Preconditions; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.table.functions.TableFunction; /** - * A simple implementation of {@link PythonFunction}. + * The wrapper of user defined python table function. */ @Internal -public final class SimplePythonFunction implements PythonFunction { +public class PythonTableFunction extends TableFunction implements PythonFunction { private static final long serialVersionUID = 1L; - /** - * Serialized representation of the user-defined python function. - */ - private final byte[] serializedPythonFunction; - - /** - * Python execution environment. - */ + private final String name; + private final byte[] serializedScalarFunction; + private final TypeInformation[] inputTypes; + private final TypeInformation resultType; + private final PythonFunctionKind pythonFunctionKind; + private final boolean deterministic; private final PythonEnv pythonEnv; - /** - * The kind of the user-defined python function. - */ - private final PythonFunctionKind pythonFunctionKind; + public PythonTableFunction( + String name, + byte[] serializedScalarFunction, + TypeInformation[] inputTypes, + TypeInformation resultType, + PythonFunctionKind pythonFunctionKind, + boolean deterministic, + PythonEnv pythonEnv) { + this.name = name; + this.serializedScalarFunction = serializedScalarFunction; + this.inputTypes = inputTypes; + this.resultType = resultType; + this.pythonFunctionKind = pythonFunctionKind; + this.deterministic = deterministic; + this.pythonEnv = pythonEnv; + } - public SimplePythonFunction(byte[] serializedPythonFunction, PythonEnv pythonEnv, PythonFunctionKind pythonFunctionKind) { - this.serializedPythonFunction = Preconditions.checkNotNull(serializedPythonFunction); - this.pythonEnv = Preconditions.checkNotNull(pythonEnv); - this.pythonFunctionKind = Preconditions.checkNotNull(pythonFunctionKind); + public void eval(Object... args) { + throw new UnsupportedOperationException( + "This method is a placeholder and should not be called."); } @Override public byte[] getSerializedPythonFunction() { - return serializedPythonFunction; + return serializedScalarFunction; } @Override @@ -64,4 +74,24 @@ public final class SimplePythonFunction implements PythonFunction { public PythonFunctionKind getPythonFunctionKind() { return pythonFunctionKind; } + + @Override + public boolean isDeterministic() { + return deterministic; + } + + @Override + public TypeInformation[] getParameterTypes(Class[] signature) { + return inputTypes; + } + + @Override + public TypeInformation getResultType() { + return resultType; + } + + @Override + public String toString() { + return name; + } } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/PythonFunctionCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/PythonFunctionCodeGenerator.scala deleted file mode 100644 index 152aafd387..0000000000 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/PythonFunctionCodeGenerator.scala +++ /dev/null @@ -1,230 +0,0 @@ -/* - * 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.table.planner.codegen - -import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.java.typeutils.RowTypeInfo -import org.apache.flink.table.functions.python.{PythonEnv, PythonFunction, PythonFunctionKind} -import org.apache.flink.table.functions.{ScalarFunction, TableFunction, UserDefinedFunction} -import org.apache.flink.table.planner.codegen.CodeGenUtils.{newName, primitiveDefaultValue, primitiveTypeTermForType} -import org.apache.flink.table.planner.codegen.Indenter.toISC -import org.apache.flink.table.runtime.generated.GeneratedFunction -import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter -import org.apache.flink.types.Row - -/** - * A code generator for generating Python [[UserDefinedFunction]]s. - */ -object PythonFunctionCodeGenerator { - - private val PYTHON_SCALAR_FUNCTION_NAME = "PythonScalarFunction" - - private val PYTHON_TABLE_FUNCTION_NAME = "PythonTableFunction" - - /** - * Generates a [[ScalarFunction]] for the specified Python user-defined function. - * - * @param ctx The context of the code generator - * @param name name of the user-defined function - * @param serializedScalarFunction serialized Python scalar function - * @param inputTypes input data types - * @param resultType expected result type - * @param pythonFunctionKind the kind of the Python function - * @param deterministic the determinism of the function's results - * @param pythonEnv the Python execution environment - * @return instance of generated ScalarFunction - */ - def generateScalarFunction( - ctx: CodeGeneratorContext, - name: String, - serializedScalarFunction: Array[Byte], - inputTypes: Array[TypeInformation[_]], - resultType: TypeInformation[_], - pythonFunctionKind: PythonFunctionKind, - deterministic: Boolean, - pythonEnv: PythonEnv): ScalarFunction = { - val funcName = newName(PYTHON_SCALAR_FUNCTION_NAME) - val resultLogicType = TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType(resultType) - val resultTypeTerm = primitiveTypeTermForType(resultLogicType) - val defaultResultValue = primitiveDefaultValue(resultLogicType) - val inputParamCode = inputTypes.zipWithIndex.map { case (inputType, index) => - s"${primitiveTypeTermForType( - TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType(inputType))} in$index" - }.mkString(", ") - - val typeInfoTypeTerm = classOf[TypeInformation[_]].getCanonicalName - val pythonEnvTypeTerm = classOf[PythonEnv].getCanonicalName - val pythonFunctionKindTypeTerm = classOf[PythonFunctionKind].getCanonicalName - - val resultTypeNameTerm = - ctx.addReusableObject(resultType, "resultType", typeInfoTypeTerm) - val serializedScalarFunctionNameTerm = - ctx.addReusableObject(serializedScalarFunction, "serializedScalarFunction", "byte[]") - val pythonEnvNameTerm = ctx.addReusableObject(pythonEnv, "pythonEnv", pythonEnvTypeTerm) - val inputTypesCode = inputTypes - .map(ctx.addReusableObject(_, "inputType", typeInfoTypeTerm)) - .mkString(", ") - val pythonFunctionKindNameTerm = - ctx.addReusableObject(pythonFunctionKind, "pythonFunctionKind", pythonFunctionKindTypeTerm) - - val funcCode = j""" - |public class $funcName extends ${classOf[ScalarFunction].getCanonicalName} - | implements ${classOf[PythonFunction].getCanonicalName} { - | - | private static final long serialVersionUID = 1L; - | - | ${ctx.reuseMemberCode()} - | - | public $funcName(Object[] references) throws Exception { - | ${ctx.reuseInitCode()} - | } - | - | public $resultTypeTerm eval($inputParamCode) { - | return $defaultResultValue; - | } - | - | @Override - | public $typeInfoTypeTerm[] getParameterTypes(Class[] signature) { - | return new $typeInfoTypeTerm[]{$inputTypesCode}; - | } - | - | @Override - | public $typeInfoTypeTerm getResultType(Class[] signature) { - | return $resultTypeNameTerm; - | } - | - | @Override - | public byte[] getSerializedPythonFunction() { - | return $serializedScalarFunctionNameTerm; - | } - | - | @Override - | public $pythonEnvTypeTerm getPythonEnv() { - | return $pythonEnvNameTerm; - | } - | - | @Override - | public $pythonFunctionKindTypeTerm getPythonFunctionKind() { - | return $pythonFunctionKindNameTerm; - | } - | - | @Override - | public boolean isDeterministic() { - | return $deterministic; - | } - | - | @Override - | public String toString() { - | return "$name"; - | } - |} - |""".stripMargin - new GeneratedFunction(funcName, funcCode, ctx.references.toArray) - .newInstance(Thread.currentThread().getContextClassLoader) - } - - /** - * Generates a [[TableFunction]] for the specified Python user-defined function. - * - * @param ctx The context of the code generator - * @param name name of the user-defined function - * @param serializedTableFunction serialized Python table function - * @param inputTypes input data types - * @param resultTypes expected result types - * @param deterministic the determinism of the function's results - * @param pythonEnv the Python execution environment - * @return instance of generated TableFunction - */ - def generateTableFunction( - ctx: CodeGeneratorContext, - name: String, - serializedTableFunction: Array[Byte], - inputTypes: Array[TypeInformation[_]], - resultTypes: Array[TypeInformation[_]], - deterministic: Boolean, - pythonEnv: PythonEnv): TableFunction[_] = { - val funcName = newName(PYTHON_TABLE_FUNCTION_NAME) - val inputParamCode = inputTypes.zipWithIndex.map { case (inputType, index) => - s"${primitiveTypeTermForType( - TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType(inputType))} in$index" - }.mkString(", ") - - val rowTypeTerm = classOf[Row].getCanonicalName - val typeInfoTypeTerm = classOf[TypeInformation[_]].getCanonicalName - val rowTypeInfoTerm = classOf[RowTypeInfo].getCanonicalName - val pythonEnvTypeTerm = classOf[PythonEnv].getCanonicalName - - val serializedTableFunctionNameTerm = - ctx.addReusableObject(serializedTableFunction, "serializedTableFunction", "byte[]") - val pythonEnvNameTerm = ctx.addReusableObject(pythonEnv, "pythonEnv", pythonEnvTypeTerm) - val inputTypesCode = inputTypes - .map(ctx.addReusableObject(_, "inputType", typeInfoTypeTerm)) - .mkString(", ") - val resultTypesCode = resultTypes - .map(ctx.addReusableObject(_, "resultType", typeInfoTypeTerm)) - .mkString(", ") - val funcCode = j""" - |public class $funcName extends ${classOf[TableFunction[_]].getCanonicalName}<$rowTypeTerm> - | implements ${classOf[PythonFunction].getCanonicalName} { - | - | private static final long serialVersionUID = 1L; - | - | ${ctx.reuseMemberCode()} - | - | public $funcName(Object[] references) throws Exception { - | ${ctx.reuseInitCode()} - | } - | - | public void eval($inputParamCode) { - | } - | - | @Override - | public $typeInfoTypeTerm[] getParameterTypes(Class[] signature) { - | return new $typeInfoTypeTerm[]{$inputTypesCode}; - | } - | - | @Override - | public $typeInfoTypeTerm<$rowTypeTerm> getResultType() { - | return new $rowTypeInfoTerm(new $typeInfoTypeTerm[]{$resultTypesCode}); - | } - | - | @Override - | public byte[] getSerializedPythonFunction() { - | return $serializedTableFunctionNameTerm; - | } - | - | @Override - | public $pythonEnvTypeTerm getPythonEnv() { - | return $pythonEnvNameTerm; - | } - | - | @Override - | public boolean isDeterministic() { - | return $deterministic; - | } - | - | @Override - | public String toString() { - | return "$name"; - | } - |} - |""".stripMargin - new GeneratedFunction(funcName, funcCode, ctx.references.toArray) - .newInstance(Thread.currentThread().getContextClassLoader) - } -} diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonBase.scala index 745cd3f26f..128cf0cff6 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonBase.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonBase.scala @@ -22,7 +22,7 @@ import org.apache.calcite.rex.{RexCall, RexLiteral, RexNode} import org.apache.calcite.sql.`type`.SqlTypeName import org.apache.flink.table.api.TableException import org.apache.flink.table.functions.UserDefinedFunction -import org.apache.flink.table.functions.python.{PythonFunction, PythonFunctionInfo, SimplePythonFunction} +import org.apache.flink.table.functions.python.{PythonFunction, PythonFunctionInfo} import org.apache.flink.table.planner.functions.utils.{ScalarSqlFunction, TableSqlFunction} import scala.collection.mutable @@ -70,13 +70,7 @@ trait CommonPythonBase { } } - // Extracts the necessary information for Python function execution, such as - // the serialized Python function, the Python env, etc - val pythonFunction = new SimplePythonFunction( - func.asInstanceOf[PythonFunction].getSerializedPythonFunction, - func.asInstanceOf[PythonFunction].getPythonEnv, - func.asInstanceOf[PythonFunction].getPythonFunctionKind) - new PythonFunctionInfo(pythonFunction, inputs.toArray) + new PythonFunctionInfo(func.asInstanceOf[PythonFunction], inputs.toArray) } protected def createPythonFunctionInfo( diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/utils/python/PythonTableUtils.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/utils/python/PythonTableUtils.scala index e427923645..b1c16d90c1 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/utils/python/PythonTableUtils.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/utils/python/PythonTableUtils.scala @@ -30,10 +30,7 @@ import org.apache.flink.api.common.typeinfo.{BasicArrayTypeInfo, BasicTypeInfo, import org.apache.flink.api.java.io.CollectionInputFormat import org.apache.flink.api.java.typeutils.{MapTypeInfo, ObjectArrayTypeInfo, RowTypeInfo} import org.apache.flink.core.io.InputSplit -import org.apache.flink.table.api.{TableConfig, TableSchema, Types} -import org.apache.flink.table.functions.{ScalarFunction, TableFunction} -import org.apache.flink.table.functions.python.{PythonEnv, PythonFunctionKind} -import org.apache.flink.table.planner.codegen.{CodeGeneratorContext, PythonFunctionCodeGenerator} +import org.apache.flink.table.api.{TableSchema, Types} import org.apache.flink.table.sources.InputFormatTableSource import org.apache.flink.types.Row @@ -41,65 +38,6 @@ import scala.collection.JavaConversions._ object PythonTableUtils { - /** - * Creates a [[ScalarFunction]] for the specified Python ScalarFunction. - * - * @param funcName class name of the user-defined function. Must be a valid Java class identifier - * @param serializedScalarFunction serialized Python scalar function - * @param inputTypes input data types - * @param resultType expected result type - * @param pythonFunctionKind the kind of the Python function - * @param deterministic the determinism of the function's results - * @param pythonEnv the Python execution environment - * @return A generated Java ScalarFunction representation for the specified Python ScalarFunction - */ - def createPythonScalarFunction( - config: TableConfig, - funcName: String, - serializedScalarFunction: Array[Byte], - inputTypes: Array[TypeInformation[_]], - resultType: TypeInformation[_], - pythonFunctionKind: PythonFunctionKind, - deterministic: Boolean, - pythonEnv: PythonEnv): ScalarFunction = - PythonFunctionCodeGenerator.generateScalarFunction( - CodeGeneratorContext(config), - funcName, - serializedScalarFunction, - inputTypes, - resultType, - pythonFunctionKind, - deterministic, - pythonEnv) - - /** - * Creates a [[TableFunction]] for the specified Python TableFunction. - * - * @param funcName class name of the user-defined function. Must be a valid Java class identifier - * @param serializedTableFunction serialized Python table function - * @param inputTypes input data types - * @param resultTypes expected result types - * @param deterministic the determinism of the function's results - * @param pythonEnv the Python execution environment - * @return A generated Java TableFunction representation for the specified Python TableFunction - */ - def createPythonTableFunction( - config: TableConfig, - funcName: String, - serializedTableFunction: Array[Byte], - inputTypes: Array[TypeInformation[_]], - resultTypes: Array[TypeInformation[_]], - deterministic: Boolean, - pythonEnv: PythonEnv): TableFunction[_] = - PythonFunctionCodeGenerator.generateTableFunction( - CodeGeneratorContext(config), - funcName, - serializedTableFunction, - inputTypes, - resultTypes, - deterministic, - pythonEnv) - /** * Wrap the unpickled python data with an InputFormat. It will be passed to * PythonInputFormatTableSource later. diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/PythonFunctionCodeGenerator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/PythonFunctionCodeGenerator.scala deleted file mode 100644 index 31952219ad..0000000000 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/PythonFunctionCodeGenerator.scala +++ /dev/null @@ -1,230 +0,0 @@ -/* - * 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.table.codegen - -import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.java.typeutils.RowTypeInfo -import org.apache.flink.table.codegen.CodeGenUtils.{newName, primitiveDefaultValue, primitiveTypeTermForTypeInfo} -import org.apache.flink.table.codegen.Indenter.toISC -import org.apache.flink.table.functions.{ScalarFunction, TableFunction, UserDefinedFunction} -import org.apache.flink.table.functions.python.{PythonEnv, PythonFunction, PythonFunctionKind} -import org.apache.flink.table.utils.EncodingUtils -import org.apache.flink.types.Row - -/** - * A code generator for generating Python [[UserDefinedFunction]]s. - */ -object PythonFunctionCodeGenerator extends Compiler[UserDefinedFunction] { - - private val PYTHON_SCALAR_FUNCTION_NAME = "PythonScalarFunction" - - private val PYTHON_TABLE_FUNCTION_NAME = "PythonTableFunction" - - /** - * Generates a [[ScalarFunction]] for the specified Python user-defined function. - * - * @param name name of the user-defined function - * @param serializedScalarFunction serialized Python scalar function - * @param inputTypes input data types - * @param resultType expected result type - * @param pythonFunctionKind the kind of the Python function - * @param deterministic the determinism of the function's results - * @param pythonEnv the Python execution environment - * @return instance of generated ScalarFunction - */ - def generateScalarFunction( - name: String, - serializedScalarFunction: Array[Byte], - inputTypes: Array[TypeInformation[_]], - resultType: TypeInformation[_], - pythonFunctionKind: PythonFunctionKind, - deterministic: Boolean, - pythonEnv: PythonEnv): ScalarFunction = { - val funcName = newName(PYTHON_SCALAR_FUNCTION_NAME) - val resultTypeTerm = primitiveTypeTermForTypeInfo(resultType) - val defaultResultValue = primitiveDefaultValue(resultType) - val inputParamCode = inputTypes.zipWithIndex.map { case (inputType, index) => - s"${primitiveTypeTermForTypeInfo(inputType)} in$index" - }.mkString(", ") - - val encodingUtilsTypeTerm = classOf[EncodingUtils].getCanonicalName - val typeInfoTypeTerm = classOf[TypeInformation[_]].getCanonicalName - val inputTypesCode = inputTypes.map(EncodingUtils.encodeObjectToString).map { inputType => - s""" - |($typeInfoTypeTerm) $encodingUtilsTypeTerm.decodeStringToObject( - | "$inputType", $typeInfoTypeTerm.class) - |""".stripMargin - }.mkString(", ") - - val encodedResultType = EncodingUtils.encodeObjectToString(resultType) - val encodedScalarFunction = EncodingUtils.encodeBytesToBase64(serializedScalarFunction) - val encodedPythonEnv = EncodingUtils.encodeObjectToString(pythonEnv) - val pythonEnvTypeTerm = classOf[PythonEnv].getCanonicalName - val encodedPythonFunctionKind = EncodingUtils.encodeObjectToString(pythonFunctionKind) - val pythonFunctionKindTypeTerm = classOf[PythonFunctionKind].getCanonicalName - - val funcCode = j""" - |public class $funcName extends ${classOf[ScalarFunction].getCanonicalName} - | implements ${classOf[PythonFunction].getCanonicalName} { - | - | private static final long serialVersionUID = 1L; - | - | public $resultTypeTerm eval($inputParamCode) { - | return $defaultResultValue; - | } - | - | @Override - | public $typeInfoTypeTerm[] getParameterTypes(Class[] signature) { - | return new $typeInfoTypeTerm[]{$inputTypesCode}; - | } - | - | @Override - | public $typeInfoTypeTerm getResultType(Class[] signature) { - | return ($typeInfoTypeTerm) $encodingUtilsTypeTerm.decodeStringToObject( - | "$encodedResultType", $typeInfoTypeTerm.class); - | } - | - | @Override - | public byte[] getSerializedPythonFunction() { - | return $encodingUtilsTypeTerm.decodeBase64ToBytes("$encodedScalarFunction"); - | } - | - | @Override - | public $pythonEnvTypeTerm getPythonEnv() { - | return ($pythonEnvTypeTerm) $encodingUtilsTypeTerm.decodeStringToObject( - | "$encodedPythonEnv", $pythonEnvTypeTerm.class); - | } - | - | @Override - | public $pythonFunctionKindTypeTerm getPythonFunctionKind() { - | return ($pythonFunctionKindTypeTerm) $encodingUtilsTypeTerm.decodeStringToObject( - | "$encodedPythonFunctionKind", $pythonFunctionKindTypeTerm.class); - | } - | - | @Override - | public boolean isDeterministic() { - | return $deterministic; - | } - | - | @Override - | public String toString() { - | return "$name"; - | } - |} - |""".stripMargin - - val clazz = compile( - Thread.currentThread().getContextClassLoader, - funcName, - funcCode) - clazz.newInstance().asInstanceOf[ScalarFunction] - } - - - /** - * Generates a [[TableFunction]] for the specified Python user-defined function. - * - * @param name name of the user-defined function - * @param serializedTableFunction serialized Python table function - * @param inputTypes input data types - * @param resultTypes expected result types - * @param deterministic the determinism of the function's results - * @param pythonEnv the Python execution environment - * @return instance of generated TableFunction - */ - def generateTableFunction( - name: String, - serializedTableFunction: Array[Byte], - inputTypes: Array[TypeInformation[_]], - resultTypes: Array[TypeInformation[_]], - deterministic: Boolean, - pythonEnv: PythonEnv): TableFunction[_] = { - val funcName = newName(PYTHON_TABLE_FUNCTION_NAME) - val inputParamCode = inputTypes.zipWithIndex.map { case (inputType, index) => - s"${primitiveTypeTermForTypeInfo(inputType)} in$index" - }.mkString(", ") - - val encodingUtilsTypeTerm = classOf[EncodingUtils].getCanonicalName - val typeInfoTypeTerm = classOf[TypeInformation[_]].getCanonicalName - val rowTypeInfoTerm = classOf[RowTypeInfo].getCanonicalName - val rowTypeTerm = classOf[Row].getCanonicalName - val inputTypesCode = inputTypes.map(EncodingUtils.encodeObjectToString).map { inputType => - s""" - |($typeInfoTypeTerm) $encodingUtilsTypeTerm.decodeStringToObject( - | "$inputType", $typeInfoTypeTerm.class) - |""".stripMargin - }.mkString(", ") - val resultTypesCode = resultTypes.map(EncodingUtils.encodeObjectToString).map { resultType => - s""" - |($typeInfoTypeTerm) $encodingUtilsTypeTerm.decodeStringToObject( - | "$resultType", $typeInfoTypeTerm.class) - |""".stripMargin - }.mkString(", ") - - - val encodedScalarFunction = EncodingUtils.encodeBytesToBase64(serializedTableFunction) - val encodedPythonEnv = EncodingUtils.encodeObjectToString(pythonEnv) - val pythonEnvTypeTerm = classOf[PythonEnv].getCanonicalName - val funcCode = j""" - |public class $funcName extends ${classOf[TableFunction[_]].getCanonicalName}<$rowTypeTerm> - | implements ${classOf[PythonFunction].getCanonicalName} { - | - | private static final long serialVersionUID = 1L; - | - | public void eval($inputParamCode) { - | } - | - | @Override - | public $typeInfoTypeTerm[] getParameterTypes(Class[] signature) { - | return new $typeInfoTypeTerm[]{$inputTypesCode}; - | } - | - | @Override - | public $typeInfoTypeTerm<$rowTypeTerm> getResultType() { - | return new $rowTypeInfoTerm(new $typeInfoTypeTerm[]{$resultTypesCode}); - | } - | - | @Override - | public byte[] getSerializedPythonFunction() { - | return $encodingUtilsTypeTerm.decodeBase64ToBytes("$encodedScalarFunction"); - | } - | - | @Override - | public $pythonEnvTypeTerm getPythonEnv() { - | return ($pythonEnvTypeTerm) $encodingUtilsTypeTerm.decodeStringToObject( - | "$encodedPythonEnv", $pythonEnvTypeTerm.class); - | } - | - | @Override - | public boolean isDeterministic() { - | return $deterministic; - | } - | - | @Override - | public String toString() { - | return "$name"; - | } - |} - |""".stripMargin - val clazz = compile( - Thread.currentThread().getContextClassLoader, - funcName, - funcCode) - clazz.newInstance().asInstanceOf[TableFunction[_]] - } -} diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonPythonBase.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonPythonBase.scala index 784281576e..058b074781 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonPythonBase.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonPythonBase.scala @@ -21,7 +21,7 @@ import org.apache.calcite.rex.{RexCall, RexLiteral, RexNode} import org.apache.calcite.sql.`type`.SqlTypeName import org.apache.flink.table.api.TableException import org.apache.flink.table.functions.UserDefinedFunction -import org.apache.flink.table.functions.python.{PythonFunction, PythonFunctionInfo, SimplePythonFunction} +import org.apache.flink.table.functions.python.{PythonFunction, PythonFunctionInfo} import org.apache.flink.table.functions.utils.{ScalarSqlFunction, TableSqlFunction} import scala.collection.mutable @@ -68,13 +68,7 @@ trait CommonPythonBase { } } - // Extracts the necessary information for Python function execution, such as - // the serialized Python function, the Python env, etc - val pythonFunction = new SimplePythonFunction( - func.asInstanceOf[PythonFunction].getSerializedPythonFunction, - func.asInstanceOf[PythonFunction].getPythonEnv, - func.asInstanceOf[PythonFunction].getPythonFunctionKind) - new PythonFunctionInfo(pythonFunction, inputs.toArray) + new PythonFunctionInfo(func.asInstanceOf[PythonFunction], inputs.toArray) } protected def createPythonFunctionInfo( diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/util/python/PythonTableUtils.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/util/python/PythonTableUtils.scala index fb451d4192..094945c344 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/util/python/PythonTableUtils.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/util/python/PythonTableUtils.scala @@ -31,9 +31,6 @@ import org.apache.flink.api.java.io.CollectionInputFormat import org.apache.flink.api.java.typeutils.{MapTypeInfo, ObjectArrayTypeInfo, RowTypeInfo} import org.apache.flink.core.io.InputSplit import org.apache.flink.table.api.{TableSchema, Types} -import org.apache.flink.table.codegen.PythonFunctionCodeGenerator -import org.apache.flink.table.functions.{ScalarFunction, TableFunction} -import org.apache.flink.table.functions.python.{PythonEnv, PythonFunctionKind} import org.apache.flink.table.sources.InputFormatTableSource import org.apache.flink.types.Row @@ -41,61 +38,6 @@ import scala.collection.JavaConversions._ object PythonTableUtils { - /** - * Creates a [[ScalarFunction]] for the specified Python ScalarFunction. - * - * @param funcName class name of the user-defined function. Must be a valid Java class identifier - * @param serializedScalarFunction serialized Python scalar function - * @param inputTypes input data types - * @param resultType expected result type - * @param pythonFunctionKind the kind of the Python function - * @param deterministic the determinism of the function's results - * @param pythonEnv the Python execution environment - * @return A generated Java ScalarFunction representation for the specified Python ScalarFunction - */ - def createPythonScalarFunction( - funcName: String, - serializedScalarFunction: Array[Byte], - inputTypes: Array[TypeInformation[_]], - resultType: TypeInformation[_], - pythonFunctionKind: PythonFunctionKind, - deterministic: Boolean, - pythonEnv: PythonEnv): ScalarFunction = - PythonFunctionCodeGenerator.generateScalarFunction( - funcName, - serializedScalarFunction, - inputTypes, - resultType, - pythonFunctionKind, - deterministic, - pythonEnv) - - /** - * Creates a [[TableFunction]] for the specified Python TableFunction. - * - * @param funcName class name of the user-defined function. Must be a valid Java class identifier - * @param serializedTableFunction serialized Python table function - * @param inputTypes input data types - * @param resultTypes expected result types - * @param deterministic the determinism of the function's results - * @param pythonEnv the Python execution environment - * @return A generated Java TableFunction representation for the specified Python TableFunction - */ - def createPythonTableFunction( - funcName: String, - serializedTableFunction: Array[Byte], - inputTypes: Array[TypeInformation[_]], - resultTypes: Array[TypeInformation[_]], - deterministic: Boolean, - pythonEnv: PythonEnv): TableFunction[_] = - PythonFunctionCodeGenerator.generateTableFunction( - funcName, - serializedTableFunction, - inputTypes, - resultTypes, - deterministic, - pythonEnv) - /** * Wrap the unpickled python data with an InputFormat. It will be passed to * PythonInputFormatTableSource later. -- Gitee From 271ab6045161296744035d6737e66e529aa7d7f8 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Wed, 11 Mar 2020 22:33:29 +0800 Subject: [PATCH 130/885] [FLINK-16530][docs] Add documentation about "GROUPING SETS" and "CUBE" support in streaming mode This closes #11379 --- docs/dev/table/sql/queries.md | 4 +++- docs/dev/table/sql/queries.zh.md | 12 +++++++----- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/docs/dev/table/sql/queries.md b/docs/dev/table/sql/queries.md index de8d90622d..eba61350e6 100644 --- a/docs/dev/table/sql/queries.md +++ b/docs/dev/table/sql/queries.md @@ -471,7 +471,8 @@ SELECT DISTINCT users FROM Orders

    Grouping sets, Rollup, Cube
    - Batch + Batch Streaming + Result Updating
    {% highlight sql %} @@ -479,6 +480,7 @@ SELECT SUM(amount) FROM Orders GROUP BY GROUPING SETS ((user), (product)) {% endhighlight %} +

    Note: Streaming mode Grouping sets, Rollup and Cube are only supported in Blink planner.

    GroupBy 聚合
    批处理 流处理
    - 可自动更新结果 + 结果更新

    注意: GroupBy 在流处理表中会产生更新结果(updating result)。详情请阅读 动态表流概念 。 @@ -459,7 +459,7 @@ WINDOW w AS (

    Distinct
    批处理 流处理
    - 可自动更新结果 + 结果更新
    {% highlight sql %} @@ -471,7 +471,8 @@ SELECT DISTINCT users FROM Orders
    Grouping sets, Rollup, Cube
    - 批处理 + 批处理 流处理 + 结果更新
    {% highlight sql %} @@ -479,6 +480,7 @@ SELECT SUM(amount) FROM Orders GROUP BY GROUPING SETS ((user), (product)) {% endhighlight %} +

    Note: 流式 Grouping sets、Rollup 以及 Cube 只在 Blink planner 中支持。

    Outer Equi-join
    批处理 流处理 - 可自动更新结果 + 结果更新

    目前仅支持 equi-join ,即 join 的联合条件至少拥有一个相等谓词。不支持任何 cross join 和 theta join。

    @@ -869,7 +871,7 @@ WHERE rownum <= N [AND conditions] - `WHERE rownum <= N`: Flink 需要 `rownum <= N` 才能识别一个查询是否为 Top-N 查询。 其中, N 代表最大或最小的 N 条记录会被保留。 - `[AND conditions]`: 在 where 语句中,可以随意添加其他的查询条件,但其他条件只允许通过 `AND` 与 `rownum <= N` 结合使用。 -流处理模式需注意 TopN 查询 可自动更新结果。 Flink SQL 会根据排序键对输入的流进行排序;若 top N 的记录发生了变化,变化的部分会以撤销、更新记录的形式发送到下游。 +流处理模式需注意 TopN 查询的结果会带有更新。 Flink SQL 会根据排序键对输入的流进行排序;若 top N 的记录发生了变化,变化的部分会以撤销、更新记录的形式发送到下游。 推荐使用一个支持更新的存储作为 Top-N 查询的 sink 。另外,若 top N 记录需要存储到外部存储,则结果表需要拥有相同与 Top-N 查询相同的唯一键。 Top-N 的唯一键是分区列和 rownum 列的结合,另外 Top-N 查询也可以获得上游的唯一键。以下面的任务为例,`product_id` 是 `ShopSales` 的唯一键,然后 Top-N 的唯一键是 [`category`, `rownum`] 和 [`product_id`] 。 -- Gitee From 091d1d38f7f2c9ec0c97606682486d777758041b Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Sun, 8 Mar 2020 11:19:06 +0100 Subject: [PATCH 131/885] [FLINK-16374][AZP] Disable java e2e tests in misc profile Because the java e2e tests are executed in the e2e profile --- tools/travis_watchdog.sh | 22 +++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/tools/travis_watchdog.sh b/tools/travis_watchdog.sh index 4a1bb67a63..055ed75c15 100755 --- a/tools/travis_watchdog.sh +++ b/tools/travis_watchdog.sh @@ -304,17 +304,17 @@ case $TEST in echo "Previous build failure detected, skipping bash end-to-end tests.\n" echo "==============================================================================\n" fi - fi - if [ $EXIT_CODE == 0 ]; then - echo "\n\n==============================================================================\n" - echo "Running java end-to-end tests\n" - echo "==============================================================================\n" - - run_with_watchdog "$MVN_E2E -DdistDir=$(readlink -e build-target)" - else - echo "\n==============================================================================\n" - echo "Previous build failure detected, skipping java end-to-end tests.\n" - fi + if [ $EXIT_CODE == 0 ]; then + echo "\n\n==============================================================================\n" + echo "Running java end-to-end tests\n" + echo "==============================================================================\n" + + run_with_watchdog "$MVN_E2E -DdistDir=$(readlink -e build-target)" + else + echo "\n==============================================================================\n" + echo "Previous build failure detected, skipping java end-to-end tests.\n" + fi + fi ;; esac -- Gitee From 10e8918b9ee3fa939d102a29d98f852790be22e9 Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Mon, 9 Mar 2020 21:53:27 +0100 Subject: [PATCH 132/885] [FLINK-16456][e2e] Increase memory off heap of 'Heavy deployment end-to-end test' Increasing from default / calculated 134217728b to 200m. --- flink-end-to-end-tests/test-scripts/test_heavy_deployment.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/flink-end-to-end-tests/test-scripts/test_heavy_deployment.sh b/flink-end-to-end-tests/test-scripts/test_heavy_deployment.sh index 6bba15f044..4285214145 100755 --- a/flink-end-to-end-tests/test-scripts/test_heavy_deployment.sh +++ b/flink-end-to-end-tests/test-scripts/test_heavy_deployment.sh @@ -36,6 +36,7 @@ set_config_key "taskmanager.memory.network.max" "128mb" set_config_key "taskmanager.network.request-backoff.max" "60000" set_config_key "taskmanager.memory.segment-size" "8kb" set_config_key "taskmanager.memory.jvm-metaspace.size" "64m" +set_config_key "taskmanager.memory.framework.off-heap.size" "200m" set_config_key "taskmanager.numberOfTaskSlots" "20" # 20 slots per TM -- Gitee From bf1195232a49cce1897c1fa86c5af9ee005212c6 Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Sun, 8 Mar 2020 10:22:54 +0100 Subject: [PATCH 133/885] [FLINK-16417][e2e] Increase offheap memory for high-parallel-iterations test This closes #11364 --- .../test-scripts/test_high_parallelism_iterations.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-end-to-end-tests/test-scripts/test_high_parallelism_iterations.sh b/flink-end-to-end-tests/test-scripts/test_high_parallelism_iterations.sh index 54a441bd7b..cc1fb13a20 100755 --- a/flink-end-to-end-tests/test-scripts/test_high_parallelism_iterations.sh +++ b/flink-end-to-end-tests/test-scripts/test_high_parallelism_iterations.sh @@ -32,7 +32,7 @@ TEST_PROGRAM_JAR=${END_TO_END_DIR}/$TEST/target/$TEST_PROGRAM_NAME.jar set_config_key "taskmanager.numberOfTaskSlots" "$SLOTS_PER_TM" set_config_key "taskmanager.memory.network.min" "160m" set_config_key "taskmanager.memory.network.max" "160m" -set_config_key "taskmanager.memory.framework.off-heap.size" "160m" +set_config_key "taskmanager.memory.framework.off-heap.size" "300m" print_mem_use start_cluster -- Gitee From 1b628d4a7d92f9c79c31f3fe90911940e0676b22 Mon Sep 17 00:00:00 2001 From: Seth Wiesman Date: Fri, 13 Dec 2019 13:15:45 -0600 Subject: [PATCH 134/885] [FLINK-15337][docs] Add vendor table to documentation --- docs/ops/deployment/index.md | 147 ++++++++++++++++++++++++++++++++ docs/ops/deployment/index.zh.md | 147 ++++++++++++++++++++++++++++++++ 2 files changed, 294 insertions(+) diff --git a/docs/ops/deployment/index.md b/docs/ops/deployment/index.md index e82299d6de..85fde97c62 100644 --- a/docs/ops/deployment/index.md +++ b/docs/ops/deployment/index.md @@ -3,6 +3,7 @@ title: "Clusters & Deployment" nav-id: deployment nav-parent_id: ops nav-pos: 1 +nav-show_overview: true --- + +When deciding how and where to run Flink, there's a wide range of options available. + +* This will be replaced by the TOC +{:toc} + +## Deployment Targets + +Apache Flink ships with first class support for a number of common deployment targets. + +
    +
    +
    +
    + Local +
    +
    + Run Flink locally for basic testing and experimentation +
    Learn more +
    +
    +
    +
    +
    +
    + Standalone +
    +
    + A simple solution for running Flink on bare metal or VM's +
    Learn more +
    +
    +
    +
    +
    +
    + Yarn +
    +
    + Deploy Flink on-top Apache Hadoop's resource manager +
    Learn more +
    +
    +
    +
    +
    +
    +
    +
    + Mesos +
    +
    + A generic resource manager for running distriubted systems +
    Learn more +
    +
    +
    +
    +
    +
    + Docker +
    +
    + A popular solution for running Flink within a containerized environment +
    Learn more +
    +
    +
    +
    +
    +
    + Kubernetes +
    +
    + An automated system for deploying containerized applications +
    Learn more +
    +
    +
    +
    + +## Vendor Solutions + +A number of vendors offer managed or fully hosted Flink solutions. +None of these vendors are officially supported or endorsed by the Apache Flink PMC. +Please refer to vendor maintained documentation on how to use these products. + + + +#### AliCloud Realtime Compute + +[Website](https://www.alibabacloud.com/products/realtime-compute) + +Supported Environments: +AliCloud + +#### Amazon EMR + +[Website](https://aws.amazon.com/emr/) + +Supported Environments: +AWS + +#### Amazon Kinesis Data Analytics For Java + +[Website](https://docs.aws.amazon.com/kinesisanalytics/latest/java/what-is.html) + +Supported Environments: +AWS + +#### Cloudera + +[Website](https://www.cloudera.com/) + +Supported Environment: +AWS +Azure +Google Cloud +On-Premise + +#### Eventador + +[Website](https://eventador.io) + +Supported Environment: +AWS + +#### Huawei Cloud Stream Service + +[Website](https://www.huaweicloud.com/en-us/product/cs.html) + +Supported Environment: +Huawei Cloud + +#### Ververica Platform + +[Website](https://www.ververica.com/platform-overview) + +Supported Environments: +AliCloud +AWS +Azure +Google Cloud +On-Premise diff --git a/docs/ops/deployment/index.zh.md b/docs/ops/deployment/index.zh.md index 196e3808cb..4499abb5c0 100644 --- a/docs/ops/deployment/index.zh.md +++ b/docs/ops/deployment/index.zh.md @@ -3,6 +3,7 @@ title: "集群与部署" nav-id: deployment nav-parent_id: ops nav-pos: 1 +nav-show_overview: true --- + +When deciding how and where to run Flink, there's a wide range of options available. + +* This will be replaced by the TOC +{:toc} + +## Deployment Targets + +Apache Flink ships with first class support for a number of common deployment targets. + +
    +
    +
    +
    + Local +
    +
    + Run Flink locally for basic testing and experimentation +
    Learn more +
    +
    +
    +
    +
    +
    + Standalone +
    +
    + A simple solution for running Flink on bare metal or VM's +
    Learn more +
    +
    +
    +
    +
    +
    + Yarn +
    +
    + Deploy Flink on-top Apache Hadoop's resource manager +
    Learn more +
    +
    +
    +
    +
    +
    +
    +
    + Mesos +
    +
    + A generic resource manager for running distriubted systems +
    Learn more +
    +
    +
    +
    +
    +
    + Docker +
    +
    + A popular solution for running Flink within a containerized environment +
    Learn more +
    +
    +
    +
    +
    +
    + Kubernetes +
    +
    + An automated system for deploying containerized applications +
    Learn more +
    +
    +
    +
    + +## Vendor Solutions + +A number of vendors offer managed or fully hosted Flink solutions. +None of these vendors are officially supported or endorsed by the Apache Flink PMC. +Please refer to vendor maintained documentation on how to use these products. + + + +#### AliCloud Realtime Compute + +[Website](https://www.alibabacloud.com/products/realtime-compute) + +Supported Environments: +AliCloud + +#### Amazon EMR + +[Website](https://aws.amazon.com/emr/) + +Supported Environments: +AWS + +#### Amazon Kinesis Data Analytics For Java + +[Website](https://docs.aws.amazon.com/kinesisanalytics/latest/java/what-is.html) + +Supported Environments: +AWS + +#### Cloudera + +[Website](https://www.cloudera.com/) + +Supported Environment: +AWS +Azure +Google Cloud +On-Premise + +#### Eventador + +[Website](https://eventador.io) + +Supported Environment: +AWS + +#### Huawei Cloud Stream Service + +[Website](https://www.huaweicloud.com/en-us/product/cs.html) + +Supported Environment: +Huawei Cloud + +#### Ververica Platform + +[Website](https://www.ververica.com/platform-overview) + +Supported Environments: +AliCloud +AWS +Azure +Google Cloud +On-Premise -- Gitee From 81c48e51248239e8b57655616ebb061b4edb846a Mon Sep 17 00:00:00 2001 From: felixzheng Date: Wed, 11 Mar 2020 21:01:02 +0800 Subject: [PATCH 135/885] [FLINK-16546][yarn] Fix logging bug in YarnClusterDescriptor#startAppMaster --- .../main/java/org/apache/flink/yarn/YarnClusterDescriptor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java index 553da43ca1..b0c7330b1c 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java @@ -851,7 +851,7 @@ public class YarnClusterDescriptor implements ClusterDescriptor { throw e; } finally { if (tmpJobGraphFile != null && !tmpJobGraphFile.delete()) { - LOG.warn("Fail to delete temporary file {}.", tmpConfigurationFile.toPath()); + LOG.warn("Fail to delete temporary file {}.", tmpJobGraphFile.toPath()); } } } -- Gitee From 51323757b6b5e1c7fe3bb7333bf297871ae8890c Mon Sep 17 00:00:00 2001 From: wangxianghu Date: Thu, 13 Feb 2020 11:27:46 +0800 Subject: [PATCH 136/885] [hotfix] Fix some typos in flink-examples-streaming --- .../flink/streaming/examples/async/AsyncIOExample.java | 7 +++---- .../flink/streaming/examples/iteration/IterateExample.java | 2 +- .../apache/flink/streaming/examples/join/WindowJoin.java | 2 -- .../streaming/examples/sideoutput/SideOutputExample.java | 2 +- .../statemachine/generator/EventsGeneratorSource.java | 2 +- 5 files changed, 6 insertions(+), 9 deletions(-) diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java index b9f92fe593..97de5fc390 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java @@ -108,7 +108,7 @@ public class AsyncIOExample { /** - * An sample of {@link AsyncFunction} using a thread pool and executing working threads + * An example of {@link AsyncFunction} using a thread pool and executing working threads * to simulate multiple async operations. * *

    For the real use case in production environment, the thread pool may stay in the @@ -142,7 +142,6 @@ public class AsyncIOExample { @Override public void open(Configuration parameters) throws Exception { super.open(parameters); - executorService = Executors.newFixedThreadPool(30); } @@ -258,10 +257,10 @@ public class AsyncIOExample { env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime); } - // create input stream of an single integer + // create input stream of a single integer DataStream inputStream = env.addSource(new SimpleSource(maxCount)); - // create async function, which will *wait* for a while to simulate the process of async i/o + // create async function, which will "wait" for a while to simulate the process of async i/o AsyncFunction function = new SampleAsyncFunction(sleepFactor, failRatio, shutdownWaitTS); diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java index 13db58ae18..ecf57eedeb 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/iteration/IterateExample.java @@ -81,7 +81,7 @@ public class IterateExample { // create an iterative data stream from the input with 5 second timeout IterativeStream> it = inputStream.map(new InputMap()) - .iterate(5000); + .iterate(5000L); // apply the step function to get the next Fibonacci number // increment the counter and split the output with the output selector diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java index 01effd1afa..d2b179469b 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/join/WindowJoin.java @@ -85,9 +85,7 @@ public class WindowJoin { return grades.join(salaries) .where(new NameKeySelector()) .equalTo(new NameKeySelector()) - .window(TumblingEventTimeWindows.of(Time.milliseconds(windowSize))) - .apply(new JoinFunction, Tuple2, Tuple3>() { @Override diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/sideoutput/SideOutputExample.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/sideoutput/SideOutputExample.java index dca7389f39..f7fcf55d76 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/sideoutput/SideOutputExample.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/sideoutput/SideOutputExample.java @@ -33,7 +33,7 @@ import org.apache.flink.util.Collector; import org.apache.flink.util.OutputTag; /** - * An example that illustrates the use of side outputs. + * An example that illustrates the use of side output. * *

    This is a modified version of {@link org.apache.flink.streaming.examples.windowing.WindowWordCount} * that has a filter in the tokenizer and only emits some words for counting diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/generator/EventsGeneratorSource.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/generator/EventsGeneratorSource.java index 74fa9ed7fe..22f675c12a 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/generator/EventsGeneratorSource.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/generator/EventsGeneratorSource.java @@ -38,7 +38,7 @@ public class EventsGeneratorSource extends RichParallelSourceFunction { public EventsGeneratorSource(double errorProbability, int delayPerRecordMillis) { checkArgument(errorProbability >= 0.0 && errorProbability <= 1.0, "error probability must be in [0.0, 1.0]"); - checkArgument(delayPerRecordMillis >= 0, "deplay must be >= 0"); + checkArgument(delayPerRecordMillis >= 0, "delay must be >= 0"); this.errorProbability = errorProbability; this.delayPerRecordMillis = delayPerRecordMillis; -- Gitee From 3ef2c0870992c7ef79f03622c9c7d12b07798547 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Wed, 11 Mar 2020 12:58:58 +0100 Subject: [PATCH 137/885] [FLINK-16545][build] Remove Eclipse-specific plugins --- flink-clients/pom.xml | 49 ------------- .../flink-examples-streaming/pom.xml | 31 --------- flink-formats/flink-avro/pom.xml | 56 --------------- flink-tests/pom.xml | 69 ------------------- pom.xml | 65 ----------------- 5 files changed, 270 deletions(-) diff --git a/flink-clients/pom.xml b/flink-clients/pom.xml index 1481b6eb99..fa2c5e5266 100644 --- a/flink-clients/pom.xml +++ b/flink-clients/pom.xml @@ -154,54 +154,5 @@ under the License. - - - - - - org.eclipse.m2e - lifecycle-mapping - 1.0.0 - - - - - - - org.apache.maven.plugins - - - maven-assembly-plugin - - - [2.4,) - - - single - - - - - - - - - org.apache.maven.plugins - maven-clean-plugin - [1,) - - clean - - - - - - - - - - - - diff --git a/flink-examples/flink-examples-streaming/pom.xml b/flink-examples/flink-examples-streaming/pom.xml index 520b897eee..bd6b97265e 100644 --- a/flink-examples/flink-examples-streaming/pom.xml +++ b/flink-examples/flink-examples-streaming/pom.xml @@ -478,37 +478,6 @@ under the License. - - - - - - org.eclipse.m2e - lifecycle-mapping - 1.0.0 - - - - - - org.apache.maven.plugins - maven-dependency-plugin - [2.9,) - - unpack - - - - - - - - - - - - - diff --git a/flink-formats/flink-avro/pom.xml b/flink-formats/flink-avro/pom.xml index d145dba569..4dba749999 100644 --- a/flink-formats/flink-avro/pom.xml +++ b/flink-formats/flink-avro/pom.xml @@ -258,62 +258,6 @@ under the License. - - - - - - org.eclipse.m2e - lifecycle-mapping - 1.0.0 - - - - - - org.apache.maven.plugins - maven-assembly-plugin - [2.4,) - - single - - - - - - - - - org.apache.maven.plugins - maven-clean-plugin - [1,) - - clean - - - - - - - - - org.apache.avro - avro-maven-plugin - [1.7.7,) - - schema - - - - - - - - - - - - diff --git a/flink-tests/pom.xml b/flink-tests/pom.xml index e8add1a0fe..4941a62162 100644 --- a/flink-tests/pom.xml +++ b/flink-tests/pom.xml @@ -638,74 +638,5 @@ under the License. - - - - - - org.eclipse.m2e - lifecycle-mapping - 1.0.0 - - - - - - - org.apache.maven.plugins - - - maven-assembly-plugin - - - [2.4,) - - - single - - - - - - - - - org.apache.maven.plugins - maven-clean-plugin - [1,) - - clean - - - - - - - - - - net.alchim31.maven - - - scala-maven-plugin - - - [3.1.4,) - - - compile - testCompile - - - - - - - - - - - - diff --git a/pom.xml b/pom.xml index 21e89ad24f..689114656f 100644 --- a/pom.xml +++ b/pom.xml @@ -1834,71 +1834,6 @@ under the License. - - - org.eclipse.m2e - lifecycle-mapping - 1.0.0 - - - - - - pl.project13.maven - git-commit-id-plugin - [2.1.5,) - - revision - - - - - - - - - org.apache.maven.plugins - maven-checkstyle-plugin - [2.12.1,) - - check - - - - - - - - - org.apache.maven.plugins - maven-enforcer-plugin - [1.0.0,) - - enforce - - - - - - - - - org.apache.maven.plugins - maven-remote-resources-plugin - [0.0.0,) - - process - - - - - - - - - - - org.scalastyle -- Gitee From 9c6aa3287a55636294be78798fdc61c18c84e0d6 Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Tue, 18 Feb 2020 13:43:23 +0100 Subject: [PATCH 138/885] [FLINK-11720][connectors] Bump ElasticSearch5 to 5.3.3 The ES5 connector has caused numerous issues in end to end and integration tests (on CI and during release test). The NOTICE file has been updated according to this maven shade output: [INFO] --- maven-shade-plugin:3.1.1:shade (shade-flink) @ flink-connector-elasticsearch5_2.11 --- [INFO] Including org.apache.flink:flink-connector-elasticsearch-base_2.11:jar:1.11-SNAPSHOT in the shaded jar. [INFO] Including org.elasticsearch.client:transport:jar:5.3.3 in the shaded jar. [INFO] Including org.elasticsearch:elasticsearch:jar:5.3.3 in the shaded jar. [INFO] Including org.apache.lucene:lucene-core:jar:6.4.2 in the shaded jar. [INFO] Including org.apache.lucene:lucene-analyzers-common:jar:6.4.2 in the shaded jar. [INFO] Including org.apache.lucene:lucene-backward-codecs:jar:6.4.2 in the shaded jar. [INFO] Including org.apache.lucene:lucene-grouping:jar:6.4.2 in the shaded jar. [INFO] Including org.apache.lucene:lucene-highlighter:jar:6.4.2 in the shaded jar. [INFO] Including org.apache.lucene:lucene-join:jar:6.4.2 in the shaded jar. [INFO] Including org.apache.lucene:lucene-memory:jar:6.4.2 in the shaded jar. [INFO] Including org.apache.lucene:lucene-misc:jar:6.4.2 in the shaded jar. [INFO] Including org.apache.lucene:lucene-queries:jar:6.4.2 in the shaded jar. [INFO] Including org.apache.lucene:lucene-queryparser:jar:6.4.2 in the shaded jar. [INFO] Including org.apache.lucene:lucene-sandbox:jar:6.4.2 in the shaded jar. [INFO] Including org.apache.lucene:lucene-spatial:jar:6.4.2 in the shaded jar. [INFO] Including org.apache.lucene:lucene-spatial-extras:jar:6.4.2 in the shaded jar. [INFO] Including org.apache.lucene:lucene-spatial3d:jar:6.4.2 in the shaded jar. [INFO] Including org.apache.lucene:lucene-suggest:jar:6.4.2 in the shaded jar. [INFO] Including org.elasticsearch:securesm:jar:1.1 in the shaded jar. [INFO] Including net.sf.jopt-simple:jopt-simple:jar:5.0.2 in the shaded jar. [INFO] Including com.carrotsearch:hppc:jar:0.7.1 in the shaded jar. [INFO] Including joda-time:joda-time:jar:2.5 in the shaded jar. [INFO] Including org.yaml:snakeyaml:jar:1.25 in the shaded jar. [INFO] Including com.fasterxml.jackson.core:jackson-core:jar:2.10.1 in the shaded jar. [INFO] Including com.fasterxml.jackson.dataformat:jackson-dataformat-smile:jar:2.10.1 in the shaded jar. [INFO] Including com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:jar:2.10.1 in the shaded jar. [INFO] Including com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:jar:2.10.1 in the shaded jar. [INFO] Including com.tdunning:t-digest:jar:3.0 in the shaded jar. [INFO] Including org.hdrhistogram:HdrHistogram:jar:2.1.6 in the shaded jar. [INFO] Including net.java.dev.jna:jna:jar:4.2.2 in the shaded jar. [INFO] Including org.elasticsearch.plugin:transport-netty3-client:jar:5.3.3 in the shaded jar. [INFO] Including org.elasticsearch.plugin:transport-netty4-client:jar:5.3.3 in the shaded jar. [INFO] Including io.netty:netty-buffer:jar:4.1.7.Final in the shaded jar. [INFO] Including io.netty:netty-codec:jar:4.1.7.Final in the shaded jar. [INFO] Including io.netty:netty-codec-http:jar:4.1.7.Final in the shaded jar. [INFO] Including io.netty:netty-common:jar:4.1.7.Final in the shaded jar. [INFO] Including io.netty:netty-handler:jar:4.1.7.Final in the shaded jar. [INFO] Including io.netty:netty-resolver:jar:4.1.7.Final in the shaded jar. [INFO] Including io.netty:netty-transport:jar:4.1.7.Final in the shaded jar. [INFO] Including org.elasticsearch.plugin:reindex-client:jar:5.3.3 in the shaded jar. [INFO] Including org.elasticsearch.client:rest:jar:5.3.3 in the shaded jar. [INFO] Including org.apache.httpcomponents:httpclient:jar:4.5.3 in the shaded jar. [INFO] Including org.apache.httpcomponents:httpcore:jar:4.4.6 in the shaded jar. [INFO] Including org.apache.httpcomponents:httpasyncclient:jar:4.1.2 in the shaded jar. [INFO] Including org.apache.httpcomponents:httpcore-nio:jar:4.4.5 in the shaded jar. [INFO] Including commons-codec:commons-codec:jar:1.10 in the shaded jar. [INFO] Including commons-logging:commons-logging:jar:1.1.3 in the shaded jar. [INFO] Including org.elasticsearch.plugin:lang-mustache-client:jar:5.3.3 in the shaded jar. [INFO] Including com.github.spullara.mustache.java:compiler:jar:0.9.3 in the shaded jar. [INFO] Including org.elasticsearch.plugin:percolator-client:jar:5.3.3 in the shaded jar. [INFO] Including io.netty:netty:jar:3.10.6.Final in the shaded jar. [INFO] Including org.apache.flink:force-shading:jar:1.11-SNAPSHOT in the shaded jar. [WARNING] Discovered module-info.class. Shading will break its strong encapsulation. [WARNING] Discovered module-info.class. Shading will break its strong encapsulation. [WARNING] Discovered module-info.class. Shading will break its strong encapsulation. [WARNING] Discovered module-info.class. Shading will break its strong encapsulation. [INFO] Replacing original artifact with shaded artifact. [INFO] Replacing /Users/robert/Projects/flink/flink-connectors/flink-connector-elasticsearch5/target/flink-connector-elasticsearch5_2.11-1.11-SNAPSHOT.jar with /Users/robert/Projects/flink/flink-connectors/flink-connector-elasticsearch5/target/flink-connector-elasticsearch5_2.11-1.11-SNAPSHOT-shaded.jar [INFO] Dependency-reduced POM written at: /Users/robert/Projects/flink/flink-connectors/flink-connector-elasticsearch5/target/dependency-reduced-pom.xml [INFO] [INFO] --- maven-surefire-plugin:2.22.1:test (integration-tests) @ flink-connector-elasticsearch5_2.11 --- --- .../pom.xml | 2 +- .../elasticsearch/ElasticsearchSinkBase.java | 24 +++++-- .../ElasticsearchSinkBaseTest.java | 11 ++-- .../flink-connector-elasticsearch5/pom.xml | 2 +- .../src/main/resources/META-INF/NOTICE | 64 +++++++++---------- ...eddedElasticsearchNodeEnvironmentImpl.java | 2 +- flink-end-to-end-tests/run-nightly-tests.sh | 2 +- .../test-scripts/test_quickstarts.sh | 2 +- tools/travis/splits/split_misc.sh | 2 +- tools/travis/splits/split_misc_hadoopfree.sh | 2 +- 10 files changed, 65 insertions(+), 48 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index 113a4756cd..f5c4252bd7 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -37,7 +37,7 @@ under the License. - 5.1.2 + 5.3.3 diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index c8df2c0f9f..c285112d63 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -29,6 +29,7 @@ import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.util.InstantiationUtil; import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkProcessor; import org.elasticsearch.action.bulk.BulkRequest; @@ -389,6 +390,7 @@ public abstract class ElasticsearchSinkBase extends } private class BulkProcessorListener implements BulkProcessor.Listener { + @Override public void beforeBulk(long executionId, BulkRequest request) { } @@ -398,6 +400,7 @@ public abstract class ElasticsearchSinkBase extends BulkItemResponse itemResponse; Throwable failure; RestStatus restStatus; + DocWriteRequest actionRequest; try { for (int i = 0; i < response.getItems().length; i++) { @@ -407,10 +410,19 @@ public abstract class ElasticsearchSinkBase extends LOG.error("Failed Elasticsearch item request: {}", itemResponse.getFailureMessage(), failure); restStatus = itemResponse.getFailure().getStatus(); + actionRequest = request.requests().get(i); if (restStatus == null) { - failureHandler.onFailure(request.requests().get(i), failure, -1, failureRequestIndexer); + if (actionRequest instanceof ActionRequest) { + failureHandler.onFailure((ActionRequest) actionRequest, failure, -1, failureRequestIndexer); + } else { + throw new UnsupportedOperationException("The sink currently only supports ActionRequests"); + } } else { - failureHandler.onFailure(request.requests().get(i), failure, restStatus.getStatus(), failureRequestIndexer); + if (actionRequest instanceof ActionRequest) { + failureHandler.onFailure((ActionRequest) actionRequest, failure, restStatus.getStatus(), failureRequestIndexer); + } else { + throw new UnsupportedOperationException("The sink currently only supports ActionRequests"); + } } } } @@ -431,8 +443,12 @@ public abstract class ElasticsearchSinkBase extends LOG.error("Failed Elasticsearch bulk request: {}", failure.getMessage(), failure); try { - for (ActionRequest action : request.requests()) { - failureHandler.onFailure(action, failure, -1, failureRequestIndexer); + for (DocWriteRequest writeRequest : request.requests()) { + if (writeRequest instanceof ActionRequest) { + failureHandler.onFailure((ActionRequest) writeRequest, failure, -1, failureRequestIndexer); + } else { + throw new UnsupportedOperationException("The sink currently only supports ActionRequests"); + } } } catch (Throwable t) { // fail the sink and skip the rest of the items diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java index f0e401ef2b..0648ed917d 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java @@ -26,6 +26,7 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkProcessor; @@ -419,7 +420,7 @@ public class ElasticsearchSinkBaseTest { private transient BulkRequest nextBulkRequest = new BulkRequest(); private transient MultiShotLatch flushLatch = new MultiShotLatch(); - private List mockItemFailuresList; + private List mockItemFailuresList; private Throwable nextBulkFailure; public DummyElasticsearchSink( @@ -454,7 +455,7 @@ public class ElasticsearchSinkBaseTest { *

    The list is used with corresponding order to the requests in the bulk, i.e. the first * request uses the response at index 0, the second requests uses the response at index 1, etc. */ - public void setMockItemFailuresListForNextBulkItemResponses(List mockItemFailuresList) { + public void setMockItemFailuresListForNextBulkItemResponses(List mockItemFailuresList) { this.mockItemFailuresList = mockItemFailuresList; } @@ -506,14 +507,14 @@ public class ElasticsearchSinkBaseTest { if (nextBulkFailure == null) { BulkItemResponse[] mockResponses = new BulkItemResponse[currentBulkRequest.requests().size()]; for (int i = 0; i < currentBulkRequest.requests().size(); i++) { - Throwable mockItemFailure = mockItemFailuresList.get(i); + Exception mockItemFailure = mockItemFailuresList.get(i); if (mockItemFailure == null) { // the mock response for the item is success - mockResponses[i] = new BulkItemResponse(i, "opType", mock(DocWriteResponse.class)); + mockResponses[i] = new BulkItemResponse(i, DocWriteRequest.OpType.INDEX, mock(DocWriteResponse.class)); } else { // the mock response for the item is failure - mockResponses[i] = new BulkItemResponse(i, "opType", new BulkItemResponse.Failure("index", "type", "id", mockItemFailure)); + mockResponses[i] = new BulkItemResponse(i, DocWriteRequest.OpType.INDEX, new BulkItemResponse.Failure("index", "type", "id", mockItemFailure)); } } diff --git a/flink-connectors/flink-connector-elasticsearch5/pom.xml b/flink-connectors/flink-connector-elasticsearch5/pom.xml index bbec5a5937..0a846d8b8a 100644 --- a/flink-connectors/flink-connector-elasticsearch5/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch5/pom.xml @@ -37,7 +37,7 @@ under the License. - 5.1.2 + 5.3.3 diff --git a/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/NOTICE index 5ab36a2704..c30871cf6f 100644 --- a/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/NOTICE @@ -16,44 +16,44 @@ This project bundles the following dependencies under the Apache Software Licens - commons-codec:commons-codec:1.10 - commons-logging:commons-logging:1.1.3 - io.netty:netty:3.10.6.Final -- io.netty:netty-buffer:4.1.6.Final -- io.netty:netty-codec:4.1.6.Final -- io.netty:netty-codec-http:4.1.6.Final -- io.netty:netty-common:4.1.6.Final -- io.netty:netty-handler:4.1.6.Final -- io.netty:netty-resolver:4.1.6.Final -- io.netty:netty-transport:4.1.6.Final -- joda.time:joda-time:2.5 +- io.netty:netty-buffer:4.1.7.Final +- io.netty:netty-codec:4.1.7.Final +- io.netty:netty-codec-http:4.1.7.Final +- io.netty:netty-common:4.1.7.Final +- io.netty:netty-handler:4.1.7.Final +- io.netty:netty-resolver:4.1.7.Final +- io.netty:netty-transport:4.1.7.Final +- joda-time:joda-time:2.5 - net.java.dev.jna:jna:4.2.2 - org.apache.httpcomponents:httpasynclcient:4.1.2 - org.apache.httpcomponents:httpclient:4.5.3 - org.apache.httpcomponents:httpcore:4.4.6 - org.apache.httpcomponents:httpcore-nio:4.4.5 -- org.apache.lucene:lucene-analyzers-common:6.3.0 -- org.apache.lucene:lucene-backward-codecs:6.3.0 -- org.apache.lucene:lucene-core:6.3.0 -- org.apache.lucene:lucene-grouping:6.3.0 -- org.apache.lucene:lucene-highlighter:6.3.0 -- org.apache.lucene:lucene-join:6.3.0 -- org.apache.lucene:lucene-memory:6.3.0 -- org.apache.lucene:lucene-misc:6.3.0 -- org.apache.lucene:lucene-queries:6.3.0 -- org.apache.lucene:lucene-queryparser:6.3.0 -- org.apache.lucene:lucene-sandbox:6.3.0 -- org.apache.lucene:lucene-spatial:6.3.0 -- org.apache.lucene:lucene-spatial-extras:6.3.0 -- org.apache.lucene:lucene-spatial3d:6.3.0 -- org.apache.lucene:lucene-suggest:6.3.0 -- org.elasticsearch:elasticsearch:5.1.2 +- org.apache.lucene:lucene-analyzers-common:6.4.2 +- org.apache.lucene:lucene-backward-codecs:6.4.2 +- org.apache.lucene:lucene-core:6.4.2 +- org.apache.lucene:lucene-grouping:6.4.2 +- org.apache.lucene:lucene-highlighter:6.4.2 +- org.apache.lucene:lucene-join:6.4.2 +- org.apache.lucene:lucene-memory:6.4.2 +- org.apache.lucene:lucene-misc:6.4.2 +- org.apache.lucene:lucene-queries:6.4.2 +- org.apache.lucene:lucene-queryparser:6.4.2 +- org.apache.lucene:lucene-sandbox:6.4.2 +- org.apache.lucene:lucene-spatial:6.4.2 +- org.apache.lucene:lucene-spatial-extras:6.4.2 +- org.apache.lucene:lucene-spatial3d:6.4.2 +- org.apache.lucene:lucene-suggest:6.4.2 +- org.elasticsearch:elasticsearch:5.3.3 - org.elasticsearch:securesm:1.1 -- org.elasticsearch.client:rest:5.1.2 -- org.elasticsearch.client:transport:5.1.2 -- org.elasticsearch.plugin:lang-mustache-client:5.1.2 -- org.elasticsearch.plugin:percolator-client:5.1.2 -- org.elasticsearch.plugin:reindex-client:5.1.2 -- org.elasticsearch.plugin:transport-netty3-client:5.1.2 -- org.elasticsearch.plugin:transport-netty4-client:5.1.2 -- org.yaml:snakeyaml:1.15 +- org.elasticsearch.client:rest:5.3.3 +- org.elasticsearch.client:transport:5.3.3 +- org.elasticsearch.plugin:lang-mustache-client:5.3.3 +- org.elasticsearch.plugin:percolator-client:5.3.3 +- org.elasticsearch.plugin:reindex-client:5.3.3 +- org.elasticsearch.plugin:transport-netty3-client:5.3.3 +- org.elasticsearch.plugin:transport-netty4-client:5.3.3 +- org.yaml:snakeyaml:1.25 This project bundles the following dependencies under the MIT license. (https://opensource.org/licenses/MIT) diff --git a/flink-connectors/flink-connector-elasticsearch5/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java b/flink-connectors/flink-connector-elasticsearch5/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java index 16e292d3c7..3bd20255d8 100644 --- a/flink-connectors/flink-connector-elasticsearch5/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java +++ b/flink-connectors/flink-connector-elasticsearch5/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java @@ -23,8 +23,8 @@ import org.apache.flink.streaming.connectors.elasticsearch5.ElasticsearchSinkITC import org.elasticsearch.client.Client; import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.node.InternalSettingsPreparer; import org.elasticsearch.node.Node; -import org.elasticsearch.node.internal.InternalSettingsPreparer; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.transport.Netty3Plugin; diff --git a/flink-end-to-end-tests/run-nightly-tests.sh b/flink-end-to-end-tests/run-nightly-tests.sh index 9174ed654b..a7b68517eb 100755 --- a/flink-end-to-end-tests/run-nightly-tests.sh +++ b/flink-end-to-end-tests/run-nightly-tests.sh @@ -157,7 +157,7 @@ run_test "Streaming File Sink end-to-end test" "$END_TO_END_DIR/test-scripts/tes run_test "Streaming File Sink s3 end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_file_sink.sh s3" "skip_check_exceptions" run_test "Stateful stream job upgrade end-to-end test" "$END_TO_END_DIR/test-scripts/test_stateful_stream_job_upgrade.sh 2 4" -run_test "Elasticsearch (v5.1.2) sink end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 5 https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-5.1.2.tar.gz" +run_test "Elasticsearch (v5.3.3) sink end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 5 https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-5.3.3.tar.gz" run_test "Elasticsearch (v6.3.1) sink end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 6 https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-6.3.1.tar.gz" run_test "Quickstarts Java nightly end-to-end test" "$END_TO_END_DIR/test-scripts/test_quickstarts.sh java" diff --git a/flink-end-to-end-tests/test-scripts/test_quickstarts.sh b/flink-end-to-end-tests/test-scripts/test_quickstarts.sh index 3999f96509..e1d71c66ad 100755 --- a/flink-end-to-end-tests/test-scripts/test_quickstarts.sh +++ b/flink-end-to-end-tests/test-scripts/test_quickstarts.sh @@ -95,7 +95,7 @@ else echo "Success: Elasticsearch5SinkExample.class and other user classes are included in the jar." fi -setup_elasticsearch "https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-5.1.2.tar.gz" +setup_elasticsearch "https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-5.3.3.tar.gz" wait_elasticsearch_working function shutdownAndCleanup { diff --git a/tools/travis/splits/split_misc.sh b/tools/travis/splits/split_misc.sh index 814775a193..30f1d92736 100755 --- a/tools/travis/splits/split_misc.sh +++ b/tools/travis/splits/split_misc.sh @@ -57,7 +57,7 @@ run_test "Streaming File Sink end-to-end test" "$END_TO_END_DIR/test-scripts/tes run_test "Streaming File Sink s3 end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_file_sink.sh s3" "skip_check_exceptions" run_test "Stateful stream job upgrade end-to-end test" "$END_TO_END_DIR/test-scripts/test_stateful_stream_job_upgrade.sh 2 4" -run_test "Elasticsearch (v5.1.2) sink end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 5 https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-5.1.2.tar.gz" +run_test "Elasticsearch (v5.3.3) sink end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 5 https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-5.3.3.tar.gz" run_test "Elasticsearch (v6.3.1) sink end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 6 https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-6.3.1.tar.gz" run_test "Quickstarts Java nightly end-to-end test" "$END_TO_END_DIR/test-scripts/test_quickstarts.sh java" diff --git a/tools/travis/splits/split_misc_hadoopfree.sh b/tools/travis/splits/split_misc_hadoopfree.sh index 609f257aab..bf7ac04da5 100755 --- a/tools/travis/splits/split_misc_hadoopfree.sh +++ b/tools/travis/splits/split_misc_hadoopfree.sh @@ -56,7 +56,7 @@ run_test "Streaming File Sink end-to-end test" "$END_TO_END_DIR/test-scripts/tes run_test "Streaming File Sink s3 end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_file_sink.sh s3" "skip_check_exceptions" run_test "Stateful stream job upgrade end-to-end test" "$END_TO_END_DIR/test-scripts/test_stateful_stream_job_upgrade.sh 2 4" -run_test "Elasticsearch (v5.1.2) sink end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 5 https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-5.1.2.tar.gz" +run_test "Elasticsearch (v5.3.3) sink end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 5 https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-5.3.3.tar.gz" run_test "Elasticsearch (v6.3.1) sink end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 6 https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-6.3.1.tar.gz" run_test "Quickstarts Java nightly end-to-end test" "$END_TO_END_DIR/test-scripts/test_quickstarts.sh java" -- Gitee From d723d0012c6b5e41e9d56784bc424e3942747225 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Thu, 12 Mar 2020 14:34:55 +0800 Subject: [PATCH 139/885] [FLINK-16526][table-planner-blink] Fix exception when computed column expression references a keyword column name This is a followup fix of FLINK-16018. This closes #11380 --- .../operations/SqlToOperationConverter.java | 21 ++++++++--- .../planner/calcite/FlinkPlannerImpl.scala | 2 +- .../planner/plan/stream/sql/TableScanTest.xml | 35 +++++++++++-------- .../planner/catalog/CatalogTableITCase.scala | 18 +++++----- .../plan/stream/sql/TableScanTest.scala | 23 ++++++++++++ 5 files changed, 71 insertions(+), 28 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/operations/SqlToOperationConverter.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/operations/SqlToOperationConverter.java index 6e7c96cff3..bf8b4f5b1c 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/operations/SqlToOperationConverter.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/operations/SqlToOperationConverter.java @@ -78,10 +78,13 @@ import org.apache.flink.util.StringUtils; import org.apache.calcite.rel.RelRoot; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlDialect; import org.apache.calcite.sql.SqlIdentifier; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlNodeList; +import org.apache.calcite.sql.dialect.CalciteSqlDialect; +import org.apache.calcite.sql.parser.SqlParser; import org.apache.calcite.sql.validate.SqlValidator; import java.util.HashMap; @@ -102,8 +105,8 @@ import java.util.stream.Collectors; * {@link org.apache.flink.table.delegation.Planner}. */ public class SqlToOperationConverter { - private FlinkPlannerImpl flinkPlanner; - private CatalogManager catalogManager; + private final FlinkPlannerImpl flinkPlanner; + private final CatalogManager catalogManager; //~ Constructors ----------------------------------------------------------- @@ -514,7 +517,7 @@ public class SqlToOperationConverter { builder.field(call.operand(1).toString(), TypeConversions.fromLogicalToDataType( FlinkTypeFactory.toLogicalType(validatedType)), - validatedExpr.toString()); + getQuotedSqlString(validatedExpr)); // add computed column into all field list String fieldName = call.operand(1).toString(); allFieldNamesToTypes.put(fieldName, validatedType); @@ -533,12 +536,22 @@ public class SqlToOperationConverter { DataType exprDataType = TypeConversions.fromLogicalToDataType( FlinkTypeFactory.toLogicalType(validatedType)); // use the qualified SQL expression string - builder.watermark(rowtimeAttribute, validated.toString(), exprDataType); + builder.watermark(rowtimeAttribute, getQuotedSqlString(validated), exprDataType); }); return builder.build(); } + private String getQuotedSqlString(SqlNode sqlNode) { + SqlParser.Config parserConfig = flinkPlanner.config().getParserConfig(); + SqlDialect dialect = new CalciteSqlDialect(SqlDialect.EMPTY_CONTEXT + .withQuotedCasing(parserConfig.unquotedCasing()) + .withConformance(parserConfig.conformance()) + .withUnquotedCasing(parserConfig.unquotedCasing()) + .withIdentifierQuoteString(parserConfig.quoting().string)); + return sqlNode.toSqlString(dialect).getSql(); + } + private PlannerQueryOperation toQueryOperation(FlinkPlannerImpl planner, SqlNode validated) { // transform to a relational tree RelRoot relational = planner.rel(validated); diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala index 0ad903be21..6426436cef 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala @@ -46,7 +46,7 @@ import scala.collection.JavaConverters._ * The main difference is that we do not create a new RelOptPlanner in the ready() method. */ class FlinkPlannerImpl( - config: FrameworkConfig, + val config: FrameworkConfig, catalogReaderSupplier: JFunction[JBoolean, CalciteCatalogReader], typeFactory: FlinkTypeFactory, cluster: RelOptCluster) { diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml index bce11d9aa6..693a1713f2 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml @@ -19,26 +19,22 @@ limitations under the License. - - - 1]]> - - (a, 1)]) +- WatermarkAssigner(rowtime=[ts], watermark=[-(ts, 1:INTERVAL SECOND)]) +- TableSourceScan(table=[[default_catalog, default_database, src, source: [CollectionTableSource(ts, a, b)]]], fields=[ts, a, b]) ]]> - - - - - - - - - - + + + + + + + + + + + diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala index 6b16dfc640..9e2d628af0 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala @@ -408,11 +408,11 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase { toRow(2, "2019-09-10 9:23:44") ) val expected = List( - toRow(1, "1990-02-10 12:34:56", 1), - toRow(2, "2019-09-10 9:23:41", 2), - toRow(3, "2019-09-10 9:23:42", 3), - toRow(1, "2019-09-10 9:23:43", 1), - toRow(2, "2019-09-10 9:23:44", 2) + toRow(1, "1990-02-10 12:34:56", 1, "1990-02-10 12:34:56"), + toRow(2, "2019-09-10 9:23:41", 2, "2019-09-10 9:23:41"), + toRow(3, "2019-09-10 9:23:42", 3, "2019-09-10 9:23:42"), + toRow(1, "2019-09-10 9:23:43", 1, "2019-09-10 9:23:43"), + toRow(2, "2019-09-10 9:23:44", 2, "2019-09-10 9:23:44") ) TestCollectionTableFactory.initData(sourceData) tableEnv.registerFunction("my_udf", Func0) @@ -421,7 +421,8 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase { |create table t1( | a int, | `time` varchar, - | c as my_udf(a) + | c as my_udf(a), + | d as `time` |) with ( | 'connector' = 'COLLECTION' |) @@ -431,7 +432,8 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase { |create table t2( | a int, | `time` varchar, - | c int not null + | c int not null, + | d varchar |) with ( | 'connector' = 'COLLECTION' |) @@ -439,7 +441,7 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase { val query = """ |insert into t2 - |select t1.a, t1.`time`, t1.c from t1 + |select t1.a, t1.`time`, t1.c, t1.d from t1 """.stripMargin tableEnv.sqlUpdate(sourceDDL) tableEnv.sqlUpdate(sinkDDL) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala index 017cc86f91..edc9c3b1d2 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala @@ -97,4 +97,27 @@ class TableScanTest extends TableTestBase { """.stripMargin) util.verifyPlan("SELECT * FROM t1") } + + @Test + def testKeywordsWithWatermarkComputedColumn(): Unit = { + // Create table with field as atom expression. + util.tableEnv.registerFunction("my_udf", Func0) + util.addTable( + s""" + |create table t1( + | a int, + | b varchar, + | `time` time, + | mytime as `time`, + | `current_time` as current_time, + | json_row ROW<`timestamp` TIMESTAMP(3)>, + | `timestamp` AS json_row.`timestamp`, + | WATERMARK FOR `timestamp` AS `timestamp` + |) with ( + | 'connector' = 'COLLECTION', + | 'is-bounded' = 'false' + |) + """.stripMargin) + util.verifyPlan("SELECT * FROM t1") + } } -- Gitee From 663af45c7f403eb6724852915bf2078241927258 Mon Sep 17 00:00:00 2001 From: zhengcanbin Date: Thu, 12 Mar 2020 15:36:10 +0800 Subject: [PATCH 140/885] [FLINK-16493][k8s] Use enum type instead of string type for KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE This closes #11346 . --- .../generated/kubernetes_config_configuration.html | 6 +++--- .../configuration/KubernetesConfigOptions.java | 10 +++++----- .../kubernetes/kubeclient/Fabric8FlinkKubeClient.java | 5 +++-- .../decorators/AbstractServiceDecorator.java | 5 +++-- .../decorators/ExternalServiceDecorator.java | 5 ++--- .../decorators/InternalServiceDecorator.java | 4 ++-- .../parameters/KubernetesJobManagerParameters.java | 5 ++--- .../decorators/ExternalServiceDecoratorTest.java | 9 ++++++--- .../parameters/KubernetesJobManagerParametersTest.java | 4 ++-- 9 files changed, 28 insertions(+), 25 deletions(-) diff --git a/docs/_includes/generated/kubernetes_config_configuration.html b/docs/_includes/generated/kubernetes_config_configuration.html index 86f8ec6b30..1c9dff1e13 100644 --- a/docs/_includes/generated/kubernetes_config_configuration.html +++ b/docs/_includes/generated/kubernetes_config_configuration.html @@ -88,9 +88,9 @@

    kubernetes.rest-service.exposed.type
    "LoadBalancer"StringIt could be ClusterIP/NodePort/LoadBalancer(default). When set to ClusterIP, the rest servicewill not be created.LoadBalancer

    Enum

    Possible values: [ClusterIP, NodePort, LoadBalancer]
    The type of the rest service (ClusterIP or NodePort or LoadBalancer). When set to ClusterIP, the rest service will not be created.
    kubernetes.service.create-timeout
    taskmanager.memory.jvm-metaspace.size
    96 mb256 mb MemorySize JVM Metaspace Size for the TaskExecutors.
    taskmanager.memory.jvm-metaspace.size
    96 mb256 mb MemorySize JVM Metaspace Size for the TaskExecutors.
    - - - - - - - - - - - - - - - - - - - - - - - - - -
    APIsDescription
    add_python_file(file_path) -

    Adds python file dependencies which could be python files, python packages or local directories. They will be added to the PYTHONPATH of the python UDF worker.

    -{% highlight python %} -table_env.add_python_file(file_path) -{% endhighlight %} -
    set_python_requirements(requirements_file_path, requirements_cache_dir=None) -

    Specifies a requirements.txt file which defines the third-party dependencies. These dependencies will be installed to a temporary directory and added to the PYTHONPATH of the python UDF worker. For the dependencies which could not be accessed in the cluster, a directory which contains the installation packages of these dependencies could be specified using the parameter "requirements_cached_dir". It will be uploaded to the cluster to support offline installation.

    -{% highlight python %} -# commands executed in shell -echo numpy==1.16.5 > requirements.txt -pip download -d cached_dir -r requirements.txt --no-binary :all: - -# python code -table_env.set_python_requirements("requirements.txt", "cached_dir") -{% endhighlight %} -

    Please make sure the installation packages matches the platform of the cluster and the python version used. These packages will be installed using pip, so also make sure the version of Pip (version >= 7.1.0) and the version of SetupTools (version >= 37.0.0).

    -
    add_python_archive(archive_path, target_dir=None) -

    Adds a python archive file dependency. The file will be extracted to the working directory of python UDF worker. If the parameter "target_dir" is specified, the archive file will be extracted to a directory named "target_dir". Otherwise, the archive file will be extracted to a directory with the same name of the archive file.

    -{% highlight python %} -# command executed in shell -# assert the relative path of python interpreter is py_env/bin/python -zip -r py_env.zip py_env - -# python code -table_env.add_python_archive("py_env.zip") -# or -table_env.add_python_archive("py_env.zip", "myenv") - -# the files contained in the archive file can be accessed in UDF -def my_udf(): - with open("myenv/py_env/data/data.txt") as f: - ... -{% endhighlight %} -

    Please make sure the uploaded python environment matches the platform that the cluster is running on. Currently only zip-format is supported. i.e. zip, jar, whl, egg, etc.

    -
    set_python_executable(python_exec) -

    Sets the path of the python interpreter which is used to execute the python udf workers, e.g., "/usr/local/bin/python3".

    -{% highlight python %} -table_env.add_python_archive("py_env.zip") -table_env.get_config().set_python_executable("py_env.zip/py_env/bin/python") +table_env.sql_query("SELECT string, bigint, hash_code(bigint) FROM MyTable") {% endhighlight %} -

    Please make sure that the specified environment matches the platform that the cluster is running on.

    -
    +There are many ways to define a Python scalar function besides extending the base class `ScalarFunction`. +Please refer to the [Python Scalar Function]({{ site.baseurl }}/dev/table/python/python_udfs.html#scalar-functions) documentation for more details. diff --git a/docs/dev/table/functions/udfs.zh.md b/docs/dev/table/functions/udfs.zh.md index 5eec2f1692..c35f56a186 100644 --- a/docs/dev/table/functions/udfs.zh.md +++ b/docs/dev/table/functions/udfs.zh.md @@ -134,35 +134,12 @@ object TimestampModifier extends ScalarFunction {
    -Note Python 3.5+ and apache-beam==2.19.0 are required to run the Python scalar function. +In order to define a Python scalar function, one can extend the base class `ScalarFunction` in `pyflink.table.udf` and implement an evaluation method. The behavior of a Python scalar function is determined by the evaluation method which is named `eval`. -Note By default PyFlink uses the command “python” to run the python udf workers. Before starting cluster, run the following command to confirm that it meets the requirements: - -{% highlight bash %} -$ python --version -# the version printed here must be 3.5+ -$ python -m pip install apache-beam==2.19.0 -{% endhighlight %} - -It supports to use both Java/Scala scalar functions and Python scalar functions in Python Table API and SQL. In order to define a Python scalar function, one can extend the base class `ScalarFunction` in `pyflink.table.udf` and implement an evaluation method. The behavior of a Python scalar function is determined by the evaluation method. An evaluation method must be named `eval`. Evaluation method can also support variable arguments, such as `eval(*args)`. - -The following example shows how to define your own Java and Python hash code functions, register them in the TableEnvironment, and call them in a query. Note that you can configure your scalar function via a constructor before it is registered: +The following example shows how to define your own Python hash code function, register it in the TableEnvironment, and call it in a query. Note that you can configure your scalar function via a constructor before it is registered: {% highlight python %} -''' -Java code: - -// The Java class must have a public no-argument constructor and can be founded in current Java classloader. -public class HashCode extends ScalarFunction { - private int factor = 12; - - public int eval(String s) { - return s.hashCode() * factor; - } -} -''' - -class PyHashCode(ScalarFunction): +class HashCode(ScalarFunction): def __init__(self): self.factor = 12 @@ -171,128 +148,18 @@ class PyHashCode(ScalarFunction): table_env = BatchTableEnvironment.create(env) -# register the Java function -table_env.register_java_function("hashCode", "my.java.function.HashCode") - # register the Python function -table_env.register_function("py_hash_code", udf(PyHashCode(), DataTypes.BIGINT(), DataTypes.BIGINT())) +table_env.register_function("hash_code", udf(HashCode(), DataTypes.BIGINT(), DataTypes.BIGINT())) # use the function in Python Table API -my_table.select("string, bigint, string.hashCode(), hashCode(string), bigint.py_hash_code(), py_hash_code(bigint)") +my_table.select("string, bigint, string.hash_code(), hash_code(string)") # use the function in SQL API -table_env.sql_query("SELECT string, bigint, hashCode(string), py_hash_code(bigint) FROM MyTable") -{% endhighlight %} - -There are many ways to define a Python scalar function besides extending the base class `ScalarFunction`. The following example shows the different ways to define a Python scalar function which takes two columns of bigint as input parameters and returns the sum of them as the result. - -{% highlight python %} -# option 1: extending the base class `ScalarFunction` -class Add(ScalarFunction): - def eval(self, i, j): - return i + j - -add = udf(Add(), [DataTypes.BIGINT(), DataTypes.BIGINT()], DataTypes.BIGINT()) - -# option 2: Python function -@udf(input_types=[DataTypes.BIGINT(), DataTypes.BIGINT()], result_type=DataTypes.BIGINT()) -def add(i, j): - return i + j - -# option 3: lambda function -add = udf(lambda i, j: i + j, [DataTypes.BIGINT(), DataTypes.BIGINT()], DataTypes.BIGINT()) - -# option 4: callable function -class CallableAdd(object): - def __call__(self, i, j): - return i + j - -add = udf(CallableAdd(), [DataTypes.BIGINT(), DataTypes.BIGINT()], DataTypes.BIGINT()) - -# option 5: partial function -def partial_add(i, j, k): - return i + j + k - -add = udf(functools.partial(partial_add, k=1), [DataTypes.BIGINT(), DataTypes.BIGINT()], - DataTypes.BIGINT()) - -# register the Python function -table_env.register_function("add", add) -# use the function in Python Table API -my_table.select("add(a, b)") -{% endhighlight %} - -If the python scalar function depends on third-party dependencies, you can specify the dependencies with the following table APIs or through command line directly when submitting the job. - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    APIsDescription
    add_python_file(file_path) -

    Adds python file dependencies which could be python files, python packages or local directories. They will be added to the PYTHONPATH of the python UDF worker.

    -{% highlight python %} -table_env.add_python_file(file_path) -{% endhighlight %} -
    set_python_requirements(requirements_file_path, requirements_cache_dir=None) -

    Specifies a requirements.txt file which defines the third-party dependencies. These dependencies will be installed to a temporary directory and added to the PYTHONPATH of the python UDF worker. For the dependencies which could not be accessed in the cluster, a directory which contains the installation packages of these dependencies could be specified using the parameter "requirements_cached_dir". It will be uploaded to the cluster to support offline installation.

    -{% highlight python %} -# commands executed in shell -echo numpy==1.16.5 > requirements.txt -pip download -d cached_dir -r requirements.txt --no-binary :all: - -# python code -table_env.set_python_requirements("requirements.txt", "cached_dir") -{% endhighlight %} -

    Please make sure the installation packages matches the platform of the cluster and the python version used. These packages will be installed using pip, so also make sure the version of Pip (version >= 7.1.0) and the version of SetupTools (version >= 37.0.0).

    -
    add_python_archive(archive_path, target_dir=None) -

    Adds a python archive file dependency. The file will be extracted to the working directory of python UDF worker. If the parameter "target_dir" is specified, the archive file will be extracted to a directory named "target_dir". Otherwise, the archive file will be extracted to a directory with the same name of the archive file.

    -{% highlight python %} -# command executed in shell -# assert the relative path of python interpreter is py_env/bin/python -zip -r py_env.zip py_env - -# python code -table_env.add_python_archive("py_env.zip") -# or -table_env.add_python_archive("py_env.zip", "myenv") - -# the files contained in the archive file can be accessed in UDF -def my_udf(): - with open("myenv/py_env/data/data.txt") as f: - ... -{% endhighlight %} -

    Please make sure the uploaded python environment matches the platform that the cluster is running on. Currently only zip-format is supported. i.e. zip, jar, whl, egg, etc.

    -
    set_python_executable(python_exec) -

    Sets the path of the python interpreter which is used to execute the python udf workers, e.g., "/usr/local/bin/python3".

    -{% highlight python %} -table_env.add_python_archive("py_env.zip") -table_env.get_config().set_python_executable("py_env.zip/py_env/bin/python") +table_env.sql_query("SELECT string, bigint, hash_code(bigint) FROM MyTable") {% endhighlight %} -

    Please make sure that the specified environment matches the platform that the cluster is running on.

    -
    +There are many ways to define a Python scalar function besides extending the base class `ScalarFunction`. +Please refer to the [Python Scalar Function]({{ site.baseurl }}/zh/dev/table/python/python_udfs.html#scalar-functions) documentation for more details.
    diff --git a/docs/dev/table/python/dependency_management.md b/docs/dev/table/python/dependency_management.md new file mode 100644 index 0000000000..adb8cbe616 --- /dev/null +++ b/docs/dev/table/python/dependency_management.md @@ -0,0 +1,94 @@ +--- +title: "Dependency Management" +nav-parent_id: python_tableapi +nav-pos: 100 +--- + + +If third-party dependencies are used, you can specify the dependencies with the following Python Table APIs or through command line arguments directly when submitting the job. + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    APIsDescription
    add_python_file(file_path) +

    Adds python file dependencies which could be python files, python packages or local directories. They will be added to the PYTHONPATH of the python UDF worker.

    +{% highlight python %} +table_env.add_python_file(file_path) +{% endhighlight %} +
    set_python_requirements(requirements_file_path, requirements_cache_dir=None) +

    Specifies a requirements.txt file which defines the third-party dependencies. These dependencies will be installed to a temporary directory and added to the PYTHONPATH of the python UDF worker. For the dependencies which could not be accessed in the cluster, a directory which contains the installation packages of these dependencies could be specified using the parameter "requirements_cached_dir". It will be uploaded to the cluster to support offline installation.

    +{% highlight python %} +# commands executed in shell +echo numpy==1.16.5 > requirements.txt +pip download -d cached_dir -r requirements.txt --no-binary :all: + +# python code +table_env.set_python_requirements("requirements.txt", "cached_dir") +{% endhighlight %} +

    Please make sure the installation packages matches the platform of the cluster and the python version used. These packages will be installed using pip, so also make sure the version of Pip (version >= 7.1.0) and the version of SetupTools (version >= 37.0.0).

    +
    add_python_archive(archive_path, target_dir=None) +

    Adds a python archive file dependency. The file will be extracted to the working directory of python UDF worker. If the parameter "target_dir" is specified, the archive file will be extracted to a directory named "target_dir". Otherwise, the archive file will be extracted to a directory with the same name of the archive file.

    +{% highlight python %} +# command executed in shell +# assert the relative path of python interpreter is py_env/bin/python +zip -r py_env.zip py_env + +# python code +table_env.add_python_archive("py_env.zip") +# or +table_env.add_python_archive("py_env.zip", "myenv") + +# the files contained in the archive file can be accessed in UDF +def my_udf(): + with open("myenv/py_env/data/data.txt") as f: + ... +{% endhighlight %} +

    Please make sure the uploaded python environment matches the platform that the cluster is running on. Currently only zip-format is supported. i.e. zip, jar, whl, egg, etc.

    +
    set_python_executable(python_exec) +

    Sets the path of the python interpreter which is used to execute the python udf workers, e.g., "/usr/local/bin/python3".

    +{% highlight python %} +table_env.add_python_archive("py_env.zip") +table_env.get_config().set_python_executable("py_env.zip/py_env/bin/python") +{% endhighlight %} +

    Please make sure that the specified environment matches the platform that the cluster is running on.

    +
    diff --git a/docs/dev/table/python/dependency_management.zh.md b/docs/dev/table/python/dependency_management.zh.md new file mode 100644 index 0000000000..5aaf771191 --- /dev/null +++ b/docs/dev/table/python/dependency_management.zh.md @@ -0,0 +1,94 @@ +--- +title: "依赖管理" +nav-parent_id: python_tableapi +nav-pos: 100 +--- + + +If third-party dependencies are used, you can specify the dependencies with the following Python Table APIs or through command line arguments directly when submitting the job. + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    APIsDescription
    add_python_file(file_path) +

    Adds python file dependencies which could be python files, python packages or local directories. They will be added to the PYTHONPATH of the python UDF worker.

    +{% highlight python %} +table_env.add_python_file(file_path) +{% endhighlight %} +
    set_python_requirements(requirements_file_path, requirements_cache_dir=None) +

    Specifies a requirements.txt file which defines the third-party dependencies. These dependencies will be installed to a temporary directory and added to the PYTHONPATH of the python UDF worker. For the dependencies which could not be accessed in the cluster, a directory which contains the installation packages of these dependencies could be specified using the parameter "requirements_cached_dir". It will be uploaded to the cluster to support offline installation.

    +{% highlight python %} +# commands executed in shell +echo numpy==1.16.5 > requirements.txt +pip download -d cached_dir -r requirements.txt --no-binary :all: + +# python code +table_env.set_python_requirements("requirements.txt", "cached_dir") +{% endhighlight %} +

    Please make sure the installation packages matches the platform of the cluster and the python version used. These packages will be installed using pip, so also make sure the version of Pip (version >= 7.1.0) and the version of SetupTools (version >= 37.0.0).

    +
    add_python_archive(archive_path, target_dir=None) +

    Adds a python archive file dependency. The file will be extracted to the working directory of python UDF worker. If the parameter "target_dir" is specified, the archive file will be extracted to a directory named "target_dir". Otherwise, the archive file will be extracted to a directory with the same name of the archive file.

    +{% highlight python %} +# command executed in shell +# assert the relative path of python interpreter is py_env/bin/python +zip -r py_env.zip py_env + +# python code +table_env.add_python_archive("py_env.zip") +# or +table_env.add_python_archive("py_env.zip", "myenv") + +# the files contained in the archive file can be accessed in UDF +def my_udf(): + with open("myenv/py_env/data/data.txt") as f: + ... +{% endhighlight %} +

    Please make sure the uploaded python environment matches the platform that the cluster is running on. Currently only zip-format is supported. i.e. zip, jar, whl, egg, etc.

    +
    set_python_executable(python_exec) +

    Sets the path of the python interpreter which is used to execute the python udf workers, e.g., "/usr/local/bin/python3".

    +{% highlight python %} +table_env.add_python_archive("py_env.zip") +table_env.get_config().set_python_executable("py_env.zip/py_env/bin/python") +{% endhighlight %} +

    Please make sure that the specified environment matches the platform that the cluster is running on.

    +
    diff --git a/docs/dev/table/python/index.md b/docs/dev/table/python/index.md new file mode 100644 index 0000000000..4163374148 --- /dev/null +++ b/docs/dev/table/python/index.md @@ -0,0 +1,35 @@ +--- +title: "Python Table API" +nav-id: python_tableapi +nav-parent_id: tableapi +nav-pos: 100 +nav-show_overview: true +--- + + +Python Table API allows users to develop [Table API]({{ site.baseurl }}/dev/table/tableApi.html) programs using the Python language. +Apache Flink has provided Python Table API support since 1.9.0. + +## Where to go next? + +- [Installation]({{ site.baseurl }}/dev/table/python/installation.html): Introduction of how to set up the Python Table API execution environment. +- [User-defined Functions]({{ site.baseurl }}/dev/table/python/python_udfs.html): Explanation of how to define Python user-defined functions. +- [Dependency Management]({{ site.baseurl }}/dev/table/python/dependency_management.html): Specification of how to use third-part dependencies in a Python Table API program. +- [Configuration]({{ site.baseurl }}/dev/table/python/python_config.html): Description of the config options available for Python Table API programs. diff --git a/docs/dev/table/python/index.zh.md b/docs/dev/table/python/index.zh.md new file mode 100644 index 0000000000..fa1323c2de --- /dev/null +++ b/docs/dev/table/python/index.zh.md @@ -0,0 +1,35 @@ +--- +title: "Python Table API" +nav-id: python_tableapi +nav-parent_id: tableapi +nav-pos: 100 +nav-show_overview: true +--- + + +Python Table API allows users to develop [Table API]({{ site.baseurl }}/zh/dev/table/tableApi.html) programs using the Python language. +Apache Flink has provided Python Table API support since 1.9.0. + +## Where to go next? + +- [环境安装]({{ site.baseurl }}/zh/dev/table/python/installation.html): Introduction of how to set up the Python Table API execution environment. +- [自定义函数]({{ site.baseurl }}/zh/dev/table/python/python_udfs.html): Explanation of how to define Python user-defined functions. +- [依赖管理]({{ site.baseurl }}/zh/dev/table/python/dependency_management.html): Specification of how to use third-part dependencies in a Python Table API program. +- [配置]({{ site.baseurl }}/zh/dev/table/python/python_config.html): Description of the config options available for Python Table API programs. diff --git a/docs/dev/table/python/installation.md b/docs/dev/table/python/installation.md new file mode 100644 index 0000000000..0dffb77d20 --- /dev/null +++ b/docs/dev/table/python/installation.md @@ -0,0 +1,44 @@ +--- +title: "Installation" +nav-parent_id: python_tableapi +nav-pos: 10 +--- + + +* This will be replaced by the TOC +{:toc} + +## Environment Requirements +Note Python 3.5+ is required for PyFlink. Please run the following command to make sure that it meets the requirements: + +{% highlight bash %} +$ python --version +# the version printed here must be 3.5+ +{% endhighlight %} + +## Installation of PyFlink + +PyFlink has already been deployed to PyPi and can be installed as following: + +{% highlight bash %} +$ python -m pip install apache-flink +{% endhighlight %} + +You can also build PyFlink from source by following the [development guide]({{ site.baseurl }}/flinkDev/building.html#build-pyflink). diff --git a/docs/dev/table/python/installation.zh.md b/docs/dev/table/python/installation.zh.md new file mode 100644 index 0000000000..e07660a402 --- /dev/null +++ b/docs/dev/table/python/installation.zh.md @@ -0,0 +1,44 @@ +--- +title: "环境安装" +nav-parent_id: python_tableapi +nav-pos: 10 +--- + + +* This will be replaced by the TOC +{:toc} + +## Environment Requirements +Note Python 3.5+ is required for PyFlink. Please run the following command to make sure that it meets the requirements: + +{% highlight bash %} +$ python --version +# the version printed here must be 3.5+ +{% endhighlight %} + +## Installation of PyFlink + +PyFlink has already been deployed to PyPi and can be installed as following: + +{% highlight bash %} +$ python -m pip install apache-flink +{% endhighlight %} + +You can also build PyFlink from source by following the [development guide]({{ site.baseurl }}/zh/flinkDev/building.html#build-pyflink). diff --git a/docs/dev/table/python/python_config.md b/docs/dev/table/python/python_config.md new file mode 100644 index 0000000000..e25fbf5403 --- /dev/null +++ b/docs/dev/table/python/python_config.md @@ -0,0 +1,32 @@ +--- +title: "Configuration" +nav-parent_id: python_tableapi +nav-pos: 120 +--- + + +Depending on the requirements of a Python Table API program, it might be necessary to adjust certain parameters for optimization. +All the config options available for Java/Scala Table API program could also be used in the Python Table API program. +You could refer to the [Table API Configuration]({{ site.baseurl }}/dev/table/config.html) for more details on all the available config options for Java/Scala Table API programs. +It has also provided examples on how to set the config options in a Table API program. + +## Python Options + +{% include generated/python_configuration.html %} diff --git a/docs/dev/table/python/python_config.zh.md b/docs/dev/table/python/python_config.zh.md new file mode 100644 index 0000000000..fa4f59d832 --- /dev/null +++ b/docs/dev/table/python/python_config.zh.md @@ -0,0 +1,32 @@ +--- +title: "配置" +nav-parent_id: python_tableapi +nav-pos: 120 +--- + + +Depending on the requirements of a Python Table API program, it might be necessary to adjust certain parameters for optimization. +All the config options available for Java/Scala Table API program could also be used in the Python Table API program. +You could refer to the [Table API Configuration]({{ site.baseurl }}/zh/dev/table/config.html) for more details on all the available config options for Java/Scala Table API programs. +It has also provided examples on how to set the config options in a Table API program. + +## Python Options + +{% include generated/python_configuration.html %} diff --git a/docs/dev/table/python/python_udfs.md b/docs/dev/table/python/python_udfs.md new file mode 100644 index 0000000000..44a30ffb0c --- /dev/null +++ b/docs/dev/table/python/python_udfs.md @@ -0,0 +1,125 @@ +--- +title: "User-defined Functions" +nav-parent_id: python_tableapi +nav-pos: 20 +--- + + +User-defined functions are important features, because they significantly extend the expressiveness of Python Table API programs. + +* This will be replaced by the TOC +{:toc} + +## Scalar Functions +It supports to use Python scalar functions in Python Table API programs. In order to define a Python scalar function, +one can extend the base class `ScalarFunction` in `pyflink.table.udf` and implement an evaluation method. +The behavior of a Python scalar function is defined by the evaluation method which is named `eval`. +The evaluation method can support variable arguments, such as `eval(*args)`. + +The following example shows how to define your own Python hash code function, register it in the TableEnvironment, and call it in a query. +Note that you can configure your scalar function via a constructor before it is registered: + +{% highlight python %} +class HashCode(ScalarFunction): + def __init__(self): + self.factor = 12 + + def eval(self, s): + return hash(s) * self.factor + +table_env = BatchTableEnvironment.create(env) + +# register the Python function +table_env.register_function("hash_code", udf(HashCode(), DataTypes.BIGINT(), DataTypes.BIGINT())) + +# use the Python function in Python Table API +my_table.select("string, bigint, bigint.hash_code(), hash_code(bigint)") + +# use the Python function in SQL API +table_env.sql_query("SELECT string, bigint, hash_code(bigint) FROM MyTable") +{% endhighlight %} + +It also supports to use Java/Scala scalar functions in Python Table API programs. + +{% highlight python %} +''' +Java code: + +// The Java class must have a public no-argument constructor and can be founded in current Java classloader. +public class HashCode extends ScalarFunction { + private int factor = 12; + + public int eval(String s) { + return s.hashCode() * factor; + } +} +''' + +table_env = BatchTableEnvironment.create(env) + +# register the Java function +table_env.register_java_function("hash_code", "my.java.function.HashCode") + +# use the Java function in Python Table API +my_table.select("string.hash_code(), hash_code(string)") + +# use the Java function in SQL API +table_env.sql_query("SELECT string, bigint, hash_code(string) FROM MyTable") +{% endhighlight %} + +There are many ways to define a Python scalar function besides extending the base class `ScalarFunction`. +The following examples show the different ways to define a Python scalar function which takes two columns of +bigint as the input parameters and returns the sum of them as the result. + +{% highlight python %} +# option 1: extending the base class `ScalarFunction` +class Add(ScalarFunction): + def eval(self, i, j): + return i + j + +add = udf(Add(), [DataTypes.BIGINT(), DataTypes.BIGINT()], DataTypes.BIGINT()) + +# option 2: Python function +@udf(input_types=[DataTypes.BIGINT(), DataTypes.BIGINT()], result_type=DataTypes.BIGINT()) +def add(i, j): + return i + j + +# option 3: lambda function +add = udf(lambda i, j: i + j, [DataTypes.BIGINT(), DataTypes.BIGINT()], DataTypes.BIGINT()) + +# option 4: callable function +class CallableAdd(object): + def __call__(self, i, j): + return i + j + +add = udf(CallableAdd(), [DataTypes.BIGINT(), DataTypes.BIGINT()], DataTypes.BIGINT()) + +# option 5: partial function +def partial_add(i, j, k): + return i + j + k + +add = udf(functools.partial(partial_add, k=1), [DataTypes.BIGINT(), DataTypes.BIGINT()], + DataTypes.BIGINT()) + +# register the Python function +table_env.register_function("add", add) +# use the function in Python Table API +my_table.select("add(a, b)") +{% endhighlight %} diff --git a/docs/dev/table/python/python_udfs.zh.md b/docs/dev/table/python/python_udfs.zh.md new file mode 100644 index 0000000000..7acba1fd57 --- /dev/null +++ b/docs/dev/table/python/python_udfs.zh.md @@ -0,0 +1,125 @@ +--- +title: "自定义函数" +nav-parent_id: python_tableapi +nav-pos: 20 +--- + + +User-defined functions are important features, because they significantly extend the expressiveness of Python Table API programs. + +* This will be replaced by the TOC +{:toc} + +## Scalar Functions +It supports to use Python scalar functions in Python Table API programs. In order to define a Python scalar function, +one can extend the base class `ScalarFunction` in `pyflink.table.udf` and implement an evaluation method. +The behavior of a Python scalar function is defined by the evaluation method which is named `eval`. +The evaluation method can support variable arguments, such as `eval(*args)`. + +The following example shows how to define your own Python hash code function, register it in the TableEnvironment, and call it in a query. +Note that you can configure your scalar function via a constructor before it is registered: + +{% highlight python %} +class HashCode(ScalarFunction): + def __init__(self): + self.factor = 12 + + def eval(self, s): + return hash(s) * self.factor + +table_env = BatchTableEnvironment.create(env) + +# register the Python function +table_env.register_function("hash_code", udf(HashCode(), DataTypes.BIGINT(), DataTypes.BIGINT())) + +# use the Python function in Python Table API +my_table.select("string, bigint, bigint.hash_code(), hash_code(bigint)") + +# use the Python function in SQL API +table_env.sql_query("SELECT string, bigint, hash_code(bigint) FROM MyTable") +{% endhighlight %} + +It also supports to use Java/Scala scalar functions in Python Table API programs. + +{% highlight python %} +''' +Java code: + +// The Java class must have a public no-argument constructor and can be founded in current Java classloader. +public class HashCode extends ScalarFunction { + private int factor = 12; + + public int eval(String s) { + return s.hashCode() * factor; + } +} +''' + +table_env = BatchTableEnvironment.create(env) + +# register the Java function +table_env.register_java_function("hash_code", "my.java.function.HashCode") + +# use the Java function in Python Table API +my_table.select("string.hash_code(), hash_code(string)") + +# use the Java function in SQL API +table_env.sql_query("SELECT string, bigint, hash_code(string) FROM MyTable") +{% endhighlight %} + +There are many ways to define a Python scalar function besides extending the base class `ScalarFunction`. +The following examples show the different ways to define a Python scalar function which takes two columns of +bigint as the input parameters and returns the sum of them as the result. + +{% highlight python %} +# option 1: extending the base class `ScalarFunction` +class Add(ScalarFunction): + def eval(self, i, j): + return i + j + +add = udf(Add(), [DataTypes.BIGINT(), DataTypes.BIGINT()], DataTypes.BIGINT()) + +# option 2: Python function +@udf(input_types=[DataTypes.BIGINT(), DataTypes.BIGINT()], result_type=DataTypes.BIGINT()) +def add(i, j): + return i + j + +# option 3: lambda function +add = udf(lambda i, j: i + j, [DataTypes.BIGINT(), DataTypes.BIGINT()], DataTypes.BIGINT()) + +# option 4: callable function +class CallableAdd(object): + def __call__(self, i, j): + return i + j + +add = udf(CallableAdd(), [DataTypes.BIGINT(), DataTypes.BIGINT()], DataTypes.BIGINT()) + +# option 5: partial function +def partial_add(i, j, k): + return i + j + k + +add = udf(functools.partial(partial_add, k=1), [DataTypes.BIGINT(), DataTypes.BIGINT()], + DataTypes.BIGINT()) + +# register the Python function +table_env.register_function("add", add) +# use the function in Python Table API +my_table.select("add(a, b)") +{% endhighlight %} diff --git a/docs/getting-started/walkthroughs/python_table_api.md b/docs/getting-started/walkthroughs/python_table_api.md index e96d9558ce..5630fc7668 100644 --- a/docs/getting-started/walkthroughs/python_table_api.md +++ b/docs/getting-started/walkthroughs/python_table_api.md @@ -25,26 +25,14 @@ under the License. This walkthrough will quickly get you started building a pure Python Flink project. -Note Python 3.5 or higher is required to run PyFlink. Run the following command to confirm that the command “python” in current environment points to Python 3.5+: - -{% highlight bash %} -$ python --version -# the version printed here must be 3.5+ -{% endhighlight %} +Please refer to the Python Table API [installation guide]({{ site.baseurl }}/dev/table/python/installation.html) on how to set up the Python execution environments. * This will be replaced by the TOC {:toc} ## Setting up a Python Project -You can begin by creating a Python project and installing the PyFlink package. -PyFlink is available via PyPi and can be easily installed using `pip`. - -{% highlight bash %} -$ python -m pip install apache-flink -{% endhighlight %} - -You can also build PyFlink from source by following the [development guide]({{ site.baseurl }}/flinkDev/building.html#build-pyflink). +You can begin by creating a Python project and installing the PyFlink package following the [installation guide]({{ site.baseurl }}/dev/table/python/installation.html#installation-of-pyflink). ## Writing a Flink Python Table API Program diff --git a/docs/getting-started/walkthroughs/python_table_api.zh.md b/docs/getting-started/walkthroughs/python_table_api.zh.md index 0805cbe69b..794cbcf37d 100644 --- a/docs/getting-started/walkthroughs/python_table_api.zh.md +++ b/docs/getting-started/walkthroughs/python_table_api.zh.md @@ -28,19 +28,11 @@ under the License. 在该教程中,我们会从零开始,介绍如何创建一个Flink Python项目及运行Python Table API程序。 -注意 PyFlink的运行需要Python 3.5及以上版本。 - -执行以下命令以确认当前环境下的指令“python”指向Python 3.5及以上版本: - -{% highlight bash %} -$ python --version -# the version printed here must be 3.5+ -{% endhighlight %} +关于Python执行环境的要求,请参考Python Table API[环境安装]({{ site.baseurl }}/dev/dev/table/python/installation.html)。 ## 创建一个Python Table API项目 -首先,使用您最熟悉的IDE创建一个Python项目。之后执行命令`python -m pip install apache-flink`从PyPI下载安装PyFlink包。 -如果您想从源码安装,请参考[构建PyFlink]({{ site.baseurl }}/zh/flinkDev/building.html#build-pyflink)了解详细信息。 +首先,使用您最熟悉的IDE创建一个Python项目,然后安装PyFlink包,请参考[PyFlink安装指南]({{ site.baseurl }}/zh/dev/table/python/installation.html#installation-of-pyflink)了解详细信息。 ## 编写一个Flink Python Table API程序 diff --git a/docs/ops/python_shell.md b/docs/ops/python_shell.md index ac571c0ce0..ebcb7291df 100644 --- a/docs/ops/python_shell.md +++ b/docs/ops/python_shell.md @@ -27,12 +27,7 @@ It can be used in a local setup as well as in a cluster setup. See the [local setup page](deployment/local.html) for more information about how to setup a local Flink. You can also [build a local setup from source](../flinkDev/building.html). -Note The Python Shell will run the command “python”. Please run the following command to confirm that the command “python” in current environment points to Python 3.5+: - -{% highlight bash %} -$ python --version -# the version printed here must be 3.5+ -{% endhighlight %} +Note The Python Shell will run the command “python”. Note The Python Shell will run the command “python”. Please refer to the Python Table API [installation guide]({{ site.baseurl }}/dev/table/python/installation.html) on how to set up the Python execution environments. To use the shell with an integrated Flink cluster, you can simply install PyFlink with PyPi and execute the shell directly: diff --git a/docs/ops/python_shell.zh.md b/docs/ops/python_shell.zh.md index 88846d8685..e5f2a6c609 100644 --- a/docs/ops/python_shell.zh.md +++ b/docs/ops/python_shell.zh.md @@ -27,12 +27,7 @@ Flink附带了一个集成的交互式Python Shell。 本地安装Flink,请看[本地安装](deployment/local.html)页面。 您也可以从源码安装Flink,请看[从源码构建 Flink](../flinkDev/building.html)页面。 -注意 Python Shell会调用“python”命令。请执行以下命令以确认当前环境下的指令“python”指向Python 3.5及以上版本: - -{% highlight bash %} -$ python --version -# the version printed here must be 3.5+ -{% endhighlight %} +注意 Python Shell会调用“python”命令。关于Python执行环境的要求,请参考Python Table API[环境安装]({{ site.baseurl }}/dev/dev/table/python/installation.html)。 你可以通过PyPi安装PyFlink,然后使用Python Shell: -- Gitee From 22915d051d614249ea2d2eae4a2366b5533136d8 Mon Sep 17 00:00:00 2001 From: Leonard Xu Date: Fri, 13 Mar 2020 15:12:08 +0800 Subject: [PATCH 158/885] [FLINK-16464][sql-client] Result-mode tableau may shift when content contains Chinese string in SQL CLI (#11334) * [FLINK-16464][sql-client]result-mode tableau may shift when content contains Chinese String in SQL CLI * [FLINK-16464][sql-client]result-mode tableau may shift when content contains Chinese String in SQL CLI * address comment * add license and unit tests * revert pom header format --- flink-table/flink-sql-client/pom.xml | 8 + .../client/cli/CliTableauResultView.java | 34 +- .../flink/table/client/cli/CliUtils.java | 31 ++ .../src/main/resources/META-INF/NOTICE | 1 + .../resources/META-INF/licenses/LICENSE.icu4j | 414 ++++++++++++++++++ .../client/cli/CliTableauResultViewTest.java | 61 ++- .../flink/table/client/cli/CliUtilsTest.java | 25 ++ 7 files changed, 553 insertions(+), 21 deletions(-) create mode 100644 flink-table/flink-sql-client/src/main/resources/META-INF/licenses/LICENSE.icu4j diff --git a/flink-table/flink-sql-client/pom.xml b/flink-table/flink-sql-client/pom.xml index 6a6cf4d7dd..9fcae75fd8 100644 --- a/flink-table/flink-sql-client/pom.xml +++ b/flink-table/flink-sql-client/pom.xml @@ -109,6 +109,13 @@ under the License. 3.9.0 + + + com.ibm.icu + icu4j + 65.1 + + org.apache.flink @@ -521,6 +528,7 @@ under the License. org.jline:* + com.ibm.icu:icu4j diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliTableauResultView.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliTableauResultView.java index e82a026b1d..9ae606d512 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliTableauResultView.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliTableauResultView.java @@ -49,6 +49,8 @@ import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Stream; +import static org.apache.flink.table.client.cli.CliUtils.getStringDisplayWidth; +import static org.apache.flink.table.client.cli.CliUtils.isFullWidth; import static org.apache.flink.table.client.cli.CliUtils.rowToString; /** @@ -275,11 +277,12 @@ public class CliTableauResultView implements AutoCloseable { int idx = 0; for (String col : cols) { sb.append(" "); - if (col.length() <= colWidths[idx]) { - sb.append(StringUtils.repeat(' ', colWidths[idx] - col.length())); + int displayWidth = getStringDisplayWidth(col); + if (displayWidth <= colWidths[idx]) { + sb.append(StringUtils.repeat(' ', colWidths[idx] - displayWidth)); sb.append(col); } else { - sb.append(col, 0, colWidths[idx] - COLUMN_TRUNCATED_FLAG.length()); + sb.append(truncateString(col, colWidths[idx] - COLUMN_TRUNCATED_FLAG.length())); sb.append(COLUMN_TRUNCATED_FLAG); } sb.append(" |"); @@ -389,7 +392,7 @@ public class CliTableauResultView implements AutoCloseable { // fill column width with real data for (String[] row : rows) { for (int i = 0; i < row.length; ++i) { - colWidths[i] = Math.max(colWidths[i], row[i].length()); + colWidths[i] = Math.max(colWidths[i], getStringDisplayWidth(row[i])); } } @@ -401,4 +404,27 @@ public class CliTableauResultView implements AutoCloseable { return colWidths; } + private String truncateString(String col, int targetWidth) { + int passedWidth = 0; + int i = 0; + for (; i < col.length(); i++) { + if (isFullWidth(Character.codePointAt(col, i))) { + passedWidth += 2; + } else { + passedWidth += 1; + } + if (passedWidth >= targetWidth) { + break; + } + } + String substring = col.substring(0, i); + + // pad with ' ' before the column + int lackedWidth = targetWidth - getStringDisplayWidth(substring); + if (lackedWidth > 0){ + substring = StringUtils.repeat(' ', lackedWidth) + substring; + } + return substring; + } + } diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliUtils.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliUtils.java index cf87f3edc4..6b2161fcf4 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliUtils.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliUtils.java @@ -23,6 +23,8 @@ import org.apache.flink.table.types.DataType; import org.apache.flink.table.utils.EncodingUtils; import org.apache.flink.types.Row; +import com.ibm.icu.lang.UCharacter; +import com.ibm.icu.lang.UProperty; import org.jline.utils.AttributedString; import org.jline.utils.AttributedStringBuilder; import org.jline.utils.AttributedStyle; @@ -111,4 +113,33 @@ public final class CliUtils { } return typesAsString; } + + public static int getStringDisplayWidth(String str) { + int numOfFullWidthCh = (int) str.codePoints().filter(codePoint -> isFullWidth(codePoint)).count(); + return str.length() + numOfFullWidthCh; + } + + /** + * Check codePoint is FullWidth or not according to Unicode Standard version 12.0.0. + * See http://unicode.org/reports/tr11/ + */ + public static boolean isFullWidth(int codePoint) { + int value = UCharacter.getIntPropertyValue(codePoint, UProperty.EAST_ASIAN_WIDTH); + switch (value) { + case UCharacter.EastAsianWidth.NEUTRAL: + return false; + case UCharacter.EastAsianWidth.AMBIGUOUS: + return false; + case UCharacter.EastAsianWidth.HALFWIDTH: + return false; + case UCharacter.EastAsianWidth.FULLWIDTH: + return true; + case UCharacter.EastAsianWidth.NARROW: + return false; + case UCharacter.EastAsianWidth.WIDE: + return true; + default: + throw new RuntimeException("unknown UProperty.EAST_ASIAN_WIDTH: " + value); + } + } } diff --git a/flink-table/flink-sql-client/src/main/resources/META-INF/NOTICE b/flink-table/flink-sql-client/src/main/resources/META-INF/NOTICE index 02be9302cc..17c8d7579e 100644 --- a/flink-table/flink-sql-client/src/main/resources/META-INF/NOTICE +++ b/flink-table/flink-sql-client/src/main/resources/META-INF/NOTICE @@ -9,3 +9,4 @@ See bundled license files for details. - org.jline:jline-terminal:3.9.0 - org.jline:jline-reader:3.9.0 +- com.ibm.icu:icu4j:65.1 diff --git a/flink-table/flink-sql-client/src/main/resources/META-INF/licenses/LICENSE.icu4j b/flink-table/flink-sql-client/src/main/resources/META-INF/licenses/LICENSE.icu4j new file mode 100644 index 0000000000..2e01e36876 --- /dev/null +++ b/flink-table/flink-sql-client/src/main/resources/META-INF/licenses/LICENSE.icu4j @@ -0,0 +1,414 @@ +COPYRIGHT AND PERMISSION NOTICE (ICU 58 and later) + +Copyright © 1991-2019 Unicode, Inc. All rights reserved. +Distributed under the Terms of Use in https://www.unicode.org/copyright.html. + +Permission is hereby granted, free of charge, to any person obtaining +a copy of the Unicode data files and any associated documentation +(the "Data Files") or Unicode software and any associated documentation +(the "Software") to deal in the Data Files or Software +without restriction, including without limitation the rights to use, +copy, modify, merge, publish, distribute, and/or sell copies of +the Data Files or Software, and to permit persons to whom the Data Files +or Software are furnished to do so, provided that either +(a) this copyright and permission notice appear with all copies +of the Data Files or Software, or +(b) this copyright and permission notice appear in associated +Documentation. + +THE DATA FILES AND SOFTWARE ARE PROVIDED "AS IS", WITHOUT WARRANTY OF +ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE +WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND +NONINFRINGEMENT OF THIRD PARTY RIGHTS. +IN NO EVENT SHALL THE COPYRIGHT HOLDER OR HOLDERS INCLUDED IN THIS +NOTICE BE LIABLE FOR ANY CLAIM, OR ANY SPECIAL INDIRECT OR CONSEQUENTIAL +DAMAGES, OR ANY DAMAGES WHATSOEVER RESULTING FROM LOSS OF USE, +DATA OR PROFITS, WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE OR OTHER +TORTIOUS ACTION, ARISING OUT OF OR IN CONNECTION WITH THE USE OR +PERFORMANCE OF THE DATA FILES OR SOFTWARE. + +Except as contained in this notice, the name of a copyright holder +shall not be used in advertising or otherwise to promote the sale, +use or other dealings in these Data Files or Software without prior +written authorization of the copyright holder. + +--------------------- + +Third-Party Software Licenses + +This section contains third-party software notices and/or additional +terms for licensed third-party software components included within ICU +libraries. + +1. ICU License - ICU 1.8.1 to ICU 57.1 + +COPYRIGHT AND PERMISSION NOTICE + +Copyright (c) 1995-2016 International Business Machines Corporation and others +All rights reserved. + +Permission is hereby granted, free of charge, to any person obtaining +a copy of this software and associated documentation files (the +"Software"), to deal in the Software without restriction, including +without limitation the rights to use, copy, modify, merge, publish, +distribute, and/or sell copies of the Software, and to permit persons +to whom the Software is furnished to do so, provided that the above +copyright notice(s) and this permission notice appear in all copies of +the Software and that both the above copyright notice(s) and this +permission notice appear in supporting documentation. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT +OF THIRD PARTY RIGHTS. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR +HOLDERS INCLUDED IN THIS NOTICE BE LIABLE FOR ANY CLAIM, OR ANY +SPECIAL INDIRECT OR CONSEQUENTIAL DAMAGES, OR ANY DAMAGES WHATSOEVER +RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN ACTION OF +CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF OR IN +CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. + +Except as contained in this notice, the name of a copyright holder +shall not be used in advertising or otherwise to promote the sale, use +or other dealings in this Software without prior written authorization +of the copyright holder. + +All trademarks and registered trademarks mentioned herein are the +property of their respective owners. + +2. Chinese/Japanese Word Break Dictionary Data (cjdict.txt) + + # The Google Chrome software developed by Google is licensed under + # the BSD license. Other software included in this distribution is + # provided under other licenses, as set forth below. + # + # The BSD License + # http://opensource.org/licenses/bsd-license.php + # Copyright (C) 2006-2008, Google Inc. + # + # All rights reserved. + # + # Redistribution and use in source and binary forms, with or without + # modification, are permitted provided that the following conditions are met: + # + # Redistributions of source code must retain the above copyright notice, + # this list of conditions and the following disclaimer. + # Redistributions in binary form must reproduce the above + # copyright notice, this list of conditions and the following + # disclaimer in the documentation and/or other materials provided with + # the distribution. + # Neither the name of Google Inc. nor the names of its + # contributors may be used to endorse or promote products derived from + # this software without specific prior written permission. + # + # + # THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND + # CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, + # INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF + # MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + # DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE + # LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR + # CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF + # SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR + # BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF + # LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING + # NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS + # SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + # + # + # The word list in cjdict.txt are generated by combining three word lists + # listed below with further processing for compound word breaking. The + # frequency is generated with an iterative training against Google web + # corpora. + # + # * Libtabe (Chinese) + # - https://sourceforge.net/project/?group_id=1519 + # - Its license terms and conditions are shown below. + # + # * IPADIC (Japanese) + # - http://chasen.aist-nara.ac.jp/chasen/distribution.html + # - Its license terms and conditions are shown below. + # + # ---------COPYING.libtabe ---- BEGIN-------------------- + # + # /* + # * Copyright (c) 1999 TaBE Project. + # * Copyright (c) 1999 Pai-Hsiang Hsiao. + # * All rights reserved. + # * + # * Redistribution and use in source and binary forms, with or without + # * modification, are permitted provided that the following conditions + # * are met: + # * + # * . Redistributions of source code must retain the above copyright + # * notice, this list of conditions and the following disclaimer. + # * . Redistributions in binary form must reproduce the above copyright + # * notice, this list of conditions and the following disclaimer in + # * the documentation and/or other materials provided with the + # * distribution. + # * . Neither the name of the TaBE Project nor the names of its + # * contributors may be used to endorse or promote products derived + # * from this software without specific prior written permission. + # * + # * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + # * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + # * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS + # * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE + # * REGENTS OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, + # * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES + # * (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + # * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) + # * HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, + # * STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + # * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED + # * OF THE POSSIBILITY OF SUCH DAMAGE. + # */ + # + # /* + # * Copyright (c) 1999 Computer Systems and Communication Lab, + # * Institute of Information Science, Academia + # * Sinica. All rights reserved. + # * + # * Redistribution and use in source and binary forms, with or without + # * modification, are permitted provided that the following conditions + # * are met: + # * + # * . Redistributions of source code must retain the above copyright + # * notice, this list of conditions and the following disclaimer. + # * . Redistributions in binary form must reproduce the above copyright + # * notice, this list of conditions and the following disclaimer in + # * the documentation and/or other materials provided with the + # * distribution. + # * . Neither the name of the Computer Systems and Communication Lab + # * nor the names of its contributors may be used to endorse or + # * promote products derived from this software without specific + # * prior written permission. + # * + # * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + # * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + # * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS + # * FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE + # * REGENTS OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, + # * INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES + # * (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + # * SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) + # * HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, + # * STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + # * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED + # * OF THE POSSIBILITY OF SUCH DAMAGE. + # */ + # + # Copyright 1996 Chih-Hao Tsai @ Beckman Institute, + # University of Illinois + # c-tsai4@uiuc.edu http://casper.beckman.uiuc.edu/~c-tsai4 + # + # ---------------COPYING.libtabe-----END-------------------------------- + # + # + # ---------------COPYING.ipadic-----BEGIN------------------------------- + # + # Copyright 2000, 2001, 2002, 2003 Nara Institute of Science + # and Technology. All Rights Reserved. + # + # Use, reproduction, and distribution of this software is permitted. + # Any copy of this software, whether in its original form or modified, + # must include both the above copyright notice and the following + # paragraphs. + # + # Nara Institute of Science and Technology (NAIST), + # the copyright holders, disclaims all warranties with regard to this + # software, including all implied warranties of merchantability and + # fitness, in no event shall NAIST be liable for + # any special, indirect or consequential damages or any damages + # whatsoever resulting from loss of use, data or profits, whether in an + # action of contract, negligence or other tortuous action, arising out + # of or in connection with the use or performance of this software. + # + # A large portion of the dictionary entries + # originate from ICOT Free Software. The following conditions for ICOT + # Free Software applies to the current dictionary as well. + # + # Each User may also freely distribute the Program, whether in its + # original form or modified, to any third party or parties, PROVIDED + # that the provisions of Section 3 ("NO WARRANTY") will ALWAYS appear + # on, or be attached to, the Program, which is distributed substantially + # in the same form as set out herein and that such intended + # distribution, if actually made, will neither violate or otherwise + # contravene any of the laws and regulations of the countries having + # jurisdiction over the User or the intended distribution itself. + # + # NO WARRANTY + # + # The program was produced on an experimental basis in the course of the + # research and development conducted during the project and is provided + # to users as so produced on an experimental basis. Accordingly, the + # program is provided without any warranty whatsoever, whether express, + # implied, statutory or otherwise. The term "warranty" used herein + # includes, but is not limited to, any warranty of the quality, + # performance, merchantability and fitness for a particular purpose of + # the program and the nonexistence of any infringement or violation of + # any right of any third party. + # + # Each user of the program will agree and understand, and be deemed to + # have agreed and understood, that there is no warranty whatsoever for + # the program and, accordingly, the entire risk arising from or + # otherwise connected with the program is assumed by the user. + # + # Therefore, neither ICOT, the copyright holder, or any other + # organization that participated in or was otherwise related to the + # development of the program and their respective officials, directors, + # officers and other employees shall be held liable for any and all + # damages, including, without limitation, general, special, incidental + # and consequential damages, arising out of or otherwise in connection + # with the use or inability to use the program or any product, material + # or result produced or otherwise obtained by using the program, + # regardless of whether they have been advised of, or otherwise had + # knowledge of, the possibility of such damages at any time during the + # project or thereafter. Each user will be deemed to have agreed to the + # foregoing by his or her commencement of use of the program. The term + # "use" as used herein includes, but is not limited to, the use, + # modification, copying and distribution of the program and the + # production of secondary products from the program. + # + # In the case where the program, whether in its original form or + # modified, was distributed or delivered to or received by a user from + # any person, organization or entity other than ICOT, unless it makes or + # grants independently of ICOT any specific warranty to the user in + # writing, such person, organization or entity, will also be exempted + # from and not be held liable to the user for any such damages as noted + # above as far as the program is concerned. + # + # ---------------COPYING.ipadic-----END---------------------------------- + +3. Lao Word Break Dictionary Data (laodict.txt) + + # Copyright (c) 2013 International Business Machines Corporation + # and others. All Rights Reserved. + # + # Project: http://code.google.com/p/lao-dictionary/ + # Dictionary: http://lao-dictionary.googlecode.com/git/Lao-Dictionary.txt + # License: http://lao-dictionary.googlecode.com/git/Lao-Dictionary-LICENSE.txt + # (copied below) + # + # This file is derived from the above dictionary, with slight + # modifications. + # ---------------------------------------------------------------------- + # Copyright (C) 2013 Brian Eugene Wilson, Robert Martin Campbell. + # All rights reserved. + # + # Redistribution and use in source and binary forms, with or without + # modification, + # are permitted provided that the following conditions are met: + # + # + # Redistributions of source code must retain the above copyright notice, this + # list of conditions and the following disclaimer. Redistributions in + # binary form must reproduce the above copyright notice, this list of + # conditions and the following disclaimer in the documentation and/or + # other materials provided with the distribution. + # + # + # THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + # "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + # LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS + # FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE + # COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, + # INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES + # (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + # SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) + # HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, + # STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + # ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED + # OF THE POSSIBILITY OF SUCH DAMAGE. + # -------------------------------------------------------------------------- + +4. Burmese Word Break Dictionary Data (burmesedict.txt) + + # Copyright (c) 2014 International Business Machines Corporation + # and others. All Rights Reserved. + # + # This list is part of a project hosted at: + # github.com/kanyawtech/myanmar-karen-word-lists + # + # -------------------------------------------------------------------------- + # Copyright (c) 2013, LeRoy Benjamin Sharon + # All rights reserved. + # + # Redistribution and use in source and binary forms, with or without + # modification, are permitted provided that the following conditions + # are met: Redistributions of source code must retain the above + # copyright notice, this list of conditions and the following + # disclaimer. Redistributions in binary form must reproduce the + # above copyright notice, this list of conditions and the following + # disclaimer in the documentation and/or other materials provided + # with the distribution. + # + # Neither the name Myanmar Karen Word Lists, nor the names of its + # contributors may be used to endorse or promote products derived + # from this software without specific prior written permission. + # + # THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND + # CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, + # INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF + # MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + # DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS + # BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, + # EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED + # TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + # DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON + # ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR + # TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF + # THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF + # SUCH DAMAGE. + # -------------------------------------------------------------------------- + +5. Time Zone Database + + ICU uses the public domain data and code derived from Time Zone +Database for its time zone support. The ownership of the TZ database +is explained in BCP 175: Procedure for Maintaining the Time Zone +Database section 7. + + # 7. Database Ownership + # + # The TZ database itself is not an IETF Contribution or an IETF + # document. Rather it is a pre-existing and regularly updated work + # that is in the public domain, and is intended to remain in the + # public domain. Therefore, BCPs 78 [RFC5378] and 79 [RFC3979] do + # not apply to the TZ Database or contributions that individuals make + # to it. Should any claims be made and substantiated against the TZ + # Database, the organization that is providing the IANA + # Considerations defined in this RFC, under the memorandum of + # understanding with the IETF, currently ICANN, may act in accordance + # with all competent court orders. No ownership claims will be made + # by ICANN or the IETF Trust on the database or the code. Any person + # making a contribution to the database or code waives all rights to + # future claims in that contribution or in the TZ Database. + +6. Google double-conversion + +Copyright 2006-2011, the V8 project authors. All rights reserved. +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following + disclaimer in the documentation and/or other materials provided + with the distribution. + * Neither the name of Google Inc. nor the names of its + contributors may be used to endorse or promote products derived + from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliTableauResultViewTest.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliTableauResultViewTest.java index 3ab8ae3bf9..073cd59700 100644 --- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliTableauResultViewTest.java +++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliTableauResultViewTest.java @@ -135,6 +135,26 @@ public class CliTableauResultViewTest { null) ); + data.add( + Row.of( + null, + -1, + -1, + "这是一段中文", + BigDecimal.valueOf(-12345.06789), + Timestamp.valueOf("2020-03-04 18:39:14")) + ); + + data.add( + Row.of( + null, + -1, + -1, + "これは日本語をテストするための文です", + BigDecimal.valueOf(-12345.06789), + Timestamp.valueOf("2020-03-04 18:39:14")) + ); + streamingData = new ArrayList<>(); for (int i = 0; i < data.size(); ++i) { streamingData.add(new Tuple2<>(i % 2 == 0, data.get(i))); @@ -156,19 +176,20 @@ public class CliTableauResultViewTest { view.displayBatchResults(); view.close(); - Assert.assertEquals( - "+---------+-------------+----------------------+----------------------------+----------------+----------------------------+\n" + - "| boolean | int | bigint | varchar | decimal(10, 5) | timestamp |\n" + - "+---------+-------------+----------------------+----------------------------+----------------+----------------------------+\n" + - "| (NULL) | 1 | 2 | abc | 1.23 | 2020-03-01 18:39:14.0 |\n" + - "| false | (NULL) | 0 | | 1 | 2020-03-01 18:39:14.1 |\n" + - "| true | 2147483647 | (NULL) | abcdefg | 1234567890 | 2020-03-01 18:39:14.12 |\n" + - "| false | -2147483648 | 9223372036854775807 | (NULL) | 12345.06789 | 2020-03-01 18:39:14.123 |\n" + - "| true | 100 | -9223372036854775808 | abcdefg111 | (NULL) | 2020-03-01 18:39:14.123456 |\n" + - "| (NULL) | -1 | -1 | abcdefghijklmnopqrstuvwxyz | -12345.06789 | (NULL) |\n" + - "+---------+-------------+----------------------+----------------------------+----------------+----------------------------+\n" + - "6 row in set\n", + "+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+\n" + + "| boolean | int | bigint | varchar | decimal(10, 5) | timestamp |\n" + + "+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+\n" + + "| (NULL) | 1 | 2 | abc | 1.23 | 2020-03-01 18:39:14.0 |\n" + + "| false | (NULL) | 0 | | 1 | 2020-03-01 18:39:14.1 |\n" + + "| true | 2147483647 | (NULL) | abcdefg | 1234567890 | 2020-03-01 18:39:14.12 |\n" + + "| false | -2147483648 | 9223372036854775807 | (NULL) | 12345.06789 | 2020-03-01 18:39:14.123 |\n" + + "| true | 100 | -9223372036854775808 | abcdefg111 | (NULL) | 2020-03-01 18:39:14.123456 |\n" + + "| (NULL) | -1 | -1 | abcdefghijklmnopqrstuvwxyz | -12345.06789 | (NULL) |\n" + + "| (NULL) | -1 | -1 | 这是一段中文 | -12345.06789 | 2020-03-04 18:39:14.0 |\n" + + "| (NULL) | -1 | -1 | これは日本語をテストするた... | -12345.06789 | 2020-03-04 18:39:14.0 |\n" + + "+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+\n" + + "8 row in set\n", terminalOutput.toString()); verify(mockExecutor, times(0)).cancelQuery(anyString(), anyString()); } @@ -266,7 +287,9 @@ public class CliTableauResultViewTest { view.displayStreamResults(); view.close(); - + // note: the expected result may look irregular because every CJK(Chinese/Japanese/Korean) character's + // width < 2 in IDE by default, every CJK character usually's width is 2, you can open this source file + // by vim or just cat the file to check the regular result. Assert.assertEquals( "+-----+---------+-------------+----------------------+----------------------+----------------+----------------------------+\n" + "| +/- | boolean | int | bigint | varchar | decimal(10, 5) | timestamp |\n" + @@ -276,9 +299,11 @@ public class CliTableauResultViewTest { "| + | true | 2147483647 | (NULL) | abcdefg | 1234567890 | 2020-03-01 18:39:14.12 |\n" + "| - | false | -2147483648 | 9223372036854775807 | (NULL) | 12345.06789 | 2020-03-01 18:39:14.123 |\n" + "| + | true | 100 | -9223372036854775808 | abcdefg111 | (NULL) | 2020-03-01 18:39:14.123456 |\n" + - "| - | (NULL) | -1 | -1 | abcdefghijklmnopq... | -12345.06789 | (NULL) |\n" + + "| - | (NULL) | -1 | -1 | abcdefghijklmnop... | -12345.06789 | (NULL) |\n" + + "| + | (NULL) | -1 | -1 | 这是一段中文 | -12345.06789 | 2020-03-04 18:39:14.0 |\n" + + "| - | (NULL) | -1 | -1 | これは日本語をテ... | -12345.06789 | 2020-03-04 18:39:14.0 |\n" + "+-----+---------+-------------+----------------------+----------------------+----------------+----------------------------+\n" + - "Received a total of 6 rows\n", + "Received a total of 8 rows\n", terminalOutput.toString()); verify(mockExecutor, times(0)).cancelQuery(anyString(), anyString()); } @@ -335,7 +360,8 @@ public class CliTableauResultViewTest { "| + | (NULL) | 1 | 2 | abc | 1.23 | 2020-03-01 18:39:14.0 |\n" + "| - | false | (NULL) | 0 | | 1 | 2020-03-01 18:39:14.1 |\n" + "| + | true | 2147483647 | (NULL) | abcdefg | 1234567890 | 2020-03-01 18:39:14.12 |\n" + - "Query terminated, received a total of 3 rows\n", + "| - | false | -2147483648 | 9223372036854775807 | (NULL) | 12345.06789 | 2020-03-01 18:39:14.123 |\n" + + "Query terminated, received a total of 4 rows\n", terminalOutput.toString()); verify(mockExecutor, times(1)).cancelQuery(anyString(), anyString()); @@ -366,7 +392,8 @@ public class CliTableauResultViewTest { "+-----+---------+-------------+----------------------+----------------------+----------------+----------------------------+\n" + "| + | (NULL) | 1 | 2 | abc | 1.23 | 2020-03-01 18:39:14.0 |\n" + "| - | false | (NULL) | 0 | | 1 | 2020-03-01 18:39:14.1 |\n" + - "| + | true | 2147483647 | (NULL) | abcdefg | 1234567890 | 2020-03-01 18:39:14.12 |\n", + "| + | true | 2147483647 | (NULL) | abcdefg | 1234567890 | 2020-03-01 18:39:14.12 |\n" + + "| - | false | -2147483648 | 9223372036854775807 | (NULL) | 12345.06789 | 2020-03-01 18:39:14.123 |\n", terminalOutput.toString()); verify(mockExecutor, times(1)).cancelQuery(anyString(), anyString()); } diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliUtilsTest.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliUtilsTest.java index 8e2d32e52b..8cbac23047 100644 --- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliUtilsTest.java +++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliUtilsTest.java @@ -23,6 +23,7 @@ import org.apache.flink.types.Row; import org.junit.Test; import java.util.Arrays; +import java.util.List; import static org.junit.Assert.assertEquals; @@ -40,4 +41,28 @@ public class CliUtilsTest { row.setField(3, new Integer[][]{new Integer[]{9, 10}, new Integer[]{11, 12}}); assertEquals("[[1, 2], [3, 4], [[5, 6], [7, 8]], [[9, 10], [11, 12]]]", Arrays.toString(CliUtils.rowToString(row))); } + + @Test + public void testCharFullWidth() { + char[] chars = new char[] {'A', 'a', ',', '中', ',', 'こ'}; + boolean[] expected = new boolean[] {false, false, false, true, true, true}; + + for (int i = 0; i < chars.length; i++) { + assertEquals(expected[i], CliUtils.isFullWidth(Character.codePointAt(chars, i))); + } + } + + @Test + public void testStringDisplayWidth() { + List data = Arrays.asList( + "abcdefg,12345,ABC", + "to be or not to be that's a question.", + "这是一段中文", + "これは日本語をテストするための文です"); + int[] expected = new int[] {17, 37, 12, 36}; + + for (int i = 0; i < data.size(); i++) { + assertEquals(expected[i], CliUtils.getStringDisplayWidth(data.get(i))); + } + } } -- Gitee From d6038cc62f2d04cf078f0ac38cb8bc6fd7d9bb8e Mon Sep 17 00:00:00 2001 From: HuangXingBo Date: Fri, 13 Mar 2020 17:22:42 +0800 Subject: [PATCH 159/885] [FLINK-16524][python] Optimize the Python UDF execution to use generator to eliminate unnecessary function calls (#11374) --- .../pyflink/fn_execution/coder_impl.py | 119 +++++++++++++----- flink-python/pyflink/fn_execution/coders.py | 21 +++- .../pyflink/fn_execution/operations.py | 30 ++--- ...s_test_common.py => test_coders_common.py} | 17 ++- 4 files changed, 132 insertions(+), 55 deletions(-) rename flink-python/pyflink/fn_execution/tests/{coders_test_common.py => test_coders_common.py} (88%) diff --git a/flink-python/pyflink/fn_execution/coder_impl.py b/flink-python/pyflink/fn_execution/coder_impl.py index ea2db30809..7b6405aafc 100644 --- a/flink-python/pyflink/fn_execution/coder_impl.py +++ b/flink-python/pyflink/fn_execution/coder_impl.py @@ -19,9 +19,12 @@ import datetime import decimal import struct -import pyarrow as pa +from typing import Any +from typing import Generator +from typing import List -from apache_beam.coders.coder_impl import StreamCoderImpl +import pyarrow as pa +from apache_beam.coders.coder_impl import StreamCoderImpl, create_InputStream, create_OutputStream from pyflink.fn_execution.ResettableIO import ResettableIO from pyflink.table import Row @@ -31,11 +34,12 @@ class FlattenRowCoderImpl(StreamCoderImpl): def __init__(self, field_coders): self._field_coders = field_coders - self._filed_count = len(field_coders) - self._leading_complete_bytes_num = self._filed_count // 8 - self._remaining_bits_num = self._filed_count % 8 + self._field_count = len(field_coders) + self._leading_complete_bytes_num = self._field_count // 8 + self._remaining_bits_num = self._field_count % 8 self.null_mask_search_table = self.generate_null_mask_search_table() self.null_byte_search_table = (0x80, 0x40, 0x20, 0x10, 0x08, 0x04, 0x02, 0x01) + self.data_out_stream = create_OutputStream() @staticmethod def generate_null_mask_search_table(): @@ -51,21 +55,30 @@ class FlattenRowCoderImpl(StreamCoderImpl): return tuple(null_mask) - def encode_to_stream(self, value, out_stream, nested): - self.write_null_mask(value, out_stream) + def encode_to_stream(self, iter_value, out_stream, nested): field_coders = self._field_coders - for i in range(self._filed_count): - item = value[i] - if item is not None: - field_coders[i].encode_to_stream(item, out_stream, nested) + data_out_stream = self.data_out_stream + for value in iter_value: + self._write_null_mask(value, data_out_stream) + for i in range(self._field_count): + item = value[i] + if item is not None: + field_coders[i].encode_to_stream(item, data_out_stream, nested) + out_stream.write_var_int64(data_out_stream.size()) + out_stream.write(data_out_stream.get()) + data_out_stream._clear() def decode_from_stream(self, in_stream, nested): - null_mask = self.read_null_mask(in_stream) - field_coders = self._field_coders - return [None if null_mask[idx] else field_coders[idx].decode_from_stream( - in_stream, nested) for idx in range(0, self._filed_count)] + while in_stream.size() > 0: + in_stream.read_var_int64() + yield self._decode_one_row_from_stream(in_stream, nested) + + def _decode_one_row_from_stream(self, in_stream: create_InputStream, nested: bool) -> List: + null_mask = self._read_null_mask(in_stream) + return [None if null_mask[idx] else self._field_coders[idx].decode_from_stream( + in_stream, nested) for idx in range(0, self._field_count)] - def write_null_mask(self, value, out_stream): + def _write_null_mask(self, value, out_stream): field_pos = 0 null_byte_search_table = self.null_byte_search_table remaining_bits_num = self._remaining_bits_num @@ -84,7 +97,7 @@ class FlattenRowCoderImpl(StreamCoderImpl): b |= null_byte_search_table[i] out_stream.write_byte(b) - def read_null_mask(self, in_stream): + def _read_null_mask(self, in_stream): null_mask = [] null_mask_search_table = self.null_mask_search_table remaining_bits_num = self._remaining_bits_num @@ -106,8 +119,16 @@ class RowCoderImpl(FlattenRowCoderImpl): def __init__(self, field_coders): super(RowCoderImpl, self).__init__(field_coders) + def encode_to_stream(self, value, out_stream, nested): + field_coders = self._field_coders + self._write_null_mask(value, out_stream) + for i in range(self._field_count): + item = value[i] + if item is not None: + field_coders[i].encode_to_stream(item, out_stream, nested) + def decode_from_stream(self, in_stream, nested): - return Row(*super(RowCoderImpl, self).decode_from_stream(in_stream, nested)) + return Row(*self._decode_one_row_from_stream(in_stream, nested)) def __repr__(self): return 'RowCoderImpl[%s]' % ', '.join(str(c) for c in self._field_coders) @@ -117,18 +138,25 @@ class TableFunctionRowCoderImpl(StreamCoderImpl): def __init__(self, flatten_row_coder): self._flatten_row_coder = flatten_row_coder - self._field_count = flatten_row_coder._filed_count - - def encode_to_stream(self, value, out_stream, nested): - if value is None: + self._field_count = flatten_row_coder._field_count + + def encode_to_stream(self, iter_value, out_stream, nested): + for value in iter_value: + if value: + if self._field_count == 1: + value = self._create_tuple_result(value) + self._flatten_row_coder.encode_to_stream(value, out_stream, nested) + out_stream.write_var_int64(1) out_stream.write_byte(0x00) - else: - self._flatten_row_coder.encode_to_stream(value if self._field_count != 1 else (value,), - out_stream, nested) def decode_from_stream(self, in_stream, nested): return self._flatten_row_coder.decode_from_stream(in_stream, nested) + @staticmethod + def _create_tuple_result(value: List) -> Generator: + for result in value: + yield (result,) + def __repr__(self): return 'TableFunctionRowCoderImpl[%s]' % repr(self._flatten_row_coder) @@ -385,16 +413,20 @@ class ArrowCoderImpl(StreamCoderImpl): self._resettable_io = ResettableIO() self._batch_reader = ArrowCoderImpl._load_from_stream(self._resettable_io) self._batch_writer = pa.RecordBatchStreamWriter(self._resettable_io, self._schema) + self.data_out_stream = create_OutputStream() + self._resettable_io.set_output_stream(self.data_out_stream) - def encode_to_stream(self, cols, out_stream, nested): - self._resettable_io.set_output_stream(out_stream) - self._batch_writer.write_batch(self._create_batch(cols)) + def encode_to_stream(self, iter_cols, out_stream, nested): + data_out_stream = self.data_out_stream + for cols in iter_cols: + self._batch_writer.write_batch(self._create_batch(cols)) + out_stream.write_var_int64(data_out_stream.size()) + out_stream.write(data_out_stream.get()) + data_out_stream._clear() def decode_from_stream(self, in_stream, nested): - self._resettable_io.set_input_bytes(in_stream.read_all()) - # there is only arrow batch in the underlying input stream - table = pa.Table.from_batches([next(self._batch_reader)]) - return [c.to_pandas(date_as_object=True) for c in table.itercolumns()] + while in_stream.size() > 0: + yield self._decode_one_batch_from_stream(in_stream) @staticmethod def _load_from_stream(stream): @@ -414,5 +446,28 @@ class ArrowCoderImpl(StreamCoderImpl): arrays = [create_array(cols[i], self._schema.types[i]) for i in range(0, len(self._schema))] return pa.RecordBatch.from_arrays(arrays, self._schema) + def _decode_one_batch_from_stream(self, in_stream: create_InputStream) -> List: + self._resettable_io.set_input_bytes(in_stream.read_all(True)) + # there is only one arrow batch in the underlying input stream + table = pa.Table.from_batches([next(self._batch_reader)]) + return [c.to_pandas(date_as_object=True) for c in table.itercolumns()] + def __repr__(self): return 'ArrowCoderImpl[%s]' % self._schema + + +class PassThroughLengthPrefixCoderImpl(StreamCoderImpl): + def __init__(self, value_coder): + self._value_coder = value_coder + + def encode_to_stream(self, value, out: create_OutputStream, nested: bool) -> Any: + self._value_coder.encode_to_stream(value, out, nested) + + def decode_from_stream(self, in_stream: create_InputStream, nested: bool) -> Any: + return self._value_coder.decode_from_stream(in_stream, nested) + + def get_estimated_size_and_observables(self, value: Any, nested=False): + return 0, [] + + def __repr__(self): + return 'PassThroughLengthPrefixCoderImpl[%s]' % self._value_coder diff --git a/flink-python/pyflink/fn_execution/coders.py b/flink-python/pyflink/fn_execution/coders.py index ae6e3df93f..e6a8ac5908 100644 --- a/flink-python/pyflink/fn_execution/coders.py +++ b/flink-python/pyflink/fn_execution/coders.py @@ -23,7 +23,8 @@ import datetime import decimal import pyarrow as pa from apache_beam.coders import Coder -from apache_beam.coders.coders import FastCoder +from apache_beam.coders.coders import FastCoder, LengthPrefixCoder +from apache_beam.portability import common_urns from apache_beam.typehints import typehints from pyflink.fn_execution import coder_impl @@ -414,6 +415,24 @@ class ArrowCoder(DeterministicCoder): return 'ArrowCoder[%s]' % self._schema +class PassThroughLengthPrefixCoder(LengthPrefixCoder): + """ + Coder which doesn't prefix the length of the encoded object as the length prefix will be handled + by the wrapped value coder. + """ + def __init__(self, value_coder): + super(PassThroughLengthPrefixCoder, self).__init__(value_coder) + + def _create_impl(self): + return coder_impl.PassThroughLengthPrefixCoderImpl(self._value_coder.get_impl()) + + def __repr__(self): + return 'PassThroughLengthPrefixCoder[%s]' % self._value_coder + + +Coder.register_structured_urn( + common_urns.coders.LENGTH_PREFIX.urn, PassThroughLengthPrefixCoder) + type_name = flink_fn_execution_pb2.Schema.TypeName _type_name_mappings = { type_name.TINYINT: TinyIntCoder(), diff --git a/flink-python/pyflink/fn_execution/operations.py b/flink-python/pyflink/fn_execution/operations.py index 8f6eeea46b..ec27e72273 100644 --- a/flink-python/pyflink/fn_execution/operations.py +++ b/flink-python/pyflink/fn_execution/operations.py @@ -22,6 +22,7 @@ import cloudpickle from apache_beam.runners.worker import operation_specs from apache_beam.runners.worker import bundle_processor from apache_beam.runners.worker.operations import Operation +from apache_beam.utils.windowed_value import WindowedValue from pyflink.fn_execution import flink_fn_execution_pb2 from pyflink.serializers import PickleSerializer @@ -76,6 +77,11 @@ class StatelessFunctionOperation(Operation): str(tag)] = receiver.opcounter.element_counter.value() return metrics + def process(self, o: WindowedValue): + output_stream = self.consumer.output_stream + self._value_coder_impl.encode_to_stream(self.func(o.value), output_stream, True) + output_stream.maybe_flush() + def generate_func(self, udfs): pass @@ -166,12 +172,8 @@ class ScalarFunctionOperation(StatelessFunctionOperation): :return: the generated lambda function """ scalar_functions = [self._extract_user_defined_function(udf) for udf in udfs] - return eval('lambda value: [%s]' % ','.join(scalar_functions), self.variable_dict) - - def process(self, o): - output_stream = self.consumer.output_stream - self._value_coder_impl.encode_to_stream(self.func(o.value), output_stream, True) - output_stream.maybe_flush() + mapper = eval('lambda value: [%s]' % ','.join(scalar_functions), self.variable_dict) + return lambda it: map(mapper, it) class TableFunctionOperation(StatelessFunctionOperation): @@ -186,20 +188,8 @@ class TableFunctionOperation(StatelessFunctionOperation): :return: the generated lambda function """ table_function = self._extract_user_defined_function(udtfs[0]) - return eval('lambda value: %s' % table_function, self.variable_dict) - - def process(self, o): - output_stream = self.consumer.output_stream - for result in self._create_result(o.value): - self._value_coder_impl.encode_to_stream(result, output_stream, True) - output_stream.maybe_flush() - - def _create_result(self, value): - result = self.func(value) - if result is not None: - yield from result - yield None - + mapper = eval('lambda value: %s' % table_function, self.variable_dict) + return lambda it: map(mapper, it) @bundle_processor.BeamTransformFactory.register_urn( SCALAR_FUNCTION_URN, flink_fn_execution_pb2.UserDefinedFunctions) diff --git a/flink-python/pyflink/fn_execution/tests/coders_test_common.py b/flink-python/pyflink/fn_execution/tests/test_coders_common.py similarity index 88% rename from flink-python/pyflink/fn_execution/tests/coders_test_common.py rename to flink-python/pyflink/fn_execution/tests/test_coders_common.py index d93d9c03dd..42d1c19a64 100644 --- a/flink-python/pyflink/fn_execution/tests/coders_test_common.py +++ b/flink-python/pyflink/fn_execution/tests/test_coders_common.py @@ -22,7 +22,7 @@ import unittest from pyflink.fn_execution.coders import BigIntCoder, TinyIntCoder, BooleanCoder, \ SmallIntCoder, IntCoder, FloatCoder, DoubleCoder, BinaryCoder, CharCoder, DateCoder, \ - TimeCoder, TimestampCoder, ArrayCoder, MapCoder, DecimalCoder, FlattenRowCoder + TimeCoder, TimestampCoder, ArrayCoder, MapCoder, DecimalCoder, FlattenRowCoder, RowCoder class CodersTest(unittest.TestCase): @@ -117,7 +117,20 @@ class CodersTest(unittest.TestCase): field_coder = BigIntCoder() field_count = 10 coder = FlattenRowCoder([field_coder for _ in range(field_count)]) - self.check_coder(coder, [None if i % 2 == 0 else i for i in range(field_count)]) + v = [[None if i % 2 == 0 else i for i in range(field_count)]] + generator_result = coder.decode(coder.encode(v)) + result = [] + for item in generator_result: + result.append(item) + self.assertEqual(v, result) + + def test_row_coder(self): + from pyflink.table import Row + field_coder = BigIntCoder() + field_count = 10 + coder = RowCoder([field_coder for _ in range(field_count)]) + v = Row(*[None if i % 2 == 0 else i for i in range(field_count)]) + self.check_coder(coder, v) if __name__ == '__main__': -- Gitee From 6126600b3717368e7428a4543b524b413c58501e Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Wed, 26 Feb 2020 10:10:24 +0100 Subject: [PATCH 160/885] [FLINK-16261][table] Use type compatibility consistently to avoid casting In many cases, full type equality is not necessary. Especially when leaving the table ecosystem to sinks or UDFs a clear definition of "type compatibility" is important. This commit defines the type compatibility for all logical types to avoid unnecessary casting and less type inequality exceptions. This closes #11225. --- .../rules/ResolveCallByArgumentsRule.java | 4 +- .../strategies/AndArgumentTypeStrategy.java | 4 +- .../ExplicitArgumentTypeStrategy.java | 8 +- .../strategies/MappingTypeStrategy.java | 6 +- .../strategies/OrArgumentTypeStrategy.java | 6 +- .../strategies/OrInputTypeStrategy.java | 10 +- .../types/logical/utils/LogicalTypeCasts.java | 115 ++++++++++++++++++ .../logical/utils/LogicalTypeChecks.java | 60 --------- .../flink/table/utils/TypeMappingUtils.java | 3 +- ...java => LogicalTypeCastAvoidanceTest.java} | 80 ++++++++---- .../inference/InputTypeStrategiesTest.java | 7 ++ .../types/inference/TypeStrategiesTest.java | 7 ++ .../TypeInferenceOperandChecker.java | 13 +- .../calls/BridgingSqlFunctionCallGen.scala | 8 +- .../table/planner/sinks/TableSinkUtils.scala | 23 ++-- .../runtime/stream/sql/FunctionITCase.java | 37 +++++- 16 files changed, 268 insertions(+), 123 deletions(-) rename flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/{LogicalTypeCompatibleCheckTest.java => LogicalTypeCastAvoidanceTest.java} (76%) diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/rules/ResolveCallByArgumentsRule.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/rules/ResolveCallByArgumentsRule.java index d4258b3b2b..1c2dedf7d6 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/rules/ResolveCallByArgumentsRule.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/rules/ResolveCallByArgumentsRule.java @@ -60,6 +60,7 @@ import java.util.stream.IntStream; import static java.util.Collections.singletonList; import static org.apache.flink.table.expressions.ApiExpressionUtils.valueLiteral; +import static org.apache.flink.table.types.logical.utils.LogicalTypeCasts.supportsAvoidingCast; import static org.apache.flink.table.types.utils.TypeConversions.fromDataTypeToLegacyInfo; import static org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType; @@ -231,7 +232,8 @@ final class ResolveCallByArgumentsRule implements ResolverRule { final ResolvedExpression argument = resolvedArgs.get(pos); final DataType argumentType = argument.getOutputDataType(); final DataType expectedType = inferenceResult.getExpectedArgumentTypes().get(pos); - if (!argumentType.equals(expectedType)) { + + if (!supportsAvoidingCast(argumentType.getLogicalType(), expectedType.getLogicalType())) { return resolutionContext .postResolutionFactory() .cast(argument, expectedType); diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/AndArgumentTypeStrategy.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/AndArgumentTypeStrategy.java index f2d89ccd7d..8766141008 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/AndArgumentTypeStrategy.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/AndArgumentTypeStrategy.java @@ -33,6 +33,8 @@ import java.util.Objects; import java.util.Optional; import java.util.stream.Collectors; +import static org.apache.flink.table.types.logical.utils.LogicalTypeCasts.supportsAvoidingCast; + /** * Strategy for inferring and validating an argument using a conjunction of multiple {@link ArgumentTypeStrategy}s * into one like {@code f(NUMERIC && LITERAL)} @@ -72,7 +74,7 @@ public final class AndArgumentTypeStrategy implements ArgumentTypeStrategy { } final LogicalType inferredType = inferredDataType.get().getLogicalType(); // a more specific, casted argument type is available - if (!actualType.equals(inferredType) && !closestDataType.isPresent()) { + if (!supportsAvoidingCast(actualType, inferredType) && !closestDataType.isPresent()) { closestDataType = inferredDataType; } } diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/ExplicitArgumentTypeStrategy.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/ExplicitArgumentTypeStrategy.java index 770558e99e..43e0485b2f 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/ExplicitArgumentTypeStrategy.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/ExplicitArgumentTypeStrategy.java @@ -25,12 +25,14 @@ import org.apache.flink.table.types.inference.ArgumentTypeStrategy; import org.apache.flink.table.types.inference.CallContext; import org.apache.flink.table.types.inference.Signature.Argument; import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.utils.LogicalTypeCasts; import org.apache.flink.util.Preconditions; import java.util.Objects; import java.util.Optional; +import static org.apache.flink.table.types.logical.utils.LogicalTypeCasts.supportsAvoidingCast; +import static org.apache.flink.table.types.logical.utils.LogicalTypeCasts.supportsImplicitCast; + /** * Strategy for an argument that corresponds to an explicitly defined type. Implicit casts will be * inserted if possible. @@ -50,11 +52,11 @@ public final class ExplicitArgumentTypeStrategy implements ArgumentTypeStrategy final LogicalType actualType = callContext.getArgumentDataTypes().get(argumentPos).getLogicalType(); // if logical types match, we return the expected data type // for ensuring the expected conversion class - if (expectedType.equals(actualType)) { + if (supportsAvoidingCast(actualType, expectedType)) { return Optional.of(expectedDataType); } // type coercion - if (!LogicalTypeCasts.supportsImplicitCast(actualType, expectedType)) { + if (!supportsImplicitCast(actualType, expectedType)) { if (throwOnFailure) { throw callContext.newValidationError( "Unsupported argument type. Expected type '%s' but actual type was '%s'.", diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/MappingTypeStrategy.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/MappingTypeStrategy.java index b4047b9cc7..1fa6a0b7dc 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/MappingTypeStrategy.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/MappingTypeStrategy.java @@ -31,9 +31,11 @@ import java.util.Objects; import java.util.Optional; import java.util.stream.Collectors; +import static org.apache.flink.table.types.logical.utils.LogicalTypeCasts.supportsAvoidingCast; + /** * Type strategy that maps an {@link InputTypeStrategy} to a {@link TypeStrategy} if the input strategy - * infers identical types. + * infers compatible types. */ @Internal public final class MappingTypeStrategy implements TypeStrategy { @@ -64,7 +66,7 @@ public final class MappingTypeStrategy implements TypeStrategy { final List inferredTypes = inferredDataTypes.get().stream() .map(DataType::getLogicalType) .collect(Collectors.toList()); - if (actualTypes.equals(inferredTypes)) { + if (supportsAvoidingCast(actualTypes, inferredTypes)) { return strategy.inferType(callContext); } } diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/OrArgumentTypeStrategy.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/OrArgumentTypeStrategy.java index 67a7738fe2..c415fb57a5 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/OrArgumentTypeStrategy.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/OrArgumentTypeStrategy.java @@ -33,6 +33,8 @@ import java.util.Objects; import java.util.Optional; import java.util.stream.Collectors; +import static org.apache.flink.table.types.logical.utils.LogicalTypeCasts.supportsAvoidingCast; + /** * Strategy for inferring and validating an argument using a disjunction of multiple {@link ArgumentTypeStrategy}s * into one like {@code f(NUMERIC || STRING)}. @@ -70,9 +72,9 @@ public final class OrArgumentTypeStrategy implements ArgumentTypeStrategy { continue; } final LogicalType inferredType = inferredDataType.get().getLogicalType(); - // argument type matches exactly + // argument type matches // we prefer a strategy that does not require an implicit cast - if (actualType.equals(inferredType)) { + if (supportsAvoidingCast(actualType, inferredType)) { return inferredDataType; } // argument type requires a more specific, casted type diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/OrInputTypeStrategy.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/OrInputTypeStrategy.java index 5c861bb971..c0d8146d3b 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/OrInputTypeStrategy.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/OrInputTypeStrategy.java @@ -37,6 +37,8 @@ import java.util.Objects; import java.util.Optional; import java.util.stream.Collectors; +import static org.apache.flink.table.types.logical.utils.LogicalTypeCasts.supportsAvoidingCast; + /** * Strategy for inferring and validating the input using a disjunction of multiple {@link InputTypeStrategy}s * into one like {@code f(NUMERIC) || f(STRING)}. @@ -121,8 +123,8 @@ public final class OrInputTypeStrategy implements InputTypeStrategy { final List inferredTypes = inferredDataTypes.get().stream() .map(DataType::getLogicalType) .collect(Collectors.toList()); - // types match exactly - if (actualTypes.equals(inferredTypes)) { + // types match + if (supportsAvoidingCast(actualTypes, inferredTypes)) { return inferredDataTypes; } // type matches with some casting @@ -163,7 +165,7 @@ public final class OrInputTypeStrategy implements InputTypeStrategy { /** * Returns the common minimum argument count or null if undefined. */ - private @Nullable Integer commonMin(List counts) { + private static @Nullable Integer commonMin(List counts) { // min=5, min=3, min=0 -> min=0 // min=5, min=3, min=0, min=null -> min=null int commonMin = Integer.MAX_VALUE; @@ -183,7 +185,7 @@ public final class OrInputTypeStrategy implements InputTypeStrategy { /** * Returns the common maximum argument count or null if undefined. */ - private @Nullable Integer commonMax(List counts) { + private static @Nullable Integer commonMax(List counts) { // max=5, max=3, max=0 -> max=5 // max=5, max=3, max=0, max=null -> max=null int commonMax = Integer.MIN_VALUE; diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeCasts.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeCasts.java index d46288c345..f22f9bee09 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeCasts.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeCasts.java @@ -24,6 +24,9 @@ import org.apache.flink.table.types.logical.DistinctType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.LogicalTypeFamily; import org.apache.flink.table.types.logical.LogicalTypeRoot; +import org.apache.flink.table.types.logical.StructuredType; +import org.apache.flink.table.types.logical.VarBinaryType; +import org.apache.flink.table.types.logical.VarCharType; import java.util.Arrays; import java.util.HashMap; @@ -66,7 +69,9 @@ import static org.apache.flink.table.types.logical.LogicalTypeRoot.TIME_WITHOUT_ import static org.apache.flink.table.types.logical.LogicalTypeRoot.TINYINT; import static org.apache.flink.table.types.logical.LogicalTypeRoot.VARBINARY; import static org.apache.flink.table.types.logical.LogicalTypeRoot.VARCHAR; +import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getLength; import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.hasFamily; +import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.hasRoot; import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.isSingleFieldInterval; /** @@ -207,6 +212,43 @@ public final class LogicalTypeCasts { .build(); } + /** + * Returns whether the source type can be safely interpreted as the target type. This allows avoiding + * casts by ignoring some logical properties. This is basically a relaxed {@link LogicalType#equals(Object)}. + * + *

    In particular this means: + * + *

    Atomic, non-string types (INT, BOOLEAN, ...) and user-defined structured types must be fully + * equal (i.e. {@link LogicalType#equals(Object)}). However, a NOT NULL type can be stored in NULL + * type but not vice versa. + * + *

    Atomic, string types must be contained in the target type (e.g. CHAR(2) is contained in VARCHAR(3), + * but VARCHAR(2) is not contained in CHAR(3)). Same for binary strings. + * + *

    Constructed types (ARRAY, ROW, MAP, etc.) and user-defined distinct type must be of same kind + * but ignore field names and other logical attributes. However, all the children types + * ({@link LogicalType#getChildren()}) must be compatible. + */ + public static boolean supportsAvoidingCast(LogicalType sourceType, LogicalType targetType) { + final CastAvoidanceChecker checker = new CastAvoidanceChecker(sourceType); + return targetType.accept(checker); + } + + /** + * See {@link #supportsAvoidingCast(LogicalType, LogicalType)}. + */ + public static boolean supportsAvoidingCast(List sourceTypes, List targetTypes) { + if (sourceTypes.size() != targetTypes.size()) { + return false; + } + for (int i = 0; i < sourceTypes.size(); i++) { + if (!supportsAvoidingCast(sourceTypes.get(i), targetTypes.get(i))) { + return false; + } + } + return true; + } + /** * Returns whether the source type can be safely casted to the target type without loosing information. * @@ -379,6 +421,79 @@ public final class LogicalTypeCasts { } } + // -------------------------------------------------------------------------------------------- + + /** + * Checks if a source type can safely be interpreted as the target type. + */ + private static class CastAvoidanceChecker extends LogicalTypeDefaultVisitor { + + private final LogicalType sourceType; + + private CastAvoidanceChecker(LogicalType sourceType) { + this.sourceType = sourceType; + } + + @Override + public Boolean visit(VarCharType targetType) { + if (sourceType.isNullable() && !targetType.isNullable()) { + return false; + } + // CHAR and VARCHAR are very compatible within bounds + if ((hasRoot(sourceType, LogicalTypeRoot.CHAR) || hasRoot(sourceType, LogicalTypeRoot.VARCHAR)) && + getLength(sourceType) <= targetType.getLength()) { + return true; + } + return defaultMethod(targetType); + } + + @Override + public Boolean visit(VarBinaryType targetType) { + if (sourceType.isNullable() && !targetType.isNullable()) { + return false; + } + // BINARY and VARBINARY are very compatible within bounds + if ((hasRoot(sourceType, LogicalTypeRoot.BINARY) || hasRoot(sourceType, LogicalTypeRoot.VARBINARY)) && + getLength(sourceType) <= targetType.getLength()) { + return true; + } + return defaultMethod(targetType); + } + + @Override + public Boolean visit(StructuredType targetType) { + if (sourceType.isNullable() && !targetType.isNullable()) { + return false; + } + // structured types should be equal (modulo nullability) + return sourceType.equals(targetType) || sourceType.copy(true).equals(targetType); + } + + @Override + protected Boolean defaultMethod(LogicalType targetType) { + // quick path + if (sourceType == targetType) { + return true; + } + + if (sourceType.isNullable() && !targetType.isNullable() || + sourceType.getClass() != targetType.getClass() || // TODO drop this line once we remove legacy types + sourceType.getTypeRoot() != targetType.getTypeRoot()) { + return false; + } + + final List sourceChildren = sourceType.getChildren(); + final List targetChildren = targetType.getChildren(); + if (sourceChildren.isEmpty()) { + // handles all types that are not of family CONSTRUCTED or USER DEFINED + return sourceType.equals(targetType) || sourceType.copy(true).equals(targetType); + } else { + // handles all types of CONSTRUCTED family as well as distinct types + return supportsAvoidingCast(sourceChildren, targetChildren); + } + } + } + private LogicalTypeCasts() { // no instantiation } diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeChecks.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeChecks.java index 3b00b08116..f4562e7193 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeChecks.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeChecks.java @@ -40,12 +40,9 @@ import org.apache.flink.table.types.logical.VarCharType; import org.apache.flink.table.types.logical.YearMonthIntervalType; import org.apache.flink.table.types.logical.ZonedTimestampType; -import java.util.List; import java.util.Optional; import java.util.function.Predicate; -import static org.apache.flink.util.Preconditions.checkNotNull; - /** * Utilities for checking {@link LogicalType} and avoiding a lot of type casting and repetitive work. */ @@ -175,20 +172,6 @@ public final class LogicalTypeChecks { return logicalType.accept(SINGLE_FIELD_INTERVAL_EXTRACTOR); } - /** - * Returns true if the two given types are compatible. Types are compatible is for atomic types - * (VARCHAR, INT, BOOLEAN, etc..), they must be fully equal (i.e. {@link LogicalType#equals(Object)}), - * for complex types (ARRAY, ROW, MAP, etc..), they must be in the same type but ignore field - * names and other logical attributes, and all the children types ({@link LogicalType#getChildren()}) - * must be compatible too. - */ - public static boolean areTypesCompatible(LogicalType thisType, LogicalType thatType) { - checkNotNull(thisType); - checkNotNull(thatType); - TypeCompatibleVisitor visitor = new TypeCompatibleVisitor(thisType); - return thatType.accept(visitor); - } - private LogicalTypeChecks() { // no instantiation } @@ -377,49 +360,6 @@ public final class LogicalTypeChecks { } } - private static class TypeCompatibleVisitor extends LogicalTypeDefaultVisitor { - - private final LogicalType thisType; - - private TypeCompatibleVisitor(LogicalType thisType) { - checkNotNull(thisType); - this.thisType = thisType; - } - - @Override - protected Boolean defaultMethod(LogicalType thatType) { - checkNotNull(thatType); - if (thisType == thatType) { - return true; - } - if (thisType.getClass() != thatType.getClass() || - thisType.isNullable() != thatType.isNullable() || - thisType.getTypeRoot() != thatType.getTypeRoot()) { - return false; - } - - List thisChildren = thisType.getChildren(); - List thatChildren = thatType.getChildren(); - if (thisChildren.size() != thatChildren.size()) { - return false; - } - if (thisChildren.isEmpty()) { - // if it is an atomic type, delegate to equals method. - return thisType.equals(thatType); - } else { - // if it is composite type, only need to check children types - for (int i = 0; i < thisChildren.size(); i++) { - LogicalType thisChild = thisChildren.get(i); - LogicalType thatChild = thatChildren.get(i); - if (!areTypesCompatible(thisChild, thatChild)) { - return false; - } - } - return true; - } - } - } - /** * Searches for a type (including children) satisfying the given predicate. */ diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/utils/TypeMappingUtils.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/utils/TypeMappingUtils.java index 8239100f7e..4922ac2f3b 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/utils/TypeMappingUtils.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/utils/TypeMappingUtils.java @@ -46,6 +46,7 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.Stream; +import static org.apache.flink.table.types.logical.utils.LogicalTypeCasts.supportsAvoidingCast; import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.hasFamily; /** @@ -247,7 +248,7 @@ public final class TypeMappingUtils { LogicalType physicalFieldType, LogicalType logicalFieldType, Function exceptionSupplier) { - if (LogicalTypeChecks.areTypesCompatible(physicalFieldType, logicalFieldType)) { + if (supportsAvoidingCast(physicalFieldType, logicalFieldType)) { return; } diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypeCompatibleCheckTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypeCastAvoidanceTest.java similarity index 76% rename from flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypeCompatibleCheckTest.java rename to flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypeCastAvoidanceTest.java index cd1bd5fb26..00f6da1f64 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypeCompatibleCheckTest.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypeCastAvoidanceTest.java @@ -48,7 +48,7 @@ import org.apache.flink.table.types.logical.VarBinaryType; import org.apache.flink.table.types.logical.VarCharType; import org.apache.flink.table.types.logical.YearMonthIntervalType; import org.apache.flink.table.types.logical.ZonedTimestampType; -import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; +import org.apache.flink.table.types.logical.utils.LogicalTypeCasts; import org.junit.Test; import org.junit.runner.RunWith; @@ -58,18 +58,18 @@ import org.junit.runners.Parameterized.Parameters; import java.util.Arrays; import java.util.List; -import java.util.UUID; import java.util.stream.Collectors; +import static org.apache.flink.table.types.logical.utils.LogicalTypeCasts.supportsAvoidingCast; import static org.hamcrest.CoreMatchers.equalTo; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; /** - * Tests for {@link LogicalTypeChecks#areTypesCompatible(LogicalType, LogicalType)}. + * Tests for {@link LogicalTypeCasts#supportsAvoidingCast(LogicalType, LogicalType)}. */ @RunWith(Parameterized.class) -public class LogicalTypeCompatibleCheckTest { +public class LogicalTypeCastAvoidanceTest { @Parameters(name = "{index}: [{0} COMPATIBLE {1} => {2}") public static List testData() { @@ -77,13 +77,29 @@ public class LogicalTypeCompatibleCheckTest { new Object[][]{ {new CharType(), new CharType(5), false}, - {new VarCharType(), new VarCharType(33), false}, + {new VarCharType(30), new VarCharType(10), false}, + + {new VarCharType(10), new VarCharType(30), true}, + + {new CharType(10), new VarCharType(30), true}, + + {new BinaryType(10), new VarBinaryType(30), true}, + + {new CharType(false, 10), new VarCharType(30), true}, + + {new BinaryType(false, 10), new VarBinaryType(30), true}, + + {new VarCharType(30), new CharType(10), false}, + + {new VarBinaryType(30), new BinaryType(10), false}, {new BooleanType(), new BooleanType(false), false}, - {new BinaryType(), new BinaryType(22), false}, + {new BinaryType(10), new BinaryType(30), false}, + + {new VarBinaryType(10), new VarBinaryType(30), true}, - {new VarBinaryType(), new VarBinaryType(44), false}, + {new VarBinaryType(30), new VarBinaryType(10), false}, {new DecimalType(), new DecimalType(10, 2), false}, @@ -93,6 +109,8 @@ public class LogicalTypeCompatibleCheckTest { {new IntType(), new IntType(false), false}, + {new IntType(false), new IntType(), true}, + {new BigIntType(), new BigIntType(false), false}, {new FloatType(), new FloatType(false), false}, @@ -134,8 +152,14 @@ public class LogicalTypeCompatibleCheckTest { }, { - new MapType(new VarCharType(20), new TimestampType()), - new MapType(new VarCharType(99), new TimestampType()), + new MapType(new VarCharType(10), new TimestampType()), + new MapType(new VarCharType(30), new TimestampType()), + true + }, + + { + new MapType(new VarCharType(30), new TimestampType()), + new MapType(new VarCharType(10), new TimestampType()), false }, @@ -241,14 +265,26 @@ public class LogicalTypeCompatibleCheckTest { }, { - createUserType(new IntType(), new VarCharType()), - createUserType(new IntType(), new VarCharType()), + createUserType("User", new IntType(), new VarCharType()), + createUserType("User", new IntType(), new VarCharType()), + true + }, + + { + createUserType("User", new IntType(), new VarCharType()), + createUserType("User2", new IntType(), new VarCharType()), + false + }, + + { + createDistinctType("Money", new DecimalType(10, 2)), + createDistinctType("Money", new DecimalType(10, 2)), true }, { - createDistinctType(new DecimalType(10, 2)), - createDistinctType(new DecimalType(10, 2)), + createDistinctType("Money", new DecimalType(10, 2)), + createDistinctType("Money2", new DecimalType(10, 2)), true } } @@ -265,27 +301,27 @@ public class LogicalTypeCompatibleCheckTest { public boolean equals; @Test - public void testAreTypesCompatible() { + public void testSupportsAvoidingCast() { assertThat( - LogicalTypeChecks.areTypesCompatible(sourceType, targetType), + supportsAvoidingCast(sourceType, targetType), equalTo(equals)); - assertTrue(LogicalTypeChecks.areTypesCompatible(sourceType, sourceType.copy())); - assertTrue(LogicalTypeChecks.areTypesCompatible(targetType, targetType.copy())); + assertTrue(supportsAvoidingCast(sourceType, sourceType.copy())); + assertTrue(supportsAvoidingCast(targetType, targetType.copy())); } - private static DistinctType createDistinctType(LogicalType sourceType) { + private static DistinctType createDistinctType(String name, LogicalType sourceType) { return DistinctType.newBuilder( - ObjectIdentifier.of("cat", "db", UUID.randomUUID().toString()), + ObjectIdentifier.of("cat", "db", name), sourceType) .description("Money type desc.") .build(); } - private static StructuredType createUserType(LogicalType... children) { - return StructuredType.newBuilder(ObjectIdentifier.of("cat", "db", "User"), User.class) + private static StructuredType createUserType(String name, LogicalType... children) { + return StructuredType.newBuilder(ObjectIdentifier.of("cat", "db", name), User.class) .attributes( Arrays.stream(children) - .map(lt -> new StructuredType.StructuredAttribute(UUID.randomUUID().toString(), lt)) + .map(lt -> new StructuredType.StructuredAttribute("field", lt)) .collect(Collectors.toList())) .description("User type desc.") .setFinal(true) diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/InputTypeStrategiesTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/InputTypeStrategiesTest.java index 74b4ea4fcc..9bfb1b147f 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/InputTypeStrategiesTest.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/InputTypeStrategiesTest.java @@ -89,6 +89,13 @@ public class InputTypeStrategiesTest { .expectSignature("f(INT, BOOLEAN)") .expectArgumentTypes(DataTypes.INT().bridgedTo(int.class), DataTypes.BOOLEAN()), + // explicit sequence with ROW ignoring field names + TestSpec + .forStrategy(explicitSequence(DataTypes.ROW(DataTypes.FIELD("expected", DataTypes.INT())))) + .calledWithArgumentTypes(DataTypes.ROW(DataTypes.FIELD("actual", DataTypes.INT()))) + .expectSignature("f(ROW<`expected` INT>)") + .expectArgumentTypes(DataTypes.ROW(DataTypes.FIELD("expected", DataTypes.INT()))), + // invalid named sequence TestSpec .forStrategy( diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/TypeStrategiesTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/TypeStrategiesTest.java index 1a57dc5d74..a5a4d2d321 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/TypeStrategiesTest.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/TypeStrategiesTest.java @@ -93,6 +93,13 @@ public class TypeStrategiesTest { .inputTypes(DataTypes.INT(), DataTypes.STRING()) .expectDataType(DataTypes.BOOLEAN().bridgedTo(boolean.class)), + // (INT, CHAR(10)) -> BOOLEAN + // but avoiding casts (mapping actually expects STRING) + TestSpec + .forStrategy(createMappingTypeStrategy()) + .inputTypes(DataTypes.INT(), DataTypes.CHAR(10)) + .expectDataType(DataTypes.BOOLEAN().bridgedTo(boolean.class)), + // invalid mapping strategy TestSpec .forStrategy(createMappingTypeStrategy()) diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandChecker.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandChecker.java index 330f3c4dbd..cc84e5d3cd 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandChecker.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandChecker.java @@ -44,11 +44,13 @@ import org.apache.calcite.sql.validate.SqlValidatorNamespace; import java.util.List; +import static org.apache.flink.table.planner.calcite.FlinkTypeFactory.toLogicalType; import static org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTypeFactory; import static org.apache.flink.table.types.inference.TypeInferenceUtil.adaptArguments; import static org.apache.flink.table.types.inference.TypeInferenceUtil.createInvalidCallException; import static org.apache.flink.table.types.inference.TypeInferenceUtil.createInvalidInputException; import static org.apache.flink.table.types.inference.TypeInferenceUtil.createUnexpectedException; +import static org.apache.flink.table.types.logical.utils.LogicalTypeCasts.supportsAvoidingCast; /** * A {@link SqlOperandTypeChecker} backed by {@link TypeInference}. @@ -142,11 +144,14 @@ public final class TypeInferenceOperandChecker implements SqlOperandTypeChecker final List operands = callBinding.operands(); for (int i = 0; i < operands.size(); i++) { final LogicalType expectedType = expectedDataTypes.get(i).getLogicalType(); - final RelDataType expectedRelDataType = flinkTypeFactory.createFieldTypeFromLogicalType(expectedType); + final LogicalType argumentType = toLogicalType(callBinding.getOperandType(i)); - final SqlNode castedOperand = castTo(operands.get(i), expectedRelDataType); - callBinding.getCall().setOperand(i, castedOperand); - updateInferredType(callBinding.getValidator(), castedOperand, expectedRelDataType); + if (!supportsAvoidingCast(argumentType, expectedType)) { + final RelDataType expectedRelDataType = flinkTypeFactory.createFieldTypeFromLogicalType(expectedType); + final SqlNode castedOperand = castTo(operands.get(i), expectedRelDataType); + callBinding.getCall().setOperand(i, castedOperand); + updateInferredType(callBinding.getValidator(), castedOperand, expectedRelDataType); + } } } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/BridgingSqlFunctionCallGen.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/BridgingSqlFunctionCallGen.scala index 6380d4afbe..2acf2db61d 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/BridgingSqlFunctionCallGen.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/BridgingSqlFunctionCallGen.scala @@ -36,7 +36,7 @@ import org.apache.flink.table.types.DataType import org.apache.flink.table.types.extraction.utils.ExtractionUtils import org.apache.flink.table.types.extraction.utils.ExtractionUtils.{createMethodSignatureString, isAssignable, isMethodInvokable, primitiveToWrapper} import org.apache.flink.table.types.inference.TypeInferenceUtil -import org.apache.flink.table.types.logical.utils.LogicalTypeChecks +import org.apache.flink.table.types.logical.utils.LogicalTypeCasts.supportsAvoidingCast import org.apache.flink.table.types.logical.utils.LogicalTypeChecks.{hasRoot, isCompositeType} import org.apache.flink.table.types.logical.{LogicalType, LogicalTypeRoot, RowType} import org.apache.flink.util.Preconditions @@ -252,8 +252,7 @@ class BridgingSqlFunctionCallGen(call: RexCall) extends CallGenerator { val enrichedTypes = enrichedDataTypes.map(_.getLogicalType) operandTypes.zip(enrichedTypes).foreach { case (operandType, enrichedType) => // check that the logical type has not changed during the enrichment - // a nullability mismatch is acceptable if the enriched type can handle it - if (operandType != enrichedType && operandType.copy(true) != enrichedType) { + if (!supportsAvoidingCast(operandType, enrichedType)) { throw new CodeGenException( s"Mismatch of function's argument data type '$enrichedType' and actual " + s"argument type '$operandType'.") @@ -295,8 +294,7 @@ class BridgingSqlFunctionCallGen(call: RexCall) extends CallGenerator { : Unit = { val enrichedType = enrichedDataType.getLogicalType // check that the logical type has not changed during the enrichment - // a nullability mismatch is acceptable if the output type can handle it - if (returnType != enrichedType && returnType != enrichedType.copy(true)) { + if (!supportsAvoidingCast(enrichedType, returnType)) { throw new CodeGenException( s"Mismatch of expected output data type '$returnType' and function's " + s"output type '$enrichedType'.") diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sinks/TableSinkUtils.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sinks/TableSinkUtils.scala index 90955ee9e1..84397c9d5b 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sinks/TableSinkUtils.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sinks/TableSinkUtils.scala @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.sinks +import org.apache.calcite.rel.RelNode import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2} import org.apache.flink.api.java.typeutils.{GenericTypeInfo, PojoTypeInfo, TupleTypeInfo} import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo @@ -31,14 +32,14 @@ import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter.fromDataTy import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo import org.apache.flink.table.sinks._ import org.apache.flink.table.types.DataType -import org.apache.flink.table.types.inference.TypeTransformations.{legacyDecimalToDefaultDecimal, toNullable, legacyRawToTypeInfoRaw} -import org.apache.flink.table.types.logical.utils.{LogicalTypeCasts, LogicalTypeChecks} -import org.apache.flink.table.types.logical.{LegacyTypeInformationType, LogicalType, RowType, TypeInformationRawType} +import org.apache.flink.table.types.inference.TypeTransformations.{legacyDecimalToDefaultDecimal, legacyRawToTypeInfoRaw, toNullable} +import org.apache.flink.table.types.logical.utils.LogicalTypeCasts.{supportsAvoidingCast, supportsImplicitCast} +import org.apache.flink.table.types.logical.utils.LogicalTypeChecks +import org.apache.flink.table.types.logical.{LegacyTypeInformationType, RowType} import org.apache.flink.table.types.utils.DataTypeUtils import org.apache.flink.table.types.utils.TypeConversions.{fromLegacyInfoToDataType, fromLogicalToDataType} import org.apache.flink.table.utils.{TableSchemaUtils, TypeMappingUtils} import org.apache.flink.types.Row -import org.apache.calcite.rel.RelNode import _root_.scala.collection.JavaConversions._ @@ -67,13 +68,10 @@ object TableSinkUtils { .transform(sinkSchema.toRowDataType, legacyDecimalToDefaultDecimal, legacyRawToTypeInfoRaw) .getLogicalType .asInstanceOf[RowType] - if (LogicalTypeCasts.supportsImplicitCast(queryLogicalType, sinkLogicalType)) { + if (supportsImplicitCast(queryLogicalType, sinkLogicalType)) { // the query can be written into sink // but we may need to add a cast project if the types are not compatible - if (LogicalTypeChecks.areTypesCompatible( - nullableLogicalType(queryLogicalType), nullableLogicalType(sinkLogicalType))) { - // types are compatible excepts nullable, do not need cast project - // we ignores nullable to avoid cast project as cast non-null to nullable is redundant + if (supportsAvoidingCast(queryLogicalType, sinkLogicalType)) { query } else { // otherwise, add a cast project @@ -100,13 +98,6 @@ object TableSinkUtils { } } - /** - * Make the logical type nullable recursively. - */ - private def nullableLogicalType(logicalType: LogicalType): LogicalType = { - DataTypeUtils.transform(fromLogicalToDataType(logicalType), toNullable).getLogicalType - } - /** * It checks whether the [[TableSink]] is compatible to the INSERT INTO clause, e.g. * whether the sink is a [[PartitionableTableSink]] and the partitions are valid. diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/FunctionITCase.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/FunctionITCase.java index 0491326e45..fcc26d1a65 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/FunctionITCase.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/FunctionITCase.java @@ -453,15 +453,38 @@ public class FunctionITCase extends StreamingTestBase { TestCollectionTableFactory.reset(); TestCollectionTableFactory.initData(sourceData); - tEnv().sqlUpdate("CREATE TABLE TestTable(a INT NOT NULL, b BIGINT NOT NULL, c STRING) WITH ('connector' = 'COLLECTION')"); + tEnv().sqlUpdate("CREATE TABLE TestTable(i INT NOT NULL, b BIGINT NOT NULL, s STRING) WITH ('connector' = 'COLLECTION')"); tEnv().createTemporarySystemFunction("PrimitiveScalarFunction", PrimitiveScalarFunction.class); - tEnv().sqlUpdate("INSERT INTO TestTable SELECT a, PrimitiveScalarFunction(a, b, c), c FROM TestTable"); + tEnv().sqlUpdate("INSERT INTO TestTable SELECT i, PrimitiveScalarFunction(i, b, s), s FROM TestTable"); tEnv().execute("Test Job"); assertThat(TestCollectionTableFactory.getResult(), equalTo(sinkData)); } + @Test + public void testRowScalarFunction() throws Exception { + final List sourceData = Arrays.asList( + Row.of(1, Row.of(1, "1")), + Row.of(2, Row.of(2, "2")), + Row.of(3, Row.of(3, "3")) + ); + + TestCollectionTableFactory.reset(); + TestCollectionTableFactory.initData(sourceData); + + tEnv().sqlUpdate( + "CREATE TABLE TestTable(i INT, r ROW) " + + "WITH ('connector' = 'COLLECTION')"); + + tEnv().createTemporarySystemFunction("RowScalarFunction", RowScalarFunction.class); + // the names of the function input and r differ + tEnv().sqlUpdate("INSERT INTO TestTable SELECT i, RowScalarFunction(r) FROM TestTable"); + tEnv().execute("Test Job"); + + assertThat(TestCollectionTableFactory.getResult(), equalTo(sourceData)); + } + @Test public void testComplexScalarFunction() throws Exception { final List sourceData = Arrays.asList( @@ -687,6 +710,16 @@ public class FunctionITCase extends StreamingTestBase { } } + /** + * Function that takes and returns rows. + */ + public static class RowScalarFunction extends ScalarFunction { + public @DataTypeHint("ROW") Row eval( + @DataTypeHint("ROW") Row row) { + return row; + } + } + /** * Function that is overloaded and takes use of annotations. */ -- Gitee From 3b99f7e4d769f6e2b6e9fd8e36a542e063d6c369 Mon Sep 17 00:00:00 2001 From: Roman Khachatryan Date: Wed, 11 Mar 2020 17:55:17 +0100 Subject: [PATCH 161/885] [FLINK-16554][task][refactoring] split StreamTask 1. Extract static CheckpointingOperation and AsyncCheckpointRunnable from StreamTask to separate files to reduce complexity. No changes in logic or structure, just a simple refactoring. 2. Cleanup CheckpointingOperation and AsyncCheckpointRunnable: Don't pass StreamTask as an argument 3. move registering AsyncCheckpointRunnable Closeable to its run() method --- .../tasks/AsyncCheckpointRunnable.java | 256 +++++++++++++ .../runtime/tasks/CheckpointingOperation.java | 136 +++++++ .../streaming/runtime/tasks/StreamTask.java | 355 +----------------- .../tasks/LocalStateForwardingTest.java | 16 +- 4 files changed, 409 insertions(+), 354 deletions(-) create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AsyncCheckpointRunnable.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/CheckpointingOperation.java diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AsyncCheckpointRunnable.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AsyncCheckpointRunnable.java new file mode 100644 index 0000000000..098384a89c --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AsyncCheckpointRunnable.java @@ -0,0 +1,256 @@ +/* + * 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.runtime.tasks; + +import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.core.fs.FileSystemSafetyNet; +import org.apache.flink.runtime.checkpoint.CheckpointMetaData; +import org.apache.flink.runtime.checkpoint.CheckpointMetrics; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.streaming.api.operators.OperatorSnapshotFinalizer; +import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * This runnable executes the asynchronous parts of all involved backend snapshots for the subtask. + */ +final class AsyncCheckpointRunnable implements Runnable, Closeable { + + public static final Logger LOG = LoggerFactory.getLogger(AsyncCheckpointRunnable.class); + private final String taskName; + private final CloseableRegistry closeableRegistry; + private final Environment taskEnvironment; + + private enum AsyncCheckpointState { + RUNNING, + DISCARDED, + COMPLETED + } + + private final AsyncExceptionHandler asyncExceptionHandler; + private final Map operatorSnapshotsInProgress; + private final CheckpointMetaData checkpointMetaData; + private final CheckpointMetrics checkpointMetrics; + private final long asyncStartNanos; + private final AtomicReference asyncCheckpointState = new AtomicReference<>(AsyncCheckpointState.RUNNING); + + AsyncCheckpointRunnable( + Map operatorSnapshotsInProgress, + CheckpointMetaData checkpointMetaData, + CheckpointMetrics checkpointMetrics, + long asyncStartNanos, + String taskName, + CloseableRegistry closeableRegistry, + Environment taskEnvironment, + AsyncExceptionHandler asyncExceptionHandler) { + + this.operatorSnapshotsInProgress = checkNotNull(operatorSnapshotsInProgress); + this.checkpointMetaData = checkNotNull(checkpointMetaData); + this.checkpointMetrics = checkNotNull(checkpointMetrics); + this.asyncStartNanos = asyncStartNanos; + this.taskName = checkNotNull(taskName); + this.closeableRegistry = checkNotNull(closeableRegistry); + this.taskEnvironment = checkNotNull(taskEnvironment); + this.asyncExceptionHandler = checkNotNull(asyncExceptionHandler); + } + + @Override + public void run() { + FileSystemSafetyNet.initializeSafetyNetForThread(); + try { + closeableRegistry.registerCloseable(this); + + TaskStateSnapshot jobManagerTaskOperatorSubtaskStates = new TaskStateSnapshot(operatorSnapshotsInProgress.size()); + TaskStateSnapshot localTaskOperatorSubtaskStates = new TaskStateSnapshot(operatorSnapshotsInProgress.size()); + + for (Map.Entry entry : operatorSnapshotsInProgress.entrySet()) { + + OperatorID operatorID = entry.getKey(); + OperatorSnapshotFutures snapshotInProgress = entry.getValue(); + + // finalize the async part of all by executing all snapshot runnables + OperatorSnapshotFinalizer finalizedSnapshots = + new OperatorSnapshotFinalizer(snapshotInProgress); + + jobManagerTaskOperatorSubtaskStates.putSubtaskStateByOperatorID( + operatorID, + finalizedSnapshots.getJobManagerOwnedState()); + + localTaskOperatorSubtaskStates.putSubtaskStateByOperatorID( + operatorID, + finalizedSnapshots.getTaskLocalState()); + } + + final long asyncEndNanos = System.nanoTime(); + final long asyncDurationMillis = (asyncEndNanos - asyncStartNanos) / 1_000_000L; + + checkpointMetrics.setAsyncDurationMillis(asyncDurationMillis); + + if (asyncCheckpointState.compareAndSet(AsyncCheckpointState.RUNNING, AsyncCheckpointState.COMPLETED)) { + + reportCompletedSnapshotStates( + jobManagerTaskOperatorSubtaskStates, + localTaskOperatorSubtaskStates, + asyncDurationMillis); + + } else { + LOG.debug("{} - asynchronous part of checkpoint {} could not be completed because it was closed before.", + taskName, + checkpointMetaData.getCheckpointId()); + } + } catch (Exception e) { + if (LOG.isDebugEnabled()) { + LOG.debug("{} - asynchronous part of checkpoint {} could not be completed.", + taskName, + checkpointMetaData.getCheckpointId(), + e); + } + handleExecutionException(e); + } finally { + closeableRegistry.unregisterCloseable(this); + FileSystemSafetyNet.closeSafetyNetAndGuardedResourcesForThread(); + } + } + + private void reportCompletedSnapshotStates( + TaskStateSnapshot acknowledgedTaskStateSnapshot, + TaskStateSnapshot localTaskStateSnapshot, + long asyncDurationMillis) { + + boolean hasAckState = acknowledgedTaskStateSnapshot.hasState(); + boolean hasLocalState = localTaskStateSnapshot.hasState(); + + Preconditions.checkState(hasAckState || !hasLocalState, + "Found cached state but no corresponding primary state is reported to the job " + + "manager. This indicates a problem."); + + // we signal stateless tasks by reporting null, so that there are no attempts to assign empty state + // to stateless tasks on restore. This enables simple job modifications that only concern + // stateless without the need to assign them uids to match their (always empty) states. + taskEnvironment.getTaskStateManager().reportTaskStateSnapshots( + checkpointMetaData, + checkpointMetrics, + hasAckState ? acknowledgedTaskStateSnapshot : null, + hasLocalState ? localTaskStateSnapshot : null); + + LOG.debug("{} - finished asynchronous part of checkpoint {}. Asynchronous duration: {} ms", + taskName, checkpointMetaData.getCheckpointId(), asyncDurationMillis); + + LOG.trace("{} - reported the following states in snapshot for checkpoint {}: {}.", + taskName, checkpointMetaData.getCheckpointId(), acknowledgedTaskStateSnapshot); + } + + private void handleExecutionException(Exception e) { + + boolean didCleanup = false; + AsyncCheckpointState currentState = asyncCheckpointState.get(); + + while (AsyncCheckpointState.DISCARDED != currentState) { + + if (asyncCheckpointState.compareAndSet(currentState, AsyncCheckpointState.DISCARDED)) { + + didCleanup = true; + + try { + cleanup(); + } catch (Exception cleanupException) { + e.addSuppressed(cleanupException); + } + + Exception checkpointException = new Exception( + "Could not materialize checkpoint " + checkpointMetaData.getCheckpointId() + " for operator " + + taskName + '.', + e); + + // We only report the exception for the original cause of fail and cleanup. + // Otherwise this followup exception could race the original exception in failing the task. + try { + taskEnvironment.declineCheckpoint(checkpointMetaData.getCheckpointId(), checkpointException); + } catch (Exception unhandled) { + AsynchronousException asyncException = new AsynchronousException(unhandled); + asyncExceptionHandler.handleAsyncException("Failure in asynchronous checkpoint materialization", asyncException); + } + + currentState = AsyncCheckpointState.DISCARDED; + } else { + currentState = asyncCheckpointState.get(); + } + } + + if (!didCleanup) { + LOG.trace("Caught followup exception from a failed checkpoint thread. This can be ignored.", e); + } + } + + @Override + public void close() { + if (asyncCheckpointState.compareAndSet(AsyncCheckpointState.RUNNING, AsyncCheckpointState.DISCARDED)) { + + try { + cleanup(); + } catch (Exception cleanupException) { + LOG.warn("Could not properly clean up the async checkpoint runnable.", cleanupException); + } + } else { + logFailedCleanupAttempt(); + } + } + + private void cleanup() throws Exception { + LOG.debug( + "Cleanup AsyncCheckpointRunnable for checkpoint {} of {}.", + checkpointMetaData.getCheckpointId(), + taskName); + + Exception exception = null; + + // clean up ongoing operator snapshot results and non partitioned state handles + for (OperatorSnapshotFutures operatorSnapshotResult : operatorSnapshotsInProgress.values()) { + if (operatorSnapshotResult != null) { + try { + operatorSnapshotResult.cancel(); + } catch (Exception cancelException) { + exception = ExceptionUtils.firstOrSuppressed(cancelException, exception); + } + } + } + + if (null != exception) { + throw exception; + } + } + + private void logFailedCleanupAttempt() { + LOG.debug("{} - asynchronous checkpointing operation for checkpoint {} has " + + "already been completed. Thus, the state handles are not cleaned up.", + taskName, + checkpointMetaData.getCheckpointId()); + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/CheckpointingOperation.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/CheckpointingOperation.java new file mode 100644 index 0000000000..cb88ea0dbd --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/CheckpointingOperation.java @@ -0,0 +1,136 @@ +/* + * 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.runtime.tasks; + +import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.runtime.checkpoint.CheckpointMetaData; +import org.apache.flink.runtime.checkpoint.CheckpointMetrics; +import org.apache.flink.runtime.checkpoint.CheckpointOptions; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.state.CheckpointStreamFactory; +import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.concurrent.ExecutorService; + +final class CheckpointingOperation { + + public static final Logger LOG = LoggerFactory.getLogger(CheckpointingOperation.class); + + static void execute( + CheckpointMetaData checkpointMetaData, + CheckpointOptions checkpointOptions, + CheckpointMetrics checkpointMetrics, + CheckpointStreamFactory storageLocation, + OperatorChain operatorChain, + String taskName, + CloseableRegistry closeableRegistry, + ExecutorService threadPool, + Environment environment, + AsyncExceptionHandler asyncExceptionHandler) throws Exception { + + Preconditions.checkNotNull(checkpointMetaData); + Preconditions.checkNotNull(checkpointOptions); + Preconditions.checkNotNull(checkpointMetrics); + Preconditions.checkNotNull(storageLocation); + Preconditions.checkNotNull(operatorChain); + Preconditions.checkNotNull(closeableRegistry); + Preconditions.checkNotNull(threadPool); + Preconditions.checkNotNull(environment); + Preconditions.checkNotNull(asyncExceptionHandler); + + long startSyncPartNano = System.nanoTime(); + + HashMap operatorSnapshotsInProgress = new HashMap<>(operatorChain.getNumberOfOperators()); + try { + for (StreamOperatorWrapper operatorWrapper : operatorChain.getAllOperators(true)) { + StreamOperator op = operatorWrapper.getStreamOperator(); + OperatorSnapshotFutures snapshotInProgress = op.snapshotState( + checkpointMetaData.getCheckpointId(), + checkpointMetaData.getTimestamp(), + checkpointOptions, + storageLocation); + operatorSnapshotsInProgress.put(op.getOperatorID(), snapshotInProgress); + } + + if (LOG.isDebugEnabled()) { + LOG.debug("Finished synchronous checkpoints for checkpoint {} on task {}", + checkpointMetaData.getCheckpointId(), taskName); + } + + long startAsyncPartNano = System.nanoTime(); + + checkpointMetrics.setSyncDurationMillis((startAsyncPartNano - startSyncPartNano) / 1_000_000); + + // we are transferring ownership over snapshotInProgressList for cleanup to the thread, active on submit + threadPool.execute(new AsyncCheckpointRunnable( + operatorSnapshotsInProgress, + checkpointMetaData, + checkpointMetrics, + startAsyncPartNano, + taskName, + closeableRegistry, + environment, + asyncExceptionHandler)); + + if (LOG.isDebugEnabled()) { + LOG.debug("{} - finished synchronous part of checkpoint {}. " + + "Alignment duration: {} ms, snapshot duration {} ms", + taskName, checkpointMetaData.getCheckpointId(), + checkpointMetrics.getAlignmentDurationNanos() / 1_000_000, + checkpointMetrics.getSyncDurationMillis()); + } + } catch (Exception ex) { + // Cleanup to release resources + for (OperatorSnapshotFutures operatorSnapshotResult : operatorSnapshotsInProgress.values()) { + if (null != operatorSnapshotResult) { + try { + operatorSnapshotResult.cancel(); + } catch (Exception e) { + LOG.warn("Could not properly cancel an operator snapshot result.", e); + } + } + } + + if (LOG.isDebugEnabled()) { + LOG.debug("{} - did NOT finish synchronous part of checkpoint {}. " + + "Alignment duration: {} ms, snapshot duration {} ms", + taskName, checkpointMetaData.getCheckpointId(), + checkpointMetrics.getAlignmentDurationNanos() / 1_000_000, + checkpointMetrics.getSyncDurationMillis()); + } + + if (checkpointOptions.getCheckpointType().isSynchronous()) { + // in the case of a synchronous checkpoint, we always rethrow the exception, + // so that the task fails. + // this is because the intention is always to stop the job after this checkpointing + // operation, and without the failure, the task would go back to normal execution. + throw ex; + } else { + environment.declineCheckpoint(checkpointMetaData.getCheckpointId(), ex); + } + } + } + +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java index 0c9b2d19ac..aabd9c54b1 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java @@ -22,7 +22,6 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.core.fs.CloseableRegistry; -import org.apache.flink.core.fs.FileSystemSafetyNet; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.SimpleCounter; import org.apache.flink.runtime.checkpoint.CheckpointException; @@ -30,7 +29,6 @@ import org.apache.flink.runtime.checkpoint.CheckpointFailureReason; import org.apache.flink.runtime.checkpoint.CheckpointMetaData; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; import org.apache.flink.runtime.checkpoint.CheckpointOptions; -import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.execution.CancelTaskException; import org.apache.flink.runtime.execution.Environment; @@ -52,7 +50,6 @@ import org.apache.flink.runtime.state.CheckpointStorageWorkerView; import org.apache.flink.runtime.state.CheckpointStreamFactory; import org.apache.flink.runtime.state.StateBackend; import org.apache.flink.runtime.state.StateBackendLoader; -import org.apache.flink.runtime.state.TaskStateManager; import org.apache.flink.runtime.taskmanager.DispatcherThreadFactory; import org.apache.flink.runtime.util.ExecutorThreadFactory; import org.apache.flink.runtime.util.FatalExitExceptionHandler; @@ -60,8 +57,6 @@ import org.apache.flink.streaming.api.TimeCharacteristic; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.graph.StreamEdge; import org.apache.flink.streaming.api.operators.MailboxExecutor; -import org.apache.flink.streaming.api.operators.OperatorSnapshotFinalizer; -import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures; import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.api.operators.StreamTaskStateInitializer; import org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl; @@ -77,7 +72,6 @@ import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxExecutorFactory; import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor; import org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailbox; import org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailboxImpl; -import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; import org.apache.flink.util.SerializedValue; @@ -87,9 +81,7 @@ import org.slf4j.LoggerFactory; import javax.annotation.Nullable; -import java.io.Closeable; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.OptionalLong; @@ -99,7 +91,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.ThreadFactory; -import java.util.concurrent.atomic.AtomicReference; /** * Base class for all streaming tasks. A task is the unit of local processing that is deployed @@ -928,14 +919,17 @@ public abstract class StreamTask> checkpointMetaData.getCheckpointId(), checkpointOptions.getTargetLocation()); - CheckpointingOperation checkpointingOperation = new CheckpointingOperation( - this, + CheckpointingOperation.execute( checkpointMetaData, checkpointOptions, + checkpointMetrics, storage, - checkpointMetrics); - - checkpointingOperation.executeCheckpointing(); + operatorChain, + getName(), + getCancelables(), + getAsyncOperationsThreadPool(), + getEnvironment(), + this); } // ------------------------------------------------------------------------ @@ -1053,345 +1047,12 @@ public abstract class StreamTask> } } - /** - * This runnable executes the asynchronous parts of all involved backend snapshots for the subtask. - */ - @VisibleForTesting - protected static final class AsyncCheckpointRunnable implements Runnable, Closeable { - - private final StreamTask owner; - - private final Map operatorSnapshotsInProgress; - - private final CheckpointMetaData checkpointMetaData; - private final CheckpointMetrics checkpointMetrics; - - private final long asyncStartNanos; - - private final AtomicReference asyncCheckpointState = new AtomicReference<>( - CheckpointingOperation.AsyncCheckpointState.RUNNING); - - AsyncCheckpointRunnable( - StreamTask owner, - Map operatorSnapshotsInProgress, - CheckpointMetaData checkpointMetaData, - CheckpointMetrics checkpointMetrics, - long asyncStartNanos) { - - this.owner = Preconditions.checkNotNull(owner); - this.operatorSnapshotsInProgress = Preconditions.checkNotNull(operatorSnapshotsInProgress); - this.checkpointMetaData = Preconditions.checkNotNull(checkpointMetaData); - this.checkpointMetrics = Preconditions.checkNotNull(checkpointMetrics); - this.asyncStartNanos = asyncStartNanos; - } - - @Override - public void run() { - FileSystemSafetyNet.initializeSafetyNetForThread(); - try { - - TaskStateSnapshot jobManagerTaskOperatorSubtaskStates = - new TaskStateSnapshot(operatorSnapshotsInProgress.size()); - - TaskStateSnapshot localTaskOperatorSubtaskStates = - new TaskStateSnapshot(operatorSnapshotsInProgress.size()); - - for (Map.Entry entry : operatorSnapshotsInProgress.entrySet()) { - - OperatorID operatorID = entry.getKey(); - OperatorSnapshotFutures snapshotInProgress = entry.getValue(); - - // finalize the async part of all by executing all snapshot runnables - OperatorSnapshotFinalizer finalizedSnapshots = - new OperatorSnapshotFinalizer(snapshotInProgress); - - jobManagerTaskOperatorSubtaskStates.putSubtaskStateByOperatorID( - operatorID, - finalizedSnapshots.getJobManagerOwnedState()); - - localTaskOperatorSubtaskStates.putSubtaskStateByOperatorID( - operatorID, - finalizedSnapshots.getTaskLocalState()); - } - - final long asyncEndNanos = System.nanoTime(); - final long asyncDurationMillis = (asyncEndNanos - asyncStartNanos) / 1_000_000L; - - checkpointMetrics.setAsyncDurationMillis(asyncDurationMillis); - - if (asyncCheckpointState.compareAndSet(CheckpointingOperation.AsyncCheckpointState.RUNNING, - CheckpointingOperation.AsyncCheckpointState.COMPLETED)) { - - reportCompletedSnapshotStates( - jobManagerTaskOperatorSubtaskStates, - localTaskOperatorSubtaskStates, - asyncDurationMillis); - - } else { - LOG.debug("{} - asynchronous part of checkpoint {} could not be completed because it was closed before.", - owner.getName(), - checkpointMetaData.getCheckpointId()); - } - } catch (Exception e) { - if (LOG.isDebugEnabled()) { - LOG.debug("{} - asynchronous part of checkpoint {} could not be completed.", - owner.getName(), - checkpointMetaData.getCheckpointId(), - e); - } - handleExecutionException(e); - } finally { - owner.cancelables.unregisterCloseable(this); - FileSystemSafetyNet.closeSafetyNetAndGuardedResourcesForThread(); - } - } - - private void reportCompletedSnapshotStates( - TaskStateSnapshot acknowledgedTaskStateSnapshot, - TaskStateSnapshot localTaskStateSnapshot, - long asyncDurationMillis) { - - TaskStateManager taskStateManager = owner.getEnvironment().getTaskStateManager(); - - boolean hasAckState = acknowledgedTaskStateSnapshot.hasState(); - boolean hasLocalState = localTaskStateSnapshot.hasState(); - - Preconditions.checkState(hasAckState || !hasLocalState, - "Found cached state but no corresponding primary state is reported to the job " + - "manager. This indicates a problem."); - - // we signal stateless tasks by reporting null, so that there are no attempts to assign empty state - // to stateless tasks on restore. This enables simple job modifications that only concern - // stateless without the need to assign them uids to match their (always empty) states. - taskStateManager.reportTaskStateSnapshots( - checkpointMetaData, - checkpointMetrics, - hasAckState ? acknowledgedTaskStateSnapshot : null, - hasLocalState ? localTaskStateSnapshot : null); - - LOG.debug("{} - finished asynchronous part of checkpoint {}. Asynchronous duration: {} ms", - owner.getName(), checkpointMetaData.getCheckpointId(), asyncDurationMillis); - - LOG.trace("{} - reported the following states in snapshot for checkpoint {}: {}.", - owner.getName(), checkpointMetaData.getCheckpointId(), acknowledgedTaskStateSnapshot); - } - - private void handleExecutionException(Exception e) { - - boolean didCleanup = false; - CheckpointingOperation.AsyncCheckpointState currentState = asyncCheckpointState.get(); - - while (CheckpointingOperation.AsyncCheckpointState.DISCARDED != currentState) { - - if (asyncCheckpointState.compareAndSet( - currentState, - CheckpointingOperation.AsyncCheckpointState.DISCARDED)) { - - didCleanup = true; - - try { - cleanup(); - } catch (Exception cleanupException) { - e.addSuppressed(cleanupException); - } - - Exception checkpointException = new Exception( - "Could not materialize checkpoint " + checkpointMetaData.getCheckpointId() + " for operator " + - owner.getName() + '.', - e); - - // We only report the exception for the original cause of fail and cleanup. - // Otherwise this followup exception could race the original exception in failing the task. - try { - owner.getEnvironment().declineCheckpoint(checkpointMetaData.getCheckpointId(), checkpointException); - } catch (Exception unhandled) { - AsynchronousException asyncException = new AsynchronousException(unhandled); - owner.handleAsyncException("Failure in asynchronous checkpoint materialization", asyncException); - } - - currentState = CheckpointingOperation.AsyncCheckpointState.DISCARDED; - } else { - currentState = asyncCheckpointState.get(); - } - } - - if (!didCleanup) { - LOG.trace("Caught followup exception from a failed checkpoint thread. This can be ignored.", e); - } - } - - @Override - public void close() { - if (asyncCheckpointState.compareAndSet( - CheckpointingOperation.AsyncCheckpointState.RUNNING, - CheckpointingOperation.AsyncCheckpointState.DISCARDED)) { - - try { - cleanup(); - } catch (Exception cleanupException) { - LOG.warn("Could not properly clean up the async checkpoint runnable.", cleanupException); - } - } else { - logFailedCleanupAttempt(); - } - } - - private void cleanup() throws Exception { - LOG.debug( - "Cleanup AsyncCheckpointRunnable for checkpoint {} of {}.", - checkpointMetaData.getCheckpointId(), - owner.getName()); - - Exception exception = null; - - // clean up ongoing operator snapshot results and non partitioned state handles - for (OperatorSnapshotFutures operatorSnapshotResult : operatorSnapshotsInProgress.values()) { - if (operatorSnapshotResult != null) { - try { - operatorSnapshotResult.cancel(); - } catch (Exception cancelException) { - exception = ExceptionUtils.firstOrSuppressed(cancelException, exception); - } - } - } - - if (null != exception) { - throw exception; - } - } - - private void logFailedCleanupAttempt() { - LOG.debug("{} - asynchronous checkpointing operation for checkpoint {} has " + - "already been completed. Thus, the state handles are not cleaned up.", - owner.getName(), - checkpointMetaData.getCheckpointId()); - } - } - public CloseableRegistry getCancelables() { return cancelables; } // ------------------------------------------------------------------------ - private static final class CheckpointingOperation { - - private final StreamTask owner; - - private final CheckpointMetaData checkpointMetaData; - private final CheckpointOptions checkpointOptions; - private final CheckpointMetrics checkpointMetrics; - private final CheckpointStreamFactory storageLocation; - - private final OperatorChain operatorChain; - - private long startSyncPartNano; - private long startAsyncPartNano; - - // ------------------------ - - private final Map operatorSnapshotsInProgress; - - public CheckpointingOperation( - StreamTask owner, - CheckpointMetaData checkpointMetaData, - CheckpointOptions checkpointOptions, - CheckpointStreamFactory checkpointStorageLocation, - CheckpointMetrics checkpointMetrics) { - - this.owner = Preconditions.checkNotNull(owner); - this.checkpointMetaData = Preconditions.checkNotNull(checkpointMetaData); - this.checkpointOptions = Preconditions.checkNotNull(checkpointOptions); - this.checkpointMetrics = Preconditions.checkNotNull(checkpointMetrics); - this.storageLocation = Preconditions.checkNotNull(checkpointStorageLocation); - this.operatorChain = owner.operatorChain; - this.operatorSnapshotsInProgress = new HashMap<>(operatorChain.getNumberOfOperators()); - } - - public void executeCheckpointing() throws Exception { - startSyncPartNano = System.nanoTime(); - - try { - for (StreamOperatorWrapper operatorWrapper : operatorChain.getAllOperators(true)) { - checkpointStreamOperator(operatorWrapper.getStreamOperator()); - } - - if (LOG.isDebugEnabled()) { - LOG.debug("Finished synchronous checkpoints for checkpoint {} on task {}", - checkpointMetaData.getCheckpointId(), owner.getName()); - } - - startAsyncPartNano = System.nanoTime(); - - checkpointMetrics.setSyncDurationMillis((startAsyncPartNano - startSyncPartNano) / 1_000_000); - - // we are transferring ownership over snapshotInProgressList for cleanup to the thread, active on submit - AsyncCheckpointRunnable asyncCheckpointRunnable = new AsyncCheckpointRunnable( - owner, - operatorSnapshotsInProgress, - checkpointMetaData, - checkpointMetrics, - startAsyncPartNano); - - owner.cancelables.registerCloseable(asyncCheckpointRunnable); - owner.asyncOperationsThreadPool.execute(asyncCheckpointRunnable); - - if (LOG.isDebugEnabled()) { - LOG.debug("{} - finished synchronous part of checkpoint {}. " + - "Alignment duration: {} ms, snapshot duration {} ms", - owner.getName(), checkpointMetaData.getCheckpointId(), - checkpointMetrics.getAlignmentDurationNanos() / 1_000_000, - checkpointMetrics.getSyncDurationMillis()); - } - } catch (Exception ex) { - // Cleanup to release resources - for (OperatorSnapshotFutures operatorSnapshotResult : operatorSnapshotsInProgress.values()) { - if (null != operatorSnapshotResult) { - try { - operatorSnapshotResult.cancel(); - } catch (Exception e) { - LOG.warn("Could not properly cancel an operator snapshot result.", e); - } - } - } - - if (LOG.isDebugEnabled()) { - LOG.debug("{} - did NOT finish synchronous part of checkpoint {}. " + - "Alignment duration: {} ms, snapshot duration {} ms", - owner.getName(), checkpointMetaData.getCheckpointId(), - checkpointMetrics.getAlignmentDurationNanos() / 1_000_000, - checkpointMetrics.getSyncDurationMillis()); - } - - if (checkpointOptions.getCheckpointType().isSynchronous()) { - // in the case of a synchronous checkpoint, we always rethrow the exception, - // so that the task fails. - // this is because the intention is always to stop the job after this checkpointing - // operation, and without the failure, the task would go back to normal execution. - throw ex; - } else { - owner.getEnvironment().declineCheckpoint(checkpointMetaData.getCheckpointId(), ex); - } - } - } - - @SuppressWarnings("deprecation") - private void checkpointStreamOperator(StreamOperator op) throws Exception { - OperatorSnapshotFutures snapshotInProgress = op.snapshotState( - checkpointMetaData.getCheckpointId(), - checkpointMetaData.getTimestamp(), - checkpointOptions, - storageLocation); - operatorSnapshotsInProgress.put(op.getOperatorID(), snapshotInProgress); - } - - private enum AsyncCheckpointState { - RUNNING, - DISCARDED, - COMPLETED - } - } - @VisibleForTesting public static RecordWriterDelegate>> createRecordWriterDelegate( StreamConfig configuration, diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/LocalStateForwardingTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/LocalStateForwardingTest.java index bc864a2e5d..bd0e49e38c 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/LocalStateForwardingTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/LocalStateForwardingTest.java @@ -105,13 +105,15 @@ public class LocalStateForwardingTest extends TestLogger { OperatorID operatorID = new OperatorID(); snapshots.put(operatorID, osFuture); - StreamTask.AsyncCheckpointRunnable checkpointRunnable = - new StreamTask.AsyncCheckpointRunnable( - testStreamTask, - snapshots, - checkpointMetaData, - checkpointMetrics, - 0L); + AsyncCheckpointRunnable checkpointRunnable = new AsyncCheckpointRunnable( + snapshots, + checkpointMetaData, + checkpointMetrics, + 0L, + testStreamTask.getName(), + testStreamTask.getCancelables(), + testStreamTask.getEnvironment(), + testStreamTask); checkpointRunnable.run(); -- Gitee From 0920e6447283579551d14a0fed726f8b8d73bd3f Mon Sep 17 00:00:00 2001 From: Rong Rong Date: Sun, 10 Feb 2019 15:05:10 -0800 Subject: [PATCH 162/885] [FLINK-11088] Allow YARN to discover pre-installed keytab files This is to change how keytab files are discovered, * remove hard-coded keytab filenames * extended YARNSessionFIFOITCase to accommodate different types of security setting test cases regenerate configuration documentations --- .../flink/yarn/YARNSessionFIFOITCase.java | 17 +++-- .../yarn/YARNSessionFIFOSecuredITCase.java | 62 ++++++++++++++++++- .../java/org/apache/flink/yarn/Utils.java | 22 ++++--- .../flink/yarn/YarnClusterDescriptor.java | 24 +++++-- .../org/apache/flink/yarn/YarnConfigKeys.java | 3 +- .../flink/yarn/YarnTaskExecutorRunner.java | 19 ++++-- .../yarn/configuration/YarnConfigOptions.java | 22 +++++++ .../yarn/entrypoint/YarnEntrypointUtils.java | 19 +++--- .../yarn/YarnTaskExecutorRunnerTest.java | 40 ++++++++++-- 9 files changed, 184 insertions(+), 44 deletions(-) diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java index 056182bb7c..fbff39da95 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java @@ -20,9 +20,7 @@ package org.apache.flink.yarn; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; -import org.apache.flink.configuration.SecurityOptions; import org.apache.flink.test.testdata.WordCountData; -import org.apache.flink.test.util.SecureTestEnvironment; import org.apache.flink.testutils.logging.TestLoggerResource; import org.apache.flink.yarn.cli.FlinkYarnSessionCli; import org.apache.flink.yarn.configuration.YarnConfigOptions; @@ -47,8 +45,10 @@ import org.slf4j.event.Level; import java.io.File; import java.util.ArrayList; +import java.util.Collections; import java.util.EnumSet; import java.util.List; +import java.util.Map; import java.util.concurrent.TimeUnit; import static org.apache.flink.yarn.util.YarnTestUtils.getTestJarPath; @@ -85,13 +85,13 @@ public class YARNSessionFIFOITCase extends YarnTestBase { @Test(timeout = 60000) public void testDetachedMode() throws Exception { - runTest(this::runDetachedModeTest); + runTest(() -> runDetachedModeTest(Collections.emptyMap())); } /** * Test regular operation, including command line parameter parsing. */ - void runDetachedModeTest() throws Exception { + void runDetachedModeTest(Map securityProperties) throws Exception { runTest(() -> { LOG.info("Starting testDetachedMode()"); @@ -116,11 +116,10 @@ public class YARNSessionFIFOITCase extends YarnTestBase { args.add("-tm"); args.add("1024m"); - if (SecureTestEnvironment.getTestKeytab() != null) { - args.add("-D" + SecurityOptions.KERBEROS_LOGIN_KEYTAB.key() + "=" + SecureTestEnvironment.getTestKeytab()); - } - if (SecureTestEnvironment.getHadoopServicePrincipal() != null) { - args.add("-D" + SecurityOptions.KERBEROS_LOGIN_PRINCIPAL.key() + "=" + SecureTestEnvironment.getHadoopServicePrincipal()); + if (securityProperties != null) { + for (Map.Entry property : securityProperties.entrySet()) { + args.add("-D" + property.getKey() + "=" + property.getValue()); + } } args.add("--name"); diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java index 4db588d4dc..a9c9aa3b46 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java @@ -24,6 +24,7 @@ import org.apache.flink.runtime.security.SecurityConfiguration; import org.apache.flink.runtime.security.SecurityUtils; import org.apache.flink.test.util.SecureTestEnvironment; import org.apache.flink.test.util.TestingSecurityContext; +import org.apache.flink.yarn.configuration.YarnConfigOptions; import org.apache.flink.yarn.util.TestHadoopModuleFactory; import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; @@ -41,7 +42,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.concurrent.Callable; /** @@ -68,9 +71,9 @@ public class YARNSessionFIFOSecuredITCase extends YARNSessionFIFOITCase { Configuration flinkConfig = new Configuration(); flinkConfig.setString(SecurityOptions.KERBEROS_LOGIN_KEYTAB, - SecureTestEnvironment.getTestKeytab()); + SecureTestEnvironment.getTestKeytab()); flinkConfig.setString(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL, - SecureTestEnvironment.getHadoopServicePrincipal()); + SecureTestEnvironment.getHadoopServicePrincipal()); // Setting customized security module class. TestHadoopModuleFactory.hadoopConfiguration = YARN_CONFIGURATION; @@ -108,7 +111,60 @@ public class YARNSessionFIFOSecuredITCase extends YARNSessionFIFOITCase { @Override public void testDetachedMode() throws Exception { runTest(() -> { - runDetachedModeTest(); + Map securityProperties = new HashMap<>(); + if (SecureTestEnvironment.getTestKeytab() != null) { + securityProperties.put(SecurityOptions.KERBEROS_LOGIN_KEYTAB.key(), SecureTestEnvironment.getTestKeytab()); + } + if (SecureTestEnvironment.getHadoopServicePrincipal() != null) { + securityProperties.put(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL.key(), SecureTestEnvironment.getHadoopServicePrincipal()); + } + runDetachedModeTest(securityProperties); + final String[] mustHave = {"Login successful for user", "using keytab file"}; + final boolean jobManagerRunsWithKerberos = verifyStringsInNamedLogFiles( + mustHave, + "jobmanager.log"); + final boolean taskManagerRunsWithKerberos = verifyStringsInNamedLogFiles( + mustHave, "taskmanager.log"); + + Assert.assertThat( + "The JobManager and the TaskManager should both run with Kerberos.", + jobManagerRunsWithKerberos && taskManagerRunsWithKerberos, + Matchers.is(true)); + + final List amRMTokens = Lists.newArrayList(AMRMTokenIdentifier.KIND_NAME.toString()); + final String jobmanagerContainerId = getContainerIdByLogName("jobmanager.log"); + final String taskmanagerContainerId = getContainerIdByLogName("taskmanager.log"); + final boolean jobmanagerWithAmRmToken = verifyTokenKindInContainerCredentials(amRMTokens, jobmanagerContainerId); + final boolean taskmanagerWithAmRmToken = verifyTokenKindInContainerCredentials(amRMTokens, taskmanagerContainerId); + + Assert.assertThat( + "The JobManager should have AMRMToken.", + jobmanagerWithAmRmToken, + Matchers.is(true)); + Assert.assertThat( + "The TaskManager should not have AMRMToken.", + taskmanagerWithAmRmToken, + Matchers.is(false)); + }); + } + + @Test(timeout = 60000) // timeout after a minute. + public void testDetachedModeSecureWithPreInstallKeytab() throws Exception { + runTest(() -> { + LOG.info("Starting testDetachedModeSecureWithPreInstallKeytab()"); + Map securityProperties = new HashMap<>(); + if (SecureTestEnvironment.getTestKeytab() != null) { + // client login keytab + securityProperties.put(SecurityOptions.KERBEROS_LOGIN_KEYTAB.key(), SecureTestEnvironment.getTestKeytab()); + // pre-install Yarn local keytab, since both reuse the same temporary folder "tmp" + securityProperties.put(YarnConfigOptions.LOCALIZED_KEYTAB_PATH.key(), SecureTestEnvironment.getTestKeytab()); + // unset keytab localization + securityProperties.put(YarnConfigOptions.SHIP_LOCAL_KEYTAB.key(), "false"); + } + if (SecureTestEnvironment.getHadoopServicePrincipal() != null) { + securityProperties.put(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL.key(), SecureTestEnvironment.getHadoopServicePrincipal()); + } + runDetachedModeTest(securityProperties); final String[] mustHave = {"Login successful for user", "using keytab file"}; final boolean jobManagerRunsWithKerberos = verifyStringsInNamedLogFiles( mustHave, diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java index 2a25cba689..66e99cf7e5 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java @@ -72,7 +72,7 @@ public final class Utils { private static final Logger LOG = LoggerFactory.getLogger(Utils.class); /** Keytab file name populated in YARN container. */ - public static final String KEYTAB_FILE_NAME = "krb5.keytab"; + public static final String DEFAULT_KEYTAB_FILE = "krb5.keytab"; /** KRB5 file name populated in YARN container for secure IT run. */ public static final String KRB5_FILE_NAME = "krb5.conf"; @@ -424,14 +424,16 @@ public final class Utils { String yarnClientUsername = env.get(YarnConfigKeys.ENV_HADOOP_USER_NAME); require(yarnClientUsername != null, "Environment variable %s not set", YarnConfigKeys.ENV_HADOOP_USER_NAME); - final String remoteKeytabPath = env.get(YarnConfigKeys.KEYTAB_PATH); - final String remoteKeytabPrincipal = env.get(YarnConfigKeys.KEYTAB_PRINCIPAL); + final String remoteKeytabPath = env.get(YarnConfigKeys.REMOTE_KEYTAB_PATH); + final String localKeytabPath = env.get(YarnConfigKeys.LOCAL_KEYTAB_PATH); + final String keytabPrincipal = env.get(YarnConfigKeys.KEYTAB_PRINCIPAL); final String remoteYarnConfPath = env.get(YarnConfigKeys.ENV_YARN_SITE_XML_PATH); final String remoteKrb5Path = env.get(YarnConfigKeys.ENV_KRB5_PATH); if (log.isDebugEnabled()) { log.debug("TM:remote keytab path obtained {}", remoteKeytabPath); - log.debug("TM:remote keytab principal obtained {}", remoteKeytabPrincipal); + log.debug("TM:local keytab path obtained {}", localKeytabPath); + log.debug("TM:keytab principal obtained {}", keytabPrincipal); log.debug("TM:remote yarn conf path obtained {}", remoteYarnConfPath); log.debug("TM:remote krb5 path obtained {}", remoteKrb5Path); } @@ -489,7 +491,7 @@ public final class Utils { taskManagerLocalResources.put(KRB5_FILE_NAME, krb5ConfResource); } if (keytabResource != null) { - taskManagerLocalResources.put(KEYTAB_FILE_NAME, keytabResource); + taskManagerLocalResources.put(localKeytabPath, keytabResource); } // prepare additional files to be shipped @@ -533,9 +535,13 @@ public final class Utils { containerEnv.put(YarnConfigKeys.ENV_HADOOP_USER_NAME, UserGroupInformation.getCurrentUser().getUserName()); - if (remoteKeytabPath != null && remoteKeytabPrincipal != null) { - containerEnv.put(YarnConfigKeys.KEYTAB_PATH, remoteKeytabPath); - containerEnv.put(YarnConfigKeys.KEYTAB_PRINCIPAL, remoteKeytabPrincipal); + if (remoteKeytabPath != null && localKeytabPath != null && keytabPrincipal != null) { + containerEnv.put(YarnConfigKeys.REMOTE_KEYTAB_PATH, remoteKeytabPath); + containerEnv.put(YarnConfigKeys.LOCAL_KEYTAB_PATH, localKeytabPath); + containerEnv.put(YarnConfigKeys.KEYTAB_PRINCIPAL, keytabPrincipal); + } else if (localKeytabPath != null && keytabPrincipal != null) { + containerEnv.put(YarnConfigKeys.LOCAL_KEYTAB_PATH, localKeytabPath); + containerEnv.put(YarnConfigKeys.KEYTAB_PRINCIPAL, keytabPrincipal); } ctx.setEnvironment(containerEnv); diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java index b0c7330b1c..f3c6f3c7fe 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java @@ -899,13 +899,17 @@ public class YarnClusterDescriptor implements ClusterDescriptor { } } - // setup security tokens Path remotePathKeytab = null; + String localizedKeytabPath = null; String keytab = configuration.getString(SecurityOptions.KERBEROS_LOGIN_KEYTAB); if (keytab != null) { - LOG.info("Adding keytab {} to the AM container local resource bucket", keytab); - remotePathKeytab = setupSingleLocalResource( - Utils.KEYTAB_FILE_NAME, + boolean localizeKeytab = flinkConfiguration.getBoolean(YarnConfigOptions.SHIP_LOCAL_KEYTAB); + localizedKeytabPath = flinkConfiguration.getString(YarnConfigOptions.LOCALIZED_KEYTAB_PATH); + if (localizeKeytab) { + // Localize the keytab to YARN containers via local resource. + LOG.info("Adding keytab {} to the AM container local resource bucket", keytab); + remotePathKeytab = setupSingleLocalResource( + localizedKeytabPath, fs, appId, new Path(keytab), @@ -913,6 +917,10 @@ public class YarnClusterDescriptor implements ClusterDescriptor { homeDir, "", fileReplication); + } else { + // // Assume Keytab is pre-installed in the container. + localizedKeytabPath = flinkConfiguration.getString(YarnConfigOptions.LOCALIZED_KEYTAB_PATH); + } } final boolean hasLogback = logConfigFilePath != null && logConfigFilePath.endsWith(CONFIG_FILE_LOGBACK_NAME); @@ -925,6 +933,7 @@ public class YarnClusterDescriptor implements ClusterDescriptor { hasKrb5, clusterSpecification.getMasterMemoryMB()); + // setup security tokens if (UserGroupInformation.isSecurityEnabled()) { // set HDFS delegation tokens when security is enabled LOG.info("Adding delegation token to the AM container."); @@ -953,10 +962,13 @@ public class YarnClusterDescriptor implements ClusterDescriptor { // https://github.com/apache/hadoop/blob/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnApplicationSecurity.md#identity-on-an-insecure-cluster-hadoop_user_name appMasterEnv.put(YarnConfigKeys.ENV_HADOOP_USER_NAME, UserGroupInformation.getCurrentUser().getUserName()); - if (remotePathKeytab != null) { - appMasterEnv.put(YarnConfigKeys.KEYTAB_PATH, remotePathKeytab.toString()); + if (localizedKeytabPath != null) { + appMasterEnv.put(YarnConfigKeys.LOCAL_KEYTAB_PATH, localizedKeytabPath); String principal = configuration.getString(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL); appMasterEnv.put(YarnConfigKeys.KEYTAB_PRINCIPAL, principal); + if (remotePathKeytab != null) { + appMasterEnv.put(YarnConfigKeys.REMOTE_KEYTAB_PATH, remotePathKeytab.toString()); + } } //To support Yarn Secure Integration Test Scenario diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnConfigKeys.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnConfigKeys.java index c73ec3e74a..477ff27991 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnConfigKeys.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnConfigKeys.java @@ -36,7 +36,8 @@ public class YarnConfigKeys { public static final String FLINK_JAR_PATH = "_FLINK_JAR_PATH"; // the Flink jar resource location (in HDFS). public static final String FLINK_YARN_FILES = "_FLINK_YARN_FILES"; // the root directory for all yarn application files - public static final String KEYTAB_PATH = "_KEYTAB_PATH"; + public static final String REMOTE_KEYTAB_PATH = "_REMOTE_KEYTAB_PATH"; + public static final String LOCAL_KEYTAB_PATH = "_LOCAL_KEYTAB_PATH"; public static final String KEYTAB_PRINCIPAL = "_KEYTAB_PRINCIPAL"; public static final String ENV_HADOOP_USER_NAME = "HADOOP_USER_NAME"; public static final String ENV_ZOOKEEPER_NAMESPACE = "_ZOOKEEPER_NAMESPACE"; diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskExecutorRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskExecutorRunner.java index 5a05232cc2..b806829576 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskExecutorRunner.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskExecutorRunner.java @@ -129,18 +129,25 @@ public class YarnTaskExecutorRunner { private static void setupConfigurationFromVariables(Configuration configuration, String currDir, Map variables) throws IOException { final String yarnClientUsername = variables.get(YarnConfigKeys.ENV_HADOOP_USER_NAME); - final String remoteKeytabPath = variables.get(YarnConfigKeys.KEYTAB_PATH); + final String remoteKeytabPath = variables.get(YarnConfigKeys.REMOTE_KEYTAB_PATH); LOG.info("TM: remote keytab path obtained {}", remoteKeytabPath); - final String remoteKeytabPrincipal = variables.get(YarnConfigKeys.KEYTAB_PRINCIPAL); - LOG.info("TM: remote keytab principal obtained {}", remoteKeytabPrincipal); + final String localKeytabPath = variables.get(YarnConfigKeys.LOCAL_KEYTAB_PATH); + LOG.info("TM: local keytab path obtained {}", localKeytabPath); + + final String keytabPrincipal = variables.get(YarnConfigKeys.KEYTAB_PRINCIPAL); + LOG.info("TM: keytab principal obtained {}", keytabPrincipal); // tell akka to die in case of an error configuration.setBoolean(AkkaOptions.JVM_EXIT_ON_FATAL_ERROR, true); String keytabPath = null; if (remoteKeytabPath != null) { - File f = new File(currDir, Utils.KEYTAB_FILE_NAME); + File f = new File(currDir, localKeytabPath); + keytabPath = f.getAbsolutePath(); + LOG.info("keytab path: {}", keytabPath); + } else if (localKeytabPath != null) { + File f = new File(localKeytabPath); keytabPath = f.getAbsolutePath(); LOG.info("keytab path: {}", keytabPath); } @@ -150,9 +157,9 @@ public class YarnTaskExecutorRunner { LOG.info("YARN daemon is running as: {} Yarn client user obtainer: {}", currentUser.getShortUserName(), yarnClientUsername); - if (keytabPath != null && remoteKeytabPrincipal != null) { + if (keytabPath != null && keytabPrincipal != null) { configuration.setString(SecurityOptions.KERBEROS_LOGIN_KEYTAB, keytabPath); - configuration.setString(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL, remoteKeytabPrincipal); + configuration.setString(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL, keytabPrincipal); } // use the hostname passed by job manager diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptions.java b/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptions.java index fa2e00f30d..4b26efb01a 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptions.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptions.java @@ -19,6 +19,7 @@ package org.apache.flink.yarn.configuration; import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.SecurityOptions; import org.apache.flink.configuration.description.Description; import java.util.List; @@ -238,6 +239,27 @@ public class YarnConfigOptions { .noDefaultValue() .withDescription("Specify YARN node label for the YARN application."); + public static final ConfigOption SHIP_LOCAL_KEYTAB = + key("yarn.security.kerberos.ship-local-keytab") + .booleanType() + .defaultValue(true) + .withDescription( + "When this is true Flink will ship the keytab file configured via " + + SecurityOptions.KERBEROS_LOGIN_KEYTAB.key() + + " as a localized YARN resource."); + + public static final ConfigOption LOCALIZED_KEYTAB_PATH = + key("yarn.security.kerberos.localized-keytab-path") + .stringType() + .defaultValue("krb5.keytab") + .withDescription( + "Local (on NodeManager) path where kerberos keytab file will be" + + " localized to. If " + SHIP_LOCAL_KEYTAB.key() + " set to " + + "true, Flink willl ship the keytab file as a YARN local " + + "resource. In this case, the path is relative to the local " + + "resource directory. If set to false, Flink" + + " will try to directly locate the keytab from the path itself."); + // ------------------------------------------------------------------------ /** This class is not meant to be instantiated. */ diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java index b7ffca5c6e..19af8925f7 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java @@ -29,7 +29,6 @@ import org.apache.flink.configuration.SecurityOptions; import org.apache.flink.configuration.WebOptions; import org.apache.flink.runtime.clusterframework.BootstrapTools; import org.apache.flink.util.Preconditions; -import org.apache.flink.yarn.Utils; import org.apache.flink.yarn.YarnConfigKeys; import org.apache.hadoop.security.UserGroupInformation; @@ -49,7 +48,7 @@ public class YarnEntrypointUtils { public static Configuration loadConfiguration(String workingDirectory, Map env) { Configuration configuration = GlobalConfiguration.loadConfiguration(workingDirectory); - final String remoteKeytabPrincipal = env.get(YarnConfigKeys.KEYTAB_PRINCIPAL); + final String keytabPrincipal = env.get(YarnConfigKeys.KEYTAB_PRINCIPAL); final String zooKeeperNamespace = env.get(YarnConfigKeys.ENV_ZOOKEEPER_NAMESPACE); @@ -90,16 +89,22 @@ public class YarnEntrypointUtils { final String keytabPath; - if (env.get(YarnConfigKeys.KEYTAB_PATH) == null) { + if (env.get(YarnConfigKeys.LOCAL_KEYTAB_PATH) == null) { // keytab not exist keytabPath = null; } else { - File f = new File(workingDirectory, Utils.KEYTAB_FILE_NAME); - keytabPath = f.getAbsolutePath(); + File f; + f = new File(env.get(YarnConfigKeys.LOCAL_KEYTAB_PATH)); + if (f.exists()) { // keytab file exist in host environment. + keytabPath = f.getAbsolutePath(); + } else { + f = new File(workingDirectory, env.get(YarnConfigKeys.LOCAL_KEYTAB_PATH)); + keytabPath = f.getAbsolutePath(); + } } - if (keytabPath != null && remoteKeytabPrincipal != null) { + if (keytabPath != null && keytabPrincipal != null) { configuration.setString(SecurityOptions.KERBEROS_LOGIN_KEYTAB, keytabPath); - configuration.setString(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL, remoteKeytabPrincipal); + configuration.setString(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL, keytabPrincipal); } final String localDirs = env.get(ApplicationConstants.Environment.LOCAL_DIRS.key()); diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnTaskExecutorRunnerTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnTaskExecutorRunnerTest.java index 18726ad515..3f4565ade9 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnTaskExecutorRunnerTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnTaskExecutorRunnerTest.java @@ -24,6 +24,7 @@ import org.apache.flink.runtime.security.SecurityUtils; import org.apache.flink.runtime.security.modules.HadoopModule; import org.apache.flink.runtime.security.modules.SecurityModule; import org.apache.flink.util.TestLogger; +import org.apache.flink.yarn.configuration.YarnConfigOptions; import org.junit.Test; @@ -34,6 +35,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; @@ -44,12 +46,14 @@ import static org.junit.Assert.fail; public class YarnTaskExecutorRunnerTest extends TestLogger { @Test - public void testKerberosKeytabConfiguration() throws Exception { + public void testDefaultKerberosKeytabConfiguration() throws Exception { final String resourceDirPath = Paths.get("src", "test", "resources").toAbsolutePath().toString(); final Map envs = new HashMap<>(2); envs.put(YarnConfigKeys.KEYTAB_PRINCIPAL, "testuser1@domain"); - envs.put(YarnConfigKeys.KEYTAB_PATH, resourceDirPath); + envs.put(YarnConfigKeys.REMOTE_KEYTAB_PATH, resourceDirPath); + // Local keytab path will be populated from default YarnConfigOptions.LOCALIZED_KEYTAB_PATH + envs.put(YarnConfigKeys.LOCAL_KEYTAB_PATH, YarnConfigOptions.LOCALIZED_KEYTAB_PATH.defaultValue()); Configuration configuration = new Configuration(); YarnTaskExecutorRunner.setupConfigurationAndInstallSecurityContext(configuration, resourceDirPath, envs); @@ -60,13 +64,41 @@ public class YarnTaskExecutorRunnerTest extends TestLogger { if (moduleOpt.isPresent()) { HadoopModule hadoopModule = (HadoopModule) moduleOpt.get(); assertThat(hadoopModule.getSecurityConfig().getPrincipal(), is("testuser1@domain")); - assertThat(hadoopModule.getSecurityConfig().getKeytab(), is(new File(resourceDirPath, Utils.KEYTAB_FILE_NAME).getAbsolutePath())); + assertThat(hadoopModule.getSecurityConfig().getKeytab(), is(new File(resourceDirPath, YarnConfigOptions.LOCALIZED_KEYTAB_PATH.defaultValue()).getAbsolutePath())); } else { fail("Can not find HadoopModule!"); } - assertThat(configuration.getString(SecurityOptions.KERBEROS_LOGIN_KEYTAB), is(new File(resourceDirPath, Utils.KEYTAB_FILE_NAME).getAbsolutePath())); + assertThat(configuration.getString(SecurityOptions.KERBEROS_LOGIN_KEYTAB), is(new File(resourceDirPath, YarnConfigOptions.LOCALIZED_KEYTAB_PATH.defaultValue()).getAbsolutePath())); assertThat(configuration.getString(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL), is("testuser1@domain")); } + @Test + public void testPreInstallKerberosKeytabConfiguration() throws Exception { + final String resourceDirPath = Paths.get("src", "test", "resources").toAbsolutePath().toString(); + + final Map envs = new HashMap<>(2); + envs.put(YarnConfigKeys.KEYTAB_PRINCIPAL, "testuser1@domain"); + // Try directly resolving local path when no remote keytab path is provided. + envs.put(YarnConfigKeys.LOCAL_KEYTAB_PATH, "src/test/resources/krb5.keytab"); + + Configuration configuration = new Configuration(); + YarnTaskExecutorRunner.setupConfigurationAndInstallSecurityContext(configuration, resourceDirPath, envs); + + final List modules = SecurityUtils.getInstalledModules(); + Optional moduleOpt = modules.stream().filter(module -> module instanceof HadoopModule).findFirst(); + + if (moduleOpt.isPresent()) { + HadoopModule hadoopModule = (HadoopModule) moduleOpt.get(); + assertThat(hadoopModule.getSecurityConfig().getPrincipal(), is("testuser1@domain")); + // Using containString verification as the absolute path varies depending on runtime environment + assertThat(hadoopModule.getSecurityConfig().getKeytab(), containsString("src/test/resources/krb5.keytab")); + } else { + fail("Can not find HadoopModule!"); + } + + assertThat(configuration.getString(SecurityOptions.KERBEROS_LOGIN_KEYTAB), containsString("src/test/resources/krb5.keytab")); + assertThat(configuration.getString(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL), is("testuser1@domain")); + + } } -- Gitee From ea8f0e3ecc9fa9f4bcb150d04b62b94a6ea43757 Mon Sep 17 00:00:00 2001 From: Rong Rong Date: Wed, 11 Mar 2020 10:04:33 -0700 Subject: [PATCH 163/885] [FLINK-11088] Refactor keytab resolution to Utils --- .../security/SecurityConfiguration.java | 8 +- .../yarn/YARNSessionFIFOSecuredITCase.java | 100 +++++++----------- .../java/org/apache/flink/yarn/Utils.java | 32 +++++- .../flink/yarn/YarnTaskExecutorRunner.java | 15 +-- .../yarn/entrypoint/YarnEntrypointUtils.java | 17 +-- 5 files changed, 78 insertions(+), 94 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityConfiguration.java index 66c3a8b213..279300cbfe 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityConfiguration.java @@ -135,13 +135,15 @@ public class SecurityConfiguration { if (!StringUtils.isBlank(keytab)) { // principal is required if (StringUtils.isBlank(principal)) { - throw new IllegalConfigurationException("Kerberos login configuration is invalid; keytab requires a principal."); + throw new IllegalConfigurationException("Kerberos login configuration is invalid: keytab requires a principal."); } // check the keytab is readable File keytabFile = new File(keytab); - if (!keytabFile.exists() || !keytabFile.isFile() || !keytabFile.canRead()) { - throw new IllegalConfigurationException("Kerberos login configuration is invalid; keytab is unreadable"); + if (!keytabFile.exists() || !keytabFile.isFile()) { + throw new IllegalConfigurationException("Kerberos login configuration is invalid: keytab [" + keytab + "] doesn't exist!"); + } else if (!keytabFile.canRead()) { + throw new IllegalConfigurationException("Kerberos login configuration is invalid: keytab [" + keytab + "] is unreadable!"); } } } diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java index a9c9aa3b46..b210c4429e 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java @@ -108,92 +108,70 @@ public class YARNSessionFIFOSecuredITCase extends YARNSessionFIFOITCase { } @Test(timeout = 60000) // timeout after a minute. - @Override - public void testDetachedMode() throws Exception { + public void testDetachedModeSecureWithPreInstallKeytab() throws Exception { runTest(() -> { Map securityProperties = new HashMap<>(); if (SecureTestEnvironment.getTestKeytab() != null) { + // client login keytab securityProperties.put(SecurityOptions.KERBEROS_LOGIN_KEYTAB.key(), SecureTestEnvironment.getTestKeytab()); + // pre-install Yarn local keytab, since both reuse the same temporary folder "tmp" + securityProperties.put(YarnConfigOptions.LOCALIZED_KEYTAB_PATH.key(), SecureTestEnvironment.getTestKeytab()); + // unset keytab localization + securityProperties.put(YarnConfigOptions.SHIP_LOCAL_KEYTAB.key(), "false"); } if (SecureTestEnvironment.getHadoopServicePrincipal() != null) { securityProperties.put(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL.key(), SecureTestEnvironment.getHadoopServicePrincipal()); } runDetachedModeTest(securityProperties); - final String[] mustHave = {"Login successful for user", "using keytab file"}; - final boolean jobManagerRunsWithKerberos = verifyStringsInNamedLogFiles( - mustHave, - "jobmanager.log"); - final boolean taskManagerRunsWithKerberos = verifyStringsInNamedLogFiles( - mustHave, "taskmanager.log"); - - Assert.assertThat( - "The JobManager and the TaskManager should both run with Kerberos.", - jobManagerRunsWithKerberos && taskManagerRunsWithKerberos, - Matchers.is(true)); - - final List amRMTokens = Lists.newArrayList(AMRMTokenIdentifier.KIND_NAME.toString()); - final String jobmanagerContainerId = getContainerIdByLogName("jobmanager.log"); - final String taskmanagerContainerId = getContainerIdByLogName("taskmanager.log"); - final boolean jobmanagerWithAmRmToken = verifyTokenKindInContainerCredentials(amRMTokens, jobmanagerContainerId); - final boolean taskmanagerWithAmRmToken = verifyTokenKindInContainerCredentials(amRMTokens, taskmanagerContainerId); - - Assert.assertThat( - "The JobManager should have AMRMToken.", - jobmanagerWithAmRmToken, - Matchers.is(true)); - Assert.assertThat( - "The TaskManager should not have AMRMToken.", - taskmanagerWithAmRmToken, - Matchers.is(false)); + verifyResultContainsKerberosKeytab(); }); } @Test(timeout = 60000) // timeout after a minute. - public void testDetachedModeSecureWithPreInstallKeytab() throws Exception { + @Override + public void testDetachedMode() throws Exception { runTest(() -> { - LOG.info("Starting testDetachedModeSecureWithPreInstallKeytab()"); Map securityProperties = new HashMap<>(); if (SecureTestEnvironment.getTestKeytab() != null) { - // client login keytab securityProperties.put(SecurityOptions.KERBEROS_LOGIN_KEYTAB.key(), SecureTestEnvironment.getTestKeytab()); - // pre-install Yarn local keytab, since both reuse the same temporary folder "tmp" - securityProperties.put(YarnConfigOptions.LOCALIZED_KEYTAB_PATH.key(), SecureTestEnvironment.getTestKeytab()); - // unset keytab localization - securityProperties.put(YarnConfigOptions.SHIP_LOCAL_KEYTAB.key(), "false"); } if (SecureTestEnvironment.getHadoopServicePrincipal() != null) { securityProperties.put(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL.key(), SecureTestEnvironment.getHadoopServicePrincipal()); } runDetachedModeTest(securityProperties); - final String[] mustHave = {"Login successful for user", "using keytab file"}; - final boolean jobManagerRunsWithKerberos = verifyStringsInNamedLogFiles( - mustHave, - "jobmanager.log"); - final boolean taskManagerRunsWithKerberos = verifyStringsInNamedLogFiles( - mustHave, "taskmanager.log"); - - Assert.assertThat( - "The JobManager and the TaskManager should both run with Kerberos.", - jobManagerRunsWithKerberos && taskManagerRunsWithKerberos, - Matchers.is(true)); - - final List amRMTokens = Lists.newArrayList(AMRMTokenIdentifier.KIND_NAME.toString()); - final String jobmanagerContainerId = getContainerIdByLogName("jobmanager.log"); - final String taskmanagerContainerId = getContainerIdByLogName("taskmanager.log"); - final boolean jobmanagerWithAmRmToken = verifyTokenKindInContainerCredentials(amRMTokens, jobmanagerContainerId); - final boolean taskmanagerWithAmRmToken = verifyTokenKindInContainerCredentials(amRMTokens, taskmanagerContainerId); - - Assert.assertThat( - "The JobManager should have AMRMToken.", - jobmanagerWithAmRmToken, - Matchers.is(true)); - Assert.assertThat( - "The TaskManager should not have AMRMToken.", - taskmanagerWithAmRmToken, - Matchers.is(false)); + verifyResultContainsKerberosKeytab(); }); } + private static void verifyResultContainsKerberosKeytab() throws Exception { + final String[] mustHave = {"Login successful for user", "using keytab file"}; + final boolean jobManagerRunsWithKerberos = verifyStringsInNamedLogFiles( + mustHave, + "jobmanager.log"); + final boolean taskManagerRunsWithKerberos = verifyStringsInNamedLogFiles( + mustHave, "taskmanager.log"); + + Assert.assertThat( + "The JobManager and the TaskManager should both run with Kerberos.", + jobManagerRunsWithKerberos && taskManagerRunsWithKerberos, + Matchers.is(true)); + + final List amRMTokens = Lists.newArrayList(AMRMTokenIdentifier.KIND_NAME.toString()); + final String jobmanagerContainerId = getContainerIdByLogName("jobmanager.log"); + final String taskmanagerContainerId = getContainerIdByLogName("taskmanager.log"); + final boolean jobmanagerWithAmRmToken = verifyTokenKindInContainerCredentials(amRMTokens, jobmanagerContainerId); + final boolean taskmanagerWithAmRmToken = verifyTokenKindInContainerCredentials(amRMTokens, taskmanagerContainerId); + + Assert.assertThat( + "The JobManager should have AMRMToken.", + jobmanagerWithAmRmToken, + Matchers.is(true)); + Assert.assertThat( + "The TaskManager should not have AMRMToken.", + taskmanagerWithAmRmToken, + Matchers.is(false)); + } + /* For secure cluster testing, it is enough to run only one test and override below test methods * to keep the overall build time minimal */ diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java index 66e99cf7e5..7c8b2bfc99 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java @@ -359,6 +359,37 @@ public final class Utils { StringInterner.weakIntern(val)); } + /** + * Resolve keytab path either as absolute path or relative to working directory. + * + * @param workingDir current working directory + * @param keytabPath configured keytab path. + * @return resolved keytab path, or null if not found. + */ + public static String resolveKeytabPath(String workingDir, String keytabPath) { + String keytab = null; + if (keytabPath != null) { + File f; + f = new File(keytabPath); + if (f.exists()) { + keytab = f.getAbsolutePath(); + LOG.info("Resolved keytab path: {}", keytab); + } else { + // try using relative paths, this is the case when the keytab was shipped + // as a local resource + f = new File(workingDir, keytabPath); + if (f.exists()) { + keytab = f.getAbsolutePath(); + LOG.info("Resolved keytab path: {}", keytab); + } else { + LOG.warn("Could not resolve keytab path with: {}", keytabPath); + keytab = null; + } + } + } + return keytab; + } + /** * Private constructor to prevent instantiation. */ @@ -600,5 +631,4 @@ public final class Utils { throw new RuntimeException(String.format(message, values)); } } - } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskExecutorRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskExecutorRunner.java index b806829576..bf28974420 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskExecutorRunner.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskExecutorRunner.java @@ -41,7 +41,6 @@ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.File; import java.io.IOException; import java.lang.reflect.UndeclaredThrowableException; import java.util.Map; @@ -129,9 +128,6 @@ public class YarnTaskExecutorRunner { private static void setupConfigurationFromVariables(Configuration configuration, String currDir, Map variables) throws IOException { final String yarnClientUsername = variables.get(YarnConfigKeys.ENV_HADOOP_USER_NAME); - final String remoteKeytabPath = variables.get(YarnConfigKeys.REMOTE_KEYTAB_PATH); - LOG.info("TM: remote keytab path obtained {}", remoteKeytabPath); - final String localKeytabPath = variables.get(YarnConfigKeys.LOCAL_KEYTAB_PATH); LOG.info("TM: local keytab path obtained {}", localKeytabPath); @@ -141,16 +137,7 @@ public class YarnTaskExecutorRunner { // tell akka to die in case of an error configuration.setBoolean(AkkaOptions.JVM_EXIT_ON_FATAL_ERROR, true); - String keytabPath = null; - if (remoteKeytabPath != null) { - File f = new File(currDir, localKeytabPath); - keytabPath = f.getAbsolutePath(); - LOG.info("keytab path: {}", keytabPath); - } else if (localKeytabPath != null) { - File f = new File(localKeytabPath); - keytabPath = f.getAbsolutePath(); - LOG.info("keytab path: {}", keytabPath); - } + String keytabPath = Utils.resolveKeytabPath(currDir, localKeytabPath); UserGroupInformation currentUser = UserGroupInformation.getCurrentUser(); diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java index 19af8925f7..3da17bb5ff 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java @@ -29,13 +29,13 @@ import org.apache.flink.configuration.SecurityOptions; import org.apache.flink.configuration.WebOptions; import org.apache.flink.runtime.clusterframework.BootstrapTools; import org.apache.flink.util.Preconditions; +import org.apache.flink.yarn.Utils; import org.apache.flink.yarn.YarnConfigKeys; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.api.ApplicationConstants; import org.slf4j.Logger; -import java.io.File; import java.io.IOException; import java.util.Map; @@ -87,20 +87,7 @@ public class YarnEntrypointUtils { ConfigConstants.YARN_TASK_MANAGER_ENV_PREFIX, ResourceManagerOptions.CONTAINERIZED_TASK_MANAGER_ENV_PREFIX); - final String keytabPath; - - if (env.get(YarnConfigKeys.LOCAL_KEYTAB_PATH) == null) { // keytab not exist - keytabPath = null; - } else { - File f; - f = new File(env.get(YarnConfigKeys.LOCAL_KEYTAB_PATH)); - if (f.exists()) { // keytab file exist in host environment. - keytabPath = f.getAbsolutePath(); - } else { - f = new File(workingDirectory, env.get(YarnConfigKeys.LOCAL_KEYTAB_PATH)); - keytabPath = f.getAbsolutePath(); - } - } + final String keytabPath = Utils.resolveKeytabPath(workingDirectory, env.get(YarnConfigKeys.LOCAL_KEYTAB_PATH)); if (keytabPath != null && keytabPrincipal != null) { configuration.setString(SecurityOptions.KERBEROS_LOGIN_KEYTAB, keytabPath); -- Gitee From 7f86f7409ae4b8ea99478296df589054bfb31ecb Mon Sep 17 00:00:00 2001 From: Rong Rong Date: Wed, 11 Mar 2020 10:04:33 -0700 Subject: [PATCH 164/885] [FLINK-11088] Regenerate YARN/kerberos config docs --- .../generated/yarn_config_configuration.html | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/docs/_includes/generated/yarn_config_configuration.html b/docs/_includes/generated/yarn_config_configuration.html index 3c32b856a0..bdc83bba22 100644 --- a/docs/_includes/generated/yarn_config_configuration.html +++ b/docs/_includes/generated/yarn_config_configuration.html @@ -110,6 +110,18 @@

    String When a Flink job is submitted to YARN, the JobManager’s host and the number of available processing slots is written into a properties file, so that the Flink client is able to pick those details up. This configuration parameter allows changing the default location of that file (for example for environments sharing a Flink installation between users).
    yarn.security.kerberos.localized-keytab-path
    "krb5.keytab"StringLocal (on NodeManager) path where kerberos keytab file will be localized to. If yarn.security.kerberos.ship-local-keytab set to true, Flink willl ship the keytab file as a YARN local resource. In this case, the path is relative to the local resource directory. If set to false, Flink will try to directly locate the keytab from the path itself.
    yarn.security.kerberos.ship-local-keytab
    trueBooleanWhen this is true Flink will ship the keytab file configured via security.kerberos.login.keytab as a localized YARN resource.
    yarn.ship-directories
    (none)
    table.exec.shuffle-mode

    Batch
    "batch" StringSets exec shuffle mode. Only batch or pipeline can be set. + Sets exec shuffle mode. Only batch or pipelined can be set. batch: the job will run stage by stage. -pipeline: the job will run in streaming mode, but it may cause resource deadlock that receiver waits for resource to start when the sender holds resource to wait to send data to the receiver.
    table.exec.sort.async-merge-enabled

    Batch
    @@ -345,9 +345,9 @@ catalogs: -### Changing the Current Catalog And Database +### 修改当前的 Catalog 和数据库 -Flink will always search for tables, views, and UDF's in the current catalog and database. +Flink 始终在当前的 Catalog 和数据库中寻找表、视图和 UDF。
    @@ -364,7 +364,7 @@ Flink SQL> USE myDB;
    -Metadata from catalogs that are not the current catalog are accessible by providing fully qualified names in the form `catalog.database.object`. +通过提供全限定名 `catalog.database.object` 来访问不在当前 Catalog 中的元数据信息。
    @@ -379,7 +379,7 @@ Flink SQL> SELECT * FROM not_the_current_catalog.not_the_current_db.my_table;
    -### List Available Catalogs +### 列出可用的 Catalog
    @@ -395,7 +395,7 @@ Flink SQL> show catalogs;
    -### List Available Databases +### 列出可用的数据库
    @@ -410,7 +410,7 @@ Flink SQL> show databases;
    -### List Available Tables +### 列出可用的表
    -- Gitee From 4035f03a1de24b7e292466e9fa0a0de9cb10f358 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Mon, 16 Mar 2020 13:31:51 +0800 Subject: [PATCH 174/885] [FLINK-16454][build] Update the copyright with 2020 year in NOTICE files --- NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- .../flink-connector-hive/src/main/resources/META-INF/NOTICE | 4 ++-- .../src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- flink-dist/src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- .../flink-azure-fs-hadoop/src/main/resources/META-INF/NOTICE | 2 +- .../flink-fs-hadoop-shaded/src/main/resources/META-INF/NOTICE | 2 +- .../flink-oss-fs-hadoop/src/main/resources/META-INF/NOTICE | 2 +- .../flink-s3-fs-hadoop/src/main/resources/META-INF/NOTICE | 2 +- .../flink-s3-fs-presto/src/main/resources/META-INF/NOTICE | 2 +- .../flink-swift-fs-hadoop/src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- flink-mesos/src/main/resources/META-INF/NOTICE | 2 +- .../flink-metrics-datadog/src/main/resources/META-INF/NOTICE | 2 +- .../flink-metrics-graphite/src/main/resources/META-INF/NOTICE | 2 +- .../flink-metrics-influxdb/src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- .../flink-ml-uber/src/main/resources/META-INF/NOTICE | 2 +- flink-python/src/main/resources/META-INF/NOTICE | 2 +- flink-runtime-web/src/main/resources/META-INF/NOTICE | 2 +- flink-runtime/src/main/resources/META-INF/NOTICE | 2 +- .../flink-sql-client/src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- .../flink-table-planner/src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- .../flink-test-utils/src/main/resources/META-INF/NOTICE | 2 +- tools/releasing/NOTICE-binary_PREAMBLE.txt | 2 +- 39 files changed, 40 insertions(+), 40 deletions(-) diff --git a/NOTICE b/NOTICE index 35f4f6e897..707f8d92f3 100644 --- a/NOTICE +++ b/NOTICE @@ -1,5 +1,5 @@ Apache Flink -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-connectors/flink-connector-cassandra/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-connector-cassandra/src/main/resources/META-INF/NOTICE index 5d23402d1e..726ff6f5cc 100644 --- a/flink-connectors/flink-connector-cassandra/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-connector-cassandra/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-connector-cassandra -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/NOTICE index c30871cf6f..2e7f042548 100644 --- a/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-connector-elasticsearch5/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-connector-elasticsearch5 -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-connectors/flink-connector-hive/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-connector-hive/src/main/resources/META-INF/NOTICE index e6eedee597..a1d75604d4 100644 --- a/flink-connectors/flink-connector-hive/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-connector-hive/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-connector-hive -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). @@ -10,4 +10,4 @@ This project bundles the following dependencies under the Apache Software Licens - org.apache.parquet:parquet-format:1.10.0 - org.apache.parquet:parquet-column:1.10.0 - org.apache.parquet:parquet-common:1.10.0 -- org.apache.parquet:parquet-encoding:1.10.0 \ No newline at end of file +- org.apache.parquet:parquet-encoding:1.10.0 diff --git a/flink-connectors/flink-connector-kinesis/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-connector-kinesis/src/main/resources/META-INF/NOTICE index e3af608ef4..daffb7eb38 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-connector-kinesis/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-connector-kinesis -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-connectors/flink-connector-twitter/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-connector-twitter/src/main/resources/META-INF/NOTICE index 40b0687b87..50e1e58208 100644 --- a/flink-connectors/flink-connector-twitter/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-connector-twitter/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-connector-twitter -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE index 43c60595c9..542a1f6cc3 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-sql-connector-elasticsearch6 -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE index 0a27b7bc3b..9f68ec03da 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-sql-connector-elasticsearch7 -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-connectors/flink-sql-connector-hive-1.2.2/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-hive-1.2.2/src/main/resources/META-INF/NOTICE index ae2657902d..a884504525 100644 --- a/flink-connectors/flink-sql-connector-hive-1.2.2/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-hive-1.2.2/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-sql-connector-hive-1.2.2 -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-connectors/flink-sql-connector-hive-2.2.0/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-hive-2.2.0/src/main/resources/META-INF/NOTICE index d7664ccaf1..24634360a1 100644 --- a/flink-connectors/flink-sql-connector-hive-2.2.0/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-hive-2.2.0/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-sql-connector-hive-2.2.0 -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-connectors/flink-sql-connector-hive-2.3.6/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-hive-2.3.6/src/main/resources/META-INF/NOTICE index a2b038f00b..0692589da5 100644 --- a/flink-connectors/flink-sql-connector-hive-2.3.6/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-hive-2.3.6/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-sql-connector-hive-2.3.6 -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-connectors/flink-sql-connector-hive-3.1.2/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-hive-3.1.2/src/main/resources/META-INF/NOTICE index 71df3aae2b..430761c146 100644 --- a/flink-connectors/flink-sql-connector-hive-3.1.2/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-hive-3.1.2/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-sql-connector-hive-3.1.2 -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-connectors/flink-sql-connector-kafka-0.10/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-kafka-0.10/src/main/resources/META-INF/NOTICE index 1aa05251ae..fefdb3e980 100644 --- a/flink-connectors/flink-sql-connector-kafka-0.10/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-kafka-0.10/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-sql-connector-kafka-0.10 -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-connectors/flink-sql-connector-kafka-0.11/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-kafka-0.11/src/main/resources/META-INF/NOTICE index 69516d36fc..25e8b22713 100644 --- a/flink-connectors/flink-sql-connector-kafka-0.11/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-kafka-0.11/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-sql-connector-kafka-0.11 -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-connectors/flink-sql-connector-kafka/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-kafka/src/main/resources/META-INF/NOTICE index b99bbc44f4..cd7ec2d952 100644 --- a/flink-connectors/flink-sql-connector-kafka/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-kafka/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-sql-connector-kafka -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-dist/src/main/resources/META-INF/NOTICE b/flink-dist/src/main/resources/META-INF/NOTICE index 2c9da733a0..0eca371987 100644 --- a/flink-dist/src/main/resources/META-INF/NOTICE +++ b/flink-dist/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-dist -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-examples/flink-examples-build-helper/flink-examples-streaming-state-machine/src/main/resources/META-INF/NOTICE b/flink-examples/flink-examples-build-helper/flink-examples-streaming-state-machine/src/main/resources/META-INF/NOTICE index 9161b8b1a9..d5005995b7 100644 --- a/flink-examples/flink-examples-build-helper/flink-examples-streaming-state-machine/src/main/resources/META-INF/NOTICE +++ b/flink-examples/flink-examples-build-helper/flink-examples-streaming-state-machine/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-examples-streaming-state-machine -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-filesystems/flink-azure-fs-hadoop/src/main/resources/META-INF/NOTICE b/flink-filesystems/flink-azure-fs-hadoop/src/main/resources/META-INF/NOTICE index 8fe7725e83..d65b624a0e 100644 --- a/flink-filesystems/flink-azure-fs-hadoop/src/main/resources/META-INF/NOTICE +++ b/flink-filesystems/flink-azure-fs-hadoop/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-azure-fs-hadoop -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This project includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-filesystems/flink-fs-hadoop-shaded/src/main/resources/META-INF/NOTICE b/flink-filesystems/flink-fs-hadoop-shaded/src/main/resources/META-INF/NOTICE index 3633e55b45..5513bf58fb 100644 --- a/flink-filesystems/flink-fs-hadoop-shaded/src/main/resources/META-INF/NOTICE +++ b/flink-filesystems/flink-fs-hadoop-shaded/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-fs-hadoop-shaded -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This project includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/META-INF/NOTICE b/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/META-INF/NOTICE index b4ecba50eb..3aea254a8b 100644 --- a/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/META-INF/NOTICE +++ b/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-oss-fs-hadoop -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This project includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-filesystems/flink-s3-fs-hadoop/src/main/resources/META-INF/NOTICE b/flink-filesystems/flink-s3-fs-hadoop/src/main/resources/META-INF/NOTICE index 8705a6a9d7..26a1bc54b7 100644 --- a/flink-filesystems/flink-s3-fs-hadoop/src/main/resources/META-INF/NOTICE +++ b/flink-filesystems/flink-s3-fs-hadoop/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-s3-fs-hadoop -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) diff --git a/flink-filesystems/flink-s3-fs-presto/src/main/resources/META-INF/NOTICE b/flink-filesystems/flink-s3-fs-presto/src/main/resources/META-INF/NOTICE index 6a64e1ab4b..cc80ddac06 100644 --- a/flink-filesystems/flink-s3-fs-presto/src/main/resources/META-INF/NOTICE +++ b/flink-filesystems/flink-s3-fs-presto/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-s3-fs-presto -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This project includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-filesystems/flink-swift-fs-hadoop/src/main/resources/META-INF/NOTICE b/flink-filesystems/flink-swift-fs-hadoop/src/main/resources/META-INF/NOTICE index 4343aafbb2..050c6d439b 100644 --- a/flink-filesystems/flink-swift-fs-hadoop/src/main/resources/META-INF/NOTICE +++ b/flink-filesystems/flink-swift-fs-hadoop/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-swift-fs-hadoop -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This project includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-formats/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE index 26541c32ee..9417323f42 100644 --- a/flink-formats/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE +++ b/flink-formats/flink-avro-confluent-registry/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-avro-confluent-registry -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-mesos/src/main/resources/META-INF/NOTICE b/flink-mesos/src/main/resources/META-INF/NOTICE index 98c836ad9a..0e12ee2491 100644 --- a/flink-mesos/src/main/resources/META-INF/NOTICE +++ b/flink-mesos/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-mesos -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-metrics/flink-metrics-datadog/src/main/resources/META-INF/NOTICE b/flink-metrics/flink-metrics-datadog/src/main/resources/META-INF/NOTICE index 949e9aa384..1b4409b82f 100644 --- a/flink-metrics/flink-metrics-datadog/src/main/resources/META-INF/NOTICE +++ b/flink-metrics/flink-metrics-datadog/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-metrics-datadog -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-metrics/flink-metrics-graphite/src/main/resources/META-INF/NOTICE b/flink-metrics/flink-metrics-graphite/src/main/resources/META-INF/NOTICE index 6c225e1965..c88e9e3504 100644 --- a/flink-metrics/flink-metrics-graphite/src/main/resources/META-INF/NOTICE +++ b/flink-metrics/flink-metrics-graphite/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-metrics-graphite -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-metrics/flink-metrics-influxdb/src/main/resources/META-INF/NOTICE b/flink-metrics/flink-metrics-influxdb/src/main/resources/META-INF/NOTICE index 6f923d88f6..ad79869c5b 100644 --- a/flink-metrics/flink-metrics-influxdb/src/main/resources/META-INF/NOTICE +++ b/flink-metrics/flink-metrics-influxdb/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-metrics-influxdb -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-metrics/flink-metrics-prometheus/src/main/resources/META-INF/NOTICE b/flink-metrics/flink-metrics-prometheus/src/main/resources/META-INF/NOTICE index 33354b276d..92cdf5a7a6 100644 --- a/flink-metrics/flink-metrics-prometheus/src/main/resources/META-INF/NOTICE +++ b/flink-metrics/flink-metrics-prometheus/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-metrics-prometheus -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-ml-parent/flink-ml-uber/src/main/resources/META-INF/NOTICE b/flink-ml-parent/flink-ml-uber/src/main/resources/META-INF/NOTICE index db087abe66..656a326b24 100644 --- a/flink-ml-parent/flink-ml-uber/src/main/resources/META-INF/NOTICE +++ b/flink-ml-parent/flink-ml-uber/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-ml-lib -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-python/src/main/resources/META-INF/NOTICE b/flink-python/src/main/resources/META-INF/NOTICE index a8c4a364d8..d45e29693b 100644 --- a/flink-python/src/main/resources/META-INF/NOTICE +++ b/flink-python/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-python -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-runtime-web/src/main/resources/META-INF/NOTICE b/flink-runtime-web/src/main/resources/META-INF/NOTICE index 35d45ed76b..81fc4113c8 100644 --- a/flink-runtime-web/src/main/resources/META-INF/NOTICE +++ b/flink-runtime-web/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-runtime-web -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-runtime/src/main/resources/META-INF/NOTICE b/flink-runtime/src/main/resources/META-INF/NOTICE index 82ee6f71cf..de76bed680 100644 --- a/flink-runtime/src/main/resources/META-INF/NOTICE +++ b/flink-runtime/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-runtime -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-table/flink-sql-client/src/main/resources/META-INF/NOTICE b/flink-table/flink-sql-client/src/main/resources/META-INF/NOTICE index 17c8d7579e..0f583197a3 100644 --- a/flink-table/flink-sql-client/src/main/resources/META-INF/NOTICE +++ b/flink-table/flink-sql-client/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-sql-client -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-table/flink-table-planner-blink/src/main/resources/META-INF/NOTICE b/flink-table/flink-table-planner-blink/src/main/resources/META-INF/NOTICE index a7c22a41df..9770943b43 100644 --- a/flink-table/flink-table-planner-blink/src/main/resources/META-INF/NOTICE +++ b/flink-table/flink-table-planner-blink/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-table-planner-blink -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-table/flink-table-planner/src/main/resources/META-INF/NOTICE b/flink-table/flink-table-planner/src/main/resources/META-INF/NOTICE index 1a015be628..1227b0bc0f 100644 --- a/flink-table/flink-table-planner/src/main/resources/META-INF/NOTICE +++ b/flink-table/flink-table-planner/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-table-planner -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-table/flink-table-runtime-blink/src/main/resources/META-INF/NOTICE b/flink-table/flink-table-runtime-blink/src/main/resources/META-INF/NOTICE index e7fc1dc410..3aac7ecdee 100644 --- a/flink-table/flink-table-runtime-blink/src/main/resources/META-INF/NOTICE +++ b/flink-table/flink-table-runtime-blink/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-table-runtime-blink -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-test-utils-parent/flink-test-utils/src/main/resources/META-INF/NOTICE b/flink-test-utils-parent/flink-test-utils/src/main/resources/META-INF/NOTICE index 6393e25f65..86474a1986 100644 --- a/flink-test-utils-parent/flink-test-utils/src/main/resources/META-INF/NOTICE +++ b/flink-test-utils-parent/flink-test-utils/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-test-utils -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/tools/releasing/NOTICE-binary_PREAMBLE.txt b/tools/releasing/NOTICE-binary_PREAMBLE.txt index 278e89e2e6..236d22d1d3 100644 --- a/tools/releasing/NOTICE-binary_PREAMBLE.txt +++ b/tools/releasing/NOTICE-binary_PREAMBLE.txt @@ -4,7 +4,7 @@ // ------------------------------------------------------------------ Apache Flink -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) -- Gitee From 1063d56154caddc6d116605fb6590ec06ab6e5d1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E5=86=9B?= <25563794+zhangjun888@users.noreply.github.com> Date: Tue, 17 Mar 2020 10:47:18 +0800 Subject: [PATCH 175/885] [FLINK-16413][hive] Reduce hive source parallelism when limit push down This closes #11405 --- .../connectors/hive/HiveTableSource.java | 7 +++- .../connectors/hive/HiveTableSourceTest.java | 33 +++++++++++++++++++ 2 files changed, 39 insertions(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java index 49d0c9a8e0..3addbe33ba 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java @@ -27,6 +27,7 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.ObjectPath; import org.apache.flink.table.catalog.hive.client.HiveMetastoreClientFactory; @@ -148,6 +149,7 @@ public class HiveTableSource implements DataStreamSource source = execEnv.createInput(inputFormat, typeInfo); + int parallelism = flinkConf.get(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM); if (flinkConf.get(HiveOptions.TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM)) { int max = flinkConf.get(HiveOptions.TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM_MAX); if (max < 1) { @@ -168,8 +170,11 @@ public class HiveTableSource implements } catch (IOException e) { throw new FlinkHiveException(e); } - source.setParallelism(Math.min(Math.max(1, splitNum), max)); + parallelism = Math.min(splitNum, max); } + parallelism = limit > 0 ? Math.min(parallelism, (int) limit / 1000) : parallelism; + parallelism = Math.max(1, parallelism); + source.setParallelism(parallelism); return source.name(explainSource()); } diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSourceTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSourceTest.java index 372d2a543b..93f21f041f 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSourceTest.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSourceTest.java @@ -24,6 +24,8 @@ import org.apache.flink.connectors.hive.read.HiveTableInputFormat; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.transformations.OneInputTransformation; +import org.apache.flink.streaming.api.transformations.PartitionTransformation; import org.apache.flink.table.HiveVersionTestUtil; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.TableEnvironment; @@ -410,6 +412,37 @@ public class HiveTableSourceTest { Assert.assertEquals(2, transformation.getParallelism()); } + @Test + public void testParallelismOnLimitPushDown() { + final String catalogName = "hive"; + final String dbName = "source_db"; + final String tblName = "test_parallelism_limit_pushdown"; + hiveShell.execute("CREATE TABLE source_db.test_parallelism_limit_pushdown " + + "(year STRING, value INT) partitioned by (pt int);"); + HiveTestUtils.createTextTableInserter(hiveShell, dbName, tblName) + .addRow(new Object[]{"2014", 3}) + .addRow(new Object[]{"2014", 4}) + .commit("pt=0"); + HiveTestUtils.createTextTableInserter(hiveShell, dbName, tblName) + .addRow(new Object[]{"2015", 2}) + .addRow(new Object[]{"2015", 5}) + .commit("pt=1"); + TableEnvironment tEnv = HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(); + tEnv.getConfig().getConfiguration().setBoolean( + HiveOptions.TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM, false); + tEnv.getConfig().getConfiguration().setInteger( + ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2); + tEnv.registerCatalog(catalogName, hiveCatalog); + Table table = tEnv.sqlQuery("select * from hive.source_db.test_parallelism_limit_pushdown limit 1"); + PlannerBase planner = (PlannerBase) ((TableEnvironmentImpl) tEnv).getPlanner(); + RelNode relNode = planner.optimize(TableTestUtil.toRelNode(table)); + ExecNode execNode = planner.translateToExecNodePlan(toScala(Collections.singletonList(relNode))).get(0); + @SuppressWarnings("unchecked") + Transformation transformation = execNode.translateToPlan(planner); + Assert.assertEquals(1, ((PartitionTransformation) ((OneInputTransformation) transformation).getInput()) + .getInput().getParallelism()); + } + @Test public void testSourceConfig() throws Exception { // vector reader not available for 1.x and we're not testing orc for 2.0.x -- Gitee From 9242daf12ae858db9cab9ec72bdbeaa61e37a948 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E5=86=9B?= <25563794+zhangjun888@users.noreply.github.com> Date: Tue, 17 Mar 2020 14:13:00 +0800 Subject: [PATCH 176/885] [FLINK-16623][sql-client] Add the shorthand 'desc' for describe in sql client This closes #11422 --- .../java/org/apache/flink/table/client/cli/CliClient.java | 1 + .../org/apache/flink/table/client/cli/SqlCommandParser.java | 4 ++++ .../apache/flink/table/client/cli/SqlCommandParserTest.java | 3 +++ 3 files changed, 8 insertions(+) diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliClient.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliClient.java index a028c9c681..3521de685b 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliClient.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliClient.java @@ -285,6 +285,7 @@ public class CliClient { case USE: callUseDatabase(cmdCall); break; + case DESC: case DESCRIBE: callDescribe(cmdCall); break; diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/SqlCommandParser.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/SqlCommandParser.java index 03e2f639bc..4877fa6718 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/SqlCommandParser.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/SqlCommandParser.java @@ -115,6 +115,10 @@ public final class SqlCommandParser { "(CREATE\\s+CATALOG\\s+.*)", SINGLE_OPERAND), + DESC( + "DESC\\s+(.*)", + SINGLE_OPERAND), + DESCRIBE( "DESCRIBE\\s+(.*)", SINGLE_OPERAND), diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/SqlCommandParserTest.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/SqlCommandParserTest.java index 2509c3cdfd..522c3aa788 100644 --- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/SqlCommandParserTest.java +++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/SqlCommandParserTest.java @@ -42,6 +42,9 @@ public class SqlCommandParserTest { testValidSqlCommand(" SHOW TABLES ", new SqlCommandCall(SqlCommand.SHOW_TABLES)); testValidSqlCommand("SHOW FUNCTIONS", new SqlCommandCall(SqlCommand.SHOW_FUNCTIONS)); testValidSqlCommand(" SHOW FUNCTIONS ", new SqlCommandCall(SqlCommand.SHOW_FUNCTIONS)); + testValidSqlCommand("DESC MyTable", new SqlCommandCall(SqlCommand.DESC, new String[]{"MyTable"})); + testValidSqlCommand("DESC MyTable ", new SqlCommandCall(SqlCommand.DESC, new String[]{"MyTable"})); + testInvalidSqlCommand("DESC "); // no table name testValidSqlCommand("DESCRIBE MyTable", new SqlCommandCall(SqlCommand.DESCRIBE, new String[]{"MyTable"})); testValidSqlCommand("DESCRIBE MyTable ", new SqlCommandCall(SqlCommand.DESCRIBE, new String[]{"MyTable"})); testInvalidSqlCommand("DESCRIBE "); // no table name -- Gitee From 428c77870a6df94f9e2fb32819f8afba932c7216 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Sat, 14 Mar 2020 18:39:50 +0800 Subject: [PATCH 177/885] [FLINK-16607][python] Update flink-fn-execution.proto adding more schema information --- flink-python/pyflink/fn_execution/coders.py | 12 +- .../fn_execution/flink_fn_execution_pb2.py | 470 +++++++++++++++--- .../pyflink/proto/flink-fn-execution.proto | 51 +- .../runtime/typeutils/PythonTypeUtils.java | 31 +- 4 files changed, 482 insertions(+), 82 deletions(-) diff --git a/flink-python/pyflink/fn_execution/coders.py b/flink-python/pyflink/fn_execution/coders.py index e6a8ac5908..eb47cf70b0 100644 --- a/flink-python/pyflink/fn_execution/coders.py +++ b/flink-python/pyflink/fn_execution/coders.py @@ -464,15 +464,15 @@ def from_proto(field_type): return coder if field_type_name == type_name.ROW: return RowCoder([from_proto(f.type) for f in field_type.row_schema.fields]) - if field_type_name == type_name.DATETIME: - return TimestampCoder(field_type.date_time_type.precision) + if field_type_name == type_name.TIMESTAMP: + return TimestampCoder(field_type.timestamp_info.precision) elif field_type_name == type_name.ARRAY: return ArrayCoder(from_proto(field_type.collection_element_type)) elif field_type_name == type_name.MAP: - return MapCoder(from_proto(field_type.map_type.key_type), - from_proto(field_type.map_type.value_type)) + return MapCoder(from_proto(field_type.map_info.key_type), + from_proto(field_type.map_info.value_type)) elif field_type_name == type_name.DECIMAL: - return DecimalCoder(field_type.decimal_type.precision, - field_type.decimal_type.scale) + return DecimalCoder(field_type.decimal_info.precision, + field_type.decimal_info.scale) else: raise ValueError("field_type %s is not supported." % field_type) diff --git a/flink-python/pyflink/fn_execution/flink_fn_execution_pb2.py b/flink-python/pyflink/fn_execution/flink_fn_execution_pb2.py index 3ca7fa5a46..18d4475001 100644 --- a/flink-python/pyflink/fn_execution/flink_fn_execution_pb2.py +++ b/flink-python/pyflink/fn_execution/flink_fn_execution_pb2.py @@ -36,7 +36,7 @@ DESCRIPTOR = _descriptor.FileDescriptor( name='flink-fn-execution.proto', package='org.apache.flink.fn_execution.v1', syntax='proto3', - serialized_pb=_b('\n\x18\x66link-fn-execution.proto\x12 org.apache.flink.fn_execution.v1\"\xfc\x01\n\x13UserDefinedFunction\x12\x0f\n\x07payload\x18\x01 \x01(\x0c\x12K\n\x06inputs\x18\x02 \x03(\x0b\x32;.org.apache.flink.fn_execution.v1.UserDefinedFunction.Input\x1a\x86\x01\n\x05Input\x12\x44\n\x03udf\x18\x01 \x01(\x0b\x32\x35.org.apache.flink.fn_execution.v1.UserDefinedFunctionH\x00\x12\x15\n\x0binputOffset\x18\x02 \x01(\x05H\x00\x12\x17\n\rinputConstant\x18\x03 \x01(\x0cH\x00\x42\x07\n\x05input\"[\n\x14UserDefinedFunctions\x12\x43\n\x04udfs\x18\x01 \x03(\x0b\x32\x35.org.apache.flink.fn_execution.v1.UserDefinedFunction\"\x80\t\n\x06Schema\x12>\n\x06\x66ields\x18\x01 \x03(\x0b\x32..org.apache.flink.fn_execution.v1.Schema.Field\x1a\x97\x01\n\x07MapType\x12\x44\n\x08key_type\x18\x01 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x12\x46\n\nvalue_type\x18\x02 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x1a!\n\x0c\x44\x61teTimeType\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a/\n\x0b\x44\x65\x63imalType\x12\x11\n\tprecision\x18\x01 \x01(\x05\x12\r\n\x05scale\x18\x02 \x01(\x05\x1a\xec\x03\n\tFieldType\x12\x44\n\ttype_name\x18\x01 \x01(\x0e\x32\x31.org.apache.flink.fn_execution.v1.Schema.TypeName\x12\x10\n\x08nullable\x18\x02 \x01(\x08\x12U\n\x17\x63ollection_element_type\x18\x03 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldTypeH\x00\x12\x44\n\x08map_type\x18\x04 \x01(\x0b\x32\x30.org.apache.flink.fn_execution.v1.Schema.MapTypeH\x00\x12>\n\nrow_schema\x18\x05 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.SchemaH\x00\x12O\n\x0e\x64\x61te_time_type\x18\x06 \x01(\x0b\x32\x35.org.apache.flink.fn_execution.v1.Schema.DateTimeTypeH\x00\x12L\n\x0c\x64\x65\x63imal_type\x18\x07 \x01(\x0b\x32\x34.org.apache.flink.fn_execution.v1.Schema.DecimalTypeH\x00\x42\x0b\n\ttype_info\x1al\n\x05\x46ield\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x13\n\x0b\x64\x65scription\x18\x02 \x01(\t\x12@\n\x04type\x18\x03 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\"\xea\x01\n\x08TypeName\x12\x07\n\x03ROW\x10\x00\x12\x0b\n\x07TINYINT\x10\x01\x12\x0c\n\x08SMALLINT\x10\x02\x12\x07\n\x03INT\x10\x03\x12\n\n\x06\x42IGINT\x10\x04\x12\x0b\n\x07\x44\x45\x43IMAL\x10\x05\x12\t\n\x05\x46LOAT\x10\x06\x12\n\n\x06\x44OUBLE\x10\x07\x12\x08\n\x04\x44\x41TE\x10\x08\x12\x08\n\x04TIME\x10\t\x12\x0c\n\x08\x44\x41TETIME\x10\n\x12\x0b\n\x07\x42OOLEAN\x10\x0b\x12\n\n\x06\x42INARY\x10\x0c\x12\r\n\tVARBINARY\x10\r\x12\x08\n\x04\x43HAR\x10\x0e\x12\x0b\n\x07VARCHAR\x10\x0f\x12\t\n\x05\x41RRAY\x10\x10\x12\x07\n\x03MAP\x10\x11\x12\x0c\n\x08MULTISET\x10\x12\x42-\n\x1forg.apache.flink.fnexecution.v1B\nFlinkFnApib\x06proto3') + serialized_pb=_b('\n\x18\x66link-fn-execution.proto\x12 org.apache.flink.fn_execution.v1\"\xfc\x01\n\x13UserDefinedFunction\x12\x0f\n\x07payload\x18\x01 \x01(\x0c\x12K\n\x06inputs\x18\x02 \x03(\x0b\x32;.org.apache.flink.fn_execution.v1.UserDefinedFunction.Input\x1a\x86\x01\n\x05Input\x12\x44\n\x03udf\x18\x01 \x01(\x0b\x32\x35.org.apache.flink.fn_execution.v1.UserDefinedFunctionH\x00\x12\x15\n\x0binputOffset\x18\x02 \x01(\x05H\x00\x12\x17\n\rinputConstant\x18\x03 \x01(\x0cH\x00\x42\x07\n\x05input\"[\n\x14UserDefinedFunctions\x12\x43\n\x04udfs\x18\x01 \x03(\x0b\x32\x35.org.apache.flink.fn_execution.v1.UserDefinedFunction\"\xe6\x0f\n\x06Schema\x12>\n\x06\x66ields\x18\x01 \x03(\x0b\x32..org.apache.flink.fn_execution.v1.Schema.Field\x1a\x97\x01\n\x07MapInfo\x12\x44\n\x08key_type\x18\x01 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x12\x46\n\nvalue_type\x18\x02 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x1a\x1d\n\x08TimeInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a\"\n\rTimestampInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a,\n\x17LocalZonedTimestampInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a\'\n\x12ZonedTimestampInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a/\n\x0b\x44\x65\x63imalInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x12\r\n\x05scale\x18\x02 \x01(\x05\x1a\x1c\n\nBinaryInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\x1f\n\rVarBinaryInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\x1a\n\x08\x43harInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\x1d\n\x0bVarCharInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\xb0\x08\n\tFieldType\x12\x44\n\ttype_name\x18\x01 \x01(\x0e\x32\x31.org.apache.flink.fn_execution.v1.Schema.TypeName\x12\x10\n\x08nullable\x18\x02 \x01(\x08\x12U\n\x17\x63ollection_element_type\x18\x03 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldTypeH\x00\x12\x44\n\x08map_info\x18\x04 \x01(\x0b\x32\x30.org.apache.flink.fn_execution.v1.Schema.MapInfoH\x00\x12>\n\nrow_schema\x18\x05 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.SchemaH\x00\x12L\n\x0c\x64\x65\x63imal_info\x18\x06 \x01(\x0b\x32\x34.org.apache.flink.fn_execution.v1.Schema.DecimalInfoH\x00\x12\x46\n\ttime_info\x18\x07 \x01(\x0b\x32\x31.org.apache.flink.fn_execution.v1.Schema.TimeInfoH\x00\x12P\n\x0etimestamp_info\x18\x08 \x01(\x0b\x32\x36.org.apache.flink.fn_execution.v1.Schema.TimestampInfoH\x00\x12\x66\n\x1alocal_zoned_timestamp_info\x18\t \x01(\x0b\x32@.org.apache.flink.fn_execution.v1.Schema.LocalZonedTimestampInfoH\x00\x12[\n\x14zoned_timestamp_info\x18\n \x01(\x0b\x32;.org.apache.flink.fn_execution.v1.Schema.ZonedTimestampInfoH\x00\x12J\n\x0b\x62inary_info\x18\x0b \x01(\x0b\x32\x33.org.apache.flink.fn_execution.v1.Schema.BinaryInfoH\x00\x12Q\n\x0fvar_binary_info\x18\x0c \x01(\x0b\x32\x36.org.apache.flink.fn_execution.v1.Schema.VarBinaryInfoH\x00\x12\x46\n\tchar_info\x18\r \x01(\x0b\x32\x31.org.apache.flink.fn_execution.v1.Schema.CharInfoH\x00\x12M\n\rvar_char_info\x18\x0e \x01(\x0b\x32\x34.org.apache.flink.fn_execution.v1.Schema.VarCharInfoH\x00\x42\x0b\n\ttype_info\x1al\n\x05\x46ield\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x13\n\x0b\x64\x65scription\x18\x02 \x01(\t\x12@\n\x04type\x18\x03 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\"\x9b\x02\n\x08TypeName\x12\x07\n\x03ROW\x10\x00\x12\x0b\n\x07TINYINT\x10\x01\x12\x0c\n\x08SMALLINT\x10\x02\x12\x07\n\x03INT\x10\x03\x12\n\n\x06\x42IGINT\x10\x04\x12\x0b\n\x07\x44\x45\x43IMAL\x10\x05\x12\t\n\x05\x46LOAT\x10\x06\x12\n\n\x06\x44OUBLE\x10\x07\x12\x08\n\x04\x44\x41TE\x10\x08\x12\x08\n\x04TIME\x10\t\x12\r\n\tTIMESTAMP\x10\n\x12\x0b\n\x07\x42OOLEAN\x10\x0b\x12\n\n\x06\x42INARY\x10\x0c\x12\r\n\tVARBINARY\x10\r\x12\x08\n\x04\x43HAR\x10\x0e\x12\x0b\n\x07VARCHAR\x10\x0f\x12\t\n\x05\x41RRAY\x10\x10\x12\x07\n\x03MAP\x10\x11\x12\x0c\n\x08MULTISET\x10\x12\x12\x19\n\x15LOCAL_ZONED_TIMESTAMP\x10\x13\x12\x13\n\x0fZONED_TIMESTAMP\x10\x14\x42-\n\x1forg.apache.flink.fnexecution.v1B\nFlinkFnApib\x06proto3') ) @@ -88,7 +88,7 @@ _SCHEMA_TYPENAME = _descriptor.EnumDescriptor( options=None, type=None), _descriptor.EnumValueDescriptor( - name='DATETIME', index=10, number=10, + name='TIMESTAMP', index=10, number=10, options=None, type=None), _descriptor.EnumValueDescriptor( @@ -123,11 +123,19 @@ _SCHEMA_TYPENAME = _descriptor.EnumDescriptor( name='MULTISET', index=18, number=18, options=None, type=None), + _descriptor.EnumValueDescriptor( + name='LOCAL_ZONED_TIMESTAMP', index=19, number=19, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='ZONED_TIMESTAMP', index=20, number=20, + options=None, + type=None), ], containing_type=None, options=None, - serialized_start=1329, - serialized_end=1563, + serialized_start=2150, + serialized_end=2433, ) _sym_db.RegisterEnumDescriptor(_SCHEMA_TYPENAME) @@ -248,22 +256,22 @@ _USERDEFINEDFUNCTIONS = _descriptor.Descriptor( ) -_SCHEMA_MAPTYPE = _descriptor.Descriptor( - name='MapType', - full_name='org.apache.flink.fn_execution.v1.Schema.MapType', +_SCHEMA_MAPINFO = _descriptor.Descriptor( + name='MapInfo', + full_name='org.apache.flink.fn_execution.v1.Schema.MapInfo', filename=None, file=DESCRIPTOR, containing_type=None, fields=[ _descriptor.FieldDescriptor( - name='key_type', full_name='org.apache.flink.fn_execution.v1.Schema.MapType.key_type', index=0, + name='key_type', full_name='org.apache.flink.fn_execution.v1.Schema.MapInfo.key_type', index=0, number=1, type=11, cpp_type=10, label=1, has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( - name='value_type', full_name='org.apache.flink.fn_execution.v1.Schema.MapType.value_type', index=1, + name='value_type', full_name='org.apache.flink.fn_execution.v1.Schema.MapInfo.value_type', index=1, number=2, type=11, cpp_type=10, label=1, has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, @@ -285,15 +293,15 @@ _SCHEMA_MAPTYPE = _descriptor.Descriptor( serialized_end=637, ) -_SCHEMA_DATETIMETYPE = _descriptor.Descriptor( - name='DateTimeType', - full_name='org.apache.flink.fn_execution.v1.Schema.DateTimeType', +_SCHEMA_TIMEINFO = _descriptor.Descriptor( + name='TimeInfo', + full_name='org.apache.flink.fn_execution.v1.Schema.TimeInfo', filename=None, file=DESCRIPTOR, containing_type=None, fields=[ _descriptor.FieldDescriptor( - name='precision', full_name='org.apache.flink.fn_execution.v1.Schema.DateTimeType.precision', index=0, + name='precision', full_name='org.apache.flink.fn_execution.v1.Schema.TimeInfo.precision', index=0, number=1, type=5, cpp_type=1, label=1, has_default_value=False, default_value=0, message_type=None, enum_type=None, containing_type=None, @@ -312,25 +320,115 @@ _SCHEMA_DATETIMETYPE = _descriptor.Descriptor( oneofs=[ ], serialized_start=639, - serialized_end=672, + serialized_end=668, +) + +_SCHEMA_TIMESTAMPINFO = _descriptor.Descriptor( + name='TimestampInfo', + full_name='org.apache.flink.fn_execution.v1.Schema.TimestampInfo', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='precision', full_name='org.apache.flink.fn_execution.v1.Schema.TimestampInfo.precision', index=0, + number=1, type=5, cpp_type=1, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=670, + serialized_end=704, +) + +_SCHEMA_LOCALZONEDTIMESTAMPINFO = _descriptor.Descriptor( + name='LocalZonedTimestampInfo', + full_name='org.apache.flink.fn_execution.v1.Schema.LocalZonedTimestampInfo', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='precision', full_name='org.apache.flink.fn_execution.v1.Schema.LocalZonedTimestampInfo.precision', index=0, + number=1, type=5, cpp_type=1, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=706, + serialized_end=750, +) + +_SCHEMA_ZONEDTIMESTAMPINFO = _descriptor.Descriptor( + name='ZonedTimestampInfo', + full_name='org.apache.flink.fn_execution.v1.Schema.ZonedTimestampInfo', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='precision', full_name='org.apache.flink.fn_execution.v1.Schema.ZonedTimestampInfo.precision', index=0, + number=1, type=5, cpp_type=1, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=752, + serialized_end=791, ) -_SCHEMA_DECIMALTYPE = _descriptor.Descriptor( - name='DecimalType', - full_name='org.apache.flink.fn_execution.v1.Schema.DecimalType', +_SCHEMA_DECIMALINFO = _descriptor.Descriptor( + name='DecimalInfo', + full_name='org.apache.flink.fn_execution.v1.Schema.DecimalInfo', filename=None, file=DESCRIPTOR, containing_type=None, fields=[ _descriptor.FieldDescriptor( - name='precision', full_name='org.apache.flink.fn_execution.v1.Schema.DecimalType.precision', index=0, + name='precision', full_name='org.apache.flink.fn_execution.v1.Schema.DecimalInfo.precision', index=0, number=1, type=5, cpp_type=1, label=1, has_default_value=False, default_value=0, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( - name='scale', full_name='org.apache.flink.fn_execution.v1.Schema.DecimalType.scale', index=1, + name='scale', full_name='org.apache.flink.fn_execution.v1.Schema.DecimalInfo.scale', index=1, number=2, type=5, cpp_type=1, label=1, has_default_value=False, default_value=0, message_type=None, enum_type=None, containing_type=None, @@ -348,8 +446,128 @@ _SCHEMA_DECIMALTYPE = _descriptor.Descriptor( extension_ranges=[], oneofs=[ ], - serialized_start=674, - serialized_end=721, + serialized_start=793, + serialized_end=840, +) + +_SCHEMA_BINARYINFO = _descriptor.Descriptor( + name='BinaryInfo', + full_name='org.apache.flink.fn_execution.v1.Schema.BinaryInfo', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='length', full_name='org.apache.flink.fn_execution.v1.Schema.BinaryInfo.length', index=0, + number=1, type=5, cpp_type=1, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=842, + serialized_end=870, +) + +_SCHEMA_VARBINARYINFO = _descriptor.Descriptor( + name='VarBinaryInfo', + full_name='org.apache.flink.fn_execution.v1.Schema.VarBinaryInfo', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='length', full_name='org.apache.flink.fn_execution.v1.Schema.VarBinaryInfo.length', index=0, + number=1, type=5, cpp_type=1, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=872, + serialized_end=903, +) + +_SCHEMA_CHARINFO = _descriptor.Descriptor( + name='CharInfo', + full_name='org.apache.flink.fn_execution.v1.Schema.CharInfo', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='length', full_name='org.apache.flink.fn_execution.v1.Schema.CharInfo.length', index=0, + number=1, type=5, cpp_type=1, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=905, + serialized_end=931, +) + +_SCHEMA_VARCHARINFO = _descriptor.Descriptor( + name='VarCharInfo', + full_name='org.apache.flink.fn_execution.v1.Schema.VarCharInfo', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='length', full_name='org.apache.flink.fn_execution.v1.Schema.VarCharInfo.length', index=0, + number=1, type=5, cpp_type=1, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=933, + serialized_end=962, ) _SCHEMA_FIELDTYPE = _descriptor.Descriptor( @@ -381,7 +599,7 @@ _SCHEMA_FIELDTYPE = _descriptor.Descriptor( is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( - name='map_type', full_name='org.apache.flink.fn_execution.v1.Schema.FieldType.map_type', index=3, + name='map_info', full_name='org.apache.flink.fn_execution.v1.Schema.FieldType.map_info', index=3, number=4, type=11, cpp_type=10, label=1, has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, @@ -395,19 +613,68 @@ _SCHEMA_FIELDTYPE = _descriptor.Descriptor( is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( - name='date_time_type', full_name='org.apache.flink.fn_execution.v1.Schema.FieldType.date_time_type', index=5, + name='decimal_info', full_name='org.apache.flink.fn_execution.v1.Schema.FieldType.decimal_info', index=5, number=6, type=11, cpp_type=10, label=1, has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( - name='decimal_type', full_name='org.apache.flink.fn_execution.v1.Schema.FieldType.decimal_type', index=6, + name='time_info', full_name='org.apache.flink.fn_execution.v1.Schema.FieldType.time_info', index=6, number=7, type=11, cpp_type=10, label=1, has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='timestamp_info', full_name='org.apache.flink.fn_execution.v1.Schema.FieldType.timestamp_info', index=7, + number=8, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='local_zoned_timestamp_info', full_name='org.apache.flink.fn_execution.v1.Schema.FieldType.local_zoned_timestamp_info', index=8, + number=9, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='zoned_timestamp_info', full_name='org.apache.flink.fn_execution.v1.Schema.FieldType.zoned_timestamp_info', index=9, + number=10, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='binary_info', full_name='org.apache.flink.fn_execution.v1.Schema.FieldType.binary_info', index=10, + number=11, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='var_binary_info', full_name='org.apache.flink.fn_execution.v1.Schema.FieldType.var_binary_info', index=11, + number=12, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='char_info', full_name='org.apache.flink.fn_execution.v1.Schema.FieldType.char_info', index=12, + number=13, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='var_char_info', full_name='org.apache.flink.fn_execution.v1.Schema.FieldType.var_char_info', index=13, + number=14, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), ], extensions=[ ], @@ -423,8 +690,8 @@ _SCHEMA_FIELDTYPE = _descriptor.Descriptor( name='type_info', full_name='org.apache.flink.fn_execution.v1.Schema.FieldType.type_info', index=0, containing_type=None, fields=[]), ], - serialized_start=724, - serialized_end=1216, + serialized_start=965, + serialized_end=2037, ) _SCHEMA_FIELD = _descriptor.Descriptor( @@ -467,8 +734,8 @@ _SCHEMA_FIELD = _descriptor.Descriptor( extension_ranges=[], oneofs=[ ], - serialized_start=1218, - serialized_end=1326, + serialized_start=2039, + serialized_end=2147, ) _SCHEMA = _descriptor.Descriptor( @@ -488,7 +755,7 @@ _SCHEMA = _descriptor.Descriptor( ], extensions=[ ], - nested_types=[_SCHEMA_MAPTYPE, _SCHEMA_DATETIMETYPE, _SCHEMA_DECIMALTYPE, _SCHEMA_FIELDTYPE, _SCHEMA_FIELD, ], + nested_types=[_SCHEMA_MAPINFO, _SCHEMA_TIMEINFO, _SCHEMA_TIMESTAMPINFO, _SCHEMA_LOCALZONEDTIMESTAMPINFO, _SCHEMA_ZONEDTIMESTAMPINFO, _SCHEMA_DECIMALINFO, _SCHEMA_BINARYINFO, _SCHEMA_VARBINARYINFO, _SCHEMA_CHARINFO, _SCHEMA_VARCHARINFO, _SCHEMA_FIELDTYPE, _SCHEMA_FIELD, ], enum_types=[ _SCHEMA_TYPENAME, ], @@ -499,7 +766,7 @@ _SCHEMA = _descriptor.Descriptor( oneofs=[ ], serialized_start=411, - serialized_end=1563, + serialized_end=2433, ) _USERDEFINEDFUNCTION_INPUT.fields_by_name['udf'].message_type = _USERDEFINEDFUNCTION @@ -515,33 +782,68 @@ _USERDEFINEDFUNCTION_INPUT.oneofs_by_name['input'].fields.append( _USERDEFINEDFUNCTION_INPUT.fields_by_name['inputConstant'].containing_oneof = _USERDEFINEDFUNCTION_INPUT.oneofs_by_name['input'] _USERDEFINEDFUNCTION.fields_by_name['inputs'].message_type = _USERDEFINEDFUNCTION_INPUT _USERDEFINEDFUNCTIONS.fields_by_name['udfs'].message_type = _USERDEFINEDFUNCTION -_SCHEMA_MAPTYPE.fields_by_name['key_type'].message_type = _SCHEMA_FIELDTYPE -_SCHEMA_MAPTYPE.fields_by_name['value_type'].message_type = _SCHEMA_FIELDTYPE -_SCHEMA_MAPTYPE.containing_type = _SCHEMA -_SCHEMA_DATETIMETYPE.containing_type = _SCHEMA -_SCHEMA_DECIMALTYPE.containing_type = _SCHEMA +_SCHEMA_MAPINFO.fields_by_name['key_type'].message_type = _SCHEMA_FIELDTYPE +_SCHEMA_MAPINFO.fields_by_name['value_type'].message_type = _SCHEMA_FIELDTYPE +_SCHEMA_MAPINFO.containing_type = _SCHEMA +_SCHEMA_TIMEINFO.containing_type = _SCHEMA +_SCHEMA_TIMESTAMPINFO.containing_type = _SCHEMA +_SCHEMA_LOCALZONEDTIMESTAMPINFO.containing_type = _SCHEMA +_SCHEMA_ZONEDTIMESTAMPINFO.containing_type = _SCHEMA +_SCHEMA_DECIMALINFO.containing_type = _SCHEMA +_SCHEMA_BINARYINFO.containing_type = _SCHEMA +_SCHEMA_VARBINARYINFO.containing_type = _SCHEMA +_SCHEMA_CHARINFO.containing_type = _SCHEMA +_SCHEMA_VARCHARINFO.containing_type = _SCHEMA _SCHEMA_FIELDTYPE.fields_by_name['type_name'].enum_type = _SCHEMA_TYPENAME _SCHEMA_FIELDTYPE.fields_by_name['collection_element_type'].message_type = _SCHEMA_FIELDTYPE -_SCHEMA_FIELDTYPE.fields_by_name['map_type'].message_type = _SCHEMA_MAPTYPE +_SCHEMA_FIELDTYPE.fields_by_name['map_info'].message_type = _SCHEMA_MAPINFO _SCHEMA_FIELDTYPE.fields_by_name['row_schema'].message_type = _SCHEMA -_SCHEMA_FIELDTYPE.fields_by_name['date_time_type'].message_type = _SCHEMA_DATETIMETYPE -_SCHEMA_FIELDTYPE.fields_by_name['decimal_type'].message_type = _SCHEMA_DECIMALTYPE +_SCHEMA_FIELDTYPE.fields_by_name['decimal_info'].message_type = _SCHEMA_DECIMALINFO +_SCHEMA_FIELDTYPE.fields_by_name['time_info'].message_type = _SCHEMA_TIMEINFO +_SCHEMA_FIELDTYPE.fields_by_name['timestamp_info'].message_type = _SCHEMA_TIMESTAMPINFO +_SCHEMA_FIELDTYPE.fields_by_name['local_zoned_timestamp_info'].message_type = _SCHEMA_LOCALZONEDTIMESTAMPINFO +_SCHEMA_FIELDTYPE.fields_by_name['zoned_timestamp_info'].message_type = _SCHEMA_ZONEDTIMESTAMPINFO +_SCHEMA_FIELDTYPE.fields_by_name['binary_info'].message_type = _SCHEMA_BINARYINFO +_SCHEMA_FIELDTYPE.fields_by_name['var_binary_info'].message_type = _SCHEMA_VARBINARYINFO +_SCHEMA_FIELDTYPE.fields_by_name['char_info'].message_type = _SCHEMA_CHARINFO +_SCHEMA_FIELDTYPE.fields_by_name['var_char_info'].message_type = _SCHEMA_VARCHARINFO _SCHEMA_FIELDTYPE.containing_type = _SCHEMA _SCHEMA_FIELDTYPE.oneofs_by_name['type_info'].fields.append( _SCHEMA_FIELDTYPE.fields_by_name['collection_element_type']) _SCHEMA_FIELDTYPE.fields_by_name['collection_element_type'].containing_oneof = _SCHEMA_FIELDTYPE.oneofs_by_name['type_info'] _SCHEMA_FIELDTYPE.oneofs_by_name['type_info'].fields.append( - _SCHEMA_FIELDTYPE.fields_by_name['map_type']) -_SCHEMA_FIELDTYPE.fields_by_name['map_type'].containing_oneof = _SCHEMA_FIELDTYPE.oneofs_by_name['type_info'] + _SCHEMA_FIELDTYPE.fields_by_name['map_info']) +_SCHEMA_FIELDTYPE.fields_by_name['map_info'].containing_oneof = _SCHEMA_FIELDTYPE.oneofs_by_name['type_info'] _SCHEMA_FIELDTYPE.oneofs_by_name['type_info'].fields.append( _SCHEMA_FIELDTYPE.fields_by_name['row_schema']) _SCHEMA_FIELDTYPE.fields_by_name['row_schema'].containing_oneof = _SCHEMA_FIELDTYPE.oneofs_by_name['type_info'] _SCHEMA_FIELDTYPE.oneofs_by_name['type_info'].fields.append( - _SCHEMA_FIELDTYPE.fields_by_name['date_time_type']) -_SCHEMA_FIELDTYPE.fields_by_name['date_time_type'].containing_oneof = _SCHEMA_FIELDTYPE.oneofs_by_name['type_info'] + _SCHEMA_FIELDTYPE.fields_by_name['decimal_info']) +_SCHEMA_FIELDTYPE.fields_by_name['decimal_info'].containing_oneof = _SCHEMA_FIELDTYPE.oneofs_by_name['type_info'] +_SCHEMA_FIELDTYPE.oneofs_by_name['type_info'].fields.append( + _SCHEMA_FIELDTYPE.fields_by_name['time_info']) +_SCHEMA_FIELDTYPE.fields_by_name['time_info'].containing_oneof = _SCHEMA_FIELDTYPE.oneofs_by_name['type_info'] +_SCHEMA_FIELDTYPE.oneofs_by_name['type_info'].fields.append( + _SCHEMA_FIELDTYPE.fields_by_name['timestamp_info']) +_SCHEMA_FIELDTYPE.fields_by_name['timestamp_info'].containing_oneof = _SCHEMA_FIELDTYPE.oneofs_by_name['type_info'] +_SCHEMA_FIELDTYPE.oneofs_by_name['type_info'].fields.append( + _SCHEMA_FIELDTYPE.fields_by_name['local_zoned_timestamp_info']) +_SCHEMA_FIELDTYPE.fields_by_name['local_zoned_timestamp_info'].containing_oneof = _SCHEMA_FIELDTYPE.oneofs_by_name['type_info'] +_SCHEMA_FIELDTYPE.oneofs_by_name['type_info'].fields.append( + _SCHEMA_FIELDTYPE.fields_by_name['zoned_timestamp_info']) +_SCHEMA_FIELDTYPE.fields_by_name['zoned_timestamp_info'].containing_oneof = _SCHEMA_FIELDTYPE.oneofs_by_name['type_info'] +_SCHEMA_FIELDTYPE.oneofs_by_name['type_info'].fields.append( + _SCHEMA_FIELDTYPE.fields_by_name['binary_info']) +_SCHEMA_FIELDTYPE.fields_by_name['binary_info'].containing_oneof = _SCHEMA_FIELDTYPE.oneofs_by_name['type_info'] _SCHEMA_FIELDTYPE.oneofs_by_name['type_info'].fields.append( - _SCHEMA_FIELDTYPE.fields_by_name['decimal_type']) -_SCHEMA_FIELDTYPE.fields_by_name['decimal_type'].containing_oneof = _SCHEMA_FIELDTYPE.oneofs_by_name['type_info'] + _SCHEMA_FIELDTYPE.fields_by_name['var_binary_info']) +_SCHEMA_FIELDTYPE.fields_by_name['var_binary_info'].containing_oneof = _SCHEMA_FIELDTYPE.oneofs_by_name['type_info'] +_SCHEMA_FIELDTYPE.oneofs_by_name['type_info'].fields.append( + _SCHEMA_FIELDTYPE.fields_by_name['char_info']) +_SCHEMA_FIELDTYPE.fields_by_name['char_info'].containing_oneof = _SCHEMA_FIELDTYPE.oneofs_by_name['type_info'] +_SCHEMA_FIELDTYPE.oneofs_by_name['type_info'].fields.append( + _SCHEMA_FIELDTYPE.fields_by_name['var_char_info']) +_SCHEMA_FIELDTYPE.fields_by_name['var_char_info'].containing_oneof = _SCHEMA_FIELDTYPE.oneofs_by_name['type_info'] _SCHEMA_FIELD.fields_by_name['type'].message_type = _SCHEMA_FIELDTYPE _SCHEMA_FIELD.containing_type = _SCHEMA _SCHEMA.fields_by_name['fields'].message_type = _SCHEMA_FIELD @@ -575,24 +877,73 @@ _sym_db.RegisterMessage(UserDefinedFunctions) Schema = _reflection.GeneratedProtocolMessageType('Schema', (_message.Message,), dict( - MapType = _reflection.GeneratedProtocolMessageType('MapType', (_message.Message,), dict( - DESCRIPTOR = _SCHEMA_MAPTYPE, + MapInfo = _reflection.GeneratedProtocolMessageType('MapInfo', (_message.Message,), dict( + DESCRIPTOR = _SCHEMA_MAPINFO, + __module__ = 'flink_fn_execution_pb2' + # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.Schema.MapInfo) + )) + , + + TimeInfo = _reflection.GeneratedProtocolMessageType('TimeInfo', (_message.Message,), dict( + DESCRIPTOR = _SCHEMA_TIMEINFO, + __module__ = 'flink_fn_execution_pb2' + # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.Schema.TimeInfo) + )) + , + + TimestampInfo = _reflection.GeneratedProtocolMessageType('TimestampInfo', (_message.Message,), dict( + DESCRIPTOR = _SCHEMA_TIMESTAMPINFO, + __module__ = 'flink_fn_execution_pb2' + # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.Schema.TimestampInfo) + )) + , + + LocalZonedTimestampInfo = _reflection.GeneratedProtocolMessageType('LocalZonedTimestampInfo', (_message.Message,), dict( + DESCRIPTOR = _SCHEMA_LOCALZONEDTIMESTAMPINFO, + __module__ = 'flink_fn_execution_pb2' + # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.Schema.LocalZonedTimestampInfo) + )) + , + + ZonedTimestampInfo = _reflection.GeneratedProtocolMessageType('ZonedTimestampInfo', (_message.Message,), dict( + DESCRIPTOR = _SCHEMA_ZONEDTIMESTAMPINFO, + __module__ = 'flink_fn_execution_pb2' + # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.Schema.ZonedTimestampInfo) + )) + , + + DecimalInfo = _reflection.GeneratedProtocolMessageType('DecimalInfo', (_message.Message,), dict( + DESCRIPTOR = _SCHEMA_DECIMALINFO, + __module__ = 'flink_fn_execution_pb2' + # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.Schema.DecimalInfo) + )) + , + + BinaryInfo = _reflection.GeneratedProtocolMessageType('BinaryInfo', (_message.Message,), dict( + DESCRIPTOR = _SCHEMA_BINARYINFO, + __module__ = 'flink_fn_execution_pb2' + # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.Schema.BinaryInfo) + )) + , + + VarBinaryInfo = _reflection.GeneratedProtocolMessageType('VarBinaryInfo', (_message.Message,), dict( + DESCRIPTOR = _SCHEMA_VARBINARYINFO, __module__ = 'flink_fn_execution_pb2' - # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.Schema.MapType) + # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.Schema.VarBinaryInfo) )) , - DateTimeType = _reflection.GeneratedProtocolMessageType('DateTimeType', (_message.Message,), dict( - DESCRIPTOR = _SCHEMA_DATETIMETYPE, + CharInfo = _reflection.GeneratedProtocolMessageType('CharInfo', (_message.Message,), dict( + DESCRIPTOR = _SCHEMA_CHARINFO, __module__ = 'flink_fn_execution_pb2' - # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.Schema.DateTimeType) + # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.Schema.CharInfo) )) , - DecimalType = _reflection.GeneratedProtocolMessageType('DecimalType', (_message.Message,), dict( - DESCRIPTOR = _SCHEMA_DECIMALTYPE, + VarCharInfo = _reflection.GeneratedProtocolMessageType('VarCharInfo', (_message.Message,), dict( + DESCRIPTOR = _SCHEMA_VARCHARINFO, __module__ = 'flink_fn_execution_pb2' - # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.Schema.DecimalType) + # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.Schema.VarCharInfo) )) , @@ -614,9 +965,16 @@ Schema = _reflection.GeneratedProtocolMessageType('Schema', (_message.Message,), # @@protoc_insertion_point(class_scope:org.apache.flink.fn_execution.v1.Schema) )) _sym_db.RegisterMessage(Schema) -_sym_db.RegisterMessage(Schema.MapType) -_sym_db.RegisterMessage(Schema.DateTimeType) -_sym_db.RegisterMessage(Schema.DecimalType) +_sym_db.RegisterMessage(Schema.MapInfo) +_sym_db.RegisterMessage(Schema.TimeInfo) +_sym_db.RegisterMessage(Schema.TimestampInfo) +_sym_db.RegisterMessage(Schema.LocalZonedTimestampInfo) +_sym_db.RegisterMessage(Schema.ZonedTimestampInfo) +_sym_db.RegisterMessage(Schema.DecimalInfo) +_sym_db.RegisterMessage(Schema.BinaryInfo) +_sym_db.RegisterMessage(Schema.VarBinaryInfo) +_sym_db.RegisterMessage(Schema.CharInfo) +_sym_db.RegisterMessage(Schema.VarCharInfo) _sym_db.RegisterMessage(Schema.FieldType) _sym_db.RegisterMessage(Schema.Field) diff --git a/flink-python/pyflink/proto/flink-fn-execution.proto b/flink-python/pyflink/proto/flink-fn-execution.proto index 0cca33ef27..02489b4083 100644 --- a/flink-python/pyflink/proto/flink-fn-execution.proto +++ b/flink-python/pyflink/proto/flink-fn-execution.proto @@ -64,7 +64,7 @@ message Schema { DOUBLE = 7; DATE = 8; TIME = 9; - DATETIME = 10; + TIMESTAMP = 10; BOOLEAN = 11; BINARY = 12; VARBINARY = 13; @@ -73,31 +73,68 @@ message Schema { ARRAY = 16; MAP = 17; MULTISET = 18; + LOCAL_ZONED_TIMESTAMP = 19; + ZONED_TIMESTAMP = 20; } - message MapType { + message MapInfo { FieldType key_type = 1; FieldType value_type = 2; } - message DateTimeType { + message TimeInfo { int32 precision = 1; } - message DecimalType { + message TimestampInfo { + int32 precision = 1; + } + + message LocalZonedTimestampInfo { + int32 precision = 1; + } + + message ZonedTimestampInfo { + int32 precision = 1; + } + + message DecimalInfo { int32 precision = 1; int32 scale = 2; } + message BinaryInfo { + int32 length = 1; + } + + message VarBinaryInfo { + int32 length = 1; + } + + message CharInfo { + int32 length = 1; + } + + message VarCharInfo { + int32 length = 1; + } + message FieldType { TypeName type_name = 1; bool nullable = 2; oneof type_info { FieldType collection_element_type = 3; - MapType map_type = 4; + MapInfo map_info = 4; Schema row_schema = 5; - DateTimeType date_time_type = 6; - DecimalType decimal_type = 7; + DecimalInfo decimal_info = 6; + TimeInfo time_info = 7; + TimestampInfo timestamp_info = 8; + LocalZonedTimestampInfo local_zoned_timestamp_info = 9; + ZonedTimestampInfo zoned_timestamp_info = 10; + BinaryInfo binary_info = 11; + VarBinaryInfo var_binary_info = 12; + CharInfo char_info = 13; + VarCharInfo var_char_info = 14; } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java index 383f75f481..97d6cf6a60 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java @@ -353,6 +353,7 @@ public final class PythonTypeUtils { public FlinkFnApi.Schema.FieldType visit(BinaryType binaryType) { return FlinkFnApi.Schema.FieldType.newBuilder() .setTypeName(FlinkFnApi.Schema.TypeName.BINARY) + .setBinaryInfo(FlinkFnApi.Schema.BinaryInfo.newBuilder().setLength(binaryType.getLength())) .setNullable(binaryType.isNullable()) .build(); } @@ -361,6 +362,7 @@ public final class PythonTypeUtils { public FlinkFnApi.Schema.FieldType visit(VarBinaryType varBinaryType) { return FlinkFnApi.Schema.FieldType.newBuilder() .setTypeName(FlinkFnApi.Schema.TypeName.VARBINARY) + .setVarBinaryInfo(FlinkFnApi.Schema.VarBinaryInfo.newBuilder().setLength(varBinaryType.getLength())) .setNullable(varBinaryType.isNullable()) .build(); } @@ -369,6 +371,7 @@ public final class PythonTypeUtils { public FlinkFnApi.Schema.FieldType visit(CharType charType) { return FlinkFnApi.Schema.FieldType.newBuilder() .setTypeName(FlinkFnApi.Schema.TypeName.CHAR) + .setCharInfo(FlinkFnApi.Schema.CharInfo.newBuilder().setLength(charType.getLength())) .setNullable(charType.isNullable()) .build(); } @@ -377,6 +380,7 @@ public final class PythonTypeUtils { public FlinkFnApi.Schema.FieldType visit(VarCharType varCharType) { return FlinkFnApi.Schema.FieldType.newBuilder() .setTypeName(FlinkFnApi.Schema.TypeName.VARCHAR) + .setVarCharInfo(FlinkFnApi.Schema.VarCharInfo.newBuilder().setLength(varCharType.getLength())) .setNullable(varCharType.isNullable()) .build(); } @@ -393,6 +397,7 @@ public final class PythonTypeUtils { public FlinkFnApi.Schema.FieldType visit(TimeType timeType) { return FlinkFnApi.Schema.FieldType.newBuilder() .setTypeName(FlinkFnApi.Schema.TypeName.TIME) + .setTimeInfo(FlinkFnApi.Schema.TimeInfo.newBuilder().setPrecision(timeType.getPrecision())) .setNullable(timeType.isNullable()) .build(); } @@ -401,13 +406,13 @@ public final class PythonTypeUtils { public FlinkFnApi.Schema.FieldType visit(TimestampType timestampType) { FlinkFnApi.Schema.FieldType.Builder builder = FlinkFnApi.Schema.FieldType.newBuilder() - .setTypeName(FlinkFnApi.Schema.TypeName.DATETIME) + .setTypeName(FlinkFnApi.Schema.TypeName.TIMESTAMP) .setNullable(timestampType.isNullable()); - FlinkFnApi.Schema.DateTimeType.Builder dateTimeBuilder = - FlinkFnApi.Schema.DateTimeType.newBuilder() + FlinkFnApi.Schema.TimestampInfo.Builder timestampInfoBuilder = + FlinkFnApi.Schema.TimestampInfo.newBuilder() .setPrecision(timestampType.getPrecision()); - builder.setDateTimeType(dateTimeBuilder.build()); + builder.setTimestampInfo(timestampInfoBuilder); return builder.build(); } @@ -418,11 +423,11 @@ public final class PythonTypeUtils { .setTypeName(FlinkFnApi.Schema.TypeName.DECIMAL) .setNullable(decimalType.isNullable()); - FlinkFnApi.Schema.DecimalType.Builder decimalTypeBuilder = - FlinkFnApi.Schema.DecimalType.newBuilder() + FlinkFnApi.Schema.DecimalInfo.Builder decimalInfoBuilder = + FlinkFnApi.Schema.DecimalInfo.newBuilder() .setPrecision(decimalType.getPrecision()) .setScale(decimalType.getScale()); - builder.setDecimalType(decimalTypeBuilder); + builder.setDecimalInfo(decimalInfoBuilder); return builder.build(); } @@ -445,11 +450,11 @@ public final class PythonTypeUtils { .setTypeName(FlinkFnApi.Schema.TypeName.MAP) .setNullable(mapType.isNullable()); - FlinkFnApi.Schema.MapType.Builder mapBuilder = - FlinkFnApi.Schema.MapType.newBuilder() + FlinkFnApi.Schema.MapInfo.Builder mapBuilder = + FlinkFnApi.Schema.MapInfo.newBuilder() .setKeyType(mapType.getKeyType().accept(this)) .setValueType(mapType.getValueType().accept(this)); - builder.setMapType(mapBuilder.build()); + builder.setMapInfo(mapBuilder.build()); return builder.build(); } @@ -484,11 +489,11 @@ public final class PythonTypeUtils { .setNullable(logicalType.isNullable()); // Because we can't get precision and scale from legacy BIG_DEC_TYPE_INFO, // we set the precision and scale to default value compatible with python. - FlinkFnApi.Schema.DecimalType.Builder decimalTypeBuilder = - FlinkFnApi.Schema.DecimalType.newBuilder() + FlinkFnApi.Schema.DecimalInfo.Builder decimalTypeBuilder = + FlinkFnApi.Schema.DecimalInfo.newBuilder() .setPrecision(38) .setScale(18); - builder.setDecimalType(decimalTypeBuilder); + builder.setDecimalInfo(decimalTypeBuilder); return builder.build(); } } -- Gitee From fac0cf15765764fcf2c6c64fca159e161ac042dd Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Tue, 17 Mar 2020 16:49:40 +0800 Subject: [PATCH 178/885] [FLINK-16047][table-planner-blink] Fix Blink planner produces wrong aggregate results with state clean up This closes #11419 --- .../harness/GroupAggregateHarnessTest.scala | 137 ++++++++++++++++++ .../operators/aggregate/GroupAggFunction.java | 7 + .../runtime/util/BaseRowHarnessAssertor.java | 12 +- .../table/runtime/util/StreamRecordUtils.java | 9 ++ 4 files changed, 164 insertions(+), 1 deletion(-) create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/harness/GroupAggregateHarnessTest.scala diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/harness/GroupAggregateHarnessTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/harness/GroupAggregateHarnessTest.scala new file mode 100644 index 0000000000..b566aba14f --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/harness/GroupAggregateHarnessTest.scala @@ -0,0 +1,137 @@ +/* + * 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.table.planner.runtime.harness + +import org.apache.flink.api.common.time.Time +import org.apache.flink.api.scala._ +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.api.scala.internal.StreamTableEnvironmentImpl +import org.apache.flink.table.api.{EnvironmentSettings, Types} +import org.apache.flink.table.planner.runtime.utils.StreamingWithStateTestBase.StateBackendMode +import org.apache.flink.table.runtime.util.BaseRowHarnessAssertor +import org.apache.flink.table.runtime.util.StreamRecordUtils.{binaryrow, retractBinaryRow} +import org.apache.flink.types.Row +import org.junit.runner.RunWith +import org.junit.runners.Parameterized +import org.junit.{Before, Test} + +import java.lang.{Long => JLong} +import java.util.concurrent.ConcurrentLinkedQueue + +import scala.collection.mutable + +@RunWith(classOf[Parameterized]) +class GroupAggregateHarnessTest(mode: StateBackendMode) extends HarnessTestBase(mode) { + + @Before + override def before(): Unit = { + super.before() + val setting = EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build() + val config = new TestTableConfig + this.tEnv = StreamTableEnvironmentImpl.create(env, setting, config) + } + + @Test + def testAggregateWithRetraction(): Unit = { + val data = new mutable.MutableList[(String, String, Long)] + val t = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c) + tEnv.createTemporaryView("T", t) + + val sql = + """ + |SELECT a, SUM(c) + |FROM ( + | SELECT a, b, SUM(c) as c + | FROM T GROUP BY a, b + |)GROUP BY a + """.stripMargin + val t1 = tEnv.sqlQuery(sql) + + val queryConfig = new TestStreamQueryConfig(Time.seconds(2), Time.seconds(3)) + val testHarness = createHarnessTester(t1.toRetractStream[Row](queryConfig), "GroupAggregate") + val assertor = new BaseRowHarnessAssertor(Array( Types.STRING, Types.LONG)) + + testHarness.open() + + val expectedOutput = new ConcurrentLinkedQueue[Object]() + + // register cleanup timer with 3001 + testHarness.setProcessingTime(1) + + // accumulate + testHarness.processElement(new StreamRecord(binaryrow("aaa", 1L: JLong), 1)) + expectedOutput.add(new StreamRecord(binaryrow("aaa", 1L: JLong), 1)) + + // accumulate + testHarness.processElement(new StreamRecord(binaryrow("bbb", 1L: JLong), 2)) + expectedOutput.add(new StreamRecord(binaryrow("bbb", 1L: JLong), 2)) + + // retract for insertion + testHarness.processElement(new StreamRecord(binaryrow("aaa", 2L: JLong), 3)) + expectedOutput.add(new StreamRecord(retractBinaryRow( "aaa", 1L: JLong), 3)) + expectedOutput.add(new StreamRecord(binaryrow("aaa", 3L: JLong), 3)) + + // retract for deletion + testHarness.processElement(new StreamRecord(retractBinaryRow("aaa", 2L: JLong), 3)) + expectedOutput.add(new StreamRecord(retractBinaryRow("aaa", 3L: JLong), 3)) + expectedOutput.add(new StreamRecord(binaryrow("aaa", 1L: JLong), 3)) + + // accumulate + testHarness.processElement(new StreamRecord(binaryrow("ccc", 3L: JLong), 4)) + expectedOutput.add(new StreamRecord(binaryrow("ccc", 3L: JLong), 4)) + + // trigger cleanup timer and register cleanup timer with 6002 + testHarness.setProcessingTime(3002) + + // retract after clean up + testHarness.processElement(new StreamRecord(retractBinaryRow("ccc", 3L: JLong), 4)) + // not output + + // accumulate + testHarness.processElement(new StreamRecord(binaryrow("aaa", 4L: JLong), 5)) + expectedOutput.add(new StreamRecord(binaryrow("aaa", 4L: JLong), 5)) + testHarness.processElement(new StreamRecord(binaryrow("bbb", 2L: JLong), 6)) + expectedOutput.add(new StreamRecord(binaryrow("bbb", 2L: JLong), 6)) + + // retract + testHarness.processElement(new StreamRecord(binaryrow("aaa", 5L: JLong), 7)) + expectedOutput.add(new StreamRecord(retractBinaryRow("aaa", 4L: JLong), 7)) + expectedOutput.add(new StreamRecord(binaryrow("aaa", 9L: JLong), 7)) + + // accumulate + testHarness.processElement(new StreamRecord(binaryrow("eee", 6L: JLong), 8)) + expectedOutput.add(new StreamRecord(binaryrow("eee", 6L: JLong), 8)) + + // retract + testHarness.processElement(new StreamRecord(binaryrow("aaa", 7L: JLong), 9)) + expectedOutput.add(new StreamRecord(retractBinaryRow("aaa", 9L: JLong), 9)) + expectedOutput.add(new StreamRecord(binaryrow("aaa", 16L: JLong), 9)) + testHarness.processElement(new StreamRecord(binaryrow("bbb", 3L: JLong), 10)) + expectedOutput.add(new StreamRecord(retractBinaryRow("bbb", 2L: JLong), 10)) + expectedOutput.add(new StreamRecord(binaryrow("bbb", 5L: JLong), 10)) + + val result = testHarness.getOutput + + assertor.assertOutputEqualsSorted("result mismatch", expectedOutput, result) + + testHarness.close() + } + +} diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/GroupAggFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/GroupAggFunction.java index d7a400b8f3..75d8ef1615 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/GroupAggFunction.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/GroupAggFunction.java @@ -36,6 +36,7 @@ import org.apache.flink.util.Collector; import static org.apache.flink.table.dataformat.util.BaseRowUtil.ACCUMULATE_MSG; import static org.apache.flink.table.dataformat.util.BaseRowUtil.RETRACT_MSG; import static org.apache.flink.table.dataformat.util.BaseRowUtil.isAccumulateMsg; +import static org.apache.flink.table.dataformat.util.BaseRowUtil.isRetractMsg; /** * Aggregate Function used for the groupby (without window) aggregate. @@ -141,6 +142,12 @@ public class GroupAggFunction extends KeyedProcessFunctionWithCleanupState Date: Wed, 11 Mar 2020 21:55:21 +0100 Subject: [PATCH 179/885] [FLINK-16551][WebUI][tests] Improve assertions --- .../flink/runtime/webmonitor/WebFrontendITCase.java | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebFrontendITCase.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebFrontendITCase.java index 3df9517e0c..37b160ea16 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebFrontendITCase.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebFrontendITCase.java @@ -64,6 +64,8 @@ import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.stream.Collectors; +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; @@ -120,8 +122,7 @@ public class WebFrontendITCase extends TestLogger { @Test public void getFrontPage() throws Exception { String fromHTTP = TestBaseUtils.getFromHTTP("http://localhost:" + getRestPort() + "/index.html"); - String text = "Apache Flink Web Dashboard"; - assertTrue("Startpage should contain " + text, fromHTTP.contains(text)); + assertThat(fromHTTP, containsString("Apache Flink Web Dashboard")); } private int getRestPort() { @@ -195,11 +196,11 @@ public class WebFrontendITCase extends TestLogger { FileUtils.writeStringToFile(logFiles.logFile, "job manager log"); String logs = TestBaseUtils.getFromHTTP("http://localhost:" + getRestPort() + "/jobmanager/log"); - assertTrue(logs.contains("job manager log")); + assertThat(logs, containsString("job manager log")); FileUtils.writeStringToFile(logFiles.stdOutFile, "job manager out"); logs = TestBaseUtils.getFromHTTP("http://localhost:" + getRestPort() + "/jobmanager/stdout"); - assertTrue(logs.contains("job manager out")); + assertThat(logs, containsString("job manager out")); } @Test @@ -217,11 +218,11 @@ public class WebFrontendITCase extends TestLogger { //we check for job manager log files, since no separate taskmanager logs exist FileUtils.writeStringToFile(logFiles.logFile, "job manager log"); String logs = TestBaseUtils.getFromHTTP("http://localhost:" + getRestPort() + "/taskmanagers/" + id + "/log"); - assertTrue(logs.contains("job manager log")); + assertThat(logs, containsString("job manager log")); FileUtils.writeStringToFile(logFiles.stdOutFile, "job manager out"); logs = TestBaseUtils.getFromHTTP("http://localhost:" + getRestPort() + "/taskmanagers/" + id + "/stdout"); - assertTrue(logs.contains("job manager out")); + assertThat(logs, containsString("job manager out")); } @Test -- Gitee From 6cf07d374a34742a919a1dc1edf4eb1c1f44e831 Mon Sep 17 00:00:00 2001 From: Canbin Zheng Date: Tue, 17 Mar 2020 23:12:26 +0800 Subject: [PATCH 180/885] [FLINK-16494][k8s] Switch to enum type for config option KubernetesConfigOptions.CONTAINER_IMAGE_PULL_POLICY This closes #11406 . --- .../kubernetes_config_configuration.html | 6 +++--- .../configuration/KubernetesConfigOptions.java | 17 +++++++++++++---- .../decorators/InitJobManagerDecorator.java | 2 +- .../decorators/InitTaskManagerDecorator.java | 2 +- .../AbstractKubernetesParameters.java | 4 ++-- .../parameters/KubernetesParameters.java | 4 +++- .../flink/kubernetes/KubernetesTestBase.java | 5 +++-- .../decorators/InitJobManagerDecoratorTest.java | 2 +- .../InitTaskManagerDecoratorTest.java | 2 +- .../KubernetesJobManagerFactoryTest.java | 2 +- .../KubernetesTaskManagerFactoryTest.java | 2 +- 11 files changed, 30 insertions(+), 18 deletions(-) diff --git a/docs/_includes/generated/kubernetes_config_configuration.html b/docs/_includes/generated/kubernetes_config_configuration.html index 1c9dff1e13..aad4edf5e1 100644 --- a/docs/_includes/generated/kubernetes_config_configuration.html +++ b/docs/_includes/generated/kubernetes_config_configuration.html @@ -34,9 +34,9 @@
    - - - + + + diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java index fff5c13bd8..0ddb84801d 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java @@ -66,11 +66,11 @@ public class KubernetesConfigOptions { .withDescription("The number of cpu used by task manager. By default, the cpu is set " + "to the number of slots per TaskManager"); - public static final ConfigOption CONTAINER_IMAGE_PULL_POLICY = + public static final ConfigOption CONTAINER_IMAGE_PULL_POLICY = key("kubernetes.container.image.pull-policy") - .stringType() - .defaultValue("IfNotPresent") - .withDescription("Kubernetes image pull policy. Valid values are Always, Never, and IfNotPresent. " + + .enumType(ImagePullPolicy.class) + .defaultValue(ImagePullPolicy.IfNotPresent) + .withDescription("The Kubernetes container image pull policy (IfNotPresent or Always or Never). " + "The default policy is IfNotPresent to avoid putting pressure to image repository."); public static final ConfigOption> CONTAINER_IMAGE_PULL_SECRETS = @@ -156,6 +156,15 @@ public class KubernetesConfigOptions { LoadBalancer } + /** + * The container image pull policy. + */ + public enum ImagePullPolicy { + IfNotPresent, + Always, + Never + } + /** This class is not meant to be instantiated. */ private KubernetesConfigOptions() {} } diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecorator.java index 18e7c405a0..2836dea4d0 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecorator.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecorator.java @@ -83,7 +83,7 @@ public class InitJobManagerDecorator extends AbstractKubernetesStepDecorator { return new ContainerBuilder(container) .withName(kubernetesJobManagerParameters.getJobManagerMainContainerName()) .withImage(kubernetesJobManagerParameters.getImage()) - .withImagePullPolicy(kubernetesJobManagerParameters.getImagePullPolicy()) + .withImagePullPolicy(kubernetesJobManagerParameters.getImagePullPolicy().name()) .withResources(requirements) .withPorts(getContainerPorts()) .withEnv(getCustomizedEnvs()) diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecorator.java index 4d521cf72b..ef118c2db3 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecorator.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecorator.java @@ -77,7 +77,7 @@ public class InitTaskManagerDecorator extends AbstractKubernetesStepDecorator { return new ContainerBuilder(container) .withName(kubernetesTaskManagerParameters.getTaskManagerMainContainerName()) .withImage(kubernetesTaskManagerParameters.getImage()) - .withImagePullPolicy(kubernetesTaskManagerParameters.getImagePullPolicy()) + .withImagePullPolicy(kubernetesTaskManagerParameters.getImagePullPolicy().name()) .withResources(resourceRequirements) .withPorts(new ContainerPortBuilder() .withName(Constants.TASK_MANAGER_RPC_PORT_NAME) diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/AbstractKubernetesParameters.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/AbstractKubernetesParameters.java index 93b8524424..5f94a55872 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/AbstractKubernetesParameters.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/AbstractKubernetesParameters.java @@ -77,8 +77,8 @@ public abstract class AbstractKubernetesParameters implements KubernetesParamete } @Override - public String getImagePullPolicy() { - return flinkConfig.getString(KubernetesConfigOptions.CONTAINER_IMAGE_PULL_POLICY); + public KubernetesConfigOptions.ImagePullPolicy getImagePullPolicy() { + return flinkConfig.get(KubernetesConfigOptions.CONTAINER_IMAGE_PULL_POLICY); } @Override diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesParameters.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesParameters.java index 2ce4c89967..00163412f4 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesParameters.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesParameters.java @@ -18,6 +18,8 @@ package org.apache.flink.kubernetes.kubeclient.parameters; +import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; + import io.fabric8.kubernetes.api.model.LocalObjectReference; import java.util.Map; @@ -34,7 +36,7 @@ public interface KubernetesParameters { String getImage(); - String getImagePullPolicy(); + KubernetesConfigOptions.ImagePullPolicy getImagePullPolicy(); LocalObjectReference[] getImagePullSecrets(); diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesTestBase.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesTestBase.java index 05de0368ef..d53330bc3a 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesTestBase.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesTestBase.java @@ -46,7 +46,8 @@ public class KubernetesTestBase extends TestLogger { protected static final String NAMESPACE = "test"; protected static final String CLUSTER_ID = "my-flink-cluster1"; protected static final String CONTAINER_IMAGE = "flink-k8s-test:latest"; - protected static final String CONTAINER_IMAGE_PULL_POLICY = "IfNotPresent"; + protected static final KubernetesConfigOptions.ImagePullPolicy CONTAINER_IMAGE_PULL_POLICY = + KubernetesConfigOptions.ImagePullPolicy.IfNotPresent; @Rule public MixedKubernetesServer server = new MixedKubernetesServer(true, true); @@ -67,7 +68,7 @@ public class KubernetesTestBase extends TestLogger { flinkConfig.setString(KubernetesConfigOptions.NAMESPACE, NAMESPACE); flinkConfig.setString(KubernetesConfigOptions.CLUSTER_ID, CLUSTER_ID); flinkConfig.setString(KubernetesConfigOptions.CONTAINER_IMAGE, CONTAINER_IMAGE); - flinkConfig.setString(KubernetesConfigOptions.CONTAINER_IMAGE_PULL_POLICY, CONTAINER_IMAGE_PULL_POLICY); + flinkConfig.set(KubernetesConfigOptions.CONTAINER_IMAGE_PULL_POLICY, CONTAINER_IMAGE_PULL_POLICY); flinkConfDir = temporaryFolder.newFolder().getAbsoluteFile(); writeFlinkConfiguration(); diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecoratorTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecoratorTest.java index ba10a7234a..8024792a0b 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecoratorTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecoratorTest.java @@ -82,7 +82,7 @@ public class InitJobManagerDecoratorTest extends KubernetesJobManagerTestBase { @Test public void testMainContainerImage() { assertEquals(CONTAINER_IMAGE, this.resultMainContainer.getImage()); - assertEquals(CONTAINER_IMAGE_PULL_POLICY, this.resultMainContainer.getImagePullPolicy()); + assertEquals(CONTAINER_IMAGE_PULL_POLICY.name(), this.resultMainContainer.getImagePullPolicy()); } @Test diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecoratorTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecoratorTest.java index 59bb7cf8b6..8ab07e008c 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecoratorTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecoratorTest.java @@ -85,7 +85,7 @@ public class InitTaskManagerDecoratorTest extends KubernetesTaskManagerTestBase @Test public void testMainContainerImagePullPolicy() { - assertEquals(CONTAINER_IMAGE_PULL_POLICY, this.resultMainContainer.getImagePullPolicy()); + assertEquals(CONTAINER_IMAGE_PULL_POLICY.name(), this.resultMainContainer.getImagePullPolicy()); } @Test diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactoryTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactoryTest.java index 054b645b5d..7b17c9fd44 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactoryTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactoryTest.java @@ -109,7 +109,7 @@ public class KubernetesJobManagerFactoryTest extends KubernetesJobManagerTestBas final Container resultedMainContainer = resultPodSpec.getContainers().get(0); assertEquals(KubernetesJobManagerParameters.JOB_MANAGER_MAIN_CONTAINER_NAME, resultedMainContainer.getName()); assertEquals(CONTAINER_IMAGE, resultedMainContainer.getImage()); - assertEquals(CONTAINER_IMAGE_PULL_POLICY, resultedMainContainer.getImagePullPolicy()); + assertEquals(CONTAINER_IMAGE_PULL_POLICY.name(), resultedMainContainer.getImagePullPolicy()); assertEquals(3, resultedMainContainer.getEnv().size()); assertTrue(resultedMainContainer.getEnv() diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesTaskManagerFactoryTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesTaskManagerFactoryTest.java index b54d0839ca..14f6c48eb0 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesTaskManagerFactoryTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesTaskManagerFactoryTest.java @@ -66,7 +66,7 @@ public class KubernetesTaskManagerFactoryTest extends KubernetesTaskManagerTestB KubernetesTaskManagerParameters.TASK_MANAGER_MAIN_CONTAINER_NAME, resultMainContainer.getName()); assertEquals(CONTAINER_IMAGE, resultMainContainer.getImage()); - assertEquals(CONTAINER_IMAGE_PULL_POLICY, resultMainContainer.getImagePullPolicy()); + assertEquals(CONTAINER_IMAGE_PULL_POLICY.name(), resultMainContainer.getImagePullPolicy()); assertEquals(1, resultMainContainer.getPorts().size()); assertEquals(1, resultMainContainer.getCommand().size()); assertEquals(3, resultMainContainer.getArgs().size()); -- Gitee From 0a8eb8d86c5f2563e5dc8d0a533ff0cd24343869 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Fri, 13 Mar 2020 16:15:22 +0800 Subject: [PATCH 181/885] [FLINK-16586][network] Build ResultSubpartitionInfo and InputChannelInfo in respective constructors In the constructors of ResultSubpartition and InputChannel, the respective ResultSubpartitionInfo and InputChannelInfo should be created as well. These infos would be used for interacting with ChannelStateReader and ChannelStateWriter future. This cloese #11400. --- .../deployment/TaskDeploymentDescriptor.java | 14 +++++------ .../io/network/NettyShuffleEnvironment.java | 9 +++++--- .../ReleaseOnConsumptionResultPartition.java | 2 ++ .../io/network/partition/ResultPartition.java | 10 ++++++++ .../partition/ResultPartitionFactory.java | 5 ++++ .../network/partition/ResultSubpartition.java | 10 ++++++++ .../partition/consumer/InputChannel.java | 11 +++++++++ .../partition/consumer/SingleInputGate.java | 10 ++++++++ .../consumer/SingleInputGateFactory.java | 2 ++ .../runtime/shuffle/ShuffleEnvironment.java | 9 ++++---- .../flink/runtime/taskmanager/Task.java | 5 ++-- .../TaskDeploymentDescriptorBuilder.java | 10 ++++---- .../partition/InputGateFairnessTest.java | 4 +++- .../partition/ResultPartitionBuilder.java | 8 +++++++ .../partition/ResultPartitionFactoryTest.java | 2 +- .../partition/ResultPartitionTest.java | 22 ++++++++++++++++++ .../TaskExecutorPartitionTrackerImplTest.java | 5 ++-- .../consumer/SingleInputGateBuilder.java | 8 +++++++ .../consumer/SingleInputGateTest.java | 23 +++++++++++++++++++ .../TaskExecutorSubmissionTest.java | 9 ++++---- .../flink/runtime/taskmanager/TaskTest.java | 10 ++++---- .../runtime/taskmanager/TestTaskBuilder.java | 9 ++++---- .../StreamNetworkBenchmarkEnvironment.java | 5 ++-- 23 files changed, 162 insertions(+), 40 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java index ac5b770b9e..cf36a128e9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java @@ -35,7 +35,7 @@ import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.io.Serializable; -import java.util.Collection; +import java.util.List; /** * A task deployment descriptor contains all the information necessary to deploy a task on a task manager. @@ -134,10 +134,10 @@ public final class TaskDeploymentDescriptor implements Serializable { private final int attemptNumber; /** The list of produced intermediate result partition deployment descriptors. */ - private final Collection producedPartitions; + private final List producedPartitions; /** The list of consumed intermediate result partitions. */ - private final Collection inputGates; + private final List inputGates; /** Slot number to run the sub task in on the target machine. */ private final int targetSlotNumber; @@ -156,8 +156,8 @@ public final class TaskDeploymentDescriptor implements Serializable { int attemptNumber, int targetSlotNumber, @Nullable JobManagerTaskRestore taskRestore, - Collection resultPartitionDeploymentDescriptors, - Collection inputGateDeploymentDescriptors) { + List resultPartitionDeploymentDescriptors, + List inputGateDeploymentDescriptors) { this.jobId = Preconditions.checkNotNull(jobId); @@ -256,11 +256,11 @@ public final class TaskDeploymentDescriptor implements Serializable { return targetSlotNumber; } - public Collection getProducedPartitions() { + public List getProducedPartitions() { return producedPartitions; } - public Collection getInputGates() { + public List getInputGates() { return inputGates; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironment.java index 2d2d6f35ff..604cf5c934 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironment.java @@ -53,6 +53,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.Arrays; import java.util.Collection; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; @@ -185,14 +186,15 @@ public class NettyShuffleEnvironment implements ShuffleEnvironment createResultPartitionWriters( ShuffleIOOwnerContext ownerContext, - Collection resultPartitionDeploymentDescriptors) { + List resultPartitionDeploymentDescriptors) { synchronized (lock) { Preconditions.checkState(!isClosed, "The NettyShuffleEnvironment has already been shut down."); ResultPartition[] resultPartitions = new ResultPartition[resultPartitionDeploymentDescriptors.size()]; int counter = 0; for (ResultPartitionDeploymentDescriptor rpdd : resultPartitionDeploymentDescriptors) { - resultPartitions[counter++] = resultPartitionFactory.create(ownerContext.getOwnerName(), rpdd); + resultPartitions[counter] = resultPartitionFactory.create(ownerContext.getOwnerName(), counter, rpdd); + counter++; } registerOutputMetrics(config.isNetworkDetailedMetrics(), ownerContext.getOutputGroup(), resultPartitions); @@ -204,7 +206,7 @@ public class NettyShuffleEnvironment implements ShuffleEnvironment createInputGates( ShuffleIOOwnerContext ownerContext, PartitionProducerStateProvider partitionProducerStateProvider, - Collection inputGateDeploymentDescriptors) { + List inputGateDeploymentDescriptors) { synchronized (lock) { Preconditions.checkState(!isClosed, "The NettyShuffleEnvironment has already been shut down."); @@ -217,6 +219,7 @@ public class NettyShuffleEnvironment implements ShuffleEnvironment bufferPoolFactory) { super( owningTaskName, + partitionIndex, partitionId, partitionType, subpartitions, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java index e814191fb7..ccd3fa9ab9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java @@ -30,6 +30,7 @@ import org.apache.flink.runtime.io.network.partition.consumer.LocalInputChannel; import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel; import org.apache.flink.runtime.jobgraph.DistributionPattern; import org.apache.flink.runtime.taskexecutor.TaskExecutor; +import org.apache.flink.util.Preconditions; import org.apache.flink.util.function.FunctionWithException; import org.slf4j.Logger; @@ -76,6 +77,8 @@ public class ResultPartition implements ResultPartitionWriter, BufferPoolOwner { private final String owningTaskName; + private final int partitionIndex; + protected final ResultPartitionID partitionId; /** Type of this partition. Defines the concrete subpartition implementation to use. */ @@ -106,6 +109,7 @@ public class ResultPartition implements ResultPartitionWriter, BufferPoolOwner { public ResultPartition( String owningTaskName, + int partitionIndex, ResultPartitionID partitionId, ResultPartitionType partitionType, ResultSubpartition[] subpartitions, @@ -115,6 +119,8 @@ public class ResultPartition implements ResultPartitionWriter, BufferPoolOwner { FunctionWithException bufferPoolFactory) { this.owningTaskName = checkNotNull(owningTaskName); + Preconditions.checkArgument(0 <= partitionIndex, "The partition index must be positive."); + this.partitionIndex = partitionIndex; this.partitionId = checkNotNull(partitionId); this.partitionType = checkNotNull(partitionType); this.subpartitions = checkNotNull(subpartitions); @@ -152,6 +158,10 @@ public class ResultPartition implements ResultPartitionWriter, BufferPoolOwner { return partitionId; } + int getPartitionIndex() { + return partitionIndex; + } + @Override public int getNumberOfSubpartitions() { return subpartitions.length; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactory.java index e924bd0bbb..f52ffed133 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactory.java @@ -90,9 +90,11 @@ public class ResultPartitionFactory { public ResultPartition create( String taskNameWithSubtaskAndId, + int partitionIndex, ResultPartitionDeploymentDescriptor desc) { return create( taskNameWithSubtaskAndId, + partitionIndex, desc.getShuffleDescriptor().getResultPartitionID(), desc.getPartitionType(), desc.getNumberOfSubpartitions(), @@ -103,6 +105,7 @@ public class ResultPartitionFactory { @VisibleForTesting public ResultPartition create( String taskNameWithSubtaskAndId, + int partitionIndex, ResultPartitionID id, ResultPartitionType type, int numberOfSubpartitions, @@ -117,6 +120,7 @@ public class ResultPartitionFactory { ResultPartition partition = forcePartitionReleaseOnConsumption || !type.isBlocking() ? new ReleaseOnConsumptionResultPartition( taskNameWithSubtaskAndId, + partitionIndex, id, type, subpartitions, @@ -126,6 +130,7 @@ public class ResultPartitionFactory { bufferPoolFactory) : new ResultPartition( taskNameWithSubtaskAndId, + partitionIndex, id, type, subpartitions, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java index 31b184ac99..d139df00e2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.io.network.partition; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.runtime.checkpoint.channel.ResultSubpartitionInfo; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferConsumer; @@ -34,6 +35,9 @@ public abstract class ResultSubpartition { /** The index of the subpartition at the parent partition. */ protected final int index; + /** The info of the subpartition to identify it globally within a task. */ + protected final ResultSubpartitionInfo subpartitionInfo; + /** The parent partition this subpartition belongs to. */ protected final ResultPartition parent; @@ -42,6 +46,7 @@ public abstract class ResultSubpartition { public ResultSubpartition(int index, ResultPartition parent) { this.index = index; this.parent = parent; + this.subpartitionInfo = new ResultSubpartitionInfo(parent.getPartitionIndex(), index); } /** @@ -52,6 +57,11 @@ public abstract class ResultSubpartition { return parent.bufferCompressor != null && buffer.isBuffer() && buffer.readableBytes() > 0; } + @VisibleForTesting + ResultSubpartitionInfo getSubpartitionInfo() { + return subpartitionInfo; + } + /** * Gets the total numbers of buffers (data buffers plus events). */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java index 61703f856d..b96626b51b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java @@ -18,7 +18,9 @@ package org.apache.flink.runtime.io.network.partition.consumer; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.metrics.Counter; +import org.apache.flink.runtime.checkpoint.channel.InputChannelInfo; import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.execution.CancelTaskException; import org.apache.flink.runtime.io.network.buffer.Buffer; @@ -47,6 +49,9 @@ public abstract class InputChannel { protected final int channelIndex; + /** The info of the input channel to identify it globally within a task. */ + protected final InputChannelInfo channelInfo; + protected final ResultPartitionID partitionId; protected final SingleInputGate inputGate; @@ -88,6 +93,7 @@ public abstract class InputChannel { this.inputGate = checkNotNull(inputGate); this.channelIndex = channelIndex; + this.channelInfo = new InputChannelInfo(inputGate.getGateIndex(), channelIndex); this.partitionId = checkNotNull(partitionId); this.initialBackoff = initial; @@ -106,6 +112,11 @@ public abstract class InputChannel { return channelIndex; } + @VisibleForTesting + InputChannelInfo getChannelInfo() { + return channelInfo; + } + public ResultPartitionID getPartitionId() { return partitionId; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java index 677b320630..b59af428e0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java @@ -37,6 +37,7 @@ import org.apache.flink.runtime.jobgraph.DistributionPattern; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.shuffle.NettyShuffleDescriptor; +import org.apache.flink.util.Preconditions; import org.apache.flink.util.function.SupplierWithException; import org.slf4j.Logger; @@ -109,6 +110,8 @@ public class SingleInputGate extends InputGate { /** The name of the owning task, for logging purposes. */ private final String owningTaskName; + private final int gateIndex; + /** * The ID of the consumed intermediate result. Each input gate consumes partitions of the * intermediate result specified by this ID. This ID also identifies the input gate at the @@ -175,6 +178,7 @@ public class SingleInputGate extends InputGate { public SingleInputGate( String owningTaskName, + int gateIndex, IntermediateDataSetID consumedResultId, final ResultPartitionType consumedPartitionType, int consumedSubpartitionIndex, @@ -184,6 +188,8 @@ public class SingleInputGate extends InputGate { @Nullable BufferDecompressor bufferDecompressor) { this.owningTaskName = checkNotNull(owningTaskName); + Preconditions.checkArgument(0 <= gateIndex, "The gate index must be positive."); + this.gateIndex = gateIndex; this.consumedResultId = checkNotNull(consumedResultId); this.consumedPartitionType = checkNotNull(consumedPartitionType); @@ -254,6 +260,10 @@ public class SingleInputGate extends InputGate { return numberOfInputChannels; } + public int getGateIndex() { + return gateIndex; + } + /** * Returns the type of this input channel's consumed result partition. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java index a6d36418c8..2fc8b7252b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java @@ -108,6 +108,7 @@ public class SingleInputGateFactory { */ public SingleInputGate create( @Nonnull String owningTaskName, + int gateIndex, @Nonnull InputGateDeploymentDescriptor igdd, @Nonnull PartitionProducerStateProvider partitionProducerStateProvider, @Nonnull InputChannelMetrics metrics) { @@ -125,6 +126,7 @@ public class SingleInputGateFactory { SingleInputGate inputGate = new SingleInputGate( owningTaskName, + gateIndex, igdd.getConsumedResultId(), igdd.getConsumedPartitionType(), igdd.getConsumedSubpartitionIndex(), diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleEnvironment.java index e6a4802de1..215f1f803f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleEnvironment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleEnvironment.java @@ -31,6 +31,7 @@ import org.apache.flink.runtime.io.network.partition.consumer.InputGate; import java.io.IOException; import java.util.Collection; +import java.util.List; /** * Interface for the implementation of shuffle service local environment. @@ -121,11 +122,11 @@ public interface ShuffleEnvironment

    createResultPartitionWriters( ShuffleIOOwnerContext ownerContext, - Collection resultPartitionDeploymentDescriptors); + List resultPartitionDeploymentDescriptors); /** * Release local resources occupied by the given partitions. @@ -154,12 +155,12 @@ public interface ShuffleEnvironment

    createInputGates( ShuffleIOOwnerContext ownerContext, PartitionProducerStateProvider partitionProducerStateProvider, - Collection inputGateDeploymentDescriptors); + List inputGateDeploymentDescriptors); /** * Update a gate with the newly available partition information, previously unknown. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java index 1e1505cb80..30e7c8a1a3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java @@ -95,6 +95,7 @@ import java.lang.reflect.InvocationTargetException; import java.net.URL; import java.util.Collection; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; @@ -286,8 +287,8 @@ public class Task implements Runnable, TaskSlotPayload, TaskActions, PartitionPr AllocationID slotAllocationId, int subtaskIndex, int attemptNumber, - Collection resultPartitionDeploymentDescriptors, - Collection inputGateDeploymentDescriptors, + List resultPartitionDeploymentDescriptors, + List inputGateDeploymentDescriptors, int targetSlotNumber, MemoryManager memManager, IOManager ioManager, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorBuilder.java index 4b9a40735f..64ae2fd3af 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorBuilder.java @@ -34,8 +34,8 @@ import org.apache.flink.util.SerializedValue; import javax.annotation.Nullable; import java.io.IOException; -import java.util.Collection; import java.util.Collections; +import java.util.List; /** * Builder for {@link TaskDeploymentDescriptor}. @@ -48,8 +48,8 @@ public class TaskDeploymentDescriptorBuilder { private AllocationID allocationId; private int subtaskIndex; private int attemptNumber; - private Collection producedPartitions; - private Collection inputGates; + private List producedPartitions; + private List inputGates; private int targetSlotNumber; @Nullable @@ -116,12 +116,12 @@ public class TaskDeploymentDescriptorBuilder { } public TaskDeploymentDescriptorBuilder setProducedPartitions( - Collection producedPartitions) { + List producedPartitions) { this.producedPartitions = producedPartitions; return this; } - public TaskDeploymentDescriptorBuilder setInputGates(Collection inputGates) { + public TaskDeploymentDescriptorBuilder setInputGates(List inputGates) { this.inputGates = inputGates; return this; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java index a6572affd6..4230abc41b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java @@ -331,7 +331,9 @@ public class InputGateFairnessTest { int consumedSubpartitionIndex, int numberOfInputChannels) { - super(owningTaskName, + super( + owningTaskName, + 0, consumedResultId, ResultPartitionType.PIPELINED, consumedSubpartitionIndex, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionBuilder.java index b57452c88a..a0c9faabcb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionBuilder.java @@ -40,6 +40,8 @@ public class ResultPartitionBuilder { private BoundedBlockingSubpartitionType blockingSubpartitionType = BoundedBlockingSubpartitionType.AUTO; + private int partitionIndex = 0; + private int numberOfSubpartitions = 1; private int numTargetKeyGroups = 1; @@ -65,6 +67,11 @@ public class ResultPartitionBuilder { private String compressionCodec = "LZ4"; + public ResultPartitionBuilder setResultPartitionIndex(int partitionIndex) { + this.partitionIndex = partitionIndex; + return this; + } + public ResultPartitionBuilder setResultPartitionId(ResultPartitionID partitionId) { this.partitionId = partitionId; return this; @@ -167,6 +174,7 @@ public class ResultPartitionBuilder { return resultPartitionFactory.create( "Result Partition task", + partitionIndex, partitionId, partitionType, numberOfSubpartitions, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactoryTest.java index add46287e8..7f27dbd492 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactoryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactoryTest.java @@ -112,6 +112,6 @@ public class ResultPartitionFactoryTest extends TestLogger { true ); - return factory.create("test", descriptor); + return factory.create("test", 0, descriptor); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java index 02b6eed212..011aa72862 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.io.network.partition; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.checkpoint.channel.ResultSubpartitionInfo; import org.apache.flink.runtime.io.disk.FileChannelManager; import org.apache.flink.runtime.io.disk.FileChannelManagerImpl; import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; @@ -76,6 +77,27 @@ public class ResultPartitionTest { fileChannelManager.close(); } + @Test + public void testResultSubpartitionInfo() { + final int numPartitions = 2; + final int numSubpartitions = 3; + + for (int i = 0; i < numPartitions; i++) { + final ResultPartition partition = new ResultPartitionBuilder() + .setResultPartitionIndex(i) + .setNumberOfSubpartitions(numSubpartitions) + .build(); + + ResultSubpartition[] subpartitions = partition.getAllPartitions(); + for (int j = 0; j < subpartitions.length; j++) { + ResultSubpartitionInfo subpartitionInfo = subpartitions[j].getSubpartitionInfo(); + + assertEquals(i, subpartitionInfo.getPartitionIdx()); + assertEquals(j, subpartitionInfo.getSubPartitionIdx()); + } + } + } + /** * Tests the schedule or update consumers message sending behaviour depending on the relevant flags. */ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImplTest.java index 44325fe451..ec8af9c31d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImplTest.java @@ -38,6 +38,7 @@ import org.junit.Test; import java.io.IOException; import java.util.Collection; import java.util.Collections; +import java.util.List; import java.util.concurrent.CompletableFuture; import static org.hamcrest.CoreMatchers.not; @@ -168,7 +169,7 @@ public class TaskExecutorPartitionTrackerImplTest extends TestLogger { } @Override - public Collection createResultPartitionWriters(ShuffleIOOwnerContext ownerContext, Collection resultPartitionDeploymentDescriptors) { + public Collection createResultPartitionWriters(ShuffleIOOwnerContext ownerContext, List resultPartitionDeploymentDescriptors) { return backingShuffleEnvironment.createResultPartitionWriters(ownerContext, resultPartitionDeploymentDescriptors); } @@ -186,7 +187,7 @@ public class TaskExecutorPartitionTrackerImplTest extends TestLogger { } @Override - public Collection createInputGates(ShuffleIOOwnerContext ownerContext, PartitionProducerStateProvider partitionProducerStateProvider, Collection inputGateDeploymentDescriptors) { + public Collection createInputGates(ShuffleIOOwnerContext ownerContext, PartitionProducerStateProvider partitionProducerStateProvider, List inputGateDeploymentDescriptors) { return backingShuffleEnvironment.createInputGates(ownerContext, partitionProducerStateProvider, inputGateDeploymentDescriptors); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateBuilder.java index 5e46c8e96f..76317664d3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateBuilder.java @@ -42,6 +42,8 @@ public class SingleInputGateBuilder { private int consumedSubpartitionIndex = 0; + private int gateIndex = 0; + private int numberOfChannels = 1; private PartitionProducerStateProvider partitionProducerStateProvider = NO_OP_PRODUCER_CHECKER; @@ -69,6 +71,11 @@ public class SingleInputGateBuilder { return this; } + SingleInputGateBuilder setSingleInputGateIndex(int gateIndex) { + this.gateIndex = gateIndex; + return this; + } + public SingleInputGateBuilder setNumberOfChannels(int numberOfChannels) { this.numberOfChannels = numberOfChannels; return this; @@ -98,6 +105,7 @@ public class SingleInputGateBuilder { public SingleInputGate build() { return new SingleInputGate( "Single Input Gate", + gateIndex, intermediateDataSetID, partitionType, consumedSubpartitionIndex, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java index 5606f9e927..df920dc33a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java @@ -21,6 +21,7 @@ package org.apache.flink.runtime.io.network.partition.consumer; import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.core.memory.MemorySegmentFactory; import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; +import org.apache.flink.runtime.checkpoint.channel.InputChannelInfo; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; import org.apache.flink.runtime.event.TaskEvent; @@ -385,6 +386,7 @@ public class SingleInputGateTest extends InputGateTestBase { netEnv.getNetworkBufferPool()) .create( "TestTask", + 0, gateDesc, SingleInputGateBuilder.NO_OP_PRODUCER_CHECKER, InputChannelTestUtils.newUnregisteredInputChannelMetrics()); @@ -653,6 +655,27 @@ public class SingleInputGateTest extends InputGateTestBase { } } + @Test + public void testSingleInputGateInfo() { + final int numSingleInputGates = 2; + final int numInputChannels = 3; + + for (int i = 0; i < numSingleInputGates; i++) { + final SingleInputGate gate = new SingleInputGateBuilder() + .setSingleInputGateIndex(i) + .setNumberOfChannels(numInputChannels) + .build(); + + int channelCounter = 0; + for (InputChannel inputChannel : gate.getInputChannels().values()) { + InputChannelInfo channelInfo = inputChannel.getChannelInfo(); + + assertEquals(i, channelInfo.getGateIdx()); + assertEquals(channelCounter++, channelInfo.getInputChannelIdx()); + } + } + } + // --------------------------------------------------------------------------------------------- private static Map createInputGateWithLocalChannels( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java index e82e9b3c3e..d26601c2cc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java @@ -82,6 +82,7 @@ import java.io.IOException; import java.net.URL; import java.util.Collection; import java.util.Collections; +import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; @@ -753,8 +754,8 @@ public class TaskExecutorSubmissionTest extends TestLogger { ExecutionAttemptID eid, Class abstractInvokable, int maxNumberOfSubtasks, - Collection producedPartitions, - Collection inputGates + List producedPartitions, + List inputGates ) throws IOException { Preconditions.checkNotNull(producedPartitions); Preconditions.checkNotNull(inputGates); @@ -782,8 +783,8 @@ public class TaskExecutorSubmissionTest extends TestLogger { Configuration jobConfiguration, Configuration taskConfiguration, String invokableClassName, - Collection producedPartitions, - Collection inputGates, + List producedPartitions, + List inputGates, Collection requiredJarFiles, Collection requiredClasspaths, int targetSlotNumber) throws IOException { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java index 41ae741f19..a5750826e0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java @@ -65,9 +65,9 @@ import javax.annotation.Nonnull; import java.io.IOException; import java.net.InetSocketAddress; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; @@ -255,7 +255,7 @@ public class TaskTest extends TestLogger { shuffleDescriptor, 1, false); - testExecutionFailsInNetworkRegistration(Collections.singleton(dummyPartition), Collections.emptyList()); + testExecutionFailsInNetworkRegistration(Collections.singletonList(dummyPartition), Collections.emptyList()); } @Test @@ -266,12 +266,12 @@ public class TaskTest extends TestLogger { ResultPartitionType.PIPELINED, 0, new ShuffleDescriptor[] { dummyChannel }); - testExecutionFailsInNetworkRegistration(Collections.emptyList(), Collections.singleton(dummyGate)); + testExecutionFailsInNetworkRegistration(Collections.emptyList(), Collections.singletonList(dummyGate)); } private void testExecutionFailsInNetworkRegistration( - Collection resultPartitions, - Collection inputGates) throws Exception { + List resultPartitions, + List inputGates) throws Exception { final String errorMessage = "Network buffer pool has already been destroyed."; final ResultPartitionConsumableNotifier consumableNotifier = new NoOpResultPartitionConsumableNotifier(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TestTaskBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TestTaskBuilder.java index 2c9a5354ee..2c5f341a29 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TestTaskBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TestTaskBuilder.java @@ -61,6 +61,7 @@ import org.apache.flink.util.SerializedValue; import java.lang.reflect.Field; import java.util.Collection; import java.util.Collections; +import java.util.List; import java.util.concurrent.Executor; import static org.mockito.Mockito.mock; @@ -81,8 +82,8 @@ public final class TestTaskBuilder { private Configuration taskManagerConfig = new Configuration(); private ExecutionConfig executionConfig = new ExecutionConfig(); private Collection requiredJarFileBlobKeys = Collections.emptyList(); - private Collection resultPartitions = Collections.emptyList(); - private Collection inputGates = Collections.emptyList(); + private List resultPartitions = Collections.emptyList(); + private List inputGates = Collections.emptyList(); private JobID jobId = new JobID(); private AllocationID allocationID = new AllocationID(); private ExecutionAttemptID executionAttemptId = new ExecutionAttemptID(); @@ -141,12 +142,12 @@ public final class TestTaskBuilder { return this; } - public TestTaskBuilder setResultPartitions(Collection resultPartitions) { + public TestTaskBuilder setResultPartitions(List resultPartitions) { this.resultPartitions = resultPartitions; return this; } - public TestTaskBuilder setInputGates(Collection inputGates) { + public TestTaskBuilder setInputGates(List inputGates) { this.inputGates = inputGates; return this; } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBenchmarkEnvironment.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBenchmarkEnvironment.java index 60a4ba201b..a7da7ea837 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBenchmarkEnvironment.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBenchmarkEnvironment.java @@ -264,10 +264,11 @@ public class StreamNetworkBenchmarkEnvironment { private InputGate createInputGateWithMetrics( SingleInputGateFactory gateFactory, InputGateDeploymentDescriptor gateDescriptor, - int channelIndex) { + int gateIndex) { final SingleInputGate singleGate = gateFactory.create( - "receiving task[" + channelIndex + "]", + "receiving task[" + gateIndex + "]", + gateIndex, gateDescriptor, SingleInputGateBuilder.NO_OP_PRODUCER_CHECKER, InputChannelTestUtils.newUnregisteredInputChannelMetrics()); -- Gitee From ddec07323701a9998e854c378468617bafffa1f5 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Mon, 16 Mar 2020 12:13:35 +0800 Subject: [PATCH 182/885] [hotfix][task][tests] Remove useless import from TaskTest class This cloese #11400. --- .../test/java/org/apache/flink/runtime/taskmanager/TaskTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java index a5750826e0..aa5501e20f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java @@ -83,7 +83,6 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; -- Gitee From 8bddbc2a254d15675c6bdc8ecfabdc731b6b2f66 Mon Sep 17 00:00:00 2001 From: Matt Welke Date: Sun, 15 Mar 2020 18:08:14 -0400 Subject: [PATCH 183/885] Fix typo --- docs/getting-started/walkthroughs/datastream_api.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/getting-started/walkthroughs/datastream_api.md b/docs/getting-started/walkthroughs/datastream_api.md index 7aef86e804..7281a65767 100644 --- a/docs/getting-started/walkthroughs/datastream_api.md +++ b/docs/getting-started/walkthroughs/datastream_api.md @@ -317,7 +317,7 @@ val transactions: DataStream[Transaction] = env #### Partitioning Events & Detecting Fraud -The `transactions` stream contains a lot of transactions from a large number of users, such that it needs to be processed in parallel my multiple fraud detection tasks. Since fraud occurs on a per-account basis, you must ensure that all transactions for the same account are processed by the same parallel task of the fraud detector operator. +The `transactions` stream contains a lot of transactions from a large number of users, such that it needs to be processed in parallel by multiple fraud detection tasks. Since fraud occurs on a per-account basis, you must ensure that all transactions for the same account are processed by the same parallel task of the fraud detector operator. To ensure that the same physical task processes all records for a particular key, you can partition a stream using `DataStream#keyBy`. The `process()` call adds an operator that applies a function to each partitioned element in the stream. -- Gitee From cd51ffc00de58eae6baa6f2e7c31d9fdf54016b4 Mon Sep 17 00:00:00 2001 From: Rui Li Date: Wed, 18 Mar 2020 16:50:06 +0800 Subject: [PATCH 184/885] [FLINK-16433][table-api] TableEnvironmentImpl doesn't clear buffered operations when it fails This closes #11317 --- .../flink/table/api/TableEnvironment.java | 5 ++++ .../table/api/TableEnvironmentITCase.scala | 23 ++++++++++++++++++- 2 files changed, 27 insertions(+), 1 deletion(-) diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java index 37a0dd1990..f086dd8f88 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java @@ -858,6 +858,11 @@ public interface TableEnvironment { * other hand some values might be evaluated according to the state from the time when * this method is called (e.g. timeZone). * + *

    Once the execution finishes, any previously defined DMLs will be cleared, no matter + * whether the execution succeeds or not. Therefore, if you want to retry in case of + * failures, you have to re-define the DMLs, i.e. by calling {@link #sqlUpdate(String)}, + * before you call this method again. + * * @param jobName Desired name of the job * @return The result of the job execution, containing elapsed time and accumulators. * @throws Exception which occurs during job execution. diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentITCase.scala index 24ee0c203e..5501e1a468 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentITCase.scala @@ -38,7 +38,7 @@ import org.junit.Assert.{assertEquals, assertFalse, assertTrue} import org.junit.rules.TemporaryFolder import org.junit.runner.RunWith import org.junit.runners.Parameterized -import org.junit.{Before, Rule, Test} +import org.junit.{Assert, Before, Rule, Test} import _root_.java.io.File import _root_.java.util @@ -249,6 +249,27 @@ class TableEnvironmentITCase(tableEnvName: String, isStreaming: Boolean) { assertEquals(getExpectedLastValues.sorted, sink.getAppendResults.sorted) } + @Test + def testClearOperation(): Unit = { + val tableEnv = TableEnvironmentImpl.create(settings) + tableEnv.sqlUpdate("create table dest1(x map) with('connector' = 'COLLECTION')") + tableEnv.sqlUpdate("create table dest2(x int) with('connector' = 'COLLECTION')") + tableEnv.sqlUpdate("create table src(x int) with('connector' = 'COLLECTION')") + + try { + // it would fail due to query and sink type mismatch + tableEnv.sqlUpdate("insert into dest1 select count(*) from src") + tableEnv.execute("insert dest1") + Assert.fail("insert is expected to fail due to type mismatch") + } catch { + case _: Exception => //expected + } + + tableEnv.sqlUpdate("drop table dest1") + tableEnv.sqlUpdate("insert into dest2 select x from src") + tableEnv.execute("insert dest2") + } + private def registerCsvTableSink( tEnv: TableEnvironment, fieldNames: Array[String], -- Gitee From a7dccf1945d7b2b4abc726c4ca213677df17d8bc Mon Sep 17 00:00:00 2001 From: Hequn Cheng Date: Wed, 18 Mar 2020 17:40:50 +0800 Subject: [PATCH 185/885] [FLINK-16565][python][ml] Make Pipeline Json compitable between Java and Python if all PipelineStages are Java ones (#11420) --- flink-python/pyflink/ml/api/base.py | 69 +++++++++++++++++-- .../pyflink/ml/tests/test_pipeline.py | 28 ++++++++ .../pyflink/ml/tests/test_pipeline_it_case.py | 31 +++++++++ 3 files changed, 121 insertions(+), 7 deletions(-) diff --git a/flink-python/pyflink/ml/api/base.py b/flink-python/pyflink/ml/api/base.py index 8888df3a83..3674e6cc0f 100644 --- a/flink-python/pyflink/ml/api/base.py +++ b/flink-python/pyflink/ml/api/base.py @@ -23,7 +23,8 @@ from abc import ABCMeta, abstractmethod from pyflink.table.table_environment import TableEnvironment from pyflink.table.table import Table from pyflink.ml.api.param import WithParams, Params -from py4j.java_gateway import get_field +from pyflink.java_gateway import get_gateway +from py4j.java_gateway import get_field, get_java_class as java_class class PipelineStage(WithParams): @@ -265,11 +266,65 @@ class Pipeline(Estimator, Model, Transformer): return input def to_json(self) -> str: - import jsonpickle - return str(jsonpickle.encode(self, keys=True)) + """ + If all PipelineStages in this Pipeline are Java ones, this method will return a + Java json string, which can be loaded either from a Python Pipeline or a Java Pipeline, + otherwise, it returns a Python json string which can only be loaded from a Python Pipeline. + """ + # if all PipelineStages are Java ones, we use Java toJson() to generate Json string + # so that the string can also be loaded from Java side. + if all([type(stage) in [JavaTransformer, JavaEstimator, JavaModel] + for stage in self.get_stages()]): + j_pipeline = get_gateway().jvm.org.apache.flink.ml.api.core.Pipeline() + for stage in self.get_stages(): + stage._convert_params_to_java(stage._j_obj) + j_pipeline.appendStage(stage._j_obj) + return j_pipeline.toJson() + else: + import jsonpickle + return str(jsonpickle.encode(self, keys=True)) def load_json(self, json: str) -> None: - import jsonpickle - pipeline = jsonpickle.decode(json, keys=True) - for stage in pipeline.get_stages(): - self.append_stage(stage) + """ + This method can either load from a Java Pipeline json or a Python Pipeline json. + """ + # noinspection PyBroadException + try: + # try to load json with Python method + import jsonpickle + pipeline = jsonpickle.decode(json, keys=True) + for stage in pipeline.get_stages(): + self.append_stage(stage) + except Exception as outer_e: + try: + # if can't load json with Python method, try to load with Java method + gw = get_gateway() + j_pipeline = gw.jvm.org.apache.flink.ml.api.core.Pipeline() + j_pipeline.loadJson(json) + + for j_stage in j_pipeline.getStages(): + j_stage_class = j_stage.getClass() + j_transformer_class = java_class( + gw.jvm.org.apache.flink.ml.api.core.Transformer) + j_estimator_class = java_class(gw.jvm.org.apache.flink.ml.api.core.Estimator) + j_model_class = java_class(gw.jvm.org.apache.flink.ml.api.core.Model) + if j_transformer_class.isAssignableFrom(j_stage_class): + self.append_stage(JavaTransformer(j_stage)) + elif j_estimator_class.isAssignableFrom(j_stage_class): + self.append_stage(JavaEstimator(j_stage)) + elif j_model_class.isAssignableFrom(j_stage_class): + self.append_stage(JavaModel(j_stage)) + else: + raise TypeError( + "Unexpected Java PipelineStage %s. Class should be a %s, " + "%s or a %s." % + (j_stage_class.getCanonicalName(), + j_transformer_class.getCanonicalName(), + j_estimator_class.getCanonicalName(), + j_model_class.getCanonicalName())) + except Exception as inner_e: + raise RuntimeError( + "Cannot load the JSON as either a Java Pipeline or a Python Pipeline.\n" + "Python Pipeline load failed due to: %s.\n" + "Java Pipeline load failed due to: %s." % + (outer_e, inner_e)) diff --git a/flink-python/pyflink/ml/tests/test_pipeline.py b/flink-python/pyflink/ml/tests/test_pipeline.py index 31c3068741..19707fc2c8 100644 --- a/flink-python/pyflink/ml/tests/test_pipeline.py +++ b/flink-python/pyflink/ml/tests/test_pipeline.py @@ -21,6 +21,7 @@ from pyflink.ml.api import JavaTransformer, Transformer, Estimator, Model, \ Pipeline, JavaEstimator, JavaModel from pyflink.ml.api.param.base import WithParams, ParamInfo, TypeConverters from pyflink import keyword +from pyflink.testing.test_case_utils import MLTestCase class PipelineTest(unittest.TestCase): @@ -72,6 +73,33 @@ class PipelineTest(unittest.TestCase): self.assertEqual("a_ja_mb_mjb_mc_d", PipelineTest.describe_pipeline(pipeline_model)) +class ValidationPipelineTest(MLTestCase): + + def test_pipeline_from_invalid_json(self): + invalid_json = '[a:aa]' + + # load json + p = Pipeline() + with self.assertRaises(RuntimeError) as context: + p.load_json(invalid_json) + exception_str = str(context.exception) + + # only assert key error message as the whole message is very long. + self.assertTrue( + 'Cannot load the JSON as either a Java Pipeline or a Python Pipeline.' + in exception_str) + self.assertTrue( + 'Python Pipeline load failed due to: Expecting value: line 1 column 2 (char 1).' + in exception_str) + self.assertTrue( + 'Java Pipeline load failed due to: An error occurred while calling o0.loadJson.' + in exception_str) + self.assertTrue( + 'Caused by: org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.' + 'JsonParseException: Unrecognized token \'a\'' + in exception_str) + + class SelfDescribe(WithParams): self_desc = ParamInfo("selfDesc", "selfDesc", type_converter=TypeConverters.to_string) diff --git a/flink-python/pyflink/ml/tests/test_pipeline_it_case.py b/flink-python/pyflink/ml/tests/test_pipeline_it_case.py index 655d20c4df..c291b8b0d0 100644 --- a/flink-python/pyflink/ml/tests/test_pipeline_it_case.py +++ b/flink-python/pyflink/ml/tests/test_pipeline_it_case.py @@ -169,3 +169,34 @@ class PythonPipelineTest(MLTestCase): # the first input is false since 0 + 0 is smaller than the max_sum 14. # the second input is true since 12 + 3 is bigger than the max_sum 14. self.assert_equals(actual, ["false", "true"]) + + def test_pipeline_from_and_to_java_json(self): + # json generated from Java api + java_json = '[{"stageClassName":"org.apache.flink.ml.pipeline.' \ + 'UserDefinedPipelineStages$SelectColumnTransformer",' \ + '"stageJson":"{\\"selectedCols\\":\\"[\\\\\\"a\\\\\\",' \ + '\\\\\\"b\\\\\\"]\\"}"}]' + + # load json + p = Pipeline() + p.load_json(java_json) + python_json = p.to_json() + + t_env = MLEnvironmentFactory().get_default().get_stream_table_environment() + + table_sink = source_sink_utils.TestAppendSink( + ['a', 'b'], [DataTypes.BIGINT(), DataTypes.BIGINT()]) + t_env.register_table_sink("TestJsonResults", table_sink) + + source_table = t_env.from_elements([(1, 2, 3, 4), (4, 3, 2, 1)], ['a', 'b', 'c', 'd']) + transformer = p.get_stages()[0] + transformer\ + .transform(t_env, source_table)\ + .insert_into("TestJsonResults") + + # execute + t_env.execute('JavaPipelineITCase') + actual = source_sink_utils.results() + + self.assert_equals(actual, ["1,2", "4,3"]) + self.assertEqual(python_json, java_json) -- Gitee From b83060dff6d403b6994b6646b3f29a374f599530 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Mon, 16 Mar 2020 16:11:31 +0800 Subject: [PATCH 186/885] [FLINK-16606][python] Throw exceptions for the data types which are not currently supported --- flink-python/pyflink/table/tests/test_calc.py | 26 ++--- .../pyflink/table/tests/test_descriptor.py | 2 +- .../pyflink/table/tests/test_types.py | 6 +- flink-python/pyflink/table/tests/test_udf.py | 16 +-- flink-python/pyflink/table/types.py | 108 +++++++++++++++--- 5 files changed, 114 insertions(+), 44 deletions(-) diff --git a/flink-python/pyflink/table/tests/test_calc.py b/flink-python/pyflink/table/tests/test_calc.py index 1580cc73ed..eed505ab8f 100644 --- a/flink-python/pyflink/table/tests/test_calc.py +++ b/flink-python/pyflink/table/tests/test_calc.py @@ -64,18 +64,17 @@ class StreamTableCalcTests(PyFlinkStreamTableTestCase): def test_from_element(self): t_env = self.t_env field_names = ["a", "b", "c", "d", "e", "f", "g", "h", - "i", "j", "k", "l", "m", "n", "o", "p", "q", "r", "s"] + "i", "j", "k", "l", "m", "n", "o", "p", "q", "r"] field_types = [DataTypes.BIGINT(), DataTypes.DOUBLE(), DataTypes.STRING(), DataTypes.STRING(), DataTypes.DATE(), DataTypes.TIME(), - DataTypes.TIMESTAMP(), - DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(), - DataTypes.INTERVAL(DataTypes.DAY(), DataTypes.SECOND()), + DataTypes.TIMESTAMP(3), + DataTypes.INTERVAL(DataTypes.SECOND(3)), DataTypes.ARRAY(DataTypes.DOUBLE()), DataTypes.ARRAY(DataTypes.DOUBLE(False)), DataTypes.ARRAY(DataTypes.STRING()), DataTypes.ARRAY(DataTypes.DATE()), - DataTypes.DECIMAL(10, 0), + DataTypes.DECIMAL(38, 18), DataTypes.ROW([DataTypes.FIELD("a", DataTypes.BIGINT()), DataTypes.FIELD("b", DataTypes.DOUBLE())]), DataTypes.MAP(DataTypes.STRING(), DataTypes.DOUBLE()), @@ -89,7 +88,7 @@ class StreamTableCalcTests(PyFlinkStreamTableTestCase): t_env.register_table_sink("Results", table_sink) t = t_env.from_elements( [(1, 1.0, "hi", "hello", datetime.date(1970, 1, 2), datetime.time(1, 0, 0), - datetime.datetime(1970, 1, 2, 0, 0), datetime.datetime(1970, 1, 2, 0, 0), + datetime.datetime(1970, 1, 2, 0, 0), datetime.timedelta(days=1, microseconds=10), [1.0, None], array.array("d", [1.0, 2.0]), ["abc"], [datetime.date(1970, 1, 2)], Decimal(1), Row("a", "b")(1, 2.0), @@ -101,7 +100,7 @@ class StreamTableCalcTests(PyFlinkStreamTableTestCase): actual = source_sink_utils.results() expected = ['1,1.0,hi,hello,1970-01-02,01:00:00,1970-01-02 00:00:00.0,' - '1970-01-02 00:00:00.0,86400000,[1.0, null],[1.0, 2.0],[abc],[1970-01-02],' + '86400000,[1.0, null],[1.0, 2.0],[abc],[1970-01-02],' '1,1,2.0,{key=1.0},[65, 66, 67, 68],[1.0, 2.0],[3.0, 4.0]'] self.assert_equals(actual, expected) @@ -110,18 +109,17 @@ class StreamTableCalcTests(PyFlinkStreamTableTestCase): .new_instance().use_blink_planner() .in_batch_mode().build()) field_names = ["a", "b", "c", "d", "e", "f", "g", "h", - "i", "j", "k", "l", "m", "n", "o", "p", "q", "r"] + "i", "j", "k", "l", "m", "n", "o", "p", "q"] field_types = [DataTypes.BIGINT(), DataTypes.DOUBLE(), DataTypes.STRING(), DataTypes.STRING(), DataTypes.DATE(), DataTypes.TIME(), - DataTypes.TIMESTAMP(), - DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(), - DataTypes.INTERVAL(DataTypes.DAY(), DataTypes.SECOND()), + DataTypes.TIMESTAMP(3), + DataTypes.INTERVAL(DataTypes.SECOND(3)), DataTypes.ARRAY(DataTypes.DOUBLE()), DataTypes.ARRAY(DataTypes.DOUBLE(False)), DataTypes.ARRAY(DataTypes.STRING()), DataTypes.ARRAY(DataTypes.DATE()), - DataTypes.DECIMAL(10, 0), + DataTypes.DECIMAL(38, 18), DataTypes.ROW([DataTypes.FIELD("a", DataTypes.BIGINT()), DataTypes.FIELD("b", DataTypes.DOUBLE())]), DataTypes.MAP(DataTypes.STRING(), DataTypes.DOUBLE()), @@ -135,7 +133,7 @@ class StreamTableCalcTests(PyFlinkStreamTableTestCase): t_env.register_table_sink("Results", table_sink) t = t_env.from_elements( [(1, 1.0, "hi", "hello", datetime.date(1970, 1, 2), datetime.time(1, 0, 0), - datetime.datetime(1970, 1, 2, 0, 0), datetime.datetime(1970, 1, 2, 0, 0), + datetime.datetime(1970, 1, 2, 0, 0), datetime.timedelta(days=1, microseconds=10), [1.0, None], array.array("d", [1.0, 2.0]), ["abc"], [datetime.date(1970, 1, 2)], Decimal(1), Row("a", "b")(1, 2.0), @@ -147,7 +145,7 @@ class StreamTableCalcTests(PyFlinkStreamTableTestCase): actual = source_sink_utils.results() expected = ['1,1.0,hi,hello,1970-01-02,01:00:00,1970-01-02 00:00:00.0,' - '1970-01-02 00:00:00.0,86400000,[1.0, null],[1.0, 2.0],[abc],[1970-01-02],' + '86400000,[1.0, null],[1.0, 2.0],[abc],[1970-01-02],' '1.000000000000000000,1,2.0,{key=1.0},[65, 66, 67, 68],[3.0, 4.0]'] self.assert_equals(actual, expected) diff --git a/flink-python/pyflink/table/tests/test_descriptor.py b/flink-python/pyflink/table/tests/test_descriptor.py index 82e47aec89..002f499866 100644 --- a/flink-python/pyflink/table/tests/test_descriptor.py +++ b/flink-python/pyflink/table/tests/test_descriptor.py @@ -766,7 +766,7 @@ class SchemaDescriptorTests(PyFlinkTestCase): .field("int_field", DataTypes.INT())\ .field("long_field", DataTypes.BIGINT())\ .field("string_field", DataTypes.STRING())\ - .field("timestamp_field", DataTypes.TIMESTAMP())\ + .field("timestamp_field", DataTypes.TIMESTAMP(3))\ .field("time_field", DataTypes.TIME())\ .field("date_field", DataTypes.DATE())\ .field("double_field", DataTypes.DOUBLE())\ diff --git a/flink-python/pyflink/table/tests/test_types.py b/flink-python/pyflink/table/tests/test_types.py index a8490ab853..415224dfb8 100644 --- a/flink-python/pyflink/table/tests/test_types.py +++ b/flink-python/pyflink/table/tests/test_types.py @@ -790,7 +790,7 @@ class DataTypeConvertTests(unittest.TestCase): DataTypes.DOUBLE(), DataTypes.DATE(), DataTypes.TIME(), - DataTypes.TIMESTAMP()] + DataTypes.TIMESTAMP(3)] java_types = [_to_java_type(item) for item in test_types] @@ -802,7 +802,7 @@ class DataTypeConvertTests(unittest.TestCase): gateway = get_gateway() JDataTypes = gateway.jvm.DataTypes java_types = [JDataTypes.TIME(3).notNull(), - JDataTypes.TIMESTAMP().notNull(), + JDataTypes.TIMESTAMP(3).notNull(), JDataTypes.VARBINARY(100).notNull(), JDataTypes.BINARY(2).notNull(), JDataTypes.VARCHAR(30).notNull(), @@ -812,7 +812,7 @@ class DataTypeConvertTests(unittest.TestCase): converted_python_types = [_from_java_type(item) for item in java_types] expected = [DataTypes.TIME(3, False), - DataTypes.TIMESTAMP().not_null(), + DataTypes.TIMESTAMP(3).not_null(), DataTypes.VARBINARY(100, False), DataTypes.BINARY(2, False), DataTypes.VARCHAR(30, False), diff --git a/flink-python/pyflink/table/tests/test_udf.py b/flink-python/pyflink/table/tests/test_udf.py index 474e0ee66c..e12b71ac6d 100644 --- a/flink-python/pyflink/table/tests/test_udf.py +++ b/flink-python/pyflink/table/tests/test_udf.py @@ -176,14 +176,14 @@ class UserDefinedFunctionTests(object): DataTypes.SMALLINT(), DataTypes.INT(), DataTypes.BIGINT(), - DataTypes.DECIMAL(20, 10), + DataTypes.DECIMAL(38, 18), DataTypes.FLOAT(), DataTypes.DOUBLE(), DataTypes.BOOLEAN(), DataTypes.STRING(), DataTypes.DATE(), DataTypes.TIME(), - DataTypes.TIMESTAMP()], + DataTypes.TIMESTAMP(3)], result_type=DataTypes.BIGINT())) self.t_env.register_function( @@ -388,10 +388,10 @@ class UserDefinedFunctionTests(object): "date_func", udf(date_func, [DataTypes.DATE()], DataTypes.DATE())) self.t_env.register_function( - "time_func", udf(time_func, [DataTypes.TIME(3)], DataTypes.TIME(3))) + "time_func", udf(time_func, [DataTypes.TIME()], DataTypes.TIME())) self.t_env.register_function( - "timestamp_func", udf(timestamp_func, [DataTypes.TIMESTAMP()], DataTypes.TIMESTAMP())) + "timestamp_func", udf(timestamp_func, [DataTypes.TIMESTAMP(3)], DataTypes.TIMESTAMP(3))) self.t_env.register_function( "array_func", udf(array_func, [DataTypes.ARRAY(DataTypes.ARRAY(DataTypes.BIGINT()))], @@ -414,8 +414,8 @@ class UserDefinedFunctionTests(object): [DataTypes.BIGINT(), DataTypes.BIGINT(), DataTypes.TINYINT(), DataTypes.BOOLEAN(), DataTypes.SMALLINT(), DataTypes.INT(), DataTypes.FLOAT(), DataTypes.DOUBLE(), DataTypes.BYTES(), - DataTypes.STRING(), DataTypes.DATE(), DataTypes.TIME(3), - DataTypes.TIMESTAMP(), DataTypes.ARRAY(DataTypes.BIGINT()), + DataTypes.STRING(), DataTypes.DATE(), DataTypes.TIME(), + DataTypes.TIMESTAMP(3), DataTypes.ARRAY(DataTypes.BIGINT()), DataTypes.MAP(DataTypes.BIGINT(), DataTypes.STRING()), DataTypes.DECIMAL(38, 18), DataTypes.DECIMAL(38, 18)]) self.t_env.register_table_sink("Results", table_sink) @@ -441,8 +441,8 @@ class UserDefinedFunctionTests(object): DataTypes.FIELD("i", DataTypes.BYTES()), DataTypes.FIELD("j", DataTypes.STRING()), DataTypes.FIELD("k", DataTypes.DATE()), - DataTypes.FIELD("l", DataTypes.TIME(3)), - DataTypes.FIELD("m", DataTypes.TIMESTAMP()), + DataTypes.FIELD("l", DataTypes.TIME()), + DataTypes.FIELD("m", DataTypes.TIMESTAMP(3)), DataTypes.FIELD("n", DataTypes.ARRAY(DataTypes.ARRAY(DataTypes.BIGINT()))), DataTypes.FIELD("o", DataTypes.MAP(DataTypes.BIGINT(), DataTypes.STRING())), DataTypes.FIELD("p", DataTypes.DECIMAL(38, 18)), diff --git a/flink-python/pyflink/table/types.py b/flink-python/pyflink/table/types.py index f099dd3bef..d3ab0f2f48 100644 --- a/flink-python/pyflink/table/types.py +++ b/flink-python/pyflink/table/types.py @@ -1621,33 +1621,77 @@ def _to_java_type(data_type): BigIntType: Types.LONG(), FloatType: Types.FLOAT(), DoubleType: Types.DOUBLE(), - DecimalType: Types.DECIMAL(), DateType: Types.SQL_DATE(), - TimeType: Types.SQL_TIME(), - TimestampType: Types.SQL_TIMESTAMP(), - LocalZonedTimestampType: Types.SQL_TIMESTAMP(), - CharType: Types.STRING(), - VarCharType: Types.STRING(), - BinaryType: Types.PRIMITIVE_ARRAY(Types.BYTE()), - VarBinaryType: Types.PRIMITIVE_ARRAY(Types.BYTE()) } - # NullType - if isinstance(data_type, NullType): - # null type is still not supported in Java - raise NotImplementedError - # basic types - elif type(data_type) in _python_java_types_mapping: + if type(data_type) in _python_java_types_mapping: return _python_java_types_mapping[type(data_type)] + # DecimalType + elif isinstance(data_type, DecimalType): + if data_type.precision == 38 and data_type.scale == 18: + return Types.DECIMAL() + else: + raise TypeError("The precision must be 38 and the scale must be 18 for DecimalType, " + "got %s" % repr(data_type)) + + # TimeType + elif isinstance(data_type, TimeType): + if data_type.precision == 0: + return Types.SQL_TIME() + else: + raise TypeError("The precision must be 0 for TimeType, got %s" % repr(data_type)) + + # TimestampType + elif isinstance(data_type, TimestampType): + if data_type.precision == 3: + return Types.SQL_TIMESTAMP() + else: + raise TypeError("The precision must be 3 for TimestampType, got %s" % repr(data_type)) + + # LocalZonedTimestampType + elif isinstance(data_type, LocalZonedTimestampType): + if data_type.precision == 3: + return gateway.jvm.org.apache.flink.api.common.typeinfo.Types.INSTANT + else: + raise TypeError("The precision must be 3 for LocalZonedTimestampType, got %s" + % repr(data_type)) + + # VarCharType + elif isinstance(data_type, VarCharType): + if data_type.length == 0x7fffffff: + return Types.STRING() + else: + raise TypeError("The length limit must be 0x7fffffff(2147483647) for VarCharType, " + "got %s" % repr(data_type)) + + # VarBinaryType + elif isinstance(data_type, VarBinaryType): + if data_type.length == 0x7fffffff: + return Types.PRIMITIVE_ARRAY(Types.BYTE()) + else: + raise TypeError("The length limit must be 0x7fffffff(2147483647) for VarBinaryType, " + "got %s" % repr(data_type)) + # YearMonthIntervalType elif isinstance(data_type, YearMonthIntervalType): - return Types.INTERVAL_MONTHS() + if data_type.resolution == YearMonthIntervalType.YearMonthResolution.MONTH and \ + data_type.precision == 2: + return Types.INTERVAL_MONTHS() + else: + raise TypeError("The resolution must be YearMonthResolution.MONTH and the precision " + "must be 2 for YearMonthIntervalType, got %s" % repr(data_type)) # DayTimeIntervalType elif isinstance(data_type, DayTimeIntervalType): - return Types.INTERVAL_MILLIS() + if data_type.resolution == DayTimeIntervalType.DayTimeResolution.SECOND and \ + data_type.day_precision == 2 and data_type.fractional_precision == 3: + return Types.INTERVAL_MILLIS() + else: + raise TypeError("The resolution must be DayTimeResolution.SECOND, the day_precision " + "must be 2 and the fractional_precision must be 3 for " + "DayTimeIntervalType, got %s" % repr(data_type)) # ArrayType elif isinstance(data_type, ArrayType): @@ -1677,7 +1721,7 @@ def _to_java_type(data_type): return _to_java_type(data_type.sql_type()) else: - raise TypeError("Not supported type: %s" % data_type) + raise TypeError("Not supported type: %s" % repr(data_type)) def _is_instance_of(java_data_type, java_class): @@ -1731,7 +1775,7 @@ def _from_java_type(j_data_type): elif _is_instance_of(logical_type, gateway.jvm.TimeType): data_type = DataTypes.TIME(logical_type.getPrecision(), logical_type.isNullable()) elif _is_instance_of(logical_type, gateway.jvm.TimestampType): - data_type = DataTypes.TIMESTAMP(nullable=logical_type.isNullable()) + data_type = DataTypes.TIMESTAMP(precision=3, nullable=logical_type.isNullable()) elif _is_instance_of(logical_type, gateway.jvm.BooleanType): data_type = DataTypes.BOOLEAN(logical_type.isNullable()) elif _is_instance_of(logical_type, gateway.jvm.TinyIntType): @@ -2252,6 +2296,8 @@ class DataTypes(object): define such a type as well. The null type is an extension to the SQL standard. + + .. note:: `NullType` is still not supported yet. """ return NullType() @@ -2263,6 +2309,8 @@ class DataTypes(object): :param length: int, the string representation length. It must have a value between 1 and 2147483647(0x7fffffff) (both inclusive). :param nullable: boolean, whether the type can be null (None) or not. + + .. note:: `CharType` is still not supported yet. """ return CharType(length, nullable) @@ -2274,6 +2322,9 @@ class DataTypes(object): :param length: int, the maximum string representation length. It must have a value between 1 and 2147483647(0x7fffffff) (both inclusive). :param nullable: boolean, whether the type can be null (None) or not. + + .. note:: The length limit must be 0x7fffffff(2147483647) currently. + .. seealso:: :func:`~DataTypes.STRING` """ return VarCharType(length, nullable) @@ -2284,6 +2335,8 @@ class DataTypes(object): This is a shortcut for ``DataTypes.VARCHAR(2147483647)``. :param nullable: boolean, whether the type can be null (None) or not. + + .. seealso:: :func:`~DataTypes.VARCHAR` """ return DataTypes.VARCHAR(0x7fffffff, nullable) @@ -2305,6 +2358,8 @@ class DataTypes(object): :param length: int, the number of bytes. It must have a value between 1 and 2147483647(0x7fffffff) (both inclusive). :param nullable: boolean, whether the type can be null (None) or not. + + .. note:: `BinaryType` is still not supported yet. """ return BinaryType(length, nullable) @@ -2316,6 +2371,9 @@ class DataTypes(object): :param length: int, the maximum number of bytes. It must have a value between 1 and 2147483647(0x7fffffff) (both inclusive). :param nullable: boolean, whether the type can be null (None) or not. + + .. note:: The length limit must be 0x7fffffff(2147483647) currently. + .. seealso:: :func:`~DataTypes.BYTES` """ return VarBinaryType(length, nullable) @@ -2326,6 +2384,8 @@ class DataTypes(object): defined maximum length. This is a shortcut for ``DataTypes.VARBINARY(2147483647)``. :param nullable: boolean, whether the type can be null (None) or not. + + .. seealso:: :func:`~DataTypes.VARBINARY` """ return DataTypes.VARBINARY(0x7fffffff, nullable) @@ -2339,6 +2399,8 @@ class DataTypes(object): :param scale: the number of digits on right side of dot. It must have a value between 0 and precision (both inclusive). :param nullable: boolean, whether the type can be null (None) or not. + + .. note:: The precision must be 38 and the scale must be 18 currently. """ return DecimalType(precision, scale, nullable) @@ -2424,6 +2486,8 @@ class DataTypes(object): :param precision: int, the number of digits of fractional seconds. It must have a value between 0 and 9 (both inclusive). :param nullable: boolean, whether the type can be null (None) or not. + + .. note:: The precision must be 0 currently. """ return TimeType(precision, nullable) @@ -2447,6 +2511,8 @@ class DataTypes(object): :param precision: int, the number of digits of fractional seconds. It must have a value between 0 and 9 (both inclusive). (default: 6) :param nullable: boolean, whether the type can be null (None) or not. + + .. note:: The precision must be 3 currently. """ return TimestampType(precision, nullable) @@ -2468,6 +2534,9 @@ class DataTypes(object): :param precision: int, the number of digits of fractional seconds. It must have a value between 0 and 9 (both inclusive). (default: 6) :param nullable: boolean, whether the type can be null (None) or not. + + .. note:: `LocalZonedTimestampType` is currently only supported in blink planner and the + precision must be 3. """ return LocalZonedTimestampType(precision, nullable) @@ -2553,6 +2622,7 @@ class DataTypes(object): between 0 and 9 (both inclusive), (default: 6). :return: the specified :class:`Resolution`. + .. note:: the precision must be 3 currently. .. seealso:: :func:`~pyflink.table.DataTypes.INTERVAL` """ return Resolution(Resolution.IntervalUnit.SECOND, precision) @@ -2646,6 +2716,8 @@ class DataTypes(object): :param upper_resolution: :class:`Resolution`, the upper resolution of the interval. :param lower_resolution: :class:`Resolution`, the lower resolution of the interval. + .. note:: the upper_resolution must be `MONTH` for `YearMonthIntervalType`, `SECOND` for + `DayTimeIntervalType` and the lower_resolution must be None currently. .. seealso:: :func:`~pyflink.table.DataTypes.SECOND` .. seealso:: :func:`~pyflink.table.DataTypes.MINUTE` .. seealso:: :func:`~pyflink.table.DataTypes.HOUR` -- Gitee From cf7cc89ac99554dc51da5cbffc3b76fe32ef5fea Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 4 Mar 2020 18:01:18 +0100 Subject: [PATCH 187/885] [FLINK-16373] Make JobManagerLeaderListener thread safe The JobManagerLeaderListener used by the JobLeaderService was not thread safe. Stopping the listener while notifying a new leader could lead to an IllegalStateException where the rpcConnection which was supposed to be started was concurrently closed by the stop call. This closes #11313. --- .../taskexecutor/JobLeaderService.java | 130 ++++++++++-------- .../taskexecutor/JobLeaderServiceTest.java | 116 ++++++++++++++++ 2 files changed, 186 insertions(+), 60 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/JobLeaderServiceTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java index eed6f114fa..f89585e277 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java @@ -40,9 +40,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.annotation.Nullable; +import javax.annotation.concurrent.GuardedBy; +import javax.annotation.concurrent.ThreadSafe; import java.util.Map; import java.util.Objects; +import java.util.Optional; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; @@ -228,20 +231,26 @@ public class JobLeaderService { /** * Leader listener which tries to establish a connection to a newly detected job leader. */ + @ThreadSafe private final class JobManagerLeaderListener implements LeaderRetrievalListener { + private final Object lock = new Object(); + /** Job id identifying the job to look for a leader. */ private final JobID jobId; /** Rpc connection to the job leader. */ - private volatile RegisteredRpcConnection rpcConnection; + @GuardedBy("lock") + private RegisteredRpcConnection rpcConnection; + + /** Leader id of the current job leader. */ + @GuardedBy("lock") + @Nullable + private JobMasterId currentJobMasterId; /** State of the listener. */ private volatile boolean stopped; - /** Leader id of the current job leader. */ - private volatile JobMasterId currentJobMasterId; - private JobManagerLeaderListener(JobID jobId) { this.jobId = Preconditions.checkNotNull(jobId); @@ -250,91 +259,92 @@ public class JobLeaderService { currentJobMasterId = null; } + private JobMasterId getCurrentJobMasterId() { + synchronized (lock) { + return currentJobMasterId; + } + } + public void stop() { - stopped = true; + synchronized (lock) { + if (!stopped) { + stopped = true; - if (rpcConnection != null) { - rpcConnection.close(); + if (rpcConnection != null) { + rpcConnection.close(); + } + } } } public void reconnect() { - if (stopped) { - LOG.debug("Cannot reconnect because the JobManagerLeaderListener has already been stopped."); - } else { - final RegisteredRpcConnection currentRpcConnection = rpcConnection; - - if (currentRpcConnection != null) { - if (currentRpcConnection.isConnected()) { - - if (currentRpcConnection.tryReconnect()) { - // double check for concurrent stop operation - if (stopped) { - currentRpcConnection.close(); - } - } else { - LOG.debug("Could not reconnect to the JobMaster {}.", currentRpcConnection.getTargetAddress()); - } + synchronized (lock) { + if (stopped) { + LOG.debug("Cannot reconnect because the JobManagerLeaderListener has already been stopped."); + } else { + if (rpcConnection != null) { + Preconditions.checkState( + rpcConnection.tryReconnect(), + "Illegal concurrent modification of the JobManagerLeaderListener rpc connection."); } else { - LOG.debug("Ongoing registration to JobMaster {}.", currentRpcConnection.getTargetAddress()); + LOG.debug("Cannot reconnect to an unknown JobMaster."); } - } else { - LOG.debug("Cannot reconnect to an unknown JobMaster."); } } } @Override public void notifyLeaderAddress(final @Nullable String leaderAddress, final @Nullable UUID leaderId) { - if (stopped) { - LOG.debug("{}'s leader retrieval listener reported a new leader for job {}. " + - "However, the service is no longer running.", JobLeaderService.class.getSimpleName(), jobId); - } else { - final JobMasterId jobMasterId = JobMasterId.fromUuidOrNull(leaderId); - - LOG.debug("New leader information for job {}. Address: {}, leader id: {}.", - jobId, leaderAddress, jobMasterId); - - if (leaderAddress == null || leaderAddress.isEmpty()) { - // the leader lost leadership but there is no other leader yet. - if (rpcConnection != null) { - rpcConnection.close(); - } + Optional jobManagerLostLeadership = Optional.empty(); - jobLeaderListener.jobManagerLostLeadership(jobId, currentJobMasterId); - - currentJobMasterId = jobMasterId; + synchronized (lock) { + if (stopped) { + LOG.debug("{}'s leader retrieval listener reported a new leader for job {}. " + + "However, the service is no longer running.", JobLeaderService.class.getSimpleName(), jobId); } else { - currentJobMasterId = jobMasterId; + final JobMasterId jobMasterId = JobMasterId.fromUuidOrNull(leaderId); - if (rpcConnection != null) { - // check if we are already trying to connect to this leader - if (!Objects.equals(jobMasterId, rpcConnection.getTargetLeaderId())) { + LOG.debug("New leader information for job {}. Address: {}, leader id: {}.", + jobId, leaderAddress, jobMasterId); + + if (leaderAddress == null || leaderAddress.isEmpty()) { + // the leader lost leadership but there is no other leader yet. + if (rpcConnection != null) { rpcConnection.close(); + } + jobManagerLostLeadership = Optional.ofNullable(currentJobMasterId); + currentJobMasterId = jobMasterId; + } else { + currentJobMasterId = jobMasterId; + + if (rpcConnection != null) { + // check if we are already trying to connect to this leader + if (!Objects.equals(jobMasterId, rpcConnection.getTargetLeaderId())) { + rpcConnection.close(); + + rpcConnection = new JobManagerRegisteredRpcConnection( + LOG, + leaderAddress, + jobMasterId, + rpcService.getExecutor()); + } + } else { rpcConnection = new JobManagerRegisteredRpcConnection( LOG, leaderAddress, jobMasterId, rpcService.getExecutor()); } - } else { - rpcConnection = new JobManagerRegisteredRpcConnection( - LOG, - leaderAddress, - jobMasterId, - rpcService.getExecutor()); - } - // double check for a concurrent stop operation - if (stopped) { - rpcConnection.close(); - } else { LOG.info("Try to register at job manager {} with leader id {}.", leaderAddress, leaderId); rpcConnection.start(); } } } + + // send callbacks outside of the lock scope + jobManagerLostLeadership.ifPresent(oldJobMasterId -> jobLeaderListener.jobManagerLostLeadership(jobId, oldJobMasterId)); } @Override @@ -378,7 +388,7 @@ public class JobLeaderService { @Override protected void onRegistrationSuccess(JMTMRegistrationSuccess success) { // filter out old registration attempts - if (Objects.equals(getTargetLeaderId(), currentJobMasterId)) { + if (Objects.equals(getTargetLeaderId(), getCurrentJobMasterId())) { log.info("Successful registration at job manager {} for job {}.", getTargetAddress(), jobId); jobLeaderListener.jobManagerGainedLeadership(jobId, getTargetGateway(), success); @@ -390,7 +400,7 @@ public class JobLeaderService { @Override protected void onRegistrationFailure(Throwable failure) { // filter out old registration attempts - if (Objects.equals(getTargetLeaderId(), currentJobMasterId)) { + if (Objects.equals(getTargetLeaderId(), getCurrentJobMasterId())) { log.info("Failed to register at job manager {} for job {}.", getTargetAddress(), jobId); jobLeaderListener.handleError(failure); } else { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/JobLeaderServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/JobLeaderServiceTest.java new file mode 100644 index 0000000000..51595e85b4 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/JobLeaderServiceTest.java @@ -0,0 +1,116 @@ +/* + * 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.runtime.taskexecutor; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.core.testutils.CheckedThread; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServicesBuilder; +import org.apache.flink.runtime.jobmaster.JMTMRegistrationSuccess; +import org.apache.flink.runtime.jobmaster.JobMasterGateway; +import org.apache.flink.runtime.jobmaster.JobMasterId; +import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService; +import org.apache.flink.runtime.registration.RetryingRegistrationConfiguration; +import org.apache.flink.runtime.rpc.TestingRpcServiceResource; +import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; +import org.apache.flink.util.TestLogger; + +import org.junit.ClassRule; +import org.junit.Test; + +import java.util.UUID; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; + +/** + * Tests for the {@link JobLeaderService}. + */ +public class JobLeaderServiceTest extends TestLogger { + + @ClassRule + public static final TestingRpcServiceResource RPC_SERVICE_RESOURCE = new TestingRpcServiceResource(); + + /** + * Tests that we can concurrently modify the JobLeaderService and complete the leader retrieval operation. + * See FLINK-16373. + */ + @Test + public void handlesConcurrentJobAdditionsAndLeaderChanges() throws Exception { + final JobLeaderService jobLeaderService = new JobLeaderService( + new LocalTaskManagerLocation(), + RetryingRegistrationConfiguration.defaultConfiguration()); + + final TestingJobLeaderListener jobLeaderListener = new TestingJobLeaderListener(); + final int numberOperations = 20; + final BlockingQueue instantiatedLeaderRetrievalServices = new ArrayBlockingQueue<>(numberOperations); + + final HighAvailabilityServices haServices = new TestingHighAvailabilityServicesBuilder() + .setJobMasterLeaderRetrieverFunction( + leaderForJobId -> { + final SettableLeaderRetrievalService leaderRetrievalService = new SettableLeaderRetrievalService(); + instantiatedLeaderRetrievalServices.offer(leaderRetrievalService); + return leaderRetrievalService; + }) + .build(); + + jobLeaderService.start( + "foobar", + RPC_SERVICE_RESOURCE.getTestingRpcService(), + haServices, + jobLeaderListener); + + final CheckedThread addJobAction = new CheckedThread() { + @Override + public void go() throws Exception { + for (int i = 0; i < numberOperations; i++) { + final JobID jobId = JobID.generate(); + jobLeaderService.addJob(jobId, "foobar"); + Thread.yield(); + jobLeaderService.removeJob(jobId); + } + } + }; + addJobAction.start(); + + for (int i = 0; i < numberOperations; i++) { + final SettableLeaderRetrievalService leaderRetrievalService = instantiatedLeaderRetrievalServices.take(); + leaderRetrievalService.notifyListener("foobar", UUID.randomUUID()); + } + + addJobAction.sync(); + } + + private static final class TestingJobLeaderListener implements JobLeaderListener { + + @Override + public void jobManagerGainedLeadership(JobID jobId, JobMasterGateway jobManagerGateway, JMTMRegistrationSuccess registrationMessage) { + // ignored + } + + @Override + public void jobManagerLostLeadership(JobID jobId, JobMasterId jobMasterId) { + // ignored + } + + @Override + public void handleError(Throwable throwable) { + // ignored + } + } +} -- Gitee From 3cd1eabee8b23bc903d5dbf0b04df39d4e634c50 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 4 Mar 2020 18:08:13 +0100 Subject: [PATCH 188/885] [hotfix] Minor cleanups in JobLeaderService --- .../apache/flink/runtime/taskexecutor/JobLeaderService.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java index f89585e277..b444cd09b7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java @@ -294,7 +294,7 @@ public class JobLeaderService { } @Override - public void notifyLeaderAddress(final @Nullable String leaderAddress, final @Nullable UUID leaderId) { + public void notifyLeaderAddress(@Nullable final String leaderAddress, @Nullable final UUID leaderId) { Optional jobManagerLostLeadership = Optional.empty(); synchronized (lock) { @@ -446,8 +446,8 @@ public class JobLeaderService { @Override protected CompletableFuture invokeRegistration( JobMasterGateway gateway, - JobMasterId jobMasterId, - long timeoutMillis) throws Exception { + JobMasterId fencingToken, + long timeoutMillis) { return gateway.registerTaskManager(taskManagerRpcAddress, taskManagerLocation, Time.milliseconds(timeoutMillis)); } } -- Gitee From 9a0794528c98612f4dfcc723529419fdb992918f Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 17 Mar 2020 15:08:07 +0100 Subject: [PATCH 189/885] [hotfix] Add JavaDoc to RegisteredRpcConnection#tryReconnect --- .../runtime/registration/RegisteredRpcConnection.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegisteredRpcConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegisteredRpcConnection.java index 8c78ede8e1..b257e17a60 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegisteredRpcConnection.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegisteredRpcConnection.java @@ -100,6 +100,13 @@ public abstract class RegisteredRpcConnection Date: Tue, 17 Mar 2020 12:11:43 +0100 Subject: [PATCH 190/885] [FLINK-16635] Remove pinned dependency for okio and okhttp from flink-metrics-influxdb With FLINK-12147 we bumped the influxdb-java version from 2.14 to 2.16. At the same time we still have okio and okhttp fixed to an incompatible version. This commit removes the dependency management entries for these dependencies so that the influxdb reporter bundles the correct dependencies. --- flink-metrics/flink-metrics-influxdb/pom.xml | 15 --------------- .../src/main/resources/META-INF/NOTICE | 4 ++-- 2 files changed, 2 insertions(+), 17 deletions(-) diff --git a/flink-metrics/flink-metrics-influxdb/pom.xml b/flink-metrics/flink-metrics-influxdb/pom.xml index 81d90cd471..1d6289ab0e 100644 --- a/flink-metrics/flink-metrics-influxdb/pom.xml +++ b/flink-metrics/flink-metrics-influxdb/pom.xml @@ -89,21 +89,6 @@ under the License. - - - - com.squareup.okhttp3 - okhttp - 3.11.0 - - - com.squareup.okio - okio - 1.14.0 - - - - diff --git a/flink-metrics/flink-metrics-influxdb/src/main/resources/META-INF/NOTICE b/flink-metrics/flink-metrics-influxdb/src/main/resources/META-INF/NOTICE index ad79869c5b..2cca5650b7 100644 --- a/flink-metrics/flink-metrics-influxdb/src/main/resources/META-INF/NOTICE +++ b/flink-metrics/flink-metrics-influxdb/src/main/resources/META-INF/NOTICE @@ -8,8 +8,8 @@ This project bundles the following dependencies under the Apache Software Licens - com.squareup.moshi:moshi:1.5.0 - com.squareup.okhttp3:logging-interceptor:3.11.0 -- com.squareup.okhttp3:okhttp:3.11.0 -- com.squareup.okio:okio:1.14.0 +- com.squareup.okhttp3:okhttp:3.14.3 +- com.squareup.okio:okio:1.17.2 - com.squareup.retrofit2:converter-moshi:2.4.0 - com.squareup.retrofit2:retrofit:2.4.0 -- Gitee From 405cf8f429c6a5031c21597abe9193bedcb8e15b Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 17 Mar 2020 14:38:09 +0100 Subject: [PATCH 191/885] [FLINK-16635] Bump influxdb-java version to 2.17 This commit bumps influxdb-java version from 2.16 to 2.17. This resolves a dependency convergence problem within the influxdb-java dependency. This closes #11428. --- flink-metrics/flink-metrics-influxdb/pom.xml | 2 +- .../src/main/resources/META-INF/NOTICE | 12 ++++++------ 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/flink-metrics/flink-metrics-influxdb/pom.xml b/flink-metrics/flink-metrics-influxdb/pom.xml index 1d6289ab0e..71cfdbf88b 100644 --- a/flink-metrics/flink-metrics-influxdb/pom.xml +++ b/flink-metrics/flink-metrics-influxdb/pom.xml @@ -57,7 +57,7 @@ under the License. org.influxdb influxdb-java - 2.16 + 2.17 diff --git a/flink-metrics/flink-metrics-influxdb/src/main/resources/META-INF/NOTICE b/flink-metrics/flink-metrics-influxdb/src/main/resources/META-INF/NOTICE index 2cca5650b7..cfd39b4c17 100644 --- a/flink-metrics/flink-metrics-influxdb/src/main/resources/META-INF/NOTICE +++ b/flink-metrics/flink-metrics-influxdb/src/main/resources/META-INF/NOTICE @@ -6,13 +6,13 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) -- com.squareup.moshi:moshi:1.5.0 -- com.squareup.okhttp3:logging-interceptor:3.11.0 -- com.squareup.okhttp3:okhttp:3.14.3 +- com.squareup.moshi:moshi:1.8.0 +- com.squareup.okhttp3:logging-interceptor:3.14.4 +- com.squareup.okhttp3:okhttp:3.14.4 - com.squareup.okio:okio:1.17.2 -- com.squareup.retrofit2:converter-moshi:2.4.0 -- com.squareup.retrofit2:retrofit:2.4.0 +- com.squareup.retrofit2:converter-moshi:2.6.2 +- com.squareup.retrofit2:retrofit:2.6.2 This project bundles the following dependencies under the MIT license. (https://opensource.org/licenses/MIT) -- org.influxdb:influxdb-java:2.14 +- org.influxdb:influxdb-java:2.17 -- Gitee From 65037f0235cc540f14cf5f04f7ecb0ad157948c1 Mon Sep 17 00:00:00 2001 From: CrazyTomatoOo <1650459038@qq.com> Date: Tue, 17 Mar 2020 09:14:20 +0800 Subject: [PATCH 192/885] [FLINK-16083][docs-zh] Translate "Dynamic Table" page of "Streaming Concepts" into Chinese This closes #11423 --- docs/dev/table/common.zh.md | 2 + docs/dev/table/streaming/dynamic_tables.zh.md | 122 +++++++++--------- 2 files changed, 64 insertions(+), 60 deletions(-) diff --git a/docs/dev/table/common.zh.md b/docs/dev/table/common.zh.md index adbc695804..4670a205a4 100644 --- a/docs/dev/table/common.zh.md +++ b/docs/dev/table/common.zh.md @@ -950,6 +950,8 @@ val table2: Table = tableEnv.fromDataStream(stream, 'myLong, 'myString) {% top %} + + ### 将表转换成 DataStream 或 DataSet `Table` 可以被转换成 `DataStream` 或 `DataSet`。通过这种方式,定制的 DataSet 或 DataStream 程序就可以在 Table API 或者 SQL 的查询结果上运行了。 diff --git a/docs/dev/table/streaming/dynamic_tables.zh.md b/docs/dev/table/streaming/dynamic_tables.zh.md index a9d12d37ee..2906568661 100644 --- a/docs/dev/table/streaming/dynamic_tables.zh.md +++ b/docs/dev/table/streaming/dynamic_tables.zh.md @@ -22,127 +22,127 @@ specific language governing permissions and limitations under the License. --> -SQL and the relational algebra have not been designed with streaming data in mind. As a consequence, there are few conceptual gaps between relational algebra (and SQL) and stream processing. +SQL 和关系代数在设计时并未考虑流数据。因此,在关系代数(和 SQL)之间几乎没有概念上的差异。 -This page discusses these differences and explains how Flink can achieve the same semantics on unbounded data as a regular database engine on bounded data. +本文会讨论这种差异,并介绍 Flink 如何在无界数据集上实现与数据库引擎在有界数据上的处理具有相同的语义。 * This will be replaced by the TOC {:toc} -Relational Queries on Data Streams +DataStream 上的关系查询 ---------------------------------- -The following table compares traditional relational algebra and stream processing with respect to input data, execution, and output results. +下表比较了传统的关系代数和流处理与输入数据、执行和输出结果的关系。

    kubernetes.container.image.pull-policy
    "IfNotPresent"StringKubernetes image pull policy. Valid values are Always, Never, and IfNotPresent. The default policy is IfNotPresent to avoid putting pressure to image repository.IfNotPresent

    Enum

    Possible values: [IfNotPresent, Always, Never]
    The Kubernetes container image pull policy (IfNotPresent or Always or Never). The default policy is IfNotPresent to avoid putting pressure to image repository.
    kubernetes.container.image.pull-secrets
    - - + + - - + + - - + + - - + +
    Relational Algebra / SQLStream Processing关系代数 / SQL流处理
    Relations (or tables) are bounded (multi-)sets of tuples.A stream is an infinite sequences of tuples.关系(或表)是有界(多)元组集合。流是一个无限元组序列。
    A query that is executed on batch data (e.g., a table in a relational database) has access to the complete input data.A streaming query cannot access all data when it is started and has to "wait" for data to be streamed in.对批数据(例如关系数据库中的表)执行的查询可以访问完整的输入数据。流式查询在启动时不能访问所有数据,必须“等待”数据流入。
    A batch query terminates after it produced a fixed sized result.A streaming query continuously updates its result based on the received records and never completes.批处理查询在产生固定大小的结果后终止。流查询不断地根据接收到的记录更新其结果,并且始终不会结束。
    -Despite these differences, processing streams with relational queries and SQL is not impossible. Advanced relational database systems offer a feature called *Materialized Views*. A materialized view is defined as a SQL query, just like a regular virtual view. In contrast to a virtual view, a materialized view caches the result of the query such that the query does not need to be evaluated when the view is accessed. A common challenge for caching is to prevent a cache from serving outdated results. A materialized view becomes outdated when the base tables of its definition query are modified. *Eager View Maintenance* is a technique to update a materialized view as soon as its base tables are updated. +尽管存在这些差异,但是使用关系查询和 SQL 处理流并不是不可能的。高级关系数据库系统提供了一个称为 *物化视图(Materialized Views)* 的特性。物化视图被定义为一条 SQL 查询,就像常规的虚拟视图一样。与虚拟视图相反,物化视图缓存查询的结果,因此在访问视图时不需要对查询进行计算。缓存的一个常见难题是防止缓存为过期的结果提供服务。当其定义查询的基表被修改时,物化视图将过期。 *即时视图维护(Eager View Maintenance)* 是一种一旦更新了物化视图的基表就立即更新视图的技术。 -The connection between eager view maintenance and SQL queries on streams becomes obvious if we consider the following: +如果我们考虑以下问题,那么即时视图维护和流上的SQL查询之间的联系就会变得显而易见: -- A database table is the result of a *stream* of `INSERT`, `UPDATE`, and `DELETE` DML statements, often called *changelog stream*. -- A materialized view is defined as a SQL query. In order to update the view, the query continuously processes the changelog streams of the view's base relations. -- The materialized view is the result of the streaming SQL query. +- 数据库表是 `INSERT`、`UPDATE` 和 `DELETE` DML 语句的 *stream* 的结果,通常称为 *changelog stream* 。 +- 物化视图被定义为一条 SQL 查询。为了更新视图,查询不断地处理视图的基本关系的changelog 流。 +- 物化视图是流式 SQL 查询的结果。 -With these points in mind, we introduce following concept of *Dynamic tables* in the next section. +了解了这些要点之后,我们将在下一节中介绍 *动态表(Dynamic tables)* 的概念。 -Dynamic Tables & Continuous Queries +动态表 & 连续查询(Continuous Query) --------------------------------------- -*Dynamic tables* are the core concept of Flink's Table API and SQL support for streaming data. In contrast to the static tables that represent batch data, dynamic tables are changing over time. They can be queried like static batch tables. Querying dynamic tables yields a *Continuous Query*. A continuous query never terminates and produces a dynamic table as result. The query continuously updates its (dynamic) result table to reflect the changes on its (dynamic) input tables. Essentially, a continuous query on a dynamic table is very similar to a query that defines a materialized view. +*动态表* 是 Flink 的支持流数据的 Table API 和 SQL 的核心概念。与表示批处理数据的静态表不同,动态表是随时间变化的。可以像查询静态批处理表一样查询它们。查询动态表将生成一个 *连续查询* 。一个连续查询永远不会终止,结果会生成一个动态表。查询不断更新其(动态)结果表,以反映其(动态)输入表上的更改。本质上,动态表上的连续查询非常类似于定义物化视图的查询。 -It is important to note that the result of a continuous query is always semantically equivalent to the result of the same query being executed in batch mode on a snapshot of the input tables. +需要注意的是,连续查询的结果在语义上总是等价于以批处理模式在输入表快照上执行的相同查询的结果。 -The following figure visualizes the relationship of streams, dynamic tables, and continuous queries: +下图显示了流、动态表和连续查询之间的关系:
    Dynamic tables
    -1. A stream is converted into a dynamic table. -1. A continuous query is evaluated on the dynamic table yielding a new dynamic table. -1. The resulting dynamic table is converted back into a stream. +1. 将流转换为动态表。 +2. 在动态表上计算一个连续查询,生成一个新的动态表。 +3. 生成的动态表被转换回流。 -**Note:** Dynamic tables are foremost a logical concept. Dynamic tables are not necessarily (fully) materialized during query execution. +**注意:** 动态表首先是一个逻辑概念。在查询执行期间不一定(完全)物化动态表。 -In the following, we will explain the concepts of dynamic tables and continuous queries with a stream of click events that have the following schema: +在下面,我们将解释动态表和连续查询的概念,并使用具有以下模式的单击事件流: {% highlight plain %} [ - user: VARCHAR, // the name of the user - cTime: TIMESTAMP, // the time when the URL was accessed - url: VARCHAR // the URL that was accessed by the user + user: VARCHAR, // 用户名 + cTime: TIMESTAMP, // 访问 URL 的时间 + url: VARCHAR // 用户访问的 URL ] {% endhighlight %} -Defining a Table on a Stream +在流上定义表 ---------------------------- -In order to process a stream with a relational query, it has to be converted into a `Table`. Conceptually, each record of the stream is interpreted as an `INSERT` modification on the resulting table. Essentially, we are building a table from an `INSERT`-only changelog stream. +为了使用关系查询处理流,必须将其转换成 `Table`。从概念上讲,流的每条记录都被解释为对结果表的 `INSERT` 操作。本质上我们正在从一个 `INSERT`-only 的 changelog 流构建表。 -The following figure visualizes how the stream of click event (left-hand side) is converted into a table (right-hand side). The resulting table is continuously growing as more records of the click stream are inserted. +下图显示了单击事件流(左侧)如何转换为表(右侧)。当插入更多的单击流记录时,结果表将不断增长。
    Append mode
    -**Note:** A table which is defined on a stream is internally not materialized. +**注意:** 在流上定义的表在内部没有物化。 -### Continuous Queries +### 连续查询 ---------------------- -A continuous query is evaluated on a dynamic table and produces a new dynamic table as result. In contrast to a batch query, a continuous query never terminates and updates its result table according to the updates on its input tables. At any point in time, the result of a continuous query is semantically equivalent to the result of the same query being executed in batch mode on a snapshot of the input tables. +在动态表上计算一个连续查询,并生成一个新的动态表。与批处理查询不同,连续查询从不终止,并根据其输入表上的更新更新其结果表。在任何时候,连续查询的结果在语义上与以批处理模式在输入表快照上执行的相同查询的结果相同。 -In the following we show two example queries on a `clicks` table that is defined on the stream of click events. +在接下来的代码中,我们将展示 `clicks` 表上的两个示例查询,这个表是在点击事件流上定义的。 -The first query is a simple `GROUP-BY COUNT` aggregation query. It groups the `clicks` table on the `user` field and counts the number of visited URLs. The following figure shows how the query is evaluated over time as the `clicks` table is updated with additional rows. +第一个查询是一个简单的 `GROUP-BY COUNT` 聚合查询。它基于 `user` 字段对 `clicks` 表进行分组,并统计访问的 URL 的数量。下面的图显示了当 `clicks` 表被附加的行更新时,查询是如何被评估的。
    Continuous Non-Windowed Query
    -When the query is started, the `clicks` table (left-hand side) is empty. The query starts to compute the result table, when the first row is inserted into the `clicks` table. After the first row `[Mary, ./home]` was inserted, the result table (right-hand side, top) consists of a single row `[Mary, 1]`. When the second row `[Bob, ./cart]` is inserted into the `clicks` table, the query updates the result table and inserts a new row `[Bob, 1]`. The third row `[Mary, ./prod?id=1]` yields an update of an already computed result row such that `[Mary, 1]` is updated to `[Mary, 2]`. Finally, the query inserts a third row `[Liz, 1]` into the result table, when the fourth row is appended to the `clicks` table. +当查询开始,`clicks` 表(左侧)是空的。当第一行数据被插入到 `clicks` 表时,查询开始计算结果表。第一行数据 `[Mary,./home]` 插入后,结果表(右侧,上部)由一行 `[Mary, 1]` 组成。当第二行 `[Bob, ./cart]` 插入到 `clicks` 表时,查询会更新结果表并插入了一行新数据 `[Bob, 1]`。第三行 `[Mary, ./prod?id=1]` 将产生已计算的结果行的更新,`[Mary, 1]` 更新成 `[Mary, 2]`。最后,当第四行数据加入 `clicks` 表时,查询将第三行 `[Liz, 1]` 插入到结果表中。 -The second query is similar to the first one but groups the `clicks` table in addition to the `user` attribute also on an [hourly tumbling window]({{ site.baseurl }}/dev/table/sql/index.html#group-windows) before it counts the number of URLs (time-based computations such as windows are based on special [time attributes](time_attributes.html), which are discussed later.). Again, the figure shows the input and output at different points in time to visualize the changing nature of dynamic tables. +第二条查询与第一条类似,但是除了用户属性之外,还将 `clicks` 分组至[每小时滚动窗口]({{ site.baseurl }}/zh/dev/table/sql/index.html#group-windows)中,然后计算 url 数量(基于时间的计算,例如基于特定[时间属性](time_attributes.html)的窗口,后面会讨论)。同样,该图显示了不同时间点的输入和输出,以可视化动态表的变化特性。
    Continuous Group-Window Query
    -As before, the input table `clicks` is shown on the left. The query continuously computes results every hour and updates the result table. The clicks table contains four rows with timestamps (`cTime`) between `12:00:00` and `12:59:59`. The query computes two results rows from this input (one for each `user`) and appends them to the result table. For the next window between `13:00:00` and `13:59:59`, the `clicks` table contains three rows, which results in another two rows being appended to the result table. The result table is updated, as more rows are appended to `clicks` over time. +与前面一样,左边显示了输入表 `clicks`。查询每小时持续计算结果并更新结果表。clicks表包含四行带有时间戳(`cTime`)的数据,时间戳在 `12:00:00` 和 `12:59:59` 之间。查询从这个输入计算出两个结果行(每个 `user` 一个),并将它们附加到结果表中。对于 `13:00:00` 和 `13:59:59` 之间的下一个窗口,`clicks` 表包含三行,这将导致另外两行被追加到结果表。随着时间的推移,更多的行被添加到 `click` 中,结果表将被更新。 -### Update and Append Queries +### 更新和追加查询 -Although the two example queries appear to be quite similar (both compute a grouped count aggregate), they differ in one important aspect: -- The first query updates previously emitted results, i.e., the changelog stream that defines the result table contains `INSERT` and `UPDATE` changes. -- The second query only appends to the result table, i.e., the changelog stream of the result table only consists of `INSERT` changes. +虽然这两个示例查询看起来非常相似(都计算分组计数聚合),但它们在一个重要方面不同: +- 第一个查询更新先前输出的结果,即定义结果表的 changelog 流包含 `INSERT` 和 `UPDATE` 操作。 +- 第二个查询只附加到结果表,即结果表的 changelog 流只包含 `INSERT` 操作。 -Whether a query produces an append-only table or an updated table has some implications: -- Queries that produce update changes usually have to maintain more state (see the following section). -- The conversion of an append-only table into a stream is different from the conversion of an updated table (see the [Table to Stream Conversion](#table-to-stream-conversion) section). +一个查询是产生一个只追加的表还是一个更新的表有一些含义: +- 产生更新更改的查询通常必须维护更多的状态(请参阅以下部分)。 +- 将 append-only 的表转换为流与将已更新的表转换为流是不同的(参阅[表到流的转换](#table-to-stream-conversion)章节)。 -### Query Restrictions +### 查询限制 -Many, but not all, semantically valid queries can be evaluated as continuous queries on streams. Some queries are too expensive to compute, either due to the size of state that they need to maintain or because computing updates is too expensive. +许多(但不是全部)语义上有效的查询可以作为流上的连续查询进行评估。有些查询代价太高而无法计算,这可能是由于它们需要维护的状态大小,也可能是由于计算更新代价太高。 -- **State Size:** Continuous queries are evaluated on unbounded streams and are often supposed to run for weeks or months. Hence, the total amount of data that a continuous query processes can be very large. Queries that have to update previously emitted results need to maintain all emitted rows in order to be able to update them. For instance, the first example query needs to store the URL count for each user to be able to increase the count and sent out a new result when the input table receives a new row. If only registered users are tracked, the number of counts to maintain might not be too high. However, if non-registered users get a unique user name assigned, the number of counts to maintain would grow over time and might eventually cause the query to fail. +- **状态大小:** 连续查询在无界流上计算,通常应该运行数周或数月。因此,连续查询处理的数据总量可能非常大。必须更新先前输出的结果的查询需要维护所有输出的行,以便能够更新它们。例如,第一个查询示例需要存储每个用户的 URL 计数,以便能够增加该计数并在输入表接收新行时发送新结果。如果只跟踪注册用户,则要维护的计数数量可能不会太高。但是,如果未注册的用户分配了一个惟一的用户名,那么要维护的计数数量将随着时间增长,并可能最终导致查询失败。 {% highlight sql %} SELECT user, COUNT(url) @@ -150,7 +150,7 @@ FROM clicks GROUP BY user; {% endhighlight %} -- **Computing Updates:** Some queries require to recompute and update a large fraction of the emitted result rows even if only a single input record is added or updated. Clearly, such queries are not well suited to be executed as continuous queries. An example is the following query which computes for each user a `RANK` based on the time of the last click. As soon as the `clicks` table receives a new row, the `lastAction` of the user is updated and a new rank must be computed. However since two rows cannot have the same rank, all lower ranked rows need to be updated as well. +- **计算更新:** 有些查询需要重新计算和更新大量已输出的结果行,即使只添加或更新一条输入记录。显然,这样的查询不适合作为连续查询执行。下面的查询就是一个例子,它根据最后一次单击的时间为每个用户计算一个 `RANK`。一旦 `click` 表接收到一个新行,用户的 `lastAction` 就会更新,并必须计算一个新的排名。然而,由于两行不能具有相同的排名,所以所有较低排名的行也需要更新。 {% highlight sql %} SELECT user, RANK() OVER (ORDER BY lastLogin) @@ -159,31 +159,33 @@ FROM ( ); {% endhighlight %} -The [Query Configuration](query_configuration.html) page discusses parameters to control the execution of continuous queries. Some parameters can be used to trade the size of maintained state for result accuracy. +[查询配置](query_configuration.html)章节讨论了控制连续查询执行的参数。一些参数可以用来在维持状态的大小和获得结果的准确性之间做取舍。 -Table to Stream Conversion + + +表到流的转换 -------------------------- -A dynamic table can be continuously modified by `INSERT`, `UPDATE`, and `DELETE` changes just like a regular database table. It might be a table with a single row, which is constantly updated, an insert-only table without `UPDATE` and `DELETE` modifications, or anything in between. +动态表可以像普通数据库表一样通过 `INSERT`、`UPDATE` 和 `DELETE` 来不断修改。它可能是一个只有一行、不断更新的表,也可能是一个 insert-only 的表,没有 `UPDATE` 和 `DELETE` 修改,或者介于两者之间的其他表。 -When converting a dynamic table into a stream or writing it to an external system, these changes need to be encoded. Flink's Table API and SQL support three ways to encode the changes of a dynamic table: +在将动态表转换为流或将其写入外部系统时,需要对这些更改进行编码。Flink的 Table API 和 SQL 支持三种方式来编码一个动态表的变化: -* **Append-only stream:** A dynamic table that is only modified by `INSERT` changes can be converted into a stream by emitting the inserted rows. +* **Append-only 流:** 仅通过 `INSERT` 操作修改的动态表可以通过输出插入的行转换为流。 -* **Retract stream:** A retract stream is a stream with two types of messages, *add messages* and *retract messages*. A dynamic table is converted into an retract stream by encoding an `INSERT` change as add message, a `DELETE` change as retract message, and an `UPDATE` change as a retract message for the updated (previous) row and an add message for the updating (new) row. The following figure visualizes the conversion of a dynamic table into a retract stream. +* **Retract 流:** retract 流包含两种类型的 message: *add messages* 和 *retract messages* 。通过将`INSERT` 操作编码为 add message、将 `DELETE` 操作编码为 retract message、将 `UPDATE` 操作编码为更新(先前)行的 retract message 和更新(新)行的 add message,将动态表转换为 retract 流。下图显示了将动态表转换为 retract 流的过程。
    Dynamic tables


    -* **Upsert stream:** An upsert stream is a stream with two types of messages, *upsert messages* and *delete messages*. A dynamic table that is converted into an upsert stream requires a (possibly composite) unique key. A dynamic table with unique key is converted into a stream by encoding `INSERT` and `UPDATE` changes as upsert messages and `DELETE` changes as delete messages. The stream consuming operator needs to be aware of the unique key attribute in order to apply messages correctly. The main difference to a retract stream is that `UPDATE` changes are encoded with a single message and hence more efficient. The following figure visualizes the conversion of a dynamic table into an upsert stream. +* **Upsert 流:** upsert 流包含两种类型的 message: *upsert messages* 和*delete messages*。转换为 upsert 流的动态表需要(可能是组合的)唯一键。通过将 `INSERT` 和 `UPDATE` 操作编码为 upsert message,将 `DELETE` 操作编码为 delete message ,将具有唯一键的动态表转换为流。消费流的算子需要知道唯一键的属性,以便正确地应用 message。与 retract 流的主要区别在于 `UPDATE` 操作是用单个 message 编码的,因此效率更高。下图显示了将动态表转换为 upsert 流的过程。
    Dynamic tables


    -The API to convert a dynamic table into a `DataStream` is discussed on the [Common Concepts]({{ site.baseurl }}/dev/table/common.html#convert-a-table-into-a-datastream) page. Please note that only append and retract streams are supported when converting a dynamic table into a `DataStream`. The `TableSink` interface to emit a dynamic table to an external system are discussed on the [TableSources and TableSinks](../sourceSinks.html#define-a-tablesink) page. +在[通用概念]({{ site.baseurl }}/zh/dev/table/common.html#convert-a-table-into-a-datastream)中讨论了将动态表转换为 `DataStream` 的 API。请注意,在将动态表转换为 `DataStream` 时,只支持 append 流和 retract 流。在 [TableSources 和 TableSinks](../sourceSinks.html#define-a-tablesink) 章节讨论向外部系统输出动态表的 `TableSink` 接口。 {% top %} -- Gitee From 1276507fa68880866f227dcf5f607ffb0ca9ffdf Mon Sep 17 00:00:00 2001 From: JunZhang Date: Thu, 19 Mar 2020 11:26:13 +0800 Subject: [PATCH 193/885] [FLINK-16646][orc] Flink read orc file throw a NullPointerException after re-open This closes #11434 --- .../org/apache/flink/orc/OrcInputFormat.java | 4 +++ .../flink/orc/OrcRowInputFormatTest.java | 27 +++++++++++++++++++ 2 files changed, 31 insertions(+) diff --git a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/OrcInputFormat.java b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/OrcInputFormat.java index 777005b326..f49eef2598 100644 --- a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/OrcInputFormat.java +++ b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/OrcInputFormat.java @@ -127,6 +127,10 @@ public abstract class OrcInputFormat extends FileInputFormat { this.reader.close(); } this.reader = null; + } + + @Override + public void closeInputFormat() throws IOException{ this.schema = null; } diff --git a/flink-formats/flink-orc/src/test/java/org/apache/flink/orc/OrcRowInputFormatTest.java b/flink-formats/flink-orc/src/test/java/org/apache/flink/orc/OrcRowInputFormatTest.java index 9481c72951..dc984c9b08 100644 --- a/flink-formats/flink-orc/src/test/java/org/apache/flink/orc/OrcRowInputFormatTest.java +++ b/flink-formats/flink-orc/src/test/java/org/apache/flink/orc/OrcRowInputFormatTest.java @@ -928,6 +928,33 @@ public class OrcRowInputFormatTest { assertEquals(1920800, cnt); } + @Test + public void testReadFileInManySplits() throws IOException { + + rowOrcInputFormat = new OrcRowInputFormat(getPath(TEST_FILE_FLAT), TEST_SCHEMA_FLAT, new Configuration()); + rowOrcInputFormat.selectFields(0, 1); + + FileInputSplit[] splits = rowOrcInputFormat.createInputSplits(4); + assertEquals(4, splits.length); + rowOrcInputFormat.openInputFormat(); + + long cnt = 0; + // read all splits + for (FileInputSplit split : splits) { + + // open split + rowOrcInputFormat.open(split); + // read and count all rows + while (!rowOrcInputFormat.reachedEnd()) { + assertNotNull(rowOrcInputFormat.nextRecord(null)); + cnt++; + } + rowOrcInputFormat.close(); + } + // check that all rows have been read + assertEquals(1920800, cnt); + } + @Test public void testReadFileWithFilter() throws IOException { -- Gitee From 4f4c6826631095c51855db8762dfe43e2f92c2c3 Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Wed, 11 Mar 2020 15:23:19 +0800 Subject: [PATCH 194/885] [hotfix][doc] Fix typos and broken links in memory configuration docs. --- docs/ops/memory/mem_migration.md | 6 +++--- docs/ops/memory/mem_migration.zh.md | 6 +++--- docs/ops/memory/mem_setup.md | 8 ++++---- docs/ops/memory/mem_setup.zh.md | 8 ++++---- docs/ops/memory/mem_trouble.md | 8 ++++---- docs/ops/memory/mem_trouble.zh.md | 8 ++++---- docs/ops/memory/mem_tuning.md | 6 +++--- docs/ops/memory/mem_tuning.zh.md | 6 +++--- 8 files changed, 28 insertions(+), 28 deletions(-) diff --git a/docs/ops/memory/mem_migration.md b/docs/ops/memory/mem_migration.md index dff7508b00..e191940937 100644 --- a/docs/ops/memory/mem_migration.md +++ b/docs/ops/memory/mem_migration.md @@ -123,7 +123,7 @@ The following options are deprecated but if they are still used they will be int Although, the network memory configuration has not changed too much it is recommended to verify its configuration. It can change if other memory components have new sizes, e.g. the total memory which the network can be a fraction of. -See also [new detailed memory model](mem_setup.html#detailed-memory-model). +See also [new detailed memory model](mem_detail.html). The container cut-off configuration options, [`containerized.heap-cutoff-ratio`](config.html#containerized-heap-cutoff-ratio) and [`containerized.heap-cutoff-min`](config.html#containerized-heap-cutoff-min), have no effect for task manager processes anymore @@ -158,7 +158,7 @@ Additionally, you can now have more direct control over the JVM heap assigned to ([`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size)), see also [Task (Operator) Heap Memory](mem_setup.html#task-operator-heap-memory). The JVM heap memory is also used by the heap state backends ([MemoryStateBackend](../state/state_backends.html#the-memorystatebackend) -or [FsStateBackend](../state/state_backends.html#the-fsstatebackend) if it is chosen for streaming jobs. +or [FsStateBackend](../state/state_backends.html#the-fsstatebackend)) if it is chosen for streaming jobs. A part of the JVM heap is now always reserved for Flink framework ([`taskmanager.memory.framework.heap.size`](../config.html#taskmanager-memory-framework-heap-size)). @@ -217,7 +217,7 @@ The other direct or native off-heap memory consumers can now be addressed by the * Task off-heap memory ([`taskmanager.memory.task.off-heap.size`](../config.html#taskmanager-memory-task-off-heap-size)) * Framework off-heap memory ([`taskmanager.memory.framework.off-heap.size`](../config.html#taskmanager-memory-framework-off-heap-size)) * JVM metaspace ([`taskmanager.memory.jvm-metaspace.size`](../config.html#taskmanager-memory-jvm-metaspace-size)) -* JVM overhead (see also [detailed new memory model](mem_setup.html#detailed-memory-model)) +* JVM overhead (see also [detailed new memory model](mem_detail.html)) Note The job manager still has container cut-off memory configuration options. The mentioned configuration options remain valid for the job manager in the same way as before. diff --git a/docs/ops/memory/mem_migration.zh.md b/docs/ops/memory/mem_migration.zh.md index dff7508b00..e191940937 100644 --- a/docs/ops/memory/mem_migration.zh.md +++ b/docs/ops/memory/mem_migration.zh.md @@ -123,7 +123,7 @@ The following options are deprecated but if they are still used they will be int Although, the network memory configuration has not changed too much it is recommended to verify its configuration. It can change if other memory components have new sizes, e.g. the total memory which the network can be a fraction of. -See also [new detailed memory model](mem_setup.html#detailed-memory-model). +See also [new detailed memory model](mem_detail.html). The container cut-off configuration options, [`containerized.heap-cutoff-ratio`](config.html#containerized-heap-cutoff-ratio) and [`containerized.heap-cutoff-min`](config.html#containerized-heap-cutoff-min), have no effect for task manager processes anymore @@ -158,7 +158,7 @@ Additionally, you can now have more direct control over the JVM heap assigned to ([`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size)), see also [Task (Operator) Heap Memory](mem_setup.html#task-operator-heap-memory). The JVM heap memory is also used by the heap state backends ([MemoryStateBackend](../state/state_backends.html#the-memorystatebackend) -or [FsStateBackend](../state/state_backends.html#the-fsstatebackend) if it is chosen for streaming jobs. +or [FsStateBackend](../state/state_backends.html#the-fsstatebackend)) if it is chosen for streaming jobs. A part of the JVM heap is now always reserved for Flink framework ([`taskmanager.memory.framework.heap.size`](../config.html#taskmanager-memory-framework-heap-size)). @@ -217,7 +217,7 @@ The other direct or native off-heap memory consumers can now be addressed by the * Task off-heap memory ([`taskmanager.memory.task.off-heap.size`](../config.html#taskmanager-memory-task-off-heap-size)) * Framework off-heap memory ([`taskmanager.memory.framework.off-heap.size`](../config.html#taskmanager-memory-framework-off-heap-size)) * JVM metaspace ([`taskmanager.memory.jvm-metaspace.size`](../config.html#taskmanager-memory-jvm-metaspace-size)) -* JVM overhead (see also [detailed new memory model](mem_setup.html#detailed-memory-model)) +* JVM overhead (see also [detailed new memory model](mem_detail.html)) Note The job manager still has container cut-off memory configuration options. The mentioned configuration options remain valid for the job manager in the same way as before. diff --git a/docs/ops/memory/mem_setup.md b/docs/ops/memory/mem_setup.md index 7e27ee2cf4..32cfd9c554 100644 --- a/docs/ops/memory/mem_setup.md +++ b/docs/ops/memory/mem_setup.md @@ -47,7 +47,7 @@ and by the JVM to run the process. The *total Flink memory* consumption includes
    -If you run FIink locally (e.g. from your IDE) without creating a cluster, then only a subset of the memory configuration +If you run Flink locally (e.g. from your IDE) without creating a cluster, then only a subset of the memory configuration options are relevant, see also [local execution](mem_detail.html#local-execution) for more details. Otherwise, the simplest way to setup memory in Flink is to configure either of the two following options: @@ -55,7 +55,7 @@ Otherwise, the simplest way to setup memory in Flink is to configure either of t * Total process memory ([`taskmanager.memory.process.size`](../config.html#taskmanager-memory-process-size)) The rest of the memory components will be adjusted automatically, based on default values or additionally configured options. -[Here](mem_detail.html#detailed-memory-model) are more details about the other memory components. +[Here](mem_detail.html) are more details about the other memory components. Configuring *total Flink memory* is better suited for standalone deployments where you want to declare how much memory is given to Flink itself. The *total Flink memory* splits up into JVM heap, [managed memory size](#managed-memory) @@ -87,7 +87,7 @@ to specify explicitly both [task heap](#task-operator-heap-memory) and [managed It gives more control over the available JVM heap to Flink’s tasks and its [managed memory](#managed-memory). The rest of the memory components will be adjusted automatically, based on default values or additionally configured options. -[Here](mem_detail.html#detailed-memory-model) are more details about the other memory components. +[Here](mem_detail.html) are more details about the other memory components. Note If you have configured the task heap and managed memory explicitly, it is recommended to set neither *total process memory* nor *total Flink memory*. Otherwise, it may easily lead to memory configuration conflicts. @@ -130,4 +130,4 @@ see also [JVM parameters](mem_detail.html#jvm-parameters). Note The *network memory* is also part of JVM *direct memory* but it is managed by Flink and guaranteed to never exceed its configured size. Therefore, resizing the *network memory* will not help in this situation. -See also [the detailed memory model](mem_detail.html#detailed-memory-model). +See also [the detailed memory model](mem_detail.html). diff --git a/docs/ops/memory/mem_setup.zh.md b/docs/ops/memory/mem_setup.zh.md index 7e27ee2cf4..32cfd9c554 100644 --- a/docs/ops/memory/mem_setup.zh.md +++ b/docs/ops/memory/mem_setup.zh.md @@ -47,7 +47,7 @@ and by the JVM to run the process. The *total Flink memory* consumption includes
    -If you run FIink locally (e.g. from your IDE) without creating a cluster, then only a subset of the memory configuration +If you run Flink locally (e.g. from your IDE) without creating a cluster, then only a subset of the memory configuration options are relevant, see also [local execution](mem_detail.html#local-execution) for more details. Otherwise, the simplest way to setup memory in Flink is to configure either of the two following options: @@ -55,7 +55,7 @@ Otherwise, the simplest way to setup memory in Flink is to configure either of t * Total process memory ([`taskmanager.memory.process.size`](../config.html#taskmanager-memory-process-size)) The rest of the memory components will be adjusted automatically, based on default values or additionally configured options. -[Here](mem_detail.html#detailed-memory-model) are more details about the other memory components. +[Here](mem_detail.html) are more details about the other memory components. Configuring *total Flink memory* is better suited for standalone deployments where you want to declare how much memory is given to Flink itself. The *total Flink memory* splits up into JVM heap, [managed memory size](#managed-memory) @@ -87,7 +87,7 @@ to specify explicitly both [task heap](#task-operator-heap-memory) and [managed It gives more control over the available JVM heap to Flink’s tasks and its [managed memory](#managed-memory). The rest of the memory components will be adjusted automatically, based on default values or additionally configured options. -[Here](mem_detail.html#detailed-memory-model) are more details about the other memory components. +[Here](mem_detail.html) are more details about the other memory components. Note If you have configured the task heap and managed memory explicitly, it is recommended to set neither *total process memory* nor *total Flink memory*. Otherwise, it may easily lead to memory configuration conflicts. @@ -130,4 +130,4 @@ see also [JVM parameters](mem_detail.html#jvm-parameters). Note The *network memory* is also part of JVM *direct memory* but it is managed by Flink and guaranteed to never exceed its configured size. Therefore, resizing the *network memory* will not help in this situation. -See also [the detailed memory model](mem_detail.html#detailed-memory-model). +See also [the detailed memory model](mem_detail.html). diff --git a/docs/ops/memory/mem_trouble.md b/docs/ops/memory/mem_trouble.md index cd6463bc1b..f3e6b137d7 100644 --- a/docs/ops/memory/mem_trouble.md +++ b/docs/ops/memory/mem_trouble.md @@ -29,7 +29,7 @@ under the License. If you see an *IllegalConfigurationException* thrown from *TaskExecutorProcessUtils*, it usually indicates that there is either an invalid configuration value (e.g. negative memory size, fraction that is greater than 1, etc.) -or configuration conflicts. Check the documentation chapters related to the [memory components](mem_setup.html#detailed-memory-model) +or configuration conflicts. Check the documentation chapters related to the [memory components](mem_detail.html) mentioned in the exception message. ## OutOfMemoryError: Java heap space @@ -44,7 +44,7 @@ is advanced and should only be changed if you are sure that the Flink framework The exception usually indicates that the JVM *direct memory* limit is too small or that there is a *direct memory leak*. Check whether user code or other external dependencies use the JVM *direct memory* and that it is properly accounted for. -You can try to increase its limit by adjusting [direct off-heap memory](mem_setup.html#detailed-memory-model). +You can try to increase its limit by adjusting [direct off-heap memory](mem_detail.html). See also [how to configure off-heap memory](mem_setup.html#configure-off-heap-memory-direct-or-native) and the [JVM arguments](mem_detail.html#jvm-parameters) which Flink sets. @@ -55,7 +55,7 @@ You can try to increase the [JVM metaspace option](../config.html#taskmanager-me ## IOException: Insufficient number of network buffers -The exception usually indicates that the size of the configured [network memory](mem_setup.html#detailed-memory-model) +The exception usually indicates that the size of the configured [network memory](mem_detail.html) is not big enough. You can try to increase the *network memory* by adjusting the following options: * [`taskmanager.memory.network.min`](../config.html#taskmanager-memory-network-min) * [`taskmanager.memory.network.max`](../config.html#taskmanager-memory-network-max) @@ -70,5 +70,5 @@ monitoring system or from the error messages when a container gets killed by the If [RocksDBStateBackend](../state/state_backends.html#the-rocksdbstatebackend) is used and the memory controlling is disabled, you can try to increase the [managed memory](mem_setup.html#managed-memory). -Alternatively, you can increase the [JVM overhead](mem_setup.html#detailed-memory-model). +Alternatively, you can increase the [JVM overhead](mem_detail.html). See also [how to configure memory for containers](mem_tuning.html#configure-memory-for-containers). diff --git a/docs/ops/memory/mem_trouble.zh.md b/docs/ops/memory/mem_trouble.zh.md index cd6463bc1b..f3e6b137d7 100644 --- a/docs/ops/memory/mem_trouble.zh.md +++ b/docs/ops/memory/mem_trouble.zh.md @@ -29,7 +29,7 @@ under the License. If you see an *IllegalConfigurationException* thrown from *TaskExecutorProcessUtils*, it usually indicates that there is either an invalid configuration value (e.g. negative memory size, fraction that is greater than 1, etc.) -or configuration conflicts. Check the documentation chapters related to the [memory components](mem_setup.html#detailed-memory-model) +or configuration conflicts. Check the documentation chapters related to the [memory components](mem_detail.html) mentioned in the exception message. ## OutOfMemoryError: Java heap space @@ -44,7 +44,7 @@ is advanced and should only be changed if you are sure that the Flink framework The exception usually indicates that the JVM *direct memory* limit is too small or that there is a *direct memory leak*. Check whether user code or other external dependencies use the JVM *direct memory* and that it is properly accounted for. -You can try to increase its limit by adjusting [direct off-heap memory](mem_setup.html#detailed-memory-model). +You can try to increase its limit by adjusting [direct off-heap memory](mem_detail.html). See also [how to configure off-heap memory](mem_setup.html#configure-off-heap-memory-direct-or-native) and the [JVM arguments](mem_detail.html#jvm-parameters) which Flink sets. @@ -55,7 +55,7 @@ You can try to increase the [JVM metaspace option](../config.html#taskmanager-me ## IOException: Insufficient number of network buffers -The exception usually indicates that the size of the configured [network memory](mem_setup.html#detailed-memory-model) +The exception usually indicates that the size of the configured [network memory](mem_detail.html) is not big enough. You can try to increase the *network memory* by adjusting the following options: * [`taskmanager.memory.network.min`](../config.html#taskmanager-memory-network-min) * [`taskmanager.memory.network.max`](../config.html#taskmanager-memory-network-max) @@ -70,5 +70,5 @@ monitoring system or from the error messages when a container gets killed by the If [RocksDBStateBackend](../state/state_backends.html#the-rocksdbstatebackend) is used and the memory controlling is disabled, you can try to increase the [managed memory](mem_setup.html#managed-memory). -Alternatively, you can increase the [JVM overhead](mem_setup.html#detailed-memory-model). +Alternatively, you can increase the [JVM overhead](mem_detail.html). See also [how to configure memory for containers](mem_tuning.html#configure-memory-for-containers). diff --git a/docs/ops/memory/mem_tuning.md b/docs/ops/memory/mem_tuning.md index 9d83d00d8d..de54513411 100644 --- a/docs/ops/memory/mem_tuning.md +++ b/docs/ops/memory/mem_tuning.md @@ -31,7 +31,7 @@ depending on the use case and which options are important in which case. ## Configure memory for standalone deployment It is recommended to configure [total Flink memory](mem_setup.html#configure-total-memory) -([`taskmanager.memory.flink.size`](../config.html#taskmanager-memory-flink-size)) or its [components](mem_setup.html#detailed-memory-model) +([`taskmanager.memory.flink.size`](../config.html#taskmanager-memory-flink-size)) or its [components](mem_detail.html) for [standalone deployment](../deployment/cluster_setup.html) where you want to declare how much memory is given to Flink itself. Additionally, you can adjust *JVM metaspace* if it causes [problems](mem_trouble.html#outofmemoryerror-metaspace). @@ -47,7 +47,7 @@ It declares how much memory in total should be assigned to the Flink *JVM proces Note If you configure the *total Flink memory* Flink will implicitly add JVM memory components to derive the *total process memory* and request a container with the memory of that derived size, -see also [detailed Memory Model](mem_setup.html#detailed-memory-model). +see also [detailed Memory Model](mem_detail.html).
    Warning: If Flink or user code allocates unmanaged off-heap (native) memory beyond the container size @@ -63,7 +63,7 @@ will dictate the optimal memory configurations of your cluster. ### Heap state backend When running a stateless job or using a heap state backend ([MemoryStateBackend](../state/state_backends.html#the-memorystatebackend) -or [FsStateBackend](../state/state_backends.html#the-fsstatebackend), set [managed memory](mem_setup.html#managed-memory) to zero. +or [FsStateBackend](../state/state_backends.html#the-fsstatebackend)), set [managed memory](mem_setup.html#managed-memory) to zero. This will ensure that the maximum amount of memory is allocated for user code on the JVM. ### RocksDB state backend diff --git a/docs/ops/memory/mem_tuning.zh.md b/docs/ops/memory/mem_tuning.zh.md index 9d83d00d8d..de54513411 100644 --- a/docs/ops/memory/mem_tuning.zh.md +++ b/docs/ops/memory/mem_tuning.zh.md @@ -31,7 +31,7 @@ depending on the use case and which options are important in which case. ## Configure memory for standalone deployment It is recommended to configure [total Flink memory](mem_setup.html#configure-total-memory) -([`taskmanager.memory.flink.size`](../config.html#taskmanager-memory-flink-size)) or its [components](mem_setup.html#detailed-memory-model) +([`taskmanager.memory.flink.size`](../config.html#taskmanager-memory-flink-size)) or its [components](mem_detail.html) for [standalone deployment](../deployment/cluster_setup.html) where you want to declare how much memory is given to Flink itself. Additionally, you can adjust *JVM metaspace* if it causes [problems](mem_trouble.html#outofmemoryerror-metaspace). @@ -47,7 +47,7 @@ It declares how much memory in total should be assigned to the Flink *JVM proces Note If you configure the *total Flink memory* Flink will implicitly add JVM memory components to derive the *total process memory* and request a container with the memory of that derived size, -see also [detailed Memory Model](mem_setup.html#detailed-memory-model). +see also [detailed Memory Model](mem_detail.html).
    Warning: If Flink or user code allocates unmanaged off-heap (native) memory beyond the container size @@ -63,7 +63,7 @@ will dictate the optimal memory configurations of your cluster. ### Heap state backend When running a stateless job or using a heap state backend ([MemoryStateBackend](../state/state_backends.html#the-memorystatebackend) -or [FsStateBackend](../state/state_backends.html#the-fsstatebackend), set [managed memory](mem_setup.html#managed-memory) to zero. +or [FsStateBackend](../state/state_backends.html#the-fsstatebackend)), set [managed memory](mem_setup.html#managed-memory) to zero. This will ensure that the maximum amount of memory is allocated for user code on the JVM. ### RocksDB state backend -- Gitee From ba38b036cd4add37ad851cd0c6ca7142c8e2a863 Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Wed, 11 Mar 2020 11:03:42 +0800 Subject: [PATCH 195/885] [FLINK-15991][docs-zh] Translate the index of "Memory Configuration" into Chinese. This closes #11401 --- docs/ops/memory/index.zh.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ops/memory/index.zh.md b/docs/ops/memory/index.zh.md index e2190e1281..df2ae8f4d6 100644 --- a/docs/ops/memory/index.zh.md +++ b/docs/ops/memory/index.zh.md @@ -1,5 +1,5 @@ --- -nav-title: 'Memory Configuration' +nav-title: '内存配置' nav-id: ops_mem nav-parent_id: ops nav-pos: 5 -- Gitee From b133cc1fc7627db7c0e4d7af028ff11b4e871eff Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Wed, 11 Mar 2020 19:51:21 +0800 Subject: [PATCH 196/885] [FLINK-15991][docs-zh] Translate the "Set up Task Executor Memory" page into Chinese. This closes #11401 --- docs/ops/memory/mem_detail.zh.md | 18 ++-- docs/ops/memory/mem_migration.zh.md | 32 +++---- docs/ops/memory/mem_setup.zh.md | 126 +++++++++++++--------------- docs/ops/memory/mem_trouble.zh.md | 6 +- docs/ops/memory/mem_tuning.zh.md | 18 ++-- 5 files changed, 93 insertions(+), 107 deletions(-) diff --git a/docs/ops/memory/mem_detail.zh.md b/docs/ops/memory/mem_detail.zh.md index a5b476f1dd..0c56c6cdfd 100644 --- a/docs/ops/memory/mem_detail.zh.md +++ b/docs/ops/memory/mem_detail.zh.md @@ -42,13 +42,13 @@ which affect the size of the respective components: |   **Component**   |   **Configuration options**   |   **Description**   | | :-------------------------------------------------------------------------------- | :------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------ | :----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | | [Framework Heap Memory](#framework-memory) | [`taskmanager.memory.framework.heap.size`](../config.html#taskmanager-memory-framework-heap-size) | JVM heap memory dedicated to Flink framework (advanced option) | -| [Task Heap Memory](mem_setup.html#task-operator-heap-memory) | [`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size) | JVM heap memory dedicated to Flink application to run operators and user code | -| [Managed memory](mem_setup.html#managed-memory) | [`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size)
    [`taskmanager.memory.managed.fraction`](../config.html#taskmanager-memory-managed-fraction) | Native memory managed by Flink, reserved for sorting, hash tables, caching of intermediate results and RocksDB state backend | -| [Framework Off-heap Memory](#framework-memory) | [`taskmanager.memory.framework.off-heap.size`](../config.html#taskmanager-memory-framework-off-heap-size) | [Off-heap direct (or native) memory](mem_setup.html#configure-off-heap-memory-direct-or-native) dedicated to Flink framework (advanced option) | -| [Task Off-heap Memory](mem_setup.html#configure-off-heap-memory-direct-or-native) | [`taskmanager.memory.task.off-heap.size`](../config.html#taskmanager-memory-task-off-heap-size) | [Off-heap direct (or native) memory](mem_setup.html#configure-off-heap-memory-direct-or-native) dedicated to Flink application to run operators | -| Network Memory | [`taskmanager.memory.network.min`](../config.html#taskmanager-memory-network-min)
    [`taskmanager.memory.network.max`](../config.html#taskmanager-memory-network-max)
    [`taskmanager.memory.network.fraction`](../config.html#taskmanager-memory-network-fraction) | Direct memory reserved for data record exchange between tasks (e.g. buffering for the transfer over the network), it is a [capped fractionated component](#capped-fractionated-components) of the [total Flink memory](mem_setup.html#configure-total-memory) | +| [Task Heap Memory](mem_setup.html#任务算子堆内存) | [`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size) | JVM heap memory dedicated to Flink application to run operators and user code | +| [Managed memory](mem_setup.html#托管内存) | [`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size)
    [`taskmanager.memory.managed.fraction`](../config.html#taskmanager-memory-managed-fraction) | Native memory managed by Flink, reserved for sorting, hash tables, caching of intermediate results and RocksDB state backend | +| [Framework Off-heap Memory](#framework-memory) | [`taskmanager.memory.framework.off-heap.size`](../config.html#taskmanager-memory-framework-off-heap-size) | [Off-heap direct (or native) memory](mem_setup.html#配置堆外内存直接内存或本地内存) dedicated to Flink framework (advanced option) | +| [Task Off-heap Memory](mem_setup.html#配置堆外内存直接内存或本地内存) | [`taskmanager.memory.task.off-heap.size`](../config.html#taskmanager-memory-task-off-heap-size) | [Off-heap direct (or native) memory](mem_setup.html#配置堆外内存直接内存或本地内存) dedicated to Flink application to run operators | +| Network Memory | [`taskmanager.memory.network.min`](../config.html#taskmanager-memory-network-min)
    [`taskmanager.memory.network.max`](../config.html#taskmanager-memory-network-max)
    [`taskmanager.memory.network.fraction`](../config.html#taskmanager-memory-network-fraction) | Direct memory reserved for data record exchange between tasks (e.g. buffering for the transfer over the network), it is a [capped fractionated component](#capped-fractionated-components) of the [total Flink memory](mem_setup.html#配置总内存) | | [JVM metaspace](#jvm-parameters) | [`taskmanager.memory.jvm-metaspace.size`](../config.html#taskmanager-memory-jvm-metaspace-size) | Metaspace size of the Flink JVM process | -| JVM Overhead | [`taskmanager.memory.jvm-overhead.min`](../config.html#taskmanager-memory-jvm-overhead-min)
    [`taskmanager.memory.jvm-overhead.max`](../config.html#taskmanager-memory-jvm-overhead-max)
    [`taskmanager.memory.jvm-overhead.fraction`](../config.html#taskmanager-memory-jvm-overhead-fraction) | Native memory reserved for other JVM overhead: e.g. thread stacks, code cache, garbage collection space etc, it is a [capped fractionated component](#capped-fractionated-components) of the [total process memory](mem_setup.html#configure-total-memory) | +| JVM Overhead | [`taskmanager.memory.jvm-overhead.min`](../config.html#taskmanager-memory-jvm-overhead-min)
    [`taskmanager.memory.jvm-overhead.max`](../config.html#taskmanager-memory-jvm-overhead-max)
    [`taskmanager.memory.jvm-overhead.fraction`](../config.html#taskmanager-memory-jvm-overhead-fraction) | Native memory reserved for other JVM overhead: e.g. thread stacks, code cache, garbage collection space etc, it is a [capped fractionated component](#capped-fractionated-components) of the [total process memory](mem_setup.html#配置总内存) | {:.table-bordered}
    @@ -68,7 +68,7 @@ The separation of framework and task memory can be used in future releases for f ## Capped Fractionated Components This section describes the configuration details of the following options which can be a fraction of a certain -[total memory](mem_setup.html#configure-total-memory): +[total memory](mem_setup.html#配置总内存): * *Network memory* can be a fraction of the *total Flink memory* * *JVM overhead* can be a fraction of the *total process memory* @@ -139,8 +139,8 @@ then all components are ignored except for the following:
    All of the components listed above can be but do not have to be explicitly configured for the local execution. -If they are not configured they are set to their default values. [Task heap memory](mem_setup.html#task-operator-heap-memory) and -*task off-heap memory* are considered to be infinite (*Long.MAX_VALUE* bytes) and [managed memory](mem_setup.html#managed-memory) +If they are not configured they are set to their default values. [Task heap memory](mem_setup.html#任务算子堆内存) and +*task off-heap memory* are considered to be infinite (*Long.MAX_VALUE* bytes) and [managed memory](mem_setup.html#托管内存) has a default value of 128Mb only for the local execution mode. Note The task heap size is not related in any way to the real heap size in this case. diff --git a/docs/ops/memory/mem_migration.zh.md b/docs/ops/memory/mem_migration.zh.md index e191940937..5b3cdeb35e 100644 --- a/docs/ops/memory/mem_migration.zh.md +++ b/docs/ops/memory/mem_migration.zh.md @@ -36,7 +36,7 @@ were removed or their semantics changed. This guide will help you to migrate the
    Note Before version *1.10*, Flink did not require that memory related options are set at all -as they all had default values. The [new memory configuration](mem_setup.html#configure-total-memory) requires +as they all had default values. The [new memory configuration](mem_setup.html#配置总内存) requires that at least one subset of the following options is configured explicitly, otherwise the configuration will fail: * [`taskmanager.memory.flink.size`](../config.html#taskmanager-memory-flink-size) * [`taskmanager.memory.process.size`](../config.html#taskmanager-memory-process-size) @@ -143,7 +143,7 @@ they will be directly translated into the following new options: It is also recommended to use these new options instead of the legacy ones as they might be completely removed in the following releases. -See also [how to configure total memory now](mem_setup.html#configure-total-memory). +See also [how to configure total memory now](mem_setup.html#配置总内存). ## JVM Heap Memory @@ -152,11 +152,11 @@ which included any other usages of heap memory. This rest was always implicitly see also [how to migrate managed memory](#managed-memory). Now, if only *total Flink memory* or *total process memory* is configured, then the JVM heap is also derived as the rest of -what is left after subtracting all other components from the total memory, see also [how to configure total memory](mem_setup.html#configure-total-memory). +what is left after subtracting all other components from the total memory, see also [how to configure total memory](mem_setup.html#配置总内存). Additionally, you can now have more direct control over the JVM heap assigned to the operator tasks ([`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size)), -see also [Task (Operator) Heap Memory](mem_setup.html#task-operator-heap-memory). +see also [Task (Operator) Heap Memory](mem_setup.html#任务算子堆内存). The JVM heap memory is also used by the heap state backends ([MemoryStateBackend](../state/state_backends.html#the-memorystatebackend) or [FsStateBackend](../state/state_backends.html#the-fsstatebackend)) if it is chosen for streaming jobs. @@ -166,7 +166,7 @@ See also [Framework memory](mem_detail.html#framework-memory). ## Managed Memory -See also [how to configure managed memory now](mem_setup.html#managed-memory). +See also [how to configure managed memory now](mem_setup.html#托管内存). ### Explicit Size @@ -180,15 +180,15 @@ If not set explicitly, the managed memory could be previously specified as a fra of the total memory minus network memory and container cut-off (only for [Yarn](../deployment/yarn_setup.html) and [Mesos](../deployment/mesos.html) deployments). This option has been completely removed and will have no effect if still used. Please, use the new option [`taskmanager.memory.managed.fraction`](../config.html#taskmanager-memory-managed-fraction) instead. -This new option will set the [managed memory](mem_setup.html#managed-memory) to the specified fraction of the -[total Flink memory](mem_setup.html#configure-total-memory) if its size is not set explicitly by +This new option will set the [managed memory](mem_setup.html#托管内存) to the specified fraction of the +[total Flink memory](mem_setup.html#配置总内存) if its size is not set explicitly by [`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size). ### RocksDB state If the [RocksDBStateBackend](../state/state_backends.html#the-rocksdbstatebackend) is chosen for a streaming job, -its native memory consumption should now be accounted for in [managed memory](mem_setup.html#managed-memory). -The RocksDB memory allocation is limited by the [managed memory](mem_setup.html#managed-memory) size. +its native memory consumption should now be accounted for in [managed memory](mem_setup.html#托管内存). +The RocksDB memory allocation is limited by the [managed memory](mem_setup.html#托管内存) size. This should prevent the killing of containers on [Yarn](../deployment/yarn_setup.html) or [Mesos](../deployment/mesos.html). You can disable the RocksDB memory control by setting [state.backend.rocksdb.memory.managed](../config.html#state-backend-rocksdb-memory-managed) to `false`. See also [how to migrate container cut-off](#container-cut-off-memory). @@ -196,9 +196,9 @@ to `false`. See also [how to migrate container cut-off](#container-cut-off-memor ### Other changes Additionally, the following changes have been made: -* The [managed memory](mem_setup.html#managed-memory) is always off-heap now. The configuration option `taskmanager.memory.off-heap` is removed and will have no effect anymore. -* The [managed memory](mem_setup.html#managed-memory) now uses native memory which is not direct memory. It means that the managed memory is no longer accounted for in the JVM direct memory limit. -* The [managed memory](mem_setup.html#managed-memory) is always lazily allocated now. The configuration option `taskmanager.memory.preallocate` is removed and will have no effect anymore. +* The [managed memory](mem_setup.html#托管内存) is always off-heap now. The configuration option `taskmanager.memory.off-heap` is removed and will have no effect anymore. +* The [managed memory](mem_setup.html#托管内存) now uses native memory which is not direct memory. It means that the managed memory is no longer accounted for in the JVM direct memory limit. +* The [managed memory](mem_setup.html#托管内存) is always lazily allocated now. The configuration option `taskmanager.memory.preallocate` is removed and will have no effect anymore. ## Container Cut-Off Memory @@ -209,9 +209,9 @@ will have no effect on the task manager process anymore. The new memory model in described further, to address these concerns. In streaming jobs which use [RocksDBStateBackend](../state/state_backends.html#the-rocksdbstatebackend), the RocksDB -native memory consumption should be accounted for as a part of the [managed memory](mem_setup.html#managed-memory) now. -The RocksDB memory allocation is also limited by the configured size of the [managed memory](mem_setup.html#managed-memory). -See also [migrating managed memory](#managed-memory) and [how to configure managed memory now](mem_setup.html#managed-memory). +native memory consumption should be accounted for as a part of the [managed memory](mem_setup.html#托管内存) now. +The RocksDB memory allocation is also limited by the configured size of the [managed memory](mem_setup.html#托管内存). +See also [migrating managed memory](#managed-memory) and [how to configure managed memory now](mem_setup.html#托管内存). The other direct or native off-heap memory consumers can now be addressed by the following new configuration options: * Task off-heap memory ([`taskmanager.memory.task.off-heap.size`](../config.html#taskmanager-memory-task-off-heap-size)) @@ -228,7 +228,7 @@ This section describes the changes of the default `flink-conf.yaml` shipped with The total memory (`taskmanager.heap.size`) is replaced by [`taskmanager.memory.process.size`](../config.html#taskmanager-memory-process-size) in the default `flink-conf.yaml`. The value is also increased from 1024Mb to 1568Mb. -See also [how to configure total memory now](mem_setup.html#configure-total-memory). +See also [how to configure total memory now](mem_setup.html#配置总内存).
    Warning: If you use the new default `flink-conf.yaml` it can result in different sizes of diff --git a/docs/ops/memory/mem_setup.zh.md b/docs/ops/memory/mem_setup.zh.md index 32cfd9c554..2a3a3e81b9 100644 --- a/docs/ops/memory/mem_setup.zh.md +++ b/docs/ops/memory/mem_setup.zh.md @@ -1,5 +1,5 @@ --- -title: "Set up Task Executor Memory" +title: "配置 TaskExecutor 内存" nav-parent_id: ops_mem nav-pos: 1 --- @@ -22,112 +22,98 @@ specific language governing permissions and limitations under the License. --> -Apache Flink provides efficient workloads on top of the JVM by tightly controlling the memory usage of its various components. -While the community strives to offer sensible defaults to all configurations, the full breadth of applications -that users deploy on Flink means this isn't always possible. To provide the most production value to our users, -Flink allows both high level and fine-grained tuning of memory allocation within clusters. +Apache Flink 基于 JVM 的高效处理能力,依赖于其对各组件内存用量的细致掌控。 +考虑到用户在 Flink 上运行的应用的多样性,尽管社区已经努力为所有配置项提供合理的默认值,仍无法满足所有情况下的需求。 +为了给用户生产提供最大化的价值, Flink 允许用户在整体上以及细粒度上对集群的内存分配进行调整。 * toc {:toc} -The further described memory configuration is applicable starting with the release version *1.10*. If you upgrade Flink -from earlier versions, check the [migration guide](mem_migration.html) because many changes were introduced with the *1.10* release. +本文接下来介绍的内存配置方法适用于 *1.10* 及以上版本。 +Flink 在 1.10 版本中对内存配置部分进行了较大幅度的改动,从早期版本升级的用户请参考[升级指南](mem_migration.html)。 -Note This memory setup guide is relevant only for task executors! -Check [job manager related configuration options](../config.html#jobmanager-heap-size) for the memory setup of job manager. +提示 本篇内存配置文档仅针对 TaskExecutor!关于 JobManager 的内存配置请参考 [JobManager 相关配置参数](../config.html#jobmanager-heap-size)。 -## Configure Total Memory +## 配置总内存 -The *total process memory* of Flink JVM processes consists of memory consumed by Flink application (*total Flink memory*) -and by the JVM to run the process. The *total Flink memory* consumption includes usage of JVM heap, -*managed memory* (managed by Flink) and other direct (or native) memory. +Flink JVM 进程的*进程总内存(Total Process Memory)*包含了由 Flink 应用使用的内存(*Flink 总内存*)以及由运行 Flink 的 JVM 使用的内存。 +其中,*Flink 总内存(Total Flink Memory)*包括 JVM 堆内存(Heap Memory)、*托管内存(Managed Memory)*以及其他直接内存(Direct Memory)或本地内存(Native Memory)。
    Simple memory model

    -If you run Flink locally (e.g. from your IDE) without creating a cluster, then only a subset of the memory configuration -options are relevant, see also [local execution](mem_detail.html#local-execution) for more details. +如果你是在本地运行 Flink(例如在 IDE 中)而非创建一个集群,那么本文介绍的配置并非所有都是适用的,详情请参考[本地执行](mem_detail.html#local-execution)。 -Otherwise, the simplest way to setup memory in Flink is to configure either of the two following options: -* Total Flink memory ([`taskmanager.memory.flink.size`](../config.html#taskmanager-memory-flink-size)) -* Total process memory ([`taskmanager.memory.process.size`](../config.html#taskmanager-memory-process-size)) +其他情况下,配置 Flink 内存最简单的方法就是配置下列两个参数中的任意一个。 +* Flink 总内存([`taskmanager.memory.flink.size`](../config.html#taskmanager-memory-flink-size)) +* 进程总内存([`taskmanager.memory.process.size`](../config.html#taskmanager-memory-process-size)) -The rest of the memory components will be adjusted automatically, based on default values or additionally configured options. -[Here](mem_detail.html) are more details about the other memory components. +Flink 会根据默认值或其他配置参数自动调整剩余内存部分的大小。关于各内存部分的更多细节,请参考[相关文档](mem_detail.html)。 -Configuring *total Flink memory* is better suited for standalone deployments where you want to declare how much memory -is given to Flink itself. The *total Flink memory* splits up into JVM heap, [managed memory size](#managed-memory) -and *direct memory*. +对于独立部署模式(Standalone Deployment),如果你希望指定由 Flink 应用本身使用的内存大小,最好选择配置 *Flink 总内存*。 +*Flink 总内存*会进一步划分为 JVM 堆内存、[托管内存](#托管内存)和*直接内存*。 -If you configure *total process memory* you declare how much memory in total should be assigned to the Flink *JVM process*. -For the containerized deployments it corresponds to the size of the requested container, see also -[how to configure memory for containers](mem_tuning.html#configure-memory-for-containers) -([Kubernetes](../deployment/kubernetes.html), [Yarn](../deployment/yarn_setup.html) or [Mesos](../deployment/mesos.html)). +通过配置*进程总内存*可以指定由 Flink *JVM 进程*使用的总内存大小。 +对于容器化部署模式(Containerized Deployment),这相当于申请的容器(Container)大小,详情请参考[如何配置容器内存](mem_tuning.html#configure-memory-for-containers)([Kubernetes](../deployment/kubernetes.html)、[Yarn](../deployment/yarn_setup.html) 或 [Mesos](../deployment/mesos.html))。 -Another way to setup the memory is to set [task heap](#task-operator-heap-memory) and [managed memory](#managed-memory) -([`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size) and [`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size)). -This more fine-grained approach is described in more detail [here](#configure-heap-and-managed-memory). +此外,还可以通过设置[任务堆内存(Task Heap Memory)](#任务算子堆内存)和[托管内存](#托管内存)的方式进行内存配置([`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size) 和 [`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size))。 +这是一种更细粒度的配置方式,更多细节请参考[相关文档](#配置堆内存和托管内存)。 -Note One of the three mentioned ways has to be used to configure Flink’s memory (except for local execution), or the Flink startup will fail. -This means that one of the following option subsets, which do not have default values, have to be configured explicitly: +提示 以上三种方式中,用户需要至少选择其中一种进行配置(本地运行除外),否则 Flink 将无法启动。 +这意味着,用户需要从以下无默认值的配置参数(或参数组合)中选择一个给出明确的配置: * [`taskmanager.memory.flink.size`](../config.html#taskmanager-memory-flink-size) * [`taskmanager.memory.process.size`](../config.html#taskmanager-memory-process-size) -* [`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size) and [`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size) +* [`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size) 和 [`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size) -Note Explicitly configuring both *total process memory* and *total Flink memory* is not recommended. -It may lead to deployment failures due to potential memory configuration conflicts. Additional configuration -of other memory components also requires caution as it can produce further configuration conflicts. +提示 不建议同时设置*进程总内存*和 *Flink 总内存*。 +这可能会造成内存配置冲突,从而导致部署失败。 +额外配置其他内存部分时,同样需要注意可能产生的配置冲突。 -## Configure Heap and Managed Memory +## 配置堆内存和托管内存 -As mentioned before in [total memory description](#configure-total-memory), another way to setup memory in Flink is -to specify explicitly both [task heap](#task-operator-heap-memory) and [managed memory](#managed-memory). -It gives more control over the available JVM heap to Flink’s tasks and its [managed memory](#managed-memory). +如[配置总内存](#配置总内存)中所述,另一种配置 Flink 内存的方式是同时设置[任务堆内存](#任务算子堆内存)和[托管内存](#托管内存)。 +通过这种方式,用户可以更好地掌控用于 Flink 任务的 JVM 堆内存及 Flink 的[托管内存](#托管内存)大小。 -The rest of the memory components will be adjusted automatically, based on default values or additionally configured options. -[Here](mem_detail.html) are more details about the other memory components. +Flink 会根据默认值或其他配置参数自动调整剩余内存部分的大小。关于各内存部分的更多细节,请参考[相关文档](mem_detail.html)。 -Note If you have configured the task heap and managed memory explicitly, it is recommended to set neither -*total process memory* nor *total Flink memory*. Otherwise, it may easily lead to memory configuration conflicts. +提示 如果已经明确设置了任务堆内存和托管内存,建议不要再设置*进程总内存*或 *Flink 总内存*,否则可能会造成内存配置冲突。 -### Task (Operator) Heap Memory +### 任务(算子)堆内存 -If you want to guarantee that a certain amount of JVM heap is available for your user code, you can set the *task heap memory* -explicitly ([`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size)). -It will be added to the JVM heap size and will be dedicated to Flink’s operators running the user code. +如果希望确保指定大小的 JVM 堆内存给用户代码使用,可以明确指定*任务堆内存*([`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size))。 +指定的内存将被包含在总的 JVM 堆空间中,专门用于 Flink 算子及用户代码的执行。 -### Managed Memory +### 托管内存 -*Managed memory* is managed by Flink and is allocated as native memory (off-heap). The following workloads use *managed memory*: -* Streaming jobs can use it for [RocksDB state backend](../state/state_backends.html#the-rocksdbstatebackend). -* [Batch jobs](../../dev/batch) can use it for sorting, hash tables, caching of intermediate results. +*托管内存*是由 Flink 负责分配和管理的本地(堆外)内存。 +以下场景需要使用*托管内存*: +* 流处理作业中用于 [RocksDB State Backend](../state/state_backends.html#the-rocksdbstatebackend)。 +* [批处理作业](../../dev/batch)中用于排序、哈希表及缓存中间结果。 -The size of *managed memory* can be -* either configured explicitly via [`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size) -* or computed as a fraction of *total Flink memory* via [`taskmanager.memory.managed.fraction`](../config.html#taskmanager-memory-managed-fraction). +可以通过以下两种范式指定*托管内存*的大小: +* 通过 [`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size) 明确指定其大小。 +* 通过 [`taskmanager.memory.managed.fraction`](../config.html#taskmanager-memory-managed-fraction) 指定在*Flink 总内存*中的占比。 -*Size* will override *fraction*, if both are set. -If neither *size* nor *fraction* is explicitly configured, the [default fraction](../config.html#taskmanager-memory-managed-fraction) will be used. +当同时指定二者时,会优先采用指定的大小(Size)。 +若二者均未指定,会根据[默认占比](../config.html#taskmanager-memory-managed-fraction)进行计算。 -See also [how to configure memory for state backends](mem_tuning.html#configure-memory-for-state-backends) and [batch jobs](mem_tuning.html#configure-memory-for-batch-jobs). +请同时参考[如何配置 State Backend 内存](mem_tuning.html#configure-memory-for-state-backends)以及[如何配置批处理作业内存](mem_tuning.html#configure-memory-for-batch-jobs)。 -## Configure Off-Heap Memory (direct or native) +## 配置堆外内存(直接内存或本地内存) -The off-heap memory which is allocated by user code should be accounted for in *task off-heap memory* -([`taskmanager.memory.task.off-heap.size`](../config.html#taskmanager-memory-task-off-heap-size)). +用户代码中分配的堆外内存被归为*任务堆外内存(Task Off-Heap Memory),可以通过 [`taskmanager.memory.task.off-heap.size`](../config.html#taskmanager-memory-task-off-heap-size) 指定。 -Note You can also adjust the [framework off-heap memory](mem_detail.html#framework-memory). This option is advanced -and only recommended to be changed if you are sure that the Flink framework needs more memory. +提示 你也可以调整[框架推外内存(Framework Off-Heap Memory)](mem_detail.html#framework-memory)。 +这是一个进阶配置,建议仅在确定 Flink 框架需要更多的内存时调整该配置。 -Flink includes the *framework off-heap memory* and *task off-heap memory* into the *direct memory* limit of the JVM, -see also [JVM parameters](mem_detail.html#jvm-parameters). +Flink 将*框架堆外内存*和*任务堆外内存*都计算在 JVM 的*直接内存*限制中,请参考 [JVM 参数](mem_detail.html#jvm-parameters)。 -Note Although, native non-direct memory usage can be accounted for as a part of the -*framework off-heap memory* or *task off-heap memory*, it will result in a higher JVM's *direct memory* limit in this case. +提示 本地内存(非直接内存)也可以被归在*框架堆外内存*或*任务推外内存*中,在这种情况下 JVM 的*直接内存*限制可能会高于实际需求。 -Note The *network memory* is also part of JVM *direct memory* but it is managed by Flink and guaranteed -to never exceed its configured size. Therefore, resizing the *network memory* will not help in this situation. +提示 *网络内存(Network Memory)*同样被计算在 JVM *直接内存*中。 +Flink 会负责管理网络内存,保证其实际用量不会超过配置大小。 +因此,调整*网络内存*的大小不会对其他堆外内存有实质上的影响。 -See also [the detailed memory model](mem_detail.html). +请参考[内存模型详解](mem_detail.html)。 diff --git a/docs/ops/memory/mem_trouble.zh.md b/docs/ops/memory/mem_trouble.zh.md index f3e6b137d7..4f584bc9dc 100644 --- a/docs/ops/memory/mem_trouble.zh.md +++ b/docs/ops/memory/mem_trouble.zh.md @@ -35,7 +35,7 @@ mentioned in the exception message. ## OutOfMemoryError: Java heap space The exception usually indicates that the JVM heap is too small. You can try to increase the JVM heap size -by increasing [total memory](mem_setup.html#configure-total-memory) or [task heap memory](mem_setup.html#task-operator-heap-memory). +by increasing [total memory](mem_setup.html#配置总内存) or [task heap memory](mem_setup.html#任务算子堆内存). Note You can also increase the [framework heap memory](mem_detail.html#framework-memory) but this option is advanced and should only be changed if you are sure that the Flink framework itself needs more memory. @@ -45,7 +45,7 @@ is advanced and should only be changed if you are sure that the Flink framework The exception usually indicates that the JVM *direct memory* limit is too small or that there is a *direct memory leak*. Check whether user code or other external dependencies use the JVM *direct memory* and that it is properly accounted for. You can try to increase its limit by adjusting [direct off-heap memory](mem_detail.html). -See also [how to configure off-heap memory](mem_setup.html#configure-off-heap-memory-direct-or-native) and +See also [how to configure off-heap memory](mem_setup.html#配置堆外内存直接内存或本地内存)) and the [JVM arguments](mem_detail.html#jvm-parameters) which Flink sets. ## OutOfMemoryError: Metaspace @@ -68,7 +68,7 @@ this usually indicates that Flink has not reserved enough native memory. You can monitoring system or from the error messages when a container gets killed by the deployment environment. If [RocksDBStateBackend](../state/state_backends.html#the-rocksdbstatebackend) is used and the memory controlling is disabled, -you can try to increase the [managed memory](mem_setup.html#managed-memory). +you can try to increase the [managed memory](mem_setup.html#托管内存). Alternatively, you can increase the [JVM overhead](mem_detail.html). See also [how to configure memory for containers](mem_tuning.html#configure-memory-for-containers). diff --git a/docs/ops/memory/mem_tuning.zh.md b/docs/ops/memory/mem_tuning.zh.md index de54513411..61131f0c38 100644 --- a/docs/ops/memory/mem_tuning.zh.md +++ b/docs/ops/memory/mem_tuning.zh.md @@ -30,7 +30,7 @@ depending on the use case and which options are important in which case. ## Configure memory for standalone deployment -It is recommended to configure [total Flink memory](mem_setup.html#configure-total-memory) +It is recommended to configure [total Flink memory](mem_setup.html#配置总内存) ([`taskmanager.memory.flink.size`](../config.html#taskmanager-memory-flink-size)) or its [components](mem_detail.html) for [standalone deployment](../deployment/cluster_setup.html) where you want to declare how much memory is given to Flink itself. Additionally, you can adjust *JVM metaspace* if it causes [problems](mem_trouble.html#outofmemoryerror-metaspace). @@ -40,7 +40,7 @@ only physical resources of the executing machine matter in this case. ## Configure memory for containers -It is recommended to configure [total process memory](mem_setup.html#configure-total-memory) +It is recommended to configure [total process memory](mem_setup.html#配置总内存) ([`taskmanager.memory.process.size`](../config.html#taskmanager-memory-process-size)) for the containerized deployments ([Kubernetes](../deployment/kubernetes.html), [Yarn](../deployment/yarn_setup.html) or [Mesos](../deployment/mesos.html)). It declares how much memory in total should be assigned to the Flink *JVM process* and corresponds to the size of the requested container. @@ -63,25 +63,25 @@ will dictate the optimal memory configurations of your cluster. ### Heap state backend When running a stateless job or using a heap state backend ([MemoryStateBackend](../state/state_backends.html#the-memorystatebackend) -or [FsStateBackend](../state/state_backends.html#the-fsstatebackend)), set [managed memory](mem_setup.html#managed-memory) to zero. +or [FsStateBackend](../state/state_backends.html#the-fsstatebackend)), set [managed memory](mem_setup.html#托管内存) to zero. This will ensure that the maximum amount of memory is allocated for user code on the JVM. ### RocksDB state backend The [RocksDBStateBackend](../state/state_backends.html#the-rocksdbstatebackend) uses native memory. By default, -RocksDB is setup to limit native memory allocation to the size of the [managed memory](mem_setup.html#managed-memory). +RocksDB is setup to limit native memory allocation to the size of the [managed memory](mem_setup.html#托管内存). Therefore, it is important to reserve enough *managed memory* for your state use case. If you disable the default RocksDB memory control, task executors can be killed in containerized deployments if RocksDB allocates memory above the limit of the requested container size -(the [total process memory](mem_setup.html#configure-total-memory)). +(the [total process memory](mem_setup.html#配置总内存)). See also [how to tune RocksDB memory](../state/large_state_tuning.html#tuning-rocksdb-memory) and [state.backend.rocksdb.memory.managed](../config.html#state-backend-rocksdb-memory-managed). ## Configure memory for batch jobs -Flink's batch operators leverage [managed memory](../memory/mem_setup.html#managed-memory) to run more efficiently. +Flink's batch operators leverage [managed memory](../memory/mem_setup.html#托管内存) to run more efficiently. In doing so, some operations can be performed directly on raw data without having to be deserialized into Java objects. -This means that [managed memory](../memory/mem_setup.html#managed-memory) configurations have practical effects -on the performance of your applications. Flink will attempt to allocate and use as much [managed memory](../memory/mem_setup.html#managed-memory) +This means that [managed memory](../memory/mem_setup.html#托管内存) configurations have practical effects +on the performance of your applications. Flink will attempt to allocate and use as much [managed memory](../memory/mem_setup.html#托管内存) as configured for batch jobs but not go beyond its limits. This prevents `OutOfMemoryError`'s because Flink knows precisely -how much memory it has to leverage. If the [managed memory](../memory/mem_setup.html#managed-memory) is not sufficient, +how much memory it has to leverage. If the [managed memory](../memory/mem_setup.html#托管内存) is not sufficient, Flink will gracefully spill to disk. -- Gitee From 1f852b7ee0c2c77be364409b78cee4a634b9e80e Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Thu, 12 Mar 2020 17:32:36 +0800 Subject: [PATCH 197/885] [FLINK-15991][docs-zh] Translate the "Detailed Memory Model" page into Chinese. This closes #11401 --- docs/ops/memory/mem_detail.zh.md | 171 ++++++++++++++-------------- docs/ops/memory/mem_migration.zh.md | 2 +- docs/ops/memory/mem_setup.zh.md | 6 +- docs/ops/memory/mem_trouble.zh.md | 6 +- 4 files changed, 90 insertions(+), 95 deletions(-) diff --git a/docs/ops/memory/mem_detail.zh.md b/docs/ops/memory/mem_detail.zh.md index 0c56c6cdfd..2f646f2de3 100644 --- a/docs/ops/memory/mem_detail.zh.md +++ b/docs/ops/memory/mem_detail.zh.md @@ -1,5 +1,5 @@ --- -title: "Detailed Memory Model" +title: "内存模型详解" nav-parent_id: ops_mem nav-pos: 2 --- @@ -22,128 +22,123 @@ specific language governing permissions and limitations under the License. --> -This section gives a detailed description of all components in Flink’s memory model of task executor. -Check [memory configuration guide](mem_setup.html) for the basic memory setup. +本文将详细介绍 Flink TaskExecutor 内存模型中的所有组成部分。 +关于基本的内存配置方法,请参考[配置指南](mem_setup.html)。 * toc {:toc} -## Overview +## 概述
    Simple memory model

    - -The following table lists all memory components, depicted above, and references Flink configuration options -which affect the size of the respective components: - -|   **Component**   |   **Configuration options**   |   **Description**   | -| :-------------------------------------------------------------------------------- | :------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------ | :----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | -| [Framework Heap Memory](#framework-memory) | [`taskmanager.memory.framework.heap.size`](../config.html#taskmanager-memory-framework-heap-size) | JVM heap memory dedicated to Flink framework (advanced option) | -| [Task Heap Memory](mem_setup.html#任务算子堆内存) | [`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size) | JVM heap memory dedicated to Flink application to run operators and user code | -| [Managed memory](mem_setup.html#托管内存) | [`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size)
    [`taskmanager.memory.managed.fraction`](../config.html#taskmanager-memory-managed-fraction) | Native memory managed by Flink, reserved for sorting, hash tables, caching of intermediate results and RocksDB state backend | -| [Framework Off-heap Memory](#framework-memory) | [`taskmanager.memory.framework.off-heap.size`](../config.html#taskmanager-memory-framework-off-heap-size) | [Off-heap direct (or native) memory](mem_setup.html#配置堆外内存直接内存或本地内存) dedicated to Flink framework (advanced option) | -| [Task Off-heap Memory](mem_setup.html#配置堆外内存直接内存或本地内存) | [`taskmanager.memory.task.off-heap.size`](../config.html#taskmanager-memory-task-off-heap-size) | [Off-heap direct (or native) memory](mem_setup.html#配置堆外内存直接内存或本地内存) dedicated to Flink application to run operators | -| Network Memory | [`taskmanager.memory.network.min`](../config.html#taskmanager-memory-network-min)
    [`taskmanager.memory.network.max`](../config.html#taskmanager-memory-network-max)
    [`taskmanager.memory.network.fraction`](../config.html#taskmanager-memory-network-fraction) | Direct memory reserved for data record exchange between tasks (e.g. buffering for the transfer over the network), it is a [capped fractionated component](#capped-fractionated-components) of the [total Flink memory](mem_setup.html#配置总内存) | -| [JVM metaspace](#jvm-parameters) | [`taskmanager.memory.jvm-metaspace.size`](../config.html#taskmanager-memory-jvm-metaspace-size) | Metaspace size of the Flink JVM process | -| JVM Overhead | [`taskmanager.memory.jvm-overhead.min`](../config.html#taskmanager-memory-jvm-overhead-min)
    [`taskmanager.memory.jvm-overhead.max`](../config.html#taskmanager-memory-jvm-overhead-max)
    [`taskmanager.memory.jvm-overhead.fraction`](../config.html#taskmanager-memory-jvm-overhead-fraction) | Native memory reserved for other JVM overhead: e.g. thread stacks, code cache, garbage collection space etc, it is a [capped fractionated component](#capped-fractionated-components) of the [total process memory](mem_setup.html#配置总内存) | +提示 任务堆外内存也包括了用户代码使用的本地内存(非直接内存)。 + +如上图所示,下表中列出了 Flink TaskExecutor 内存模型的所有组成部分,以影响其大小的相关配置参数。 + +|   **组成部分**   |   **配置参数**   |   **描述**   | +| :----------------------------------------------------------------------------- | :--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | :----------------------------------------------------------------------------------------------------------------------------------------- | +| [框架堆内存(Framework Heap Memory)](#框架内存) | [`taskmanager.memory.framework.heap.size`](../config.html#taskmanager-memory-framework-heap-size) | 用于 Flink 框架的 JVM 堆内存(进阶配置)。 | +| [任务堆内存(Task Heap Memory)](mem_setup.html#任务算子堆内存) | [`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size) | 用于 Flink 应用的算子及用户代码的 JVM 堆内存。 | +| [托管内存(Managed memory)](mem_setup.html#托管内存) | [`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size)
    [`taskmanager.memory.managed.fraction`](../config.html#taskmanager-memory-managed-fraction) | 由 Flink 管理的用于排序、哈希表、缓存中间结果及 RocksDB State Backend 的本地内存。 | +| [框架堆外内存(Framework Off-heap Memory)](#框架内存) | [`taskmanager.memory.framework.off-heap.size`](../config.html#taskmanager-memory-framework-off-heap-size) | 用于 Flink 框架的[堆外内存(直接内存或本地内存)](mem_setup.html#配置堆外内存直接内存或本地内存)(进阶配置)。 | +| [任务堆外内存(Task Off-heap Memory)](mem_setup.html#配置堆外内存直接内存或本地内存) | [`taskmanager.memory.task.off-heap.size`](../config.html#taskmanager-memory-task-off-heap-size) | 用于 Flink 应用的算计及用户代码的[堆外内存(直接内存或本地内存)](mem_setup.html#配置堆外内存直接内存或本地内存)。 | +| 网络内存(Network Memory) | [`taskmanager.memory.network.min`](../config.html#taskmanager-memory-network-min)
    [`taskmanager.memory.network.max`](../config.html#taskmanager-memory-network-max)
    [`taskmanager.memory.network.fraction`](../config.html#taskmanager-memory-network-fraction) | 用于任务之间数据传输的直接内存(例如网络传输缓冲)。该内存部分为基于 [Flink 总内存](mem_setup.html#配置总内存)的[受限的等比内存部分](#受限的等比内存部分)。 | +| [JVM Metaspace](#jvm-参数) | [`taskmanager.memory.jvm-metaspace.size`](../config.html#taskmanager-memory-jvm-metaspace-size) | Flink JVM 进程的 Metaspace。 | +| JVM 开销 | [`taskmanager.memory.jvm-overhead.min`](../config.html#taskmanager-memory-jvm-overhead-min)
    [`taskmanager.memory.jvm-overhead.max`](../config.html#taskmanager-memory-jvm-overhead-max)
    [`taskmanager.memory.jvm-overhead.fraction`](../config.html#taskmanager-memory-jvm-overhead-fraction) | 用于其他 JVM 开销的本地内存,例如栈空间、垃圾回收空间等。该内存部分为基于[进程总内存](mem_setup.html#配置总内存)的[受限的等比内存部分](#受限的等比内存部分)。 | {:.table-bordered}
    -As you can see, the size of some memory components can be simply set by the respective option. -Other components can be tuned using multiple options. - -## Framework Memory +我们可以看到,有些内存部分的大小可以直接通过一个配置参数进行设置,有些则需要根据多个参数进行调整。 -The *framework heap memory* and *framework off-heap memory* options are not supposed to be changed without a good reason. -Adjust them only if you are sure that Flink needs more memory for some internal data structures or operations. -It can be related to a particular deployment environment or job structure, like high parallelism. -In addition, Flink dependencies, such as Hadoop may consume more direct or native memory in certain setups. +## 框架内存 -Note Neither heap nor off-heap versions of framework and task memory are currently isolated within Flink. -The separation of framework and task memory can be used in future releases for further optimizations. +通常情况下,不建议对*框架堆内存*和*框架堆外内存*进行调整。 +除非你非常肯定 Flink 的内部数据结构及操作需要更多的内存。 +这可能与具体的部署环境及作业结构有关,例如非常高的并发度。 +此外,Flink 的部分依赖(例如 Hadoop)在某些特定的情况下也可能会需要更多的直接内存或本地内存。 -## Capped Fractionated Components +提示 不管是堆内存还是堆外内存,Flink 中的框架内存和任务内存之间目前是没有隔离的。 +对框架和任务内存的区分,主要是为了在后续版本中做进一步优化。 -This section describes the configuration details of the following options which can be a fraction of a certain -[total memory](mem_setup.html#配置总内存): +## 受限的等比内存部分 -* *Network memory* can be a fraction of the *total Flink memory* -* *JVM overhead* can be a fraction of the *total process memory* +本节介绍下列内存部分的配置方法,它们都可以通过指定在[总内存](mem_setup.html#配置总内存)中所占比例的方式进行配置。 +* *网络内存*:可以配置占用 *Flink 总内存*的固定比例 +* *JVM 开销*:可以配置占用*进程总内存*的固定比例 -See also [detailed memory model](#overview). +请同时参考[概述部分](#概述)。 -The size of those components always has to be between its maximum and minimum value, otherwise Flink startup will fail. -The maximum and minimum values have defaults or can be explicitly set by corresponding configuration options. -For example, if only the following memory options are set: -- total Flink memory = 1000Mb, -- network min = 64Mb, -- network max = 128Mb, -- network fraction = 0.1 +这些内存部分的大小必须在相应的最大值、最小值范围内,否则 Flink 将无法启动。 +最大值、最小值具有默认值,也可以通过相应的配置参数进行设置。 +例如,如果仅配置下列参数: +- Flink 总内存 = 1000Mb +- 网络内存最小值 = 64Mb +- 网络内存最大值 = 128Mb +- 网络内存占比 = 0.1 -then the network memory will be 1000Mb x 0.1 = 100Mb which is within the range 64-128Mb. +那么网络内存的实际大小将会是 1000Mb x 0.1 = 100Mb,在 64-128Mb 的范围内。 -Notice if you configure the same maximum and minimum value it effectively means that its size is fixed to that value. +如果将最大值、最小值设置成相同大小,那相当于明确指定了该内存部分的大小。 -If the component memory is not explicitly configured, then Flink will use the fraction to calculate the memory size -based on the total memory. The calculated value is capped by its corresponding min/max options. -For example, if only the following memory options are set: -- total Flink memory = 1000Mb, -- network min = 128Mb, -- network max = 256Mb, -- network fraction = 0.1 +如果没有明确指定内存部分的大小,Flink 会根据总内存和占比计算出该内存部分的大小。 +计算得到的内存大小将受限于相应的最大值、最小。 +例如,如果仅配置下列参数: +- Flink 总内存 = 1000Mb +- 网络内存最小值 = 128Mb +- 网络内存最大值 = 256Mb +- 网络内存占比 = 0.1 -then the network memory will be 128Mb because the size derived from fraction is 100Mb and it is less than the minimum. +那么网络内存的实际大小将会是 128Mb,因为根据总内存和占比计算得到的内存大小 100Mb 小于最小值。 -It can also happen that the fraction is ignored if the sizes of the total memory and its other components are defined. -In this case, the network memory is the rest of the total memory. The derived value still has to be within its min/max -range otherwise the configuration fails. For example, suppose only the following memory options are set: -- total Flink memory = 1000Mb, -- task heap = 100Mb, -- network min = 64Mb, -- network max = 256Mb, -- network fraction = 0.1 +如果配置了总内存和其他内存部分的大小,那么 Flink 也有可能会忽略给定的占比。 +这种情况下,受限的等比内存部分的实际大小是总内存减去其他所有内存部分后剩余的部分。 +这样推导得出的内存大小必须符合最大值、最小值范围,否则 Flink 将无法启动。 +例如,如果仅配置下列参数: +- Flink 总内存 = 1000Mb, +- 任务堆内存 = 100Mb, +- 网络内存最小值 = 64Mb +- 网络内存最大值 = 256Mb +- 网络内存占比 = 0.1 -All other components of the total Flink memory have default values, including the default managed memory fraction. -Then the network memory is not the fraction (1000Mb x 0.1 = 100Mb) but the rest of the total Flink memory -which will either be within the range 64-256Mb or fail. +Flink 总内存中所有其他内存部分均有默认大小(包括托管内存的默认占比)。 +因此,网络内存的实际大小不是根据占比算出的大小(1000Mb x 0.1 = 100Mb),而是 Flink 总内存中剩余的部分。 +这个剩余部分的大小必须在 64-256Mb 的范围内,否则将会启动失败。 -## JVM Parameters +## JVM 参数 -Flink explicitly adds the following memory related JVM arguments while starting the task executor process, -based on the configured or derived memory component sizes: +Flink 启动 TaskExecutor 进程时,会根据配置的和自动推导出的各内存部分大小,显式地设置以下 JVM 参数: -|   **JVM Arguments**   |   **Value**   | -| :---------------------------------------- | :----------------------------------------- | -| *-Xmx* and *-Xms* | Framework + Task Heap Memory | -| *-XX:MaxDirectMemorySize* | Framework + Task Off-Heap + Network Memory | -| *-XX:MaxMetaspaceSize* | JVM Metaspace | +|   **JVM 参数**   |   **值**   | +| :---------------------------------- | :------------------------------- | +| *-Xmx* 和 *-Xms* | 框架堆内存 + 任务堆内存 | +| *-XX:MaxDirectMemorySize* | 框架堆外内存 + 任务堆外内存 + 网络内存 | +| *-XX:MaxMetaspaceSize* | JVM Metaspace | {:.table-bordered}
    -See also [detailed memory model](#overview). +请同时参考[概述部分](#概述)。 -## Local Execution -If you start Flink locally on your machine as a single java program without creating a cluster (e.g. from your IDE) -then all components are ignored except for the following: +## 本地执行 +如果你是将 Flink 作为一个单独的 Java 程序运行在你的电脑本地而非创建一个集群(例如在 IDE 中),那么只有下列配置会生效,其他配置参数则不会起到任何效果: -|   **Memory component**   |   **Relevant options**   |   **Default value for the local execution**   | -| :------------------------------------------- | :-------------------------------------------------------------------------------------------- | :---------------------------------------------------------------------------- | -| Task heap | [`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size) | infinite | -| Task off-heap | [`taskmanager.memory.task.off-heap.size`](../config.html#taskmanager-memory-task-off-heap-size) | infinite | -| Managed memory | [`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size) | 128Mb | -| Network memory | [`taskmanager.memory.network.min`](../config.html#taskmanager-memory-network-min)
    [`taskmanager.memory.network.max`](../config.html#taskmanager-memory-network-max) | 64Mb | +|   **组成部分**   |   **配置参数**   |   **本地执行时的默认值**   | +| :---------------------------------- | :------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | :------------------------------------------ | +| 任务堆内存 | [`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size) | 无穷大 | +| 任务堆外内存 | [`taskmanager.memory.task.off-heap.size`](../config.html#taskmanager-memory-task-off-heap-size) | 无穷大 | +| 托管内存 | [`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size) | 128Mb | +| 网络内存 | [`taskmanager.memory.network.min`](../config.html#taskmanager-memory-network-min)
    [`taskmanager.memory.network.max`](../config.html#taskmanager-memory-network-max) | 64Mb | {:.table-bordered}
    -All of the components listed above can be but do not have to be explicitly configured for the local execution. -If they are not configured they are set to their default values. [Task heap memory](mem_setup.html#任务算子堆内存) and -*task off-heap memory* are considered to be infinite (*Long.MAX_VALUE* bytes) and [managed memory](mem_setup.html#托管内存) -has a default value of 128Mb only for the local execution mode. +本地执行模式下,上面列出的所有内存部分均可以但不是必须进行配置。 +如果未配置,则会采用默认值。 +其中,[*任务堆内存*](mem_setup.html#任务算子堆内存)和*任务堆外内存*的默认值无穷大(*Long.MAX_VALUE*字节),以及[托管内存](mem_setup.html#托管内存)的默认值 128Mb 均只针对本地执行模式。 -Note The task heap size is not related in any way to the real heap size in this case. -It can become relevant for future optimizations coming with next releases. The actual JVM heap size of the started -local process is not controlled by Flink and depends on how you start the process. -If you want to control the JVM heap size you have to explicitly pass the corresponding JVM arguments, e.g. *-Xmx*, *-Xms*. +提示 这种情况下,任务堆内存的大小与实际的堆空间大小无关。 +该配置参数可能与后续版本中的进一步优化相关。 +本地执行模式下,JVM 堆空间的实际大小不受 Flink 掌控,而是取决于本地执行进程是如何启动的。 +如果希望控制 JVM 的堆空间大小,可以在启动进程时明确地指定相关的 JVM 参数,即 *-Xmx* 和 *-Xms*. diff --git a/docs/ops/memory/mem_migration.zh.md b/docs/ops/memory/mem_migration.zh.md index 5b3cdeb35e..d42bb7c787 100644 --- a/docs/ops/memory/mem_migration.zh.md +++ b/docs/ops/memory/mem_migration.zh.md @@ -162,7 +162,7 @@ or [FsStateBackend](../state/state_backends.html#the-fsstatebackend)) if it is c A part of the JVM heap is now always reserved for Flink framework ([`taskmanager.memory.framework.heap.size`](../config.html#taskmanager-memory-framework-heap-size)). -See also [Framework memory](mem_detail.html#framework-memory). +See also [Framework memory](mem_detail.html#框架内存). ## Managed Memory diff --git a/docs/ops/memory/mem_setup.zh.md b/docs/ops/memory/mem_setup.zh.md index 2a3a3e81b9..7fd329a7d6 100644 --- a/docs/ops/memory/mem_setup.zh.md +++ b/docs/ops/memory/mem_setup.zh.md @@ -44,7 +44,7 @@ Flink JVM 进程的*进程总内存(Total Process Memory)*包含了由 Flink
    -如果你是在本地运行 Flink(例如在 IDE 中)而非创建一个集群,那么本文介绍的配置并非所有都是适用的,详情请参考[本地执行](mem_detail.html#local-execution)。 +如果你是在本地运行 Flink(例如在 IDE 中)而非创建一个集群,那么本文介绍的配置并非所有都是适用的,详情请参考[本地执行](mem_detail.html#本地执行)。 其他情况下,配置 Flink 内存最简单的方法就是配置下列两个参数中的任意一个。 * Flink 总内存([`taskmanager.memory.flink.size`](../config.html#taskmanager-memory-flink-size)) @@ -105,10 +105,10 @@ Flink 会根据默认值或其他配置参数自动调整剩余内存部分的 用户代码中分配的堆外内存被归为*任务堆外内存(Task Off-Heap Memory),可以通过 [`taskmanager.memory.task.off-heap.size`](../config.html#taskmanager-memory-task-off-heap-size) 指定。 -提示 你也可以调整[框架推外内存(Framework Off-Heap Memory)](mem_detail.html#framework-memory)。 +提示 你也可以调整[框架推外内存(Framework Off-Heap Memory)](mem_detail.html#框架内存)。 这是一个进阶配置,建议仅在确定 Flink 框架需要更多的内存时调整该配置。 -Flink 将*框架堆外内存*和*任务堆外内存*都计算在 JVM 的*直接内存*限制中,请参考 [JVM 参数](mem_detail.html#jvm-parameters)。 +Flink 将*框架堆外内存*和*任务堆外内存*都计算在 JVM 的*直接内存*限制中,请参考 [JVM 参数](mem_detail.html#jvm-参数)。 提示 本地内存(非直接内存)也可以被归在*框架堆外内存*或*任务推外内存*中,在这种情况下 JVM 的*直接内存*限制可能会高于实际需求。 diff --git a/docs/ops/memory/mem_trouble.zh.md b/docs/ops/memory/mem_trouble.zh.md index 4f584bc9dc..e29bc8e7de 100644 --- a/docs/ops/memory/mem_trouble.zh.md +++ b/docs/ops/memory/mem_trouble.zh.md @@ -37,7 +37,7 @@ mentioned in the exception message. The exception usually indicates that the JVM heap is too small. You can try to increase the JVM heap size by increasing [total memory](mem_setup.html#配置总内存) or [task heap memory](mem_setup.html#任务算子堆内存). -Note You can also increase the [framework heap memory](mem_detail.html#framework-memory) but this option +Note You can also increase the [framework heap memory](mem_detail.html#框架内存) but this option is advanced and should only be changed if you are sure that the Flink framework itself needs more memory. ## OutOfMemoryError: Direct buffer memory @@ -46,11 +46,11 @@ The exception usually indicates that the JVM *direct memory* limit is too small Check whether user code or other external dependencies use the JVM *direct memory* and that it is properly accounted for. You can try to increase its limit by adjusting [direct off-heap memory](mem_detail.html). See also [how to configure off-heap memory](mem_setup.html#配置堆外内存直接内存或本地内存)) and -the [JVM arguments](mem_detail.html#jvm-parameters) which Flink sets. +the [JVM arguments](mem_detail.html#jvm-参数) which Flink sets. ## OutOfMemoryError: Metaspace -The exception usually indicates that [JVM metaspace limit](mem_detail.html#jvm-parameters) is configured too small. +The exception usually indicates that [JVM metaspace limit](mem_detail.html#jvm-参数) is configured too small. You can try to increase the [JVM metaspace option](../config.html#taskmanager-memory-jvm-metaspace-size). ## IOException: Insufficient number of network buffers -- Gitee From 61d0e0e5e229cfc5a7e43c70d592f8b9fd460575 Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Fri, 13 Mar 2020 12:41:09 +0800 Subject: [PATCH 198/885] [FLINK-15991][docs-zh] Translate the "Memory tuning guide" page into Chinese. This closes #11401 --- docs/ops/memory/mem_setup.zh.md | 4 +- docs/ops/memory/mem_trouble.zh.md | 2 +- docs/ops/memory/mem_tuning.zh.md | 77 +++++++++++++------------------ 3 files changed, 36 insertions(+), 47 deletions(-) diff --git a/docs/ops/memory/mem_setup.zh.md b/docs/ops/memory/mem_setup.zh.md index 7fd329a7d6..0e30b53744 100644 --- a/docs/ops/memory/mem_setup.zh.md +++ b/docs/ops/memory/mem_setup.zh.md @@ -56,7 +56,7 @@ Flink 会根据默认值或其他配置参数自动调整剩余内存部分的 *Flink 总内存*会进一步划分为 JVM 堆内存、[托管内存](#托管内存)和*直接内存*。 通过配置*进程总内存*可以指定由 Flink *JVM 进程*使用的总内存大小。 -对于容器化部署模式(Containerized Deployment),这相当于申请的容器(Container)大小,详情请参考[如何配置容器内存](mem_tuning.html#configure-memory-for-containers)([Kubernetes](../deployment/kubernetes.html)、[Yarn](../deployment/yarn_setup.html) 或 [Mesos](../deployment/mesos.html))。 +对于容器化部署模式(Containerized Deployment),这相当于申请的容器(Container)大小,详情请参考[如何配置容器内存](mem_tuning.html#容器container的内存配置)([Kubernetes](../deployment/kubernetes.html)、[Yarn](../deployment/yarn_setup.html) 或 [Mesos](../deployment/mesos.html))。 此外,还可以通过设置[任务堆内存(Task Heap Memory)](#任务算子堆内存)和[托管内存](#托管内存)的方式进行内存配置([`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size) 和 [`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size))。 这是一种更细粒度的配置方式,更多细节请参考[相关文档](#配置堆内存和托管内存)。 @@ -99,7 +99,7 @@ Flink 会根据默认值或其他配置参数自动调整剩余内存部分的 当同时指定二者时,会优先采用指定的大小(Size)。 若二者均未指定,会根据[默认占比](../config.html#taskmanager-memory-managed-fraction)进行计算。 -请同时参考[如何配置 State Backend 内存](mem_tuning.html#configure-memory-for-state-backends)以及[如何配置批处理作业内存](mem_tuning.html#configure-memory-for-batch-jobs)。 +请同时参考[如何配置 State Backend 内存](mem_tuning.html#state-backend-的内存配置)以及[如何配置批处理作业内存](mem_tuning.html#批处理作业的内存配置)。 ## 配置堆外内存(直接内存或本地内存) diff --git a/docs/ops/memory/mem_trouble.zh.md b/docs/ops/memory/mem_trouble.zh.md index e29bc8e7de..3e661f98ec 100644 --- a/docs/ops/memory/mem_trouble.zh.md +++ b/docs/ops/memory/mem_trouble.zh.md @@ -71,4 +71,4 @@ If [RocksDBStateBackend](../state/state_backends.html#the-rocksdbstatebackend) i you can try to increase the [managed memory](mem_setup.html#托管内存). Alternatively, you can increase the [JVM overhead](mem_detail.html). -See also [how to configure memory for containers](mem_tuning.html#configure-memory-for-containers). +See also [how to configure memory for containers](mem_tuning.html#容器container的内存配置). diff --git a/docs/ops/memory/mem_tuning.zh.md b/docs/ops/memory/mem_tuning.zh.md index 61131f0c38..90926621de 100644 --- a/docs/ops/memory/mem_tuning.zh.md +++ b/docs/ops/memory/mem_tuning.zh.md @@ -1,5 +1,5 @@ --- -title: "Memory tuning guide" +title: "调优指南" nav-parent_id: ops_mem nav-pos: 3 --- @@ -22,66 +22,55 @@ specific language governing permissions and limitations under the License. --> -In addition to the [main memory setup guide](mem_setup.html), this section explains how to setup memory of task executors -depending on the use case and which options are important in which case. +本文在的基本的[配置指南](mem_setup.html)的基础上,介绍如何根据具体的使用场景调整 TaskExecutor 的内存配置,以及在不同使用场景下分别需要重点关注哪些配置参数。 * toc {:toc} -## Configure memory for standalone deployment +## 独立部署模式(Standalone Deployment)下的内存配置 -It is recommended to configure [total Flink memory](mem_setup.html#配置总内存) -([`taskmanager.memory.flink.size`](../config.html#taskmanager-memory-flink-size)) or its [components](mem_detail.html) -for [standalone deployment](../deployment/cluster_setup.html) where you want to declare how much memory is given to Flink itself. -Additionally, you can adjust *JVM metaspace* if it causes [problems](mem_trouble.html#outofmemoryerror-metaspace). +[独立部署模式](../deployment/cluster_setup.html),我们通常更关注 Flink 应用本身使用的内存大小。 +建议配置 [Flink 总内存](mem_setup.html#配置总内存)([`taskmanager.memory.flink.size`](../config.html#taskmanager-memory-flink-size))或者它的[组成部分](mem_detail.html)。 +此外,如果出现 [Metaspace 不足的问题](mem_trouble.html#outofmemoryerror-metaspace),可以调整 *JVM Metaspace* 的大小。 -The *total Process memory* is not relevant because *JVM overhead* is not controlled by Flink or deployment environment, -only physical resources of the executing machine matter in this case. +这种情况下通常无需配置*进程总内存*,因为不管是 Flink 还是部署环境都不会对 *JVM 开销* 进行限制,它只与机器的物理资源相关。 -## Configure memory for containers +## 容器(Container)的内存配置 -It is recommended to configure [total process memory](mem_setup.html#配置总内存) -([`taskmanager.memory.process.size`](../config.html#taskmanager-memory-process-size)) for the containerized deployments -([Kubernetes](../deployment/kubernetes.html), [Yarn](../deployment/yarn_setup.html) or [Mesos](../deployment/mesos.html)). -It declares how much memory in total should be assigned to the Flink *JVM process* and corresponds to the size of the requested container. +在容器化部署模式(Containerized Deployment)下([Kubernetes](../deployment/kubernetes.html)、[Yarn](../deployment/yarn_setup.html) 或 [Mesos](../deployment/mesos.html)),建议配置[进程总内存](mem_setup.html#配置总内存)([`taskmanager.memory.process.size`](../config.html#taskmanager-memory-process-size))。 +该配置参数用于指定分配给 Flink *JVM 进程*的总内存,也就是需要申请的容器大小。 -Note If you configure the *total Flink memory* Flink will implicitly add JVM memory components -to derive the *total process memory* and request a container with the memory of that derived size, -see also [detailed Memory Model](mem_detail.html). +提示 如果配置了 *Flink 总内存*,Flink 会自动加上 JVM 相关的内存部分,根据推算出的*进程总内存*大小申请容器。 +请参考[内存模型详解](mem_detail.html)。
    - Warning: If Flink or user code allocates unmanaged off-heap (native) memory beyond the container size - the job can fail because the deployment environment can kill the offending containers. + 注意: 如果 Flink 或者用户代码分配超过容器大小的非托管的堆外(本地)内存,部署环境可能会杀掉超用内存的容器,造成作业执行失败。
    -See also description of [container memory exceeded](mem_trouble.html#container-memory-exceeded) failure. +请参考[容器内存超用](mem_trouble.html#container-memory-exceeded)中的相关描述。 -## Configure memory for state backends +## State Backend 的内存配置 -When deploying a Flink streaming application, the type of [state backend](../state/state_backends.html) used -will dictate the optimal memory configurations of your cluster. +在部署 Flink 流处理应用时,可以根据 [State Backend](../state/state_backends.html) 的类型对集群的配置进行优化。 -### Heap state backend +### Heap State Backend -When running a stateless job or using a heap state backend ([MemoryStateBackend](../state/state_backends.html#the-memorystatebackend) -or [FsStateBackend](../state/state_backends.html#the-fsstatebackend)), set [managed memory](mem_setup.html#托管内存) to zero. -This will ensure that the maximum amount of memory is allocated for user code on the JVM. +执行无状态作业或者使用 Heap State Backend([MemoryStateBackend](../state/state_backends.html#memorystatebackend) +或 [FsStateBackend](../state/state_backends.html#fsstatebackend))时,建议将[托管内存](mem_setup.html#托管内存)设置为 0。 +这样能够最大化分配给 JVM 上用户代码的内存。 -### RocksDB state backend +### RocksDB State Backend -The [RocksDBStateBackend](../state/state_backends.html#the-rocksdbstatebackend) uses native memory. By default, -RocksDB is setup to limit native memory allocation to the size of the [managed memory](mem_setup.html#托管内存). -Therefore, it is important to reserve enough *managed memory* for your state use case. If you disable the default RocksDB memory control, -task executors can be killed in containerized deployments if RocksDB allocates memory above the limit of the requested container size -(the [total process memory](mem_setup.html#配置总内存)). -See also [how to tune RocksDB memory](../state/large_state_tuning.html#tuning-rocksdb-memory) -and [state.backend.rocksdb.memory.managed](../config.html#state-backend-rocksdb-memory-managed). +[RocksDBStateBackend](../state/state_backends.html#rocksdbstatebackend) 使用本地内存。 +默认情况下,RocksDB 会限制其内存用量不超过用户配置的[*托管内存*](mem_setup.html#托管内存)。 +因此,使用这种方式存储状态时,配置足够多的*托管内存*是十分重要的。 +如果你关闭了 RocksDB 的内存控制,那么在容器化部署模式下如果 RocksDB 分配的内存超出了申请容器的大小([进程总内存](mem_setup.html#配置总内存)),可能会造成 TaskExecutor 被部署环境杀掉。 +请同时参考[如何调整 RocksDB 内存](../state/large_state_tuning.html#tuning-rocksdb-memory)以及 [state.backend.rocksdb.memory.managed](../config.html#state-backend-rocksdb-memory-managed)。 -## Configure memory for batch jobs +## 批处理作业的内存配置 -Flink's batch operators leverage [managed memory](../memory/mem_setup.html#托管内存) to run more efficiently. -In doing so, some operations can be performed directly on raw data without having to be deserialized into Java objects. -This means that [managed memory](../memory/mem_setup.html#托管内存) configurations have practical effects -on the performance of your applications. Flink will attempt to allocate and use as much [managed memory](../memory/mem_setup.html#托管内存) -as configured for batch jobs but not go beyond its limits. This prevents `OutOfMemoryError`'s because Flink knows precisely -how much memory it has to leverage. If the [managed memory](../memory/mem_setup.html#托管内存) is not sufficient, -Flink will gracefully spill to disk. +Flink 批处理算子使用[托管内存](../memory/mem_setup.html#托管内存)来提高处理效率。 +算子运行时,部分操作可以直接在原始数据上进行,而无需将数据反序列化成 Java 对象。 +这意味着[托管内存](../memory/mem_setup.html#托管内存)对应用的性能具有实质上的影响。 +因此 Flink 会在不超过其配置限额的前提下,尽可能分配更多的[托管内存](../memory/mem_setup.html#托管内存)。 +Flink 明确知道可以使用的内存大小,因此可以有效避免 `OutOfMemoryError` 的发生。 +当[托管内存](../memory/mem_setup.html#托管内存)不足时,Flink 会优雅的将数据落盘。 -- Gitee From 3209719d7eaa28c6dc74251ce2ce2c5227d9a300 Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Fri, 13 Mar 2020 13:25:54 +0800 Subject: [PATCH 199/885] [FLINK-15991][docs-zh] Translate the "Troubleshooting" page into Chinese. This closes #11401 --- docs/ops/memory/mem_trouble.zh.md | 47 ++++++++++++++----------------- docs/ops/memory/mem_tuning.zh.md | 2 +- 2 files changed, 22 insertions(+), 27 deletions(-) diff --git a/docs/ops/memory/mem_trouble.zh.md b/docs/ops/memory/mem_trouble.zh.md index 3e661f98ec..2e3377c041 100644 --- a/docs/ops/memory/mem_trouble.zh.md +++ b/docs/ops/memory/mem_trouble.zh.md @@ -1,5 +1,5 @@ --- -title: "Troubleshooting" +title: "常见问题" nav-parent_id: ops_mem nav-pos: 4 --- @@ -27,48 +27,43 @@ under the License. ## IllegalConfigurationException -If you see an *IllegalConfigurationException* thrown from *TaskExecutorProcessUtils*, it usually indicates -that there is either an invalid configuration value (e.g. negative memory size, fraction that is greater than 1, etc.) -or configuration conflicts. Check the documentation chapters related to the [memory components](mem_detail.html) -mentioned in the exception message. +如果遇到从 *TaskExecutorProcessUtils* 抛出的 *IllegalConfigurationException* 异常,这通常说明您的配置参数中存在无效值(例如内存大小为负数、占比大于 1 等)或者配置冲突。 +请根据异常信息,确认[内存模型详解](mem_detail.html)中与出错的内存部分对应章节的内容。 ## OutOfMemoryError: Java heap space -The exception usually indicates that the JVM heap is too small. You can try to increase the JVM heap size -by increasing [total memory](mem_setup.html#配置总内存) or [task heap memory](mem_setup.html#任务算子堆内存). +该异常说明 JVM 的堆空间过小。 +可以通过增大[总内存](mem_setup.html#配置总内存)或[任务堆内存](mem_setup.html#任务算子堆内存)的方法来增大 JVM 堆空间。 -Note You can also increase the [framework heap memory](mem_detail.html#框架内存) but this option -is advanced and should only be changed if you are sure that the Flink framework itself needs more memory. +提示 也可以增大[框架堆内存](mem_detail.html#框架内存)。这是一个进阶配置,只有在确认是 Flink 框架自身需要更多内存时才应该去调整。 ## OutOfMemoryError: Direct buffer memory -The exception usually indicates that the JVM *direct memory* limit is too small or that there is a *direct memory leak*. -Check whether user code or other external dependencies use the JVM *direct memory* and that it is properly accounted for. -You can try to increase its limit by adjusting [direct off-heap memory](mem_detail.html). -See also [how to configure off-heap memory](mem_setup.html#配置堆外内存直接内存或本地内存)) and -the [JVM arguments](mem_detail.html#jvm-参数) which Flink sets. +该异常通常说明 JVM 的*直接内存*限制过小,或者存在*直接内存泄漏(Direct Memory Leak)*。 +请确认用户代码及外部依赖中是否使用了 JVM *直接内存*,以及如果使用了直接内存,是否配置了足够的内存空间。 +可以通过调整[堆外内存](mem_detail.html)来增大直接内存限制。 +请同时参考[如何配置堆外内存](mem_setup.html#配置堆外内存直接内存或本地内存))以及 Flink 设置的 [JVM 参数](mem_detail.html#jvm-参数)。 ## OutOfMemoryError: Metaspace -The exception usually indicates that [JVM metaspace limit](mem_detail.html#jvm-参数) is configured too small. -You can try to increase the [JVM metaspace option](../config.html#taskmanager-memory-jvm-metaspace-size). +该异常说明 [JVM Metaspace 限制](mem_detail.html#jvm-参数)过小。 +可以尝试调整 [JVM Metaspace 参数](../config.html#taskmanager-memory-jvm-metaspace-size)。 ## IOException: Insufficient number of network buffers -The exception usually indicates that the size of the configured [network memory](mem_detail.html) -is not big enough. You can try to increase the *network memory* by adjusting the following options: +该异常通常说明[网络内存](mem_detail.html)过小。 +可以通过调整以下配置参数增大*网络内存*: * [`taskmanager.memory.network.min`](../config.html#taskmanager-memory-network-min) * [`taskmanager.memory.network.max`](../config.html#taskmanager-memory-network-max) * [`taskmanager.memory.network.fraction`](../config.html#taskmanager-memory-network-fraction) -## Container Memory Exceeded +## 容器(Container)内存超用 -If a task executor container tries to allocate memory beyond its requested size (Yarn, Mesos or Kubernetes), -this usually indicates that Flink has not reserved enough native memory. You can observe this either by using an external -monitoring system or from the error messages when a container gets killed by the deployment environment. +如果 TaskExecutor 容器尝试分配超过其申请大小的内存(Yarn、Mesos 或 Kubernetes),这通常说明 Flink 没有预留出足够的本地内存。 +可以通过外部监控系统或者容器被部署环境杀掉时的错误信息判断是否存在容器内存超用。 -If [RocksDBStateBackend](../state/state_backends.html#the-rocksdbstatebackend) is used and the memory controlling is disabled, -you can try to increase the [managed memory](mem_setup.html#托管内存). +如果使用了 [RocksDBStateBackend](../state/state_backends.html#rocksdbstatebackend) 且没有开启内存控制,也可以尝试增大[托管内存](mem_setup.html#托管内存)。 -Alternatively, you can increase the [JVM overhead](mem_detail.html). -See also [how to configure memory for containers](mem_tuning.html#容器container的内存配置). +此外,还可以尝试增大 [JVM 开销](mem_detail.html)。 + +请参考[如何配置容器内存](mem_tuning.html#容器container的内存配置)。 diff --git a/docs/ops/memory/mem_tuning.zh.md b/docs/ops/memory/mem_tuning.zh.md index 90926621de..f1e3a8c23a 100644 --- a/docs/ops/memory/mem_tuning.zh.md +++ b/docs/ops/memory/mem_tuning.zh.md @@ -46,7 +46,7 @@ under the License.
    注意: 如果 Flink 或者用户代码分配超过容器大小的非托管的堆外(本地)内存,部署环境可能会杀掉超用内存的容器,造成作业执行失败。
    -请参考[容器内存超用](mem_trouble.html#container-memory-exceeded)中的相关描述。 +请参考[容器内存超用](mem_trouble.html#容器container内存超用)中的相关描述。 ## State Backend 的内存配置 -- Gitee From 6c1f94cd46e99fec774b8676912a28b4fded0ce6 Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Fri, 13 Mar 2020 17:18:12 +0800 Subject: [PATCH 200/885] [FLINK-15991][docs-zh] Translate the "Migration Guide" page into Chinese. This closes #11401 --- docs/ops/memory/mem_migration.zh.md | 207 +++++++++++++--------------- 1 file changed, 96 insertions(+), 111 deletions(-) diff --git a/docs/ops/memory/mem_migration.zh.md b/docs/ops/memory/mem_migration.zh.md index d42bb7c787..d51683a4d9 100644 --- a/docs/ops/memory/mem_migration.zh.md +++ b/docs/ops/memory/mem_migration.zh.md @@ -1,5 +1,5 @@ --- -title: "Migration Guide" +title: "升级指南" nav-parent_id: ops_mem nav-pos: 5 --- @@ -22,67 +22,63 @@ specific language governing permissions and limitations under the License. --> -The [memory setup of task managers](mem_setup.html) has changed a lot with the 1.10 release. Many configuration options -were removed or their semantics changed. This guide will help you to migrate the memory configuration from Flink -[<= *1.9*](https://ci.apache.org/projects/flink/flink-docs-release-1.9/ops/mem_setup.html) to >= *1.10*. +在 1.10 版本中,Flink 的 [TaskExecutor 内存配置方法](mem_setup.html)发生了较大的变化。 +部分配置参数被移除了,或是语义上发生了变化。 +本篇升级指南将介绍如何将 [*Flink 1.9 及以前版本*](https://ci.apache.org/projects/flink/flink-docs-release-1.9/ops/mem_setup.html)的内存配置升级到 *Flink 1.10 及以后版本*。 * toc {:toc}
    - Warning: It is important to review this guide because the legacy and new memory configuration can - result in different sizes of memory components. If you try to reuse your Flink configuration from older versions - before 1.10, it can result in changes to the behavior, performance or even configuration failures of your application. + 注意: 请仔细阅读本篇升级指南。 + 使用原本的和新的内存配制方法可能会使内存组成部分具有截然不同的大小。 + 未经调整直接沿用 Flink 1.10 以前版本的配置文件,可能导致应用的行为、性能发生变化,甚至造成应用执行失败。
    -Note Before version *1.10*, Flink did not require that memory related options are set at all -as they all had default values. The [new memory configuration](mem_setup.html#配置总内存) requires -that at least one subset of the following options is configured explicitly, otherwise the configuration will fail: +提示 在 *1.10* 版本之前,Flink 不要求用户一定要配置内存相关的参数,因为这些参数都具有默认值。 +[新的内存配置](mem_setup.html#配置总内存)要求用户至少指定下列配置参数(或参数组合)的其中之一,否则 Flink 将无法启动。 * [`taskmanager.memory.flink.size`](../config.html#taskmanager-memory-flink-size) * [`taskmanager.memory.process.size`](../config.html#taskmanager-memory-process-size) -* [`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size) and [`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size) +* [`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size) 和 [`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size) -The [default `flink-conf.yaml`](#default-configuration-in-flink-confyaml) shipped with Flink sets [`taskmanager.memory.process.size`](../config.html#taskmanager-memory-process-size) -to make the default memory configuration consistent. +Flink 自带的[默认 flink-conf.yaml](#flink-confyaml-中的默认配置) 文件指定了 [`taskmanager.memory.process.size`](../config.html#taskmanager-memory-process-size),以便与此前的行为保持一致。 -This [spreadsheet](https://docs.google.com/spreadsheets/d/1mJaMkMPfDJJ-w6nMXALYmTc4XxiV30P5U7DzgwLkSoE) can also help -to evaluate and compare the results of the legacy and new memory computations. +可以使用这张[电子表格](https://docs.google.com/spreadsheets/d/1mJaMkMPfDJJ-w6nMXALYmTc4XxiV30P5U7DzgwLkSoE)来估算和比较原本的和新的内存配置下的计算结果。 -## Changes in Configuration Options +## 配置参数变化 -This chapter shortly lists all changes to Flink's memory configuration options introduced with the *1.10* release. -It also references other chapters for more details about migrating to the new configuration options. +本节简要列出了 *Flink 1.10* 引入的配置参数变化,并援引其他章节中关于如何升级到新配置参数的相关描述。 -The following options are completely removed. If they are still used, they will be ignored. +下列配置参数已被彻底移除,配置它们将不会产生任何效果。 - - + + - + - +
    Removed optionNote移除的配置参数备注
    taskmanager.memory.fraction
    - Check the description of the new option taskmanager.memory.managed.fraction. - The new option has different semantics and the value of the deprecated option usually has to be adjusted. - See also how to migrate managed memory. + 请参考新配置参数 taskmanager.memory.managed.fraction 的相关描述。 + 新的配置参数与被移除的配置参数在语义上有所差别,因此其配置值通常也需要做出适当调整。 + 请参考如何升级托管内存
    taskmanager.memory.off-heap
    On-heap managed memory is no longer supported. See also how to migrate managed memory.Flink 不再支持堆上的(On-Heap)托管内存。请参考如何升级托管内存
    taskmanager.memory.preallocate
    Pre-allocation is no longer supported and managed memory is always allocated lazily. See also how to migrate managed memory.Flink 不再支持内存预分配,今后托管内存将都是惰性分配的。请参考如何升级托管内存
    -The following options are deprecated but if they are still used they will be interpreted as new options for backwards compatibility: +下列配置参数将被弃用,出于向后兼容性考虑,配置它们将被解读成对应的新配置参数。 @@ -96,15 +92,15 @@ The following options are deprecated but if they are still used they will be int - + @@ -121,116 +117,105 @@ The following options are deprecated but if they are still used they will be int
    taskmanager.heap.size
    - See also how to migrate total memory. + 请参考如何升级总内存
    taskmanager.memory.size
    taskmanager.memory.managed.size, see also how to migrate managed memory.taskmanager.memory.managed.size。请参考如何升级托管内存
    taskmanager.network.memory.min
    -Although, the network memory configuration has not changed too much it is recommended to verify its configuration. -It can change if other memory components have new sizes, e.g. the total memory which the network can be a fraction of. -See also [new detailed memory model](mem_detail.html). +尽管网络内存的配置参数没有发生太多变化,我们仍建议您检查其配置结果。 +网络内存的大小可能会受到其他内存部分大小变化的影响,例如总内存变化时,根据占比计算出的网络内存也可能发生变化。 +请参考[内存模型详解](mem_detail.html)。 -The container cut-off configuration options, [`containerized.heap-cutoff-ratio`](config.html#containerized-heap-cutoff-ratio) -and [`containerized.heap-cutoff-min`](config.html#containerized-heap-cutoff-min), have no effect for task manager processes anymore -but they still have the same semantics for the job manager process. See also [how to migrate container cut-off](#container-cut-off-memory). +容器切除(Cut-Off)内存相关的配置参数([`containerized.heap-cutoff-ratio`](config.html#containerized-heap-cutoff-ratio) +和 [`containerized.heap-cutoff-min`](config.html#containerized-heap-cutoff-min))将不再对 TaskExecutor 进程生效。 +对于 JobManager 进程,它们仍具有与此前相同的语义。 +请参考[如何升级容器切除内存](#容器切除cut-off内存)。 -## Total Memory (Previously Heap Memory) +## 总内存(原堆内存) -The previous options which were responsible for the total memory used by Flink are `taskmanager.heap.size` or `taskmanager.heap.mb`. -Despite their naming, they included not only JVM heap but also other off-heap memory components. The options have been deprecated. +在原本的内存配置方法中,用于指定用于 Flink 的总内存的配置参数是 `taskmanager.heap.size` 或 `taskmanager.heap.mb`。 +尽管这两个参数以“堆(Heap)”命名,实际上它们指定的内存既包含了 JVM 堆内存,也包含了其他堆外内存部分。 +这两个配置参数目前已被弃用。 -The Mesos integration also had a separate option with the same semantics: `mesos.resourcemanager.tasks.mem` which has also been removed. +Flink 在 Mesos 上还有另一个具有同样语义的配置参数 `mesos.resourcemanager.tasks.mem`,目前也已经被弃用。 -If the mentioned legacy options are used without specifying the corresponding new options, -they will be directly translated into the following new options: -* Total Flink memory ([`taskmanager.memory.flink.size`](../config.html#taskmanager-memory-flink-size)) for standalone deployments -* Total process memory ([`taskmanager.memory.process.size`](../config.html#taskmanager-memory-process-size)) for containerized deployments (Yarn or Mesos) +如果配置了上述弃用的参数,同时又没有配置与之对应的新配置参数,那它们将按如下规则对应到新的配置参数。 +* 独立部署模式(Standalone Deployment)下:Flink 总内存([`taskmanager.memory.flink.size`](../config.html#taskmanager-memory-flink-size)) +* 容器化部署模式(Containerized Deployement)下(Yarn、Mesos):进程总内存([`taskmanager.memory.process.size`](../config.html#taskmanager-memory-process-size)) -It is also recommended to use these new options instead of the legacy ones as they might be completely removed in the following releases. +建议您尽早使用新的配置参数取代启用的配置参数,它们在今后的版本中可能会被彻底移除。 -See also [how to configure total memory now](mem_setup.html#配置总内存). +请参考[如何配置总内存](mem_setup.html#配置总内存). -## JVM Heap Memory +## JVM 堆内存 -JVM heap memory previously consisted of the managed memory (if configured to be on-heap) and the rest -which included any other usages of heap memory. This rest was always implicitly derived as the remaining part of the total memory, -see also [how to migrate managed memory](#managed-memory). +此前,JVM 堆空间由托管内存(仅在配置为堆上时)及 Flink 用到的所有其他堆内存组成。 +这里的其他堆内存是由总内存减去所有其他非堆内存得到的。 +请参考[如何升级托管内存](#托管内存)。 -Now, if only *total Flink memory* or *total process memory* is configured, then the JVM heap is also derived as the rest of -what is left after subtracting all other components from the total memory, see also [how to configure total memory](mem_setup.html#配置总内存). +现在,如果仅配置了*Flink总内存*或*进程总内存*,JVM 的堆空间依然是根据总内存减去所有其他非堆内存得到的。 +请参考[如何配置总内存](mem_setup.html#配置总内存)。 -Additionally, you can now have more direct control over the JVM heap assigned to the operator tasks -([`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size)), -see also [Task (Operator) Heap Memory](mem_setup.html#任务算子堆内存). -The JVM heap memory is also used by the heap state backends ([MemoryStateBackend](../state/state_backends.html#the-memorystatebackend) -or [FsStateBackend](../state/state_backends.html#the-fsstatebackend)) if it is chosen for streaming jobs. +此外,你现在可以更直接地控制用于任务和算子的 JVM 的堆内存([`taskmanager.memory.task.heap.size`](../config.html#taskmanager-memory-task-heap-size)),详见[任务堆内存](mem_setup.html#任务算子堆内存)。 +如果流处理作业选择使用 Heap State Backend([MemoryStateBackend](../state/state_backends.html#memorystatebackend) +或 [FsStateBackend](../state/state_backends.html#fsstatebackend)),那么它同样需要使用 JVM 堆内存。 -A part of the JVM heap is now always reserved for Flink framework -([`taskmanager.memory.framework.heap.size`](../config.html#taskmanager-memory-framework-heap-size)). -See also [Framework memory](mem_detail.html#框架内存). +Flink 现在总是会预留一部分 JVM 堆内存供框架使用([`taskmanager.memory.framework.heap.size`](../config.html#taskmanager-memory-framework-heap-size))。 +请参考[框架内存](mem_detail.html#框架内存)。 -## Managed Memory +## 托管内存 -See also [how to configure managed memory now](mem_setup.html#托管内存). +请参考[如何配置托管内存](mem_setup.html#托管内存)。 -### Explicit Size +### 明确的大小 -The previous option to configure managed memory size (`taskmanager.memory.size`) was renamed to -[`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size) and deprecated. -It is recommended to use the new option because the legacy one can be removed in future releases. +原本用于指定明确的托管内存大小的配置参数(`taskmanager.memory.size`)已被弃用,与它具有相同语义的新配置参数为 [`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size)。 +建议使用新的配置参数,原本的配置参数在今后的版本中可能会被彻底移除。 -### Fraction +### 占比 -If not set explicitly, the managed memory could be previously specified as a fraction (`taskmanager.memory.fraction`) -of the total memory minus network memory and container cut-off (only for [Yarn](../deployment/yarn_setup.html) and -[Mesos](../deployment/mesos.html) deployments). This option has been completely removed and will have no effect if still used. -Please, use the new option [`taskmanager.memory.managed.fraction`](../config.html#taskmanager-memory-managed-fraction) instead. -This new option will set the [managed memory](mem_setup.html#托管内存) to the specified fraction of the -[total Flink memory](mem_setup.html#配置总内存) if its size is not set explicitly by -[`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size). +此前,如果不指定明确的大小,也可以将托管内存配置为占用总内存减去网络内存和容器切除内存(仅在 [Yarn](../deployment/yarn_setup.html) 和 +[Mesos](../deployment/mesos.html) 上)之后剩余部分的固定比例(`taskmanager.memory.fraction`)。 +该配置参数已经被彻底移除,配置它不会产生任何效果。 +请使用新的配置参数 [`taskmanager.memory.managed.fraction`](../config.html#taskmanager-memory-managed-fraction)。 +在未通过 [`taskmanager.memory.managed.size`](../config.html#taskmanager-memory-managed-size) 指定明确大小的情况下,新的配置参数将指定[托管内存](mem_setup.html#托管内存)在 [Flink 总内存](mem_setup.html#配置总内存)中的所占比例。 -### RocksDB state +### RocksDB State Backend -If the [RocksDBStateBackend](../state/state_backends.html#the-rocksdbstatebackend) is chosen for a streaming job, -its native memory consumption should now be accounted for in [managed memory](mem_setup.html#托管内存). -The RocksDB memory allocation is limited by the [managed memory](mem_setup.html#托管内存) size. -This should prevent the killing of containers on [Yarn](../deployment/yarn_setup.html) or [Mesos](../deployment/mesos.html). -You can disable the RocksDB memory control by setting [state.backend.rocksdb.memory.managed](../config.html#state-backend-rocksdb-memory-managed) -to `false`. See also [how to migrate container cut-off](#container-cut-off-memory). +流处理作业如果选择使用 [RocksDBStateBackend](../state/state_backends.html#rocksdbstatebackend),它使用的本地内存现在也被归为[托管内存](mem_setup.html#托管内存)。 +默认情况下,RocksDB 将限制其内存用量不超过[托管内存](mem_setup.html#托管内存)大小,以避免在 [Yarn](../deployment/yarn_setup.html) 或 [Mesos](../deployment/mesos.html) 上容器被杀。你也可以通过设置 [state.backend.rocksdb.memory.managed](../config.html#state-backend-rocksdb-memory-managed) 来关闭 RocksDB 的内存控制。 +请参考[如何升级容器切除内存](#容器切除cut-off内存)。 -### Other changes +### 其他变化 -Additionally, the following changes have been made: -* The [managed memory](mem_setup.html#托管内存) is always off-heap now. The configuration option `taskmanager.memory.off-heap` is removed and will have no effect anymore. -* The [managed memory](mem_setup.html#托管内存) now uses native memory which is not direct memory. It means that the managed memory is no longer accounted for in the JVM direct memory limit. -* The [managed memory](mem_setup.html#托管内存) is always lazily allocated now. The configuration option `taskmanager.memory.preallocate` is removed and will have no effect anymore. +此外,Flink 1.10 对托管内存还引入了下列变化: +* [托管内存](mem_setup.html#托管内存)现在总是在堆外。配置参数 `taskmanager.memory.off-heap` 已被彻底移除,配置它不会产生任何效果。 +* [托管内存](mem_setup.html#托管内存)现在使用本地内存而非直接内存。这意味着托管内存将不在 JVM 直接内存限制的范围内。 +* [托管内存](mem_setup.html#托管内存)现在总是惰性分配的。配置参数 `taskmanager.memory.preallocate` 已被彻底移除,配置它不会产生任何效果。 -## Container Cut-Off Memory +## 容器切除(Cut-Off)内存 -For containerized deployments, you could previously specify a cut-off memory. This memory could accommodate for unaccounted memory allocations. -Dependencies which were not directly controlled by Flink were the main source of those allocations, e.g. RocksDB, internals of JVM, etc. -This is no longer available and the related configuration options (`containerized.heap-cutoff-ratio` and `containerized.heap-cutoff-min`) -will have no effect on the task manager process anymore. The new memory model introduced more specific memory components, -described further, to address these concerns. +在容器化部署模式(Containerized Deployment)下,此前你可以指定切除内存。 +这部分内存将预留给所有未被 Flink 计算在内的内存开销。 +其主要来源是不受 Flink 直接管理的依赖使用的内存,例如 RocksDB、JVM 内部开销等。 +相应的配置参数(`containerized.heap-cutoff-ratio` 和 `containerized.heap-cutoff-min`)现在不再对 TaskExecutor 生效。 +新的内存配置方法引入了新的内存组成部分来具体描述这些内存用量。 -In streaming jobs which use [RocksDBStateBackend](../state/state_backends.html#the-rocksdbstatebackend), the RocksDB -native memory consumption should be accounted for as a part of the [managed memory](mem_setup.html#托管内存) now. -The RocksDB memory allocation is also limited by the configured size of the [managed memory](mem_setup.html#托管内存). -See also [migrating managed memory](#managed-memory) and [how to configure managed memory now](mem_setup.html#托管内存). +流处理作业如果使用了 [RocksDBStateBackend](../state/state_backends.html#the-rocksdbstatebackend),RocksDB 使用的本地内存现在将被归为[托管内存](mem_setup.html#托管内存)。 +默认情况下,RocksDB 将限制其内存用量不超过[托管内存](mem_setup.html#托管内存)大小。 +请同时参考[如何升级托管内存](#托管内存)以及[如何配置托管内存](mem_setup.html#托管内存)。 -The other direct or native off-heap memory consumers can now be addressed by the following new configuration options: -* Task off-heap memory ([`taskmanager.memory.task.off-heap.size`](../config.html#taskmanager-memory-task-off-heap-size)) -* Framework off-heap memory ([`taskmanager.memory.framework.off-heap.size`](../config.html#taskmanager-memory-framework-off-heap-size)) -* JVM metaspace ([`taskmanager.memory.jvm-metaspace.size`](../config.html#taskmanager-memory-jvm-metaspace-size)) -* JVM overhead (see also [detailed new memory model](mem_detail.html)) +其他直接内存或堆外内存开销,现在可以通过下列配置参数进行设置: +* 任务堆外内存([`taskmanager.memory.task.off-heap.size`](../config.html#taskmanager-memory-task-off-heap-size)) +* 框架堆外内存([`taskmanager.memory.framework.off-heap.size`](../config.html#taskmanager-memory-framework-off-heap-size)) +* JVM Metaspace([`taskmanager.memory.jvm-metaspace.size`](../config.html#taskmanager-memory-jvm-metaspace-size)) +* JVM 开销(请参考[内存模型详解](mem_detail.html)) -Note The job manager still has container cut-off memory configuration options. -The mentioned configuration options remain valid for the job manager in the same way as before. +提示 JobManager 进程仍保留了容器切除内存,相关配置项和此前一样仍对 JobManager 生效。 -## Default Configuration in flink-conf.yaml +## flink-conf.yaml 中的默认配置 -This section describes the changes of the default `flink-conf.yaml` shipped with Flink. +本节描述 Flink 自带的默认 `flink-conf.yaml` 文件中的变化。 -The total memory (`taskmanager.heap.size`) is replaced by [`taskmanager.memory.process.size`](../config.html#taskmanager-memory-process-size) -in the default `flink-conf.yaml`. The value is also increased from 1024Mb to 1568Mb. -See also [how to configure total memory now](mem_setup.html#配置总内存). +在默认 `flink-conf.yaml` 文件中,原本的总内存(`taskmanager.heap.size`)被新的配置项 [`taskmanager.memory.process.size`](../config.html#taskmanager-memory-process-size) 所取代。 +默认值从 1024Mb 增加到了 1568Mb。 +请参考[如何配置总内存](mem_setup.html#配置总内存)。
    - Warning: If you use the new default `flink-conf.yaml` it can result in different sizes of - the memory components and can lead to performance changes. + 注意: 使用新的默认 `flink-conf.yaml` 可能会造成各内存部分的大小发生变化,从而产生性能变化。
    -- Gitee From 64abbdd6f274333fbdc3d6a51b2876c3110986d6 Mon Sep 17 00:00:00 2001 From: vthinkxie Date: Wed, 18 Mar 2020 15:18:49 +0800 Subject: [PATCH 201/885] [FLINK-15953][web] Fix color of RESTARTING status in Web UI This closes #11433. --- flink-runtime-web/web-dashboard/src/app/app.config.ts | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-runtime-web/web-dashboard/src/app/app.config.ts b/flink-runtime-web/web-dashboard/src/app/app.config.ts index 66224e329e..6d41b675f6 100644 --- a/flink-runtime-web/web-dashboard/src/app/app.config.ts +++ b/flink-runtime-web/web-dashboard/src/app/app.config.ts @@ -29,6 +29,7 @@ export const COLOR_MAP = { RECONCILING: '#eb2f96', IN_PROGRESS: '#faad14', SCHEDULED: '#722ed1', - COMPLETED : '#1890ff' + COMPLETED: '#1890ff', + RESTARTING: '#13c2c2' }; export const LONG_MIN_VALUE = -9223372036854776000; -- Gitee From e8d98f45e2874b7252d757f43ca3b91325cb2c5c Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Mon, 17 Feb 2020 09:51:01 +0100 Subject: [PATCH 202/885] [FLINK-15958][table-common] Introduce unresolved data types in API Introduces unresolved data types for class-based extraction, name-based resolution, and configuration-based RAW types. Unresolved types behave like regular data types but only after they have been resolved. Using unresolved data types in nested structures leads to further unresolved types. Thus, the usage of unresolved types is type-safe in API. This closes #11153. --- .../table/catalog/DataTypeFactoryImpl.java | 22 +- .../resolver/ExpressionResolverTest.java | 2 +- .../org/apache/flink/table/api/DataTypes.java | 302 ++++++++++++- .../flink/table/catalog/DataTypeFactory.java | 47 +- .../flink/table/types/AbstractDataType.java | 69 +++ .../apache/flink/table/types/DataType.java | 33 +- .../flink/table/types/UnresolvedDataType.java | 104 +++++ .../extraction/utils/DataTypeTemplate.java | 14 +- .../flink/table/types/logical/ArrayType.java | 2 +- .../flink/table/types/logical/MapType.java | 2 +- .../table/types/logical/MultisetType.java | 2 +- .../flink/table/types/logical/RawType.java | 2 +- .../flink/table/types/logical/RowType.java | 6 +- .../flink/table/types/DataTypesTest.java | 416 +++++++++++++----- .../extraction/DataTypeExtractorTest.java | 7 +- .../TypeInferenceExtractorTest.java | 2 +- .../inference/InputTypeStrategiesTest.java | 2 +- .../utils/DataTypeFactoryMock.java | 25 +- 18 files changed, 849 insertions(+), 210 deletions(-) create mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/AbstractDataType.java create mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/UnresolvedDataType.java rename flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/{inference => }/utils/DataTypeFactoryMock.java (66%) diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/DataTypeFactoryImpl.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/DataTypeFactoryImpl.java index 8a60ece0f9..9b0e76e834 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/DataTypeFactoryImpl.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/DataTypeFactoryImpl.java @@ -24,7 +24,10 @@ import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.types.AbstractDataType; import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.UnresolvedDataType; import org.apache.flink.table.types.extraction.DataTypeExtractor; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.LogicalTypeRoot; @@ -34,7 +37,6 @@ import org.apache.flink.table.types.logical.utils.LogicalTypeParser; import javax.annotation.Nullable; -import java.util.Optional; import java.util.function.Supplier; import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.hasRoot; @@ -61,19 +63,29 @@ final class DataTypeFactoryImpl implements DataTypeFactory { } @Override - public Optional createDataType(String name) { + public DataType createDataType(AbstractDataType abstractDataType) { + if (abstractDataType instanceof DataType) { + return (DataType) abstractDataType; + } else if (abstractDataType instanceof UnresolvedDataType) { + return ((UnresolvedDataType) abstractDataType).toDataType(this); + } + throw new ValidationException("Unsupported abstract data type."); + } + + @Override + public DataType createDataType(String name) { final LogicalType parsedType = LogicalTypeParser.parse(name, classLoader); final LogicalType resolvedType = parsedType.accept(resolver); - return Optional.of(fromLogicalToDataType(resolvedType)); + return fromLogicalToDataType(resolvedType); } @Override - public Optional createDataType(UnresolvedIdentifier identifier) { + public DataType createDataType(UnresolvedIdentifier identifier) { if (!identifier.getDatabaseName().isPresent()) { return createDataType(identifier.getObjectName()); } final LogicalType resolvedType = resolveType(identifier); - return Optional.of(fromLogicalToDataType(resolvedType)); + return fromLogicalToDataType(resolvedType); } @Override diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/expressions/resolver/ExpressionResolverTest.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/expressions/resolver/ExpressionResolverTest.java index 02cb57f3ad..801ed185b4 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/expressions/resolver/ExpressionResolverTest.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/expressions/resolver/ExpressionResolverTest.java @@ -41,7 +41,7 @@ import org.apache.flink.table.functions.ScalarFunctionDefinition; import org.apache.flink.table.operations.CatalogQueryOperation; import org.apache.flink.table.operations.QueryOperation; import org.apache.flink.table.types.inference.TypeInferenceUtil; -import org.apache.flink.table.types.inference.utils.DataTypeFactoryMock; +import org.apache.flink.table.types.utils.DataTypeFactoryMock; import org.apache.flink.table.types.utils.TypeConversions; import org.junit.Test; diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/DataTypes.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/DataTypes.java index 52f86dd3a9..61606e3790 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/DataTypes.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/DataTypes.java @@ -22,11 +22,15 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.table.annotation.DataTypeHint; +import org.apache.flink.table.catalog.DataTypeFactory; +import org.apache.flink.table.types.AbstractDataType; import org.apache.flink.table.types.AtomicDataType; import org.apache.flink.table.types.CollectionDataType; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.FieldsDataType; import org.apache.flink.table.types.KeyValueDataType; +import org.apache.flink.table.types.UnresolvedDataType; import org.apache.flink.table.types.logical.ArrayType; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BinaryType; @@ -46,6 +50,7 @@ import org.apache.flink.table.types.logical.MultisetType; import org.apache.flink.table.types.logical.NullType; import org.apache.flink.table.types.logical.RawType; import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.RowType.RowField; import org.apache.flink.table.types.logical.SmallIntType; import org.apache.flink.table.types.logical.TimeType; import org.apache.flink.table.types.logical.TimestampType; @@ -73,6 +78,11 @@ import java.util.stream.Stream; * A {@link DataType} can be used to declare input and/or output types of operations. This class * enumerates all pre-defined data types of the Table & SQL API. * + *

    For convenience, this class also contains methods for creating {@link UnresolvedDataType}s that + * need to be resolved at later stages. This is in particular useful for more complex types that are + * expressed as {@link Class} (see {@link #of(Class)}) or types that need to be looked up in a catalog + * (see {@link #of(String)}). + * *

    NOTE: Planners might not support every data type with the desired precision or parameter. Please * see the planner compatibility and limitations section in the website documentation before using a * data type. @@ -80,6 +90,71 @@ import java.util.stream.Stream; @PublicEvolving public final class DataTypes { + /** + * Creates an unresolved type that will be resolved to a {@link DataType} by analyzing the given + * class later. + * + *

    During the resolution, Java reflection is used which can be supported by {@link DataTypeHint} + * annotations for nested, structured types. + * + *

    It will throw an {@link ValidationException} in cases where the reflective extraction needs + * more information or simply fails. + * + *

    The following examples show how to use and enrich the extraction process: + * + *

    +	 * {@code
    +	 *   // returns INT
    +	 *   of(Integer.class)
    +	 *
    +	 *   // returns TIMESTAMP(9)
    +	 *   of(java.time.LocalDateTime.class)
    +	 *
    +	 *   // returns an anonymous, unregistered structured type
    +	 *   // that is deeply integrated into the API compared to opaque RAW types
    +	 *   class User {
    +	 *
    +	 *     // extract fields automatically
    +	 *     public String name;
    +	 *     public int age;
    +	 *
    +	 *     // enrich the extraction with precision information
    +	 *     public @DataTypeHint("DECIMAL(10,2)") BigDecimal accountBalance;
    +	 *
    +	 *     // enrich the extraction with forcing using RAW types
    +	 *     public @DataTypeHint(forceRawPattern = "scala.") Address address;
    +	 *
    +	 *     // enrich the extraction by specifying defaults
    +	 *     public @DataTypeHint(defaultSecondPrecision = 3) Log log;
    +	 *   }
    +	 *   of(User.class)
    +	 * }
    +	 * 
    + * + *

    Note: In most of the cases, the {@link UnresolvedDataType} will be automatically resolved by + * the API. At other locations, a {@link DataTypeFactory} is provided. + */ + public static UnresolvedDataType of(Class unresolvedClass) { + return new UnresolvedDataType( + () -> String.format("'%s'", unresolvedClass.getName()), + (factory) -> factory.createDataType(unresolvedClass)); + } + + /** + * Creates an unresolved type that will be resolved to a {@link DataType} by using a fully or partially + * defined name. + * + *

    It includes both built-in types (e.g. "INT") as well as user-defined types (e.g. "mycat.mydb.Money"). + * + *

    Note: In most of the cases, the {@link UnresolvedDataType} will be automatically resolved by + * the API. At other locations, a {@link DataTypeFactory} is provided. + */ + public static UnresolvedDataType of(String unresolvedName) { + return new UnresolvedDataType( + () -> unresolvedName, + (factory) -> factory.createDataType(unresolvedName)); + } + // we use SQL-like naming for data types and avoid Java keyword clashes // CHECKSTYLE.OFF: MethodName @@ -473,6 +548,25 @@ public final class DataTypes { return new CollectionDataType(new ArrayType(elementDataType.getLogicalType()), elementDataType); } + /** + * Unresolved data type of an array of elements with same subtype. + * + *

    Compared to the SQL standard, the maximum cardinality of an array cannot be specified but + * is fixed at {@link Integer#MAX_VALUE}. Also, any valid type is supported as a subtype. + * + *

    Note: Compared to {@link #ARRAY(DataType)}, this method produces an {@link UnresolvedDataType}. + * In most of the cases, the {@link UnresolvedDataType} will be automatically resolved by the API. At + * other locations, a {@link DataTypeFactory} is provided. + * + * @see ArrayType + */ + public static UnresolvedDataType ARRAY(UnresolvedDataType unresolvedElementDataType) { + Preconditions.checkNotNull(unresolvedElementDataType, "Element data type must not be null."); + return new UnresolvedDataType( + () -> String.format(ArrayType.FORMAT, unresolvedElementDataType), + factory -> ARRAY(factory.createDataType(unresolvedElementDataType))); + } + /** * Data type of a multiset (=bag). Unlike a set, it allows for multiple instances for each of its * elements with a common subtype. Each unique value (including {@code NULL}) is mapped to some @@ -488,6 +582,27 @@ public final class DataTypes { return new CollectionDataType(new MultisetType(elementDataType.getLogicalType()), elementDataType); } + /** + * Unresolved data type of a multiset (=bag). Unlike a set, it allows for multiple instances for each of its + * elements with a common subtype. Each unique value (including {@code NULL}) is mapped to some + * multiplicity. + * + *

    There is no restriction of element types; it is the responsibility of the user to ensure + * uniqueness. + * + *

    Note: Compared to {@link #MULTISET(DataType)}, this method produces an {@link UnresolvedDataType}. In + * most of the cases, the {@link UnresolvedDataType} will be automatically resolved by the API. At other + * locations, a {@link DataTypeFactory} is provided. + * + * @see MultisetType + */ + public static UnresolvedDataType MULTISET(UnresolvedDataType unresolvedElementDataType) { + Preconditions.checkNotNull(unresolvedElementDataType, "Element data type must not be null."); + return new UnresolvedDataType( + () -> String.format(MultisetType.FORMAT, unresolvedElementDataType), + factory -> MULTISET(factory.createDataType(unresolvedElementDataType))); + } + /** * Data type of an associative array that maps keys (including {@code NULL}) to values (including * {@code NULL}). A map cannot contain duplicate keys; each key can map to at most one value. @@ -506,6 +621,27 @@ public final class DataTypes { valueDataType); } + /** + * Unresolved data type of an associative array that maps keys (including {@code NULL}) to values (including + * {@code NULL}). A map cannot contain duplicate keys; each key can map to at most one value. + * + *

    There is no restriction of key types; it is the responsibility of the user to ensure uniqueness. + * The map type is an extension to the SQL standard. + * + *

    Note: Compared to {@link #MAP(DataType, DataType)}, this method produces an {@link UnresolvedDataType}. + * In most of the cases, the {@link UnresolvedDataType} will be automatically resolved by the API. At + * other locations, a {@link DataTypeFactory} is provided. + * + * @see MapType + */ + public static UnresolvedDataType MAP(AbstractDataType keyDataType, AbstractDataType valueDataType) { + Preconditions.checkNotNull(keyDataType, "Key data type must not be null."); + Preconditions.checkNotNull(valueDataType, "Value data type must not be null."); + return new UnresolvedDataType( + () -> String.format(MapType.FORMAT, keyDataType, valueDataType), + factory -> MAP(factory.createDataType(keyDataType), factory.createDataType(valueDataType))); + } + /** * Data type of a sequence of fields. A field consists of a field name, field type, and an optional * description. The most specific type of a row of a table is a row type. In this case, each column @@ -515,18 +651,64 @@ public final class DataTypes { *

    Compared to the SQL standard, an optional field description simplifies the handling with * complex structures. * + *

    Use {@link #FIELD(String, DataType)} or {@link #FIELD(String, DataType, String)} to construct + * fields. + * * @see RowType */ public static DataType ROW(Field... fields) { - final List logicalFields = Stream.of(fields) + final List logicalFields = Stream.of(fields) .map(f -> Preconditions.checkNotNull(f, "Field definition must not be null.")) - .map(f -> new RowType.RowField(f.name, f.dataType.getLogicalType(), f.description)) + .map(f -> new RowField(f.name, f.dataType.getLogicalType(), f.description)) .collect(Collectors.toList()); final Map fieldDataTypes = Stream.of(fields) .collect(Collectors.toMap(f -> f.name, f -> f.dataType)); return new FieldsDataType(new RowType(logicalFields), fieldDataTypes); } + /** + * Data type of a row type with no fields. It only exists for completeness. + * + * @see #ROW(Field...) + */ + public static DataType ROW() { + return ROW(new Field[0]); + } + + /** + * Unresolved data type of a sequence of fields. A field consists of a field name, field type, and + * an optional description. The most specific type of a row of a table is a row type. In this case, + * each column of the row corresponds to the field of the row type that has the same ordinal position + * as the column. + * + *

    Compared to the SQL standard, an optional field description simplifies the handling with + * complex structures. + * + *

    Use {@link #FIELD(String, UnresolvedDataType)} or {@link #FIELD(String, UnresolvedDataType, String)} + * to construct fields. + * + *

    Note: Compared to {@link #ROW(Field...)} )}, this method produces an {@link UnresolvedDataType} + * with {@link UnresolvedField}s. In most of the cases, the {@link UnresolvedDataType} will be + * automatically resolved by the API. At other locations, a {@link DataTypeFactory} is provided. + * + * @see RowType + */ + public static UnresolvedDataType ROW(AbstractField... fields) { + Stream.of(fields) + .forEach(f -> Preconditions.checkNotNull(f, "Field definition must not be null.")); + return new UnresolvedDataType( + () -> String.format( + RowType.FORMAT, + Stream.of(fields).map(Object::toString).collect(Collectors.joining(", "))), + factory -> { + final Field[] fieldsArray = Stream.of(fields) + .map(f -> new Field(f.name, factory.createDataType(f.dataType), f.description)) + .toArray(Field[]::new); + return ROW(fieldsArray); + } + ); + } + /** * Data type for representing untyped {@code NULL} values. A null type has no other value except * {@code NULL}, thus, it can be cast to any nullable type similar to JVM semantics. @@ -548,8 +730,8 @@ public final class DataTypes { * *

    The raw type is an extension to the SQL standard. * - *

    This method assumes that a {@link TypeSerializer} instance is present. Use {@link #RAW(TypeInformation)} - * for generating a serializer from Flink's core type system automatically in subsequent layers. + *

    This method assumes that a {@link TypeSerializer} instance is present. Use {@link #RAW(Class)} + * for automatically generating a serializer. * * @param clazz originating value class * @param serializer type serializer @@ -560,6 +742,28 @@ public final class DataTypes { return new AtomicDataType(new RawType<>(clazz, serializer)); } + /** + * Unresolved data type of an arbitrary serialized type. This type is a black box within the table + * ecosystem and is only deserialized at the edges. + * + *

    The raw type is an extension to the SQL standard. + * + *

    Compared to {@link #RAW(Class, TypeSerializer)}, this method produces an {@link UnresolvedDataType} + * where no serializer is known and a generic serializer should be used. During the resolution, a + * {@link DataTypes#RAW(Class, TypeSerializer)} with Flink's default RAW serializer is created and + * automatically configured. + * + *

    Note: In most of the cases, the {@link UnresolvedDataType} will be automatically resolved by + * the API. At other locations, a {@link DataTypeFactory} is provided. + * + * @see RawType + */ + public static UnresolvedDataType RAW(Class clazz) { + return new UnresolvedDataType( + () -> String.format(RawType.FORMAT, clazz.getName(), "?"), + factory -> factory.createRawDataType(clazz)); + } + /** * Data type of an arbitrary serialized type backed by {@link TypeInformation}. This type is * a black box within the table ecosystem and is only deserialized at the edges. @@ -680,12 +884,40 @@ public final class DataTypes { Preconditions.checkNotNull(description, "Field description must not be null.")); } + /** + * Unresolved field definition with field name and data type. + * + *

    Note: Compared to {@link #FIELD(String, DataType)}, this method produces an {@link UnresolvedField} + * containing an {@link UnresolvedDataType}. + */ + public static UnresolvedField FIELD(String name, UnresolvedDataType unresolvedDataType) { + return new UnresolvedField( + Preconditions.checkNotNull(name, "Field name must not be null."), + Preconditions.checkNotNull(unresolvedDataType, "Field data type must not be null."), + null); + } + + /** + * Unresolved field definition with field name, unresolved data type, and a description. + * + *

    Note: Compared to {@link #FIELD(String, DataType, String)}, this method produces an {@link UnresolvedField} + * containing an {@link UnresolvedDataType}. + */ + public static UnresolvedField FIELD(String name, UnresolvedDataType unresolvedDataType, String description) { + return new UnresolvedField( + Preconditions.checkNotNull(name, "Field name must not be null."), + Preconditions.checkNotNull(unresolvedDataType, "Field data type must not be null."), + Preconditions.checkNotNull(description, "Field description must not be null.")); + } + // -------------------------------------------------------------------------------------------- // Helper classes // -------------------------------------------------------------------------------------------- /** * Helper class for defining the resolution of an interval. + * + * @see #INTERVAL(Resolution) */ public static final class Resolution { @@ -790,17 +1022,22 @@ public final class DataTypes { } /** - * Helper class for defining the field of a row or structured type. + * Common helper class for resolved and unresolved fields of a row or structured type. + * + * @see #FIELD(String, DataType) + * @see #FIELD(String, DataType, String) + * @see #FIELD(String, UnresolvedDataType) + * @see #FIELD(String, UnresolvedDataType, String) */ - public static final class Field { + public static class AbstractField> { - private final String name; + protected final String name; - private final DataType dataType; + protected final T dataType; - private final @Nullable String description; + protected final @Nullable String description; - private Field(String name, DataType dataType, String description) { + private AbstractField(String name, T dataType, @Nullable String description) { this.name = name; this.dataType = dataType; this.description = description; @@ -810,13 +1047,56 @@ public final class DataTypes { return name; } - public DataType getDataType() { + public T getDataType() { return dataType; } public Optional getDescription() { return Optional.ofNullable(description); } + + @Override + public String toString() { + if (description != null) { + return String.format( + RowField.FIELD_FORMAT_WITH_DESCRIPTION, + name, + dataType, + description); + } + return String.format( + RowField.FIELD_FORMAT_NO_DESCRIPTION, + name, + dataType); + } + } + + /** + * Helper class for defining the field of a row or structured type. + * + * @see #FIELD(String, DataType) + * @see #FIELD(String, DataType, String) + */ + public static final class Field extends AbstractField { + + private Field(String name, DataType dataType, @Nullable String description) { + super(name, dataType, description); + } + } + + /** + * Helper class for defining the unresolved field of a row or structured type. + * + *

    Compared to {@link Field}, an unresolved field contains an {@link UnresolvedDataType}. + * + * @see #FIELD(String, UnresolvedDataType) + * @see #FIELD(String, UnresolvedDataType, String) + */ + public static final class UnresolvedField extends AbstractField { + + private UnresolvedField(String name, UnresolvedDataType unresolvedDataType, @Nullable String description) { + super(name, unresolvedDataType, description); + } } private DataTypes() { diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/DataTypeFactory.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/DataTypeFactory.java index 16f7a01387..0a16c56501 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/DataTypeFactory.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/DataTypeFactory.java @@ -23,12 +23,12 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.table.annotation.DataTypeHint; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.types.AbstractDataType; import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.UnresolvedDataType; import org.apache.flink.table.types.logical.DistinctType; import org.apache.flink.table.types.logical.StructuredType; -import java.util.Optional; - /** * Factory for creating fully resolved data types that can be used for planning. * @@ -38,13 +38,21 @@ import java.util.Optional; @PublicEvolving public interface DataTypeFactory { + /** + * Creates a type out of an {@link AbstractDataType}. + * + *

    If the given type is already a {@link DataType}, the factory will return it unmodified. In + * case of {@link UnresolvedDataType}, the factory will resolve it to a {@link DataType}. + */ + DataType createDataType(AbstractDataType abstractDataType); + /** * Creates a type by a fully or partially defined name. * *

    The factory will parse and resolve the name of a type to a {@link DataType}. This includes * both built-in types as well as user-defined types (see {@link DistinctType} and {@link StructuredType}). */ - Optional createDataType(String name); + DataType createDataType(String name); /** * Creates a type by a fully or partially defined identifier. @@ -52,7 +60,7 @@ public interface DataTypeFactory { *

    The factory will parse and resolve the name of a type to a {@link DataType}. This includes * both built-in types as well as user-defined types (see {@link DistinctType} and {@link StructuredType}). */ - Optional createDataType(UnresolvedIdentifier identifier); + DataType createDataType(UnresolvedIdentifier identifier); /** * Creates a type by analyzing the given class. @@ -63,36 +71,7 @@ public interface DataTypeFactory { *

    It will throw an {@link ValidationException} in cases where the reflective extraction needs * more information or simply fails. * - *

    The following examples show how to use and enrich the extraction process: - * - *

    -	 * {@code
    -	 *   // returns INT
    -	 *   createDataType(Integer.class)
    -	 *
    -	 *   // returns TIMESTAMP(9)
    -	 *   createDataType(java.time.LocalDateTime.class)
    -	 *
    -	 *   // returns an anonymous, unregistered structured type
    -	 *   // that is deeply integrated into the API compared to opaque RAW types
    -	 *   class User {
    -	 *
    -	 *     // extract fields automatically
    -	 *     public String name;
    -	 *     public int age;
    -	 *
    -	 *     // enrich the extraction with precision information
    -	 *     public @DataTypeHint("DECIMAL(10,2)") BigDecimal accountBalance;
    -	 *
    -	 *     // enrich the extraction with forcing using RAW types
    -	 *     public @DataTypeHint(forceRawPattern = "scala.") Address address;
    -	 *
    -	 *     // enrich the extraction by specifying defaults
    -	 *     public @DataTypeHint(defaultSecondPrecision = 3) Log log;
    -	 *   }
    -	 *   createDataType(User.class)
    -	 * }
    -	 * 
    + *

    See {@link DataTypes#of(Class)} for further examples. */ DataType createDataType(Class clazz); diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/AbstractDataType.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/AbstractDataType.java new file mode 100644 index 0000000000..22b5971849 --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/AbstractDataType.java @@ -0,0 +1,69 @@ +/* + * 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.table.types; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.table.types.logical.LogicalType; + +/** + * Highest abstraction that describes the data type of a value in the table ecosystem. This class + * describes two kinds of data types: + * + *

    Fully resolved data types that can be used directly to declare input and/or output types of + * operations. This kind is represented in subclasses of {@link DataType}. + * + *

    Partially resolved data types that can be resolved to {@link DataType} but require a lookup in + * a catalog or configuration first. This kind is represented in subclasses of {@link UnresolvedDataType}. + * + * @param kind of data type returned after mutation + */ +@PublicEvolving +public interface AbstractDataType> { + + /** + * Adds a hint that null values are not expected in the data for this type. + * + * @return a new, reconfigured data type instance + */ + T notNull(); + + /** + * Adds a hint that null values are expected in the data for this type (default behavior). + * + *

    This method exists for explicit declaration of the default behavior or for invalidation of + * a previous call to {@link #notNull()}. + * + * @return a new, reconfigured data type instance + */ + T nullable(); + + /** + * Adds a hint that data should be represented using the given class when entering or leaving + * the table ecosystem. + * + *

    A supported conversion class depends on the logical type and its nullability property. + * + *

    Please see the implementation of {@link LogicalType#supportsInputConversion(Class)}, + * {@link LogicalType#supportsOutputConversion(Class)}, or the documentation for more information + * about supported conversions. + * + * @return a new, reconfigured data type instance + */ + T bridgedTo(Class newConversionClass); +} diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/DataType.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/DataType.java index 335c7c3b81..e414ef18e0 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/DataType.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/DataType.java @@ -50,7 +50,7 @@ import java.util.Objects; * @see DataTypes for a list of supported data types and instances of this class. */ @PublicEvolving -public abstract class DataType implements Serializable { +public abstract class DataType implements AbstractDataType, Serializable { protected final LogicalType logicalType; @@ -84,37 +84,6 @@ public abstract class DataType implements Serializable { return conversionClass; } - /** - * Adds a hint that null values are not expected in the data for this type. - * - * @return a new, reconfigured data type instance - */ - public abstract DataType notNull(); - - /** - * Adds a hint that null values are expected in the data for this type (default behavior). - * - *

    This method exists for explicit declaration of the default behavior or for invalidation of - * a previous call to {@link #notNull()}. - * - * @return a new, reconfigured data type instance - */ - public abstract DataType nullable(); - - /** - * Adds a hint that data should be represented using the given class when entering or leaving - * the table ecosystem. - * - *

    A supported conversion class depends on the logical type and its nullability property. - * - *

    Please see the implementation of {@link LogicalType#supportsInputConversion(Class)}, - * {@link LogicalType#supportsOutputConversion(Class)}, or the documentation for more information - * about supported conversions. - * - * @return a new, reconfigured data type instance - */ - public abstract DataType bridgedTo(Class newConversionClass); - public abstract R accept(DataTypeVisitor visitor); @Override diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/UnresolvedDataType.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/UnresolvedDataType.java new file mode 100644 index 0000000000..64ca28cf18 --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/UnresolvedDataType.java @@ -0,0 +1,104 @@ +/* + * 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.table.types; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.table.catalog.DataTypeFactory; +import org.apache.flink.table.types.logical.LogicalType; + +import javax.annotation.Nullable; + +import java.util.function.Function; +import java.util.function.Supplier; + +/** + * Partially resolved data type that requires a lookup in a catalog or configuration before creating + * the corresponding {@link LogicalType}. + * + *

    Users are able to influence the nullability and conversion class even if the actual {@link LogicalType} + * is not fully known yet. The information is stored and verified when resolving to {@link DataType} + * lazily. + */ +@PublicEvolving +public final class UnresolvedDataType implements AbstractDataType { + + private static final String FORMAT = "[%s]"; // indicates that this is an unresolved type + + private final @Nullable Boolean isNullable; + + private final @Nullable Class conversionClass; + + private final Supplier description; + + private final Function resolutionFactory; + + private UnresolvedDataType( + @Nullable Boolean isNullable, + @Nullable Class conversionClass, + Supplier description, + Function resolutionFactory) { + this.isNullable = isNullable; + this.conversionClass = conversionClass; + this.description = description; + this.resolutionFactory = resolutionFactory; + } + + public UnresolvedDataType( + Supplier description, + Function resolutionFactory) { + this(null, null, description, resolutionFactory); + } + + /** + * Converts this instance to a resolved {@link DataType} possibly enriched with additional + * nullability and conversion class information. + */ + public DataType toDataType(DataTypeFactory factory) { + DataType resolvedDataType = resolutionFactory.apply(factory); + if (isNullable == Boolean.TRUE) { + resolvedDataType = resolvedDataType.nullable(); + } else if (isNullable == Boolean.FALSE) { + resolvedDataType = resolvedDataType.notNull(); + } + if (conversionClass != null) { + resolvedDataType = resolvedDataType.bridgedTo(conversionClass); + } + return resolvedDataType; + } + + @Override + public UnresolvedDataType notNull() { + return new UnresolvedDataType(false, conversionClass, description, resolutionFactory); + } + + @Override + public UnresolvedDataType nullable() { + return new UnresolvedDataType(true, conversionClass, description, resolutionFactory); + } + + @Override + public UnresolvedDataType bridgedTo(Class newConversionClass) { + return new UnresolvedDataType(isNullable, newConversionClass, description, resolutionFactory); + } + + @Override + public String toString() { + return String.format(FORMAT, description.get()); + } +} diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/extraction/utils/DataTypeTemplate.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/extraction/utils/DataTypeTemplate.java index 2d81f5d0e4..e7b99059bc 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/extraction/utils/DataTypeTemplate.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/extraction/utils/DataTypeTemplate.java @@ -35,7 +35,6 @@ import java.util.Objects; import java.util.function.Function; import static org.apache.flink.table.types.extraction.utils.ExtractionUtils.createRawType; -import static org.apache.flink.table.types.extraction.utils.ExtractionUtils.extractionError; /** * Internal representation of a {@link DataTypeHint}. @@ -315,14 +314,11 @@ public final class DataTypeTemplate { return createRawType(typeFactory, template.rawSerializer, conversionClass); } // regular type that must be resolvable - return typeFactory.createDataType(typeName) - .map(dataType -> { - if (conversionClass != null) { - return dataType.bridgedTo(conversionClass); - } - return dataType; - }) - .orElseThrow(() -> extractionError("Could not resolve type with name '%s'.", typeName)); + final DataType resolvedDataType = typeFactory.createDataType(typeName); + if (conversionClass != null) { + return resolvedDataType.bridgedTo(conversionClass); + } + return resolvedDataType; } // extracted data type else if (conversionClass != null) { diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/ArrayType.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/ArrayType.java index d8122af694..9340ff215e 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/ArrayType.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/ArrayType.java @@ -38,7 +38,7 @@ import java.util.Set; @PublicEvolving public final class ArrayType extends LogicalType { - private static final String FORMAT = "ARRAY<%s>"; + public static final String FORMAT = "ARRAY<%s>"; private static final Set INPUT_OUTPUT_CONVERSION = conversionSet( "org.apache.flink.table.dataformat.BinaryArray"); diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/MapType.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/MapType.java index cd0ab71814..6fb6ae7b5f 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/MapType.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/MapType.java @@ -40,7 +40,7 @@ import java.util.Set; @PublicEvolving public final class MapType extends LogicalType { - private static final String FORMAT = "MAP<%s, %s>"; + public static final String FORMAT = "MAP<%s, %s>"; private static final Set INPUT_OUTPUT_CONVERSION = conversionSet( Map.class.getName(), diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/MultisetType.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/MultisetType.java index cd4b7ee6c2..324ab1dd44 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/MultisetType.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/MultisetType.java @@ -42,7 +42,7 @@ import java.util.Set; @PublicEvolving public final class MultisetType extends LogicalType { - private static final String FORMAT = "MULTISET<%s>"; + public static final String FORMAT = "MULTISET<%s>"; private static final Set INPUT_OUTPUT_CONVERSION = conversionSet( Map.class.getName(), diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/RawType.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/RawType.java index c795464f58..d25ce29db0 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/RawType.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/RawType.java @@ -43,7 +43,7 @@ import java.util.Set; @PublicEvolving public final class RawType extends LogicalType { - private static final String FORMAT = "RAW('%s', '%s')"; + public static final String FORMAT = "RAW('%s', '%s')"; private static final Set INPUT_OUTPUT_CONVERSION = conversionSet( byte[].class.getName(), diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/RowType.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/RowType.java index c3a5cd1137..2baf805e55 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/RowType.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/RowType.java @@ -52,7 +52,7 @@ import static org.apache.flink.table.utils.EncodingUtils.escapeSingleQuotes; @PublicEvolving public final class RowType extends LogicalType { - private static final String FORMAT = "ROW<%s>"; + public static final String FORMAT = "ROW<%s>"; private static final Set INPUT_OUTPUT_CONVERSION = conversionSet( Row.class.getName(), @@ -65,9 +65,9 @@ public final class RowType extends LogicalType { */ public static final class RowField implements Serializable { - private static final String FIELD_FORMAT_WITH_DESCRIPTION = "%s %s '%s'"; + public static final String FIELD_FORMAT_WITH_DESCRIPTION = "%s %s '%s'"; - private static final String FIELD_FORMAT_NO_DESCRIPTION = "%s %s"; + public static final String FIELD_FORMAT_NO_DESCRIPTION = "%s %s"; private final String name; diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/DataTypesTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/DataTypesTest.java index 1c7e50cae0..ec37360640 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/DataTypesTest.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/DataTypesTest.java @@ -20,6 +20,7 @@ package org.apache.flink.table.types; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.common.typeutils.base.VoidSerializer; +import org.apache.flink.api.java.typeutils.GenericTypeInfo; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.types.logical.ArrayType; import org.apache.flink.table.types.logical.BigIntType; @@ -49,6 +50,7 @@ import org.apache.flink.table.types.logical.VarCharType; import org.apache.flink.table.types.logical.YearMonthIntervalType; import org.apache.flink.table.types.logical.YearMonthIntervalType.YearMonthResolution; import org.apache.flink.table.types.logical.ZonedTimestampType; +import org.apache.flink.table.types.utils.DataTypeFactoryMock; import org.apache.flink.table.types.utils.LogicalTypeDataTypeConverter; import org.apache.flink.types.Row; @@ -58,10 +60,13 @@ import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameter; import org.junit.runners.Parameterized.Parameters; +import javax.annotation.Nullable; + import java.math.BigDecimal; import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.Optional; import static org.apache.flink.table.api.DataTypes.ARRAY; import static org.apache.flink.table.api.DataTypes.BIGINT; @@ -108,130 +113,343 @@ import static org.junit.Assert.assertThat; @RunWith(Parameterized.class) public class DataTypesTest { - @Parameters(name = "{index}: {0}=[Logical: {1}, Class: {2}]") - public static List dataTypes() { + @Parameters(name = "{index}: {0}") + public static List testData() { return Arrays.asList( - new Object[][]{ - {CHAR(2), new CharType(2), String.class}, - - {VARCHAR(2), new VarCharType(2), String.class}, - - {STRING(), new VarCharType(VarCharType.MAX_LENGTH), String.class}, - - {BOOLEAN(), new BooleanType(), Boolean.class}, - - {BINARY(42), new BinaryType(42), byte[].class}, - - {VARBINARY(42), new VarBinaryType(42), byte[].class}, - - {BYTES(), new VarBinaryType(VarBinaryType.MAX_LENGTH), byte[].class}, - - {DECIMAL(10, 10), new DecimalType(10, 10), BigDecimal.class}, - - {TINYINT(), new TinyIntType(), Byte.class}, - - {SMALLINT(), new SmallIntType(), Short.class}, - - {INT(), new IntType(), Integer.class}, - - {BIGINT(), new BigIntType(), Long.class}, - - {FLOAT(), new FloatType(), Float.class}, - - {DOUBLE(), new DoubleType(), Double.class}, + TestSpec + .forDataType(CHAR(2)) + .expectLogicalType(new CharType(2)) + .expectConversionClass(String.class), + + TestSpec + .forDataType(VARCHAR(2)) + .expectLogicalType(new VarCharType(2)) + .expectConversionClass(String.class), + + TestSpec + .forDataType(STRING()) + .expectLogicalType(new VarCharType(VarCharType.MAX_LENGTH)) + .expectConversionClass(String.class), + + TestSpec + .forDataType(BOOLEAN()) + .expectLogicalType(new BooleanType()) + .expectConversionClass(Boolean.class), + + TestSpec + .forDataType(BINARY(42)) + .expectLogicalType(new BinaryType(42)) + .expectConversionClass(byte[].class), + + TestSpec + .forDataType(VARBINARY(42)) + .expectLogicalType(new VarBinaryType(42)) + .expectConversionClass(byte[].class), + + TestSpec + .forDataType(BYTES()) + .expectLogicalType(new VarBinaryType(VarBinaryType.MAX_LENGTH)) + .expectConversionClass(byte[].class), + + TestSpec + .forDataType(DECIMAL(10, 10)) + .expectLogicalType(new DecimalType(10, 10)) + .expectConversionClass(BigDecimal.class), + + TestSpec + .forDataType(TINYINT()) + .expectLogicalType(new TinyIntType()) + .expectConversionClass(Byte.class), + + TestSpec + .forDataType(SMALLINT()) + .expectLogicalType(new SmallIntType()) + .expectConversionClass(Short.class), + + TestSpec + .forDataType(INT()) + .expectLogicalType(new IntType()) + .expectConversionClass(Integer.class), + + TestSpec + .forDataType(BIGINT()) + .expectLogicalType(new BigIntType()) + .expectConversionClass(Long.class), + + TestSpec + .forDataType(FLOAT()) + .expectLogicalType(new FloatType()) + .expectConversionClass(Float.class), + + TestSpec + .forDataType(DOUBLE()) + .expectLogicalType(new DoubleType()) + .expectConversionClass(Double.class), + + TestSpec + .forDataType(DATE()) + .expectLogicalType(new DateType()) + .expectConversionClass(java.time.LocalDate.class), + + TestSpec + .forDataType(TIME(3)) + .expectLogicalType(new TimeType(3)) + .expectConversionClass(java.time.LocalTime.class), + + TestSpec + .forDataType(TIME()) + .expectLogicalType(new TimeType(0)) + .expectConversionClass(java.time.LocalTime.class), + + TestSpec + .forDataType(TIMESTAMP(3)) + .expectLogicalType(new TimestampType(3)) + .expectConversionClass(java.time.LocalDateTime.class), + + TestSpec + .forDataType(TIMESTAMP()) + .expectLogicalType(new TimestampType(6)) + .expectConversionClass(java.time.LocalDateTime.class), + + TestSpec + .forDataType(TIMESTAMP_WITH_TIME_ZONE(3)) + .expectLogicalType(new ZonedTimestampType(3)) + .expectConversionClass(java.time.OffsetDateTime.class), + + TestSpec + .forDataType(TIMESTAMP_WITH_TIME_ZONE()) + .expectLogicalType(new ZonedTimestampType(6)) + .expectConversionClass(java.time.OffsetDateTime.class), + + TestSpec + .forDataType(TIMESTAMP_WITH_LOCAL_TIME_ZONE(3)) + .expectLogicalType(new LocalZonedTimestampType(3)) + .expectConversionClass(java.time.Instant.class), + + TestSpec + .forDataType(TIMESTAMP_WITH_LOCAL_TIME_ZONE()) + .expectLogicalType(new LocalZonedTimestampType(6)) + .expectConversionClass(java.time.Instant.class), + + TestSpec + .forDataType(INTERVAL(MINUTE(), SECOND(3))) + .expectLogicalType(new DayTimeIntervalType(MINUTE_TO_SECOND, DEFAULT_DAY_PRECISION, 3)) + .expectConversionClass(java.time.Duration.class), + + TestSpec + .forDataType(INTERVAL(MONTH())) + .expectLogicalType(new YearMonthIntervalType(YearMonthResolution.MONTH)) + .expectConversionClass(java.time.Period.class), + + TestSpec + .forDataType(ARRAY(ARRAY(INT()))) + .expectLogicalType(new ArrayType(new ArrayType(new IntType()))) + .expectConversionClass(Integer[][].class), + + TestSpec + .forDataType(MULTISET(MULTISET(INT()))) + .expectLogicalType(new MultisetType(new MultisetType(new IntType()))) + .expectConversionClass(Map.class), + + TestSpec + .forDataType(MAP(INT(), SMALLINT())) + .expectLogicalType(new MapType(new IntType(), new SmallIntType())) + .expectConversionClass(Map.class), + + TestSpec + .forDataType(ROW(FIELD("field1", CHAR(2)), FIELD("field2", BOOLEAN()))) + .expectLogicalType(new RowType( + Arrays.asList( + new RowType.RowField("field1", new CharType(2)), + new RowType.RowField("field2", new BooleanType())))) + .expectConversionClass(Row.class), + + TestSpec + .forDataType(NULL()) + .expectLogicalType(new NullType()) + .expectConversionClass(Object.class), + + TestSpec + .forDataType(RAW(Types.GENERIC(DataTypesTest.class))) + .expectLogicalType(new TypeInformationRawType<>(Types.GENERIC(DataTypesTest.class))) + .expectConversionClass(DataTypesTest.class), + + TestSpec + .forDataType(RAW(Void.class, VoidSerializer.INSTANCE)) + .expectLogicalType(new RawType<>(Void.class, VoidSerializer.INSTANCE)) + .expectConversionClass(Void.class), + + TestSpec + .forUnresolvedDataType(DataTypes.of("INT")) + .expectUnresolvedString("[INT]") + .lookupReturns(INT()) + .expectLogicalType(new IntType()), + + TestSpec + .forUnresolvedDataType(DataTypes.of(Integer.class)) + .expectUnresolvedString("['java.lang.Integer']") + .expectResolvedDataType(INT()), + + TestSpec + .forUnresolvedDataType(DataTypes.of(java.sql.Timestamp.class).notNull()) + .expectUnresolvedString("['java.sql.Timestamp']") + .expectResolvedDataType(TIMESTAMP(9).notNull().bridgedTo(java.sql.Timestamp.class)), + + TestSpec + .forUnresolvedDataType( + DataTypes.of(java.sql.Timestamp.class).bridgedTo(java.time.LocalDateTime.class)) + .expectUnresolvedString("['java.sql.Timestamp']") + .expectResolvedDataType(TIMESTAMP(9).bridgedTo(java.time.LocalDateTime.class)), + + TestSpec + .forUnresolvedDataType(MAP(DataTypes.of("INT"), DataTypes.of("STRING"))) + .expectUnresolvedString("[MAP<[INT], [STRING]>]") + .expectResolvedDataType(MAP(DataTypes.INT(), DataTypes.STRING())), + + TestSpec + .forUnresolvedDataType(MAP(DataTypes.of("INT"), STRING().notNull())) + .expectUnresolvedString("[MAP<[INT], STRING NOT NULL>]") + .expectResolvedDataType(MAP(INT(), STRING().notNull())), + + TestSpec + .forUnresolvedDataType(MULTISET(DataTypes.of("STRING"))) + .expectUnresolvedString("[MULTISET<[STRING]>]") + .expectResolvedDataType(MULTISET(DataTypes.STRING())), + + TestSpec + .forUnresolvedDataType(ARRAY(DataTypes.of("STRING"))) + .expectUnresolvedString("[ARRAY<[STRING]>]") + .expectResolvedDataType(ARRAY(DataTypes.STRING())), + + TestSpec + .forUnresolvedDataType( + ARRAY(DataTypes.of("INT").notNull()).bridgedTo(int[].class)) + .expectUnresolvedString("[ARRAY<[INT]>]") + .expectResolvedDataType(ARRAY(INT().notNull()).bridgedTo(int[].class)), + + TestSpec + .forUnresolvedDataType( + ROW(FIELD("field1", DataTypes.of("CHAR(2)")), FIELD("field2", BOOLEAN()))) + .expectUnresolvedString("[ROW]") + .expectResolvedDataType(ROW(FIELD("field1", CHAR(2)), FIELD("field2", BOOLEAN()))), + + TestSpec + .forUnresolvedDataType( + ARRAY( + ROW( + FIELD("f0", DataTypes.of("ARRAY")), + FIELD("f1", ARRAY(INT()))))) + .expectUnresolvedString("[ARRAY<[ROW], f1 ARRAY>]>]") + .expectResolvedDataType( + ARRAY( + ROW( + FIELD("f0", ARRAY(INT())), + FIELD("f1", ARRAY(INT())))) + ), + + TestSpec + .forUnresolvedDataType(RAW(Object.class)) + .expectUnresolvedString("[RAW('java.lang.Object', '?')]") + .lookupReturns(DataTypes.RAW(new GenericTypeInfo<>(Object.class))) + .expectResolvedDataType(DataTypes.RAW(new GenericTypeInfo<>(Object.class))) + ); + } - {DATE(), new DateType(), java.time.LocalDate.class}, + @Parameter + public TestSpec testSpec; - {TIME(3), new TimeType(3), java.time.LocalTime.class}, + @Test + public void testLogicalType() { + if (testSpec.expectedLogicalType != null) { + final DataType dataType = testSpec.typeFactory.createDataType(testSpec.abstractDataType); - {TIME(), new TimeType(0), java.time.LocalTime.class}, + assertThat(dataType, hasLogicalType(testSpec.expectedLogicalType)); - {TIMESTAMP(3), new TimestampType(3), java.time.LocalDateTime.class}, + assertThat(toDataType(testSpec.expectedLogicalType), equalTo(dataType)); - {TIMESTAMP(), new TimestampType(6), java.time.LocalDateTime.class}, + assertThat(toLogicalType(dataType), equalTo(testSpec.expectedLogicalType)); + } + } - {TIMESTAMP_WITH_TIME_ZONE(3), - new ZonedTimestampType(3), - java.time.OffsetDateTime.class}, + @Test + public void testConversionClass() { + if (testSpec.expectedConversionClass != null) { + final DataType dataType = testSpec.typeFactory.createDataType(testSpec.abstractDataType); + assertThat(dataType, hasConversionClass(testSpec.expectedConversionClass)); + } + } - {TIMESTAMP_WITH_TIME_ZONE(), - new ZonedTimestampType(6), - java.time.OffsetDateTime.class}, + @Test + public void testUnresolvedString() { + if (testSpec.expectedUnresolvedString != null) { + assertThat(testSpec.abstractDataType.toString(), equalTo(testSpec.expectedUnresolvedString)); + } + } - {TIMESTAMP_WITH_LOCAL_TIME_ZONE(3), - new LocalZonedTimestampType(3), - java.time.Instant.class}, + @Test + public void testResolvedDataType() { + if (testSpec.expectedResolvedDataType != null) { + final DataType dataType = testSpec.typeFactory.createDataType(testSpec.abstractDataType); + assertThat(dataType, equalTo(testSpec.expectedResolvedDataType)); + } + } - {TIMESTAMP_WITH_LOCAL_TIME_ZONE(), - new LocalZonedTimestampType(6), - java.time.Instant.class}, + // -------------------------------------------------------------------------------------------- - {INTERVAL(MINUTE(), SECOND(3)), - new DayTimeIntervalType(MINUTE_TO_SECOND, DEFAULT_DAY_PRECISION, 3), - java.time.Duration.class}, + private static class TestSpec { - {INTERVAL(MONTH()), - new YearMonthIntervalType(YearMonthResolution.MONTH), - java.time.Period.class}, + private final DataTypeFactoryMock typeFactory = new DataTypeFactoryMock(); - {ARRAY(ARRAY(INT())), - new ArrayType(new ArrayType(new IntType())), - Integer[][].class}, + private final AbstractDataType abstractDataType; - {MULTISET(MULTISET(INT())), - new MultisetType(new MultisetType(new IntType())), - Map.class}, + private @Nullable LogicalType expectedLogicalType; - {MAP(INT(), SMALLINT()), - new MapType(new IntType(), new SmallIntType()), - Map.class}, + private @Nullable Class expectedConversionClass; - {ROW(FIELD("field1", CHAR(2)), FIELD("field2", BOOLEAN())), - new RowType( - Arrays.asList( - new RowType.RowField("field1", new CharType(2)), - new RowType.RowField("field2", new BooleanType()))), - Row.class}, + private @Nullable String expectedUnresolvedString; - {NULL(), new NullType(), Object.class}, + private @Nullable DataType expectedResolvedDataType; - {RAW(Types.GENERIC(DataTypesTest.class)), - new TypeInformationRawType<>(Types.GENERIC(DataTypesTest.class)), - DataTypesTest.class}, + private TestSpec(AbstractDataType abstractDataType) { + this.abstractDataType = abstractDataType; + } - {RAW(Void.class, VoidSerializer.INSTANCE), - new RawType<>(Void.class, VoidSerializer.INSTANCE), - Void.class} - } - ); - } + static TestSpec forDataType(DataType dataType) { + return new TestSpec(dataType); + } - @Parameter - public DataType dataType; + static TestSpec forUnresolvedDataType(UnresolvedDataType unresolvedDataType) { + return new TestSpec(unresolvedDataType); + } - @Parameter(1) - public LogicalType expectedLogicalType; + TestSpec expectLogicalType(LogicalType expectedLogicalType) { + this.expectedLogicalType = expectedLogicalType; + return this; + } - @Parameter(2) - public Class expectedConversionClass; + TestSpec expectConversionClass(Class expectedConversionClass) { + this.expectedConversionClass = expectedConversionClass; + return this; + } - @Test - public void testLogicalType() { - assertThat(dataType, hasLogicalType(expectedLogicalType)); - } + TestSpec expectUnresolvedString(String expectedUnresolvedString) { + this.expectedUnresolvedString = expectedUnresolvedString; + return this; + } - @Test - public void testConversionClass() { - assertThat(dataType, hasConversionClass(expectedConversionClass)); - } + TestSpec expectResolvedDataType(DataType expectedResolvedDataType) { + this.expectedResolvedDataType = expectedResolvedDataType; + return this; + } - @Test - public void testLogicalTypeToDataTypeConversion() { - assertThat(toDataType(expectedLogicalType), equalTo(dataType)); - } + TestSpec lookupReturns(DataType dataType) { + this.typeFactory.dataType = Optional.of(dataType); + return this; + } - @Test - public void testDataTypeToLogicalTypeConversion() { - assertThat(toLogicalType(dataType), equalTo(expectedLogicalType)); + @Override + public String toString() { + return abstractDataType.toString(); + } } } diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/extraction/DataTypeExtractorTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/extraction/DataTypeExtractorTest.java index 4db5b6d0f9..68ac8e6141 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/extraction/DataTypeExtractorTest.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/extraction/DataTypeExtractorTest.java @@ -32,7 +32,6 @@ import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.FieldsDataType; import org.apache.flink.table.types.extraction.utils.DataTypeHintMock; import org.apache.flink.table.types.extraction.utils.DataTypeTemplate; -import org.apache.flink.table.types.inference.utils.DataTypeFactoryMock; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BooleanType; import org.apache.flink.table.types.logical.IntType; @@ -40,6 +39,7 @@ import org.apache.flink.table.types.logical.MapType; import org.apache.flink.table.types.logical.StructuredType; import org.apache.flink.table.types.logical.TypeInformationRawType; import org.apache.flink.table.types.logical.VarCharType; +import org.apache.flink.table.types.utils.DataTypeFactoryMock; import org.apache.flink.types.Row; import org.hamcrest.Matcher; @@ -402,16 +402,15 @@ public class DataTypeExtractorTest { */ static class TestSpec { - private final Function extractor; + private final DataTypeFactoryMock typeFactory = new DataTypeFactoryMock(); - private DataTypeFactoryMock typeFactory; + private final Function extractor; private @Nullable DataType expectedDataType; private @Nullable String expectedErrorMessage; private TestSpec(Function extractor) { - this.typeFactory = new DataTypeFactoryMock(); this.extractor = extractor; } diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/extraction/TypeInferenceExtractorTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/extraction/TypeInferenceExtractorTest.java index 5f64acc984..ee75f6b16b 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/extraction/TypeInferenceExtractorTest.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/extraction/TypeInferenceExtractorTest.java @@ -34,7 +34,7 @@ import org.apache.flink.table.types.inference.InputTypeStrategy; import org.apache.flink.table.types.inference.TypeInference; import org.apache.flink.table.types.inference.TypeStrategies; import org.apache.flink.table.types.inference.TypeStrategy; -import org.apache.flink.table.types.inference.utils.DataTypeFactoryMock; +import org.apache.flink.table.types.utils.DataTypeFactoryMock; import org.apache.flink.types.Row; import org.hamcrest.Matcher; diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/InputTypeStrategiesTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/InputTypeStrategiesTest.java index 9bfb1b147f..2389352754 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/InputTypeStrategiesTest.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/InputTypeStrategiesTest.java @@ -23,8 +23,8 @@ import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.functions.FunctionKind; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.inference.utils.CallContextMock; -import org.apache.flink.table.types.inference.utils.DataTypeFactoryMock; import org.apache.flink.table.types.inference.utils.FunctionDefinitionMock; +import org.apache.flink.table.types.utils.DataTypeFactoryMock; import org.junit.Rule; import org.junit.Test; diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/utils/DataTypeFactoryMock.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/utils/DataTypeFactoryMock.java similarity index 66% rename from flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/utils/DataTypeFactoryMock.java rename to flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/utils/DataTypeFactoryMock.java index e55a26f182..12003728ac 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/inference/utils/DataTypeFactoryMock.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/utils/DataTypeFactoryMock.java @@ -16,14 +16,16 @@ * limitations under the License. */ -package org.apache.flink.table.types.inference.utils; +package org.apache.flink.table.types.utils; +import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.DataTypeFactory; import org.apache.flink.table.catalog.UnresolvedIdentifier; +import org.apache.flink.table.types.AbstractDataType; import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.UnresolvedDataType; import org.apache.flink.table.types.extraction.DataTypeExtractor; import org.apache.flink.table.types.logical.utils.LogicalTypeParser; -import org.apache.flink.table.types.utils.TypeConversions; import java.util.Optional; @@ -39,17 +41,28 @@ public class DataTypeFactoryMock implements DataTypeFactory { public Optional> expectedClass = Optional.empty(); @Override - public Optional createDataType(String name) { - return Optional.of(TypeConversions.fromLogicalToDataType(LogicalTypeParser.parse(name))); + public DataType createDataType(AbstractDataType abstractDataType) { + if (abstractDataType instanceof DataType) { + return (DataType) abstractDataType; + } else if (abstractDataType instanceof UnresolvedDataType) { + return ((UnresolvedDataType) abstractDataType).toDataType(this); + } + throw new IllegalStateException(); } @Override - public Optional createDataType(UnresolvedIdentifier identifier) { - return dataType; + public DataType createDataType(String name) { + return TypeConversions.fromLogicalToDataType(LogicalTypeParser.parse(name)); + } + + @Override + public DataType createDataType(UnresolvedIdentifier identifier) { + return dataType.orElseThrow(() -> new ValidationException("No type found.")); } @Override public DataType createDataType(Class clazz) { + expectedClass.ifPresent(expected -> assertEquals(expected, clazz)); return DataTypeExtractor.extractFromType(this, clazz); } -- Gitee From 5ce71d4def3af1e7491cd6075b034710a5da87f0 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Fri, 28 Feb 2020 17:02:11 +0100 Subject: [PATCH 203/885] [FLINK-16344][table-planner-blink] Preserve nullability for nested types Avoids errors due to incompatible types during planning by preserving the nullability attributes of nested types. This closes #11260. --- .../flink/table/planner/calcite/FlinkTypeFactory.scala | 5 +++++ .../table/planner/runtime/stream/sql/FunctionITCase.java | 4 ++-- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeFactory.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeFactory.scala index 1c4bda528f..a134ccdee0 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeFactory.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeFactory.scala @@ -293,6 +293,11 @@ class FlinkTypeFactory(typeSystem: RelDataTypeSystem) extends JavaTypeFactoryImp case it: TimeIndicatorRelDataType => new TimeIndicatorRelDataType(it.typeSystem, it.originalType, isNullable, it.isEventTime) + // for nested rows we keep the nullability property, + // top-level rows fall back to Calcite's default handling + case rt: RelRecordType if rt.getStructKind == StructKind.PEEK_FIELDS_NO_EXPAND => + new RelRecordType(rt.getStructKind, rt.getFieldList, isNullable); + case _ => super.createTypeWithNullability(relDataType, isNullable) } diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/FunctionITCase.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/FunctionITCase.java index fcc26d1a65..f250f8b73a 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/FunctionITCase.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/FunctionITCase.java @@ -603,7 +603,7 @@ public class FunctionITCase extends StreamingTestBase { TestCollectionTableFactory.initData(sourceData); tEnv().sqlUpdate("CREATE TABLE SourceTable(s STRING) WITH ('connector' = 'COLLECTION')"); - tEnv().sqlUpdate("CREATE TABLE SinkTable(s STRING, sa ARRAY) WITH ('connector' = 'COLLECTION')"); + tEnv().sqlUpdate("CREATE TABLE SinkTable(s STRING, sa ARRAY NOT NULL) WITH ('connector' = 'COLLECTION')"); tEnv().createTemporarySystemFunction("RowTableFunction", RowTableFunction.class); tEnv().sqlUpdate("INSERT INTO SinkTable SELECT t.s, t.sa FROM SourceTable, LATERAL TABLE(RowTableFunction(s)) t"); @@ -761,7 +761,7 @@ public class FunctionITCase extends StreamingTestBase { /** * Function that returns a row. */ - @FunctionHint(output = @DataTypeHint("ROW>")) + @FunctionHint(output = @DataTypeHint("ROW NOT NULL>")) public static class RowTableFunction extends TableFunction { public void eval(String s) { if (s == null) { -- Gitee From e30089c310892526a39a92337ab6bd4fa65f717d Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Wed, 18 Mar 2020 11:43:04 +0100 Subject: [PATCH 204/885] [FLINK-15771][tests] Harden SQLClientKafkaITCase --- .../util/kafka/SQLClientKafkaITCase.java | 22 ++++++++++++------- .../util/cache/AbstractDownloadCache.java | 4 ++-- 2 files changed, 16 insertions(+), 10 deletions(-) diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientKafkaITCase.java b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientKafkaITCase.java index aa8832a9d6..bf5e4f336b 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientKafkaITCase.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientKafkaITCase.java @@ -56,6 +56,9 @@ import java.util.Collection; import java.util.HashMap; import java.util.Map; +import static org.hamcrest.Matchers.arrayContainingInAnyOrder; +import static org.junit.Assert.assertThat; + /** * End-to-end test for the kafka SQL connectors. */ @@ -219,15 +222,18 @@ public class SQLClientKafkaITCase extends TestLogger { for (int i = 0; i < maxRetries; i++) { if (Files.exists(result)) { byte[] bytes = Files.readAllBytes(result); - String lines = new String(bytes, Charsets.UTF_8); - if (lines.split("\n").length == 4) { + String[] lines = new String(bytes, Charsets.UTF_8).split("\n"); + if (lines.length == 4) { success = true; - String expected = - "2018-03-12 08:00:00.000,Alice,This was a warning.,2,Success constant folding.\n" + - "2018-03-12 09:00:00.000,Bob,This was another warning.,1,Success constant folding.\n" + - "2018-03-12 09:00:00.000,Steve,This was another info.,2,Success constant folding.\n" + - "2018-03-12 09:00:00.000,Alice,This was a info.,1,Success constant folding.\n"; - Assert.assertEquals(expected, lines); + assertThat( + lines, + arrayContainingInAnyOrder( + "2018-03-12 08:00:00.000,Alice,This was a warning.,2,Success constant folding.", + "2018-03-12 09:00:00.000,Bob,This was another warning.,1,Success constant folding.", + "2018-03-12 09:00:00.000,Steve,This was another info.,2,Success constant folding.", + "2018-03-12 09:00:00.000,Alice,This was a info.,1,Success constant folding." + ) + ); break; } } else { diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/AbstractDownloadCache.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/AbstractDownloadCache.java index 4250f1d4ac..48a1c5fdad 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/AbstractDownloadCache.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/AbstractDownloadCache.java @@ -64,9 +64,9 @@ abstract class AbstractDownloadCache implements DownloadCache { this.downloadAttemptTimeout = DOWNLOAD_ATTEMPT_TIMEOUT.get(Duration.ofMinutes(2)); this.downloadGlobalTimeout = DOWNLOAD_GLOBAL_TIMEOUT.get(Duration.ofMinutes(2)); - this.downloadMaxRetries = DOWNLOAD_MAX_RETRIES.get(1); + this.downloadMaxRetries = DOWNLOAD_MAX_RETRIES.get(3); - log.info("Download configuration: maxAttempts={}, attemptTimeout={}, globalTimeout={}", downloadMaxRetries, downloadAttemptTimeout, downloadGlobalTimeout); + log.info("Download configuration: maxRetries={}, attemptTimeout={}, globalTimeout={}", downloadMaxRetries, downloadAttemptTimeout, downloadGlobalTimeout); } @Override -- Gitee From aeffce4b1783649ea19cf54ba6a36f377f56fee9 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Wed, 18 Mar 2020 11:45:09 +0100 Subject: [PATCH 205/885] [hotfix][tests] Fix exception in AutoClosableProcess --- .../java/org/apache/flink/tests/util/AutoClosableProcess.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/AutoClosableProcess.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/AutoClosableProcess.java index d54bfef53f..65bc156096 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/AutoClosableProcess.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/AutoClosableProcess.java @@ -128,7 +128,7 @@ public class AutoClosableProcess implements AutoCloseable { if (++retries > maxRetries || !globalDeadline.hasTimeLeft()) { String errMsg = String.format( "Process (%s) exceeded timeout (%s) or number of retries (%s).", - Arrays.toString(commands), maxRetries, globalTimeout.toMillis()); + Arrays.toString(commands), globalTimeout.toMillis(), maxRetries); throw new IOException(errMsg, e); } } -- Gitee From 6aa4a3f583633835fece5893736b5d7d0ea2ee60 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Wed, 18 Mar 2020 12:12:32 +0100 Subject: [PATCH 206/885] [FLINK-16583][sql-client] Provide classloader during pipeline creation Fixes a classloading bug that was introduced in the last release while updating the code for the new Pipeline abstractions. This closes #11438. --- .../client/gateway/local/ExecutionContext.java | 14 ++++++++------ .../table/client/gateway/local/LocalExecutor.java | 12 ++++++------ 2 files changed, 14 insertions(+), 12 deletions(-) diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java index 51de2c0e92..0529434326 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java @@ -271,12 +271,14 @@ public class ExecutionContext { } public Pipeline createPipeline(String name) { - if (streamExecEnv != null) { - StreamTableEnvironmentImpl streamTableEnv = (StreamTableEnvironmentImpl) tableEnv; - return streamTableEnv.getPipeline(name); - } else { - return execEnv.createProgramPlan(name); - } + return wrapClassLoader(() -> { + if (streamExecEnv != null) { + StreamTableEnvironmentImpl streamTableEnv = (StreamTableEnvironmentImpl) tableEnv; + return streamTableEnv.getPipeline(name); + } else { + return execEnv.createProgramPlan(name); + } + }); } /** Returns a builder for this {@link ExecutionContext}. */ diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java index 3035d91acf..1c9ea2a276 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java @@ -35,7 +35,6 @@ import org.apache.flink.core.execution.JobClient; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.Path; import org.apache.flink.core.plugin.PluginUtils; -import org.apache.flink.table.api.QueryConfig; import org.apache.flink.table.api.StreamQueryConfig; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.TableEnvironment; @@ -573,14 +572,15 @@ public class LocalExecutor implements Executor { String sessionId, ExecutionContext context, String statement) { - applyUpdate(context, context.getTableEnvironment(), context.getQueryConfig(), statement); + + applyUpdate(context, statement); //Todo: we should refactor following condition after TableEnvironment has support submit job directly. if (!INSERT_SQL_PATTERN.matcher(statement.trim()).matches()) { return null; } - // create job graph with dependencies + // create pipeline final String jobName = sessionId + ": " + statement; final Pipeline pipeline; try { @@ -683,12 +683,12 @@ public class LocalExecutor implements Executor { /** * Applies the given update statement to the given table environment with query configuration. */ - private void applyUpdate(ExecutionContext context, TableEnvironment tableEnv, QueryConfig queryConfig, String updateStatement) { - // parse and validate statement + private void applyUpdate(ExecutionContext context, String updateStatement) { + final TableEnvironment tableEnv = context.getTableEnvironment(); try { context.wrapClassLoader(() -> { if (tableEnv instanceof StreamTableEnvironment) { - ((StreamTableEnvironment) tableEnv).sqlUpdate(updateStatement, (StreamQueryConfig) queryConfig); + ((StreamTableEnvironment) tableEnv).sqlUpdate(updateStatement, (StreamQueryConfig) context.getQueryConfig()); } else { tableEnv.sqlUpdate(updateStatement); } -- Gitee From a56738fdaf4ebf89784a31fe1317fd4cd97d4de2 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Thu, 19 Mar 2020 00:33:47 +0100 Subject: [PATCH 207/885] [FLINK-16664][stream] fix wrong check for DataStreamSource#setParallelism() This closes #11446. --- .../apache/flink/streaming/api/datastream/DataStreamSource.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java index f5fb05fb5a..a6fbac84cc 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java @@ -52,7 +52,7 @@ public class DataStreamSource extends SingleOutputStreamOperator { @Override public DataStreamSource setParallelism(int parallelism) { - OperatorValidationUtils.validateMaxParallelism(parallelism, isParallel); + OperatorValidationUtils.validateParallelism(parallelism, isParallel); super.setParallelism(parallelism); return this; } -- Gitee From 43f455dc54014139bcc3379e78c73b42f551094c Mon Sep 17 00:00:00 2001 From: chen <58683632@qq.com> Date: Sun, 15 Mar 2020 21:00:26 +0800 Subject: [PATCH 208/885] [FLINK-16592][docs] The doc of Streaming File Sink has a mistake of grammar This closes #11409. --- docs/dev/connectors/streamfile_sink.md | 2 +- docs/dev/connectors/streamfile_sink.zh.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/dev/connectors/streamfile_sink.md b/docs/dev/connectors/streamfile_sink.md index bf25182179..2e302a6b42 100644 --- a/docs/dev/connectors/streamfile_sink.md +++ b/docs/dev/connectors/streamfile_sink.md @@ -84,7 +84,7 @@ import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies. DataStream input = ...; final StreamingFileSink sink = StreamingFileSink - .forRowFormat(new Path(outputPath), new SimpleStringEncoder<>("UTF-8")) + .forRowFormat(new Path(outputPath), new SimpleStringEncoder("UTF-8")) .withRollingPolicy( DefaultRollingPolicy.builder() .withRolloverInterval(TimeUnit.MINUTES.toMillis(15)) diff --git a/docs/dev/connectors/streamfile_sink.zh.md b/docs/dev/connectors/streamfile_sink.zh.md index fdf678e34e..bd74bef529 100644 --- a/docs/dev/connectors/streamfile_sink.zh.md +++ b/docs/dev/connectors/streamfile_sink.zh.md @@ -73,7 +73,7 @@ import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies. DataStream input = ...; final StreamingFileSink sink = StreamingFileSink - .forRowFormat(new Path(outputPath), new SimpleStringEncoder<>("UTF-8")) + .forRowFormat(new Path(outputPath), new SimpleStringEncoder("UTF-8")) .withRollingPolicy( DefaultRollingPolicy.builder() .withRolloverInterval(TimeUnit.MINUTES.toMillis(15)) -- Gitee From 8c3c54821dd5907fab2adab107e7391908980206 Mon Sep 17 00:00:00 2001 From: Niels Basjes Date: Thu, 19 Mar 2020 13:26:14 +0100 Subject: [PATCH 209/885] [FLINK-16432][hive] Fix dependencies in Hive Connector build This closes #11316 --- flink-connectors/flink-connector-hive/pom.xml | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/flink-connectors/flink-connector-hive/pom.xml b/flink-connectors/flink-connector-hive/pom.xml index 04efcd952e..2b43b969b6 100644 --- a/flink-connectors/flink-connector-hive/pom.xml +++ b/flink-connectors/flink-connector-hive/pom.xml @@ -706,6 +706,10 @@ under the License. jdk.tools jdk.tools + + jms + javax.jms + log4j log4j @@ -714,6 +718,10 @@ under the License. org.slf4j slf4j-log4j12 + + org.pentaho + pentaho-aggdesigner-algorithm + -- Gitee From e19bc924a7a6c7e4522b850e2020e2dc4847da57 Mon Sep 17 00:00:00 2001 From: Rui Li Date: Thu, 19 Mar 2020 20:32:18 +0800 Subject: [PATCH 210/885] [FLINK-16675][test] Fix TableEnvironmentITCase.testClearOperation failure This closes #11451 --- .../org/apache/flink/table/api/TableEnvironmentITCase.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentITCase.scala index 5501e1a468..ad003c58f4 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentITCase.scala @@ -43,6 +43,8 @@ import org.junit.{Assert, Before, Rule, Test} import _root_.java.io.File import _root_.java.util +import org.apache.flink.table.planner.factories.utils.TestCollectionTableFactory + import _root_.scala.collection.mutable @@ -251,6 +253,7 @@ class TableEnvironmentITCase(tableEnvName: String, isStreaming: Boolean) { @Test def testClearOperation(): Unit = { + TestCollectionTableFactory.reset() val tableEnv = TableEnvironmentImpl.create(settings) tableEnv.sqlUpdate("create table dest1(x map) with('connector' = 'COLLECTION')") tableEnv.sqlUpdate("create table dest2(x int) with('connector' = 'COLLECTION')") -- Gitee From 8c224fe7c1226424af57fbff1a543c4c9c01ab0a Mon Sep 17 00:00:00 2001 From: Rong Rong Date: Sat, 22 Feb 2020 09:42:43 -0800 Subject: [PATCH 211/885] [FLINK-16236][yarn][hotfix] fix YARNSessionFIFOSecuredITCase not loading the right security context factory --- .../yarn/YARNSessionFIFOSecuredITCase.java | 2 + .../TestHadoopSecurityContextFactory.java | 63 +++++++++++++++++++ ...e.security.contexts.SecurityContextFactory | 16 +++++ 3 files changed, 81 insertions(+) create mode 100644 flink-yarn-tests/src/test/java/org/apache/flink/yarn/util/TestHadoopSecurityContextFactory.java create mode 100644 flink-yarn-tests/src/test/resources/META-INF/services/org.apache.flink.runtime.security.contexts.SecurityContextFactory diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java index b210c4429e..80822c20e4 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java @@ -79,6 +79,8 @@ public class YARNSessionFIFOSecuredITCase extends YARNSessionFIFOITCase { TestHadoopModuleFactory.hadoopConfiguration = YARN_CONFIGURATION; flinkConfig.set(SecurityOptions.SECURITY_MODULE_FACTORY_CLASSES, Collections.singletonList("org.apache.flink.yarn.util.TestHadoopModuleFactory")); + flinkConfig.set(SecurityOptions.SECURITY_CONTEXT_FACTORY_CLASSES, + Collections.singletonList("org.apache.flink.yarn.util.TestHadoopSecurityContextFactory")); SecurityConfiguration securityConfig = new SecurityConfiguration(flinkConfig); diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/util/TestHadoopSecurityContextFactory.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/util/TestHadoopSecurityContextFactory.java new file mode 100644 index 0000000000..3d83d4ad73 --- /dev/null +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/util/TestHadoopSecurityContextFactory.java @@ -0,0 +1,63 @@ +/* + * 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.yarn.util; + +import org.apache.flink.runtime.security.SecurityConfiguration; +import org.apache.flink.runtime.security.contexts.HadoopSecurityContext; +import org.apache.flink.runtime.security.contexts.HadoopSecurityContextFactory; +import org.apache.flink.runtime.security.contexts.SecurityContext; +import org.apache.flink.runtime.security.contexts.SecurityContextFactory; + +import org.apache.hadoop.security.UserGroupInformation; + +/** + * Test hadoop context module factory associated with {@link TestHadoopModuleFactory}. + */ +public class TestHadoopSecurityContextFactory implements SecurityContextFactory { + + @Override + public boolean isCompatibleWith(SecurityConfiguration securityConfig) { + if (!securityConfig.getSecurityModuleFactories().contains(TestHadoopModuleFactory.class.getCanonicalName())) { + return false; + } + try { + Class.forName( + "org.apache.hadoop.security.UserGroupInformation", + false, + HadoopSecurityContextFactory.class.getClassLoader()); + return true; + } catch (ClassNotFoundException e) { + return false; + } + } + + @Override + public SecurityContext createContext(SecurityConfiguration securityConfig) { + try { + Class.forName( + "org.apache.hadoop.security.UserGroupInformation", + false, + TestHadoopSecurityContextFactory.class.getClassLoader()); + UserGroupInformation loginUser = UserGroupInformation.getLoginUser(); + return new HadoopSecurityContext(loginUser); + } catch (Exception e) { + return null; + } + } +} diff --git a/flink-yarn-tests/src/test/resources/META-INF/services/org.apache.flink.runtime.security.contexts.SecurityContextFactory b/flink-yarn-tests/src/test/resources/META-INF/services/org.apache.flink.runtime.security.contexts.SecurityContextFactory new file mode 100644 index 0000000000..6fe29ef7e5 --- /dev/null +++ b/flink-yarn-tests/src/test/resources/META-INF/services/org.apache.flink.runtime.security.contexts.SecurityContextFactory @@ -0,0 +1,16 @@ +# 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. + +org.apache.flink.yarn.util.TestHadoopSecurityContextFactory -- Gitee From 9b3492928966075fbf46b8a794fc16ade4bdef4e Mon Sep 17 00:00:00 2001 From: Rong Rong Date: Wed, 4 Mar 2020 19:06:57 -0800 Subject: [PATCH 212/885] [FLINK-16236][yarn][runtime] Change security context factory initialization 1. throws exception when isCompatible returns true but initialziation fails. 2. SecurityUtils handles logging of exception and fallback strategy to the next context factory. This closes #11189. --- .../SecurityContextInitializeException.java | 36 +++++++++++++++++++ .../flink/runtime/security/SecurityUtils.java | 10 ++++-- .../HadoopSecurityContextFactory.java | 9 ++--- .../contexts/NoOpSecurityContextFactory.java | 3 +- .../contexts/SecurityContextFactory.java | 3 +- ...rCompatibleTestSecurityContextFactory.java | 3 +- ...ncompatibleTestSecurityContextFactory.java | 3 +- .../contexts/TestSecurityContextFactory.java | 3 +- .../yarn/YARNSessionFIFOSecuredITCase.java | 5 +++ .../TestHadoopSecurityContextFactory.java | 23 +++--------- 10 files changed, 65 insertions(+), 33 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityContextInitializeException.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityContextInitializeException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityContextInitializeException.java new file mode 100644 index 0000000000..1df56c82d0 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityContextInitializeException.java @@ -0,0 +1,36 @@ +/* + * 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.runtime.security; + +import java.security.GeneralSecurityException; + +/** + * Indicates a problem with installing or uninstalling a security module or context. + */ +public class SecurityContextInitializeException extends GeneralSecurityException { + private static final long serialVersionUID = 1L; + + public SecurityContextInitializeException(String msg) { + super(msg); + } + + public SecurityContextInitializeException(String msg, Throwable cause) { + super(msg, cause); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityUtils.java index d1767a42ab..c1693923db 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityUtils.java @@ -88,9 +88,13 @@ public class SecurityUtils { try { SecurityContextFactory contextFactory = SecurityFactoryServiceLoader.findContextFactory(contextFactoryClass); if (contextFactory.isCompatibleWith(config)) { - // install the first context that's compatible. - installedContext = contextFactory.createContext(config); - break; + try { + installedContext = contextFactory.createContext(config); + // install the first context that's compatible and ignore the remaining. + break; + } catch (SecurityContextInitializeException e) { + LOG.error("Cannot instantiate security context with: " + contextFactoryClass, e); + } } else { LOG.warn("Unable to install incompatible security context factory {}", contextFactoryClass); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/security/contexts/HadoopSecurityContextFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/security/contexts/HadoopSecurityContextFactory.java index b49221a2d2..bc175c3a93 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/security/contexts/HadoopSecurityContextFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/security/contexts/HadoopSecurityContextFactory.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.security.contexts; import org.apache.flink.runtime.security.SecurityConfiguration; +import org.apache.flink.runtime.security.SecurityContextInitializeException; import org.apache.flink.runtime.security.modules.HadoopModuleFactory; import org.apache.hadoop.security.UserGroupInformation; @@ -53,17 +54,13 @@ public class HadoopSecurityContextFactory implements SecurityContextFactory { } @Override - public SecurityContext createContext(SecurityConfiguration securityConfig) { + public SecurityContext createContext(SecurityConfiguration securityConfig) throws SecurityContextInitializeException { try { - Class.forName( - "org.apache.hadoop.security.UserGroupInformation", - false, - HadoopSecurityContextFactory.class.getClassLoader()); UserGroupInformation loginUser = UserGroupInformation.getLoginUser(); return new HadoopSecurityContext(loginUser); } catch (Exception e) { LOG.error("Cannot instantiate HadoopSecurityContext.", e); - return null; + throw new SecurityContextInitializeException("Cannot instantiate HadoopSecurityContext.", e); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/security/contexts/NoOpSecurityContextFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/security/contexts/NoOpSecurityContextFactory.java index 394848afa0..dadd45ad49 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/security/contexts/NoOpSecurityContextFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/security/contexts/NoOpSecurityContextFactory.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.security.contexts; import org.apache.flink.runtime.security.SecurityConfiguration; +import org.apache.flink.runtime.security.SecurityContextInitializeException; /** * Default security context factory for {@link NoOpSecurityContext}. @@ -31,7 +32,7 @@ public class NoOpSecurityContextFactory implements SecurityContextFactory { } @Override - public SecurityContext createContext(SecurityConfiguration securityConfig) { + public SecurityContext createContext(SecurityConfiguration securityConfig) throws SecurityContextInitializeException { return new NoOpSecurityContext(); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/security/contexts/SecurityContextFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/security/contexts/SecurityContextFactory.java index c6f3fd1402..89c01e26b5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/security/contexts/SecurityContextFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/security/contexts/SecurityContextFactory.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.security.contexts; import org.apache.flink.runtime.security.SecurityConfiguration; +import org.apache.flink.runtime.security.SecurityContextInitializeException; /** * A factory for a {@link SecurityContext}. @@ -47,5 +48,5 @@ public interface SecurityContextFactory { * @param securityConfig security configuration used to create context. * @return the security context object. */ - SecurityContext createContext(SecurityConfiguration securityConfig); + SecurityContext createContext(SecurityConfiguration securityConfig) throws SecurityContextInitializeException; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/security/contexts/AnotherCompatibleTestSecurityContextFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/security/contexts/AnotherCompatibleTestSecurityContextFactory.java index 2cc79781e9..db77a7b37d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/security/contexts/AnotherCompatibleTestSecurityContextFactory.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/security/contexts/AnotherCompatibleTestSecurityContextFactory.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.security.contexts; import org.apache.flink.runtime.security.SecurityConfiguration; +import org.apache.flink.runtime.security.SecurityContextInitializeException; import java.util.concurrent.Callable; @@ -34,7 +35,7 @@ public class AnotherCompatibleTestSecurityContextFactory implements SecurityCont } @Override - public SecurityContext createContext(SecurityConfiguration securityConfig) { + public SecurityContext createContext(SecurityConfiguration securityConfig) throws SecurityContextInitializeException { return new TestSecurityContext(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/security/contexts/IncompatibleTestSecurityContextFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/security/contexts/IncompatibleTestSecurityContextFactory.java index 08268a5226..a387865ba6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/security/contexts/IncompatibleTestSecurityContextFactory.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/security/contexts/IncompatibleTestSecurityContextFactory.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.security.contexts; import org.apache.flink.runtime.security.SecurityConfiguration; +import org.apache.flink.runtime.security.SecurityContextInitializeException; import java.util.concurrent.Callable; @@ -34,7 +35,7 @@ public class IncompatibleTestSecurityContextFactory implements SecurityContextFa } @Override - public SecurityContext createContext(SecurityConfiguration securityConfig) { + public SecurityContext createContext(SecurityConfiguration securityConfig) throws SecurityContextInitializeException { return new TestSecurityContext(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/security/contexts/TestSecurityContextFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/security/contexts/TestSecurityContextFactory.java index 03c5d63f0e..9896234d49 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/security/contexts/TestSecurityContextFactory.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/security/contexts/TestSecurityContextFactory.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.security.contexts; import org.apache.flink.runtime.security.SecurityConfiguration; +import org.apache.flink.runtime.security.SecurityContextInitializeException; import java.util.concurrent.Callable; @@ -33,7 +34,7 @@ public class TestSecurityContextFactory implements SecurityContextFactory { } @Override - public SecurityContext createContext(SecurityConfiguration securityConfig) { + public SecurityContext createContext(SecurityConfiguration securityConfig) throws SecurityContextInitializeException { return new TestSecurityContext(); } diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java index 80822c20e4..198c87ab5b 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java @@ -22,6 +22,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.SecurityOptions; import org.apache.flink.runtime.security.SecurityConfiguration; import org.apache.flink.runtime.security.SecurityUtils; +import org.apache.flink.runtime.security.contexts.HadoopSecurityContext; import org.apache.flink.test.util.SecureTestEnvironment; import org.apache.flink.test.util.TestingSecurityContext; import org.apache.flink.yarn.configuration.YarnConfigOptions; @@ -88,6 +89,10 @@ public class YARNSessionFIFOSecuredITCase extends YARNSessionFIFOITCase { try { TestingSecurityContext.install(securityConfig, SecureTestEnvironment.getClientSecurityConfigurationMap()); + // This is needed to ensure that SecurityUtils are run within a ugi.doAs section + // Since we already logged in here in @BeforeClass, even a no-op security context will still work. + Assert.assertTrue("HadoopSecurityContext must be installed", + SecurityUtils.getInstalledContext() instanceof HadoopSecurityContext); SecurityUtils.getInstalledContext().runSecured(new Callable() { @Override public Integer call() { diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/util/TestHadoopSecurityContextFactory.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/util/TestHadoopSecurityContextFactory.java index 3d83d4ad73..88c3077a67 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/util/TestHadoopSecurityContextFactory.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/util/TestHadoopSecurityContextFactory.java @@ -19,8 +19,8 @@ package org.apache.flink.yarn.util; import org.apache.flink.runtime.security.SecurityConfiguration; +import org.apache.flink.runtime.security.SecurityContextInitializeException; import org.apache.flink.runtime.security.contexts.HadoopSecurityContext; -import org.apache.flink.runtime.security.contexts.HadoopSecurityContextFactory; import org.apache.flink.runtime.security.contexts.SecurityContext; import org.apache.flink.runtime.security.contexts.SecurityContextFactory; @@ -33,31 +33,16 @@ public class TestHadoopSecurityContextFactory implements SecurityContextFactory @Override public boolean isCompatibleWith(SecurityConfiguration securityConfig) { - if (!securityConfig.getSecurityModuleFactories().contains(TestHadoopModuleFactory.class.getCanonicalName())) { - return false; - } - try { - Class.forName( - "org.apache.hadoop.security.UserGroupInformation", - false, - HadoopSecurityContextFactory.class.getClassLoader()); - return true; - } catch (ClassNotFoundException e) { - return false; - } + return securityConfig.getSecurityModuleFactories().contains(TestHadoopModuleFactory.class.getCanonicalName()); } @Override - public SecurityContext createContext(SecurityConfiguration securityConfig) { + public SecurityContext createContext(SecurityConfiguration securityConfig) throws SecurityContextInitializeException { try { - Class.forName( - "org.apache.hadoop.security.UserGroupInformation", - false, - TestHadoopSecurityContextFactory.class.getClassLoader()); UserGroupInformation loginUser = UserGroupInformation.getLoginUser(); return new HadoopSecurityContext(loginUser); } catch (Exception e) { - return null; + throw new SecurityContextInitializeException("Cannot instantiate test security context", e); } } } -- Gitee From b4828d0e1abfa6679f9f2cc522b5d197a1779325 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 19 Mar 2020 10:46:32 +0100 Subject: [PATCH 213/885] [hotfix] Add log4j2 properties file to flink-yarn-tests --- ...test.properties => log4j2-test.properties} | 32 +++++++------------ 1 file changed, 11 insertions(+), 21 deletions(-) rename flink-yarn-tests/src/test/resources/{log4j-test.properties => log4j2-test.properties} (48%) diff --git a/flink-yarn-tests/src/test/resources/log4j-test.properties b/flink-yarn-tests/src/test/resources/log4j2-test.properties similarity index 48% rename from flink-yarn-tests/src/test/resources/log4j-test.properties rename to flink-yarn-tests/src/test/resources/log4j2-test.properties index 42ae7ddf08..1d2e1708ef 100644 --- a/flink-yarn-tests/src/test/resources/log4j-test.properties +++ b/flink-yarn-tests/src/test/resources/log4j2-test.properties @@ -16,26 +16,16 @@ # limitations under the License. ################################################################################ -log4j.rootLogger=INFO, console +# Set root logger level to OFF to not flood build logs +# set manually to INFO for debugging purposes +rootLogger.level = OFF +rootLogger.appenderRef.test.ref = TestLogger -# Log all infos in the given file -log4j.appender.console=org.apache.log4j.ConsoleAppender -log4j.appender.console.layout=org.apache.log4j.PatternLayout -log4j.appender.console.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n +appender.testlogger.name = TestLogger +appender.testlogger.type = CONSOLE +appender.testlogger.target = SYSTEM_ERR +appender.testlogger.layout.type = PatternLayout +appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n -# suppress the irrelevant (wrong) warnings from the netty channel handler -log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, console - -# log whats going on between the tests -log4j.logger.org.apache.flink.yarn.YARNSessionFIFOITCase=INFO -log4j.logger.org.apache.flink.yarn.YARNSessionCapacitySchedulerITCase=INFO -log4j.logger.org.apache.flink.yarn.YarnHighAvailability=INFO -log4j.logger.org.apache.hadoop=OFF -log4j.logger.org.apache.flink.runtime.leaderelection=INFO -log4j.logger.org.apache.flink.runtime.leaderretrieval=INFO - -log4j.logger.org.apache.directory=OFF -log4j.logger.org.mortbay.log=OFF -log4j.logger.net.sf.ehcache=OFF -log4j.logger.org.apache.hadoop.metrics2=OFF -log4j.logger.org.apache.hadoop.conf.Configuration=OFF +logger.hadoop.name = org.apache.hadoop +logger.hadoop.level = OFF -- Gitee From c0416f2f5043e50602f82797cd6a127fb53ca49f Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Fri, 13 Mar 2020 11:52:22 +0100 Subject: [PATCH 214/885] [FLINK-13000][tests] Remove unused JobID parameters This closes #11404. --- ...StrategyNGAbortPendingCheckpointsTest.java | 2 +- ...egionStrategyNGConcurrentFailoverTest.java | 2 +- .../ExecutionGraphRestartTest.java | 4 ++-- .../ExecutionGraphSuspendTest.java | 6 +----- .../ExecutionGraphTestUtils.java | 20 +++++++------------ .../runtime/executiongraph/ExecutionTest.java | 9 --------- .../ExecutionVertexInputConstraintTest.java | 2 +- .../executiongraph/FinalizeOnMasterTest.java | 9 ++------- .../executiongraph/GlobalModVersionTest.java | 2 +- .../utils/SimpleSlotProvider.java | 9 +++------ ...ToInputsLocationsRetrieverAdapterTest.java | 9 ++++----- .../adapter/DefaultExecutionTopologyTest.java | 3 --- 12 files changed, 23 insertions(+), 54 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AdaptedRestartPipelinedRegionStrategyNGAbortPendingCheckpointsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AdaptedRestartPipelinedRegionStrategyNGAbortPendingCheckpointsTest.java index 18899dbbd3..3f75d90e3d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AdaptedRestartPipelinedRegionStrategyNGAbortPendingCheckpointsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AdaptedRestartPipelinedRegionStrategyNGAbortPendingCheckpointsTest.java @@ -148,7 +148,7 @@ public class AdaptedRestartPipelinedRegionStrategyNGAbortPendingCheckpointsTest .setJobGraph(jobGraph) .setRestartStrategy(new FixedDelayRestartStrategy(10, 0)) .setFailoverStrategyFactory(AdaptedRestartPipelinedRegionStrategyNG::new) - .setSlotProvider(new SimpleSlotProvider(jobGraph.getJobID(), 2, taskManagerGateway)) + .setSlotProvider(new SimpleSlotProvider(2, taskManagerGateway)) .build(); enableCheckpointing(executionGraph); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AdaptedRestartPipelinedRegionStrategyNGConcurrentFailoverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AdaptedRestartPipelinedRegionStrategyNGConcurrentFailoverTest.java index 993effc41e..ace5333e0b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AdaptedRestartPipelinedRegionStrategyNGConcurrentFailoverTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AdaptedRestartPipelinedRegionStrategyNGConcurrentFailoverTest.java @@ -263,7 +263,7 @@ public class AdaptedRestartPipelinedRegionStrategyNGConcurrentFailoverTest exten final JobGraph jg = new JobGraph(TEST_JOB_ID, "testjob", v1, v2); - final SimpleSlotProvider slotProvider = new SimpleSlotProvider(TEST_JOB_ID, DEFAULT_PARALLELISM); + final SimpleSlotProvider slotProvider = new SimpleSlotProvider(DEFAULT_PARALLELISM); final JobMasterPartitionTracker partitionTracker = new JobMasterPartitionTrackerImpl( jg.getJobID(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java index 8e5b1993b9..28eb45c046 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java @@ -101,7 +101,7 @@ public class ExecutionGraphRestartTest extends TestLogger { public void testNoManualRestart() throws Exception { ExecutionGraph eg = TestingExecutionGraphBuilder .newBuilder() - .setSlotProvider(new SimpleSlotProvider(TEST_JOB_ID, NUM_TASKS)) + .setSlotProvider(new SimpleSlotProvider(NUM_TASKS)) .setJobGraph(createJobGraph()) .build(); @@ -558,7 +558,7 @@ public class ExecutionGraphRestartTest extends TestLogger { final int parallelism = 10; final JobVertex vertex = createNoOpVertex(parallelism); final NotCancelAckingTaskGateway taskManagerGateway = new NotCancelAckingTaskGateway(); - final SlotProvider slots = new SimpleSlotProvider(TEST_JOB_ID, parallelism, taskManagerGateway); + final SlotProvider slots = new SimpleSlotProvider(parallelism, taskManagerGateway); final TestRestartStrategy restartStrategy = TestRestartStrategy.manuallyTriggered(); final JobGraph jobGraph = new JobGraph(TEST_JOB_ID, "Test Job", vertex); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSuspendTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSuspendTest.java index 738d77f83f..2eba86bb63 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSuspendTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSuspendTest.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.executiongraph; -import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; import org.apache.flink.runtime.execution.ExecutionState; @@ -284,16 +283,13 @@ public class ExecutionGraphSuspendTest extends TestLogger { } private static ExecutionGraph createExecutionGraph(TaskManagerGateway gateway, int parallelism) throws Exception { - final JobID jobId = new JobID(); - final JobVertex vertex = new JobVertex("vertex"); vertex.setInvokableClass(NoOpInvokable.class); vertex.setParallelism(parallelism); - final SlotProvider slotProvider = new SimpleSlotProvider(jobId, parallelism, gateway); + final SlotProvider slotProvider = new SimpleSlotProvider(parallelism, gateway); ExecutionGraph simpleTestGraph = ExecutionGraphTestUtils.createSimpleTestGraph( - jobId, slotProvider, new FixedDelayRestartStrategy(0, 0), vertex); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java index 2da1cd664f..b318038f33 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.executiongraph; -import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.time.Time; @@ -308,7 +307,7 @@ public class ExecutionGraphTestUtils { public static ExecutionGraph createSimpleTestGraph(RestartStrategy restartStrategy) throws Exception { JobVertex vertex = createNoOpVertex(10); - return createSimpleTestGraph(new JobID(), new SimpleAckingTaskManagerGateway(), restartStrategy, vertex); + return createSimpleTestGraph(new SimpleAckingTaskManagerGateway(), restartStrategy, vertex); } /** @@ -316,15 +315,14 @@ public class ExecutionGraphTestUtils { * *

    The execution graph uses {@link NoRestartStrategy} as the restart strategy. */ - public static ExecutionGraph createSimpleTestGraph(JobID jid, JobVertex... vertices) throws Exception { - return createSimpleTestGraph(jid, new SimpleAckingTaskManagerGateway(), new NoRestartStrategy(), vertices); + public static ExecutionGraph createSimpleTestGraph(JobVertex... vertices) throws Exception { + return createSimpleTestGraph(new SimpleAckingTaskManagerGateway(), new NoRestartStrategy(), vertices); } /** * Creates an execution graph containing the given vertices and the given restart strategy. */ public static ExecutionGraph createSimpleTestGraph( - JobID jid, TaskManagerGateway taskManagerGateway, RestartStrategy restartStrategy, JobVertex... vertices) throws Exception { @@ -334,39 +332,35 @@ public class ExecutionGraphTestUtils { numSlotsNeeded += vertex.getParallelism(); } - SlotProvider slotProvider = new SimpleSlotProvider(jid, numSlotsNeeded, taskManagerGateway); + SlotProvider slotProvider = new SimpleSlotProvider(numSlotsNeeded, taskManagerGateway); - return createSimpleTestGraph(jid, slotProvider, restartStrategy, vertices); + return createSimpleTestGraph(slotProvider, restartStrategy, vertices); } public static ExecutionGraph createSimpleTestGraph( - JobID jid, SlotProvider slotProvider, RestartStrategy restartStrategy, JobVertex... vertices) throws Exception { - return createExecutionGraph(jid, slotProvider, restartStrategy, TestingUtils.defaultExecutor(), vertices); + return createExecutionGraph(slotProvider, restartStrategy, TestingUtils.defaultExecutor(), vertices); } public static ExecutionGraph createExecutionGraph( - JobID jid, SlotProvider slotProvider, RestartStrategy restartStrategy, ScheduledExecutorService executor, JobVertex... vertices) throws Exception { - return createExecutionGraph(jid, slotProvider, restartStrategy, executor, Time.seconds(10L), vertices); + return createExecutionGraph(slotProvider, restartStrategy, executor, Time.seconds(10L), vertices); } public static ExecutionGraph createExecutionGraph( - JobID jid, SlotProvider slotProvider, RestartStrategy restartStrategy, ScheduledExecutorService executor, Time timeout, JobVertex... vertices) throws Exception { - checkNotNull(jid); checkNotNull(restartStrategy); checkNotNull(vertices); checkNotNull(timeout); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionTest.java index d939429dbe..0cee31e321 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionTest.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.executiongraph; -import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.checkpoint.JobManagerTaskRestore; import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; @@ -101,7 +100,6 @@ public class ExecutionTest extends TestLogger { slotProvider.addSlot(jobVertexId, 0, slotFuture); ExecutionGraph executionGraph = ExecutionGraphTestUtils.createSimpleTestGraph( - new JobID(), slotProvider, new NoRestartStrategy(), jobVertex); @@ -159,7 +157,6 @@ public class ExecutionTest extends TestLogger { slotProvider.addSlot(jobVertexId, 0, CompletableFuture.completedFuture(slot)); ExecutionGraph executionGraph = ExecutionGraphTestUtils.createSimpleTestGraph( - new JobID(), slotProvider, new NoRestartStrategy(), jobVertex); @@ -205,7 +202,6 @@ public class ExecutionTest extends TestLogger { slotProvider.addSlot(jobVertexId, 0, CompletableFuture.completedFuture(slot)); ExecutionGraph executionGraph = ExecutionGraphTestUtils.createSimpleTestGraph( - new JobID(), slotProvider, new NoRestartStrategy(), jobVertex); @@ -253,7 +249,6 @@ public class ExecutionTest extends TestLogger { slotProvider.addSlot(jobVertexId, 0, slotFuture); final ExecutionGraph executionGraph = ExecutionGraphTestUtils.createSimpleTestGraph( - new JobID(), slotProvider, new NoRestartStrategy(), jobVertex); @@ -361,7 +356,6 @@ public class ExecutionTest extends TestLogger { slotOwner); ExecutionGraph executionGraph = ExecutionGraphTestUtils.createSimpleTestGraph( - new JobID(), slotProvider, new NoRestartStrategy(), jobVertex); @@ -410,7 +404,6 @@ public class ExecutionTest extends TestLogger { slotOwner); ExecutionGraph executionGraph = ExecutionGraphTestUtils.createSimpleTestGraph( - new JobID(), slotProvider, new NoRestartStrategy(), jobVertex); @@ -459,7 +452,6 @@ public class ExecutionTest extends TestLogger { (LogicalSlot logicalSlot) -> returnedSlotFuture.complete(logicalSlot.getSlotRequestId())); ExecutionGraph executionGraph = ExecutionGraphTestUtils.createSimpleTestGraph( - new JobID(), slotProvider, new NoRestartStrategy(), jobVertex); @@ -526,7 +518,6 @@ public class ExecutionTest extends TestLogger { return slotFuture; }); final ExecutionGraph executionGraph = ExecutionGraphTestUtils.createSimpleTestGraph( - new JobID(), slotProvider, new NoRestartStrategy(), jobVertex); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexInputConstraintTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexInputConstraintTest.java index 77728332b1..f61653c2c9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexInputConstraintTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexInputConstraintTest.java @@ -247,7 +247,7 @@ public class ExecutionVertexInputConstraintTest extends TestLogger { } final JobGraph jobGraph = new JobGraph(orderedVertices.toArray(new JobVertex[0])); - final SlotProvider slotProvider = new SimpleSlotProvider(jobGraph.getJobID(), numSlots); + final SlotProvider slotProvider = new SimpleSlotProvider(numSlots); return TestingExecutionGraphBuilder .newBuilder() diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/FinalizeOnMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/FinalizeOnMasterTest.java index b39d336e2c..06c0f7417d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/FinalizeOnMasterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/FinalizeOnMasterTest.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.executiongraph; -import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; import org.apache.flink.runtime.jobgraph.JobVertex; @@ -42,8 +41,6 @@ public class FinalizeOnMasterTest extends TestLogger { @Test public void testFinalizeIsCalledUponSuccess() throws Exception { - final JobID jid = new JobID(); - final JobVertex vertex1 = spy(new JobVertex("test vertex 1")); vertex1.setInvokableClass(NoOpInvokable.class); vertex1.setParallelism(3); @@ -52,7 +49,7 @@ public class FinalizeOnMasterTest extends TestLogger { vertex2.setInvokableClass(NoOpInvokable.class); vertex2.setParallelism(2); - final ExecutionGraph eg = createSimpleTestGraph(jid, vertex1, vertex2); + final ExecutionGraph eg = createSimpleTestGraph(vertex1, vertex2); eg.start(ComponentMainThreadExecutorServiceAdapter.forMainThread()); eg.scheduleForExecution(); assertEquals(JobStatus.RUNNING, eg.getState()); @@ -71,13 +68,11 @@ public class FinalizeOnMasterTest extends TestLogger { @Test public void testFinalizeIsNotCalledUponFailure() throws Exception { - final JobID jid = new JobID(); - final JobVertex vertex = spy(new JobVertex("test vertex 1")); vertex.setInvokableClass(NoOpInvokable.class); vertex.setParallelism(1); - final ExecutionGraph eg = createSimpleTestGraph(jid, vertex); + final ExecutionGraph eg = createSimpleTestGraph(vertex); eg.start(ComponentMainThreadExecutorServiceAdapter.forMainThread()); eg.scheduleForExecution(); assertEquals(JobStatus.RUNNING, eg.getState()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/GlobalModVersionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/GlobalModVersionTest.java index bb07813355..a1d736d6e3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/GlobalModVersionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/GlobalModVersionTest.java @@ -161,7 +161,7 @@ public class GlobalModVersionTest extends TestLogger { JobGraph jg = new JobGraph(jid, "testjob", jv); - final SimpleSlotProvider slotProvider = new SimpleSlotProvider(jid, parallelism); + final SimpleSlotProvider slotProvider = new SimpleSlotProvider(parallelism); // build a simple execution graph with on job vertex, parallelism 2 final ExecutionGraph graph = TestingExecutionGraphBuilder diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleSlotProvider.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleSlotProvider.java index 09dc90e6de..b4a698bc8f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleSlotProvider.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleSlotProvider.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.executiongraph.utils; -import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; @@ -48,7 +47,6 @@ import java.util.HashMap; import java.util.concurrent.CompletableFuture; import static org.apache.flink.util.Preconditions.checkArgument; -import static org.apache.flink.util.Preconditions.checkNotNull; /** * A testing utility slot provider that simply has a predefined pool of slots. @@ -61,12 +59,11 @@ public class SimpleSlotProvider implements SlotProvider, SlotOwner { private final HashMap allocatedSlots; - public SimpleSlotProvider(JobID jobId, int numSlots) { - this(jobId, numSlots, new SimpleAckingTaskManagerGateway()); + public SimpleSlotProvider(int numSlots) { + this(numSlots, new SimpleAckingTaskManagerGateway()); } - public SimpleSlotProvider(JobID jobId, int numSlots, TaskManagerGateway taskManagerGateway) { - checkNotNull(jobId, "jobId"); + public SimpleSlotProvider(int numSlots, TaskManagerGateway taskManagerGateway) { checkArgument(numSlots >= 0, "numSlots must be >= 0"); this.slots = new ArrayDeque<>(numSlots); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/ExecutionGraphToInputsLocationsRetrieverAdapterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/ExecutionGraphToInputsLocationsRetrieverAdapterTest.java index d9eae6bb57..ceb9911b9b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/ExecutionGraphToInputsLocationsRetrieverAdapterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/ExecutionGraphToInputsLocationsRetrieverAdapterTest.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.scheduler; -import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.executiongraph.ExecutionGraph; import org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils; import org.apache.flink.runtime.executiongraph.ExecutionVertex; @@ -64,7 +63,7 @@ public class ExecutionGraphToInputsLocationsRetrieverAdapterTest extends TestLog consumer.connectNewDataSetAsInput(producer1, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); consumer.connectNewDataSetAsInput(producer2, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); - final ExecutionGraph eg = ExecutionGraphTestUtils.createSimpleTestGraph(new JobID(), producer1, producer2, consumer); + final ExecutionGraph eg = ExecutionGraphTestUtils.createSimpleTestGraph(producer1, producer2, consumer); final ExecutionGraphToInputsLocationsRetrieverAdapter inputsLocationsRetriever = new ExecutionGraphToInputsLocationsRetrieverAdapter(eg); @@ -93,7 +92,7 @@ public class ExecutionGraphToInputsLocationsRetrieverAdapterTest extends TestLog public void testGetEmptyTaskManagerLocationIfVertexNotScheduled() throws Exception { final JobVertex jobVertex = ExecutionGraphTestUtils.createNoOpVertex(1); - final ExecutionGraph eg = ExecutionGraphTestUtils.createSimpleTestGraph(new JobID(), jobVertex); + final ExecutionGraph eg = ExecutionGraphTestUtils.createSimpleTestGraph(jobVertex); final ExecutionGraphToInputsLocationsRetrieverAdapter inputsLocationsRetriever = new ExecutionGraphToInputsLocationsRetrieverAdapter(eg); @@ -112,7 +111,7 @@ public class ExecutionGraphToInputsLocationsRetrieverAdapterTest extends TestLog final JobVertex jobVertex = ExecutionGraphTestUtils.createNoOpVertex(1); final TestingLogicalSlot testingLogicalSlot = new TestingLogicalSlotBuilder().createTestingLogicalSlot(); - final ExecutionGraph eg = ExecutionGraphTestUtils.createSimpleTestGraph(new JobID(), jobVertex); + final ExecutionGraph eg = ExecutionGraphTestUtils.createSimpleTestGraph(jobVertex); final ExecutionGraphToInputsLocationsRetrieverAdapter inputsLocationsRetriever = new ExecutionGraphToInputsLocationsRetrieverAdapter(eg); @@ -136,7 +135,7 @@ public class ExecutionGraphToInputsLocationsRetrieverAdapterTest extends TestLog public void testGetNonExistingExecutionVertexWillThrowException() throws Exception { final JobVertex jobVertex = ExecutionGraphTestUtils.createNoOpVertex(1); - final ExecutionGraph eg = ExecutionGraphTestUtils.createSimpleTestGraph(new JobID(), jobVertex); + final ExecutionGraph eg = ExecutionGraphTestUtils.createSimpleTestGraph(jobVertex); final ExecutionGraphToInputsLocationsRetrieverAdapter inputsLocationsRetriever = new ExecutionGraphToInputsLocationsRetrieverAdapter(eg); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/DefaultExecutionTopologyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/DefaultExecutionTopologyTest.java index cb58c68b69..9323b597d4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/DefaultExecutionTopologyTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/DefaultExecutionTopologyTest.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.scheduler.adapter; -import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.executiongraph.ExecutionEdge; import org.apache.flink.runtime.executiongraph.ExecutionGraph; import org.apache.flink.runtime.executiongraph.ExecutionVertex; @@ -82,7 +81,6 @@ public class DefaultExecutionTopologyTest extends TestLogger { jobVertices[0].setInputDependencyConstraint(ALL); jobVertices[1].setInputDependencyConstraint(ANY); executionGraph = createSimpleTestGraph( - new JobID(), taskManagerGateway, triggeredRestartStrategy, jobVertices); @@ -176,7 +174,6 @@ public class DefaultExecutionTopologyTest extends TestLogger { jobVertices[1].updateCoLocationGroup(coLocationGroup); return createSimpleTestGraph( - new JobID(), taskManagerGateway, triggeredRestartStrategy, jobVertices); -- Gitee From b9943dc47761ef8ad2c620350125351309047052 Mon Sep 17 00:00:00 2001 From: hequn8128 Date: Thu, 19 Mar 2020 17:11:38 +0800 Subject: [PATCH 215/885] [FLINK-16676][python][test] Fix ValidationPipelineTest.test_pipeline_from_invalid_json failure in Azure --- flink-python/pyflink/ml/tests/test_pipeline.py | 16 +++++----------- 1 file changed, 5 insertions(+), 11 deletions(-) diff --git a/flink-python/pyflink/ml/tests/test_pipeline.py b/flink-python/pyflink/ml/tests/test_pipeline.py index 19707fc2c8..6167183928 100644 --- a/flink-python/pyflink/ml/tests/test_pipeline.py +++ b/flink-python/pyflink/ml/tests/test_pipeline.py @@ -84,20 +84,14 @@ class ValidationPipelineTest(MLTestCase): p.load_json(invalid_json) exception_str = str(context.exception) - # only assert key error message as the whole message is very long. + # NOTE: only check the general error message since the detailed error message + # would be different in different environment. self.assertTrue( 'Cannot load the JSON as either a Java Pipeline or a Python Pipeline.' in exception_str) - self.assertTrue( - 'Python Pipeline load failed due to: Expecting value: line 1 column 2 (char 1).' - in exception_str) - self.assertTrue( - 'Java Pipeline load failed due to: An error occurred while calling o0.loadJson.' - in exception_str) - self.assertTrue( - 'Caused by: org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.' - 'JsonParseException: Unrecognized token \'a\'' - in exception_str) + self.assertTrue('Python Pipeline load failed due to:' in exception_str) + self.assertTrue('Java Pipeline load failed due to:' in exception_str) + self.assertTrue('JsonParseException' in exception_str) class SelfDescribe(WithParams): -- Gitee From 75b240d49c0944818a3de37af07040432958ad5b Mon Sep 17 00:00:00 2001 From: Niels Basjes Date: Wed, 26 Feb 2020 18:27:13 +0100 Subject: [PATCH 216/885] [hotfix][docs] Fix broken links --- docs/ops/plugins.md | 2 +- docs/ops/plugins.zh.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ops/plugins.md b/docs/ops/plugins.md index 759d5dcf35..ea203ff418 100644 --- a/docs/ops/plugins.md +++ b/docs/ops/plugins.md @@ -70,7 +70,7 @@ possible across Flink core, plugins, and user code. ## File Systems -All [file systems](filesystems/index) **except MapR** are pluggable. That means they can and should +All [file systems](filesystems) **except MapR** are pluggable. That means they can and should be used as plugins. To use a pluggable file system, copy the corresponding JAR file from the `opt` directory to a directory under `plugins` directory of your Flink distribution before starting Flink, e.g. diff --git a/docs/ops/plugins.zh.md b/docs/ops/plugins.zh.md index 759d5dcf35..ea203ff418 100644 --- a/docs/ops/plugins.zh.md +++ b/docs/ops/plugins.zh.md @@ -70,7 +70,7 @@ possible across Flink core, plugins, and user code. ## File Systems -All [file systems](filesystems/index) **except MapR** are pluggable. That means they can and should +All [file systems](filesystems) **except MapR** are pluggable. That means they can and should be used as plugins. To use a pluggable file system, copy the corresponding JAR file from the `opt` directory to a directory under `plugins` directory of your Flink distribution before starting Flink, e.g. -- Gitee From b237b0081473016aff558f6c1393357d9409d1e3 Mon Sep 17 00:00:00 2001 From: Niels Basjes Date: Wed, 4 Mar 2020 10:29:16 +0100 Subject: [PATCH 217/885] [FLINK-16293] Document using plugins in Docker and Kubernetes This closes #11228. --- docs/ops/deployment/docker.md | 31 +++++++++++++++++++++ docs/ops/deployment/docker.zh.md | 31 +++++++++++++++++++++ docs/ops/deployment/kubernetes.md | 25 +++++++++++++---- docs/ops/deployment/kubernetes.zh.md | 13 +++++++++ docs/ops/deployment/native_kubernetes.md | 26 +++++++++++++++++ docs/ops/deployment/native_kubernetes.zh.md | 26 +++++++++++++++++ 6 files changed, 146 insertions(+), 6 deletions(-) diff --git a/docs/ops/deployment/docker.md b/docs/ops/deployment/docker.md index 0c86074c69..a716206461 100644 --- a/docs/ops/deployment/docker.md +++ b/docs/ops/deployment/docker.md @@ -68,6 +68,37 @@ Therefore, one needs to build a dedicated container image for every job. The `flink-container` module contains a `build.sh` script which can be used to create such an image. Please see the [instructions](https://github.com/apache/flink/blob/{{ site.github_branch }}/flink-container/docker/README.md) for more details. +## Using plugins +As described in the [plugins]({{ site.baseurl }}/ops/plugins.html) documentation page: in order to use plugins they must be +copied to the correct location in the Flink installation for them to work. + +When running Flink from one of the provided Docker images by default no plugins have been activated. +The simplest way to enable plugins is to modify the provided official Flink docker images by adding +an additional layer. This does however assume you have a docker registry available where you can push images to and +that is accessible by your cluster. + +As an example assume you want to enable the [S3]({{ site.baseurl }}/ops/filesystems/s3.html) plugins in your installation. + +Create a Dockerfile with a content something like this: {% highlight dockerfile %} +# On which specific version of Flink is this based? +# Check https://hub.docker.com/_/flink?tab=tags for current options +FROM flink:{{ site.version }}-scala_2.12 + +# Install Flink S3 FS Presto plugin +RUN mkdir /opt/flink/plugins/s3-fs-presto && cp /opt/flink/opt/flink-s3-fs-presto* /opt/flink/plugins/s3-fs-presto + +# Install Flink S3 FS Hadoop plugin +RUN mkdir /opt/flink/plugins/s3-fs-hadoop && cp /opt/flink/opt/flink-s3-fs-hadoop* /opt/flink/plugins/s3-fs-hadoop +{% endhighlight %} + +Then build and push that image to your registry +{% highlight bash %} +docker build -t docker.example.nl/flink:{{ site.version }}-scala_2.12-s3 . +docker push docker.example.nl/flink:{{ site.version }}-scala_2.12-s3 +{% endhighlight %} + +Now you can reference this image in your cluster deployment and the installed plugins are available for use. + ## Flink with Docker Compose [Docker Compose](https://docs.docker.com/compose/) is a convenient way to run a diff --git a/docs/ops/deployment/docker.zh.md b/docs/ops/deployment/docker.zh.md index d82d736d86..69ac45e797 100644 --- a/docs/ops/deployment/docker.zh.md +++ b/docs/ops/deployment/docker.zh.md @@ -68,6 +68,37 @@ Therefore, one needs to build a dedicated container image for every job. The `flink-container` module contains a `build.sh` script which can be used to create such an image. Please see the [instructions](https://github.com/apache/flink/blob/{{ site.github_branch }}/flink-container/docker/README.md) for more details. +## Using plugins +As described in the [plugins]({{ site.baseurl }}/zh/ops/plugins.html) documentation page: in order to use plugins they must be +copied to the correct location in the Flink installation for them to work. + +When running Flink from one of the provided Docker images by default no plugins have been activated. +The simplest way to enable plugins is to modify the provided official Flink docker images by adding +an additional layer. This does however assume you have a docker registry available where you can push images to and +that is accessible by your cluster. + +As an example assume you want to enable the [S3]({{ site.baseurl }}/zh/ops/filesystems/s3.html) plugins in your installation. + +Create a Dockerfile with a content something like this: {% highlight dockerfile %} +# On which specific version of Flink is this based? +# Check https://hub.docker.com/_/flink?tab=tags for current options +FROM flink:{{ site.version }}-scala_2.12 + +# Install Flink S3 FS Presto plugin +RUN mkdir /opt/flink/plugins/s3-fs-presto && cp /opt/flink/opt/flink-s3-fs-presto* /opt/flink/plugins/s3-fs-presto + +# Install Flink S3 FS Hadoop plugin +RUN mkdir /opt/flink/plugins/s3-fs-hadoop && cp /opt/flink/opt/flink-s3-fs-hadoop* /opt/flink/plugins/s3-fs-hadoop +{% endhighlight %} + +Then build and push that image to your registry +{% highlight bash %} +docker build -t docker.example.nl/flink:{{ site.version }}-scala_2.12-s3 . +docker push docker.example.nl/flink:{{ site.version }}-scala_2.12-s3 +{% endhighlight %} + +Now you can reference this image in your cluster deployment and then these plugins are available for use. + ## Flink with Docker Compose [Docker Compose](https://docs.docker.com/compose/) is a convenient way to run a diff --git a/docs/ops/deployment/kubernetes.md b/docs/ops/deployment/kubernetes.md index f30f21a228..d798bfff85 100644 --- a/docs/ops/deployment/kubernetes.md +++ b/docs/ops/deployment/kubernetes.md @@ -36,13 +36,13 @@ Please follow [Kubernetes' setup guide](https://kubernetes.io/docs/setup/) in or If you want to run Kubernetes locally, we recommend using [MiniKube](https://kubernetes.io/docs/setup/minikube/).

    - Note: If using MiniKube please make sure to execute `minikube ssh 'sudo ip link set docker0 promisc on'` before deploying a Flink cluster. - Otherwise Flink components are not able to self reference themselves through a Kubernetes service. + Note: If using MiniKube please make sure to execute `minikube ssh 'sudo ip link set docker0 promisc on'` before deploying a Flink cluster. + Otherwise Flink components are not able to self reference themselves through a Kubernetes service.
    ## Flink session cluster on Kubernetes -A Flink session cluster is executed as a long-running Kubernetes Deployment. +A Flink session cluster is executed as a long-running Kubernetes Deployment. Note that you can run multiple Flink jobs on a session cluster. Each job needs to be submitted to the cluster after the cluster has been deployed. @@ -91,15 +91,28 @@ In order to terminate the Flink session cluster, use `kubectl`: ## Flink job cluster on Kubernetes -A Flink job cluster is a dedicated cluster which runs a single job. -The job is part of the image and, thus, there is no extra job submission needed. +A Flink job cluster is a dedicated cluster which runs a single job. +The job is part of the image and, thus, there is no extra job submission needed. ### Creating the job-specific image The Flink job cluster image needs to contain the user code jars of the job for which the cluster is started. Therefore, one needs to build a dedicated container image for every job. Please follow these [instructions](https://github.com/apache/flink/blob/{{ site.github_branch }}/flink-container/docker/README.md) to build the Docker image. - + +### Using plugins + +As described in the [plugins]({{ site.baseurl }}/ops/plugins.html) documentation page: in order to use plugins they must be +copied to the correct location in the flink installation for them to work. + +The simplest way to enable plugins for use on Kubernetes is to modify the provided Flink docker image by adding +an additional layer. This does however assume you have a docker registry available where you can push images to and +that is accessible by your Kubernetes cluster. + +How this can be done is described on the [Docker Setup]({{ site.baseurl }}/zh/ops/deployment/docker.html#using-plugins) page. + +With such an image created you can now start your Kubernetes based Flink cluster which can use the enabled plugins. + ### Deploy Flink job cluster on Kubernetes In order to deploy the a job cluster on Kubernetes please follow these [instructions](https://github.com/apache/flink/blob/{{ site.github_branch }}/flink-container/kubernetes/README.md#deploy-flink-job-cluster). diff --git a/docs/ops/deployment/kubernetes.zh.md b/docs/ops/deployment/kubernetes.zh.md index 014c9372a8..f5dd31a15b 100644 --- a/docs/ops/deployment/kubernetes.zh.md +++ b/docs/ops/deployment/kubernetes.zh.md @@ -100,6 +100,19 @@ The Flink job cluster image needs to contain the user code jars of the job for w Therefore, one needs to build a dedicated container image for every job. Please follow these [instructions](https://github.com/apache/flink/blob/{{ site.github_branch }}/flink-container/docker/README.md) to build the Docker image. +### Using plugins + +As described in the [plugins]({{ site.baseurl }}/zh/ops/plugins.html) documentation page: in order to use plugins they must be +copied to the correct location in the flink installation for them to work. + +The simplest way to enable plugins for use on Kubernetes is to modify the provided Flink docker image by adding +an additional layer. This does however assume you have a docker registry available where you can push images to and +that is accessible by your Kubernetes cluster. + +How this can be done is described on the [Docker Setup]({{ site.baseurl }}/zh/ops/deployment/docker.html#using-plugins) page. + +With such an image created you can now start your Kubernetes based Flink cluster which can use the enabled plugins. + ### Deploy Flink job cluster on Kubernetes In order to deploy the a job cluster on Kubernetes please follow these [instructions](https://github.com/apache/flink/blob/{{ site.github_branch }}/flink-container/kubernetes/README.md#deploy-flink-job-cluster). diff --git a/docs/ops/deployment/native_kubernetes.md b/docs/ops/deployment/native_kubernetes.md index d543609222..b3806113ec 100644 --- a/docs/ops/deployment/native_kubernetes.md +++ b/docs/ops/deployment/native_kubernetes.md @@ -154,6 +154,32 @@ appender.console.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %-60c %x - %m If the pod is running, you can use `kubectl exec -it bash` to tunnel in and view the logs or debug the process. +## Using plugins + +As described in the [plugins]({{ site.baseurl }}/ops/plugins.html) documentation page: in order to use plugins they must be +copied to the correct location in the flink installation for them to work. + +The simplest way to enable plugins for use on Kubernetes is to modify the provided official Flink docker images by adding +an additional layer. This does however assume you have a docker registry available where you can push images to and +that is accessible by your Kubernetes cluster. + +How this can be done is described on the [Docker Setup]({{ site.baseurl }}/ops/deployment/docker.html#using-plugins) page. + +With such an image created you can now start your Kubernetes based Flink session cluster with the additional parameter +`kubernetes.container.image` which must specify the image that was created: `docker.example.nl/flink:{{ site.version }}-2.12-s3` + +Extending the above example command to start the session cluster makes it this: + +{% highlight bash %} +./bin/kubernetes-session.sh \ + -Dkubernetes.cluster-id= \ + -Dtaskmanager.memory.process.size=4096m \ + -Dkubernetes.taskmanager.cpu=2 \ + -Dtaskmanager.numberOfTaskSlots=4 \ + -Dresourcemanager.taskmanager-timeout=3600000 \ + -Dkubernetes.container.image=docker.example.nl/flink:{{ site.version }}-2.12-s3 +{% endhighlight %} + ## Kubernetes concepts ### Namespaces diff --git a/docs/ops/deployment/native_kubernetes.zh.md b/docs/ops/deployment/native_kubernetes.zh.md index 1337dc0c86..b9c328d84d 100644 --- a/docs/ops/deployment/native_kubernetes.zh.md +++ b/docs/ops/deployment/native_kubernetes.zh.md @@ -154,6 +154,32 @@ appender.console.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %-60c %x - %m If the pod is running, you can use `kubectl exec -it bash` to tunnel in and view the logs or debug the process. +## Using plugins + +As described in the [plugins]({{ site.baseurl }}/zh/ops/plugins.html) documentation page: in order to use plugins they must be +copied to the correct location in the flink installation for them to work. + +The simplest way to enable plugins for use on Kubernetes is to modify the provided official Flink docker images by adding +an additional layer. This does however assume you have a docker registry available where you can push images to and +that is accessible by your Kubernetes cluster. + +How this can be done is described on the [Docker Setup]({{ site.baseurl }}/zh/ops/deployment/docker.html#using-plugins) page. + +With such an image created you can now start your Kubernetes based Flink session cluster with the additional parameter +`kubernetes.container.image` which must specify the image that was created: `docker.example.nl/flink:{{ site.version }}-2.12-s3` + +Extending the above example command to start the session cluster makes it this: + +{% highlight bash %} +./bin/kubernetes-session.sh \ + -Dkubernetes.cluster-id= \ + -Dtaskmanager.memory.process.size=4096m \ + -Dkubernetes.taskmanager.cpu=2 \ + -Dtaskmanager.numberOfTaskSlots=4 \ + -Dresourcemanager.taskmanager-timeout=3600000 \ + -Dkubernetes.container.image=docker.example.nl/flink:{{ site.version }}-2.12-s3 +{% endhighlight %} + ## Kubernetes concepts ### Namespaces -- Gitee From 6ee5dbabfa4e0efa7ac8f1f8e03a75c303ffc622 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Tue, 17 Mar 2020 21:23:22 +0100 Subject: [PATCH 218/885] [FLINK-15852][cli] Prioritize ExecutorCLI over YarnSessionCLI for active CLI This closes #11435. --- .../apache/flink/client/cli/CliFrontend.java | 26 +++++------- .../flink/yarn/FlinkYarnSessionCliTest.java | 42 +++++++++++++++++++ 2 files changed, 52 insertions(+), 16 deletions(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java index f688c42bb0..c38c49c639 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java @@ -174,10 +174,7 @@ public class CliFrontend { LOG.info("Running 'run' command."); final Options commandOptions = CliFrontendParser.getRunCommandOptions(); - - final Options commandLineOptions = CliFrontendParser.mergeOptions(commandOptions, customCommandLineOptions); - - final CommandLine commandLine = CliFrontendParser.parse(commandLineOptions, args, true); + final CommandLine commandLine = getCommandLine(commandOptions, args, true); final ProgramOptions programOptions = new ProgramOptions(commandLine); @@ -304,10 +301,7 @@ public class CliFrontend { LOG.info("Running 'list' command."); final Options commandOptions = CliFrontendParser.getListCommandOptions(); - - final Options commandLineOptions = CliFrontendParser.mergeOptions(commandOptions, customCommandLineOptions); - - final CommandLine commandLine = CliFrontendParser.parse(commandLineOptions, args, false); + final CommandLine commandLine = getCommandLine(commandOptions, args, false); ListOptions listOptions = new ListOptions(commandLine); @@ -427,8 +421,7 @@ public class CliFrontend { LOG.info("Running 'stop-with-savepoint' command."); final Options commandOptions = CliFrontendParser.getStopCommandOptions(); - final Options commandLineOptions = CliFrontendParser.mergeOptions(commandOptions, customCommandLineOptions); - final CommandLine commandLine = CliFrontendParser.parse(commandLineOptions, args, false); + final CommandLine commandLine = getCommandLine(commandOptions, args, false); final StopOptions stopOptions = new StopOptions(commandLine); if (stopOptions.isPrintHelp()) { @@ -474,10 +467,7 @@ public class CliFrontend { LOG.info("Running 'cancel' command."); final Options commandOptions = CliFrontendParser.getCancelCommandOptions(); - - final Options commandLineOptions = CliFrontendParser.mergeOptions(commandOptions, customCommandLineOptions); - - final CommandLine commandLine = CliFrontendParser.parse(commandLineOptions, args, false); + final CommandLine commandLine = getCommandLine(commandOptions, args, false); CancelOptions cancelOptions = new CancelOptions(commandLine); @@ -550,6 +540,11 @@ public class CliFrontend { } } + public CommandLine getCommandLine(final Options commandOptions, final String[] args, final boolean stopAtNonOptions) throws CliArgsException { + final Options commandLineOptions = CliFrontendParser.mergeOptions(commandOptions, customCommandLineOptions); + return CliFrontendParser.parse(commandLineOptions, args, stopAtNonOptions); + } + /** * Executes the SAVEPOINT action. * @@ -1021,6 +1016,7 @@ public class CliFrontend { public static List loadCustomCommandLines(Configuration configuration, String configurationDirectory) { List customCommandLines = new ArrayList<>(); + customCommandLines.add(new ExecutorCLI(configuration)); // Command line interface of the YARN session, with a special initialization here // to prefix all options with y/yarn. @@ -1036,8 +1032,6 @@ public class CliFrontend { LOG.warn("Could not load CLI class {}.", flinkYarnSessionCLI, e); } - customCommandLines.add(new ExecutorCLI(configuration)); - // Tips: DefaultCLI must be added at last, because getActiveCustomCommandLine(..) will get the // active CustomCommandLine in order and DefaultCLI isActive always return true. customCommandLines.add(new DefaultCLI(configuration)); diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java index 9a4299c8eb..a3831d6122 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java @@ -18,6 +18,11 @@ package org.apache.flink.yarn; +import org.apache.flink.client.cli.CliArgsException; +import org.apache.flink.client.cli.CliFrontend; +import org.apache.flink.client.cli.CliFrontendParser; +import org.apache.flink.client.cli.CustomCommandLine; +import org.apache.flink.client.cli.ExecutorCLI; import org.apache.flink.client.deployment.ClusterClientFactory; import org.apache.flink.client.deployment.ClusterClientServiceLoader; import org.apache.flink.client.deployment.ClusterSpecification; @@ -35,6 +40,7 @@ import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; import org.apache.flink.yarn.cli.FlinkYarnSessionCli; import org.apache.flink.yarn.configuration.YarnConfigOptions; +import org.apache.flink.yarn.executors.YarnJobClusterExecutor; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.CommandLineParser; @@ -49,6 +55,7 @@ import java.io.File; import java.io.IOException; import java.nio.file.Files; import java.nio.file.StandardOpenOption; +import java.util.List; import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertEquals; @@ -161,6 +168,41 @@ public class FlinkYarnSessionCliTest extends TestLogger { assertEquals(nodeLabelCliInput, descriptor.getNodeLabel()); } + @Test + public void testExecutorCLIisPrioritised() throws Exception { + final File directoryPath = writeYarnPropertiesFile(validPropertiesFile); + + final Configuration configuration = new Configuration(); + configuration.setString(YarnConfigOptions.PROPERTIES_FILE_LOCATION, directoryPath.getAbsolutePath()); + + validateYarnCLIisActive(configuration); + + final String[] argsUnderTest = new String[] {"-e", YarnJobClusterExecutor.NAME}; + + validateExecutorCLIisPrioritised(configuration, argsUnderTest); + } + + private void validateExecutorCLIisPrioritised(Configuration configuration, String[] argsUnderTest) throws IOException, CliArgsException { + final List customCommandLines = CliFrontend.loadCustomCommandLines( + configuration, + tmp.newFile().getAbsolutePath()); + + final CliFrontend cli = new CliFrontend(configuration, customCommandLines); + final CommandLine commandLine = cli.getCommandLine( + CliFrontendParser.getRunCommandOptions(), + argsUnderTest, + true); + + final CustomCommandLine customCommandLine = cli.getActiveCustomCommandLine(commandLine); + assertTrue(customCommandLine instanceof ExecutorCLI); + } + + private void validateYarnCLIisActive(Configuration configuration) throws FlinkException, CliArgsException { + final FlinkYarnSessionCli flinkYarnSessionCli = createFlinkYarnSessionCli(configuration); + final CommandLine testCLIArgs = flinkYarnSessionCli.parseCommandLineOptions(new String[] {}, true); + assertTrue(flinkYarnSessionCli.isActive(testCLIArgs)); + } + /** * Test that the CliFrontend is able to pick up the .yarn-properties file from a specified location. */ -- Gitee From 0b9fac1b676ecc050e5d505e9e80384b7fca599b Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 19 Mar 2020 14:59:54 +0100 Subject: [PATCH 219/885] [hotfix] Make TestBulkWriterFactory publicly available for other test modules --- .../streaming/api/functions/sink/filesystem/BulkWriterTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BulkWriterTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BulkWriterTest.java index 0449e1d247..e32d46219f 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BulkWriterTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BulkWriterTest.java @@ -236,7 +236,7 @@ public class BulkWriterTest extends TestLogger { /** * A {@link BulkWriter.Factory} used for the tests. */ - private static class TestBulkWriterFactory implements BulkWriter.Factory> { + public static final class TestBulkWriterFactory implements BulkWriter.Factory> { private static final long serialVersionUID = 1L; -- Gitee From 067c044cca59f87110f14e021a8fd04d58d0abd6 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 19 Mar 2020 15:05:40 +0100 Subject: [PATCH 220/885] [FLINK-16684] Fix StreamingFileSink builder compilation for Scala This commit introduces a new type name for the row and bulk format StreamingFileSink builders in order to solve the compilation problem of Scala when using generic types with the self-type idiom. This closes #11454. --- .../sink/filesystem/StreamingFileSink.java | 33 ++++++++++-- .../filesystem/StreamingFileSinkTest.scala | 53 +++++++++++++++++++ 2 files changed, 82 insertions(+), 4 deletions(-) create mode 100644 flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/functions/sink/filesystem/StreamingFileSinkTest.scala diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/StreamingFileSink.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/StreamingFileSink.java index d8d9d6d386..b4a3684c3b 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/StreamingFileSink.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/StreamingFileSink.java @@ -165,9 +165,9 @@ public class StreamingFileSink * @return The builder where the remaining of the configuration parameters for the sink can be configured. * In order to instantiate the sink, call {@link RowFormatBuilder#build()} after specifying the desired parameters. */ - public static StreamingFileSink.RowFormatBuilder> forRowFormat( + public static StreamingFileSink.DefaultRowFormatBuilder forRowFormat( final Path basePath, final Encoder encoder) { - return new StreamingFileSink.RowFormatBuilder<>(basePath, encoder, new DateTimeBucketAssigner<>()); + return new DefaultRowFormatBuilder<>(basePath, encoder, new DateTimeBucketAssigner<>()); } /** @@ -178,9 +178,9 @@ public class StreamingFileSink * @return The builder where the remaining of the configuration parameters for the sink can be configured. * In order to instantiate the sink, call {@link RowFormatBuilder#build()} after specifying the desired parameters. */ - public static StreamingFileSink.BulkFormatBuilder> forBulkFormat( + public static StreamingFileSink.DefaultBulkFormatBuilder forBulkFormat( final Path basePath, final BulkWriter.Factory writerFactory) { - return new StreamingFileSink.BulkFormatBuilder<>(basePath, writerFactory, new DateTimeBucketAssigner<>()); + return new StreamingFileSink.DefaultBulkFormatBuilder<>(basePath, writerFactory, new DateTimeBucketAssigner<>()); } /** @@ -296,6 +296,18 @@ public class StreamingFileSink } } + /** + * Builder for the vanilla {@link StreamingFileSink} using a row format. + * @param record type + */ + public static final class DefaultRowFormatBuilder extends RowFormatBuilder> { + private static final long serialVersionUID = -8503344257202146718L; + + private DefaultRowFormatBuilder(Path basePath, Encoder encoder, BucketAssigner bucketAssigner) { + super(basePath, encoder, bucketAssigner); + } + } + /** * A builder for configuring the sink for bulk-encoding formats, e.g. Parquet/ORC. */ @@ -394,6 +406,19 @@ public class StreamingFileSink } } + /** + * Builder for the vanilla {@link StreamingFileSink} using a bulk format. + * @param record type + */ + public static final class DefaultBulkFormatBuilder extends BulkFormatBuilder> { + + private static final long serialVersionUID = 7493169281036370228L; + + private DefaultBulkFormatBuilder(Path basePath, BulkWriter.Factory writerFactory, BucketAssigner assigner) { + super(basePath, writerFactory, assigner); + } + } + // --------------------------- Sink Methods ----------------------------- @Override diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/functions/sink/filesystem/StreamingFileSinkTest.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/functions/sink/filesystem/StreamingFileSinkTest.scala new file mode 100644 index 0000000000..758c46c2b1 --- /dev/null +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/functions/sink/filesystem/StreamingFileSinkTest.scala @@ -0,0 +1,53 @@ +/* + * 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.scala.functions.sink.filesystem + +import org.apache.flink.api.common.serialization.SimpleStringEncoder +import org.apache.flink.core.fs.Path +import org.apache.flink.streaming.api.functions.sink.filesystem.BulkWriterTest.TestBulkWriterFactory +import org.apache.flink.streaming.api.functions.sink.filesystem.{OutputFileConfig, StreamingFileSink} +import org.junit.Test + +/** + * Tests for the [[org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink]] + */ +class StreamingFileSinkTest { + + /** + * Tests that the StreamingFileSink builder works with the Scala APIs. + */ + @Test + def testStreamingFileSinkRowFormatBuilderCompiles(): Unit = { + StreamingFileSink.forRowFormat(new Path("foobar"), new SimpleStringEncoder[String]()) + .withBucketCheckInterval(10L) + .withOutputFileConfig(OutputFileConfig.builder().build()) + .build() + } + + /** + * Tests that the StreamingFileSink builder works with the Scala APIs. + */ + @Test + def testStreamingFileSinkBulkFormatBuilderCompiles(): Unit = { + StreamingFileSink.forBulkFormat(new Path("foobar"), new TestBulkWriterFactory()) + .withBucketCheckInterval(10L) + .withOutputFileConfig(OutputFileConfig.builder().build()) + .build() + } +} -- Gitee From 5ccb16724769becab0003e0299d9c4a63cd52378 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Fri, 20 Mar 2020 09:46:36 +0800 Subject: [PATCH 221/885] [FLINK-16650][python] Support LocalZonedTimestampType for Python UDF in blink planner (#11439) --- .../pyflink/fn_execution/coder_impl.py | 12 +++++++ flink-python/pyflink/fn_execution/coders.py | 23 +++++++++++++ .../pyflink/fn_execution/sdk_worker_main.py | 10 +++++- .../fn_execution/tests/test_coders_common.py | 14 +++++++- .../pyflink/table/tests/test_types.py | 21 +++++++++--- flink-python/pyflink/table/tests/test_udf.py | 34 +++++++++++++++++++ flink-python/pyflink/table/types.py | 5 ++- flink-python/setup.py | 2 +- .../python/AbstractPythonFunctionRunner.java | 18 +++++++++- .../python/PythonScalarFunctionFlatMap.java | 19 ++++++++++- .../AbstractStatelessFunctionOperator.java | 22 ++++++++++-- .../BaseRowPythonScalarFunctionOperator.java | 7 ++-- .../scalar/PythonScalarFunctionOperator.java | 7 ++-- .../ArrowPythonScalarFunctionOperator.java | 7 ++-- ...eRowArrowPythonScalarFunctionOperator.java | 7 ++-- .../BaseRowPythonTableFunctionOperator.java | 7 ++-- .../table/PythonTableFunctionOperator.java | 7 ++-- ...AbstractPythonStatelessFunctionRunner.java | 6 ++-- ...ractGeneralPythonScalarFunctionRunner.java | 7 ++-- .../AbstractPythonScalarFunctionRunner.java | 7 ++-- .../BaseRowPythonScalarFunctionRunner.java | 7 ++-- .../scalar/PythonScalarFunctionRunner.java | 7 ++-- ...stractArrowPythonScalarFunctionRunner.java | 7 ++-- .../ArrowPythonScalarFunctionRunner.java | 7 ++-- ...aseRowArrowPythonScalarFunctionRunner.java | 7 ++-- .../AbstractPythonTableFunctionRunner.java | 7 ++-- .../BaseRowPythonTableFunctionRunner.java | 7 ++-- .../table/PythonTableFunctionRunner.java | 7 ++-- .../runtime/typeutils/PythonTypeUtils.java | 20 +++++++++++ ...seRowPythonScalarFunctionOperatorTest.java | 7 ++-- .../PythonScalarFunctionOperatorTest.java | 7 ++-- ...ArrowPythonScalarFunctionOperatorTest.java | 7 ++-- ...ArrowPythonScalarFunctionOperatorTest.java | 7 ++-- ...aseRowPythonTableFunctionOperatorTest.java | 4 ++- .../PythonTableFunctionOperatorTest.java | 4 ++- ...BaseRowPythonScalarFunctionRunnerTest.java | 4 ++- .../PythonScalarFunctionRunnerTest.java | 4 ++- .../ArrowPythonScalarFunctionRunnerTest.java | 1 + .../BaseRowPythonTableFunctionRunnerTest.java | 4 ++- .../table/PythonTableFunctionRunnerTest.java | 5 +-- ...hroughArrowPythonScalarFunctionRunner.java | 9 +++-- ...PassThroughPythonScalarFunctionRunner.java | 9 +++-- .../plan/nodes/common/CommonPythonBase.scala | 8 ++++- .../physical/batch/BatchExecPythonCalc.scala | 2 +- .../batch/BatchExecPythonCorrelate.scala | 2 +- .../stream/StreamExecPythonCalc.scala | 2 +- .../stream/StreamExecPythonCorrelate.scala | 2 +- .../utils/python/PythonTableUtils.scala | 8 ++++- .../table/plan/nodes/CommonPythonBase.scala | 9 ++++- .../nodes/dataset/DataSetPythonCalc.scala | 2 +- .../datastream/DataStreamPythonCalc.scala | 2 +- .../DataStreamPythonCorrelate.scala | 2 +- 52 files changed, 352 insertions(+), 77 deletions(-) diff --git a/flink-python/pyflink/fn_execution/coder_impl.py b/flink-python/pyflink/fn_execution/coder_impl.py index 7b6405aafc..f453a88e47 100644 --- a/flink-python/pyflink/fn_execution/coder_impl.py +++ b/flink-python/pyflink/fn_execution/coder_impl.py @@ -406,6 +406,18 @@ class TimestampCoderImpl(StreamCoderImpl): return datetime.datetime.utcfromtimestamp(second).replace(microsecond=microsecond) +class LocalZonedTimestampCoderImpl(TimestampCoderImpl): + + def __init__(self, precision, timezone): + super(LocalZonedTimestampCoderImpl, self).__init__(precision) + self.timezone = timezone + + def internal_to_timestamp(self, milliseconds, nanoseconds): + return self.timezone.localize( + super(LocalZonedTimestampCoderImpl, self).internal_to_timestamp( + milliseconds, nanoseconds)) + + class ArrowCoderImpl(StreamCoderImpl): def __init__(self, schema): diff --git a/flink-python/pyflink/fn_execution/coders.py b/flink-python/pyflink/fn_execution/coders.py index eb47cf70b0..da4f03b7a0 100644 --- a/flink-python/pyflink/fn_execution/coders.py +++ b/flink-python/pyflink/fn_execution/coders.py @@ -22,13 +22,16 @@ from abc import ABC import datetime import decimal import pyarrow as pa +import pytz from apache_beam.coders import Coder from apache_beam.coders.coders import FastCoder, LengthPrefixCoder +from apache_beam.options.pipeline_options import DebugOptions from apache_beam.portability import common_urns from apache_beam.typehints import typehints from pyflink.fn_execution import coder_impl from pyflink.fn_execution import flink_fn_execution_pb2 +from pyflink.fn_execution.sdk_worker_main import pipeline_options from pyflink.table import Row FLINK_SCALAR_FUNCTION_SCHEMA_CODER_URN = "flink:coder:schema:scalar_function:v1" @@ -377,6 +380,22 @@ class TimestampCoder(DeterministicCoder): return datetime.datetime +class LocalZonedTimestampCoder(DeterministicCoder): + """ + Coder for LocalZonedTimestamp. + """ + + def __init__(self, precision, timezone): + self.precision = precision + self.timezone = timezone + + def _create_impl(self): + return coder_impl.LocalZonedTimestampCoderImpl(self.precision, self.timezone) + + def to_type_hint(self): + return datetime.datetime + + class ArrowCoder(DeterministicCoder): """ Coder for Arrow. @@ -466,6 +485,10 @@ def from_proto(field_type): return RowCoder([from_proto(f.type) for f in field_type.row_schema.fields]) if field_type_name == type_name.TIMESTAMP: return TimestampCoder(field_type.timestamp_info.precision) + if field_type_name == type_name.LOCAL_ZONED_TIMESTAMP: + timezone = pytz.timezone(pipeline_options.view_as(DebugOptions).lookup_experiment( + "table.exec.timezone")) + return LocalZonedTimestampCoder(field_type.local_zoned_timestamp_info.precision, timezone) elif field_type_name == type_name.ARRAY: return ArrayCoder(from_proto(field_type.collection_element_type)) elif field_type_name == type_name.MAP: diff --git a/flink-python/pyflink/fn_execution/sdk_worker_main.py b/flink-python/pyflink/fn_execution/sdk_worker_main.py index 82d091c529..7f2c42380e 100644 --- a/flink-python/pyflink/fn_execution/sdk_worker_main.py +++ b/flink-python/pyflink/fn_execution/sdk_worker_main.py @@ -15,10 +15,12 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ - +import os import sys # force to register the operations to SDK Harness +from apache_beam.options.pipeline_options import PipelineOptions + import pyflink.fn_execution.operations # noqa # pylint: disable=unused-import # force to register the coders to SDK Harness @@ -26,5 +28,11 @@ import pyflink.fn_execution.coders # noqa # pylint: disable=unused-import import apache_beam.runners.worker.sdk_worker_main +if 'PIPELINE_OPTIONS' in os.environ: + pipeline_options = apache_beam.runners.worker.sdk_worker_main._parse_pipeline_options( + os.environ['PIPELINE_OPTIONS']) +else: + pipeline_options = PipelineOptions.from_dictionary({}) + if __name__ == '__main__': apache_beam.runners.worker.sdk_worker_main.main(sys.argv) diff --git a/flink-python/pyflink/fn_execution/tests/test_coders_common.py b/flink-python/pyflink/fn_execution/tests/test_coders_common.py index 42d1c19a64..69e9961180 100644 --- a/flink-python/pyflink/fn_execution/tests/test_coders_common.py +++ b/flink-python/pyflink/fn_execution/tests/test_coders_common.py @@ -22,7 +22,8 @@ import unittest from pyflink.fn_execution.coders import BigIntCoder, TinyIntCoder, BooleanCoder, \ SmallIntCoder, IntCoder, FloatCoder, DoubleCoder, BinaryCoder, CharCoder, DateCoder, \ - TimeCoder, TimestampCoder, ArrayCoder, MapCoder, DecimalCoder, FlattenRowCoder, RowCoder + TimeCoder, TimestampCoder, ArrayCoder, MapCoder, DecimalCoder, FlattenRowCoder, RowCoder, \ + LocalZonedTimestampCoder class CodersTest(unittest.TestCase): @@ -93,6 +94,17 @@ class CodersTest(unittest.TestCase): coder = TimestampCoder(6) self.check_coder(coder, datetime.datetime(2019, 9, 10, 18, 30, 20, 123456)) + def test_local_zoned_timestamp_coder(self): + import datetime + import pytz + timezone = pytz.timezone("Asia/Shanghai") + coder = LocalZonedTimestampCoder(3, timezone) + self.check_coder(coder, + timezone.localize(datetime.datetime(2019, 9, 10, 18, 30, 20, 123000))) + coder = LocalZonedTimestampCoder(6, timezone) + self.check_coder(coder, + timezone.localize(datetime.datetime(2019, 9, 10, 18, 30, 20, 123456))) + def test_array_coder(self): element_coder = BigIntCoder() coder = ArrayCoder(element_coder) diff --git a/flink-python/pyflink/table/tests/test_types.py b/flink-python/pyflink/table/tests/test_types.py index 415224dfb8..04a2e13231 100644 --- a/flink-python/pyflink/table/tests/test_types.py +++ b/flink-python/pyflink/table/tests/test_types.py @@ -33,7 +33,8 @@ from pyflink.table.types import (_infer_schema_from_data, _infer_type, _array_type_mappings, _merge_type, _create_type_verifier, UserDefinedType, DataTypes, Row, RowField, RowType, ArrayType, BigIntType, VarCharType, MapType, DataType, - _to_java_type, _from_java_type, ZonedTimestampType) + _to_java_type, _from_java_type, ZonedTimestampType, + LocalZonedTimestampType) class ExamplePointUDT(UserDefinedType): @@ -535,11 +536,23 @@ class TypesTests(unittest.TestCase): def test_local_zoned_timestamp_type(self): lztst = DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE() - ts = datetime.datetime(1970, 1, 1, 0, 0, 0, 0000, tzinfo=UTCOffsetTimezone(1)) - self.assertEqual(-3600000000, lztst.to_sql_type(ts)) + ts = datetime.datetime(1970, 1, 1, 0, 0, 0, 0000) + self.assertEqual(0, lztst.to_sql_type(ts)) + + import pytz + # suppose the timezone of the data is +9:00 + timezone = pytz.timezone("Asia/Tokyo") + orig_epoch = LocalZonedTimestampType.EPOCH_ORDINAL + try: + # suppose the local timezone is +8:00 + LocalZonedTimestampType.EPOCH_ORDINAL = 28800000000 + ts_tokyo = timezone.localize(ts) + self.assertEqual(-3600000000, lztst.to_sql_type(ts_tokyo)) + finally: + LocalZonedTimestampType.EPOCH_ORDINAL = orig_epoch if sys.version_info >= (3, 6): - ts2 = lztst.from_sql_type(-3600000000) + ts2 = lztst.from_sql_type(0) self.assertEqual(ts.astimezone(), ts2.astimezone()) def test_zoned_timestamp_type(self): diff --git a/flink-python/pyflink/table/tests/test_udf.py b/flink-python/pyflink/table/tests/test_udf.py index e12b71ac6d..a599aacb6d 100644 --- a/flink-python/pyflink/table/tests/test_udf.py +++ b/flink-python/pyflink/table/tests/test_udf.py @@ -15,6 +15,10 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ +import datetime + +import pytz + from pyflink.table import DataTypes from pyflink.table.udf import ScalarFunction, udf from pyflink.testing import source_sink_utils @@ -562,6 +566,36 @@ class PyFlinkBlinkStreamUserDefinedFunctionTests(UserDefinedFunctionTests, self.t_env.register_function( "non-callable-udf", udf(Plus(), DataTypes.BIGINT(), DataTypes.BIGINT())) + def test_data_types_only_supported_in_blink_planner(self): + timezone = self.t_env.get_config().get_local_timezone() + local_datetime = pytz.timezone(timezone).localize( + datetime.datetime(1970, 1, 1, 0, 0, 0, 123000)) + + def local_zoned_timestamp_func(local_zoned_timestamp_param): + assert local_zoned_timestamp_param == local_datetime, \ + 'local_zoned_timestamp_param is wrong value %s !' % local_zoned_timestamp_param + return local_zoned_timestamp_param + + self.t_env.register_function( + "local_zoned_timestamp_func", + udf(local_zoned_timestamp_func, + [DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3)], + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3))) + + table_sink = source_sink_utils.TestAppendSink( + ['a'], [DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3)]) + self.t_env.register_table_sink("Results", table_sink) + + t = self.t_env.from_elements( + [(local_datetime,)], + DataTypes.ROW([DataTypes.FIELD("a", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3))])) + + t.select("local_zoned_timestamp_func(local_zoned_timestamp_func(a))") \ + .insert_into("Results") + self.t_env.execute("test") + actual = source_sink_utils.results() + self.assert_equals(actual, ["1970-01-01T00:00:00.123Z"]) + class PyFlinkBlinkBatchUserDefinedFunctionTests(UserDefinedFunctionTests, PyFlinkBlinkBatchTableTestCase): diff --git a/flink-python/pyflink/table/types.py b/flink-python/pyflink/table/types.py index d3ab0f2f48..c193bc3aeb 100644 --- a/flink-python/pyflink/table/types.py +++ b/flink-python/pyflink/table/types.py @@ -477,6 +477,8 @@ class LocalZonedTimestampType(AtomicType): :param nullable: boolean, whether the field can be null (None) or not. """ + EPOCH_ORDINAL = calendar.timegm(time.localtime(0)) * 10 ** 6 + def __init__(self, precision=6, nullable=True): super(LocalZonedTimestampType, self).__init__(nullable) assert 0 <= precision <= 9 @@ -492,10 +494,11 @@ class LocalZonedTimestampType(AtomicType): if dt is not None: seconds = (calendar.timegm(dt.utctimetuple()) if dt.tzinfo else time.mktime(dt.timetuple())) - return int(seconds) * 10 ** 6 + dt.microsecond + return int(seconds) * 10 ** 6 + dt.microsecond + self.EPOCH_ORDINAL def from_sql_type(self, ts): if ts is not None: + ts = ts - self.EPOCH_ORDINAL return datetime.datetime.fromtimestamp(ts // 10 ** 6).replace(microsecond=ts % 10 ** 6) diff --git a/flink-python/setup.py b/flink-python/setup.py index ca4b5985fa..5f5037639f 100644 --- a/flink-python/setup.py +++ b/flink-python/setup.py @@ -225,7 +225,7 @@ run sdist. python_requires='>=3.5', install_requires=['py4j==0.10.8.1', 'python-dateutil==2.8.0', 'apache-beam==2.19.0', 'cloudpickle==1.2.2', 'avro-python3>=1.8.1,<=1.9.1', 'jsonpickle==1.2', - 'pandas>=0.23.4,<=0.25.3', 'pyarrow>=0.15.1,<=0.16.0'], + 'pandas>=0.23.4,<=0.25.3', 'pyarrow>=0.15.1,<=0.16.0', 'pytz>=2018.3'], tests_require=['pytest==4.4.1'], description='Apache Flink Python API', long_description=long_description, diff --git a/flink-python/src/main/java/org/apache/flink/python/AbstractPythonFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/python/AbstractPythonFunctionRunner.java index 597057e091..84e783826c 100644 --- a/flink-python/src/main/java/org/apache/flink/python/AbstractPythonFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/python/AbstractPythonFunctionRunner.java @@ -37,11 +37,14 @@ import org.apache.beam.runners.fnexecution.control.StageBundleFactory; import org.apache.beam.runners.fnexecution.provisioning.JobInfo; import org.apache.beam.runners.fnexecution.state.StateRequestHandler; import org.apache.beam.sdk.fn.data.FnDataReceiver; +import org.apache.beam.sdk.options.ExperimentalOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.PortablePipelineOptions; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Struct; +import java.util.Map; + /** * An base class for {@link PythonFunctionRunner}. * @@ -64,6 +67,11 @@ public abstract class AbstractPythonFunctionRunner implements PythonFunction */ private final PythonEnvironmentManager environmentManager; + /** + * The options used to configure the Python worker process. + */ + private final Map jobOptions; + /** * The bundle factory which has all job-scoped information and can be used to create a {@link StageBundleFactory}. */ @@ -112,11 +120,13 @@ public abstract class AbstractPythonFunctionRunner implements PythonFunction String taskName, FnDataReceiver resultReceiver, PythonEnvironmentManager environmentManager, - StateRequestHandler stateRequestHandler) { + StateRequestHandler stateRequestHandler, + Map jobOptions) { this.taskName = Preconditions.checkNotNull(taskName); this.resultReceiver = Preconditions.checkNotNull(resultReceiver); this.environmentManager = Preconditions.checkNotNull(environmentManager); this.stateRequestHandler = Preconditions.checkNotNull(stateRequestHandler); + this.jobOptions = Preconditions.checkNotNull(jobOptions); } @Override @@ -131,6 +141,12 @@ public abstract class AbstractPythonFunctionRunner implements PythonFunction PipelineOptionsFactory.as(PortablePipelineOptions.class); // one operator has one Python SDK harness portableOptions.setSdkWorkerParallelism(1); + ExperimentalOptions experimentalOptions = portableOptions.as(ExperimentalOptions.class); + for (Map.Entry entry : jobOptions.entrySet()) { + ExperimentalOptions.addExperiment(experimentalOptions, + String.join("=", entry.getKey(), entry.getValue())); + } + Struct pipelineOptions = PipelineOptionsTranslation.toProto(portableOptions); jobBundleFactory = createJobBundleFactory(pipelineOptions); diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/functions/python/PythonScalarFunctionFlatMap.java b/flink-python/src/main/java/org/apache/flink/table/runtime/functions/python/PythonScalarFunctionFlatMap.java index 97855093a1..10cee8dd4a 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/functions/python/PythonScalarFunctionFlatMap.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/functions/python/PythonScalarFunctionFlatMap.java @@ -53,6 +53,8 @@ import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; @@ -140,6 +142,11 @@ public final class PythonScalarFunctionFlatMap */ private final PythonConfig config; + /** + * The options used to configure the Python worker process. + */ + private final Map jobOptions; + /** * Use an AtomicBoolean because we start/stop bundles by a timer thread. */ @@ -178,6 +185,7 @@ public final class PythonScalarFunctionFlatMap this.udfInputOffsets = Preconditions.checkNotNull(udfInputOffsets); this.forwardedFields = Preconditions.checkNotNull(forwardedFields); this.config = new PythonConfig(Preconditions.checkNotNull(config)); + this.jobOptions = buildJobOptions(config); } @Override @@ -285,7 +293,8 @@ public final class PythonScalarFunctionFlatMap scalarFunctions, createPythonEnvironmentManager(), udfInputType, - udfOutputType); + udfOutputType, + jobOptions); } private PythonEnvironmentManager createPythonEnvironmentManager() throws IOException { @@ -321,6 +330,14 @@ public final class PythonScalarFunctionFlatMap } } + private Map buildJobOptions(Configuration config) { + Map jobOptions = new HashMap<>(); + if (config.containsKey("table.exec.timezone")) { + jobOptions.put("table.exec.timezone", config.getString("table.exec.timezone", null)); + } + return jobOptions; + } + @Override public TypeInformation getProducedType() { return (TypeInformation) LegacyTypeInfoDataTypeConverter diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/AbstractStatelessFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/AbstractStatelessFunctionOperator.java index 3670ac7a34..248683c25d 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/AbstractStatelessFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/AbstractStatelessFunctionOperator.java @@ -37,6 +37,8 @@ import org.apache.beam.sdk.fn.data.FnDataReceiver; import java.io.IOException; import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; import java.util.concurrent.LinkedBlockingQueue; import java.util.stream.Collectors; @@ -68,6 +70,11 @@ public abstract class AbstractStatelessFunctionOperator */ protected final int[] userDefinedFunctionInputOffsets; + /** + * The options used to configure the Python worker process. + */ + private final Map jobOptions; + /** * The user-defined function input logical type. */ @@ -108,6 +115,7 @@ public abstract class AbstractStatelessFunctionOperator this.inputType = Preconditions.checkNotNull(inputType); this.outputType = Preconditions.checkNotNull(outputType); this.userDefinedFunctionInputOffsets = Preconditions.checkNotNull(userDefinedFunctionInputOffsets); + this.jobOptions = buildJobOptions(config); } @Override @@ -140,7 +148,8 @@ public abstract class AbstractStatelessFunctionOperator return new ProjectUdfInputPythonScalarFunctionRunner( createPythonFunctionRunner( userDefinedFunctionResultReceiver, - createPythonEnvironmentManager())); + createPythonEnvironmentManager(), + jobOptions)); } /** @@ -153,7 +162,16 @@ public abstract class AbstractStatelessFunctionOperator public abstract PythonFunctionRunner createPythonFunctionRunner( FnDataReceiver resultReceiver, - PythonEnvironmentManager pythonEnvironmentManager); + PythonEnvironmentManager pythonEnvironmentManager, + Map jobOptions); + + private Map buildJobOptions(Configuration config) { + Map jobOptions = new HashMap<>(); + if (config.containsKey("table.exec.timezone")) { + jobOptions.put("table.exec.timezone", config.getString("table.exec.timezone", null)); + } + return jobOptions; + } private class ProjectUdfInputPythonScalarFunctionRunner implements PythonFunctionRunner { diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/BaseRowPythonScalarFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/BaseRowPythonScalarFunctionOperator.java index 6f9e922465..c845090dd2 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/BaseRowPythonScalarFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/BaseRowPythonScalarFunctionOperator.java @@ -33,6 +33,7 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.beam.sdk.fn.data.FnDataReceiver; import java.io.IOException; +import java.util.Map; /** * The Python {@link ScalarFunction} operator for the blink planner. @@ -80,13 +81,15 @@ public class BaseRowPythonScalarFunctionOperator extends AbstractBaseRowPythonSc @Override public PythonFunctionRunner createPythonFunctionRunner( FnDataReceiver resultReceiver, - PythonEnvironmentManager pythonEnvironmentManager) { + PythonEnvironmentManager pythonEnvironmentManager, + Map jobOptions) { return new BaseRowPythonScalarFunctionRunner( getRuntimeContext().getTaskName(), resultReceiver, scalarFunctions, pythonEnvironmentManager, userDefinedFunctionInputType, - userDefinedFunctionOutputType); + userDefinedFunctionOutputType, + jobOptions); } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperator.java index 672882e210..8fa27bfa71 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperator.java @@ -34,6 +34,7 @@ import org.apache.flink.types.Row; import org.apache.beam.sdk.fn.data.FnDataReceiver; import java.io.IOException; +import java.util.Map; /** * The Python {@link ScalarFunction} operator for the legacy planner. @@ -81,13 +82,15 @@ public class PythonScalarFunctionOperator extends AbstractRowPythonScalarFunctio @Override public PythonFunctionRunner createPythonFunctionRunner( FnDataReceiver resultReceiver, - PythonEnvironmentManager pythonEnvironmentManager) { + PythonEnvironmentManager pythonEnvironmentManager, + Map jobOptions) { return new PythonScalarFunctionRunner( getRuntimeContext().getTaskName(), resultReceiver, scalarFunctions, pythonEnvironmentManager, userDefinedFunctionInputType, - userDefinedFunctionOutputType); + userDefinedFunctionOutputType, + jobOptions); } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperator.java index 355410668d..4a686e8442 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperator.java @@ -38,6 +38,7 @@ import org.apache.arrow.vector.ipc.ArrowStreamReader; import org.apache.beam.sdk.fn.data.FnDataReceiver; import java.io.IOException; +import java.util.Map; /** * Arrow Python {@link ScalarFunction} operator for the old planner. @@ -93,7 +94,8 @@ public class ArrowPythonScalarFunctionOperator extends AbstractRowPythonScalarFu @Override public PythonFunctionRunner createPythonFunctionRunner( FnDataReceiver resultReceiver, - PythonEnvironmentManager pythonEnvironmentManager) { + PythonEnvironmentManager pythonEnvironmentManager, + Map jobOptions) { return new ArrowPythonScalarFunctionRunner( getRuntimeContext().getTaskName(), resultReceiver, @@ -101,7 +103,8 @@ public class ArrowPythonScalarFunctionOperator extends AbstractRowPythonScalarFu pythonEnvironmentManager, userDefinedFunctionInputType, userDefinedFunctionOutputType, - getPythonConfig().getMaxArrowBatchSize()); + getPythonConfig().getMaxArrowBatchSize(), + jobOptions); } @Override diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/BaseRowArrowPythonScalarFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/BaseRowArrowPythonScalarFunctionOperator.java index aa46991baf..6a20fe69ba 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/BaseRowArrowPythonScalarFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/BaseRowArrowPythonScalarFunctionOperator.java @@ -37,6 +37,7 @@ import org.apache.arrow.vector.ipc.ArrowStreamReader; import org.apache.beam.sdk.fn.data.FnDataReceiver; import java.io.IOException; +import java.util.Map; /** * Arrow Python {@link ScalarFunction} operator for the blink planner. @@ -92,7 +93,8 @@ public class BaseRowArrowPythonScalarFunctionOperator extends AbstractBaseRowPyt @Override public PythonFunctionRunner createPythonFunctionRunner( FnDataReceiver resultReceiver, - PythonEnvironmentManager pythonEnvironmentManager) { + PythonEnvironmentManager pythonEnvironmentManager, + Map jobOptions) { return new BaseRowArrowPythonScalarFunctionRunner( getRuntimeContext().getTaskName(), resultReceiver, @@ -100,7 +102,8 @@ public class BaseRowArrowPythonScalarFunctionOperator extends AbstractBaseRowPyt pythonEnvironmentManager, userDefinedFunctionInputType, userDefinedFunctionOutputType, - getPythonConfig().getMaxArrowBatchSize()); + getPythonConfig().getMaxArrowBatchSize(), + jobOptions); } @Override diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/table/BaseRowPythonTableFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/table/BaseRowPythonTableFunctionOperator.java index d138df37c5..f7dd371a73 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/table/BaseRowPythonTableFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/table/BaseRowPythonTableFunctionOperator.java @@ -43,6 +43,7 @@ import org.apache.beam.sdk.fn.data.FnDataReceiver; import org.apache.calcite.rel.core.JoinRelType; import java.io.IOException; +import java.util.Map; /** * The Python {@link TableFunction} operator for the blink planner. @@ -117,14 +118,16 @@ public class BaseRowPythonTableFunctionOperator @Override public PythonFunctionRunner createPythonFunctionRunner( FnDataReceiver resultReceiver, - PythonEnvironmentManager pythonEnvironmentManager) { + PythonEnvironmentManager pythonEnvironmentManager, + Map jobOptions) { return new BaseRowPythonTableFunctionRunner( getRuntimeContext().getTaskName(), resultReceiver, tableFunction, pythonEnvironmentManager, userDefinedFunctionInputType, - userDefinedFunctionOutputType); + userDefinedFunctionOutputType, + jobOptions); } private Projection createUdtfInputProjection() { diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/table/PythonTableFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/table/PythonTableFunctionOperator.java index a4c525993b..418d271a79 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/table/PythonTableFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/table/PythonTableFunctionOperator.java @@ -38,6 +38,7 @@ import org.apache.beam.sdk.fn.data.FnDataReceiver; import org.apache.calcite.rel.core.JoinRelType; import java.io.IOException; +import java.util.Map; /** * The Python {@link TableFunction} operator for the legacy planner. @@ -131,13 +132,15 @@ public class PythonTableFunctionOperator extends AbstractPythonTableFunctionOper @Override public PythonFunctionRunner createPythonFunctionRunner( FnDataReceiver resultReceiver, - PythonEnvironmentManager pythonEnvironmentManager) { + PythonEnvironmentManager pythonEnvironmentManager, + Map jobOptions) { return new PythonTableFunctionRunner( getRuntimeContext().getTaskName(), resultReceiver, tableFunction, pythonEnvironmentManager, userDefinedFunctionInputType, - userDefinedFunctionOutputType); + userDefinedFunctionOutputType, + jobOptions); } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/AbstractPythonStatelessFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/AbstractPythonStatelessFunctionRunner.java index c13d3afd39..5155bf6e95 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/AbstractPythonStatelessFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/AbstractPythonStatelessFunctionRunner.java @@ -51,6 +51,7 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.util.Collections; import java.util.List; +import java.util.Map; import static org.apache.beam.runners.core.construction.BeamUrns.getUrn; @@ -86,8 +87,9 @@ public abstract class AbstractPythonStatelessFunctionRunner extends Abstract PythonEnvironmentManager environmentManager, RowType inputType, RowType outputType, - String functionUrn) { - super(taskName, resultReceiver, environmentManager, StateRequestHandler.unsupported()); + String functionUrn, + Map jobOptions) { + super(taskName, resultReceiver, environmentManager, StateRequestHandler.unsupported(), jobOptions); this.functionUrn = functionUrn; this.inputType = Preconditions.checkNotNull(inputType); this.outputType = Preconditions.checkNotNull(outputType); diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/AbstractGeneralPythonScalarFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/AbstractGeneralPythonScalarFunctionRunner.java index 67b16ae7f9..eededf3e27 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/AbstractGeneralPythonScalarFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/AbstractGeneralPythonScalarFunctionRunner.java @@ -30,6 +30,8 @@ import org.apache.beam.runners.fnexecution.control.OutputReceiverFactory; import org.apache.beam.sdk.fn.data.FnDataReceiver; import org.apache.beam.sdk.util.WindowedValue; +import java.util.Map; + /** * Abstract {@link PythonFunctionRunner} used to execute Python {@link ScalarFunction}s. * @@ -51,8 +53,9 @@ public abstract class AbstractGeneralPythonScalarFunctionRunner extends Abst PythonFunctionInfo[] scalarFunctions, PythonEnvironmentManager environmentManager, RowType inputType, - RowType outputType) { - super(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType); + RowType outputType, + Map jobOptions) { + super(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, jobOptions); } @Override diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/AbstractPythonScalarFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/AbstractPythonScalarFunctionRunner.java index c5991cd184..9e0cdf8368 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/AbstractPythonScalarFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/AbstractPythonScalarFunctionRunner.java @@ -31,6 +31,8 @@ import org.apache.flink.util.Preconditions; import org.apache.beam.sdk.fn.data.FnDataReceiver; +import java.util.Map; + /** * Abstract {@link PythonFunctionRunner} used to execute Python {@link ScalarFunction}s. * @@ -49,8 +51,9 @@ public abstract class AbstractPythonScalarFunctionRunner extends AbstractPyt PythonFunctionInfo[] scalarFunctions, PythonEnvironmentManager environmentManager, RowType inputType, - RowType outputType) { - super(taskName, resultReceiver, environmentManager, inputType, outputType, SCALAR_FUNCTION_URN); + RowType outputType, + Map jobOptions) { + super(taskName, resultReceiver, environmentManager, inputType, outputType, SCALAR_FUNCTION_URN, jobOptions); this.scalarFunctions = Preconditions.checkNotNull(scalarFunctions); } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/BaseRowPythonScalarFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/BaseRowPythonScalarFunctionRunner.java index 758ac648da..4a2ee5544c 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/BaseRowPythonScalarFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/BaseRowPythonScalarFunctionRunner.java @@ -30,6 +30,8 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.beam.sdk.fn.data.FnDataReceiver; +import java.util.Map; + /** * A {@link PythonFunctionRunner} used to execute Python {@link ScalarFunction}s. * It takes {@link BaseRow} as the input and outputs a byte array. @@ -43,8 +45,9 @@ public class BaseRowPythonScalarFunctionRunner extends AbstractGeneralPythonScal PythonFunctionInfo[] scalarFunctions, PythonEnvironmentManager environmentManager, RowType inputType, - RowType outputType) { - super(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType); + RowType outputType, + Map jobOptions) { + super(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, jobOptions); } @Override diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/PythonScalarFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/PythonScalarFunctionRunner.java index ca7d80a1d6..8e48b46ec9 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/PythonScalarFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/PythonScalarFunctionRunner.java @@ -30,6 +30,8 @@ import org.apache.flink.types.Row; import org.apache.beam.sdk.fn.data.FnDataReceiver; +import java.util.Map; + /** * A {@link PythonFunctionRunner} used to execute Python {@link ScalarFunction}s. * It takes {@link Row} as the input and outputs a byte array. @@ -43,8 +45,9 @@ public class PythonScalarFunctionRunner extends AbstractGeneralPythonScalarFunct PythonFunctionInfo[] scalarFunctions, PythonEnvironmentManager environmentManager, RowType inputType, - RowType outputType) { - super(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType); + RowType outputType, + Map jobOptions) { + super(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, jobOptions); } @Override diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/AbstractArrowPythonScalarFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/AbstractArrowPythonScalarFunctionRunner.java index d218c8fddf..dda5395db1 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/AbstractArrowPythonScalarFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/AbstractArrowPythonScalarFunctionRunner.java @@ -36,6 +36,8 @@ import org.apache.beam.runners.fnexecution.control.OutputReceiverFactory; import org.apache.beam.sdk.fn.data.FnDataReceiver; import org.apache.beam.sdk.util.WindowedValue; +import java.util.Map; + /** * Abstract {@link PythonFunctionRunner} used to execute Arrow Python {@link ScalarFunction}s. * @@ -97,8 +99,9 @@ public abstract class AbstractArrowPythonScalarFunctionRunner extends Abstra PythonEnvironmentManager environmentManager, RowType inputType, RowType outputType, - int maxArrowBatchSize) { - super(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType); + int maxArrowBatchSize, + Map jobOptions) { + super(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, jobOptions); this.maxArrowBatchSize = maxArrowBatchSize; } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/ArrowPythonScalarFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/ArrowPythonScalarFunctionRunner.java index ee86327479..df3d291aac 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/ArrowPythonScalarFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/ArrowPythonScalarFunctionRunner.java @@ -30,6 +30,8 @@ import org.apache.flink.types.Row; import org.apache.beam.sdk.fn.data.FnDataReceiver; +import java.util.Map; + /** * A {@link PythonFunctionRunner} used to execute Arrow Python {@link ScalarFunction}s. * It takes {@link Row} as the input type. @@ -44,8 +46,9 @@ public class ArrowPythonScalarFunctionRunner extends AbstractArrowPythonScalarFu PythonEnvironmentManager environmentManager, RowType inputType, RowType outputType, - int maxArrowBatchSize) { - super(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, maxArrowBatchSize); + int maxArrowBatchSize, + Map jobOptions) { + super(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, maxArrowBatchSize, jobOptions); } @Override diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/BaseRowArrowPythonScalarFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/BaseRowArrowPythonScalarFunctionRunner.java index f15dec070a..60e9609334 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/BaseRowArrowPythonScalarFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/BaseRowArrowPythonScalarFunctionRunner.java @@ -30,6 +30,8 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.beam.sdk.fn.data.FnDataReceiver; +import java.util.Map; + /** * A {@link PythonFunctionRunner} used to execute Arrow Python {@link ScalarFunction}s. * It takes {@link BaseRow} as the input type. @@ -44,8 +46,9 @@ public class BaseRowArrowPythonScalarFunctionRunner extends AbstractArrowPythonS PythonEnvironmentManager environmentManager, RowType inputType, RowType outputType, - int maxBatchSize) { - super(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, maxBatchSize); + int maxBatchSize, + Map jobOptions) { + super(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, maxBatchSize, jobOptions); } @Override diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/table/AbstractPythonTableFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/table/AbstractPythonTableFunctionRunner.java index 0aa3265526..962f556ba2 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/table/AbstractPythonTableFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/table/AbstractPythonTableFunctionRunner.java @@ -34,6 +34,8 @@ import org.apache.beam.runners.fnexecution.control.OutputReceiverFactory; import org.apache.beam.sdk.fn.data.FnDataReceiver; import org.apache.beam.sdk.util.WindowedValue; +import java.util.Map; + /** * Abstract {@link PythonFunctionRunner} used to execute Python {@link TableFunction}. * @@ -58,8 +60,9 @@ public abstract class AbstractPythonTableFunctionRunner extends AbstractPyth PythonFunctionInfo tableFunction, PythonEnvironmentManager environmentManager, RowType inputType, - RowType outputType) { - super(taskName, resultReceiver, environmentManager, inputType, outputType, TABLE_FUNCTION_URN); + RowType outputType, + Map jobOptions) { + super(taskName, resultReceiver, environmentManager, inputType, outputType, TABLE_FUNCTION_URN, jobOptions); this.tableFunction = Preconditions.checkNotNull(tableFunction); } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/table/BaseRowPythonTableFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/table/BaseRowPythonTableFunctionRunner.java index 866d1826fa..3ff6ca4598 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/table/BaseRowPythonTableFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/table/BaseRowPythonTableFunctionRunner.java @@ -30,6 +30,8 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.beam.sdk.fn.data.FnDataReceiver; +import java.util.Map; + /** * A {@link PythonFunctionRunner} used to execute Python {@link TableFunction}. * It takes {@link BaseRow} as the input and outputs a byte array. @@ -42,8 +44,9 @@ public class BaseRowPythonTableFunctionRunner extends AbstractPythonTableFunctio PythonFunctionInfo tableFunction, PythonEnvironmentManager environmentManager, RowType inputType, - RowType outputType) { - super(taskName, resultReceiver, tableFunction, environmentManager, inputType, outputType); + RowType outputType, + Map jobOptions) { + super(taskName, resultReceiver, tableFunction, environmentManager, inputType, outputType, jobOptions); } @Override diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/table/PythonTableFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/table/PythonTableFunctionRunner.java index c691ca3ee5..48fa400b92 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/table/PythonTableFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/table/PythonTableFunctionRunner.java @@ -31,6 +31,8 @@ import org.apache.flink.types.Row; import org.apache.beam.sdk.fn.data.FnDataReceiver; +import java.util.Map; + /** * A {@link PythonFunctionRunner} used to execute Python {@link TableFunction}. * It takes {@link Row} as the input and outputs a byte array. @@ -44,8 +46,9 @@ public class PythonTableFunctionRunner extends AbstractPythonTableFunctionRunner PythonFunctionInfo tableFunction, PythonEnvironmentManager environmentManager, RowType inputType, - RowType outputType) { - super(taskName, resultReceiver, tableFunction, environmentManager, inputType, outputType); + RowType outputType, + Map jobOptions) { + super(taskName, resultReceiver, tableFunction, environmentManager, inputType, outputType, jobOptions); } @Override diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java index 97d6cf6a60..60f79a0a03 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java @@ -52,6 +52,7 @@ import org.apache.flink.table.types.logical.DoubleType; import org.apache.flink.table.types.logical.FloatType; import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.LegacyTypeInformationType; +import org.apache.flink.table.types.logical.LocalZonedTimestampType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.MapType; import org.apache.flink.table.types.logical.RowType; @@ -268,6 +269,11 @@ public final class PythonTypeUtils { return new SqlTimestampSerializer(timestampType.getPrecision()); } + @Override + public TypeSerializer visit(LocalZonedTimestampType localZonedTimestampType) { + return new SqlTimestampSerializer(localZonedTimestampType.getPrecision()); + } + public TypeSerializer visit(ArrayType arrayType) { LogicalType elementType = arrayType.getElementType(); TypeSerializer elementTypeSerializer = elementType.accept(this); @@ -416,6 +422,20 @@ public final class PythonTypeUtils { return builder.build(); } + @Override + public FlinkFnApi.Schema.FieldType visit(LocalZonedTimestampType localZonedTimestampType) { + FlinkFnApi.Schema.FieldType.Builder builder = + FlinkFnApi.Schema.FieldType.newBuilder() + .setTypeName(FlinkFnApi.Schema.TypeName.LOCAL_ZONED_TIMESTAMP) + .setNullable(localZonedTimestampType.isNullable()); + + FlinkFnApi.Schema.LocalZonedTimestampInfo.Builder dateTimeBuilder = + FlinkFnApi.Schema.LocalZonedTimestampInfo.newBuilder() + .setPrecision(localZonedTimestampType.getPrecision()); + builder.setLocalZonedTimestampInfo(dateTimeBuilder.build()); + return builder.build(); + } + @Override public FlinkFnApi.Schema.FieldType visit(DecimalType decimalType) { FlinkFnApi.Schema.FieldType.Builder builder = diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/BaseRowPythonScalarFunctionOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/BaseRowPythonScalarFunctionOperatorTest.java index 0c5e10eb14..ef4e618280 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/BaseRowPythonScalarFunctionOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/BaseRowPythonScalarFunctionOperatorTest.java @@ -40,6 +40,7 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.beam.sdk.fn.data.FnDataReceiver; import java.util.Collection; +import java.util.Map; import static org.apache.flink.table.runtime.util.StreamRecordUtils.baserow; @@ -115,14 +116,16 @@ public class BaseRowPythonScalarFunctionOperatorTest @Override public PythonFunctionRunner createPythonFunctionRunner( FnDataReceiver resultReceiver, - PythonEnvironmentManager pythonEnvironmentManager) { + PythonEnvironmentManager pythonEnvironmentManager, + Map jobOptions) { return new PassThroughPythonScalarFunctionRunner( getRuntimeContext().getTaskName(), resultReceiver, scalarFunctions, pythonEnvironmentManager, userDefinedFunctionInputType, - userDefinedFunctionOutputType) { + userDefinedFunctionOutputType, + jobOptions) { @Override public TypeSerializer getInputTypeSerializer() { return (BaseRowSerializer) PythonTypeUtils.toBlinkTypeSerializer(getInputType()); diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperatorTest.java index c910e45b25..459feb2a00 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperatorTest.java @@ -36,6 +36,7 @@ import org.apache.flink.types.Row; import org.apache.beam.sdk.fn.data.FnDataReceiver; import java.util.Collection; +import java.util.Map; import java.util.Queue; /** @@ -91,14 +92,16 @@ public class PythonScalarFunctionOperatorTest extends PythonScalarFunctionOperat @Override public PythonFunctionRunner createPythonFunctionRunner( FnDataReceiver resultReceiver, - PythonEnvironmentManager pythonEnvironmentManager) { + PythonEnvironmentManager pythonEnvironmentManager, + Map jobOptions) { return new PassThroughPythonScalarFunctionRunner( getRuntimeContext().getTaskName(), resultReceiver, scalarFunctions, pythonEnvironmentManager, userDefinedFunctionInputType, - userDefinedFunctionOutputType) { + userDefinedFunctionOutputType, + jobOptions) { @Override public TypeSerializer getInputTypeSerializer() { return (RowSerializer) PythonTypeUtils.toFlinkTypeSerializer(getInputType()); diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperatorTest.java index 93d0f2c271..affca785e5 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperatorTest.java @@ -38,6 +38,7 @@ import org.apache.flink.types.Row; import org.apache.beam.sdk.fn.data.FnDataReceiver; import java.util.Collection; +import java.util.Map; import java.util.Queue; /** @@ -89,7 +90,8 @@ public class ArrowPythonScalarFunctionOperatorTest extends PythonScalarFunctionO @Override public PythonFunctionRunner createPythonFunctionRunner( FnDataReceiver resultReceiver, - PythonEnvironmentManager pythonEnvironmentManager) { + PythonEnvironmentManager pythonEnvironmentManager, + Map jobOptions) { return new PassThroughArrowPythonScalarFunctionRunner( getRuntimeContext().getTaskName(), resultReceiver, @@ -97,7 +99,8 @@ public class ArrowPythonScalarFunctionOperatorTest extends PythonScalarFunctionO pythonEnvironmentManager, userDefinedFunctionInputType, userDefinedFunctionOutputType, - getPythonConfig().getMaxArrowBatchSize()) { + getPythonConfig().getMaxArrowBatchSize(), + jobOptions) { @Override public ArrowWriter createArrowWriter() { return ArrowUtils.createRowArrowWriter(root); diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/BaseRowArrowPythonScalarFunctionOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/BaseRowArrowPythonScalarFunctionOperatorTest.java index 43138f3b93..f32bf32d16 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/BaseRowArrowPythonScalarFunctionOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/BaseRowArrowPythonScalarFunctionOperatorTest.java @@ -43,6 +43,7 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.beam.sdk.fn.data.FnDataReceiver; import java.util.Collection; +import java.util.Map; import static org.apache.flink.table.runtime.util.StreamRecordUtils.baserow; @@ -111,7 +112,8 @@ public class BaseRowArrowPythonScalarFunctionOperatorTest @Override public PythonFunctionRunner createPythonFunctionRunner( FnDataReceiver resultReceiver, - PythonEnvironmentManager pythonEnvironmentManager) { + PythonEnvironmentManager pythonEnvironmentManager, + Map jobOptions) { return new PassThroughArrowPythonScalarFunctionRunner( getRuntimeContext().getTaskName(), resultReceiver, @@ -119,7 +121,8 @@ public class BaseRowArrowPythonScalarFunctionOperatorTest pythonEnvironmentManager, userDefinedFunctionInputType, userDefinedFunctionOutputType, - getPythonConfig().getMaxArrowBatchSize()) { + getPythonConfig().getMaxArrowBatchSize(), + jobOptions) { @Override public ArrowWriter createArrowWriter() { return ArrowUtils.createBaseRowArrowWriter(root); diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/table/BaseRowPythonTableFunctionOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/table/BaseRowPythonTableFunctionOperatorTest.java index f72bd9e2b2..f94bb0c279 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/table/BaseRowPythonTableFunctionOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/table/BaseRowPythonTableFunctionOperatorTest.java @@ -36,6 +36,7 @@ import org.apache.beam.sdk.fn.data.FnDataReceiver; import org.apache.calcite.rel.core.JoinRelType; import java.util.Collection; +import java.util.Map; import static org.apache.flink.table.runtime.util.StreamRecordUtils.baserow; import static org.apache.flink.table.runtime.util.StreamRecordUtils.binaryrow; @@ -94,7 +95,8 @@ public class BaseRowPythonTableFunctionOperatorTest @Override public PythonFunctionRunner createPythonFunctionRunner( FnDataReceiver resultReceiver, - PythonEnvironmentManager pythonEnvironmentManager) { + PythonEnvironmentManager pythonEnvironmentManager, + Map jobOptions) { return new PassThroughPythonTableFunctionRunner(resultReceiver) { @Override public BaseRow copy(BaseRow element) { diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/table/PythonTableFunctionOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/table/PythonTableFunctionOperatorTest.java index 4d4ddfeea4..409675b0c3 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/table/PythonTableFunctionOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/table/PythonTableFunctionOperatorTest.java @@ -34,6 +34,7 @@ import org.apache.beam.sdk.fn.data.FnDataReceiver; import org.apache.calcite.rel.core.JoinRelType; import java.util.Collection; +import java.util.Map; import java.util.Queue; /** @@ -77,7 +78,8 @@ public class PythonTableFunctionOperatorTest extends PythonTableFunctionOperator @Override public PythonFunctionRunner createPythonFunctionRunner( FnDataReceiver resultReceiver, - PythonEnvironmentManager pythonEnvironmentManager) { + PythonEnvironmentManager pythonEnvironmentManager, + Map jobOptions) { return new PassThroughPythonTableFunctionRunner(resultReceiver) { @Override public Row copy(Row element) { diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/scalar/BaseRowPythonScalarFunctionRunnerTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/scalar/BaseRowPythonScalarFunctionRunnerTest.java index 85a680be60..7f42b525ca 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/scalar/BaseRowPythonScalarFunctionRunnerTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/scalar/BaseRowPythonScalarFunctionRunnerTest.java @@ -30,6 +30,7 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.beam.sdk.fn.data.FnDataReceiver; import org.junit.Test; +import java.util.Collections; import java.util.HashMap; import static org.junit.Assert.assertEquals; @@ -88,6 +89,7 @@ public class BaseRowPythonScalarFunctionRunnerTest extends AbstractPythonScalarF pythonFunctionInfos, environmentManager, inputType, - outputType); + outputType, + Collections.emptyMap()); } } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/scalar/PythonScalarFunctionRunnerTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/scalar/PythonScalarFunctionRunnerTest.java index 7757b3f46f..66f1eed35a 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/scalar/PythonScalarFunctionRunnerTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/scalar/PythonScalarFunctionRunnerTest.java @@ -210,7 +210,8 @@ public class PythonScalarFunctionRunnerTest extends AbstractPythonScalarFunction pythonFunctionInfos, environmentManager, inputType, - outputType); + outputType, + Collections.emptyMap()); } private AbstractGeneralPythonScalarFunctionRunner createUDFRunner( @@ -236,6 +237,7 @@ public class PythonScalarFunctionRunnerTest extends AbstractPythonScalarFunction environmentManager, rowType, rowType, + Collections.emptyMap(), jobBundleFactory) { @Override public TypeSerializer getInputTypeSerializer() { diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/ArrowPythonScalarFunctionRunnerTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/ArrowPythonScalarFunctionRunnerTest.java index 8843d82f40..208b5bf22d 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/ArrowPythonScalarFunctionRunnerTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/ArrowPythonScalarFunctionRunnerTest.java @@ -173,6 +173,7 @@ public class ArrowPythonScalarFunctionRunnerTest extends AbstractPythonScalarFun inputType, outputType, maxArrowBatchSize, + Collections.emptyMap(), jobBundleFactory) { @Override public ArrowWriter createArrowWriter() { diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/table/BaseRowPythonTableFunctionRunnerTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/table/BaseRowPythonTableFunctionRunnerTest.java index a93511f4a1..23c6533b47 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/table/BaseRowPythonTableFunctionRunnerTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/table/BaseRowPythonTableFunctionRunnerTest.java @@ -30,6 +30,7 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.beam.sdk.fn.data.FnDataReceiver; import org.junit.Test; +import java.util.Collections; import java.util.HashMap; import static org.junit.Assert.assertEquals; @@ -73,6 +74,7 @@ public class BaseRowPythonTableFunctionRunnerTest extends AbstractPythonTableFun pythonFunctionInfo, environmentManager, inputType, - outputType); + outputType, + Collections.emptyMap()); } } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/table/PythonTableFunctionRunnerTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/table/PythonTableFunctionRunnerTest.java index 2c8383cc4f..5c396afa8d 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/table/PythonTableFunctionRunnerTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/table/PythonTableFunctionRunnerTest.java @@ -96,7 +96,8 @@ public class PythonTableFunctionRunnerTest extends AbstractPythonTableFunctionRu pythonFunctionInfo, environmentManager, inputType, - outputType); + outputType, + Collections.emptyMap()); } private AbstractPythonTableFunctionRunner createUDTFRunner( @@ -135,7 +136,7 @@ public class PythonTableFunctionRunnerTest extends AbstractPythonTableFunctionRu RowType inputType, RowType outputType, JobBundleFactory jobBundleFactory) { - super(taskName, resultReceiver, tableFunction, environmentManager, inputType, outputType); + super(taskName, resultReceiver, tableFunction, environmentManager, inputType, outputType, Collections.emptyMap()); this.jobBundleFactory = jobBundleFactory; } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughArrowPythonScalarFunctionRunner.java b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughArrowPythonScalarFunctionRunner.java index 09e4809bd5..ed66123502 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughArrowPythonScalarFunctionRunner.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughArrowPythonScalarFunctionRunner.java @@ -29,6 +29,7 @@ import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Struct; import java.util.ArrayList; import java.util.List; +import java.util.Map; import static org.apache.flink.table.runtime.utils.PythonTestUtils.createMockJobBundleFactory; @@ -49,8 +50,9 @@ public abstract class PassThroughArrowPythonScalarFunctionRunner extends Abs PythonEnvironmentManager environmentManager, RowType inputType, RowType outputType, - int maxArrowBatchSize) { - this(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, maxArrowBatchSize, createMockJobBundleFactory()); + int maxArrowBatchSize, + Map jobOptions) { + this(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, maxArrowBatchSize, jobOptions, createMockJobBundleFactory()); } public PassThroughArrowPythonScalarFunctionRunner( @@ -61,8 +63,9 @@ public abstract class PassThroughArrowPythonScalarFunctionRunner extends Abs RowType inputType, RowType outputType, int maxArrowBatchSize, + Map jobOptions, JobBundleFactory jobBundleFactory) { - super(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, maxArrowBatchSize); + super(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, maxArrowBatchSize, jobOptions); this.jobBundleFactory = jobBundleFactory; this.bufferedInputs = new ArrayList<>(); } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonScalarFunctionRunner.java b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonScalarFunctionRunner.java index 08dd276f77..492eef6c4c 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonScalarFunctionRunner.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonScalarFunctionRunner.java @@ -29,6 +29,7 @@ import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Struct; import java.util.ArrayList; import java.util.List; +import java.util.Map; import static org.apache.flink.table.runtime.utils.PythonTestUtils.createMockJobBundleFactory; @@ -48,8 +49,9 @@ public abstract class PassThroughPythonScalarFunctionRunner extends Abstract PythonFunctionInfo[] scalarFunctions, PythonEnvironmentManager environmentManager, RowType inputType, - RowType outputType) { - this(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, createMockJobBundleFactory()); + RowType outputType, + Map jobOptions) { + this(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, jobOptions, createMockJobBundleFactory()); } public PassThroughPythonScalarFunctionRunner( @@ -59,8 +61,9 @@ public abstract class PassThroughPythonScalarFunctionRunner extends Abstract PythonEnvironmentManager environmentManager, RowType inputType, RowType outputType, + Map jobOptions, JobBundleFactory jobBundleFactory) { - super(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType); + super(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, jobOptions); this.jobBundleFactory = jobBundleFactory; this.bufferedInputs = new ArrayList<>(); } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonBase.scala index 128cf0cff6..5a6efa20c9 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonBase.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonBase.scala @@ -20,7 +20,8 @@ package org.apache.flink.table.planner.plan.nodes.common import org.apache.calcite.rex.{RexCall, RexLiteral, RexNode} import org.apache.calcite.sql.`type`.SqlTypeName -import org.apache.flink.table.api.TableException +import org.apache.flink.configuration.Configuration +import org.apache.flink.table.api.{TableConfig, TableException} import org.apache.flink.table.functions.UserDefinedFunction import org.apache.flink.table.functions.python.{PythonFunction, PythonFunctionInfo} import org.apache.flink.table.planner.functions.utils.{ScalarSqlFunction, TableSqlFunction} @@ -84,4 +85,9 @@ trait CommonPythonBase { } } + protected def getConfig(tableConfig: TableConfig): Configuration = { + val config = new Configuration(tableConfig.getConfiguration) + config.setString("table.exec.timezone", tableConfig.getLocalTimeZone.getId) + config + } } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecPythonCalc.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecPythonCalc.scala index b9eb6a850a..f850663ca6 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecPythonCalc.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecPythonCalc.scala @@ -58,7 +58,7 @@ class BatchExecPythonCalc( inputTransform, calcProgram, "BatchExecPythonCalc", - planner.getTableConfig.getConfiguration) + getConfig(planner.getTableConfig)) ExecNode.setManagedMemoryWeight( ret, getPythonWorkerMemory(planner.getTableConfig.getConfiguration)) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecPythonCorrelate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecPythonCorrelate.scala index b341e35713..cd52714f61 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecPythonCorrelate.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecPythonCorrelate.scala @@ -76,7 +76,7 @@ class BatchExecPythonCorrelate( scan, "BatchExecPythonCorrelate", outputRowType, - planner.getTableConfig.getConfiguration, + getConfig(planner.getTableConfig), joinType) } } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecPythonCalc.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecPythonCalc.scala index 45bf644ade..5b8d8f233a 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecPythonCalc.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecPythonCalc.scala @@ -57,7 +57,7 @@ class StreamExecPythonCalc( inputTransform, calcProgram, "StreamExecPythonCalc", - planner.getTableConfig.getConfiguration) + getConfig(planner.getTableConfig)) if (inputsContainSingleton()) { ret.setParallelism(1) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecPythonCorrelate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecPythonCorrelate.scala index 4ece477540..214268d178 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecPythonCorrelate.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecPythonCorrelate.scala @@ -81,7 +81,7 @@ class StreamExecPythonCorrelate( scan, "StreamExecPythonCorrelate", outputRowType, - planner.getTableConfig.getConfiguration, + getConfig(planner.getTableConfig), joinType) } } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/utils/python/PythonTableUtils.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/utils/python/PythonTableUtils.scala index b1c16d90c1..56e4e21d5c 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/utils/python/PythonTableUtils.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/utils/python/PythonTableUtils.scala @@ -20,7 +20,7 @@ package org.apache.flink.table.planner.utils.python import java.nio.charset.StandardCharsets import java.sql.{Date, Time, Timestamp} -import java.time.{LocalDate, LocalDateTime, LocalTime} +import java.time.{Instant, LocalDate, LocalDateTime, LocalTime} import java.util.TimeZone import java.util.function.BiConsumer @@ -124,6 +124,12 @@ object PythonTableUtils { case c: Int => new Timestamp(c.toLong / 1000) } + case _ if dataType == org.apache.flink.api.common.typeinfo.Types.INSTANT => + (obj: Any) => nullSafeConvert(obj) { + case c: Long => Instant.ofEpochMilli(c / 1000) + case c: Int => Instant.ofEpochMilli(c.toLong / 1000) + } + case _ if dataType == Types.INTERVAL_MILLIS() => (obj: Any) => nullSafeConvert(obj) { case c: Long => c / 1000 case c: Int => c.toLong / 1000 diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonPythonBase.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonPythonBase.scala index 058b074781..3ca4e41b52 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonPythonBase.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonPythonBase.scala @@ -19,7 +19,8 @@ package org.apache.flink.table.plan.nodes import org.apache.calcite.rex.{RexCall, RexLiteral, RexNode} import org.apache.calcite.sql.`type`.SqlTypeName -import org.apache.flink.table.api.TableException +import org.apache.flink.configuration.Configuration +import org.apache.flink.table.api.{TableConfig, TableException} import org.apache.flink.table.functions.UserDefinedFunction import org.apache.flink.table.functions.python.{PythonFunction, PythonFunctionInfo} import org.apache.flink.table.functions.utils.{ScalarSqlFunction, TableSqlFunction} @@ -81,4 +82,10 @@ trait CommonPythonBase { createPythonFunctionInfo(pythonRexCall, inputNodes, tfc.getTableFunction) } } + + protected def getConfig(tableConfig: TableConfig): Configuration = { + val config = new Configuration(tableConfig.getConfiguration) + config.setString("table.exec.timezone", tableConfig.getLocalTimeZone.getId) + config + } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetPythonCalc.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetPythonCalc.scala index e75ed80c10..420faac0c4 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetPythonCalc.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetPythonCalc.scala @@ -81,7 +81,7 @@ class DataSetPythonCalc( val flatMapFunctionOutputRowType = TypeConversions.fromLegacyInfoToDataType( flatMapFunctionResultTypeInfo).getLogicalType.asInstanceOf[RowType] val flatMapFunction = getPythonScalarFunctionFlatMap( - tableEnv.getConfig.getConfiguration, + getConfig(tableEnv.getConfig), flatMapFunctionInputRowType, flatMapFunctionOutputRowType, calcProgram) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamPythonCalc.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamPythonCalc.scala index 37c0eef715..a40516978f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamPythonCalc.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamPythonCalc.scala @@ -84,7 +84,7 @@ class DataStreamPythonCalc( val pythonOperatorOutputRowType = TypeConversions.fromLegacyInfoToDataType( pythonOperatorResultTypeInfo).getLogicalType.asInstanceOf[RowType] val pythonOperator = getPythonScalarFunctionOperator( - planner.getConfig.getConfiguration, + getConfig(planner.getConfig), pythonOperatorInputRowType, pythonOperatorOutputRowType, calcProgram) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamPythonCorrelate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamPythonCorrelate.scala index 1e46c3571b..7703d85db4 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamPythonCorrelate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamPythonCorrelate.scala @@ -95,7 +95,7 @@ class DataStreamPythonCorrelate( val sqlFunction = pythonTableFuncRexCall.getOperator.asInstanceOf[TableSqlFunction] val pythonOperator = getPythonTableFunctionOperator( - planner.getConfig.getConfiguration, + getConfig(planner.getConfig), pythonOperatorInputRowType, pythonOperatorOutputRowType, pythonFunctionInfo, -- Gitee From 0619a5b3e000855f00d11e71ae2f1b9deef05c98 Mon Sep 17 00:00:00 2001 From: Benchao Li Date: Fri, 20 Mar 2020 14:44:19 +0800 Subject: [PATCH 222/885] [FLINK-16220][json] Fix cast exception in JsonRowSerializationSchema when serializing null fields This closes #11180 --- .../json/JsonRowSerializationSchema.java | 6 ++-- .../json/JsonRowSerializationSchemaTest.java | 35 +++++++++++++++++++ 2 files changed, 39 insertions(+), 2 deletions(-) diff --git a/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java b/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java index 284739534e..74418767db 100644 --- a/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java +++ b/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java @@ -360,7 +360,8 @@ public class JsonRowSerializationSchema implements SerializationSchema { return (mapper, reuse, object) -> { ObjectNode node; - if (reuse == null) { + // reuse could be a NullNode if last record is null. + if (reuse == null || reuse.isNull()) { node = mapper.createObjectNode(); } else { node = (ObjectNode) reuse; @@ -382,7 +383,8 @@ public class JsonRowSerializationSchema implements SerializationSchema { return (mapper, reuse, object) -> { ArrayNode node; - if (reuse == null) { + // reuse could be a NullNode if last record is null. + if (reuse == null || reuse.isNull()) { node = mapper.createArrayNode(); } else { node = (ArrayNode) reuse; diff --git a/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java b/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java index 4a0706fe24..378f92b896 100644 --- a/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java +++ b/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java @@ -91,6 +91,41 @@ public class JsonRowSerializationSchemaTest { assertEquals(row2, deserializationSchema.deserialize(bytes)); } + @Test + public void testMultiRowsWithNullValues() throws IOException { + String[] jsons = new String[] { + "{\"svt\":\"2020-02-24T12:58:09.209+0800\"}", + "{\"svt\":\"2020-02-24T12:58:09.209+0800\", \"ops\":{\"id\":\"281708d0-4092-4c21-9233-931950b6eccf\"}, " + + "\"ids\":[1, 2, 3]}", + "{\"svt\":\"2020-02-24T12:58:09.209+0800\"}", + }; + + String[] expected = new String[] { + "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":null,\"ids\":null}", + "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":{\"id\":\"281708d0-4092-4c21-9233-931950b6eccf\"}," + + "\"ids\":[1,2,3]}", + "{\"svt\":\"2020-02-24T12:58:09.209+0800\",\"ops\":null,\"ids\":null}", + }; + + TypeInformation schema = Types.ROW_NAMED( + new String[]{"svt", "ops", "ids"}, + Types.STRING, + Types.ROW_NAMED(new String[]{"id"}, Types.STRING), + Types.PRIMITIVE_ARRAY(Types.INT)); + JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema.Builder(schema) + .build(); + JsonRowSerializationSchema serializationSchema = JsonRowSerializationSchema.builder() + .withTypeInfo(schema) + .build(); + + for (int i = 0; i < jsons.length; i++) { + String json = jsons[i]; + Row row = deserializationSchema.deserialize(json.getBytes()); + String result = new String(serializationSchema.serialize(row)); + assertEquals(expected[i], result); + } + } + @Test public void testNestedSchema() { final TypeInformation rowSchema = Types.ROW_NAMED( -- Gitee From c60d11d40dc288bc467e7414be5fe7bb2f583f17 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Wed, 18 Mar 2020 20:10:17 +0800 Subject: [PATCH 223/885] [FLINK-16653][network][tests] Implement MockResultPartitionWriter base for simplifying tests At the moment there are at-least four implementations of `ResultPartitionWriter` interface used in unit tests. And there are about ten methods to be implemented for `ResultPartitionWriter` and most of them are dummy in tests. When we want to extend the methods for `ResultPartitionWriter`, the above four dummy implementations in tests have to be adjusted as well, to waste a bit efforts. Therefore the MockResultPartitionWriter is proposed to implement the basic dummy methods for `ResultPartitionWriter`, and the previous four instances can all extend it to only implement one or two methods based on specific requirements in tests. It will probably only need to adjust the MockResultPartitionWriter when extending the `ResultPartitionWriter` interface. This closes #11441. --- ...stractCollectingResultPartitionWriter.java | 45 +------- ...AvailabilityTestResultPartitionWriter.java | 57 +--------- .../network/api/writer/RecordWriterTest.java | 105 +----------------- .../partition/MockResultPartitionWriter.java | 91 +++++++++++++++ 4 files changed, 98 insertions(+), 200 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MockResultPartitionWriter.java diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/AbstractCollectingResultPartitionWriter.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/AbstractCollectingResultPartitionWriter.java index f83b6d79e1..6ad31e30fa 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/AbstractCollectingResultPartitionWriter.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/AbstractCollectingResultPartitionWriter.java @@ -18,19 +18,16 @@ package org.apache.flink.runtime.io.network.api.writer; -import org.apache.flink.runtime.io.AvailabilityProvider; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferBuilder; import org.apache.flink.runtime.io.network.buffer.BufferConsumer; import org.apache.flink.runtime.io.network.buffer.BufferProvider; -import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.MockResultPartitionWriter; -import javax.annotation.Nullable; import javax.annotation.concurrent.ThreadSafe; import java.io.IOException; import java.util.ArrayDeque; -import java.util.concurrent.CompletableFuture; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.flink.util.Preconditions.checkState; @@ -39,7 +36,7 @@ import static org.apache.flink.util.Preconditions.checkState; * {@link ResultPartitionWriter} that collects output on the List. */ @ThreadSafe -public abstract class AbstractCollectingResultPartitionWriter implements ResultPartitionWriter { +public abstract class AbstractCollectingResultPartitionWriter extends MockResultPartitionWriter { private final BufferProvider bufferProvider; private final ArrayDeque bufferConsumers = new ArrayDeque<>(); @@ -47,25 +44,6 @@ public abstract class AbstractCollectingResultPartitionWriter implements ResultP this.bufferProvider = checkNotNull(bufferProvider); } - @Override - public void setup() { - } - - @Override - public ResultPartitionID getPartitionId() { - return new ResultPartitionID(); - } - - @Override - public int getNumberOfSubpartitions() { - return 1; - } - - @Override - public int getNumTargetKeyGroups() { - return 1; - } - @Override public BufferBuilder getBufferBuilder() throws IOException, InterruptedException { return bufferProvider.requestBufferBuilderBlocking(); @@ -110,24 +88,5 @@ public abstract class AbstractCollectingResultPartitionWriter implements ResultP flushAll(); } - @Override - public void close() { - } - - @Override - public void fail(@Nullable Throwable throwable) { - throw new UnsupportedOperationException(); - } - - @Override - public void finish() { - throw new UnsupportedOperationException(); - } - - @Override - public CompletableFuture getAvailableFuture() { - return AvailabilityProvider.AVAILABLE; - } - protected abstract void deserializeBuffer(Buffer buffer) throws IOException; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/AvailabilityTestResultPartitionWriter.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/AvailabilityTestResultPartitionWriter.java index 1c96d6ae97..9d86442a4a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/AvailabilityTestResultPartitionWriter.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/AvailabilityTestResultPartitionWriter.java @@ -18,11 +18,7 @@ package org.apache.flink.runtime.io.network.api.writer; -import org.apache.flink.runtime.io.network.buffer.BufferBuilder; -import org.apache.flink.runtime.io.network.buffer.BufferConsumer; -import org.apache.flink.runtime.io.network.partition.ResultPartitionID; - -import javax.annotation.Nullable; +import org.apache.flink.runtime.io.network.partition.MockResultPartitionWriter; import java.util.concurrent.CompletableFuture; @@ -30,7 +26,7 @@ import java.util.concurrent.CompletableFuture; * A specific result partition writer implementation only used to control the output * availability state in tests. */ -public class AvailabilityTestResultPartitionWriter implements ResultPartitionWriter { +public class AvailabilityTestResultPartitionWriter extends MockResultPartitionWriter { /** This state is only valid in the first call of {@link #isAvailable()}. */ private final boolean isAvailable; @@ -44,55 +40,6 @@ public class AvailabilityTestResultPartitionWriter implements ResultPartitionWri this.isAvailable = isAvailable; } - @Override - public void setup() { - } - - @Override - public ResultPartitionID getPartitionId() { - return new ResultPartitionID(); - } - - @Override - public int getNumberOfSubpartitions() { - return 1; - } - - @Override - public int getNumTargetKeyGroups() { - return 1; - } - - @Override - public BufferBuilder getBufferBuilder() { - throw new UnsupportedOperationException(); - } - - @Override - public boolean addBufferConsumer(BufferConsumer bufferConsumer, int targetChannel) { - return true; - } - - @Override - public void flushAll() { - } - - @Override - public void flush(int subpartitionIndex) { - } - - @Override - public void close() { - } - - @Override - public void fail(@Nullable Throwable throwable) { - } - - @Override - public void finish() { - } - @Override public CompletableFuture getAvailableFuture() { return isAvailable ? AVAILABLE : future; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterTest.java index 1cbd8fe0c8..4964d93f47 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterTest.java @@ -40,9 +40,9 @@ import org.apache.flink.runtime.io.network.buffer.BufferPool; import org.apache.flink.runtime.io.network.buffer.BufferProvider; import org.apache.flink.runtime.io.network.buffer.BufferRecycler; import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; +import org.apache.flink.runtime.io.network.partition.MockResultPartitionWriter; import org.apache.flink.runtime.io.network.partition.NoOpResultPartitionConsumableNotifier; import org.apache.flink.runtime.io.network.partition.ResultPartitionBuilder; -import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.runtime.io.network.util.DeserializationUtils; import org.apache.flink.runtime.io.network.util.TestPooledBufferProvider; @@ -63,8 +63,6 @@ import org.junit.rules.TemporaryFolder; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import javax.annotation.Nullable; - import java.io.IOException; import java.util.ArrayDeque; import java.util.ArrayList; @@ -72,13 +70,11 @@ import java.util.List; import java.util.Queue; import java.util.Random; import java.util.concurrent.Callable; -import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; -import static org.apache.flink.runtime.io.AvailabilityProvider.AVAILABLE; import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.buildSingleBuffer; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -531,10 +527,9 @@ public class RecordWriterTest { /** * Partition writer that collects the added buffers/events in multiple queue. */ - static class CollectingPartitionWriter implements ResultPartitionWriter { + static class CollectingPartitionWriter extends MockResultPartitionWriter { private final Queue[] queues; private final BufferProvider bufferProvider; - private final ResultPartitionID partitionId = new ResultPartitionID(); /** * Create the partition writer. @@ -547,25 +542,11 @@ public class RecordWriterTest { this.bufferProvider = bufferProvider; } - @Override - public void setup() { - } - - @Override - public ResultPartitionID getPartitionId() { - return partitionId; - } - @Override public int getNumberOfSubpartitions() { return queues.length; } - @Override - public int getNumTargetKeyGroups() { - return 1; - } - @Override public BufferBuilder getBufferBuilder() throws IOException, InterruptedException { return bufferProvider.requestBufferBuilderBlocking(); @@ -575,33 +556,6 @@ public class RecordWriterTest { public boolean addBufferConsumer(BufferConsumer buffer, int targetChannel) throws IOException { return queues[targetChannel].add(buffer); } - - @Override - public void flushAll() { - } - - @Override - public void flush(int subpartitionIndex) { - } - - @Override - public void fail(@Nullable Throwable throwable) { - throw new UnsupportedOperationException(); - } - - @Override - public void finish() { - throw new UnsupportedOperationException(); - } - - @Override - public CompletableFuture getAvailableFuture() { - return AVAILABLE; - } - - @Override - public void close() { - } } static BufferOrEvent parseBuffer(BufferConsumer bufferConsumer, int targetChannel) throws IOException { @@ -619,70 +573,17 @@ public class RecordWriterTest { /** * Partition writer that recycles all received buffers and does no further processing. */ - private static class RecyclingPartitionWriter implements ResultPartitionWriter { + private static class RecyclingPartitionWriter extends MockResultPartitionWriter { private final BufferProvider bufferProvider; - private final ResultPartitionID partitionId = new ResultPartitionID(); private RecyclingPartitionWriter(BufferProvider bufferProvider) { this.bufferProvider = bufferProvider; } - @Override - public void setup() { - } - - @Override - public ResultPartitionID getPartitionId() { - return partitionId; - } - - @Override - public int getNumberOfSubpartitions() { - return 1; - } - - @Override - public int getNumTargetKeyGroups() { - return 1; - } - @Override public BufferBuilder getBufferBuilder() throws IOException, InterruptedException { return bufferProvider.requestBufferBuilderBlocking(); } - - @Override - public boolean addBufferConsumer(BufferConsumer bufferConsumer, int targetChannel) throws IOException { - bufferConsumer.close(); - return true; - } - - @Override - public void flushAll() { - } - - @Override - public void flush(int subpartitionIndex) { - } - - @Override - public void fail(@Nullable Throwable throwable) { - throw new UnsupportedOperationException(); - } - - @Override - public void finish() { - throw new UnsupportedOperationException(); - } - - @Override - public CompletableFuture getAvailableFuture() { - return AVAILABLE; - } - - @Override - public void close() { - } } private static class ByteArrayIO implements IOReadableWritable { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MockResultPartitionWriter.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MockResultPartitionWriter.java new file mode 100644 index 0000000000..fd6c1f8e37 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MockResultPartitionWriter.java @@ -0,0 +1,91 @@ +/* + * 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.runtime.io.network.partition; + +import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; +import org.apache.flink.runtime.io.network.buffer.BufferBuilder; +import org.apache.flink.runtime.io.network.buffer.BufferConsumer; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.concurrent.CompletableFuture; + +/** + * Dummy behaviours of {@link ResultPartitionWriter} for test purpose. + */ +public class MockResultPartitionWriter implements ResultPartitionWriter { + + private final ResultPartitionID partitionId = new ResultPartitionID(); + + @Override + public void setup() { + } + + @Override + public ResultPartitionID getPartitionId() { + return partitionId; + } + + @Override + public int getNumberOfSubpartitions() { + return 1; + } + + @Override + public int getNumTargetKeyGroups() { + return 1; + } + + @Override + public boolean addBufferConsumer(BufferConsumer bufferConsumer, int targetChannel) throws IOException { + bufferConsumer.close(); + return true; + } + + @Override + public BufferBuilder getBufferBuilder() throws IOException, InterruptedException { + throw new UnsupportedOperationException(); + } + + @Override + public void flushAll() { + } + + @Override + public void flush(int subpartitionIndex) { + } + + @Override + public void fail(@Nullable Throwable throwable) { + } + + @Override + public void finish() { + } + + @Override + public CompletableFuture getAvailableFuture() { + return AVAILABLE; + } + + @Override + public void close() { + } +} -- Gitee From 0bc38ba058165aa6751e7e4c654e37203df3aa35 Mon Sep 17 00:00:00 2001 From: Gao Yun Date: Tue, 17 Mar 2020 15:20:39 +0800 Subject: [PATCH 224/885] [FLINK-15962][network] Reduce the default chunk size to 4M in netty stack --- .../test_netty_shuffle_memory_control.sh | 4 ++-- .../runtime/io/network/netty/NettyBufferPool.java | 14 ++++++++------ 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/flink-end-to-end-tests/test-scripts/test_netty_shuffle_memory_control.sh b/flink-end-to-end-tests/test-scripts/test_netty_shuffle_memory_control.sh index cefc4cd447..71be5e1cbc 100755 --- a/flink-end-to-end-tests/test-scripts/test_netty_shuffle_memory_control.sh +++ b/flink-end-to-end-tests/test-scripts/test_netty_shuffle_memory_control.sh @@ -33,8 +33,8 @@ set_config_key "taskmanager.numberOfTaskSlots" "20" # Sets only one arena per TM for boosting the netty internal memory overhead. set_config_key "taskmanager.network.netty.num-arenas" "1" -# Limits the direct memory to be one chunk (16M) plus some margins. -set_config_key "taskmanager.memory.framework.off-heap.size" "20m" +# Limits the direct memory to be one chunk (4M) plus some margins. +set_config_key "taskmanager.memory.framework.off-heap.size" "7m" # Starts the cluster which includes one TaskManager. start_cluster diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyBufferPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyBufferPool.java index 6d2a6c8828..87bf7f6e12 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyBufferPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyBufferPool.java @@ -53,7 +53,7 @@ public class NettyBufferPool extends PooledByteBufAllocator { /** * Arenas allocate chunks of pageSize << maxOrder bytes. With these defaults, this results in - * chunks of 16 MB. + * chunks of 4 MB. * * @see #MAX_ORDER */ @@ -61,11 +61,13 @@ public class NettyBufferPool extends PooledByteBufAllocator { /** * Arenas allocate chunks of pageSize << maxOrder bytes. With these defaults, this results in - * chunks of 16 MB. + * chunks of 4 MB, which is smaller than the previous default (16 MB) to further reduce the + * netty memory overhead. According to the test result, after introducing client side zero-copy + * in FLINK-10742, 4 MB is enough to support large-scale netty shuffle. * * @see #PAGE_SIZE */ - private static final int MAX_ORDER = 11; + private static final int MAX_ORDER = 9; /** * Creates Netty's buffer pool with the specified number of direct arenas. @@ -78,8 +80,8 @@ public class NettyBufferPool extends PooledByteBufAllocator { PREFER_DIRECT, // No heap arenas, please. 0, - // Number of direct arenas. Each arena allocates a chunk of 16 MB, i.e. - // we allocate numDirectArenas * 16 MB of direct memory. This can grow + // Number of direct arenas. Each arena allocates a chunk of 4 MB, i.e. + // we allocate numDirectArenas * 4 MB of direct memory. This can grow // to multiple chunks per arena during runtime, but this should only // happen with a large amount of connections per task manager. We // control the memory allocations with low/high watermarks when writing @@ -92,7 +94,7 @@ public class NettyBufferPool extends PooledByteBufAllocator { this.numberOfArenas = numberOfArenas; // Arenas allocate chunks of pageSize << maxOrder bytes. With these - // defaults, this results in chunks of 16 MB. + // defaults, this results in chunks of 4 MB. this.chunkSize = PAGE_SIZE << MAX_ORDER; -- Gitee From b226cbd1f595b405f5da376fa3d22e1850fbf4e1 Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Tue, 17 Mar 2020 10:24:39 +0100 Subject: [PATCH 225/885] [FLINK-16633][AZP] Fix builds without s3 credentials Problem: Builds on Azure where failing if S3 credentails were not provided, because the syntax in the pipeline definition was incorrect. Solution: Use a way to access secret variables that leads to an empty value (instead of the variable name). Also, extend the check in the unit tests to "variable empty" instead of just "variable not set". --- azure-pipelines.yml | 8 +++++++- .../apache/flink/testutils/s3/S3TestCredentials.java | 11 +++++++++-- tools/azure-pipelines/build-apache-repo.yml | 3 +++ tools/azure-pipelines/jobs-template.yml | 12 ++++++------ 4 files changed, 25 insertions(+), 9 deletions(-) diff --git a/azure-pipelines.yml b/azure-pipelines.yml index 1257a49e62..470ef35abd 100644 --- a/azure-pipelines.yml +++ b/azure-pipelines.yml @@ -40,13 +40,19 @@ resources: - container: flink-build-container image: rmetzger/flink-ci:ubuntu-amd64-3528acd -# See tools/azure-pipelines/jobs-template.yml for a short summary of the caching +# Define variables: +# - See tools/azure-pipelines/jobs-template.yml for a short summary of the caching +# - See https://stackoverflow.com/questions/60742105/how-can-i-access-a-secret-value-from-an-azure-pipelines-expression +# to understand why the secrets are handled like this variables: MAVEN_CACHE_FOLDER: $(Pipeline.Workspace)/.m2/repository MAVEN_OPTS: '-Dmaven.repo.local=$(MAVEN_CACHE_FOLDER)' CACHE_KEY: maven | $(Agent.OS) | **/pom.xml, !**/target/** CACHE_FALLBACK_KEY: maven | $(Agent.OS) CACHE_FLINK_DIR: $(Pipeline.Workspace)/flink_cache + SECRET_S3_BUCKET: $[variables.IT_CASE_S3_BUCKET] + SECRET_S3_ACCESS_KEY: $[variables.IT_CASE_S3_ACCESS_KEY] + SECRET_S3_SECRET_KEY: $[variables.IT_CASE_S3_SECRET_KEY] jobs: diff --git a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/s3/S3TestCredentials.java b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/s3/S3TestCredentials.java index b1a0aecf07..5642e4803d 100644 --- a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/s3/S3TestCredentials.java +++ b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/s3/S3TestCredentials.java @@ -43,8 +43,15 @@ public class S3TestCredentials { * Checks whether S3 test credentials are available in the environment variables * of this JVM. */ - public static boolean credentialsAvailable() { - return S3_TEST_BUCKET != null && S3_TEST_ACCESS_KEY != null && S3_TEST_SECRET_KEY != null; + private static boolean credentialsAvailable() { + return isNotEmpty(S3_TEST_BUCKET) && isNotEmpty(S3_TEST_ACCESS_KEY) && isNotEmpty(S3_TEST_SECRET_KEY); + } + + /** + * Checks if a String is not null and not empty. + */ + private static boolean isNotEmpty(@Nullable String str) { + return str != null && !str.isEmpty(); } /** diff --git a/tools/azure-pipelines/build-apache-repo.yml b/tools/azure-pipelines/build-apache-repo.yml index 05ee7e9210..e881a7356a 100644 --- a/tools/azure-pipelines/build-apache-repo.yml +++ b/tools/azure-pipelines/build-apache-repo.yml @@ -46,6 +46,9 @@ variables: CACHE_KEY: maven | $(Agent.OS) | **/pom.xml, !**/target/** CACHE_FALLBACK_KEY: maven | $(Agent.OS) CACHE_FLINK_DIR: $(Pipeline.Workspace)/flink_cache + SECRET_S3_BUCKET: $[variables.IT_CASE_S3_BUCKET] + SECRET_S3_ACCESS_KEY: $[variables.IT_CASE_S3_ACCESS_KEY] + SECRET_S3_SECRET_KEY: $[variables.IT_CASE_S3_SECRET_KEY] stages: # CI / PR triggered stage: diff --git a/tools/azure-pipelines/jobs-template.yml b/tools/azure-pipelines/jobs-template.yml index 8f654b64cf..8a2a832117 100644 --- a/tools/azure-pipelines/jobs-template.yml +++ b/tools/azure-pipelines/jobs-template.yml @@ -123,9 +123,9 @@ jobs: - script: STAGE=test ${{parameters.environment}} ./tools/azure_controller.sh $(module) displayName: Test - $(module) env: - IT_CASE_S3_BUCKET: $(IT_CASE_S3_BUCKET) - IT_CASE_S3_ACCESS_KEY: $(IT_CASE_S3_ACCESS_KEY) - IT_CASE_S3_SECRET_KEY: $(IT_CASE_S3_SECRET_KEY) + IT_CASE_S3_BUCKET: $(SECRET_S3_BUCKET) + IT_CASE_S3_ACCESS_KEY: $(SECRET_S3_ACCESS_KEY) + IT_CASE_S3_SECRET_KEY: $(SECRET_S3_SECRET_KEY) - task: PublishTestResults@2 inputs: @@ -175,9 +175,9 @@ jobs: - script: ${{parameters.environment}} FLINK_DIR=`pwd`/build-target flink-end-to-end-tests/run-nightly-tests.sh displayName: Run e2e tests env: - IT_CASE_S3_BUCKET: $(IT_CASE_S3_BUCKET) - IT_CASE_S3_ACCESS_KEY: $(IT_CASE_S3_ACCESS_KEY) - IT_CASE_S3_SECRET_KEY: $(IT_CASE_S3_SECRET_KEY) + IT_CASE_S3_BUCKET: $(SECRET_S3_BUCKET) + IT_CASE_S3_ACCESS_KEY: $(SECRET_S3_ACCESS_KEY) + IT_CASE_S3_SECRET_KEY: $(SECRET_S3_SECRET_KEY) # upload debug artifacts - task: PublishPipelineArtifact@1 condition: and(succeededOrFailed(), not(eq('$(ARTIFACT_DIR)', ''))) -- Gitee From 96dc51f411e15c49b71dcb38860b1d2b7333dc13 Mon Sep 17 00:00:00 2001 From: shuai-xu Date: Mon, 24 Feb 2020 18:20:30 +0800 Subject: [PATCH 226/885] [FLINK-16011] fix the bug that with will not effect if not in the end of pattern --- .../apache/flink/cep/nfa/compiler/NFACompiler.java | 2 +- .../flink/cep/nfa/compiler/NFACompilerTest.java | 11 +++++++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java index bc692f2bc0..68a6a7c2cf 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java @@ -303,7 +303,7 @@ public class NFACompiler { currentPattern = currentPattern.getPrevious(); final Time currentWindowTime = currentPattern.getWindowTime(); - if (currentWindowTime != null && currentWindowTime.toMilliseconds() < windowTime) { + if (currentWindowTime != null && (windowTime == 0 || currentWindowTime.toMilliseconds() < windowTime)) { // the window time is the global minimum of all window times of each state windowTime = currentWindowTime.toMilliseconds(); } diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java index 821da031c6..955f6f516d 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java @@ -29,6 +29,7 @@ import org.apache.flink.cep.nfa.aftermatch.AfterMatchSkipStrategy; import org.apache.flink.cep.pattern.MalformedPatternException; import org.apache.flink.cep.pattern.Pattern; import org.apache.flink.cep.pattern.conditions.SimpleCondition; +import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.util.TestLogger; import org.apache.flink.shaded.guava18.com.google.common.collect.Sets; @@ -238,4 +239,14 @@ public class NFACompilerTest extends TestLogger { assertThat(NFACompiler.canProduceEmptyMatches(Pattern.begin("a").oneOrMore()), is(false)); assertThat(NFACompiler.canProduceEmptyMatches(Pattern.begin("a").oneOrMore().next("b").optional()), is(false)); } + + @Test + public void testWindowTimeCorrectlySet() { + Pattern pattern = Pattern.begin("start").followedBy("middle").within(Time.seconds(10)) + .followedBy("then").within(Time.seconds(20)).followedBy("end"); + + NFACompiler.NFAFactoryCompiler factory = new NFACompiler.NFAFactoryCompiler(pattern); + factory.compileFactory(); + assertEquals(10000, factory.getWindowTime()); + } } -- Gitee From 1be88b13cfb14445aaf72635bc51072264f0c32d Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 19 Mar 2020 16:29:14 +0100 Subject: [PATCH 227/885] [FLINK-16011] Distinguish between no time window and 0 length time windwo in NFACompiler This closes #11200. --- .../apache/flink/cep/nfa/compiler/NFACompiler.java | 12 +++++++----- .../flink/cep/nfa/compiler/NFACompilerTest.java | 12 +++++++++++- 2 files changed, 18 insertions(+), 6 deletions(-) diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java index 68a6a7c2cf..7b6e727ed8 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java @@ -43,6 +43,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.Stack; @@ -129,7 +130,7 @@ public class NFACompiler { private final Map> stopStates = new HashMap<>(); private final List> states = new ArrayList<>(); - private long windowTime = 0; + private Optional windowTime; private GroupPattern currentGroupPattern; private Map, Boolean> firstOfLoopMap = new HashMap<>(); private Pattern currentPattern; @@ -140,6 +141,7 @@ public class NFACompiler { NFAFactoryCompiler(final Pattern pattern) { this.currentPattern = pattern; afterMatchSkipStrategy = pattern.getAfterMatchSkipStrategy(); + windowTime = Optional.empty(); } /** @@ -172,7 +174,7 @@ public class NFACompiler { } long getWindowTime() { - return windowTime; + return windowTime.orElse(0L); } /** @@ -265,7 +267,7 @@ public class NFACompiler { */ private State createEndingState() { State endState = createState(ENDING_STATE_NAME, State.StateType.Final); - windowTime = currentPattern.getWindowTime() != null ? currentPattern.getWindowTime().toMilliseconds() : 0L; + windowTime = Optional.ofNullable(currentPattern.getWindowTime()).map(Time::toMilliseconds); return endState; } @@ -303,9 +305,9 @@ public class NFACompiler { currentPattern = currentPattern.getPrevious(); final Time currentWindowTime = currentPattern.getWindowTime(); - if (currentWindowTime != null && (windowTime == 0 || currentWindowTime.toMilliseconds() < windowTime)) { + if (currentWindowTime != null && currentWindowTime.toMilliseconds() < windowTime.orElse(Long.MAX_VALUE)) { // the window time is the global minimum of all window times of each state - windowTime = currentWindowTime.toMilliseconds(); + windowTime = Optional.of(currentWindowTime.toMilliseconds()); } } return lastSink; diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java index 955f6f516d..2275fdd85c 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java @@ -245,8 +245,18 @@ public class NFACompilerTest extends TestLogger { Pattern pattern = Pattern.begin("start").followedBy("middle").within(Time.seconds(10)) .followedBy("then").within(Time.seconds(20)).followedBy("end"); - NFACompiler.NFAFactoryCompiler factory = new NFACompiler.NFAFactoryCompiler(pattern); + NFACompiler.NFAFactoryCompiler factory = new NFACompiler.NFAFactoryCompiler<>(pattern); factory.compileFactory(); assertEquals(10000, factory.getWindowTime()); } + + @Test + public void testMultipleWindowTimeWithZeroLength() { + Pattern pattern = Pattern.begin("start").followedBy("middle").within(Time.seconds(10)) + .followedBy("then").within(Time.seconds(0)).followedBy("end"); + + NFACompiler.NFAFactoryCompiler factory = new NFACompiler.NFAFactoryCompiler<>(pattern); + factory.compileFactory(); + assertEquals(0, factory.getWindowTime()); + } } -- Gitee From 8a27bd91474d933dda6a0d99c79f303ec9f4ef58 Mon Sep 17 00:00:00 2001 From: huangxingbo Date: Fri, 20 Mar 2020 17:41:38 +0800 Subject: [PATCH 228/885] [FLINK-16691][python][docs] Improve Python UDF documentation to remind users to install PyFlink on the cluster This cloese #11462. --- docs/dev/table/python/python_udfs.md | 2 ++ docs/dev/table/python/python_udfs.zh.md | 2 ++ 2 files changed, 4 insertions(+) diff --git a/docs/dev/table/python/python_udfs.md b/docs/dev/table/python/python_udfs.md index 44a30ffb0c..fd4a99d655 100644 --- a/docs/dev/table/python/python_udfs.md +++ b/docs/dev/table/python/python_udfs.md @@ -24,6 +24,8 @@ under the License. User-defined functions are important features, because they significantly extend the expressiveness of Python Table API programs. +**NOTE:** Python UDF execution requires Python3.5+ with PyFlink installed. It's required on both the client side and the cluster side. + * This will be replaced by the TOC {:toc} diff --git a/docs/dev/table/python/python_udfs.zh.md b/docs/dev/table/python/python_udfs.zh.md index 7acba1fd57..11f896c98e 100644 --- a/docs/dev/table/python/python_udfs.zh.md +++ b/docs/dev/table/python/python_udfs.zh.md @@ -24,6 +24,8 @@ under the License. User-defined functions are important features, because they significantly extend the expressiveness of Python Table API programs. +**NOTE:** Python UDF execution requires Python3.5+ with PyFlink installed. It's required on both the client side and the cluster side. + * This will be replaced by the TOC {:toc} -- Gitee From 74b8bdee9fb0bf7cfc27ca8d992dac2a07473a0c Mon Sep 17 00:00:00 2001 From: "bowen.li" Date: Fri, 6 Mar 2020 14:05:27 -0800 Subject: [PATCH 229/885] [FLINK-16471][jdbc] develop JDBCCatalog and PostgresCatalog closes #11336 --- .../src/test/resources/log4j2-test.properties | 28 -- flink-connectors/flink-jdbc/pom.xml | 36 +- .../io/jdbc/catalog/AbstractJDBCCatalog.java | 277 +++++++++++++++ .../api/java/io/jdbc/catalog/JDBCCatalog.java | 84 +++++ .../io/jdbc/catalog/JDBCCatalogUtils.java | 54 +++ .../java/io/jdbc/catalog/PostgresCatalog.java | 323 +++++++++++++++++ .../io/jdbc/catalog/PostgresTablePath.java | 95 +++++ .../java/io/jdbc/dialect/JDBCDialects.java | 10 +- .../io/jdbc/catalog/JDBCCatalogUtilsTest.java | 44 +++ .../jdbc/catalog/PostgresCatalogITCase.java | 325 ++++++++++++++++++ .../jdbc/catalog/PostgresTablePathTest.java | 33 ++ 11 files changed, 1275 insertions(+), 34 deletions(-) delete mode 100644 flink-connectors/flink-connector-elasticsearch2/src/test/resources/log4j2-test.properties create mode 100644 flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/AbstractJDBCCatalog.java create mode 100644 flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/JDBCCatalog.java create mode 100644 flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/JDBCCatalogUtils.java create mode 100644 flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalog.java create mode 100644 flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresTablePath.java create mode 100644 flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/JDBCCatalogUtilsTest.java create mode 100644 flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogITCase.java create mode 100644 flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresTablePathTest.java diff --git a/flink-connectors/flink-connector-elasticsearch2/src/test/resources/log4j2-test.properties b/flink-connectors/flink-connector-elasticsearch2/src/test/resources/log4j2-test.properties deleted file mode 100644 index 835c2ec9a3..0000000000 --- a/flink-connectors/flink-connector-elasticsearch2/src/test/resources/log4j2-test.properties +++ /dev/null @@ -1,28 +0,0 @@ -################################################################################ -# 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. -################################################################################ - -# Set root logger level to OFF to not flood build logs -# set manually to INFO for debugging purposes -rootLogger.level = OFF -rootLogger.appenderRef.test.ref = TestLogger - -appender.testlogger.name = TestLogger -appender.testlogger.type = CONSOLE -appender.testlogger.target = SYSTEM_ERR -appender.testlogger.layout.type = PatternLayout -appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n diff --git a/flink-connectors/flink-jdbc/pom.xml b/flink-connectors/flink-jdbc/pom.xml index 3e83311d31..cb7afab76b 100644 --- a/flink-connectors/flink-jdbc/pom.xml +++ b/flink-connectors/flink-jdbc/pom.xml @@ -35,6 +35,11 @@ under the License. jar + + 42.2.10 + 0.13.3 + + @@ -53,13 +58,17 @@ under the License. provided + + - org.apache.derby - derby - 10.14.2.0 - test + org.postgresql + postgresql + ${postgres.version} + provided + + org.apache.flink flink-test-utils_${scala.binary.version} @@ -89,5 +98,24 @@ under the License. ${project.version} test + + + + + com.opentable.components + otj-pg-embedded + ${otj-pg-embedded.version} + test + + + + + + org.apache.derby + derby + 10.14.2.0 + test + + diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/AbstractJDBCCatalog.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/AbstractJDBCCatalog.java new file mode 100644 index 0000000000..523de83d22 --- /dev/null +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/AbstractJDBCCatalog.java @@ -0,0 +1,277 @@ +/* + * 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.api.java.io.jdbc.catalog; + +import org.apache.flink.api.java.io.jdbc.JDBCTableSourceSinkFactory; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.AbstractCatalog; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogDatabase; +import org.apache.flink.table.catalog.CatalogFunction; +import org.apache.flink.table.catalog.CatalogPartition; +import org.apache.flink.table.catalog.CatalogPartitionSpec; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; +import org.apache.flink.table.catalog.exceptions.FunctionAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.FunctionNotExistException; +import org.apache.flink.table.catalog.exceptions.PartitionAlreadyExistsException; +import org.apache.flink.table.catalog.exceptions.PartitionNotExistException; +import org.apache.flink.table.catalog.exceptions.PartitionSpecInvalidException; +import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException; +import org.apache.flink.table.catalog.exceptions.TablePartitionedException; +import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; +import org.apache.flink.table.catalog.stats.CatalogTableStatistics; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.factories.TableFactory; +import org.apache.flink.util.StringUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.util.Collections; +import java.util.List; +import java.util.Optional; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * Abstract catalog for any JDBC catalogs. + */ +public abstract class AbstractJDBCCatalog extends AbstractCatalog { + + private static final Logger LOG = LoggerFactory.getLogger(AbstractJDBCCatalog.class); + + protected final String username; + protected final String pwd; + protected final String baseUrl; + protected final String defaultUrl; + + public AbstractJDBCCatalog(String catalogName, String defaultDatabase, String username, String pwd, String baseUrl) { + super(catalogName, defaultDatabase); + + checkArgument(!StringUtils.isNullOrWhitespaceOnly(username)); + checkArgument(!StringUtils.isNullOrWhitespaceOnly(pwd)); + checkArgument(!StringUtils.isNullOrWhitespaceOnly(baseUrl)); + + JDBCCatalogUtils.validateJDBCUrl(baseUrl); + + this.username = username; + this.pwd = pwd; + this.baseUrl = baseUrl.endsWith("/") ? baseUrl : baseUrl + "/"; + this.defaultUrl = baseUrl + defaultDatabase; + } + + @Override + public void open() throws CatalogException { + // test connection, fail early if we cannot connect to database + try (Connection conn = DriverManager.getConnection(defaultUrl, username, pwd)) { + } catch (SQLException e) { + throw new ValidationException( + String.format("Failed connecting to %s via JDBC.", defaultUrl), e); + } + + LOG.info("Catalog {} established connection to {}", getName(), defaultUrl); + } + + @Override + public void close() throws CatalogException { + LOG.info("Catalog {} closing", getName()); + } + + // ------ table factory ------ + + public Optional getTableFactory() { + return Optional.of(new JDBCTableSourceSinkFactory()); + } + + // ------ databases ------ + + @Override + public boolean databaseExists(String databaseName) throws CatalogException { + checkArgument(!StringUtils.isNullOrWhitespaceOnly(databaseName)); + + return listDatabases().contains(databaseName); + } + + @Override + public void createDatabase(String name, CatalogDatabase database, boolean ignoreIfExists) throws DatabaseAlreadyExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade) throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterDatabase(String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists) throws DatabaseNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + // ------ tables and views ------ + + @Override + public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists) throws TableNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists) throws TableNotExistException, TableAlreadyExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void createTable(ObjectPath tablePath, CatalogBaseTable table, boolean ignoreIfExists) throws TableAlreadyExistException, DatabaseNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterTable(ObjectPath tablePath, CatalogBaseTable newTable, boolean ignoreIfNotExists) throws TableNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public List listViews(String databaseName) throws DatabaseNotExistException, CatalogException { + return Collections.emptyList(); + } + + // ------ partitions ------ + + @Override + public List listPartitions(ObjectPath tablePath) throws TableNotExistException, TableNotPartitionedException, CatalogException { + return Collections.emptyList(); + } + + @Override + public List listPartitions(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws TableNotExistException, TableNotPartitionedException, CatalogException { + return Collections.emptyList(); + } + + @Override + public List listPartitionsByFilter(ObjectPath tablePath, List filters) throws TableNotExistException, TableNotPartitionedException, CatalogException { + return Collections.emptyList(); + } + + @Override + public CatalogPartition getPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws PartitionNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public boolean partitionExists(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void createPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec, CatalogPartition partition, boolean ignoreIfExists) throws TableNotExistException, TableNotPartitionedException, PartitionSpecInvalidException, PartitionAlreadyExistsException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void dropPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec, boolean ignoreIfNotExists) throws PartitionNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec, CatalogPartition newPartition, boolean ignoreIfNotExists) throws PartitionNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + // ------ functions ------ + + @Override + public List listFunctions(String dbName) throws DatabaseNotExistException, CatalogException { + return Collections.emptyList(); + } + + @Override + public CatalogFunction getFunction(ObjectPath functionPath) throws FunctionNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public boolean functionExists(ObjectPath functionPath) throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void createFunction(ObjectPath functionPath, CatalogFunction function, boolean ignoreIfExists) throws FunctionAlreadyExistException, DatabaseNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterFunction(ObjectPath functionPath, CatalogFunction newFunction, boolean ignoreIfNotExists) throws FunctionNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void dropFunction(ObjectPath functionPath, boolean ignoreIfNotExists) throws FunctionNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + // ------ stats ------ + + @Override + public CatalogTableStatistics getTableStatistics(ObjectPath tablePath) throws TableNotExistException, CatalogException { + return CatalogTableStatistics.UNKNOWN; + } + + @Override + public CatalogColumnStatistics getTableColumnStatistics(ObjectPath tablePath) throws TableNotExistException, CatalogException { + return CatalogColumnStatistics.UNKNOWN; + } + + @Override + public CatalogTableStatistics getPartitionStatistics(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws PartitionNotExistException, CatalogException { + return CatalogTableStatistics.UNKNOWN; + } + + @Override + public CatalogColumnStatistics getPartitionColumnStatistics(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws PartitionNotExistException, CatalogException { + return CatalogColumnStatistics.UNKNOWN; + } + + @Override + public void alterTableStatistics(ObjectPath tablePath, CatalogTableStatistics tableStatistics, boolean ignoreIfNotExists) throws TableNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterTableColumnStatistics(ObjectPath tablePath, CatalogColumnStatistics columnStatistics, boolean ignoreIfNotExists) throws TableNotExistException, CatalogException, TablePartitionedException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterPartitionStatistics(ObjectPath tablePath, CatalogPartitionSpec partitionSpec, CatalogTableStatistics partitionStatistics, boolean ignoreIfNotExists) throws PartitionNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterPartitionColumnStatistics(ObjectPath tablePath, CatalogPartitionSpec partitionSpec, CatalogColumnStatistics columnStatistics, boolean ignoreIfNotExists) throws PartitionNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } +} diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/JDBCCatalog.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/JDBCCatalog.java new file mode 100644 index 0000000000..629412c744 --- /dev/null +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/JDBCCatalog.java @@ -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.api.java.io.jdbc.catalog; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogDatabase; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; + +/** + * Catalogs for relational databases via JDBC. + */ +@PublicEvolving +public class JDBCCatalog extends AbstractJDBCCatalog { + + private static final Logger LOG = LoggerFactory.getLogger(JDBCCatalog.class); + + private final Catalog internal; + + public JDBCCatalog(String catalogName, String defaultDatabase, String username, String pwd, String baseUrl) { + super(catalogName, defaultDatabase, username, pwd, baseUrl); + + internal = JDBCCatalogUtils.createCatalog(catalogName, defaultDatabase, username, pwd, baseUrl); + } + + // ------ databases ----- + + @Override + public List listDatabases() throws CatalogException { + return internal.listDatabases(); + } + + @Override + public CatalogDatabase getDatabase(String databaseName) throws DatabaseNotExistException, CatalogException { + return internal.getDatabase(databaseName); + } + + // ------ tables and views ------ + + @Override + public List listTables(String databaseName) throws DatabaseNotExistException, CatalogException { + return internal.listTables(databaseName); + } + + @Override + public CatalogBaseTable getTable(ObjectPath tablePath) throws TableNotExistException, CatalogException { + return internal.getTable(tablePath); + } + + @Override + public boolean tableExists(ObjectPath tablePath) throws CatalogException { + try { + return databaseExists(tablePath.getDatabaseName()) && + listTables(tablePath.getDatabaseName()).contains(tablePath.getObjectName()); + } catch (DatabaseNotExistException e) { + return false; + } + } +} diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/JDBCCatalogUtils.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/JDBCCatalogUtils.java new file mode 100644 index 0000000000..b9e3a19376 --- /dev/null +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/JDBCCatalogUtils.java @@ -0,0 +1,54 @@ +/* + * 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.api.java.io.jdbc.catalog; + +import org.apache.flink.api.java.io.jdbc.dialect.JDBCDialect; +import org.apache.flink.api.java.io.jdbc.dialect.JDBCDialects; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * Utils for {@link JDBCCatalog}. + */ +public class JDBCCatalogUtils { + /** + * URL has to be without database, like "jdbc:postgresql://localhost:5432/" or "jdbc:postgresql://localhost:5432" + * rather than "jdbc:postgresql://localhost:5432/db". + */ + public static void validateJDBCUrl(String url) { + String[] parts = url.trim().split("\\/+"); + + checkArgument(parts.length == 2); + } + + /** + * Create catalog instance from given information. + */ + public static AbstractJDBCCatalog createCatalog(String catalogName, String defaultDatabase, String username, String pwd, String baseUrl) { + JDBCDialect dialect = JDBCDialects.get(baseUrl).get(); + + if (dialect instanceof JDBCDialects.PostgresDialect) { + return new PostgresCatalog(catalogName, defaultDatabase, username, pwd, baseUrl); + } else { + throw new UnsupportedOperationException( + String.format("Catalog for '%s' is not supported yet.", dialect) + ); + } + } +} diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalog.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalog.java new file mode 100644 index 0000000000..d12f25473e --- /dev/null +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalog.java @@ -0,0 +1,323 @@ +/* + * 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.api.java.io.jdbc.catalog; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogDatabase; +import org.apache.flink.table.catalog.CatalogDatabaseImpl; +import org.apache.flink.table.catalog.CatalogTableImpl; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.table.types.DataType; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * Catalog for PostgreSQL. + */ +@Internal +public class PostgresCatalog extends AbstractJDBCCatalog { + + private static final Logger LOG = LoggerFactory.getLogger(PostgresCatalog.class); + + public static final String POSTGRES_TABLE_TYPE = "postgres"; + + public static final String DEFAULT_DATABASE = "postgres"; + + // ------ Postgres default objects that shouldn't be exposed to users ------ + + private static final Set builtinDatabases = new HashSet() {{ + add("template0"); + add("template1"); + }}; + + private static final Set builtinSchemas = new HashSet() {{ + add("pg_toast"); + add("pg_temp_1"); + add("pg_toast_temp_1"); + add("pg_catalog"); + add("information_schema"); + }}; + + protected PostgresCatalog(String catalogName, String defaultDatabase, String username, String pwd, String baseUrl) { + super(catalogName, defaultDatabase, username, pwd, baseUrl); + } + + // ------ databases ------ + + @Override + public List listDatabases() throws CatalogException { + List pgDatabases = new ArrayList<>(); + + try (Connection conn = DriverManager.getConnection(defaultUrl, username, pwd)) { + + PreparedStatement ps = conn.prepareStatement("SELECT datname FROM pg_database;"); + + ResultSet rs = ps.executeQuery(); + + while (rs.next()) { + String dbName = rs.getString(1); + if (!builtinDatabases.contains(dbName)) { + pgDatabases.add(rs.getString(1)); + } + } + + return pgDatabases; + } catch (Exception e) { + throw new CatalogException( + String.format("Failed listing database in catalog %s", getName()), e); + } + } + + @Override + public CatalogDatabase getDatabase(String databaseName) throws DatabaseNotExistException, CatalogException { + if (listDatabases().contains(databaseName)) { + return new CatalogDatabaseImpl(Collections.emptyMap(), null); + } else { + throw new DatabaseNotExistException(getName(), databaseName); + } + } + + // ------ tables ------ + + @Override + public List listTables(String databaseName) throws DatabaseNotExistException, CatalogException { + if (!databaseExists(databaseName)) { + throw new DatabaseNotExistException(getName(), databaseName); + } + + // get all schemas + try (Connection conn = DriverManager.getConnection(baseUrl + databaseName, username, pwd)) { + PreparedStatement ps = conn.prepareStatement("SELECT schema_name FROM information_schema.schemata;"); + + ResultSet rs = ps.executeQuery(); + + List schemas = new ArrayList<>(); + + while (rs.next()) { + String pgSchema = rs.getString(1); + if (!builtinSchemas.contains(pgSchema)) { + schemas.add(pgSchema); + } + } + + List tables = new ArrayList<>(); + + for (String schema : schemas) { + PreparedStatement stmt = conn.prepareStatement( + "SELECT * \n" + + "FROM information_schema.tables \n" + + "WHERE table_type = 'BASE TABLE' \n" + + " AND table_schema = ? \n" + + "ORDER BY table_type, table_name;"); + + stmt.setString(1, schema); + + ResultSet rstables = stmt.executeQuery(); + + while (rstables.next()) { + // position 1 is database name, position 2 is schema name, position 3 is table name + tables.add(schema + "." + rstables.getString(3)); + } + } + + return tables; + } catch (Exception e) { + throw new CatalogException( + String.format("Failed listing database in catalog %s", getName()), e); + } + } + + @Override + public CatalogBaseTable getTable(ObjectPath tablePath) throws TableNotExistException, CatalogException { + if (!tableExists(tablePath)) { + throw new TableNotExistException(getName(), tablePath); + } + + PostgresTablePath pgPath = PostgresTablePath.fromFlinkTableName(tablePath.getObjectName()); + + try (Connection conn = DriverManager.getConnection(baseUrl + tablePath.getDatabaseName(), username, pwd)) { + + PreparedStatement ps = conn.prepareStatement( + String.format("SELECT * FROM %s;", pgPath.getFullPath())); + + ResultSetMetaData rsmd = ps.getMetaData(); + + String[] names = new String[rsmd.getColumnCount()]; + DataType[] types = new DataType[rsmd.getColumnCount()]; + + for (int i = 1; i <= rsmd.getColumnCount(); i++) { + names[i - 1] = rsmd.getColumnName(i); + types[i - 1] = fromJDBCType(rsmd, i); + } + + TableSchema tableSchema = new TableSchema.Builder().fields(names, types).build(); + + return new CatalogTableImpl( + tableSchema, + new HashMap<>(), + "" + ); + } catch (Exception e) { + throw new CatalogException( + String.format("Failed getting table %s", tablePath.getFullName()), e); + } + } + + public static final String PG_BYTEA = "bytea"; + public static final String PG_BYTEA_ARRAY = "_bytea"; + public static final String PG_SMALLINT = "int2"; + public static final String PG_SMALLINT_ARRAY = "_int2"; + public static final String PG_INTEGER = "int4"; + public static final String PG_INTEGER_ARRAY = "_int4"; + public static final String PG_BIGINT = "int8"; + public static final String PG_BIGINT_ARRAY = "_int8"; + public static final String PG_REAL = "float4"; + public static final String PG_REAL_ARRAY = "_float4"; + public static final String PG_DOUBLE_PRECISION = "float8"; + public static final String PG_DOUBLE_PRECISION_ARRAY = "_float8"; + public static final String PG_NUMERIC = "numeric"; + public static final String PG_NUMERIC_ARRAY = "_numeric"; + public static final String PG_BOOLEAN = "bool"; + public static final String PG_BOOLEAN_ARRAY = "_bool"; + public static final String PG_TIMESTAMP = "timestamp"; + public static final String PG_TIMESTAMP_ARRAY = "_timestamp"; + public static final String PG_TIMESTAMPTZ = "timestamptz"; + public static final String PG_TIMESTAMPTZ_ARRAY = "_timestamptz"; + public static final String PG_DATE = "date"; + public static final String PG_DATE_ARRAY = "_date"; + public static final String PG_TIME = "time"; + public static final String PG_TIME_ARRAY = "_time"; + public static final String PG_TEXT = "text"; + public static final String PG_TEXT_ARRAY = "_text"; + public static final String PG_CHAR = "bpchar"; + public static final String PG_CHAR_ARRAY = "_bpchar"; + public static final String PG_CHARACTER = "character"; + public static final String PG_CHARACTER_ARRAY = "_character"; + public static final String PG_CHARACTER_VARYING = "varchar"; + public static final String PG_CHARACTER_VARYING_ARRAY = "_varchar"; + + private DataType fromJDBCType(ResultSetMetaData metadata, int colIndex) throws SQLException { + String pgType = metadata.getColumnTypeName(colIndex); + + int precision = metadata.getPrecision(colIndex); + + switch (pgType) { + case PG_BOOLEAN: + return DataTypes.BOOLEAN(); + case PG_BOOLEAN_ARRAY: + return DataTypes.ARRAY(DataTypes.BOOLEAN()); + case PG_BYTEA: + return DataTypes.BYTES(); + case PG_BYTEA_ARRAY: + return DataTypes.ARRAY(DataTypes.BYTES()); + case PG_SMALLINT: + return DataTypes.SMALLINT(); + case PG_SMALLINT_ARRAY: + return DataTypes.ARRAY(DataTypes.SMALLINT()); + case PG_INTEGER: + return DataTypes.INT(); + case PG_INTEGER_ARRAY: + return DataTypes.ARRAY(DataTypes.INT()); + case PG_BIGINT: + return DataTypes.BIGINT(); + case PG_BIGINT_ARRAY: + return DataTypes.ARRAY(DataTypes.BIGINT()); + case PG_REAL: + return DataTypes.FLOAT(); + case PG_REAL_ARRAY: + return DataTypes.ARRAY(DataTypes.FLOAT()); + case PG_DOUBLE_PRECISION: + return DataTypes.DOUBLE(); + case PG_DOUBLE_PRECISION_ARRAY: + return DataTypes.ARRAY(DataTypes.DOUBLE()); + case PG_NUMERIC: + return DataTypes.DECIMAL(precision, metadata.getScale(colIndex)); + case PG_NUMERIC_ARRAY: + return DataTypes.ARRAY( + DataTypes.DECIMAL(precision, metadata.getScale(colIndex))); + case PG_CHAR: + case PG_CHARACTER: + return DataTypes.CHAR(precision); + case PG_CHAR_ARRAY: + case PG_CHARACTER_ARRAY: + return DataTypes.ARRAY(DataTypes.CHAR(precision)); + case PG_CHARACTER_VARYING: + return DataTypes.VARCHAR(precision); + case PG_CHARACTER_VARYING_ARRAY: + return DataTypes.ARRAY(DataTypes.VARCHAR(precision)); + case PG_TEXT: + return DataTypes.STRING(); + case PG_TEXT_ARRAY: + return DataTypes.ARRAY(DataTypes.STRING()); + case PG_TIMESTAMP: + return DataTypes.TIMESTAMP(); + case PG_TIMESTAMP_ARRAY: + return DataTypes.ARRAY(DataTypes.TIMESTAMP()); + case PG_TIMESTAMPTZ: + return DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(); + case PG_TIMESTAMPTZ_ARRAY: + return DataTypes.ARRAY(DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()); + case PG_TIME: + return DataTypes.TIME(); + case PG_TIME_ARRAY: + return DataTypes.ARRAY(DataTypes.TIME()); + case PG_DATE: + return DataTypes.DATE(); + case PG_DATE_ARRAY: + return DataTypes.ARRAY(DataTypes.DATE()); + default: + throw new UnsupportedOperationException( + String.format("Doesn't support Postgres type '%s' yet", pgType)); + } + } + + @Override + public boolean tableExists(ObjectPath tablePath) throws CatalogException { + + List tables = null; + try { + tables = listTables(tablePath.getDatabaseName()); + } catch (DatabaseNotExistException e) { + return false; + } + + return tables.contains(PostgresTablePath.fromFlinkTableName(tablePath.getObjectName()).getFullPath()); + } + +} diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresTablePath.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresTablePath.java new file mode 100644 index 0000000000..99cc2b44e7 --- /dev/null +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresTablePath.java @@ -0,0 +1,95 @@ +/* + * 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.api.java.io.jdbc.catalog; + +import org.apache.flink.util.StringUtils; + +import java.util.Objects; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * Table path of PostgreSQL in Flink. Can be of formats "table_name" or "schema_name.table_name". + * When it's "table_name", the schema name defaults to "public". + */ +public class PostgresTablePath { + + private static final String DEFAULT_POSTGRES_SCHEMA_NAME = "public"; + + private final String pgSchemaName; + private final String pgTableName; + + public PostgresTablePath(String pgSchemaName, String pgTableName) { + checkArgument(!StringUtils.isNullOrWhitespaceOnly(pgSchemaName)); + checkArgument(!StringUtils.isNullOrWhitespaceOnly(pgTableName)); + + this.pgSchemaName = pgSchemaName; + this.pgTableName = pgTableName; + } + + public static PostgresTablePath fromFlinkTableName(String flinkTableName) { + if (flinkTableName.contains(".")) { + String[] path = flinkTableName.split("\\."); + + checkArgument(path != null && path.length == 2, + String.format("Table name '%s' is not valid. The parsed length is %d", flinkTableName, path.length)); + + return new PostgresTablePath(path[0], path[1]); + } else { + return new PostgresTablePath(DEFAULT_POSTGRES_SCHEMA_NAME, flinkTableName); + } + } + + public static String toFlinkTableName(String schema, String table) { + return new PostgresTablePath(schema, table).getFullPath(); + } + + public String getFullPath() { + return String.format("%s.%s", pgSchemaName, pgTableName); + } + + public String getFullPathWithQuotes() { + return String.format("`%s.%s`", pgSchemaName, pgTableName); + } + + @Override + public String toString() { + return getFullPathWithQuotes(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + PostgresTablePath that = (PostgresTablePath) o; + return Objects.equals(pgSchemaName, that.pgSchemaName) && + Objects.equals(pgTableName, that.pgTableName); + } + + @Override + public int hashCode() { + return Objects.hash(pgSchemaName, pgTableName); + } +} diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialects.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialects.java index fa7192efe5..743d16a015 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialects.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialects.java @@ -203,7 +203,10 @@ public final class JDBCDialects { } } - private static class MySQLDialect extends AbstractDialect { + /** + * MySQL dialect. + */ + public static class MySQLDialect extends AbstractDialect { private static final long serialVersionUID = 1L; @@ -301,7 +304,10 @@ public final class JDBCDialects { } } - private static class PostgresDialect extends AbstractDialect { + /** + * Postgres dialect. + */ + public static class PostgresDialect extends AbstractDialect { private static final long serialVersionUID = 1L; diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/JDBCCatalogUtilsTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/JDBCCatalogUtilsTest.java new file mode 100644 index 0000000000..7a4132b410 --- /dev/null +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/JDBCCatalogUtilsTest.java @@ -0,0 +1,44 @@ +/* + * 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.api.java.io.jdbc.catalog; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +/** + * Test for {@link JDBCCatalogUtils}. + */ +public class JDBCCatalogUtilsTest { + @Rule + public ExpectedException exception = ExpectedException.none(); + + @Test + public void testJDBCUrl() { + JDBCCatalogUtils.validateJDBCUrl("jdbc:postgresql://localhost:5432/"); + + JDBCCatalogUtils.validateJDBCUrl("jdbc:postgresql://localhost:5432"); + } + + @Test + public void testInvalidJDBCUrl() { + exception.expect(IllegalArgumentException.class); + JDBCCatalogUtils.validateJDBCUrl("jdbc:postgresql://localhost:5432/db"); + } +} diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogITCase.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogITCase.java new file mode 100644 index 0000000000..e103780ea1 --- /dev/null +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogITCase.java @@ -0,0 +1,325 @@ +/* + * 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.api.java.io.jdbc.catalog; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; + +import com.opentable.db.postgres.junit.EmbeddedPostgresRules; +import com.opentable.db.postgres.junit.SingleInstancePostgresRule; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.Arrays; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +/** + * Test for {@link PostgresCatalog}. + */ +public class PostgresCatalogITCase { + @Rule + public ExpectedException exception = ExpectedException.none(); + + @ClassRule + public static SingleInstancePostgresRule pg = EmbeddedPostgresRules.singleInstance(); + + protected static final String TEST_USERNAME = "postgres"; + protected static final String TEST_PWD = "postgres"; + protected static final String TEST_DB = "test"; + protected static final String TEST_SCHEMA = "test_schema"; + protected static final String TABLE1 = "t1"; + protected static final String TABLE2 = "t2"; + protected static final String TABLE3 = "t3"; + + protected static String baseUrl; + protected static Catalog catalog; + + public static Catalog createCatalog(String name, String defaultDb, String username, String pwd, String jdbcUrl) { + return new PostgresCatalog("mypg", PostgresCatalog.DEFAULT_DATABASE, username, pwd, jdbcUrl); + } + + @BeforeClass + public static void setup() throws SQLException { + // jdbc:postgresql://localhost:50807/postgres?user=postgres + String embeddedJdbcUrl = pg.getEmbeddedPostgres().getJdbcUrl(TEST_USERNAME, TEST_PWD); + // jdbc:postgresql://localhost:50807/ + baseUrl = embeddedJdbcUrl.substring(0, embeddedJdbcUrl.lastIndexOf("/") + 1); + + catalog = createCatalog("mypg", PostgresCatalog.DEFAULT_DATABASE, TEST_USERNAME, TEST_PWD, baseUrl); + + // create test database and schema + createDatabase(TEST_DB); + createSchema(TEST_DB, TEST_SCHEMA); + + // create test tables + // table: postgres.public.user1 + createTable(PostgresTablePath.fromFlinkTableName(TABLE1), getSimpleTable().pgSchemaSql); + + // table: testdb.public.user2 + // table: testdb.testschema.user3 + // table: testdb.public.datatypes + createTable(TEST_DB, PostgresTablePath.fromFlinkTableName(TABLE2), getSimpleTable().pgSchemaSql); + createTable(TEST_DB, new PostgresTablePath(TEST_SCHEMA, TABLE3), getSimpleTable().pgSchemaSql); + createTable(TEST_DB, PostgresTablePath.fromFlinkTableName("datatypes"), getDataTypesTable().pgSchemaSql); + } + + // ------ databases ------ + + @Test + public void testGetDb_DatabaseNotExistException() throws Exception { + exception.expect(DatabaseNotExistException.class); + exception.expectMessage("Database nonexistent does not exist in Catalog"); + catalog.getDatabase("nonexistent"); + } + + @Test + public void testListDatabases() { + List actual = catalog.listDatabases(); + + assertEquals( + Arrays.asList("postgres", "test"), + actual + ); + } + + @Test + public void testDbExists() throws Exception { + assertFalse(catalog.databaseExists("nonexistent")); + + assertTrue(catalog.databaseExists(PostgresCatalog.DEFAULT_DATABASE)); + } + + // ------ tables ------ + + @Test + public void testListTables() throws DatabaseNotExistException { + List actual = catalog.listTables(PostgresCatalog.DEFAULT_DATABASE); + + assertEquals(Arrays.asList("public.t1"), actual); + + actual = catalog.listTables(TEST_DB); + + assertEquals(Arrays.asList("public.datatypes", "public.t2", "test_schema.t3"), actual); + } + + @Test + public void testListTables_DatabaseNotExistException() throws DatabaseNotExistException { + exception.expect(DatabaseNotExistException.class); + catalog.listTables("postgres/nonexistschema"); + } + + @Test + public void testTableExists() { + assertFalse(catalog.tableExists(new ObjectPath(TEST_DB, "nonexist"))); + + assertTrue(catalog.tableExists(new ObjectPath(PostgresCatalog.DEFAULT_DATABASE, TABLE1))); + assertTrue(catalog.tableExists(new ObjectPath(TEST_DB, TABLE2))); + assertTrue(catalog.tableExists(new ObjectPath(TEST_DB, "test_schema.t3"))); + } + + @Test + public void testGetTables_TableNotExistException() throws TableNotExistException { + exception.expect(TableNotExistException.class); + catalog.getTable(new ObjectPath(TEST_DB, PostgresTablePath.toFlinkTableName(TEST_SCHEMA, "anytable"))); + } + + @Test + public void testGetTables_TableNotExistException_NoSchema() throws TableNotExistException { + exception.expect(TableNotExistException.class); + catalog.getTable(new ObjectPath(TEST_DB, PostgresTablePath.toFlinkTableName("nonexistschema", "anytable"))); + } + + @Test + public void testGetTables_TableNotExistException_NoDb() throws TableNotExistException { + exception.expect(TableNotExistException.class); + catalog.getTable(new ObjectPath("nonexistdb", PostgresTablePath.toFlinkTableName(TEST_SCHEMA, "anytable"))); + } + + @Test + public void testGetTable() throws org.apache.flink.table.catalog.exceptions.TableNotExistException { + // test postgres.public.user1 + TableSchema schema = getSimpleTable().schema; + + CatalogBaseTable table = catalog.getTable(new ObjectPath("postgres", TABLE1)); + + assertEquals(schema, table.getSchema()); + + table = catalog.getTable(new ObjectPath("postgres", "public.t1")); + + assertEquals(schema, table.getSchema()); + + // test testdb.public.user2 + table = catalog.getTable(new ObjectPath(TEST_DB, TABLE2)); + + assertEquals(schema, table.getSchema()); + + table = catalog.getTable(new ObjectPath(TEST_DB, "public.t2")); + + assertEquals(schema, table.getSchema()); + + // test testdb.testschema.user2 + table = catalog.getTable(new ObjectPath(TEST_DB, TEST_SCHEMA + ".t3")); + + assertEquals(schema, table.getSchema()); + + } + + @Test + public void testDataTypes() throws TableNotExistException { + CatalogBaseTable table = catalog.getTable(new ObjectPath(TEST_DB, "datatypes")); + + assertEquals(getDataTypesTable().schema, table.getSchema()); + } + + private static class TestTable { + TableSchema schema; + String pgSchemaSql; + + public TestTable(TableSchema schema, String pgSchemaSql) { + this.schema = schema; + this.pgSchemaSql = pgSchemaSql; + } + } + + private static TestTable getSimpleTable() { + return new TestTable( + TableSchema.builder() + .field("name", DataTypes.INT()) + .build(), + "name integer" + ); + } + + private static TestTable getDataTypesTable() { + return new TestTable( + TableSchema.builder() + .field("int", DataTypes.INT()) + .field("int_arr", DataTypes.ARRAY(DataTypes.INT())) + .field("bytea", DataTypes.BYTES()) + .field("bytea_arr", DataTypes.ARRAY(DataTypes.BYTES())) + .field("short", DataTypes.SMALLINT()) + .field("short_arr", DataTypes.ARRAY(DataTypes.SMALLINT())) + .field("long", DataTypes.BIGINT()) + .field("long_arr", DataTypes.ARRAY(DataTypes.BIGINT())) + .field("real", DataTypes.FLOAT()) + .field("real_arr", DataTypes.ARRAY(DataTypes.FLOAT())) + .field("double_precision", DataTypes.DOUBLE()) + .field("double_precision_arr", DataTypes.ARRAY(DataTypes.DOUBLE())) + .field("numeric", DataTypes.DECIMAL(10, 5)) + .field("numeric_arr", DataTypes.ARRAY(DataTypes.DECIMAL(10, 5))) + .field("boolean", DataTypes.BOOLEAN()) + .field("boolean_arr", DataTypes.ARRAY(DataTypes.BOOLEAN())) + .field("text", DataTypes.STRING()) + .field("text_arr", DataTypes.ARRAY(DataTypes.STRING())) + .field("char", DataTypes.CHAR(1)) + .field("char_arr", DataTypes.ARRAY(DataTypes.CHAR(1))) + .field("character", DataTypes.CHAR(3)) + .field("character_arr", DataTypes.ARRAY(DataTypes.CHAR(3))) + .field("character_varying", DataTypes.VARCHAR(20)) + .field("character_varying_arr", DataTypes.ARRAY(DataTypes.VARCHAR(20))) + .field("timestamp", DataTypes.TIMESTAMP()) + .field("timestamp_arr", DataTypes.ARRAY(DataTypes.TIMESTAMP())) + .field("timestamptz", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()) + .field("timestamptz_arr", DataTypes.ARRAY(DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE())) + .field("date", DataTypes.DATE()) + .field("date_arr", DataTypes.ARRAY(DataTypes.DATE())) + .field("time", DataTypes.TIME()) + .field("time_arr", DataTypes.ARRAY(DataTypes.TIME())) + .build(), + "int integer, " + + "int_arr integer[], " + + "bytea bytea, " + + "bytea_arr bytea[], " + + "short smallint, " + + "short_arr smallint[], " + + "long bigint, " + + "long_arr bigint[], " + + "real real, " + + "real_arr real[], " + + "double_precision double precision, " + + "double_precision_arr double precision[], " + + "numeric numeric(10, 5), " + + "numeric_arr numeric(10, 5)[], " + + "boolean boolean, " + + "boolean_arr boolean[], " + + "text text, " + + "text_arr text[], " + + "char char, " + + "char_arr char[], " + + "character character(3), " + + "character_arr character(3)[], " + + "character_varying character varying(20), " + + "character_varying_arr character varying(20)[], " + + "timestamp timestamp(6), " + + "timestamp_arr timestamp(6)[], " + + "timestamptz timestamptz, " + + "timestamptz_arr timestamptz[], " + + "date date, " + + "date_arr date[], " + + "time time(6), " + + "time_arr time(6)[]" + ); + } + + private static void createTable(PostgresTablePath tablePath, String tableSchemaSql) throws SQLException { + executeSQL(PostgresCatalog.DEFAULT_DATABASE, String.format("CREATE TABLE %s(%s);", tablePath.getFullPath(), tableSchemaSql)); + } + + private static void createTable(String db, PostgresTablePath tablePath, String tableSchemaSql) throws SQLException { + executeSQL(db, String.format("CREATE TABLE %s(%s);", tablePath.getFullPath(), tableSchemaSql)); + } + + private static void createSchema(String db, String schema) throws SQLException { + executeSQL(db, String.format("CREATE SCHEMA %s", schema)); + } + + private static void createDatabase(String database) throws SQLException { + executeSQL(String.format("CREATE DATABASE %s;", database)); + } + + private static void executeSQL(String sql) throws SQLException { + executeSQL("", sql); + } + + private static void executeSQL(String db, String sql) throws SQLException { + try (Connection conn = DriverManager.getConnection(baseUrl + db, TEST_USERNAME, TEST_PWD); + Statement statement = conn.createStatement()) { + statement.executeUpdate(sql); + } catch (SQLException e) { + throw e; + } + } + +} diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresTablePathTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresTablePathTest.java new file mode 100644 index 0000000000..46f32bc6ba --- /dev/null +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresTablePathTest.java @@ -0,0 +1,33 @@ +/* + * 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.api.java.io.jdbc.catalog; + +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +/** + * Test for {@link PostgresTablePath}. + */ +public class PostgresTablePathTest { + @Test + public void testFromFlinkTableName() { + assertEquals(new PostgresTablePath("public", "topic"), PostgresTablePath.fromFlinkTableName("public.topic")); + } +} -- Gitee From 75ad29cb9f4f377df27b71e67dbd33f36bb08bee Mon Sep 17 00:00:00 2001 From: "bowen.li" Date: Sun, 8 Mar 2020 20:58:27 -0700 Subject: [PATCH 230/885] [FLINK-16472] support precision of timestamp and time data types closes #11336 --- .../java/io/jdbc/catalog/PostgresCatalog.java | 15 ++++++------ .../jdbc/catalog/PostgresCatalogITCase.java | 24 +++++++++---------- 2 files changed, 19 insertions(+), 20 deletions(-) diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalog.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalog.java index d12f25473e..c5980737b3 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalog.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalog.java @@ -55,8 +55,6 @@ public class PostgresCatalog extends AbstractJDBCCatalog { private static final Logger LOG = LoggerFactory.getLogger(PostgresCatalog.class); - public static final String POSTGRES_TABLE_TYPE = "postgres"; - public static final String DEFAULT_DATABASE = "postgres"; // ------ Postgres default objects that shouldn't be exposed to users ------ @@ -236,6 +234,7 @@ public class PostgresCatalog extends AbstractJDBCCatalog { String pgType = metadata.getColumnTypeName(colIndex); int precision = metadata.getPrecision(colIndex); + int scale = metadata.getScale(colIndex); switch (pgType) { case PG_BOOLEAN: @@ -286,17 +285,17 @@ public class PostgresCatalog extends AbstractJDBCCatalog { case PG_TEXT_ARRAY: return DataTypes.ARRAY(DataTypes.STRING()); case PG_TIMESTAMP: - return DataTypes.TIMESTAMP(); + return DataTypes.TIMESTAMP(scale); case PG_TIMESTAMP_ARRAY: - return DataTypes.ARRAY(DataTypes.TIMESTAMP()); + return DataTypes.ARRAY(DataTypes.TIMESTAMP(scale)); case PG_TIMESTAMPTZ: - return DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(); + return DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(scale); case PG_TIMESTAMPTZ_ARRAY: - return DataTypes.ARRAY(DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()); + return DataTypes.ARRAY(DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(scale)); case PG_TIME: - return DataTypes.TIME(); + return DataTypes.TIME(scale); case PG_TIME_ARRAY: - return DataTypes.ARRAY(DataTypes.TIME()); + return DataTypes.ARRAY(DataTypes.TIME(scale)); case PG_DATE: return DataTypes.DATE(); case PG_DATE_ARRAY: diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogITCase.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogITCase.java index e103780ea1..b197bf0409 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogITCase.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogITCase.java @@ -249,14 +249,14 @@ public class PostgresCatalogITCase { .field("character_arr", DataTypes.ARRAY(DataTypes.CHAR(3))) .field("character_varying", DataTypes.VARCHAR(20)) .field("character_varying_arr", DataTypes.ARRAY(DataTypes.VARCHAR(20))) - .field("timestamp", DataTypes.TIMESTAMP()) - .field("timestamp_arr", DataTypes.ARRAY(DataTypes.TIMESTAMP())) - .field("timestamptz", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()) - .field("timestamptz_arr", DataTypes.ARRAY(DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE())) + .field("timestamp", DataTypes.TIMESTAMP(5)) + .field("timestamp_arr", DataTypes.ARRAY(DataTypes.TIMESTAMP(5))) + .field("timestamptz", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(4)) + .field("timestamptz_arr", DataTypes.ARRAY(DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(4))) .field("date", DataTypes.DATE()) .field("date_arr", DataTypes.ARRAY(DataTypes.DATE())) - .field("time", DataTypes.TIME()) - .field("time_arr", DataTypes.ARRAY(DataTypes.TIME())) + .field("time", DataTypes.TIME(3)) + .field("time_arr", DataTypes.ARRAY(DataTypes.TIME(3))) .build(), "int integer, " + "int_arr integer[], " + @@ -282,14 +282,14 @@ public class PostgresCatalogITCase { "character_arr character(3)[], " + "character_varying character varying(20), " + "character_varying_arr character varying(20)[], " + - "timestamp timestamp(6), " + - "timestamp_arr timestamp(6)[], " + - "timestamptz timestamptz, " + - "timestamptz_arr timestamptz[], " + + "timestamp timestamp(5), " + + "timestamp_arr timestamp(5)[], " + + "timestamptz timestamptz(4), " + + "timestamptz_arr timestamptz(4)[], " + "date date, " + "date_arr date[], " + - "time time(6), " + - "time_arr time(6)[]" + "time time(3), " + + "time_arr time(3)[]" ); } -- Gitee From 8a1666e8678ca79bf57ab1279ea2dfc9f6f8e8a7 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Thu, 19 Mar 2020 17:21:50 +0800 Subject: [PATCH 231/885] [hotfix][task] Remove unnecessary fields and methods from StreamTask This closes #11459 --- .../api/operators/AbstractStreamOperator.java | 2 +- .../flink/streaming/runtime/tasks/StreamTask.java | 14 -------------- .../flink/streaming/util/MockStreamTask.java | 10 ---------- .../streaming/util/MockStreamTaskBuilder.java | 5 ----- 4 files changed, 1 insertion(+), 30 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java index f876d3c7ac..532c57f1be 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java @@ -227,7 +227,7 @@ public abstract class AbstractStreamOperator LatencyStats.Granularity.SINGLE); } - this.runtimeContext = new StreamingRuntimeContext(this, environment, container.getAccumulatorMap()); + this.runtimeContext = new StreamingRuntimeContext(this, environment, environment.getAccumulatorRegistry().getUserMap()); stateKeySelector1 = config.getStatePartitioner(0, getUserCodeClassloader()); stateKeySelector2 = config.getStatePartitioner(1, getUserCodeClassloader()); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java index aabd9c54b1..82651ac892 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java @@ -19,7 +19,6 @@ package org.apache.flink.streaming.runtime.tasks; import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.core.fs.CloseableRegistry; import org.apache.flink.metrics.Counter; @@ -44,7 +43,6 @@ import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup; import org.apache.flink.runtime.operators.coordination.OperatorEvent; -import org.apache.flink.runtime.operators.coordination.OperatorEventDispatcher; import org.apache.flink.runtime.plugable.SerializationDelegate; import org.apache.flink.runtime.state.CheckpointStorageWorkerView; import org.apache.flink.runtime.state.CheckpointStreamFactory; @@ -184,9 +182,6 @@ public abstract class StreamTask> private final Thread.UncaughtExceptionHandler uncaughtExceptionHandler; - /** The map of user-defined accumulators of this task. */ - private final Map> accumulatorMap; - /** The currently active background materialization threads. */ private final CloseableRegistry cancelables = new CloseableRegistry(); @@ -272,7 +267,6 @@ public abstract class StreamTask> this.timerService = timerService; this.uncaughtExceptionHandler = Preconditions.checkNotNull(uncaughtExceptionHandler); this.configuration = new StreamConfig(getTaskConfiguration()); - this.accumulatorMap = getEnvironment().getAccumulatorRegistry().getUserMap(); this.recordWriter = createRecordWriterDelegate(configuration, environment); this.actionExecutor = Preconditions.checkNotNull(actionExecutor); this.mailboxProcessor = new MailboxProcessor(this::processInput, mailbox, actionExecutor); @@ -687,18 +681,10 @@ public abstract class StreamTask> return configuration; } - public Map> getAccumulatorMap() { - return accumulatorMap; - } - public StreamStatusMaintainer getStreamStatusMaintainer() { return operatorChain; } - public OperatorEventDispatcher getOperatorEventDispatcher() { - return operatorChain.getOperatorEventDispatcher(); - } - RecordWriterOutput[] getStreamOutputs() { return operatorChain.getStreamOutputs(); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTask.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTask.java index ffd4a5091b..d190924ea2 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTask.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTask.java @@ -19,7 +19,6 @@ package org.apache.flink.streaming.util; import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.core.fs.CloseableRegistry; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.state.CheckpointStorageWorkerView; @@ -36,7 +35,6 @@ import org.apache.flink.streaming.runtime.tasks.TimerService; import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxDefaultAction; import org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailbox; -import java.util.Map; import java.util.function.BiConsumer; /** @@ -54,7 +52,6 @@ public class MockStreamTask> extends StreamT private final CheckpointStorageWorkerView checkpointStorage; private final ProcessingTimeService processingTimeService; private final BiConsumer handleAsyncException; - private final Map> accumulatorMap; public MockStreamTask( Environment environment, @@ -68,7 +65,6 @@ public class MockStreamTask> extends StreamT CheckpointStorageWorkerView checkpointStorage, TimerService timerService, BiConsumer handleAsyncException, - Map> accumulatorMap, TaskMailbox taskMailbox, StreamTaskActionExecutor.SynchronizedStreamTaskActionExecutor taskActionExecutor ) { @@ -83,7 +79,6 @@ public class MockStreamTask> extends StreamT this.checkpointStorage = checkpointStorage; this.processingTimeService = timerService; this.handleAsyncException = handleAsyncException; - this.accumulatorMap = accumulatorMap; } @Override @@ -158,11 +153,6 @@ public class MockStreamTask> extends StreamT handleAsyncException.accept(message, exception); } - @Override - public Map> getAccumulatorMap() { - return accumulatorMap; - } - @Override public ProcessingTimeServiceFactory getProcessingTimeServiceFactory() { return mailboxExecutor -> processingTimeService; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTaskBuilder.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTaskBuilder.java index 904c5e9d5e..680fac0a86 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTaskBuilder.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTaskBuilder.java @@ -20,7 +20,6 @@ package org.apache.flink.streaming.util; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.CloseableRegistry; import org.apache.flink.runtime.execution.Environment; @@ -39,8 +38,6 @@ import org.apache.flink.streaming.runtime.tasks.TimerService; import org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailbox; import org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailboxImpl; -import java.util.Collections; -import java.util.Map; import java.util.function.BiConsumer; /** @@ -58,7 +55,6 @@ public class MockStreamTaskBuilder { private TimerService timerService = new TestProcessingTimeService(); private StreamTaskStateInitializer streamTaskStateInitializer; private BiConsumer handleAsyncException = (message, throwable) -> { }; - private Map> accumulatorMap = Collections.emptyMap(); private TaskMailbox taskMailbox = new TaskMailboxImpl(); private StreamTaskActionExecutor.SynchronizedStreamTaskActionExecutor taskActionExecutor = StreamTaskActionExecutor.synchronizedExecutor(); @@ -143,7 +139,6 @@ public class MockStreamTaskBuilder { checkpointStorage, timerService, handleAsyncException, - accumulatorMap, taskMailbox, taskActionExecutor); } -- Gitee From 6937fd82e9811847321a7d341de486a428afd1bd Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Thu, 19 Mar 2020 16:15:56 +0800 Subject: [PATCH 232/885] [hotfix][tests] Remove dead codes from SynchronousCheckpointTest This closes #11459 --- .../StreamTaskCancellationBarrierTest.java | 35 ------------------- 1 file changed, 35 deletions(-) diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskCancellationBarrierTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskCancellationBarrierTest.java index 6d55866898..52433d7a08 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskCancellationBarrierTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskCancellationBarrierTest.java @@ -21,17 +21,14 @@ package org.apache.flink.streaming.runtime.tasks; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.runtime.checkpoint.CheckpointFailureReason; -import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.streaming.api.functions.co.CoMapFunction; import org.apache.flink.streaming.api.graph.StreamConfig; -import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.StreamMap; import org.apache.flink.streaming.api.operators.co.CoStreamMap; import org.apache.flink.streaming.runtime.io.CheckpointBarrierAlignerTestBase; import org.apache.flink.streaming.runtime.io.CheckpointBarrierAlignerTestBase.CheckpointExceptionMatcher; -import org.apache.flink.streaming.runtime.tasks.StreamTaskTest.NoOpStreamTask; import org.junit.Rule; import org.junit.Test; @@ -145,38 +142,6 @@ public class StreamTaskCancellationBarrierTest { testHarness.waitForTaskCompletion(); } - // ------------------------------------------------------------------------ - // test tasks / functions - // ------------------------------------------------------------------------ - - private static class InitBlockingTask extends NoOpStreamTask> { - - private final Object lock = new Object(); - private volatile boolean running = true; - - protected InitBlockingTask(Environment env) { - super(env); - } - - @Override - protected void init() throws Exception { - super.init(); - synchronized (lock) { - while (running) { - lock.wait(); - } - } - } - - @Override - protected void cancelTask() throws Exception { - running = false; - synchronized (lock) { - lock.notifyAll(); - } - } - } - private static class IdentityMap implements MapFunction { private static final long serialVersionUID = 1L; -- Gitee From 7dafd1cd65ba8977d923482a810c907ed7d37b59 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Fri, 20 Mar 2020 13:14:35 +0800 Subject: [PATCH 233/885] [FLINK-16690][tests] Refactor StreamTaskTest to reuse TestTaskBuilder and MockStreamTaskBuilder This closes #11459 --- .../runtime/taskmanager/TestTaskBuilder.java | 8 +- ...bstractUdfStreamOperatorLifecycleTest.java | 38 +-- .../runtime/tasks/StreamTaskTest.java | 263 ++++++------------ .../flink/streaming/util/MockStreamTask.java | 13 +- .../streaming/util/MockStreamTaskBuilder.java | 17 +- 5 files changed, 134 insertions(+), 205 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TestTaskBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TestTaskBuilder.java index 2c5f341a29..c05e075e4e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TestTaskBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TestTaskBuilder.java @@ -80,6 +80,7 @@ public final class TestTaskBuilder { private KvStateService kvStateService = new KvStateService(new KvStateRegistry(), null, null); private Executor executor = TestingUtils.defaultExecutor(); private Configuration taskManagerConfig = new Configuration(); + private Configuration taskConfig = new Configuration(); private ExecutionConfig executionConfig = new ExecutionConfig(); private Collection requiredJarFileBlobKeys = Collections.emptyList(); private List resultPartitions = Collections.emptyList(); @@ -132,6 +133,11 @@ public final class TestTaskBuilder { return this; } + public TestTaskBuilder setTaskConfig(Configuration taskConfig) { + this.taskConfig = taskConfig; + return this; + } + public TestTaskBuilder setExecutionConfig(ExecutionConfig executionConfig) { this.executionConfig = executionConfig; return this; @@ -186,7 +192,7 @@ public final class TestTaskBuilder { 1, 1, invokable.getName(), - new Configuration()); + taskConfig); final BlobCacheService blobCacheService = new BlobCacheService( mock(PermanentBlobCache.class), diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java index cb24990af4..1c2240e70e 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java @@ -25,7 +25,9 @@ import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.runtime.checkpoint.CheckpointMetaData; import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.io.network.NettyShuffleEnvironmentBuilder; import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.shuffle.ShuffleEnvironment; import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.runtime.state.StateSnapshotContext; import org.apache.flink.runtime.taskmanager.Task; @@ -141,16 +143,18 @@ public class AbstractUdfStreamOperatorLifecycleTest { cfg.setOperatorID(new OperatorID()); cfg.setTimeCharacteristic(TimeCharacteristic.ProcessingTime); - Task task = StreamTaskTest.createTask(SourceStreamTask.class, cfg, taskManagerConfig); + try (ShuffleEnvironment shuffleEnvironment = new NettyShuffleEnvironmentBuilder().build()) { + Task task = StreamTaskTest.createTask(SourceStreamTask.class, shuffleEnvironment, cfg, taskManagerConfig); - task.startTaskThread(); + task.startTaskThread(); - LifecycleTrackingStreamSource.runStarted.await(); + LifecycleTrackingStreamSource.runStarted.await(); - // wait for clean termination - task.getExecutingThread().join(); - assertEquals(ExecutionState.FINISHED, task.getExecutionState()); - assertEquals(EXPECTED_CALL_ORDER_FULL, ACTUAL_ORDER_TRACKING); + // wait for clean termination + task.getExecutingThread().join(); + assertEquals(ExecutionState.FINISHED, task.getExecutionState()); + assertEquals(EXPECTED_CALL_ORDER_FULL, ACTUAL_ORDER_TRACKING); + } } @Test @@ -164,18 +168,20 @@ public class AbstractUdfStreamOperatorLifecycleTest { cfg.setOperatorID(new OperatorID()); cfg.setTimeCharacteristic(TimeCharacteristic.ProcessingTime); - Task task = StreamTaskTest.createTask(SourceStreamTask.class, cfg, taskManagerConfig); + try (ShuffleEnvironment shuffleEnvironment = new NettyShuffleEnvironmentBuilder().build()) { + Task task = StreamTaskTest.createTask(SourceStreamTask.class, shuffleEnvironment, cfg, taskManagerConfig); - task.startTaskThread(); - LifecycleTrackingStreamSource.runStarted.await(); + task.startTaskThread(); + LifecycleTrackingStreamSource.runStarted.await(); - // this should cancel the task even though it is blocked on runFinished - task.cancelExecution(); + // this should cancel the task even though it is blocked on runFinished + task.cancelExecution(); - // wait for clean termination - task.getExecutingThread().join(); - assertEquals(ExecutionState.CANCELED, task.getExecutionState()); - assertEquals(EXPECTED_CALL_ORDER_CANCEL_RUNNING, ACTUAL_ORDER_TRACKING); + // wait for clean termination + task.getExecutingThread().join(); + assertEquals(ExecutionState.CANCELED, task.getExecutionState()); + assertEquals(EXPECTED_CALL_ORDER_CANCEL_RUNNING, ACTUAL_ORDER_TRACKING); + } } private static class MockSourceFunction extends RichSourceFunction { diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java index dc9d4ead7e..1628e4c055 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java @@ -18,7 +18,6 @@ package org.apache.flink.streaming.runtime.tasks; -import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.base.StringSerializer; @@ -26,10 +25,6 @@ import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.FSDataInputStream; import org.apache.flink.core.testutils.OneShotLatch; -import org.apache.flink.runtime.blob.BlobCacheService; -import org.apache.flink.runtime.blob.PermanentBlobCache; -import org.apache.flink.runtime.blob.TransientBlobCache; -import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.checkpoint.CheckpointMetaData; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; import org.apache.flink.runtime.checkpoint.CheckpointOptions; @@ -37,38 +32,23 @@ import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.runtime.checkpoint.StateObjectCollection; import org.apache.flink.runtime.checkpoint.SubtaskState; import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; -import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.concurrent.TestingUncaughtExceptionHandler; -import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; -import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; import org.apache.flink.runtime.execution.CancelTaskException; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.execution.ExecutionState; -import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; -import org.apache.flink.runtime.executiongraph.JobInformation; -import org.apache.flink.runtime.executiongraph.TaskInformation; -import org.apache.flink.runtime.filecache.FileCache; -import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; import org.apache.flink.runtime.io.network.NettyShuffleEnvironmentBuilder; -import org.apache.flink.runtime.io.network.TaskEventDispatcher; import org.apache.flink.runtime.io.network.api.writer.AvailabilityTestResultPartitionWriter; import org.apache.flink.runtime.io.network.api.writer.RecordWriter; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; -import org.apache.flink.runtime.io.network.partition.NoOpResultPartitionConsumableNotifier; -import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier; -import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; -import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; -import org.apache.flink.runtime.memory.MemoryManager; -import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.operators.testutils.DummyEnvironment; import org.apache.flink.runtime.operators.testutils.MockEnvironment; import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; -import org.apache.flink.runtime.query.KvStateRegistry; import org.apache.flink.runtime.shuffle.ShuffleEnvironment; import org.apache.flink.runtime.state.AbstractKeyedStateBackend; import org.apache.flink.runtime.state.AbstractStateBackend; @@ -89,19 +69,14 @@ import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.runtime.state.TaskLocalStateStoreImpl; import org.apache.flink.runtime.state.TaskStateManager; import org.apache.flink.runtime.state.TaskStateManagerImpl; -import org.apache.flink.runtime.state.TestTaskStateManager; import org.apache.flink.runtime.state.memory.MemoryStateBackend; -import org.apache.flink.runtime.taskexecutor.KvStateService; -import org.apache.flink.runtime.taskexecutor.PartitionProducerStateChecker; -import org.apache.flink.runtime.taskexecutor.TestGlobalAggregateManager; import org.apache.flink.runtime.taskmanager.CheckpointResponder; import org.apache.flink.runtime.taskmanager.NoOpTaskManagerActions; -import org.apache.flink.runtime.taskmanager.NoOpTaskOperatorEventGateway; import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.runtime.taskmanager.TaskManagerActions; +import org.apache.flink.runtime.taskmanager.TestTaskBuilder; import org.apache.flink.runtime.util.FatalExitExceptionHandler; -import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo; import org.apache.flink.streaming.api.TimeCharacteristic; import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.api.graph.StreamConfig; @@ -124,11 +99,11 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer; import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxDefaultAction; import org.apache.flink.streaming.util.MockStreamConfig; +import org.apache.flink.streaming.util.MockStreamTaskBuilder; import org.apache.flink.streaming.util.TestSequentialReadingStreamOperator; import org.apache.flink.util.CloseableIterable; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.Preconditions; -import org.apache.flink.util.SerializedValue; import org.apache.flink.util.TestLogger; import org.apache.flink.util.function.RunnableWithException; import org.apache.flink.util.function.SupplierWithException; @@ -156,7 +131,6 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; -import java.util.concurrent.Executor; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.RunnableFuture; @@ -234,23 +208,29 @@ public class StreamTaskTest extends TestLogger { cfg.setTimeCharacteristic(TimeCharacteristic.ProcessingTime); final TaskManagerActions taskManagerActions = spy(new NoOpTaskManagerActions()); - final Task task = createTask(SourceStreamTask.class, cfg, new Configuration(), taskManagerActions); + try (NettyShuffleEnvironment shuffleEnvironment = new NettyShuffleEnvironmentBuilder().build()) { + final Task task = new TestTaskBuilder(shuffleEnvironment) + .setInvokable(SourceStreamTask.class) + .setTaskConfig(cfg.getConfiguration()) + .setTaskManagerActions(taskManagerActions) + .build(); - final TaskExecutionState state = new TaskExecutionState( - task.getJobID(), task.getExecutionId(), ExecutionState.RUNNING); + final TaskExecutionState state = new TaskExecutionState( + task.getJobID(), task.getExecutionId(), ExecutionState.RUNNING); - task.startTaskThread(); + task.startTaskThread(); - verify(taskManagerActions, timeout(2000L)).updateTaskExecutionState(eq(state)); + verify(taskManagerActions, timeout(2000L)).updateTaskExecutionState(eq(state)); - // send a cancel. because the operator takes a long time to deserialize, this should - // hit the task before the operator is deserialized - task.cancelExecution(); + // send a cancel. because the operator takes a long time to deserialize, this should + // hit the task before the operator is deserialized + task.cancelExecution(); - task.getExecutingThread().join(); + task.getExecutingThread().join(); - assertFalse("Task did not cancel", task.getExecutingThread().isAlive()); - assertEquals(ExecutionState.CANCELED, task.getExecutionState()); + assertFalse("Task did not cancel", task.getExecutingThread().isAlive()); + assertEquals(ExecutionState.CANCELED, task.getExecutionState()); + } } @Test @@ -265,24 +245,26 @@ public class StreamTaskTest extends TestLogger { cfg.setStreamOperator(streamSource); cfg.setTimeCharacteristic(TimeCharacteristic.ProcessingTime); - Task task = createTask(StateBackendTestSource.class, cfg, taskManagerConfig); + try (ShuffleEnvironment shuffleEnvironment = new NettyShuffleEnvironmentBuilder().build()) { + Task task = createTask(StateBackendTestSource.class, shuffleEnvironment, cfg, taskManagerConfig); - StateBackendTestSource.fail = false; - task.startTaskThread(); + StateBackendTestSource.fail = false; + task.startTaskThread(); - // wait for clean termination - task.getExecutingThread().join(); + // wait for clean termination + task.getExecutingThread().join(); - // ensure that the state backends and stream iterables are closed ... - verify(TestStreamSource.operatorStateBackend).close(); - verify(TestStreamSource.keyedStateBackend).close(); - verify(TestStreamSource.rawOperatorStateInputs).close(); - verify(TestStreamSource.rawKeyedStateInputs).close(); - // ... and disposed - verify(TestStreamSource.operatorStateBackend).dispose(); - verify(TestStreamSource.keyedStateBackend).dispose(); + // ensure that the state backends and stream iterables are closed ... + verify(TestStreamSource.operatorStateBackend).close(); + verify(TestStreamSource.keyedStateBackend).close(); + verify(TestStreamSource.rawOperatorStateInputs).close(); + verify(TestStreamSource.rawKeyedStateInputs).close(); + // ... and disposed + verify(TestStreamSource.operatorStateBackend).dispose(); + verify(TestStreamSource.keyedStateBackend).dispose(); - assertEquals(ExecutionState.FINISHED, task.getExecutionState()); + assertEquals(ExecutionState.FINISHED, task.getExecutionState()); + } } @Test @@ -297,24 +279,26 @@ public class StreamTaskTest extends TestLogger { cfg.setStreamOperator(streamSource); cfg.setTimeCharacteristic(TimeCharacteristic.ProcessingTime); - Task task = createTask(StateBackendTestSource.class, cfg, taskManagerConfig); + try (NettyShuffleEnvironment shuffleEnvironment = new NettyShuffleEnvironmentBuilder().build()) { + Task task = createTask(StateBackendTestSource.class, shuffleEnvironment, cfg, taskManagerConfig); - StateBackendTestSource.fail = true; - task.startTaskThread(); + StateBackendTestSource.fail = true; + task.startTaskThread(); - // wait for clean termination - task.getExecutingThread().join(); + // wait for clean termination + task.getExecutingThread().join(); - // ensure that the state backends and stream iterables are closed ... - verify(TestStreamSource.operatorStateBackend).close(); - verify(TestStreamSource.keyedStateBackend).close(); - verify(TestStreamSource.rawOperatorStateInputs).close(); - verify(TestStreamSource.rawKeyedStateInputs).close(); - // ... and disposed - verify(TestStreamSource.operatorStateBackend).dispose(); - verify(TestStreamSource.keyedStateBackend).dispose(); + // ensure that the state backends and stream iterables are closed ... + verify(TestStreamSource.operatorStateBackend).close(); + verify(TestStreamSource.keyedStateBackend).close(); + verify(TestStreamSource.rawOperatorStateInputs).close(); + verify(TestStreamSource.rawKeyedStateInputs).close(); + // ... and disposed + verify(TestStreamSource.operatorStateBackend).dispose(); + verify(TestStreamSource.keyedStateBackend).dispose(); - assertEquals(ExecutionState.FAILED, task.getExecutionState()); + assertEquals(ExecutionState.FAILED, task.getExecutionState()); + } } @Test @@ -322,19 +306,22 @@ public class StreamTaskTest extends TestLogger { syncLatch = new OneShotLatch(); StreamConfig cfg = new StreamConfig(new Configuration()); - Task task = createTask(CancelFailingTask.class, cfg, new Configuration()); + try (NettyShuffleEnvironment shuffleEnvironment = new NettyShuffleEnvironmentBuilder().build()) { + + Task task = createTask(CancelFailingTask.class, shuffleEnvironment, cfg, new Configuration()); - // start the task and wait until it runs - // execution state RUNNING is not enough, we need to wait until the stream task's run() method - // is entered - task.startTaskThread(); - syncLatch.await(); + // start the task and wait until it runs + // execution state RUNNING is not enough, we need to wait until the stream task's run() method + // is entered + task.startTaskThread(); + syncLatch.await(); - // cancel the execution - this should lead to smooth shutdown - task.cancelExecution(); - task.getExecutingThread().join(); + // cancel the execution - this should lead to smooth shutdown + task.cancelExecution(); + task.getExecutingThread().join(); - assertEquals(ExecutionState.CANCELED, task.getExecutionState()); + assertEquals(ExecutionState.CANCELED, task.getExecutionState()); + } } /** @@ -439,12 +426,15 @@ public class StreamTaskTest extends TestLogger { @Test public void testCancelTaskExceptionHandling() throws Exception { StreamConfig cfg = new StreamConfig(new Configuration()); - Task task = createTask(CancelThrowingTask.class, cfg, new Configuration()); - task.startTaskThread(); - task.getExecutingThread().join(); + try (NettyShuffleEnvironment shuffleEnvironment = new NettyShuffleEnvironmentBuilder().build()) { + Task task = createTask(CancelThrowingTask.class, shuffleEnvironment, cfg, new Configuration()); + + task.startTaskThread(); + task.getExecutingThread().join(); - assertEquals(ExecutionState.CANCELED, task.getExecutionState()); + assertEquals(ExecutionState.CANCELED, task.getExecutionState()); + } } /** @@ -983,7 +973,9 @@ public class StreamTaskTest extends TestLogger { try (final MockEnvironment environment = setupEnvironment(new boolean[] {true, true})) { final int numberOfProcessCalls = 10; final AvailabilityTestInputProcessor inputProcessor = new AvailabilityTestInputProcessor(numberOfProcessCalls); - final AvailabilityTestStreamTask task = new AvailabilityTestStreamTask<>(environment, inputProcessor); + final StreamTask task = new MockStreamTaskBuilder(environment) + .setStreamInputProcessor(inputProcessor) + .build(); task.invoke(); assertEquals(numberOfProcessCalls, inputProcessor.currentNumProcessCalls); @@ -995,7 +987,9 @@ public class StreamTaskTest extends TestLogger { try (final MockEnvironment environment = setupEnvironment(new boolean[] {true, false})) { final int numberOfProcessCalls = 10; final AvailabilityTestInputProcessor inputProcessor = new AvailabilityTestInputProcessor(numberOfProcessCalls); - final AvailabilityTestStreamTask task = new AvailabilityTestStreamTask<>(environment, inputProcessor); + final StreamTask task = new MockStreamTaskBuilder(environment) + .setStreamInputProcessor(inputProcessor) + .build(); final MailboxExecutor executor = task.mailboxProcessor.getMainMailboxExecutor(); final RunnableWithException completeFutureTask = () -> { @@ -1128,22 +1122,6 @@ public class StreamTaskTest extends TestLogger { protected void cleanup() throws Exception {} } - /** - * A stream task implementation used to construct a specific {@link StreamInputProcessor} for tests. - */ - private static class AvailabilityTestStreamTask> extends StreamTask { - - AvailabilityTestStreamTask(Environment environment, StreamInputProcessor inputProcessor) { - super(environment); - - this.inputProcessor = inputProcessor; - } - - @Override - protected void init() { - } - } - /** * A stream input processor implementation used to control the returned input status based on * the total number of processing calls. @@ -1196,86 +1174,17 @@ public class StreamTaskTest extends TestLogger { } } - public static Task createTask( - Class invokable, - StreamConfig taskConfig, - Configuration taskManagerConfig) throws Exception { - return createTask(invokable, taskConfig, taskManagerConfig, new TestTaskStateManager(), mock(TaskManagerActions.class)); - } - - public static Task createTask( - Class invokable, - StreamConfig taskConfig, - Configuration taskManagerConfig, - TaskManagerActions taskManagerActions) throws Exception { - return createTask(invokable, taskConfig, taskManagerConfig, new TestTaskStateManager(), taskManagerActions); - } - public static Task createTask( Class invokable, + ShuffleEnvironment shuffleEnvironment, StreamConfig taskConfig, - Configuration taskManagerConfig, - TestTaskStateManager taskStateManager, - TaskManagerActions taskManagerActions) throws Exception { - - BlobCacheService blobService = - new BlobCacheService(mock(PermanentBlobCache.class), mock(TransientBlobCache.class)); - - LibraryCacheManager libCache = mock(LibraryCacheManager.class); - when(libCache.getClassLoader(any(JobID.class))).thenReturn(StreamTaskTest.class.getClassLoader()); - - ResultPartitionConsumableNotifier consumableNotifier = new NoOpResultPartitionConsumableNotifier(); - PartitionProducerStateChecker partitionProducerStateChecker = mock(PartitionProducerStateChecker.class); - Executor executor = mock(Executor.class); - - ShuffleEnvironment shuffleEnvironment = new NettyShuffleEnvironmentBuilder().build(); - - JobInformation jobInformation = new JobInformation( - new JobID(), - "Job Name", - new SerializedValue<>(new ExecutionConfig()), - new Configuration(), - Collections.emptyList(), - Collections.emptyList()); - - TaskInformation taskInformation = new TaskInformation( - new JobVertexID(), - "Test Task", - 1, - 1, - invokable.getName(), - taskConfig.getConfiguration()); - - return new Task( - jobInformation, - taskInformation, - new ExecutionAttemptID(), - new AllocationID(), - 0, - 0, - Collections.emptyList(), - Collections.emptyList(), - 0, - mock(MemoryManager.class), - mock(IOManager.class), - shuffleEnvironment, - new KvStateService(new KvStateRegistry(), null, null), - mock(BroadcastVariableManager.class), - new TaskEventDispatcher(), - taskStateManager, - taskManagerActions, - mock(InputSplitProvider.class), - mock(CheckpointResponder.class), - new NoOpTaskOperatorEventGateway(), - new TestGlobalAggregateManager(), - blobService, - libCache, - mock(FileCache.class), - new TestingTaskManagerRuntimeInfo(taskManagerConfig, new String[] {System.getProperty("java.io.tmpdir")}), - UnregisteredMetricGroups.createUnregisteredTaskMetricGroup(), - consumableNotifier, - partitionProducerStateChecker, - executor); + Configuration taskManagerConfig) throws Exception { + + return new TestTaskBuilder(shuffleEnvironment) + .setTaskManagerConfig(taskManagerConfig) + .setInvokable(invokable) + .setTaskConfig(taskConfig.getConfiguration()) + .build(); } // ------------------------------------------------------------------------ diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTask.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTask.java index d190924ea2..77e29044af 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTask.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTask.java @@ -26,13 +26,13 @@ import org.apache.flink.runtime.util.FatalExitExceptionHandler; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.api.operators.StreamTaskStateInitializer; +import org.apache.flink.streaming.runtime.io.StreamInputProcessor; import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeServiceFactory; import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor; import org.apache.flink.streaming.runtime.tasks.TimerService; -import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxDefaultAction; import org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailbox; import java.util.function.BiConsumer; @@ -66,8 +66,9 @@ public class MockStreamTask> extends StreamT TimerService timerService, BiConsumer handleAsyncException, TaskMailbox taskMailbox, - StreamTaskActionExecutor.SynchronizedStreamTaskActionExecutor taskActionExecutor - ) { + StreamTaskActionExecutor.SynchronizedStreamTaskActionExecutor taskActionExecutor, + StreamInputProcessor inputProcessor) { + super(environment, timerService, FatalExitExceptionHandler.INSTANCE, taskActionExecutor, taskMailbox); this.name = name; this.checkpointLock = checkpointLock; @@ -79,17 +80,13 @@ public class MockStreamTask> extends StreamT this.checkpointStorage = checkpointStorage; this.processingTimeService = timerService; this.handleAsyncException = handleAsyncException; + this.inputProcessor = inputProcessor; } @Override public void init() { } - @Override - protected void processInput(MailboxDefaultAction.Controller controller) throws Exception { - controller.allActionsCompleted(); - } - @Override protected void cleanup() { mailboxProcessor.allActionsCompleted(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTaskBuilder.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTaskBuilder.java index 680fac0a86..946809e5a5 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTaskBuilder.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTaskBuilder.java @@ -20,7 +20,6 @@ package org.apache.flink.streaming.util; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; -import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.CloseableRegistry; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.state.CheckpointStorage; @@ -31,6 +30,7 @@ import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.MockStreamStatusMaintainer; import org.apache.flink.streaming.api.operators.StreamTaskStateInitializer; import org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl; +import org.apache.flink.streaming.runtime.io.StreamInputProcessor; import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer; import org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor; import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; @@ -38,6 +38,8 @@ import org.apache.flink.streaming.runtime.tasks.TimerService; import org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailbox; import org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailboxImpl; +import javax.annotation.Nullable; + import java.util.function.BiConsumer; /** @@ -47,7 +49,7 @@ public class MockStreamTaskBuilder { private final Environment environment; private String name = "Mock Task"; private Object checkpointLock = new Object(); - private StreamConfig config = new StreamConfig(new Configuration()); + private StreamConfig config; private ExecutionConfig executionConfig = new ExecutionConfig(); private CloseableRegistry closableRegistry = new CloseableRegistry(); private StreamStatusMaintainer streamStatusMaintainer = new MockStreamStatusMaintainer(); @@ -57,9 +59,12 @@ public class MockStreamTaskBuilder { private BiConsumer handleAsyncException = (message, throwable) -> { }; private TaskMailbox taskMailbox = new TaskMailboxImpl(); private StreamTaskActionExecutor.SynchronizedStreamTaskActionExecutor taskActionExecutor = StreamTaskActionExecutor.synchronizedExecutor(); + @Nullable + private StreamInputProcessor inputProcessor; public MockStreamTaskBuilder(Environment environment) throws Exception { this.environment = environment; + this.config = new StreamConfig(environment.getTaskConfiguration()); StateBackend stateBackend = new MemoryStateBackend(); this.checkpointStorage = stateBackend.createCheckpointStorage(new JobID()); @@ -126,6 +131,11 @@ public class MockStreamTaskBuilder { return this; } + public MockStreamTaskBuilder setStreamInputProcessor(StreamInputProcessor inputProcessor) { + this.inputProcessor = inputProcessor; + return this; + } + public MockStreamTask build() { return new MockStreamTask( environment, @@ -140,6 +150,7 @@ public class MockStreamTaskBuilder { timerService, handleAsyncException, taskMailbox, - taskActionExecutor); + taskActionExecutor, + inputProcessor); } } -- Gitee From 54395d9fcb3ab7b7ad6548e9003e24c24f9081c4 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Sun, 22 Mar 2020 15:35:13 +0800 Subject: [PATCH 234/885] [FLINK-16345][table-planner-blink] Fix computed column can not refer row time attribute column This closes #11424 --- .../plan/schema/CatalogSourceTable.scala | 37 +++++++++++++++++-- .../planner/plan/stream/sql/TableScanTest.xml | 34 ++++++++++++++--- .../plan/stream/sql/TableScanTest.scala | 19 ++++++++++ 3 files changed, 80 insertions(+), 10 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/CatalogSourceTable.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/CatalogSourceTable.scala index 12801cca7f..72764fe543 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/CatalogSourceTable.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/CatalogSourceTable.scala @@ -22,7 +22,7 @@ import org.apache.flink.configuration.ReadableConfig import org.apache.flink.table.api.{TableException, ValidationException} import org.apache.flink.table.catalog.CatalogTable import org.apache.flink.table.factories.{TableFactoryUtil, TableSourceFactory, TableSourceFactoryContextImpl} -import org.apache.flink.table.planner.calcite.{FlinkContext, FlinkRelBuilder} +import org.apache.flink.table.planner.calcite.{FlinkContext, FlinkRelBuilder, FlinkTypeFactory} import org.apache.flink.table.planner.catalog.CatalogSchemaTable import org.apache.flink.table.sources.{StreamTableSource, TableSource, TableSourceValidation} import org.apache.flink.table.utils.TableConnectorUtils.generateRuntimeName @@ -30,6 +30,7 @@ import org.apache.calcite.plan.{RelOptSchema, RelOptTable} import org.apache.calcite.rel.RelNode import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.logical.LogicalTableScan +import org.apache.flink.table.types.logical.{TimestampKind, TimestampType} import java.util import java.util.{List => JList} @@ -78,12 +79,16 @@ class CatalogSourceTable[T]( .getPlanner .getContext .unwrap(classOf[FlinkContext]) + val typeFactory = cluster.getTypeFactory.asInstanceOf[FlinkTypeFactory] + + // erase time indicator types in the rowType + val erasedRowType = eraseTimeIndicator(rowType, typeFactory) val tableSource = findAndCreateTableSource(flinkContext.getTableConfig.getConfiguration) val tableSourceTable = new TableSourceTable[T]( relOptSchema, schemaTable.getTableIdentifier, - rowType, + erasedRowType, statistic, tableSource, schemaTable.isStreamingMode, @@ -106,7 +111,7 @@ class CatalogSourceTable[T]( val toRexFactory = flinkContext.getSqlExprToRexConverterFactory // 2. push computed column project - val fieldNames = rowType.getFieldNames.asScala + val fieldNames = erasedRowType.getFieldNames.asScala if (columnExprs.nonEmpty) { val fieldExprs = fieldNames .map { name => @@ -139,7 +144,7 @@ class CatalogSourceTable[T]( } val rowtimeIndex = fieldNames.indexOf(rowtime) val watermarkRexNode = toRexFactory - .create(rowType) + .create(erasedRowType) .convertToRexNode(watermarkSpec.get.getWatermarkExpr) relBuilder.watermark(rowtimeIndex, watermarkRexNode) } @@ -179,4 +184,28 @@ class CatalogSourceTable[T]( tableSource.asInstanceOf[TableSource[T]] } + + /** + * Erases time indicators, i.e. converts rowtime and proctime type into regular timestamp type. + * This is required before converting this [[CatalogSourceTable]] into multiple RelNodes, + * otherwise the derived data types are mismatch. + */ + private def eraseTimeIndicator( + relDataType: RelDataType, + factory: FlinkTypeFactory): RelDataType = { + val logicalRowType = FlinkTypeFactory.toLogicalRowType(relDataType) + val fieldNames = logicalRowType.getFieldNames + val fieldTypes = logicalRowType.getFields.map { f => + if (FlinkTypeFactory.isTimeIndicatorType(f.getType)) { + val timeIndicatorType = f.getType.asInstanceOf[TimestampType] + new TimestampType( + timeIndicatorType.isNullable, + TimestampKind.REGULAR, + timeIndicatorType.getPrecision) + } else { + f.getType + } + } + factory.buildRelNodeRowType(fieldNames, fieldTypes) + } } diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml index 693a1713f2..1c54d3e258 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml @@ -49,6 +49,23 @@ LogicalProject(ts=[$0], a=[$1], b=[$2]) Calc(select=[ts, a, b], where=[>(a, 1)]) +- WatermarkAssigner(rowtime=[ts], watermark=[-(ts, 1:INTERVAL SECOND)]) +- TableSourceScan(table=[[default_catalog, default_database, src, source: [CollectionTableSource(ts, a, b)]]], fields=[ts, a, b]) +]]> + + + + + + + + + + + @@ -68,20 +85,25 @@ TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [Tes ]]> - + - + 1]]> ($1, 1)]) + +- LogicalWatermarkAssigner(rowtime=[ts], watermark=[-($0, 1:INTERVAL SECOND)]) + +- LogicalProject(ts=[$0], a=[$1], b=[$2], my_ts=[-($0, 1:INTERVAL SECOND)], proc=[PROCTIME()]) + +- LogicalTableScan(table=[[default_catalog, default_database, src, source: [CollectionTableSource(ts, a, b)]]]) ]]> (a, 1)]) ++- WatermarkAssigner(rowtime=[ts], watermark=[-(ts, 1:INTERVAL SECOND)]) + +- Calc(select=[ts, a, b, -(ts, 1:INTERVAL SECOND) AS my_ts, PROCTIME() AS proc]) + +- TableSourceScan(table=[[default_catalog, default_database, src, source: [CollectionTableSource(ts, a, b)]]], fields=[ts, a, b]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala index edc9c3b1d2..0ae7cb70e8 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala @@ -98,6 +98,25 @@ class TableScanTest extends TableTestBase { util.verifyPlan("SELECT * FROM t1") } + @Test + def testDDLWithComputedColumnReferRowtime(): Unit = { + util.addTable( + """ + |CREATE TABLE src ( + | ts TIMESTAMP(3), + | a INT, + | b DOUBLE, + | my_ts AS ts - INTERVAL '0.001' SECOND, + | proc AS PROCTIME(), + | WATERMARK FOR ts AS ts - INTERVAL '0.001' SECOND + |) WITH ( + | 'connector' = 'COLLECTION', + | 'is-bounded' = 'false' + |) + """.stripMargin) + util.verifyPlan("SELECT * FROM src WHERE a > 1") + } + @Test def testKeywordsWithWatermarkComputedColumn(): Unit = { // Create table with field as atom expression. -- Gitee From 044733db92cc3fa825923e88a51416aa8d9476c6 Mon Sep 17 00:00:00 2001 From: Rui Li Date: Mon, 23 Mar 2020 10:47:11 +0800 Subject: [PATCH 235/885] [FLINK-16532][core] Shouldn't trim whitespaces in Path This closes #11393 --- .../hive/TableEnvHiveConnectorTest.java | 15 +++++++++++++++ .../main/java/org/apache/flink/core/fs/Path.java | 16 +++++----------- .../java/org/apache/flink/core/fs/PathTest.java | 11 ----------- 3 files changed, 20 insertions(+), 22 deletions(-) diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java index 0e4f9bea5f..5a3726fc8b 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java @@ -528,6 +528,21 @@ public class TableEnvHiveConnectorTest { } } + @Test + public void testWhitespacePartValue() throws Exception { + hiveShell.execute("create database db1"); + try { + hiveShell.execute("create table db1.dest (x int) partitioned by (p string)"); + TableEnvironment tableEnv = getTableEnvWithHiveCatalog(); + tableEnv.sqlUpdate("insert into db1.dest select 1,' '"); + tableEnv.sqlUpdate("insert into db1.dest select 2,'a \t'"); + tableEnv.execute("insert"); + assertEquals("[p= , p=a %09]", hiveShell.executeQuery("show partitions db1.dest").toString()); + } finally { + hiveShell.execute("drop database db1 cascade"); + } + } + private TableEnvironment getTableEnvWithHiveCatalog() { TableEnvironment tableEnv = HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(); tableEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog); diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/Path.java b/flink-core/src/main/java/org/apache/flink/core/fs/Path.java index 1334acc214..3cb0271f2e 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/Path.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/Path.java @@ -146,24 +146,22 @@ public class Path implements IOReadableWritable, Serializable { } final URI resolved = parentUri.resolve(child.uri); - initialize(resolved.getScheme(), resolved.getAuthority(), normalizePath(resolved.getPath())); + initialize(resolved.getScheme(), resolved.getAuthority(), resolved.getPath()); } /** * Checks if the provided path string is either null or has zero length and throws * a {@link IllegalArgumentException} if any of the two conditions apply. - * In addition, leading and tailing whitespaces are removed. * * @param path * the path string to be checked - * @return The checked and trimmed path. + * @return The checked path. */ - private String checkAndTrimPathArg(String path) { + private String checkPathArg(String path) { // disallow construction of a Path from an empty string if (path == null) { throw new IllegalArgumentException("Can not create a Path from a null string"); } - path = path.trim(); if (path.length() == 0) { throw new IllegalArgumentException("Can not create a Path from an empty string"); } @@ -178,7 +176,7 @@ public class Path implements IOReadableWritable, Serializable { * the string to construct a path from */ public Path(String pathString) { - pathString = checkAndTrimPathArg(pathString); + pathString = checkPathArg(pathString); // We can't use 'new URI(String)' directly, since it assumes things are // escaped, which we don't require of Paths. @@ -228,7 +226,7 @@ public class Path implements IOReadableWritable, Serializable { * the path string */ public Path(String scheme, String authority, String path) { - path = checkAndTrimPathArg(path); + path = checkPathArg(path); initialize(scheme, authority, path); } @@ -258,10 +256,6 @@ public class Path implements IOReadableWritable, Serializable { * @return the normalized path string */ private String normalizePath(String path) { - - // remove leading and tailing whitespaces - path = path.trim(); - // remove consecutive slashes & backslashes path = path.replace("\\", "/"); path = path.replaceAll("/+", "/"); diff --git a/flink-core/src/test/java/org/apache/flink/core/fs/PathTest.java b/flink-core/src/test/java/org/apache/flink/core/fs/PathTest.java index 6d53adb2da..41029b40d7 100644 --- a/flink-core/src/test/java/org/apache/flink/core/fs/PathTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/fs/PathTest.java @@ -57,10 +57,6 @@ public class PathTest { assertEquals("/my/path/a", p.toUri().getPath()); assertNull(p.toUri().getScheme()); - p = new Path("/my/path/ "); - assertEquals("/my/path", p.toUri().getPath()); - assertNull(p.toUri().getScheme()); - p = new Path("hdfs:///my/path"); assertEquals("/my/path", p.toUri().getPath()); assertEquals("hdfs", p.toUri().getScheme()); @@ -93,13 +89,6 @@ public class PathTest { // exception expected } - try { - new Path(" "); - fail(); - } catch (Exception e) { - // exception expected - } - } @Test -- Gitee From 816bff8db8ff34caf1ee37ac3dd4897835e744dc Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Mon, 23 Mar 2020 12:57:20 +0800 Subject: [PATCH 236/885] [FLINK-16711][parquet] Parquet columnar row reader read footer from wrong end This closes #11484 --- .../formats/parquet/vector/ParquetColumnarRowSplitReader.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetColumnarRowSplitReader.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetColumnarRowSplitReader.java index e7bf0a742f..8335381f40 100644 --- a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetColumnarRowSplitReader.java +++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/vector/ParquetColumnarRowSplitReader.java @@ -117,7 +117,7 @@ public class ParquetColumnarRowSplitReader implements Closeable { this.selectedTypes = selectedTypes; this.batchSize = batchSize; // then we need to apply the predicate push down filter - ParquetMetadata footer = readFooter(conf, path, range(splitStart, splitLength)); + ParquetMetadata footer = readFooter(conf, path, range(splitStart, splitStart + splitLength)); MessageType fileSchema = footer.getFileMetaData().getSchema(); FilterCompat.Filter filter = getFilter(conf); List blocks = filterRowGroups(filter, footer.getBlocks(), fileSchema); -- Gitee From cdcc25cee9c5702c94376c3843def6c59849ec96 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Mon, 23 Mar 2020 13:55:54 +0800 Subject: [PATCH 237/885] [FLINK-16652][orc] BytesColumnVector should init buffer in Hive 3.x This closes #11485 --- .../apache/flink/orc/vector/AbstractOrcColumnVector.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/AbstractOrcColumnVector.java b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/AbstractOrcColumnVector.java index c9c64e2058..2cd518438d 100644 --- a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/AbstractOrcColumnVector.java +++ b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/vector/AbstractOrcColumnVector.java @@ -138,9 +138,11 @@ public abstract class AbstractOrcColumnVector implements bcv.isNull[0] = true; bcv.isRepeating = true; } else { - bcv.fill(value instanceof byte[] ? + byte[] bytes = value instanceof byte[] ? (byte[]) value : - value.toString().getBytes(StandardCharsets.UTF_8)); + value.toString().getBytes(StandardCharsets.UTF_8); + bcv.initBuffer(bytes.length); + bcv.fill(bytes); bcv.isNull[0] = false; } return bcv; -- Gitee From f56a075aa029832b9cbacab51649c900498e18e9 Mon Sep 17 00:00:00 2001 From: felixzheng Date: Sat, 7 Mar 2020 21:33:17 +0800 Subject: [PATCH 238/885] [FLINK-15667][k8s] Support to mount custom Hadoop Configurations This closes #11415 . --- .../kubernetes_config_configuration.html | 6 + .../KubernetesConfigOptions.java | 7 + .../decorators/FlinkConfMountDecorator.java | 2 +- .../decorators/HadoopConfMountDecorator.java | 186 ++++++++++++++++++ .../factory/KubernetesJobManagerFactory.java | 2 + .../factory/KubernetesTaskManagerFactory.java | 2 + .../AbstractKubernetesParameters.java | 29 +++ .../parameters/KubernetesParameters.java | 11 ++ .../flink/kubernetes/utils/Constants.java | 8 +- .../flink/kubernetes/KubernetesTestBase.java | 16 ++ .../HadoopConfMountDecoratorTest.java | 178 +++++++++++++++++ .../KubernetesJobManagerFactoryTest.java | 53 ++++- .../KubernetesTaskManagerFactoryTest.java | 14 +- 13 files changed, 507 insertions(+), 7 deletions(-) create mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/HadoopConfMountDecorator.java create mode 100644 flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/HadoopConfMountDecoratorTest.java diff --git a/docs/_includes/generated/kubernetes_config_configuration.html b/docs/_includes/generated/kubernetes_config_configuration.html index aad4edf5e1..6fa57f217f 100644 --- a/docs/_includes/generated/kubernetes_config_configuration.html +++ b/docs/_includes/generated/kubernetes_config_configuration.html @@ -68,6 +68,12 @@

    String The directory that logs of jobmanager and taskmanager be saved in the pod.
    kubernetes.hadoop.conf.config-map.name
    (none)StringSpecify the name of an existing ConfigMap that contains custom Hadoop configuration to be mounted on the JobManager(s) and TaskManagers.
    kubernetes.jobmanager.cpu
    1.0
    + + + + + + + + + + + + + + + + + + + + + + + + + +
    /taskmanagers/:taskmanagerid/logs
    Verb: GETResponse code: 200 OK
    Returns the list of log files on a TaskManager.
    Path parameters
    +
      +
    • taskmanagerid - 32-character hexadecimal string that identifies a task manager.
    • +
    +
    + +
    +
    +            
    +{}            
    +          
    +
    +
    + +
    +
    +            
    +{
    +  "type" : "object",
    +  "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:taskmanager:LogListInfo",
    +  "properties" : {
    +    "logs" : {
    +      "type" : "array",
    +      "items" : {
    +        "type" : "object",
    +        "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:taskmanager:LogInfo",
    +        "properties" : {
    +          "name" : {
    +            "type" : "string"
    +          },
    +          "size" : {
    +            "type" : "integer"
    +          }
    +        }
    +      }
    +    }
    +  }
    +}            
    +          
    +
    +
    diff --git a/flink-runtime-web/src/test/resources/rest_api_v1.snapshot b/flink-runtime-web/src/test/resources/rest_api_v1.snapshot index 3823602185..687405c9f4 100644 --- a/flink-runtime-web/src/test/resources/rest_api_v1.snapshot +++ b/flink-runtime-web/src/test/resources/rest_api_v1.snapshot @@ -2684,6 +2684,43 @@ } } } + }, { + "url" : "/taskmanagers/:taskmanagerid/logs", + "method" : "GET", + "status-code" : "200 OK", + "file-upload" : false, + "path-parameters" : { + "pathParameters" : [ { + "key" : "taskmanagerid" + } ] + }, + "query-parameters" : { + "queryParameters" : [ ] + }, + "request" : { + "type" : "any" + }, + "response" : { + "type" : "object", + "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:taskmanager:LogListInfo", + "properties" : { + "logs" : { + "type" : "array", + "items" : { + "type" : "object", + "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:taskmanager:LogInfo", + "properties" : { + "name" : { + "type" : "string" + }, + "size" : { + "type" : "integer" + } + } + } + } + } + } }, { "url" : "/taskmanagers/:taskmanagerid/metrics", "method" : "GET", diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java index b5f7b25175..e3e1b33ba1 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java @@ -54,6 +54,7 @@ import org.apache.flink.runtime.resourcemanager.registration.JobManagerRegistrat import org.apache.flink.runtime.resourcemanager.registration.WorkerRegistration; import org.apache.flink.runtime.resourcemanager.slotmanager.ResourceActions; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; +import org.apache.flink.runtime.rest.messages.taskmanager.LogInfo; import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerInfo; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.FencedRpcEndpoint; @@ -582,16 +583,41 @@ public abstract class ResourceManager } @Override - public CompletableFuture requestTaskManagerFileUpload(ResourceID taskManagerId, FileType fileType, Time timeout) { - log.debug("Request file {} upload from TaskExecutor {}.", fileType, taskManagerId); + public CompletableFuture requestTaskManagerFileUploadByType(ResourceID taskManagerId, FileType fileType, Time timeout) { + log.debug("Request {} file upload from TaskExecutor {}.", fileType, taskManagerId); final WorkerRegistration taskExecutor = taskExecutors.get(taskManagerId); if (taskExecutor == null) { - log.debug("Requested file {} upload from unregistered TaskExecutor {}.", fileType, taskManagerId); + log.debug("Request upload of file {} from unregistered TaskExecutor {}.", fileType, taskManagerId); return FutureUtils.completedExceptionally(new UnknownTaskExecutorException(taskManagerId)); } else { - return taskExecutor.getTaskExecutorGateway().requestFileUpload(fileType, timeout); + return taskExecutor.getTaskExecutorGateway().requestFileUploadByType(fileType, timeout); + } + } + + @Override + public CompletableFuture requestTaskManagerFileUploadByName(ResourceID taskManagerId, String fileName, Time timeout) { + log.debug("Request upload of file {} from TaskExecutor {}.", fileName, taskManagerId); + + final WorkerRegistration taskExecutor = taskExecutors.get(taskManagerId); + + if (taskExecutor == null) { + log.debug("Request upload of file {} from unregistered TaskExecutor {}.", fileName, taskManagerId); + return FutureUtils.completedExceptionally(new UnknownTaskExecutorException(taskManagerId)); + } else { + return taskExecutor.getTaskExecutorGateway().requestFileUploadByName(fileName, timeout); + } + } + + @Override + public CompletableFuture> requestTaskManagerLogList(ResourceID taskManagerId, Time timeout) { + final WorkerRegistration taskExecutor = taskExecutors.get(taskManagerId); + if (taskExecutor == null) { + log.debug("Requested log list from unregistered TaskExecutor {}.", taskManagerId); + return FutureUtils.completedExceptionally(new UnknownTaskExecutorException(taskManagerId)); + } else { + return taskExecutor.getTaskExecutorGateway().requestLogList(timeout); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java index 0d9dc4ae4c..1901bd19a1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java @@ -33,6 +33,7 @@ import org.apache.flink.runtime.jobmaster.JobMasterId; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.metrics.dump.MetricQueryService; import org.apache.flink.runtime.registration.RegistrationResponse; +import org.apache.flink.runtime.rest.messages.taskmanager.LogInfo; import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerInfo; import org.apache.flink.runtime.rpc.FencedRpcGateway; import org.apache.flink.runtime.rpc.RpcTimeout; @@ -215,5 +216,25 @@ public interface ResourceManagerGateway extends FencedRpcGateway requestTaskManagerFileUpload(ResourceID taskManagerId, FileType fileType, @RpcTimeout Time timeout); + CompletableFuture requestTaskManagerFileUploadByType(ResourceID taskManagerId, FileType fileType, @RpcTimeout Time timeout); + + /** + * Request the file upload from the given {@link TaskExecutor} to the cluster's {@link BlobServer}. The + * corresponding {@link TransientBlobKey} is returned. + * + * @param taskManagerId identifying the {@link TaskExecutor} to upload the specified file + * @param fileName name of the file to upload + * @param timeout for the asynchronous operation + * @return Future which is completed with the {@link TransientBlobKey} after uploading the file to the + * {@link BlobServer}. + */ + CompletableFuture requestTaskManagerFileUploadByName(ResourceID taskManagerId, String fileName, @RpcTimeout Time timeout); + + /** + * Request log list from the given {@link TaskExecutor}. + * @param taskManagerId identifying the {@link TaskExecutor} to get log list from + * @param timeout for the asynchronous operation + * @return Future which is completed with the historical log list + */ + CompletableFuture> requestTaskManagerLogList(ResourceID taskManagerId, @RpcTimeout Time timeout); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java index 17d1b178ba..ff4b97e3f3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.rest.handler.taskmanager; import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.blob.TransientBlobKey; import org.apache.flink.runtime.blob.TransientBlobService; import org.apache.flink.runtime.clusterframework.types.ResourceID; @@ -85,7 +86,7 @@ public abstract class AbstractTaskManagerFileHandler resourceManagerGatewayRetriever; private final TransientBlobService transientBlobService; - private final LoadingCache> fileBlobKeys; + private final LoadingCache, CompletableFuture> fileBlobKeys; protected AbstractTaskManagerFileHandler( @Nonnull GatewayRetriever leaderRetriever, @@ -106,10 +107,10 @@ public abstract class AbstractTaskManagerFileHandler>() { + new CacheLoader, CompletableFuture>() { @Override - public CompletableFuture load(ResourceID resourceId) throws Exception { - return loadTaskManagerFile(resourceId); + public CompletableFuture load(Tuple2 taskManagerIdAndFileName) throws Exception { + return loadTaskManagerFile(taskManagerIdAndFileName); } }); } @@ -118,9 +119,11 @@ public abstract class AbstractTaskManagerFileHandler respondToRequest(ChannelHandlerContext ctx, HttpRequest httpRequest, HandlerRequest handlerRequest, RestfulGateway gateway) throws RestHandlerException { final ResourceID taskManagerId = handlerRequest.getPathParameter(TaskManagerIdPathParameter.class); + String filename = getFileName(handlerRequest); + final Tuple2 taskManagerIdAndFileName = new Tuple2<>(taskManagerId, filename); final CompletableFuture blobKeyFuture; try { - blobKeyFuture = fileBlobKeys.get(taskManagerId); + blobKeyFuture = fileBlobKeys.get(taskManagerIdAndFileName); } catch (ExecutionException e) { final Throwable cause = ExceptionUtils.stripExecutionException(e); throw new RestHandlerException("Could not retrieve file blob key future.", HttpResponseStatus.INTERNAL_SERVER_ERROR, cause); @@ -169,10 +172,8 @@ public abstract class AbstractTaskManagerFileHandler requestFileUpload(ResourceManagerGateway resourceManagerGateway, ResourceID taskManagerResourceId); - - private CompletableFuture loadTaskManagerFile(ResourceID taskManagerResourceId) throws RestHandlerException { - log.debug("Load file from TaskManager {}.", taskManagerResourceId); + private CompletableFuture loadTaskManagerFile(Tuple2 taskManagerIdAndFileName) throws RestHandlerException { + log.debug("Load file from TaskManager {}.", taskManagerIdAndFileName.f0); final ResourceManagerGateway resourceManagerGateway = resourceManagerGatewayRetriever .getNow() @@ -183,10 +184,12 @@ public abstract class AbstractTaskManagerFileHandler> removalNotification) { + protected abstract CompletableFuture requestFileUpload(ResourceManagerGateway resourceManagerGateway, Tuple2 taskManagerIdAndFileName); + + private void removeBlob(RemovalNotification, CompletableFuture> removalNotification) { log.debug("Remove cached file for TaskExecutor {}.", removalNotification.getKey()); final CompletableFuture value = removalNotification.getValue(); @@ -263,4 +266,8 @@ public abstract class AbstractTaskManagerFileHandler handlerRequest) { + return null; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerHandler.java index d17dbff8e2..42b5355188 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerHandler.java @@ -37,7 +37,6 @@ import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseSt import javax.annotation.Nonnull; import java.util.Map; -import java.util.Optional; import java.util.concurrent.CompletableFuture; /** @@ -65,13 +64,18 @@ abstract class AbstractTaskManagerHandler handleRequest(@Nonnull HandlerRequest request, @Nonnull T gateway) throws RestHandlerException { - Optional resourceManagerGatewayOptional = resourceManagerGatewayRetriever.getNow(); - - ResourceManagerGateway resourceManagerGateway = resourceManagerGatewayOptional.orElseThrow( - () -> new RestHandlerException("Cannot connect to ResourceManager right now. Please try to refresh.", HttpResponseStatus.NOT_FOUND)); + ResourceManagerGateway resourceManagerGateway = getResourceManagerGateway(resourceManagerGatewayRetriever); return handleRequest(request, resourceManagerGateway); } protected abstract CompletableFuture

    handleRequest(@Nonnull HandlerRequest request, @Nonnull ResourceManagerGateway gateway) throws RestHandlerException; + + protected ResourceManagerGateway getResourceManagerGateway(GatewayRetriever resourceManagerGatewayRetriever) throws RestHandlerException { + return resourceManagerGatewayRetriever + .getNow() + .orElseThrow(() -> new RestHandlerException( + "Cannot connect to ResourceManager right now. Please try to refresh.", + HttpResponseStatus.NOT_FOUND)); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerCustomLogHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerCustomLogHandler.java new file mode 100644 index 0000000000..0ad1a05d20 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerCustomLogHandler.java @@ -0,0 +1,67 @@ +/* + * 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.runtime.rest.handler.taskmanager; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.blob.TransientBlobKey; +import org.apache.flink.runtime.blob.TransientBlobService; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.rest.handler.HandlerRequest; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.UntypedResponseMessageHeaders; +import org.apache.flink.runtime.rest.messages.taskmanager.LogFileNamePathParameter; +import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerFileMessageParameters; +import org.apache.flink.runtime.taskexecutor.TaskExecutor; +import org.apache.flink.runtime.webmonitor.RestfulGateway; +import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; + +import javax.annotation.Nonnull; + +import java.util.Map; +import java.util.concurrent.CompletableFuture; + +/** + * Rest handler which serves the custom file of the {@link TaskExecutor}. + */ +public class TaskManagerCustomLogHandler extends AbstractTaskManagerFileHandler { + + public TaskManagerCustomLogHandler( + @Nonnull GatewayRetriever leaderRetriever, + @Nonnull Time timeout, + @Nonnull Map responseHeaders, + @Nonnull UntypedResponseMessageHeaders untypedResponseMessageHeaders, + @Nonnull GatewayRetriever resourceManagerGatewayRetriever, + @Nonnull TransientBlobService transientBlobService, + @Nonnull Time cacheEntryDuration) { + super(leaderRetriever, timeout, responseHeaders, untypedResponseMessageHeaders, resourceManagerGatewayRetriever, transientBlobService, cacheEntryDuration); + + } + + @Override + protected CompletableFuture requestFileUpload(ResourceManagerGateway resourceManagerGateway, Tuple2 taskManagerIdAndFileName) { + return resourceManagerGateway.requestTaskManagerFileUploadByName(taskManagerIdAndFileName.f0, taskManagerIdAndFileName.f1, timeout); + } + + @Override + protected String getFileName(HandlerRequest handlerRequest) { + return handlerRequest.getPathParameter(LogFileNamePathParameter.class); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerLogFileHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerLogFileHandler.java index 0e90696d0e..945b5d4589 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerLogFileHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerLogFileHandler.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.rest.handler.taskmanager; import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.blob.TransientBlobKey; import org.apache.flink.runtime.blob.TransientBlobService; import org.apache.flink.runtime.clusterframework.types.ResourceID; @@ -53,7 +54,7 @@ public class TaskManagerLogFileHandler extends AbstractTaskManagerFileHandler requestFileUpload(ResourceManagerGateway resourceManagerGateway, ResourceID taskManagerResourceId) { - return resourceManagerGateway.requestTaskManagerFileUpload(taskManagerResourceId, FileType.LOG, timeout); + protected CompletableFuture requestFileUpload(ResourceManagerGateway resourceManagerGateway, Tuple2 taskManagerIdAndFileName) { + return resourceManagerGateway.requestTaskManagerFileUploadByType(taskManagerIdAndFileName.f0, FileType.LOG, timeout); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerLogListHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerLogListHandler.java new file mode 100644 index 0000000000..71dbe3f94b --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerLogListHandler.java @@ -0,0 +1,88 @@ +/* + * 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.runtime.rest.handler.taskmanager; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.resourcemanager.exceptions.UnknownTaskExecutorException; +import org.apache.flink.runtime.rest.handler.HandlerRequest; +import org.apache.flink.runtime.rest.handler.RestHandlerException; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.MessageHeaders; +import org.apache.flink.runtime.rest.messages.taskmanager.LogInfo; +import org.apache.flink.runtime.rest.messages.taskmanager.LogListInfo; +import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerIdPathParameter; +import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerMessageParameters; +import org.apache.flink.runtime.webmonitor.RestfulGateway; +import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.Preconditions; + +import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; + +import javax.annotation.Nonnull; + +import java.util.Collection; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; + +/** + * Handler which serves detailed TaskManager log list information. + */ +public class TaskManagerLogListHandler extends AbstractTaskManagerHandler { + + private final GatewayRetriever resourceManagerGatewayRetriever; + + public TaskManagerLogListHandler( + GatewayRetriever leaderRetriever, + Time timeout, + Map responseHeaders, + MessageHeaders messageHeaders, + GatewayRetriever resourceManagerGatewayRetriever) { + super(leaderRetriever, timeout, responseHeaders, messageHeaders, resourceManagerGatewayRetriever); + + this.resourceManagerGatewayRetriever = Preconditions.checkNotNull(resourceManagerGatewayRetriever); + } + + @Override + protected CompletableFuture handleRequest( + @Nonnull HandlerRequest request, + @Nonnull ResourceManagerGateway gateway) throws RestHandlerException { + final ResourceID taskManagerId = request.getPathParameter(TaskManagerIdPathParameter.class); + final ResourceManagerGateway resourceManagerGateway = getResourceManagerGateway(resourceManagerGatewayRetriever); + final CompletableFuture> logsWithLengthFuture = resourceManagerGateway.requestTaskManagerLogList(taskManagerId, timeout); + + return logsWithLengthFuture.thenApply(LogListInfo::new).exceptionally( + (throwable) -> { + final Throwable strippedThrowable = ExceptionUtils.stripCompletionException(throwable); + if (strippedThrowable instanceof UnknownTaskExecutorException) { + throw new CompletionException( + new RestHandlerException( + "Could not find TaskExecutor " + taskManagerId, + HttpResponseStatus.NOT_FOUND, + strippedThrowable + )); + } else { + throw new CompletionException(throwable); + } + }); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerStdoutFileHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerStdoutFileHandler.java index 0154224294..f68069d1ab 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerStdoutFileHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerStdoutFileHandler.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.rest.handler.taskmanager; import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.blob.TransientBlobKey; import org.apache.flink.runtime.blob.TransientBlobService; import org.apache.flink.runtime.clusterframework.types.ResourceID; @@ -53,7 +54,7 @@ public class TaskManagerStdoutFileHandler extends AbstractTaskManagerFileHandler } @Override - protected CompletableFuture requestFileUpload(ResourceManagerGateway resourceManagerGateway, ResourceID taskManagerResourceId) { - return resourceManagerGateway.requestTaskManagerFileUpload(taskManagerResourceId, FileType.STDOUT, timeout); + protected CompletableFuture requestFileUpload(ResourceManagerGateway resourceManagerGateway, Tuple2 taskManagerIdAndFileName) { + return resourceManagerGateway.requestTaskManagerFileUploadByType(taskManagerIdAndFileName.f0, FileType.STDOUT, timeout); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/LogFileNamePathParameter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/LogFileNamePathParameter.java new file mode 100644 index 0000000000..310ae24c83 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/LogFileNamePathParameter.java @@ -0,0 +1,49 @@ +/* + * 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.runtime.rest.messages.taskmanager; + +import org.apache.flink.runtime.rest.handler.taskmanager.TaskManagerCustomLogHandler; +import org.apache.flink.runtime.rest.messages.MessagePathParameter; + +/** + * TaskManager id path parameter used by {@link TaskManagerCustomLogHandler}. + */ +public class LogFileNamePathParameter extends MessagePathParameter { + + public static final String KEY = "filename"; + + public LogFileNamePathParameter() { + super(KEY); + } + + @Override + protected String convertFromString(String value) { + return value; + } + + @Override + protected String convertToString(String value) { + return value; + } + + @Override + public String getDescription() { + return "String value that identifies the file name from which to read."; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/LogInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/LogInfo.java new file mode 100644 index 0000000000..ca3c28de76 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/LogInfo.java @@ -0,0 +1,68 @@ +/* + * 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.runtime.rest.messages.taskmanager; + +import org.apache.flink.util.Preconditions; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import java.io.Serializable; +import java.util.Objects; + +/** + * Contains information about one log of TaskManager. + */ +public class LogInfo implements Serializable { + + public static final String NAME = "name"; + + public static final String SIZE = "size"; + + private static final long serialVersionUID = -7371944349031708629L; + + @JsonProperty(NAME) + private final String name; + + @JsonProperty(SIZE) + private final long size; + + @JsonCreator + public LogInfo(@JsonProperty(NAME) String name, @JsonProperty(SIZE) long size) { + this.name = Preconditions.checkNotNull(name); + this.size = size; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + LogInfo that = (LogInfo) o; + return Objects.equals(name, that.name) && size == that.size; + } + + @Override + public int hashCode() { + return Objects.hash(name, size); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/LogListInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/LogListInfo.java new file mode 100644 index 0000000000..7f8114bcb7 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/LogListInfo.java @@ -0,0 +1,73 @@ +/* + * 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.runtime.rest.messages.taskmanager; + +import org.apache.flink.runtime.rest.messages.ResponseBody; +import org.apache.flink.util.Preconditions; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import java.io.Serializable; +import java.util.Collection; +import java.util.Collections; +import java.util.Objects; + +/** + * Class containing a collection of {@link LogInfo}. + */ +public class LogListInfo implements ResponseBody, Serializable { + + public static final String FIELD_NAME_LOGS = "logs"; + + private static final long serialVersionUID = 7531494560450830517L; + + @JsonProperty(FIELD_NAME_LOGS) + private final Collection logInfos; + + @JsonCreator + public LogListInfo(@JsonProperty(FIELD_NAME_LOGS) Collection logInfos) { + this.logInfos = Preconditions.checkNotNull(logInfos); + } + + public Collection getLogInfos() { + return logInfos; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + LogListInfo that = (LogListInfo) o; + return Objects.equals(logInfos, that.logInfos); + } + + @Override + public int hashCode() { + return Objects.hash(logInfos); + } + + public static LogListInfo empty() { + return new LogListInfo(Collections.emptyList()); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerCustomLogHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerCustomLogHeaders.java new file mode 100644 index 0000000000..9c85ff0d1b --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerCustomLogHeaders.java @@ -0,0 +1,58 @@ +/* + * 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.runtime.rest.messages.taskmanager; + +import org.apache.flink.runtime.rest.HttpMethodWrapper; +import org.apache.flink.runtime.rest.handler.taskmanager.TaskManagerCustomLogHandler; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.UntypedResponseMessageHeaders; + +/** + * Headers for the {@link TaskManagerCustomLogHandler}. + */ +public class TaskManagerCustomLogHeaders implements UntypedResponseMessageHeaders { + + private static final TaskManagerCustomLogHeaders INSTANCE = new TaskManagerCustomLogHeaders(); + + private static final String URL = String.format("/taskmanagers/:%s/logs/:%s", TaskManagerIdPathParameter.KEY, LogFileNamePathParameter.KEY); + + @Override + public Class getRequestClass() { + return EmptyRequestBody.class; + } + + @Override + public TaskManagerFileMessageParameters getUnresolvedMessageParameters() { + return new TaskManagerFileMessageParameters(); + } + + @Override + public HttpMethodWrapper getHttpMethod() { + return HttpMethodWrapper.GET; + } + + @Override + public String getTargetRestEndpointURL() { + return URL; + } + + public static TaskManagerCustomLogHeaders getInstance() { + return INSTANCE; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerFileMessageParameters.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerFileMessageParameters.java new file mode 100644 index 0000000000..85181cfab6 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerFileMessageParameters.java @@ -0,0 +1,42 @@ +/* + * 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.runtime.rest.messages.taskmanager; + +import org.apache.flink.runtime.rest.handler.taskmanager.TaskManagerCustomLogHandler; +import org.apache.flink.runtime.rest.messages.MessagePathParameter; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; + +/** + * Parameters for {@link TaskManagerCustomLogHandler}. + */ +public class TaskManagerFileMessageParameters extends TaskManagerMessageParameters { + + public final LogFileNamePathParameter logFileNamePathParameter = new LogFileNamePathParameter(); + + @Override + public Collection> getPathParameters() { + return Collections.unmodifiableCollection(Arrays.asList( + logFileNamePathParameter, + taskManagerIdParameter + )); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerLogsHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerLogsHeaders.java new file mode 100644 index 0000000000..5f973f7988 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerLogsHeaders.java @@ -0,0 +1,77 @@ +/* + * 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.runtime.rest.messages.taskmanager; + +import org.apache.flink.runtime.rest.HttpMethodWrapper; +import org.apache.flink.runtime.rest.handler.taskmanager.TaskManagerLogListHandler; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.MessageHeaders; + +import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; + +/** + * Headers for the {@link TaskManagerLogListHandler}. + */ +public class TaskManagerLogsHeaders implements MessageHeaders { + + private static final TaskManagerLogsHeaders INSTANCE = new TaskManagerLogsHeaders(); + + private static final String URL = String.format("/taskmanagers/:%s/logs", TaskManagerIdPathParameter.KEY); + + private TaskManagerLogsHeaders() {} + + public static TaskManagerLogsHeaders getInstance() { + return INSTANCE; + } + + @Override + public Class getResponseClass() { + return LogListInfo.class; + } + + @Override + public HttpResponseStatus getResponseStatusCode() { + return HttpResponseStatus.OK; + } + + @Override + public String getDescription() { + return "Returns the list of log files on a TaskManager."; + } + + @Override + public Class getRequestClass() { + return EmptyRequestBody.class; + } + + @Override + public TaskManagerMessageParameters getUnresolvedMessageParameters() { + return new TaskManagerMessageParameters(); + } + + @Override + public HttpMethodWrapper getHttpMethod() { + return HttpMethodWrapper.GET; + } + + @Override + public String getTargetRestEndpointURL() { + return URL; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java index 73618187b0..70931eed07 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java @@ -83,6 +83,7 @@ import org.apache.flink.runtime.registration.RegistrationConnectionListener; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.resourcemanager.ResourceManagerId; import org.apache.flink.runtime.resourcemanager.TaskExecutorRegistration; +import org.apache.flink.runtime.rest.messages.taskmanager.LogInfo; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcService; @@ -122,6 +123,7 @@ import org.apache.flink.types.SerializableOptional; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.SerializedValue; +import org.apache.flink.util.StringUtils; import org.apache.flink.shaded.guava18.com.google.common.collect.Sets; @@ -133,6 +135,7 @@ import java.io.FileInputStream; import java.io.IOException; import java.net.InetSocketAddress; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -309,6 +312,25 @@ public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway { return CompletableFuture.completedFuture(shuffleEnvironment.getPartitionsOccupyingLocalResources().isEmpty()); } + @Override + public CompletableFuture> requestLogList(Time timeout) { + final String logDir = taskManagerConfiguration.getTaskManagerLogDir(); + if (logDir != null) { + final File[] logFiles = new File(logDir).listFiles(); + + if (logFiles == null) { + return FutureUtils.completedExceptionally(new FlinkException(String.format("There isn't a log file in TaskExecutor’s log dir %s.", logDir))); + } + + final List logsWithLength = Arrays.stream(logFiles) + .filter(File::isFile) + .map(logFile -> new LogInfo(logFile.getName(), logFile.length())) + .collect(Collectors.toList()); + return CompletableFuture.completedFuture(logsWithLength); + } + return CompletableFuture.completedFuture(Collections.emptyList()); + } + // ------------------------------------------------------------------------ // Life cycle // ------------------------------------------------------------------------ @@ -900,11 +922,8 @@ public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway { } @Override - public CompletableFuture requestFileUpload(FileType fileType, Time timeout) { - log.debug("Request file {} upload.", fileType); - + public CompletableFuture requestFileUploadByType(FileType fileType, Time timeout) { final String filePath; - switch (fileType) { case LOG: filePath = taskManagerConfiguration.getTaskManagerLogPath(); @@ -915,29 +934,19 @@ public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway { default: filePath = null; } + return requestFileUploadByFilePath(filePath, fileType.toString()); + } - if (filePath != null && !filePath.isEmpty()) { - final File file = new File(filePath); - - if (file.exists()) { - final TransientBlobCache transientBlobService = blobCacheService.getTransientBlobService(); - final TransientBlobKey transientBlobKey; - try (FileInputStream fileInputStream = new FileInputStream(file)) { - transientBlobKey = transientBlobService.putTransient(fileInputStream); - } catch (IOException e) { - log.debug("Could not upload file {}.", fileType, e); - return FutureUtils.completedExceptionally(new FlinkException("Could not upload file " + fileType + '.', e)); - } - - return CompletableFuture.completedFuture(transientBlobKey); - } else { - log.debug("The file {} does not exist on the TaskExecutor {}.", fileType, getResourceID()); - return FutureUtils.completedExceptionally(new FlinkException("The file " + fileType + " does not exist on the TaskExecutor.")); - } + @Override + public CompletableFuture requestFileUploadByName(String fileName, Time timeout) { + final String filePath; + final String logDir = taskManagerConfiguration.getTaskManagerLogDir(); + if (StringUtils.isNullOrWhitespaceOnly(logDir) || StringUtils.isNullOrWhitespaceOnly(fileName)) { + filePath = null; } else { - log.debug("The file {} is unavailable on the TaskExecutor {}.", fileType, getResourceID()); - return FutureUtils.completedExceptionally(new FlinkException("The file " + fileType + " is not available on the TaskExecutor.")); + filePath = new File(logDir, new File(fileName).getName()).getPath(); } + return requestFileUploadByFilePath(filePath, fileName); } @Override @@ -1647,6 +1656,31 @@ public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway { return jmConnection != null && Objects.equals(jmConnection.getJobMasterId(), jobMasterId); } + private CompletableFuture requestFileUploadByFilePath(String filePath, String fileTag) { + log.debug("Received file upload request for file {}", fileTag); + if (!StringUtils.isNullOrWhitespaceOnly(filePath)) { + final File file = new File(filePath); + if (file.exists()) { + final TransientBlobCache transientBlobService = blobCacheService.getTransientBlobService(); + final TransientBlobKey transientBlobKey; + try (FileInputStream fileInputStream = new FileInputStream(file)) { + transientBlobKey = transientBlobService.putTransient(fileInputStream); + } catch (IOException e) { + log.debug("Could not upload file {}.", fileTag, e); + return FutureUtils.completedExceptionally(new FlinkException("Could not upload file " + fileTag + '.', e)); + } + + return CompletableFuture.completedFuture(transientBlobKey); + } else { + log.debug("The file {} does not exist on the TaskExecutor {}.", fileTag, getResourceID()); + return FutureUtils.completedExceptionally(new FlinkException("The file " + fileTag + " does not exist on the TaskExecutor.")); + } + } else { + log.debug("The file {} is unavailable on the TaskExecutor {}.", fileTag, getResourceID()); + return FutureUtils.completedExceptionally(new FlinkException("The file " + fileTag + " is not available on the TaskExecutor.")); + } + } + // ------------------------------------------------------------------------ // Properties // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java index d61766a2fe..a6ecc60f25 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java @@ -38,12 +38,14 @@ import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.messages.TaskBackPressureResponse; import org.apache.flink.runtime.operators.coordination.OperatorEvent; import org.apache.flink.runtime.resourcemanager.ResourceManagerId; +import org.apache.flink.runtime.rest.messages.taskmanager.LogInfo; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcTimeout; import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.types.SerializableOptional; import org.apache.flink.util.SerializedValue; +import java.util.Collection; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -200,7 +202,16 @@ public interface TaskExecutorGateway extends RpcGateway, TaskExecutorOperatorEve * @param timeout for the asynchronous operation * @return Future which is completed with the {@link TransientBlobKey} of the uploaded file. */ - CompletableFuture requestFileUpload(FileType fileType, @RpcTimeout Time timeout); + CompletableFuture requestFileUploadByType(FileType fileType, @RpcTimeout Time timeout); + + /** + * Requests the file upload of the specified name to the cluster's {@link BlobServer}. + * + * @param fileName to upload + * @param timeout for the asynchronous operation + * @return Future which is completed with the {@link TransientBlobKey} of the uploaded file. + */ + CompletableFuture requestFileUploadByName(String fileName, @RpcTimeout Time timeout); /** * Returns the gateway of Metric Query Service on the TaskManager. @@ -216,6 +227,13 @@ public interface TaskExecutorGateway extends RpcGateway, TaskExecutorOperatorEve */ CompletableFuture canBeReleased(); + /** + * Requests for the historical log file names on the TaskManager. + * + * @return A Tuple2 Array with all log file names with its length. + */ + CompletableFuture> requestLogList(@RpcTimeout Time timeout); + @Override CompletableFuture sendOperatorEventToTask( ExecutionAttemptID task, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java index 8a351cd840..48ec22c0d1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java @@ -38,6 +38,7 @@ import org.slf4j.LoggerFactory; import javax.annotation.Nullable; +import java.io.File; import java.time.Duration; /** @@ -79,6 +80,9 @@ public class TaskManagerConfiguration implements TaskManagerRuntimeInfo { @Nullable private final String taskManagerStdoutPath; + @Nullable + private final String taskManagerLogDir; + private final RetryingRegistrationConfiguration retryingRegistrationConfiguration; public TaskManagerConfiguration( @@ -97,6 +101,7 @@ public class TaskManagerConfiguration implements TaskManagerRuntimeInfo { String[] alwaysParentFirstLoaderPatterns, @Nullable String taskManagerLogPath, @Nullable String taskManagerStdoutPath, + @Nullable String taskManagerLogDir, RetryingRegistrationConfiguration retryingRegistrationConfiguration) { this.numberSlots = numberSlots; @@ -114,6 +119,7 @@ public class TaskManagerConfiguration implements TaskManagerRuntimeInfo { this.alwaysParentFirstLoaderPatterns = alwaysParentFirstLoaderPatterns; this.taskManagerLogPath = taskManagerLogPath; this.taskManagerStdoutPath = taskManagerStdoutPath; + this.taskManagerLogDir = taskManagerLogDir; this.retryingRegistrationConfiguration = retryingRegistrationConfiguration; } @@ -184,6 +190,11 @@ public class TaskManagerConfiguration implements TaskManagerRuntimeInfo { return taskManagerStdoutPath; } + @Nullable + public String getTaskManagerLogDir() { + return taskManagerLogDir; + } + public RetryingRegistrationConfiguration getRetryingRegistrationConfiguration() { return retryingRegistrationConfiguration; } @@ -260,9 +271,11 @@ public class TaskManagerConfiguration implements TaskManagerRuntimeInfo { final String taskManagerLogPath = configuration.getString(ConfigConstants.TASK_MANAGER_LOG_PATH_KEY, System.getProperty("log.file")); final String taskManagerStdoutPath; + final String taskManagerLogDir; if (taskManagerLogPath != null) { final int extension = taskManagerLogPath.lastIndexOf('.'); + taskManagerLogDir = new File(taskManagerLogPath).getParent(); if (extension > 0) { taskManagerStdoutPath = taskManagerLogPath.substring(0, extension) + ".out"; @@ -271,6 +284,7 @@ public class TaskManagerConfiguration implements TaskManagerRuntimeInfo { } } else { taskManagerStdoutPath = null; + taskManagerLogDir = null; } final RetryingRegistrationConfiguration retryingRegistrationConfiguration = RetryingRegistrationConfiguration.fromConfiguration(configuration); @@ -291,6 +305,7 @@ public class TaskManagerConfiguration implements TaskManagerRuntimeInfo { alwaysParentFirstLoaderPatterns, taskManagerLogPath, taskManagerStdoutPath, + taskManagerLogDir, retryingRegistrationConfiguration); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java index 54de58242c..1ec965f0d8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java @@ -78,8 +78,10 @@ import org.apache.flink.runtime.rest.handler.legacy.files.StaticFileServerHandle import org.apache.flink.runtime.rest.handler.legacy.files.StdoutFileHandlerSpecification; import org.apache.flink.runtime.rest.handler.legacy.files.WebContentHandlerSpecification; import org.apache.flink.runtime.rest.handler.legacy.metrics.MetricFetcher; +import org.apache.flink.runtime.rest.handler.taskmanager.TaskManagerCustomLogHandler; import org.apache.flink.runtime.rest.handler.taskmanager.TaskManagerDetailsHandler; import org.apache.flink.runtime.rest.handler.taskmanager.TaskManagerLogFileHandler; +import org.apache.flink.runtime.rest.handler.taskmanager.TaskManagerLogListHandler; import org.apache.flink.runtime.rest.handler.taskmanager.TaskManagerStdoutFileHandler; import org.apache.flink.runtime.rest.handler.taskmanager.TaskManagersHandler; import org.apache.flink.runtime.rest.messages.ClusterConfigurationInfoHeaders; @@ -110,8 +112,10 @@ import org.apache.flink.runtime.rest.messages.job.JobDetailsHeaders; import org.apache.flink.runtime.rest.messages.job.SubtaskCurrentAttemptDetailsHeaders; import org.apache.flink.runtime.rest.messages.job.SubtaskExecutionAttemptAccumulatorsHeaders; import org.apache.flink.runtime.rest.messages.job.SubtaskExecutionAttemptDetailsHeaders; +import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerCustomLogHeaders; import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerDetailsHeaders; import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerLogFileHeaders; +import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerLogsHeaders; import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerStdoutFileHeaders; import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagersHeaders; import org.apache.flink.runtime.rpc.FatalErrorHandler; @@ -641,8 +645,27 @@ public class WebMonitorEndpoint extends RestServerEndp transientBlobService, cacheEntryDuration); + final TaskManagerCustomLogHandler taskManagerCustomLogHandler = new TaskManagerCustomLogHandler( + leaderRetriever, + timeout, + responseHeaders, + TaskManagerCustomLogHeaders.getInstance(), + resourceManagerRetriever, + transientBlobService, + cacheEntryDuration); + + final TaskManagerLogListHandler taskManagerLogListHandler = new TaskManagerLogListHandler( + leaderRetriever, + timeout, + responseHeaders, + TaskManagerLogsHeaders.getInstance(), + resourceManagerRetriever + ); + handlers.add(Tuple2.of(TaskManagerLogFileHeaders.getInstance(), taskManagerLogFileHandler)); handlers.add(Tuple2.of(TaskManagerStdoutFileHeaders.getInstance(), taskManagerStdoutFileHandler)); + handlers.add(Tuple2.of(TaskManagerCustomLogHeaders.getInstance(), taskManagerCustomLogHandler)); + handlers.add(Tuple2.of(TaskManagerLogsHeaders.getInstance(), taskManagerLogListHandler)); handlers.stream() .map(tuple -> tuple.f1) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/utils/TestingResourceManagerGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/utils/TestingResourceManagerGateway.java index 4d6be12717..6a2861af7e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/utils/TestingResourceManagerGateway.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/utils/TestingResourceManagerGateway.java @@ -40,6 +40,8 @@ import org.apache.flink.runtime.resourcemanager.ResourceManagerId; import org.apache.flink.runtime.resourcemanager.ResourceOverview; import org.apache.flink.runtime.resourcemanager.SlotRequest; import org.apache.flink.runtime.resourcemanager.TaskExecutorRegistration; +import org.apache.flink.runtime.resourcemanager.exceptions.UnknownTaskExecutorException; +import org.apache.flink.runtime.rest.messages.taskmanager.LogInfo; import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerInfo; import org.apache.flink.runtime.taskexecutor.FileType; import org.apache.flink.runtime.taskexecutor.SlotReport; @@ -81,7 +83,9 @@ public class TestingResourceManagerGateway implements ResourceManagerGateway { private volatile Function> registerTaskExecutorFunction; - private volatile Function, CompletableFuture> requestTaskManagerFileUploadFunction; + private volatile Function, CompletableFuture> requestTaskManagerFileUploadByTypeFunction; + + private volatile Function, CompletableFuture> requestTaskManagerFileUploadByNameFunction; private volatile Consumer> disconnectTaskExecutorConsumer; @@ -91,6 +95,8 @@ public class TestingResourceManagerGateway implements ResourceManagerGateway { private volatile Consumer> notifySlotAvailableConsumer; + private volatile Function>> requestTaskManagerLogListFunction; + public TestingResourceManagerGateway() { this( ResourceManagerId.generate(), @@ -141,8 +147,16 @@ public class TestingResourceManagerGateway implements ResourceManagerGateway { this.registerTaskExecutorFunction = registerTaskExecutorFunction; } - public void setRequestTaskManagerFileUploadFunction(Function, CompletableFuture> requestTaskManagerFileUploadFunction) { - this.requestTaskManagerFileUploadFunction = requestTaskManagerFileUploadFunction; + public void setRequestTaskManagerFileUploadByTypeFunction(Function, CompletableFuture> requestTaskManagerFileUploadByTypeFunction) { + this.requestTaskManagerFileUploadByTypeFunction = requestTaskManagerFileUploadByTypeFunction; + } + + public void setRequestTaskManagerFileUploadByNameFunction(Function, CompletableFuture> requestTaskManagerFileUploadByNameFunction) { + this.requestTaskManagerFileUploadByNameFunction = requestTaskManagerFileUploadByNameFunction; + } + + public void setRequestTaskManagerLogListFunction(Function>> requestTaskManagerLogListFunction) { + this.requestTaskManagerLogListFunction = requestTaskManagerLogListFunction; } public void setDisconnectTaskExecutorConsumer(Consumer> disconnectTaskExecutorConsumer) { @@ -299,8 +313,8 @@ public class TestingResourceManagerGateway implements ResourceManagerGateway { } @Override - public CompletableFuture requestTaskManagerFileUpload(ResourceID taskManagerId, FileType fileType, Time timeout) { - final Function, CompletableFuture> function = requestTaskManagerFileUploadFunction; + public CompletableFuture requestTaskManagerFileUploadByType(ResourceID taskManagerId, FileType fileType, Time timeout) { + final Function, CompletableFuture> function = requestTaskManagerFileUploadByTypeFunction; if (function != null) { return function.apply(Tuple2.of(taskManagerId, fileType)); @@ -309,6 +323,27 @@ public class TestingResourceManagerGateway implements ResourceManagerGateway { } } + @Override + public CompletableFuture requestTaskManagerFileUploadByName(ResourceID taskManagerId, String fileName, Time timeout) { + final Function, CompletableFuture> function = requestTaskManagerFileUploadByNameFunction; + + if (function != null) { + return function.apply(Tuple2.of(taskManagerId, fileName)); + } else { + return CompletableFuture.completedFuture(new TransientBlobKey()); + } + } + + @Override + public CompletableFuture> requestTaskManagerLogList(ResourceID taskManagerId, Time timeout) { + final Function>> function = this.requestTaskManagerLogListFunction; + if (function != null) { + return function.apply(taskManagerId); + } else { + return FutureUtils.completedExceptionally(new UnknownTaskExecutorException(taskManagerId)); + } + } + @Override public ResourceManagerId getFencingToken() { return resourceManagerId; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandlerTest.java index d7ea3e5c55..1cb4b2e7fc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandlerTest.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.rest.handler.taskmanager; import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobServer; @@ -34,6 +35,7 @@ import org.apache.flink.runtime.rest.handler.HandlerRequest; import org.apache.flink.runtime.rest.handler.HandlerRequestException; import org.apache.flink.runtime.rest.messages.EmptyRequestBody; import org.apache.flink.runtime.rest.messages.UntypedResponseMessageHeaders; +import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerFileMessageParameters; import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerIdPathParameter; import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerMessageParameters; import org.apache.flink.runtime.testingUtils.TestingUtils; @@ -123,7 +125,7 @@ public class AbstractTaskManagerFileHandlerTest extends TestLogger { handlerRequest = new HandlerRequest<>( EmptyRequestBody.getInstance(), - new TaskManagerMessageParameters(), + new TaskManagerFileMessageParameters(), Collections.singletonMap(TaskManagerIdPathParameter.KEY, EXPECTED_TASK_MANAGER_ID.getResourceIdString()), Collections.emptyMap()); } @@ -283,8 +285,8 @@ public class AbstractTaskManagerFileHandlerTest extends TestLogger { } @Override - protected CompletableFuture requestFileUpload(ResourceManagerGateway resourceManagerGateway, ResourceID taskManagerResourceId) { - assertThat(taskManagerResourceId, is(equalTo(expectedTaskManagerId))); + protected CompletableFuture requestFileUpload(ResourceManagerGateway resourceManagerGateway, Tuple2 taskManagerIdAndFileName) { + assertThat(taskManagerIdAndFileName.f0, is(equalTo(expectedTaskManagerId))); final CompletableFuture transientBlobKeyFuture = requestFileUploads.poll(); if (transientBlobKeyFuture != null) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerLogListHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerLogListHandlerTest.java new file mode 100644 index 0000000000..272a07e3cc --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerLogListHandlerTest.java @@ -0,0 +1,116 @@ +/* + * 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.runtime.rest.handler.taskmanager; + +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.resourcemanager.exceptions.UnknownTaskExecutorException; +import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway; +import org.apache.flink.runtime.rest.handler.HandlerRequest; +import org.apache.flink.runtime.rest.handler.HandlerRequestException; +import org.apache.flink.runtime.rest.handler.RestHandlerException; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.taskmanager.LogInfo; +import org.apache.flink.runtime.rest.messages.taskmanager.LogListInfo; +import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerIdPathParameter; +import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerLogsHeaders; +import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerMessageParameters; +import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.util.TestLogger; + +import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; + +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; + +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; + +/** + * Test for the {@link TaskManagerLogListHandler}. + */ +public class TaskManagerLogListHandlerTest extends TestLogger { + + private static final ResourceID EXPECTED_TASK_MANAGER_ID = ResourceID.generate(); + private TestingResourceManagerGateway resourceManagerGateway; + private TaskManagerLogListHandler taskManagerLogListHandler; + private HandlerRequest handlerRequest; + + @Before + public void setUp() throws HandlerRequestException { + resourceManagerGateway = new TestingResourceManagerGateway(); + taskManagerLogListHandler = new TaskManagerLogListHandler( + () -> CompletableFuture.completedFuture(null), + TestingUtils.TIMEOUT(), + Collections.emptyMap(), + TaskManagerLogsHeaders.getInstance(), + () -> CompletableFuture.completedFuture(resourceManagerGateway)); + handlerRequest = createRequest(EXPECTED_TASK_MANAGER_ID); + } + + @Test + public void testGetTaskManagerLogsList() throws Exception { + List logsList = Arrays.asList( + new LogInfo("taskmanager.log", 1024L), + new LogInfo("taskmanager.out", 1024L), + new LogInfo("taskmanager-2.out", 1024L)); + resourceManagerGateway.setRequestTaskManagerLogListFunction(EXPECTED_TASK_MANAGER_ID -> CompletableFuture.completedFuture(logsList)); + LogListInfo logListInfo = taskManagerLogListHandler.handleRequest(handlerRequest, resourceManagerGateway).get(); + assertThat(logListInfo.getLogInfos(), hasSize(logsList.size())); + } + + @Test + public void testGetTaskManagerLogsListForUnknownTaskExecutorException() throws Exception { + resourceManagerGateway.setRequestTaskManagerLogListFunction(EXPECTED_TASK_MANAGER_ID -> FutureUtils.completedExceptionally(new UnknownTaskExecutorException(EXPECTED_TASK_MANAGER_ID))); + try { + taskManagerLogListHandler.handleRequest(handlerRequest, resourceManagerGateway).get(); + } catch (ExecutionException e) { + final Throwable cause = e.getCause(); + assertThat(cause, is(instanceOf(RestHandlerException.class))); + + final RestHandlerException restHandlerException = (RestHandlerException) cause; + assertThat(restHandlerException.getHttpResponseStatus(), is(equalTo(HttpResponseStatus.NOT_FOUND))); + assertThat(restHandlerException.getMessage(), containsString("Could not find TaskExecutor " + EXPECTED_TASK_MANAGER_ID)); + } + } + + private static HandlerRequest createRequest(ResourceID taskManagerId) throws HandlerRequestException { + Map pathParameters = new HashMap<>(); + pathParameters.put(TaskManagerIdPathParameter.KEY, taskManagerId.toString()); + Map> queryParameters = Collections.emptyMap(); + + return new HandlerRequest<>( + EmptyRequestBody.getInstance(), + new TaskManagerMessageParameters(), + pathParameters, + queryParameters); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java index e4b61f9c81..87ba525346 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java @@ -1620,7 +1620,7 @@ public class TaskExecutorTest extends TestLogger { TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); try { CompletableFuture logFuture = - tmGateway.requestFileUpload(FileType.LOG, timeout); + tmGateway.requestFileUploadByType(FileType.LOG, timeout); logFuture.get(); } catch (Exception e) { assertThat(e.getMessage(), containsString("The file LOG does not exist on the TaskExecutor.")); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGateway.java index 1c447d3ca6..8a025aa3a8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGateway.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGateway.java @@ -39,6 +39,7 @@ import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.messages.TaskBackPressureResponse; import org.apache.flink.runtime.operators.coordination.OperatorEvent; import org.apache.flink.runtime.resourcemanager.ResourceManagerId; +import org.apache.flink.runtime.rest.messages.taskmanager.LogInfo; import org.apache.flink.runtime.rpc.RpcTimeout; import org.apache.flink.types.SerializableOptional; import org.apache.flink.util.Preconditions; @@ -46,6 +47,7 @@ import org.apache.flink.util.SerializedValue; import org.apache.flink.util.function.TriConsumer; import org.apache.flink.util.function.TriFunction; +import java.util.Collection; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.function.BiConsumer; @@ -181,7 +183,12 @@ public class TestingTaskExecutorGateway implements TaskExecutorGateway { } @Override - public CompletableFuture requestFileUpload(FileType fileType, Time timeout) { + public CompletableFuture requestFileUploadByType(FileType fileType, Time timeout) { + return FutureUtils.completedExceptionally(new UnsupportedOperationException()); + } + + @Override + public CompletableFuture requestFileUploadByName(String fileName, Time timeout) { return FutureUtils.completedExceptionally(new UnsupportedOperationException()); } @@ -212,4 +219,9 @@ public class TestingTaskExecutorGateway implements TaskExecutorGateway { public String getHostname() { return hostname; } + + @Override + public CompletableFuture> requestLogList(Time timeout) { + return FutureUtils.completedExceptionally(new UnsupportedOperationException()); + } } -- Gitee From 505a808dd42629a61dcb047f407a59a8f609c0dc Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Sun, 22 Mar 2020 13:38:58 +0100 Subject: [PATCH 243/885] [FLINK-16302][rest] Add tests for (un)marshalling LogListInfo --- .../messages/taskmanager/LogListInfoTest.java | 43 +++++++++++++++++++ 1 file changed, 43 insertions(+) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/taskmanager/LogListInfoTest.java diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/taskmanager/LogListInfoTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/taskmanager/LogListInfoTest.java new file mode 100644 index 0000000000..0a8c8028a7 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/taskmanager/LogListInfoTest.java @@ -0,0 +1,43 @@ +/* + * 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.runtime.rest.messages.taskmanager; + +import org.apache.flink.runtime.rest.messages.ResponseBody; +import org.apache.flink.runtime.rest.messages.RestResponseMarshallingTestBase; + +import java.util.Arrays; + +/** + * Tests for (un)marshalling of {@link LogListInfo}. + */ +public class LogListInfoTest extends RestResponseMarshallingTestBase { + + @Override + protected Class getTestResponseClass() { + return LogListInfo.class; + } + + @Override + protected ResponseBody getTestResponseInstance() throws Exception { + return new LogListInfo(Arrays.asList( + new LogInfo("taskmanager.log", 0), + new LogInfo("taskmanager.out", Integer.MAX_VALUE))); + } +} -- Gitee From 874310f68acd418a0a1390f9bd667e506e0efb90 Mon Sep 17 00:00:00 2001 From: ifndef-SleePy Date: Mon, 24 Feb 2020 23:49:57 +0800 Subject: [PATCH 244/885] [FLINK-14971][checkpointing] Handle ack/declined message of checkpoint in timer thread --- .../checkpoint/CheckpointCoordinator.java | 77 +++---- .../runtime/checkpoint/PendingCheckpoint.java | 82 +++++-- .../CheckpointCoordinatorFailureTest.java | 11 +- .../CheckpointCoordinatorMasterHooksTest.java | 2 + .../CheckpointCoordinatorRestoringTest.java | 10 + .../checkpoint/CheckpointCoordinatorTest.java | 28 ++- .../CheckpointStateRestoreTest.java | 2 + .../checkpoint/PendingCheckpointTest.java | 212 ++++++++++++++++-- 8 files changed, 333 insertions(+), 91 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java index 4e23df3bbc..a3692b4f0f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java @@ -656,7 +656,9 @@ public class CheckpointCoordinator { props, checkpointStorageLocation, executor, - onCompletionPromise); + timer, + onCompletionPromise, + completedCheckpointStore); if (statsTracker != null) { PendingCheckpointStats callback = statsTracker.reportPendingCheckpoint( @@ -1018,61 +1020,44 @@ public class CheckpointCoordinator { *

    Important: This method should only be called in the checkpoint lock scope. * * @param pendingCheckpoint to complete - * @throws CheckpointException if the completion failed */ - private void completePendingCheckpoint(PendingCheckpoint pendingCheckpoint) throws CheckpointException { - final long checkpointId = pendingCheckpoint.getCheckpointId(); - final CompletedCheckpoint completedCheckpoint; - + private void completePendingCheckpoint(PendingCheckpoint pendingCheckpoint) { // As a first step to complete the checkpoint, we register its state with the registry Map operatorStates = pendingCheckpoint.getOperatorStates(); sharedStateRegistry.registerAll(operatorStates.values()); - try { - try { - completedCheckpoint = pendingCheckpoint.finalizeCheckpoint(); - failureManager.handleCheckpointSuccess(pendingCheckpoint.getCheckpointId()); - } - catch (Exception e1) { - // abort the current pending checkpoint if we fails to finalize the pending checkpoint. - if (!pendingCheckpoint.isDiscarded()) { - abortPendingCheckpoint( - pendingCheckpoint, - new CheckpointException( - CheckpointFailureReason.FINALIZE_CHECKPOINT_FAILURE, e1)); - } - - throw new CheckpointException("Could not finalize the pending checkpoint " + checkpointId + '.', - CheckpointFailureReason.FINALIZE_CHECKPOINT_FAILURE, e1); - } - - // the pending checkpoint must be discarded after the finalization - Preconditions.checkState(pendingCheckpoint.isDiscarded() && completedCheckpoint != null); - - try { - completedCheckpointStore.addCheckpoint(completedCheckpoint); - } catch (Exception exception) { - // we failed to store the completed checkpoint. Let's clean up - executor.execute(new Runnable() { - @Override - public void run() { - try { - completedCheckpoint.discardOnFailedStoring(); - } catch (Throwable t) { - LOG.warn("Could not properly discard completed checkpoint {}.", completedCheckpoint.getCheckpointID(), t); + pendingCheckpoint.finalizeCheckpoint() + .whenCompleteAsync((completedCheckpoint, throwable) -> { + synchronized (lock) { + if (shutdown) { + return; + } + if (throwable != null) { + if (!pendingCheckpoint.isDiscarded()) { + abortPendingCheckpoint( + pendingCheckpoint, + new CheckpointException( + CheckpointFailureReason.FINALIZE_CHECKPOINT_FAILURE, throwable)); } + } else { + onCheckpointSuccess(completedCheckpoint); } - }); + } + }, timer); + } - throw new CheckpointException("Could not complete the pending checkpoint " + checkpointId + '.', - CheckpointFailureReason.FINALIZE_CHECKPOINT_FAILURE, exception); - } - } finally { - pendingCheckpoints.remove(checkpointId); + /** + * It's the last step of a checkpoint. + * + * @param completedCheckpoint the completed checkpoint + */ + private void onCheckpointSuccess(CompletedCheckpoint completedCheckpoint) { + final long checkpointId = completedCheckpoint.getCheckpointID(); - resumePeriodicTriggering(); - } + pendingCheckpoints.remove(checkpointId); + resumePeriodicTriggering(); + failureManager.handleCheckpointSuccess(checkpointId); rememberRecentCheckpointId(checkpointId); // drop those pending checkpoints that are at prior to the completed one diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java index 27a8513b3d..27395ea5be 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java @@ -20,6 +20,7 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.checkpoint.metadata.CheckpointMetadata; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.ExecutionVertex; import org.apache.flink.runtime.jobgraph.OperatorID; @@ -28,7 +29,6 @@ import org.apache.flink.runtime.state.CheckpointStorageLocation; import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation; import org.apache.flink.runtime.state.StateUtil; import org.apache.flink.runtime.state.StreamStateHandle; -import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.Preconditions; import org.slf4j.Logger; @@ -36,7 +36,6 @@ import org.slf4j.LoggerFactory; import javax.annotation.Nullable; -import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -46,6 +45,7 @@ import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; import java.util.concurrent.Executor; import java.util.concurrent.ScheduledFuture; @@ -111,6 +111,11 @@ public class PendingCheckpoint { /** The executor for potentially blocking I/O operations, like state disposal. */ private final Executor executor; + /** The executor for non-blocking operations. */ + private final Executor mainThreadExecutor; + + private final CompletedCheckpointStore completedCheckpointStore; + private int numAcknowledgedTasks; private boolean discarded; @@ -135,7 +140,9 @@ public class PendingCheckpoint { CheckpointProperties props, CheckpointStorageLocation targetLocation, Executor executor, - CompletableFuture onCompletionPromise) { + Executor mainThreadExecutor, + CompletableFuture onCompletionPromise, + CompletedCheckpointStore completedCheckpointStore) { checkArgument(verticesToConfirm.size() > 0, "Checkpoint needs at least one vertex that commits the checkpoint"); @@ -147,6 +154,8 @@ public class PendingCheckpoint { this.props = checkNotNull(props); this.targetLocation = checkNotNull(targetLocation); this.executor = Preconditions.checkNotNull(executor); + this.mainThreadExecutor = Preconditions.checkNotNull(mainThreadExecutor); + this.completedCheckpointStore = Preconditions.checkNotNull(completedCheckpointStore); this.operatorStates = new HashMap<>(); this.masterStates = new ArrayList<>(masterStateIdentifiers.size()); @@ -289,24 +298,37 @@ public class PendingCheckpoint { return onCompletionPromise; } - public CompletedCheckpoint finalizeCheckpoint() throws IOException { + public CompletableFuture finalizeCheckpoint() { synchronized (lock) { - checkState(!isDiscarded(), "checkpoint is discarded"); - checkState(isFullyAcknowledged(), "Pending checkpoint has not been fully acknowledged yet"); + if (isDiscarded()) { + return FutureUtils.completedExceptionally(new IllegalStateException( + "checkpoint is discarded")); + } + if (!isFullyAcknowledged()) { + return FutureUtils.completedExceptionally(new IllegalStateException( + "Pending checkpoint has not been fully acknowledged yet")); + } - // make sure we fulfill the promise with an exception if something fails - try { - // write out the metadata - final CheckpointMetadata savepoint = new CheckpointMetadata(checkpointId, operatorStates.values(), masterStates); - final CompletedCheckpointStorageLocation finalizedLocation; + // now we stop the canceller before finalization + // it simplifies the concurrent conflict issue here + cancelCanceller(); - try (CheckpointMetadataOutputStream out = targetLocation.createMetadataOutputStream()) { - Checkpoints.storeCheckpointMetadata(savepoint, out); - finalizedLocation = out.closeAndFinalizeCheckpoint(); - } + // make sure we fulfill the promise with an exception if something fails + final CompletableFuture finalizingFuture = + CompletableFuture.supplyAsync(() -> { + try { + checkState(!isDiscarded(), "The checkpoint has been discarded"); + // write out the metadata + final CheckpointMetadata savepoint = new CheckpointMetadata(checkpointId, operatorStates.values(), masterStates); + final CompletedCheckpointStorageLocation finalizedLocation; + + try (CheckpointMetadataOutputStream out = targetLocation.createMetadataOutputStream()) { + Checkpoints.storeCheckpointMetadata(savepoint, out); + finalizedLocation = out.closeAndFinalizeCheckpoint(); + } - CompletedCheckpoint completed = new CompletedCheckpoint( + CompletedCheckpoint completed = new CompletedCheckpoint( jobId, checkpointId, checkpointTimestamp, @@ -316,6 +338,25 @@ public class PendingCheckpoint { props, finalizedLocation); + try { + completedCheckpointStore.addCheckpoint(completed); + } catch (Throwable t) { + completed.discardOnFailedStoring(); + } + return completed; + } catch (Throwable t) { + LOG.warn("Could not finalize checkpoint {}.", checkpointId, t); + onCompletionPromise.completeExceptionally(t); + throw new CompletionException(t); + } + }, executor); + + return finalizingFuture.thenApplyAsync((completed) -> { + + // since canceller has been already cancelled, discarding means the coordinator must be shut down + // all the resources should be released properly when it's shutting down the coordinator + checkState(!isDiscarded(), "The checkpoint has been discarded"); + onCompletionPromise.complete(completed); // to prevent null-pointers from concurrent modification, copy reference onto stack @@ -324,7 +365,7 @@ public class PendingCheckpoint { // Finalize the statsCallback and give the completed checkpoint a // callback for discards. CompletedCheckpointStats.DiscardCallback discardCallback = - statsCallback.reportCompletedCheckpoint(finalizedLocation.getExternalPointer()); + statsCallback.reportCompletedCheckpoint(completed.getExternalPointer()); completed.setDiscardCallback(discardCallback); } @@ -332,12 +373,7 @@ public class PendingCheckpoint { dispose(false); return completed; - } - catch (Throwable t) { - onCompletionPromise.completeExceptionally(t); - ExceptionUtils.rethrowIOException(t); - return null; // silence the compiler - } + }, mainThreadExecutor); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java index e3828224c0..e6e907260e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java @@ -38,7 +38,6 @@ import java.util.List; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; @@ -104,13 +103,9 @@ public class CheckpointCoordinatorFailureTest extends TestLogger { AcknowledgeCheckpoint acknowledgeMessage = new AcknowledgeCheckpoint(jid, executionAttemptId, checkpointId, new CheckpointMetrics(), subtaskState); - try { - coord.receiveAcknowledgeMessage(acknowledgeMessage, "Unknown location"); - fail("Expected a checkpoint exception because the completed checkpoint store could not " + - "store the completed checkpoint."); - } catch (CheckpointException e) { - // ignore because we expected this exception - } + coord.receiveAcknowledgeMessage(acknowledgeMessage, "Unknown location"); + // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually + manuallyTriggeredScheduledExecutor.triggerAll(); // make sure that the pending checkpoint has been discarded after we could not complete it assertTrue(pendingCheckpoint.isDiscarded()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java index cff25719cb..a756e52ee3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java @@ -210,6 +210,8 @@ public class CheckpointCoordinatorMasterHooksTest { final long checkpointId = cc.getPendingCheckpoints().values().iterator().next().getCheckpointId(); cc.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, execId, checkpointId), "Unknown location"); + // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually + manuallyTriggeredScheduledExecutor.triggerAll(); assertEquals(0, cc.getNumberOfPendingCheckpoints()); assertEquals(1, cc.getNumberOfRetainedSuccessfulCheckpoints()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java index e098695bae..49ef8e34b4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java @@ -184,6 +184,8 @@ public class CheckpointCoordinatorRestoringTest extends TestLogger { subtaskState); coord.receiveAcknowledgeMessage(acknowledgeCheckpoint, TASK_MANAGER_LOCATION_INFO); + // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually + manuallyTriggeredScheduledExecutor.triggerAll(); } List completedCheckpoints = coord.getSuccessfulCheckpoints(); @@ -299,6 +301,8 @@ public class CheckpointCoordinatorRestoringTest extends TestLogger { coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statefulExec1.getAttemptId(), checkpointId, new CheckpointMetrics(), subtaskStatesForCheckpoint), TASK_MANAGER_LOCATION_INFO); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statelessExec1.getAttemptId(), checkpointId), TASK_MANAGER_LOCATION_INFO); + // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually + manuallyTriggeredScheduledExecutor.triggerAll(); CompletedCheckpoint success = coord.getSuccessfulCheckpoints().get(0); assertEquals(jid, success.getJobId()); @@ -326,6 +330,8 @@ public class CheckpointCoordinatorRestoringTest extends TestLogger { checkpointId = checkpointIDCounter.getLast(); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statefulExec1.getAttemptId(), checkpointId, new CheckpointMetrics(), subtaskStatesForSavepoint), TASK_MANAGER_LOCATION_INFO); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statelessExec1.getAttemptId(), checkpointId), TASK_MANAGER_LOCATION_INFO); + // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually + manuallyTriggeredScheduledExecutor.triggerAll(); assertNotNull(savepointFuture.get()); @@ -467,6 +473,8 @@ public class CheckpointCoordinatorRestoringTest extends TestLogger { taskOperatorSubtaskStates); coord.receiveAcknowledgeMessage(acknowledgeCheckpoint, TASK_MANAGER_LOCATION_INFO); + // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually + manuallyTriggeredScheduledExecutor.triggerAll(); } List completedCheckpoints = coord.getSuccessfulCheckpoints(); @@ -614,6 +622,8 @@ public class CheckpointCoordinatorRestoringTest extends TestLogger { taskOperatorSubtaskStates); coord.receiveAcknowledgeMessage(acknowledgeCheckpoint, TASK_MANAGER_LOCATION_INFO); + // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually + manuallyTriggeredScheduledExecutor.triggerAll(); } List completedCheckpoints = coord.getSuccessfulCheckpoints(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java index ce662f6a25..846c809790 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java @@ -554,6 +554,9 @@ public class CheckpointCoordinatorTest extends TestLogger { // acknowledge the other task. coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID1, checkpointId, new CheckpointMetrics(), taskOperatorSubtaskStates1), TASK_MANAGER_LOCATION_INFO); + // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually + manuallyTriggeredScheduledExecutor.triggerAll(); + // the checkpoint is internally converted to a successful checkpoint and the // pending checkpoint object is disposed assertTrue(checkpoint.isDiscarded()); @@ -593,6 +596,8 @@ public class CheckpointCoordinatorTest extends TestLogger { long checkpointIdNew = coord.getPendingCheckpoints().entrySet().iterator().next().getKey(); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID1, checkpointIdNew), TASK_MANAGER_LOCATION_INFO); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID2, checkpointIdNew), TASK_MANAGER_LOCATION_INFO); + // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually + manuallyTriggeredScheduledExecutor.triggerAll(); assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(1, coord.getNumberOfRetainedSuccessfulCheckpoints()); @@ -710,6 +715,8 @@ public class CheckpointCoordinatorTest extends TestLogger { coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID1, checkpointId2), TASK_MANAGER_LOCATION_INFO); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID1, checkpointId1), TASK_MANAGER_LOCATION_INFO); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID2, checkpointId2), TASK_MANAGER_LOCATION_INFO); + // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually + manuallyTriggeredScheduledExecutor.triggerAll(); // now, the first checkpoint should be confirmed assertEquals(1, coord.getNumberOfPendingCheckpoints()); @@ -721,6 +728,8 @@ public class CheckpointCoordinatorTest extends TestLogger { // send the last remaining ack for the second checkpoint coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID3, checkpointId2), TASK_MANAGER_LOCATION_INFO); + // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually + manuallyTriggeredScheduledExecutor.triggerAll(); // now, the second checkpoint should be confirmed assertEquals(0, coord.getNumberOfPendingCheckpoints()); @@ -873,6 +882,9 @@ public class CheckpointCoordinatorTest extends TestLogger { coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID2, checkpointId2, new CheckpointMetrics(), taskOperatorSubtaskStates22), TASK_MANAGER_LOCATION_INFO); + // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually + manuallyTriggeredScheduledExecutor.triggerAll(); + // now, the second checkpoint should be confirmed, and the first discarded // actually both pending checkpoints are discarded, and the second has been transformed // into a successful checkpoint @@ -1244,6 +1256,8 @@ public class CheckpointCoordinatorTest extends TestLogger { // acknowledge the other task. coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID1, checkpointId, new CheckpointMetrics(), taskOperatorSubtaskStates1), TASK_MANAGER_LOCATION_INFO); + // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually + manuallyTriggeredScheduledExecutor.triggerAll(); // the checkpoint is internally converted to a successful checkpoint and the // pending checkpoint object is disposed @@ -1283,6 +1297,8 @@ public class CheckpointCoordinatorTest extends TestLogger { long checkpointIdNew = coord.getPendingCheckpoints().entrySet().iterator().next().getKey(); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID1, checkpointIdNew), TASK_MANAGER_LOCATION_INFO); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID2, checkpointIdNew), TASK_MANAGER_LOCATION_INFO); + // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually + manuallyTriggeredScheduledExecutor.triggerAll(); assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(1, coord.getNumberOfRetainedSuccessfulCheckpoints()); @@ -1364,6 +1380,8 @@ public class CheckpointCoordinatorTest extends TestLogger { // 2nd checkpoint should subsume the 1st checkpoint, but not the savepoint coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID1, checkpointId2), TASK_MANAGER_LOCATION_INFO); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID2, checkpointId2), TASK_MANAGER_LOCATION_INFO); + // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually + manuallyTriggeredScheduledExecutor.triggerAll(); assertEquals(1, coord.getNumberOfPendingCheckpoints()); assertEquals(1, coord.getNumberOfRetainedSuccessfulCheckpoints()); @@ -1386,6 +1404,8 @@ public class CheckpointCoordinatorTest extends TestLogger { // 2nd savepoint should subsume the last checkpoint, but not the 1st savepoint coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID1, savepointId2), TASK_MANAGER_LOCATION_INFO); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID2, savepointId2), TASK_MANAGER_LOCATION_INFO); + // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually + manuallyTriggeredScheduledExecutor.triggerAll(); assertEquals(1, coord.getNumberOfPendingCheckpoints()); assertEquals(2, coord.getNumberOfRetainedSuccessfulCheckpoints()); @@ -1397,6 +1417,8 @@ public class CheckpointCoordinatorTest extends TestLogger { // Ack first savepoint coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID1, savepointId1), TASK_MANAGER_LOCATION_INFO); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID2, savepointId1), TASK_MANAGER_LOCATION_INFO); + // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually + manuallyTriggeredScheduledExecutor.triggerAll(); assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(3, coord.getNumberOfRetainedSuccessfulCheckpoints()); @@ -1462,11 +1484,12 @@ public class CheckpointCoordinatorTest extends TestLogger { // now, once we acknowledge one checkpoint, it should trigger the next one coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID, 1L), TASK_MANAGER_LOCATION_INFO); + // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually + manuallyTriggeredScheduledExecutor.triggerAll(); final Collection> periodicScheduledTasks = manuallyTriggeredScheduledExecutor.getPeriodicScheduledTask(); assertEquals(1, periodicScheduledTasks.size()); - final ScheduledFuture scheduledFuture = periodicScheduledTasks.iterator().next(); manuallyTriggeredScheduledExecutor.triggerPeriodicScheduledTasks(); manuallyTriggeredScheduledExecutor.triggerAll(); @@ -1663,6 +1686,7 @@ public class CheckpointCoordinatorTest extends TestLogger { coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jobId, attemptID1, checkpointId), TASK_MANAGER_LOCATION_INFO); } + manuallyTriggeredScheduledExecutor.triggerAll(); // After ACKs, all should be completed for (CompletableFuture savepointFuture : savepointFutures) { assertNotNull(savepointFuture.get()); @@ -2375,6 +2399,8 @@ public class CheckpointCoordinatorTest extends TestLogger { taskStateSnapshot); coord.receiveAcknowledgeMessage(acknowledgeCheckpoint, TASK_MANAGER_LOCATION_INFO); + // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually + manuallyTriggeredScheduledExecutor.triggerAll(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java index 958f3f15f4..4bc1df7838 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java @@ -133,6 +133,8 @@ public class CheckpointStateRestoreTest { coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statefulExec3.getAttemptId(), checkpointId, new CheckpointMetrics(), subtaskStates), TASK_MANAGER_LOCATION_INFO); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statelessExec1.getAttemptId(), checkpointId), TASK_MANAGER_LOCATION_INFO); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statelessExec2.getAttemptId(), checkpointId), TASK_MANAGER_LOCATION_INFO); + // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually + manuallyTriggeredScheduledExecutor.triggerAll(); assertEquals(1, coord.getNumberOfRetainedSuccessfulCheckpoints()); assertEquals(0, coord.getNumberOfPendingCheckpoints()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java index f172e51f12..bd2b799d0c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; import org.apache.flink.core.fs.Path; import org.apache.flink.core.fs.local.LocalFileSystem; import org.apache.flink.core.io.SimpleVersionedSerializer; @@ -26,17 +27,22 @@ import org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.Str import org.apache.flink.runtime.checkpoint.PendingCheckpoint.TaskAcknowledgeResult; import org.apache.flink.runtime.checkpoint.hooks.MasterHooks; import org.apache.flink.runtime.concurrent.Executors; +import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.executiongraph.ExecutionVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinator; +import org.apache.flink.runtime.state.CheckpointMetadataOutputStream; +import org.apache.flink.runtime.state.CheckpointStorageLocation; import org.apache.flink.runtime.state.CheckpointStorageLocationReference; +import org.apache.flink.runtime.state.CheckpointedStateScope; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.runtime.state.TestingStreamStateHandle; import org.apache.flink.runtime.state.filesystem.FsCheckpointStorageLocation; +import org.apache.flink.util.ExceptionUtils; import org.hamcrest.Matchers; import org.junit.Assert; @@ -65,6 +71,7 @@ import java.util.concurrent.ScheduledFuture; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -158,7 +165,7 @@ public class PendingCheckpointTest { future = pending.getCompletionFuture(); assertFalse(future.isDone()); - pending.abort(CheckpointFailureReason.CHECKPOINT_DECLINED); + pending.abort(CheckpointFailureReason.CHECKPOINT_EXPIRED); assertTrue(future.isDone()); // Abort subsumed @@ -166,7 +173,7 @@ public class PendingCheckpointTest { future = pending.getCompletionFuture(); assertFalse(future.isDone()); - pending.abort(CheckpointFailureReason.CHECKPOINT_DECLINED); + pending.abort(CheckpointFailureReason.CHECKPOINT_SUBSUMED); assertTrue(future.isDone()); // Finalize (all ACK'd) @@ -176,7 +183,7 @@ public class PendingCheckpointTest { assertFalse(future.isDone()); pending.acknowledgeTask(ATTEMPT_ID, null, new CheckpointMetrics()); assertTrue(pending.areTasksFullyAcknowledged()); - pending.finalizeCheckpoint(); + pending.finalizeCheckpoint().get(); assertTrue(future.isDone()); // Finalize (missing ACKs) @@ -185,10 +192,11 @@ public class PendingCheckpointTest { assertFalse(future.isDone()); try { - pending.finalizeCheckpoint(); + pending.finalizeCheckpoint().get(); fail("Did not throw expected Exception"); - } catch (IllegalStateException ignored) { + } catch (Throwable t) { // Expected + assertTrue(ExceptionUtils.findThrowable(t, IllegalStateException.class).isPresent()); } } @@ -261,7 +269,7 @@ public class PendingCheckpointTest { pending.acknowledgeTask(ATTEMPT_ID, null, new CheckpointMetrics()); verify(callback, times(1)).reportSubtaskStats(nullable(JobVertexID.class), any(SubtaskStateStats.class)); - pending.finalizeCheckpoint(); + pending.finalizeCheckpoint().get(); verify(callback, times(1)).reportCompletedCheckpoint(any(String.class)); } @@ -491,6 +499,142 @@ public class PendingCheckpointTest { assertTrue(handle2.isDisposed()); } + @Test + public void testAsyncFinalizeCheckpoint() throws Exception { + CheckpointProperties props = new CheckpointProperties(false, CheckpointType.SAVEPOINT, false, false, false, false, false); + + ManuallyTriggeredScheduledExecutor ioExecutor = new ManuallyTriggeredScheduledExecutor(); + ManuallyTriggeredScheduledExecutor mainThreadExecutor = new ManuallyTriggeredScheduledExecutor(); + + CompletedCheckpointStore completedCheckpointStore = new StandaloneCompletedCheckpointStore(10); + PendingCheckpoint pending = createPendingCheckpoint( + props, + Collections.emptyList(), + Collections.emptyList(), + ioExecutor, + mainThreadExecutor, + completedCheckpointStore); + + pending.acknowledgeTask(ATTEMPT_ID, null, new CheckpointMetrics()); + CompletableFuture checkpointCompletableFuture = pending.finalizeCheckpoint(); + + assertFalse(checkpointCompletableFuture.isDone()); + ioExecutor.triggerAll(); + + assertFalse(checkpointCompletableFuture.isDone()); + mainThreadExecutor.triggerAll(); + assertNotNull(checkpointCompletableFuture.get()); + assertEquals(1, completedCheckpointStore.getNumberOfRetainedCheckpoints()); + } + + @Test + public void testAsyncFinalizeCheckpointFailed() throws Exception { + CheckpointProperties props = new CheckpointProperties(false, CheckpointType.SAVEPOINT, false, false, false, false, false); + + ManuallyTriggeredScheduledExecutor ioExecutor = new ManuallyTriggeredScheduledExecutor(); + ManuallyTriggeredScheduledExecutor mainThreadExecutor = new ManuallyTriggeredScheduledExecutor(); + + final Map ackTasks = new HashMap<>(ACK_TASKS); + + final StandaloneCompletedCheckpointStore completedCheckpointStore = new StandaloneCompletedCheckpointStore(10); + PendingCheckpoint pending = new PendingCheckpoint( + new JobID(), + 0, + 1, + ackTasks, + Collections.emptyList(), + Collections.emptyList(), + props, + new FailingCheckpointStorageLocation(), + ioExecutor, + mainThreadExecutor, + new CompletableFuture<>(), + completedCheckpointStore); + + pending.acknowledgeTask(ATTEMPT_ID, null, new CheckpointMetrics()); + CompletableFuture checkpointCompletableFuture = pending.finalizeCheckpoint(); + + assertFalse(checkpointCompletableFuture.isDone()); + ioExecutor.triggerAll(); + + assertTrue(checkpointCompletableFuture.isCompletedExceptionally()); + assertEquals(0, completedCheckpointStore.getNumberOfRetainedCheckpoints()); + } + + @Test + public void testAbortingAfterAsyncFinalization() throws Exception { + final CheckpointProperties props = new CheckpointProperties(false, CheckpointType.SAVEPOINT, false, false, false, false, false); + + final ManuallyTriggeredScheduledExecutor ioExecutor = new ManuallyTriggeredScheduledExecutor(); + final ManuallyTriggeredScheduledExecutor mainThreadExecutor = new ManuallyTriggeredScheduledExecutor(); + + final CompletedCheckpointStore completedCheckpointStore = new StandaloneCompletedCheckpointStore(10); + final PendingCheckpoint checkpoint = createPendingCheckpoint( + props, + Collections.emptyList(), + Collections.emptyList(), + ioExecutor, + mainThreadExecutor, + completedCheckpointStore); + + checkpoint.acknowledgeTask(ATTEMPT_ID, null, new CheckpointMetrics()); + final CompletableFuture checkpointCompletableFuture = checkpoint.finalizeCheckpoint(); + + assertFalse(checkpointCompletableFuture.isDone()); + ioExecutor.triggerAll(); + + assertFalse(checkpointCompletableFuture.isDone()); + + checkpoint.abort(CheckpointFailureReason.CHECKPOINT_COORDINATOR_SHUTDOWN); + completedCheckpointStore.shutdown(JobStatus.CANCELED); + + mainThreadExecutor.triggerAll(); + // trigger the async cleanup operations + ioExecutor.triggerAll(); + assertTrue(checkpointCompletableFuture.isCompletedExceptionally()); + + assertEquals(0, completedCheckpointStore.getNumberOfRetainedCheckpoints()); + + final FsCheckpointStorageLocation location = + (FsCheckpointStorageLocation) checkpoint.getCheckpointStorageLocation(); + assertFalse(LocalFileSystem.getSharedInstance().exists(location.getCheckpointDirectory())); + } + + @Test + public void testAbortingBeforeAsyncFinalization() throws Exception { + final CheckpointProperties props = new CheckpointProperties(false, CheckpointType.SAVEPOINT, false, false, false, false, false); + + final ManuallyTriggeredScheduledExecutor ioExecutor = new ManuallyTriggeredScheduledExecutor(); + final ManuallyTriggeredScheduledExecutor mainThreadExecutor = new ManuallyTriggeredScheduledExecutor(); + + final CompletedCheckpointStore completedCheckpointStore = new StandaloneCompletedCheckpointStore(10); + final PendingCheckpoint checkpoint = createPendingCheckpoint( + props, + Collections.emptyList(), + Collections.emptyList(), + ioExecutor, + mainThreadExecutor, + completedCheckpointStore); + + checkpoint.acknowledgeTask(ATTEMPT_ID, null, new CheckpointMetrics()); + final CompletableFuture checkpointCompletableFuture = checkpoint.finalizeCheckpoint(); + + assertFalse(checkpointCompletableFuture.isDone()); + checkpoint.abort(CheckpointFailureReason.CHECKPOINT_COORDINATOR_SHUTDOWN); + completedCheckpointStore.shutdown(JobStatus.CANCELED); + + ioExecutor.triggerAll(); + + mainThreadExecutor.triggerAll(); + assertTrue(checkpointCompletableFuture.isCompletedExceptionally()); + + assertEquals(0, completedCheckpointStore.getNumberOfRetainedCheckpoints()); + + final FsCheckpointStorageLocation location = + (FsCheckpointStorageLocation) checkpoint.getCheckpointStorageLocation(); + assertFalse(LocalFileSystem.getSharedInstance().exists(location.getCheckpointDirectory())); + } + // ------------------------------------------------------------------------ private PendingCheckpoint createPendingCheckpoint(CheckpointProperties props) throws IOException { @@ -505,6 +649,20 @@ public class PendingCheckpointTest { return createPendingCheckpoint(props, Collections.emptyList(), masterStateIdentifiers, Executors.directExecutor()); } + private PendingCheckpoint createPendingCheckpoint(CheckpointProperties props, Collection operatorCoordinators, Collection masterStateIdentifiers, Executor executor) throws IOException { + return createPendingCheckpoint(props, operatorCoordinators, masterStateIdentifiers, executor, Executors.directExecutor()); + } + + private PendingCheckpoint createPendingCheckpoint( + CheckpointProperties props, + Collection operatorCoordinators, + Collection masterStateIdentifiers, + Executor executor, + Executor mainThreadExecutor) throws IOException { + + return createPendingCheckpoint(props, operatorCoordinators, masterStateIdentifiers, executor, mainThreadExecutor, new StandaloneCompletedCheckpointStore(1)); + } + private PendingCheckpoint createPendingCheckpointWithCoordinators( OperatorCoordinatorCheckpointContext... coordinators) throws IOException { @@ -536,15 +694,17 @@ public class PendingCheckpointTest { CheckpointProperties props, Collection operatorCoordinators, Collection masterStateIdentifiers, - Executor executor) throws IOException { + Executor executor, + Executor mainThreadExecutor, + CompletedCheckpointStore completedCheckpointStore) throws IOException { final Path checkpointDir = new Path(tmpFolder.newFolder().toURI()); final FsCheckpointStorageLocation location = new FsCheckpointStorageLocation( - LocalFileSystem.getSharedInstance(), - checkpointDir, checkpointDir, checkpointDir, - CheckpointStorageLocationReference.getDefault(), - 1024, - 4096); + LocalFileSystem.getSharedInstance(), + checkpointDir, checkpointDir, checkpointDir, + CheckpointStorageLocationReference.getDefault(), + 1024, + 4096); final Map ackTasks = new HashMap<>(ACK_TASKS); @@ -558,7 +718,9 @@ public class PendingCheckpointTest { props, location, executor, - new CompletableFuture<>()); + mainThreadExecutor, + new CompletableFuture<>(), + completedCheckpointStore); } private static OperatorCoordinatorCheckpointContext createOperatorCoordinator() { @@ -629,4 +791,28 @@ public class PendingCheckpointTest { return new StringSerializer(); } } + + private static class FailingCheckpointStorageLocation implements CheckpointStorageLocation { + + @Override + public CheckpointMetadataOutputStream createMetadataOutputStream() throws IOException { + throw new IOException("Create meta data output stream failed"); + } + + @Override + public void disposeOnFailure() throws IOException { + + } + + @Override + public CheckpointStorageLocationReference getLocationReference() { + return null; + } + + @Override + public CheckpointStateOutputStream createCheckpointStateOutputStream( + CheckpointedStateScope scope) throws IOException { + return null; + } + } } -- Gitee From 1a1386266ee94bf318f8105b96d2c84d74016f62 Mon Sep 17 00:00:00 2001 From: ifndef-SleePy Date: Wed, 4 Mar 2020 17:35:46 +0800 Subject: [PATCH 245/885] [FLINK-14971][checkpointing] Introduce main thread executor in CheckpointCoordinator to execute all non-IO operations instead of the timer thread --- .../checkpoint/CheckpointCoordinator.java | 90 ++++++--- .../executiongraph/ExecutionGraph.java | 3 + .../runtime/scheduler/SchedulerBase.java | 24 +-- .../CheckpointCoordinatorFailureTest.java | 8 +- .../CheckpointCoordinatorMasterHooksTest.java | 12 +- .../CheckpointCoordinatorRestoringTest.java | 34 ++-- .../checkpoint/CheckpointCoordinatorTest.java | 182 +++++++++--------- .../CheckpointCoordinatorTestingUtils.java | 16 +- .../CheckpointCoordinatorTriggeringTest.java | 74 +++---- .../CheckpointStateRestoreTest.java | 8 +- ...ecutionGraphCheckpointCoordinatorTest.java | 4 +- ...overStrategyCheckpointCoordinatorTest.java | 26 ++- ...StrategyNGAbortPendingCheckpointsTest.java | 10 +- .../ArchivedExecutionGraphTest.java | 4 +- .../scheduler/DefaultSchedulerTest.java | 45 ++++- 15 files changed, 331 insertions(+), 209 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java index a3692b4f0f..35cace779b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java @@ -22,6 +22,7 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; import org.apache.flink.runtime.checkpoint.hooks.MasterHooks; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.execution.ExecutionState; @@ -51,6 +52,7 @@ import org.apache.flink.util.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.io.IOException; @@ -151,6 +153,12 @@ public class CheckpointCoordinator { * It must be single-threaded. Eventually it will be replaced by main thread executor. */ private final ScheduledExecutor timer; + /** It can't be final for now because CheckpointCoordinator is created before + * mainThreadExecutor. So here we keep behavior same with ExecutionGraph, use a dummy executor + * in constructor and replace it with real main thread executor later. */ + @Nonnull + private ComponentMainThreadExecutor mainThreadExecutor; + /** The master checkpoint hooks executed by this checkpoint coordinator. */ private final HashMap> masterHooks; @@ -286,6 +294,11 @@ public class CheckpointCoordinator { this.failureManager = checkNotNull(failureManager); this.clock = checkNotNull(clock); + this.mainThreadExecutor = + new ComponentMainThreadExecutor.DummyComponentMainThreadExecutor( + "CheckpointCoordinator is not initialized with proper main thread executor. " + + "Call to CheckpointCoordinator.start(...) required."); + this.recentPendingCheckpoints = new ArrayDeque<>(NUM_GHOST_CHECKPOINT_IDS); this.masterHooks = new HashMap<>(); this.triggerRequestQueue = new ArrayDeque<>(); @@ -314,6 +327,19 @@ public class CheckpointCoordinator { // Configuration // -------------------------------------------------------------------------------------------- + /** + * Start the coordinator. + * + *

    + * The main thread executor is not initialized yet when the coordinator is created. + * So this method is used to pass main thread executor when it's ready. + *

    + * @param mainThreadExecutor the main thread executor + */ + public void start(ComponentMainThreadExecutor mainThreadExecutor) { + this.mainThreadExecutor = checkNotNull(mainThreadExecutor); + } + /** * Adds the given master hook to the checkpoint coordinator. This method does nothing, if * the checkpoint coordinator already contained a hook with the same ID (as defined via @@ -326,6 +352,8 @@ public class CheckpointCoordinator { public boolean addMasterHook(MasterTriggerRestoreHook hook) { checkNotNull(hook); + assertRunningMainThread(); + final String id = hook.getIdentifier(); checkArgument(!StringUtils.isNullOrWhitespaceOnly(id), "The hook has a null or empty id"); @@ -370,10 +398,11 @@ public class CheckpointCoordinator { */ public void shutdown(JobStatus jobStatus) throws Exception { synchronized (lock) { + assertRunningMainThread(); + if (!shutdown) { shutdown = true; LOG.info("Stopping checkpoint coordinator for job {}.", job); - periodicScheduling = false; periodicTriggeringSuspended = false; @@ -450,10 +479,8 @@ public class CheckpointCoordinator { checkNotNull(checkpointProperties); - // TODO, call triggerCheckpoint directly after removing timer thread - // for now, execute the trigger in timer thread to avoid competition final CompletableFuture resultFuture = new CompletableFuture<>(); - timer.execute(() -> triggerCheckpoint( + triggerCheckpoint( timestamp, checkpointProperties, targetLocation, @@ -465,7 +492,7 @@ public class CheckpointCoordinator { } else { resultFuture.completeExceptionally(throwable); } - })); + }); return resultFuture; } @@ -492,6 +519,8 @@ public class CheckpointCoordinator { boolean isPeriodic, boolean advanceToEndOfTime) { + assertRunningMainThread(); + if (advanceToEndOfTime && !(props.isSynchronous() && props.isSavepoint())) { return FutureUtils.completedExceptionally(new IllegalArgumentException( "Only synchronous savepoints are allowed to advance the watermark to MAX.")); @@ -555,7 +584,7 @@ public class CheckpointCoordinator { checkpointIdAndStorageLocation.checkpointId, checkpointIdAndStorageLocation.checkpointStorageLocation, onCompletionPromise), - timer); + mainThreadExecutor); final CompletableFuture masterStatesComplete = pendingCheckpointCompletableFuture .thenCompose(this::snapshotMasterState); @@ -563,8 +592,8 @@ public class CheckpointCoordinator { final CompletableFuture coordinatorCheckpointsComplete = pendingCheckpointCompletableFuture .thenComposeAsync((pendingCheckpoint) -> OperatorCoordinatorCheckpoints.triggerAndAcknowledgeAllCoordinatorCheckpointsWithCompletion( - coordinatorsToCheckpoint, pendingCheckpoint, timer), - timer); + coordinatorsToCheckpoint, pendingCheckpoint, mainThreadExecutor), + mainThreadExecutor); CompletableFuture.allOf(masterStatesComplete, coordinatorCheckpointsComplete) .whenCompleteAsync( @@ -591,7 +620,7 @@ public class CheckpointCoordinator { } } }, - timer); + mainThreadExecutor); } catch (Throwable throwable) { onTriggerFailure(onCompletionPromise, throwable); } @@ -636,6 +665,8 @@ public class CheckpointCoordinator { CheckpointStorageLocation checkpointStorageLocation, CompletableFuture onCompletionPromise) { + assertRunningMainThread(); + synchronized (lock) { try { // since we haven't created the PendingCheckpoint yet, we need to check the @@ -656,7 +687,7 @@ public class CheckpointCoordinator { props, checkpointStorageLocation, executor, - timer, + mainThreadExecutor, onCompletionPromise, completedCheckpointStore); @@ -670,10 +701,9 @@ public class CheckpointCoordinator { } synchronized (lock) { - pendingCheckpoints.put(checkpointID, checkpoint); - ScheduledFuture cancellerHandle = timer.schedule( + ScheduledFuture cancellerHandle = mainThreadExecutor.schedule( new CheckpointCanceller(checkpoint), checkpointTimeout, TimeUnit.MILLISECONDS); @@ -730,7 +760,7 @@ public class CheckpointCoordinator { masterStateCompletableFuture.completeExceptionally(t); } }, - timer); + mainThreadExecutor); } return masterStateCompletableFuture; } @@ -860,6 +890,8 @@ public class CheckpointCoordinator { * @param taskManagerLocationInfo The location info of the decline checkpoint message's sender */ public void receiveDeclineMessage(DeclineCheckpoint message, String taskManagerLocationInfo) { + assertRunningMainThread(); + if (shutdown || message == null) { return; } @@ -931,6 +963,8 @@ public class CheckpointCoordinator { * @throws CheckpointException If the checkpoint cannot be added to the completed checkpoint store. */ public boolean receiveAcknowledgeMessage(AcknowledgeCheckpoint message, String taskManagerLocationInfo) throws CheckpointException { + assertRunningMainThread(); + if (shutdown || message == null) { return false; } @@ -1043,7 +1077,7 @@ public class CheckpointCoordinator { onCheckpointSuccess(completedCheckpoint); } } - }, timer); + }, mainThreadExecutor); } /** @@ -1110,6 +1144,8 @@ public class CheckpointCoordinator { * @param cause of the failure */ public void failUnacknowledgedPendingCheckpointsFor(ExecutionAttemptID executionAttemptId, Throwable cause) { + assertRunningMainThread(); + synchronized (lock) { abortPendingCheckpoints( checkpoint -> !checkpoint.isAcknowledgedBy(executionAttemptId), @@ -1132,11 +1168,9 @@ public class CheckpointCoordinator { /** * Resumes suspended periodic triggering. - * - *

    NOTE: The caller of this method must hold the lock when invoking the method! */ private void resumePeriodicTriggering() { - assert(Thread.holdsLock(lock)); + assertRunningMainThread(); if (shutdown || !periodicScheduling) { return; @@ -1215,6 +1249,8 @@ public class CheckpointCoordinator { final boolean errorIfNoCheckpoint, final boolean allowNonRestoredState) throws Exception { + assertRunningMainThread(); + synchronized (lock) { if (shutdown) { throw new IllegalStateException("CheckpointCoordinator is shut down"); @@ -1391,6 +1427,8 @@ public class CheckpointCoordinator { // -------------------------------------------------------------------------------------------- public void startCheckpointScheduler() { + assertRunningMainThread(); + synchronized (lock) { if (shutdown) { throw new IllegalArgumentException("Checkpoint coordinator is shut down"); @@ -1405,6 +1443,8 @@ public class CheckpointCoordinator { } public void stopCheckpointScheduler() { + assertRunningMainThread(); + synchronized (lock) { periodicTriggeringSuspended = false; periodicScheduling = false; @@ -1437,6 +1477,7 @@ public class CheckpointCoordinator { CheckpointException exception) { assert Thread.holdsLock(lock); + assertRunningMainThread(); final PendingCheckpoint[] pendingCheckpointsToFail = pendingCheckpoints .values() @@ -1523,12 +1564,7 @@ public class CheckpointCoordinator { @Override public void run() { - try { - triggerCheckpoint(System.currentTimeMillis(), true); - } - catch (Exception e) { - LOG.error("Exception while triggering checkpoint for job {}.", job, e); - } + mainThreadExecutor.execute(() -> triggerCheckpoint(clock.absoluteTimeMillis(), true)); } } @@ -1576,6 +1612,7 @@ public class CheckpointCoordinator { @Nullable final ExecutionAttemptID executionAttemptID) { assert(Thread.holdsLock(lock)); + assertRunningMainThread(); if (!pendingCheckpoint.isDiscarded()) { try { @@ -1691,6 +1728,12 @@ public class CheckpointCoordinator { abortPendingCheckpoints(exception); } + private void assertRunningMainThread() { + if (!(mainThreadExecutor instanceof ComponentMainThreadExecutor.DummyComponentMainThreadExecutor)) { + mainThreadExecutor.assertRunningInMainThread(); + } + } + /** * The canceller of checkpoint. The checkpoint might be cancelled if it doesn't finish in a * configured period. @@ -1705,6 +1748,7 @@ public class CheckpointCoordinator { @Override public void run() { + assertRunningMainThread(); synchronized (lock) { // only do the work if the checkpoint is not discarded anyways // note that checkpoint completion discards the pending checkpoint object diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index 09a6a3aae5..7a64f91532 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -397,6 +397,9 @@ public class ExecutionGraph implements AccessExecutionGraph { public void start(@Nonnull ComponentMainThreadExecutor jobMasterMainThreadExecutor) { this.jobMasterMainThreadExecutor = jobMasterMainThreadExecutor; + if (checkpointCoordinator != null) { + checkpointCoordinator.start(this.jobMasterMainThreadExecutor); + } } // -------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java index 4664f6fea9..11f1204b2e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java @@ -798,13 +798,11 @@ public abstract class SchedulerBase implements SchedulerNG { final String taskManagerLocationInfo = retrieveTaskManagerLocation(executionAttemptID); if (checkpointCoordinator != null) { - ioExecutor.execute(() -> { - try { - checkpointCoordinator.receiveAcknowledgeMessage(ackMessage, taskManagerLocationInfo); - } catch (Throwable t) { - log.warn("Error while processing checkpoint acknowledgement message", t); - } - }); + try { + checkpointCoordinator.receiveAcknowledgeMessage(ackMessage, taskManagerLocationInfo); + } catch (Throwable t) { + log.warn("Error while processing checkpoint acknowledgement message", t); + } } else { String errorMessage = "Received AcknowledgeCheckpoint message for job {} with no CheckpointCoordinator"; if (executionGraph.getState() == JobStatus.RUNNING) { @@ -823,13 +821,11 @@ public abstract class SchedulerBase implements SchedulerNG { final String taskManagerLocationInfo = retrieveTaskManagerLocation(decline.getTaskExecutionId()); if (checkpointCoordinator != null) { - ioExecutor.execute(() -> { - try { - checkpointCoordinator.receiveDeclineMessage(decline, taskManagerLocationInfo); - } catch (Exception e) { - log.error("Error in CheckpointCoordinator while processing {}", decline, e); - } - }); + try { + checkpointCoordinator.receiveDeclineMessage(decline, taskManagerLocationInfo); + } catch (Exception e) { + log.error("Error in CheckpointCoordinator while processing {}", decline, e); + } } else { String errorMessage = "Received DeclineCheckpoint message for job {} with no CheckpointCoordinator"; if (executionGraph.getState() == JobStatus.RUNNING) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java index e6e907260e..52990ae401 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java @@ -56,7 +56,7 @@ public class CheckpointCoordinatorFailureTest extends TestLogger { public void testFailingCompletedCheckpointStoreAdd() throws Exception { JobID jid = new JobID(); - final ManuallyTriggeredScheduledExecutor manuallyTriggeredScheduledExecutor = + final ManuallyTriggeredScheduledExecutor mainThreadExecutor = new ManuallyTriggeredScheduledExecutor(); final ExecutionAttemptID executionAttemptId = new ExecutionAttemptID(); @@ -70,12 +70,12 @@ public class CheckpointCoordinatorFailureTest extends TestLogger { .setJobId(jid) .setTasks(new ExecutionVertex[] { vertex }) .setCompletedCheckpointStore(new FailingCompletedCheckpointStore()) - .setTimer(manuallyTriggeredScheduledExecutor) + .setMainThreadExecutor(mainThreadExecutor) .build(); coord.triggerCheckpoint(triggerTimestamp, false); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertEquals(1, coord.getNumberOfPendingCheckpoints()); @@ -105,7 +105,7 @@ public class CheckpointCoordinatorFailureTest extends TestLogger { coord.receiveAcknowledgeMessage(acknowledgeMessage, "Unknown location"); // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); // make sure that the pending checkpoint has been discarded after we could not complete it assertTrue(pendingCheckpoint.isDiscarded()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java index a756e52ee3..0c8827b3d4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java @@ -21,6 +21,7 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor; import org.apache.flink.runtime.concurrent.ScheduledExecutor; @@ -444,7 +445,7 @@ public class CheckpointCoordinatorMasterHooksTest { private CheckpointCoordinator instantiateCheckpointCoordinator( JobID jid, - ScheduledExecutor testingScheduledExecutor, + ScheduledExecutor mainThreadExecutor, ExecutionVertex... ackVertices) { CheckpointCoordinatorConfiguration chkConfig = new CheckpointCoordinatorConfiguration( @@ -456,7 +457,7 @@ public class CheckpointCoordinatorMasterHooksTest { true, false, 0); - return new CheckpointCoordinator( + final CheckpointCoordinator checkpointCoordinator = new CheckpointCoordinator( jid, chkConfig, new ExecutionVertex[0], @@ -467,11 +468,16 @@ public class CheckpointCoordinatorMasterHooksTest { new StandaloneCompletedCheckpointStore(10), new MemoryStateBackend(), Executors.directExecutor(), - testingScheduledExecutor, + new ManuallyTriggeredScheduledExecutor(), SharedStateRegistry.DEFAULT_FACTORY, new CheckpointFailureManager( 0, NoOpFailJobCall.INSTANCE)); + checkpointCoordinator.start( + new ComponentMainThreadExecutorServiceAdapter( + mainThreadExecutor, + Thread.currentThread())); + return checkpointCoordinator; } private static T mockGeneric(Class clazz) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java index 49ef8e34b4..04bc1549d2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java @@ -94,14 +94,14 @@ import static org.mockito.Mockito.when; public class CheckpointCoordinatorRestoringTest extends TestLogger { private static final String TASK_MANAGER_LOCATION_INFO = "Unknown location"; - private ManuallyTriggeredScheduledExecutor manuallyTriggeredScheduledExecutor; + private ManuallyTriggeredScheduledExecutor mainThreadExecutor; @Rule public TemporaryFolder tmpFolder = new TemporaryFolder(); @Before public void setUp() throws Exception { - manuallyTriggeredScheduledExecutor = new ManuallyTriggeredScheduledExecutor(); + mainThreadExecutor = new ManuallyTriggeredScheduledExecutor(); } /** @@ -147,12 +147,12 @@ public class CheckpointCoordinatorRestoringTest extends TestLogger { .setJobId(jid) .setTasks(arrayExecutionVertices) .setCompletedCheckpointStore(store) - .setTimer(manuallyTriggeredScheduledExecutor) + .setMainThreadExecutor(mainThreadExecutor) .build(); // trigger the checkpoint coord.triggerCheckpoint(timestamp, false); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertEquals(1, coord.getPendingCheckpoints().size()); long checkpointId = Iterables.getOnlyElement(coord.getPendingCheckpoints().keySet()); @@ -185,7 +185,7 @@ public class CheckpointCoordinatorRestoringTest extends TestLogger { coord.receiveAcknowledgeMessage(acknowledgeCheckpoint, TASK_MANAGER_LOCATION_INFO); // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); } List completedCheckpoints = coord.getSuccessfulCheckpoints(); @@ -274,13 +274,13 @@ public class CheckpointCoordinatorRestoringTest extends TestLogger { .setCheckpointIDCounter(checkpointIDCounter) .setCompletedCheckpointStore(store) .setTasks(new ExecutionVertex[] { stateful1, stateless1 }) - .setTimer(manuallyTriggeredScheduledExecutor) + .setMainThreadExecutor(mainThreadExecutor) .build(); //trigger a checkpoint and wait to become a completed checkpoint final CompletableFuture checkpointFuture = coord.triggerCheckpoint(timestamp, false); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertFalse(checkpointFuture.isCompletedExceptionally()); long checkpointId = checkpointIDCounter.getLast(); @@ -302,7 +302,7 @@ public class CheckpointCoordinatorRestoringTest extends TestLogger { coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statefulExec1.getAttemptId(), checkpointId, new CheckpointMetrics(), subtaskStatesForCheckpoint), TASK_MANAGER_LOCATION_INFO); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statelessExec1.getAttemptId(), checkpointId), TASK_MANAGER_LOCATION_INFO); // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); CompletedCheckpoint success = coord.getSuccessfulCheckpoints().get(0); assertEquals(jid, success.getJobId()); @@ -326,12 +326,12 @@ public class CheckpointCoordinatorRestoringTest extends TestLogger { StateObjectCollection.singleton(serializedKeyGroupStatesForSavepoint), StateObjectCollection.empty())); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); checkpointId = checkpointIDCounter.getLast(); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statefulExec1.getAttemptId(), checkpointId, new CheckpointMetrics(), subtaskStatesForSavepoint), TASK_MANAGER_LOCATION_INFO); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statelessExec1.getAttemptId(), checkpointId), TASK_MANAGER_LOCATION_INFO); // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertNotNull(savepointFuture.get()); @@ -416,12 +416,12 @@ public class CheckpointCoordinatorRestoringTest extends TestLogger { new CheckpointCoordinatorBuilder() .setJobId(jid) .setTasks(arrayExecutionVertices) - .setTimer(manuallyTriggeredScheduledExecutor) + .setMainThreadExecutor(mainThreadExecutor) .build(); // trigger the checkpoint coord.triggerCheckpoint(timestamp, false); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertEquals(1, coord.getPendingCheckpoints().size()); long checkpointId = Iterables.getOnlyElement(coord.getPendingCheckpoints().keySet()); @@ -474,7 +474,7 @@ public class CheckpointCoordinatorRestoringTest extends TestLogger { coord.receiveAcknowledgeMessage(acknowledgeCheckpoint, TASK_MANAGER_LOCATION_INFO); // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); } List completedCheckpoints = coord.getSuccessfulCheckpoints(); @@ -581,12 +581,12 @@ public class CheckpointCoordinatorRestoringTest extends TestLogger { new CheckpointCoordinatorBuilder() .setJobId(jid) .setTasks(arrayExecutionVertices) - .setTimer(manuallyTriggeredScheduledExecutor) + .setMainThreadExecutor(mainThreadExecutor) .build(); // trigger the checkpoint coord.triggerCheckpoint(timestamp, false); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertEquals(1, coord.getPendingCheckpoints().size()); long checkpointId = Iterables.getOnlyElement(coord.getPendingCheckpoints().keySet()); @@ -623,7 +623,7 @@ public class CheckpointCoordinatorRestoringTest extends TestLogger { coord.receiveAcknowledgeMessage(acknowledgeCheckpoint, TASK_MANAGER_LOCATION_INFO); // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); } List completedCheckpoints = coord.getSuccessfulCheckpoints(); @@ -823,7 +823,7 @@ public class CheckpointCoordinatorRestoringTest extends TestLogger { new CheckpointCoordinatorBuilder() .setTasks(newJobVertex1.getTaskVertices()) .setCompletedCheckpointStore(standaloneCompletedCheckpointStore) - .setTimer(manuallyTriggeredScheduledExecutor) + .setMainThreadExecutor(mainThreadExecutor) .build(); coord.restoreLatestCheckpointedState(tasks, false, true); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java index 846c809790..cf444a7587 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java @@ -109,14 +109,17 @@ public class CheckpointCoordinatorTest extends TestLogger { private static final String TASK_MANAGER_LOCATION_INFO = "Unknown location"; - private ManuallyTriggeredScheduledExecutor manuallyTriggeredScheduledExecutor; + private ManuallyTriggeredScheduledExecutor timer; + + private ManuallyTriggeredScheduledExecutor mainThreadExecutor; @Rule public TemporaryFolder tmpFolder = new TemporaryFolder(); @Before public void setUp() throws Exception { - manuallyTriggeredScheduledExecutor = new ManuallyTriggeredScheduledExecutor(); + timer = new ManuallyTriggeredScheduledExecutor(); + mainThreadExecutor = new ManuallyTriggeredScheduledExecutor(); } @Test @@ -134,7 +137,7 @@ public class CheckpointCoordinatorTest extends TestLogger { // trigger the first checkpoint. this should not succeed final CompletableFuture checkpointFuture = coord.triggerCheckpoint(timestamp, false); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertTrue(checkpointFuture.isCompletedExceptionally()); // still, nothing should be happening @@ -163,7 +166,7 @@ public class CheckpointCoordinatorTest extends TestLogger { // trigger the first checkpoint. this should not succeed final CompletableFuture checkpointFuture = coord.triggerCheckpoint(timestamp, false); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertTrue(checkpointFuture.isCompletedExceptionally()); // still, nothing should be happening @@ -192,7 +195,7 @@ public class CheckpointCoordinatorTest extends TestLogger { // trigger the first checkpoint. this should not succeed final CompletableFuture checkpointFuture = coord.triggerCheckpoint(timestamp, false); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertTrue(checkpointFuture.isCompletedExceptionally()); // still, nothing should be happening @@ -241,7 +244,7 @@ public class CheckpointCoordinatorTest extends TestLogger { // trigger the checkpoint. this should succeed final CompletableFuture checkPointFuture = coord.triggerCheckpoint(timestamp, false); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertFalse(checkPointFuture.isCompletedExceptionally()); long checkpointId = coord.getPendingCheckpoints().entrySet().iterator().next().getKey(); @@ -297,7 +300,7 @@ public class CheckpointCoordinatorTest extends TestLogger { // trigger the first checkpoint. this should succeed final CompletableFuture checkpointFuture = coord.triggerCheckpoint(timestamp, false); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertFalse(checkpointFuture.isCompletedExceptionally()); // validate that we have a pending checkpoint @@ -305,7 +308,7 @@ public class CheckpointCoordinatorTest extends TestLogger { assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); // we have one task scheduled that will cancel after timeout - assertEquals(1, manuallyTriggeredScheduledExecutor.getScheduledTasks().size()); + assertEquals(1, mainThreadExecutor.getScheduledTasks().size()); long checkpointId = coord.getPendingCheckpoints().entrySet().iterator().next().getKey(); PendingCheckpoint checkpoint = coord.getPendingCheckpoints().get(checkpointId); @@ -342,7 +345,7 @@ public class CheckpointCoordinatorTest extends TestLogger { assertTrue(checkpoint.isDiscarded()); // the canceler is also removed - assertEquals(0, manuallyTriggeredScheduledExecutor.getScheduledTasks().size()); + assertEquals(0, mainThreadExecutor.getScheduledTasks().size()); // validate that we have no new pending checkpoint assertEquals(0, coord.getNumberOfPendingCheckpoints()); @@ -383,24 +386,24 @@ public class CheckpointCoordinatorTest extends TestLogger { assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); - assertEquals(0, manuallyTriggeredScheduledExecutor.getScheduledTasks().size()); + assertEquals(0, mainThreadExecutor.getScheduledTasks().size()); // trigger the first checkpoint. this should succeed final CompletableFuture checkpointFuture1 = coord.triggerCheckpoint(timestamp, false); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertFalse(checkpointFuture1.isCompletedExceptionally()); // trigger second checkpoint, should also succeed final CompletableFuture checkpointFuture2 = coord.triggerCheckpoint(timestamp + 2, false); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertFalse(checkpointFuture2.isCompletedExceptionally()); // validate that we have a pending checkpoint assertEquals(2, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); - assertEquals(2, manuallyTriggeredScheduledExecutor.getScheduledTasks().size()); + assertEquals(2, mainThreadExecutor.getScheduledTasks().size()); Iterator> it = coord.getPendingCheckpoints().entrySet().iterator(); long checkpoint1Id = it.next().getKey(); @@ -447,7 +450,7 @@ public class CheckpointCoordinatorTest extends TestLogger { // validate that we have only one pending checkpoint left assertEquals(1, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); - assertEquals(1, manuallyTriggeredScheduledExecutor.getScheduledTasks().size()); + assertEquals(1, mainThreadExecutor.getScheduledTasks().size()); // validate that it is the same second checkpoint from earlier long checkpointIdNew = coord.getPendingCheckpoints().entrySet().iterator().next().getKey(); @@ -495,18 +498,18 @@ public class CheckpointCoordinatorTest extends TestLogger { assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); - assertEquals(0, manuallyTriggeredScheduledExecutor.getScheduledTasks().size()); + assertEquals(0, mainThreadExecutor.getScheduledTasks().size()); // trigger the first checkpoint. this should succeed final CompletableFuture checkpointFuture = coord.triggerCheckpoint(timestamp, false); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertFalse(checkpointFuture.isCompletedExceptionally()); // validate that we have a pending checkpoint assertEquals(1, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); - assertEquals(1, manuallyTriggeredScheduledExecutor.getScheduledTasks().size()); + assertEquals(1, mainThreadExecutor.getScheduledTasks().size()); long checkpointId = coord.getPendingCheckpoints().entrySet().iterator().next().getKey(); PendingCheckpoint checkpoint = coord.getPendingCheckpoints().get(checkpointId); @@ -555,7 +558,7 @@ public class CheckpointCoordinatorTest extends TestLogger { coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID1, checkpointId, new CheckpointMetrics(), taskOperatorSubtaskStates1), TASK_MANAGER_LOCATION_INFO); // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); // the checkpoint is internally converted to a successful checkpoint and the // pending checkpoint object is disposed @@ -566,7 +569,7 @@ public class CheckpointCoordinatorTest extends TestLogger { assertEquals(0, coord.getNumberOfPendingCheckpoints()); // the canceler should be removed now - assertEquals(0, manuallyTriggeredScheduledExecutor.getScheduledTasks().size()); + assertEquals(0, mainThreadExecutor.getScheduledTasks().size()); // validate that the subtasks states have registered their shared states. { @@ -591,17 +594,17 @@ public class CheckpointCoordinatorTest extends TestLogger { // --------------- final long timestampNew = timestamp + 7; coord.triggerCheckpoint(timestampNew, false); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); long checkpointIdNew = coord.getPendingCheckpoints().entrySet().iterator().next().getKey(); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID1, checkpointIdNew), TASK_MANAGER_LOCATION_INFO); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID2, checkpointIdNew), TASK_MANAGER_LOCATION_INFO); // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(1, coord.getNumberOfRetainedSuccessfulCheckpoints()); - assertEquals(0, manuallyTriggeredScheduledExecutor.getScheduledTasks().size()); + assertEquals(0, mainThreadExecutor.getScheduledTasks().size()); CompletedCheckpoint successNew = coord.getSuccessfulCheckpoints().get(0); assertEquals(jid, successNew.getJobId()); @@ -661,7 +664,7 @@ public class CheckpointCoordinatorTest extends TestLogger { .setTasksToWaitFor(new ExecutionVertex[] { ackVertex1, ackVertex2, ackVertex3 }) .setTasksToCommitTo(new ExecutionVertex[] { commitVertex }) .setCompletedCheckpointStore(new StandaloneCompletedCheckpointStore(2)) - .setTimer(manuallyTriggeredScheduledExecutor) + .setMainThreadExecutor(mainThreadExecutor) .build(); assertEquals(0, coord.getNumberOfPendingCheckpoints()); @@ -670,7 +673,7 @@ public class CheckpointCoordinatorTest extends TestLogger { // trigger the first checkpoint. this should succeed final CompletableFuture checkpointFuture1 = coord.triggerCheckpoint(timestamp1, false); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertFalse(checkpointFuture1.isCompletedExceptionally()); assertEquals(1, coord.getNumberOfPendingCheckpoints()); @@ -690,7 +693,7 @@ public class CheckpointCoordinatorTest extends TestLogger { // trigger the first checkpoint. this should succeed final CompletableFuture checkpointFuture2 = coord.triggerCheckpoint(timestamp2, false); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertFalse(checkpointFuture2.isCompletedExceptionally()); assertEquals(2, coord.getNumberOfPendingCheckpoints()); @@ -716,7 +719,7 @@ public class CheckpointCoordinatorTest extends TestLogger { coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID1, checkpointId1), TASK_MANAGER_LOCATION_INFO); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID2, checkpointId2), TASK_MANAGER_LOCATION_INFO); // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); // now, the first checkpoint should be confirmed assertEquals(1, coord.getNumberOfPendingCheckpoints()); @@ -729,7 +732,7 @@ public class CheckpointCoordinatorTest extends TestLogger { // send the last remaining ack for the second checkpoint coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID3, checkpointId2), TASK_MANAGER_LOCATION_INFO); // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); // now, the second checkpoint should be confirmed assertEquals(0, coord.getNumberOfPendingCheckpoints()); @@ -796,7 +799,7 @@ public class CheckpointCoordinatorTest extends TestLogger { .setTasksToWaitFor(new ExecutionVertex[] { ackVertex1, ackVertex2, ackVertex3 }) .setTasksToCommitTo(new ExecutionVertex[] { commitVertex }) .setCompletedCheckpointStore(new StandaloneCompletedCheckpointStore(10)) - .setTimer(manuallyTriggeredScheduledExecutor) + .setMainThreadExecutor(mainThreadExecutor) .build(); assertEquals(0, coord.getNumberOfPendingCheckpoints()); @@ -805,7 +808,7 @@ public class CheckpointCoordinatorTest extends TestLogger { // trigger the first checkpoint. this should succeed final CompletableFuture checkpointFuture1 = coord.triggerCheckpoint(timestamp1, false); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertFalse(checkpointFuture1.isCompletedExceptionally()); assertEquals(1, coord.getNumberOfPendingCheckpoints()); @@ -840,7 +843,7 @@ public class CheckpointCoordinatorTest extends TestLogger { // trigger the first checkpoint. this should succeed final CompletableFuture checkpointFuture2 = coord.triggerCheckpoint(timestamp2, false); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertFalse(checkpointFuture2.isCompletedExceptionally()); assertEquals(2, coord.getNumberOfPendingCheckpoints()); @@ -883,7 +886,7 @@ public class CheckpointCoordinatorTest extends TestLogger { coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID2, checkpointId2, new CheckpointMetrics(), taskOperatorSubtaskStates22), TASK_MANAGER_LOCATION_INFO); // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); // now, the second checkpoint should be confirmed, and the first discarded // actually both pending checkpoints are discarded, and the second has been transformed @@ -962,13 +965,13 @@ public class CheckpointCoordinatorTest extends TestLogger { .setTasksToWaitFor(new ExecutionVertex[] { ackVertex1, ackVertex2 }) .setTasksToCommitTo(new ExecutionVertex[] { commitVertex }) .setCompletedCheckpointStore(new StandaloneCompletedCheckpointStore(2)) - .setTimer(manuallyTriggeredScheduledExecutor) + .setMainThreadExecutor(mainThreadExecutor) .build(); // trigger a checkpoint, partially acknowledged final CompletableFuture checkpointFuture = coord.triggerCheckpoint(timestamp, false); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertFalse(checkpointFuture.isCompletedExceptionally()); assertEquals(1, coord.getNumberOfPendingCheckpoints()); @@ -984,7 +987,7 @@ public class CheckpointCoordinatorTest extends TestLogger { coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID1, checkpoint.getCheckpointId(), new CheckpointMetrics(), taskOperatorSubtaskStates1), TASK_MANAGER_LOCATION_INFO); // triggers cancelling - manuallyTriggeredScheduledExecutor.triggerScheduledTasks(); + mainThreadExecutor.triggerScheduledTasks(); assertTrue("Checkpoint was not canceled by the timeout", checkpoint.isDiscarded()); assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); @@ -1028,12 +1031,12 @@ public class CheckpointCoordinatorTest extends TestLogger { .setTasksToWaitFor(new ExecutionVertex[] { ackVertex1, ackVertex2 }) .setTasksToCommitTo(new ExecutionVertex[] { commitVertex }) .setCompletedCheckpointStore(new StandaloneCompletedCheckpointStore(2)) - .setTimer(manuallyTriggeredScheduledExecutor) + .setMainThreadExecutor(mainThreadExecutor) .build(); final CompletableFuture checkpointFuture = coord.triggerCheckpoint(timestamp, false); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertFalse(checkpointFuture.isCompletedExceptionally()); long checkpointId = coord.getPendingCheckpoints().keySet().iterator().next(); @@ -1093,12 +1096,12 @@ public class CheckpointCoordinatorTest extends TestLogger { .setTasksToTrigger(new ExecutionVertex[] { triggerVertex }) .setTasksToWaitFor(new ExecutionVertex[] {triggerVertex, ackVertex1, ackVertex2}) .setTasksToCommitTo(new ExecutionVertex[0]) - .setTimer(manuallyTriggeredScheduledExecutor) + .setMainThreadExecutor(mainThreadExecutor) .build(); final CompletableFuture checkpointFuture = coord.triggerCheckpoint(timestamp, false); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertFalse(checkpointFuture.isCompletedExceptionally()); assertEquals(1, coord.getNumberOfPendingCheckpoints()); @@ -1210,7 +1213,7 @@ public class CheckpointCoordinatorTest extends TestLogger { // trigger the first checkpoint. this should succeed String savepointDir = tmpFolder.newFolder().getAbsolutePath(); CompletableFuture savepointFuture = coord.triggerSavepoint(timestamp, savepointDir); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertFalse(savepointFuture.isDone()); // validate that we have a pending savepoint @@ -1257,7 +1260,7 @@ public class CheckpointCoordinatorTest extends TestLogger { // acknowledge the other task. coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID1, checkpointId, new CheckpointMetrics(), taskOperatorSubtaskStates1), TASK_MANAGER_LOCATION_INFO); // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); // the checkpoint is internally converted to a successful checkpoint and the // pending checkpoint object is disposed @@ -1291,14 +1294,14 @@ public class CheckpointCoordinatorTest extends TestLogger { // --------------- final long timestampNew = timestamp + 7; savepointFuture = coord.triggerSavepoint(timestampNew, savepointDir); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertFalse(savepointFuture.isDone()); long checkpointIdNew = coord.getPendingCheckpoints().entrySet().iterator().next().getKey(); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID1, checkpointIdNew), TASK_MANAGER_LOCATION_INFO); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID2, checkpointIdNew), TASK_MANAGER_LOCATION_INFO); // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(1, coord.getNumberOfRetainedSuccessfulCheckpoints()); @@ -1352,7 +1355,7 @@ public class CheckpointCoordinatorTest extends TestLogger { .setTasks(new ExecutionVertex[]{ vertex1, vertex2 }) .setCheckpointIDCounter(counter) .setCompletedCheckpointStore(new StandaloneCompletedCheckpointStore(10)) - .setTimer(manuallyTriggeredScheduledExecutor) + .setMainThreadExecutor(mainThreadExecutor) .build(); String savepointDir = tmpFolder.newFolder().getAbsolutePath(); @@ -1360,19 +1363,19 @@ public class CheckpointCoordinatorTest extends TestLogger { // Trigger savepoint and checkpoint CompletableFuture savepointFuture1 = coord.triggerSavepoint(timestamp, savepointDir); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); long savepointId1 = counter.getLast(); assertEquals(1, coord.getNumberOfPendingCheckpoints()); CompletableFuture checkpointFuture1 = coord.triggerCheckpoint(timestamp + 1, false); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertEquals(2, coord.getNumberOfPendingCheckpoints()); assertFalse(checkpointFuture1.isCompletedExceptionally()); CompletableFuture checkpointFuture2 = coord.triggerCheckpoint(timestamp + 2, false); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertFalse(checkpointFuture2.isCompletedExceptionally()); long checkpointId2 = counter.getLast(); assertEquals(3, coord.getNumberOfPendingCheckpoints()); @@ -1381,7 +1384,7 @@ public class CheckpointCoordinatorTest extends TestLogger { coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID1, checkpointId2), TASK_MANAGER_LOCATION_INFO); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID2, checkpointId2), TASK_MANAGER_LOCATION_INFO); // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertEquals(1, coord.getNumberOfPendingCheckpoints()); assertEquals(1, coord.getNumberOfRetainedSuccessfulCheckpoints()); @@ -1391,12 +1394,12 @@ public class CheckpointCoordinatorTest extends TestLogger { CompletableFuture checkpointFuture3 = coord.triggerCheckpoint(timestamp + 3, false); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertFalse(checkpointFuture3.isCompletedExceptionally()); assertEquals(2, coord.getNumberOfPendingCheckpoints()); CompletableFuture savepointFuture2 = coord.triggerSavepoint(timestamp + 4, savepointDir); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); long savepointId2 = counter.getLast(); assertFalse(savepointFuture2.isCompletedExceptionally()); assertEquals(3, coord.getNumberOfPendingCheckpoints()); @@ -1405,7 +1408,7 @@ public class CheckpointCoordinatorTest extends TestLogger { coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID1, savepointId2), TASK_MANAGER_LOCATION_INFO); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID2, savepointId2), TASK_MANAGER_LOCATION_INFO); // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertEquals(1, coord.getNumberOfPendingCheckpoints()); assertEquals(2, coord.getNumberOfRetainedSuccessfulCheckpoints()); @@ -1418,7 +1421,7 @@ public class CheckpointCoordinatorTest extends TestLogger { coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID1, savepointId1), TASK_MANAGER_LOCATION_INFO); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID2, savepointId1), TASK_MANAGER_LOCATION_INFO); // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(3, coord.getNumberOfRetainedSuccessfulCheckpoints()); @@ -1467,14 +1470,15 @@ public class CheckpointCoordinatorTest extends TestLogger { .setTasksToWaitFor(new ExecutionVertex[] { ackVertex }) .setTasksToCommitTo(new ExecutionVertex[] { commitVertex }) .setCompletedCheckpointStore(new StandaloneCompletedCheckpointStore(2)) - .setTimer(manuallyTriggeredScheduledExecutor) + .setTimer(timer) + .setMainThreadExecutor(mainThreadExecutor) .build(); coord.startCheckpointScheduler(); for (int i = 0; i < maxConcurrentAttempts; i++) { - manuallyTriggeredScheduledExecutor.triggerPeriodicScheduledTasks(); - manuallyTriggeredScheduledExecutor.triggerAll(); + timer.triggerPeriodicScheduledTasks(); + mainThreadExecutor.triggerAll(); } assertEquals(maxConcurrentAttempts, numCalls.get()); @@ -1485,20 +1489,20 @@ public class CheckpointCoordinatorTest extends TestLogger { // now, once we acknowledge one checkpoint, it should trigger the next one coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID, 1L), TASK_MANAGER_LOCATION_INFO); // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); final Collection> periodicScheduledTasks = - manuallyTriggeredScheduledExecutor.getPeriodicScheduledTask(); + timer.getPeriodicScheduledTask(); assertEquals(1, periodicScheduledTasks.size()); - manuallyTriggeredScheduledExecutor.triggerPeriodicScheduledTasks(); - manuallyTriggeredScheduledExecutor.triggerAll(); + timer.triggerPeriodicScheduledTasks(); + mainThreadExecutor.triggerAll(); assertEquals(maxConcurrentAttempts + 1, numCalls.get()); // no further checkpoints should happen - manuallyTriggeredScheduledExecutor.triggerPeriodicScheduledTasks(); - manuallyTriggeredScheduledExecutor.triggerAll(); + timer.triggerPeriodicScheduledTasks(); + mainThreadExecutor.triggerAll(); assertEquals(maxConcurrentAttempts + 1, numCalls.get()); coord.shutdown(JobStatus.FINISHED); @@ -1539,14 +1543,15 @@ public class CheckpointCoordinatorTest extends TestLogger { .setTasksToWaitFor(new ExecutionVertex[] { ackVertex }) .setTasksToCommitTo(new ExecutionVertex[] { commitVertex }) .setCompletedCheckpointStore(new StandaloneCompletedCheckpointStore(2)) - .setTimer(manuallyTriggeredScheduledExecutor) + .setTimer(timer) + .setMainThreadExecutor(mainThreadExecutor) .build(); coord.startCheckpointScheduler(); do { - manuallyTriggeredScheduledExecutor.triggerPeriodicScheduledTasks(); - manuallyTriggeredScheduledExecutor.triggerAll(); + timer.triggerPeriodicScheduledTasks(); + mainThreadExecutor.triggerAll(); } while (coord.getNumberOfPendingCheckpoints() < maxConcurrentAttempts); @@ -1562,8 +1567,8 @@ public class CheckpointCoordinatorTest extends TestLogger { // after a while, there should be the new checkpoints do { - manuallyTriggeredScheduledExecutor.triggerPeriodicScheduledTasks(); - manuallyTriggeredScheduledExecutor.triggerAll(); + timer.triggerPeriodicScheduledTasks(); + mainThreadExecutor.triggerAll(); } while (coord.getNumberOfPendingCheckpoints() < maxConcurrentAttempts); @@ -1612,13 +1617,14 @@ public class CheckpointCoordinatorTest extends TestLogger { .setTasksToWaitFor(new ExecutionVertex[] { ackVertex }) .setTasksToCommitTo(new ExecutionVertex[] { commitVertex }) .setCompletedCheckpointStore(new StandaloneCompletedCheckpointStore(2)) - .setTimer(manuallyTriggeredScheduledExecutor) + .setTimer(timer) + .setMainThreadExecutor(mainThreadExecutor) .build(); coord.startCheckpointScheduler(); - manuallyTriggeredScheduledExecutor.triggerPeriodicScheduledTasks(); - manuallyTriggeredScheduledExecutor.triggerAll(); + timer.triggerPeriodicScheduledTasks(); + mainThreadExecutor.triggerAll(); // no checkpoint should have started so far assertEquals(0, coord.getNumberOfPendingCheckpoints()); @@ -1626,8 +1632,8 @@ public class CheckpointCoordinatorTest extends TestLogger { currentState.set(ExecutionState.RUNNING); // the coordinator should start checkpointing now - manuallyTriggeredScheduledExecutor.triggerPeriodicScheduledTasks(); - manuallyTriggeredScheduledExecutor.triggerAll(); + timer.triggerPeriodicScheduledTasks(); + mainThreadExecutor.triggerAll(); assertTrue(coord.getNumberOfPendingCheckpoints() > 0); } @@ -1661,7 +1667,7 @@ public class CheckpointCoordinatorTest extends TestLogger { .setTasks(new ExecutionVertex[] { vertex1 }) .setCheckpointIDCounter(checkpointIDCounter) .setCompletedCheckpointStore(new StandaloneCompletedCheckpointStore(2)) - .setTimer(manuallyTriggeredScheduledExecutor) + .setMainThreadExecutor(mainThreadExecutor) .build(); List> savepointFutures = new ArrayList<>(); @@ -1678,7 +1684,7 @@ public class CheckpointCoordinatorTest extends TestLogger { assertFalse(savepointFuture.isDone()); } - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); // ACK all savepoints long checkpointId = checkpointIDCounter.getLast(); @@ -1686,7 +1692,7 @@ public class CheckpointCoordinatorTest extends TestLogger { coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jobId, attemptID1, checkpointId), TASK_MANAGER_LOCATION_INFO); } - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); // After ACKs, all should be completed for (CompletableFuture savepointFuture : savepointFutures) { assertNotNull(savepointFuture.get()); @@ -1707,7 +1713,7 @@ public class CheckpointCoordinatorTest extends TestLogger { new CheckpointCoordinatorBuilder() .setCheckpointCoordinatorConfiguration(chkConfig) .setCompletedCheckpointStore(new StandaloneCompletedCheckpointStore(2)) - .setTimer(manuallyTriggeredScheduledExecutor) + .setMainThreadExecutor(mainThreadExecutor) .build(); String savepointDir = tmpFolder.newFolder().getAbsolutePath(); @@ -1735,12 +1741,12 @@ public class CheckpointCoordinatorTest extends TestLogger { CheckpointCoordinator coord = new CheckpointCoordinatorBuilder() .setCheckpointCoordinatorConfiguration(chkConfig) - .setTimer(manuallyTriggeredScheduledExecutor) + .setMainThreadExecutor(mainThreadExecutor) .build(); CompletableFuture checkpointFuture = coord.triggerCheckpoint(timestamp, false); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertFalse(checkpointFuture.isCompletedExceptionally()); for (PendingCheckpoint checkpoint : coord.getPendingCheckpoints().values()) { @@ -1956,7 +1962,7 @@ public class CheckpointCoordinatorTest extends TestLogger { // set up the coordinator and validate the initial state CheckpointCoordinator coord = new CheckpointCoordinatorBuilder() - .setTimer(manuallyTriggeredScheduledExecutor) + .setMainThreadExecutor(mainThreadExecutor) .build(); CheckpointStatsTracker tracker = mock(CheckpointStatsTracker.class); @@ -1968,7 +1974,7 @@ public class CheckpointCoordinatorTest extends TestLogger { // Trigger a checkpoint and verify callback CompletableFuture checkpointFuture = coord.triggerCheckpoint(timestamp, false); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertFalse(checkpointFuture.isCompletedExceptionally()); verify(tracker, times(1)) @@ -1986,7 +1992,7 @@ public class CheckpointCoordinatorTest extends TestLogger { CheckpointCoordinator coord = new CheckpointCoordinatorBuilder() .setCompletedCheckpointStore(store) - .setTimer(manuallyTriggeredScheduledExecutor) + .setMainThreadExecutor(mainThreadExecutor) .build(); store.addCheckpoint(new CompletedCheckpoint( @@ -2041,7 +2047,7 @@ public class CheckpointCoordinatorTest extends TestLogger { .setJobId(jid) .setTasks(arrayExecutionVertices) .setCompletedCheckpointStore(store) - .setTimer(manuallyTriggeredScheduledExecutor) + .setMainThreadExecutor(mainThreadExecutor) .setSharedStateRegistryFactory( deleteExecutor -> { SharedStateRegistry instance = new SharedStateRegistry(deleteExecutor); @@ -2203,7 +2209,7 @@ public class CheckpointCoordinatorTest extends TestLogger { final CompletableFuture savepointFuture = coordinator .triggerSynchronousSavepoint(10L, false, "test-dir"); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); final PendingCheckpoint syncSavepoint = declineSynchronousSavepoint(jobId, coordinator, attemptID1, expectedRootCause); assertTrue(syncSavepoint.isDiscarded()); @@ -2235,7 +2241,7 @@ public class CheckpointCoordinatorTest extends TestLogger { CheckpointCoordinator coord = new CheckpointCoordinatorBuilder() .setCheckpointIDCounter(idCounter) - .setTimer(manuallyTriggeredScheduledExecutor) + .setMainThreadExecutor(mainThreadExecutor) .build(); idCounter.setOwner(coord); @@ -2250,7 +2256,7 @@ public class CheckpointCoordinatorTest extends TestLogger { null, true, false); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); try { onCompletionPromise.get(); fail("should not trigger periodic checkpoint after stop the coordinator."); @@ -2274,7 +2280,7 @@ public class CheckpointCoordinatorTest extends TestLogger { return new CheckpointCoordinatorBuilder() .setJobId(jobId) .setTasks(new ExecutionVertex[]{ vertex1, vertex2 }) - .setTimer(manuallyTriggeredScheduledExecutor) + .setMainThreadExecutor(mainThreadExecutor) .build(); } @@ -2287,7 +2293,7 @@ public class CheckpointCoordinatorTest extends TestLogger { return new CheckpointCoordinatorBuilder() .setJobId(jobId) .setTasks(new ExecutionVertex[]{ vertex1, vertex2 }) - .setTimer(manuallyTriggeredScheduledExecutor) + .setMainThreadExecutor(mainThreadExecutor) .setFailureManager(failureManager) .build(); } @@ -2316,7 +2322,7 @@ public class CheckpointCoordinatorTest extends TestLogger { .setTasksToTrigger(new ExecutionVertex[] { triggerVertex1, triggerVertex2 }) .setTasksToWaitFor(new ExecutionVertex[] { ackVertex1, ackVertex2 }) .setTasksToCommitTo(new ExecutionVertex[] {}) - .setTimer(manuallyTriggeredScheduledExecutor) + .setMainThreadExecutor(mainThreadExecutor) .build(); } @@ -2342,7 +2348,7 @@ public class CheckpointCoordinatorTest extends TestLogger { // trigger the checkpoint coord.triggerCheckpoint(timestamp, false); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertEquals(1, coord.getPendingCheckpoints().size()); long checkpointId = Iterables.getOnlyElement(coord.getPendingCheckpoints().keySet()); @@ -2400,7 +2406,7 @@ public class CheckpointCoordinatorTest extends TestLogger { coord.receiveAcknowledgeMessage(acknowledgeCheckpoint, TASK_MANAGER_LOCATION_INFO); // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java index cf36f202ce..83c4212e8c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java @@ -24,6 +24,7 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.core.fs.FSDataInputStream; import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.flink.mock.Whitebox; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor; import org.apache.flink.runtime.concurrent.ScheduledExecutor; @@ -673,6 +674,9 @@ public class CheckpointCoordinatorTestingUtils { private ScheduledExecutor timer = new ManuallyTriggeredScheduledExecutor(); + private ScheduledExecutor mainThreadExecutor = + new ManuallyTriggeredScheduledExecutor(); + private SharedStateRegistryFactory sharedStateRegistryFactory = SharedStateRegistry.DEFAULT_FACTORY; @@ -751,6 +755,11 @@ public class CheckpointCoordinatorTestingUtils { return this; } + public CheckpointCoordinatorBuilder setMainThreadExecutor(ScheduledExecutor mainThreadExecutor) { + this.mainThreadExecutor = mainThreadExecutor; + return this; + } + public CheckpointCoordinatorBuilder setSharedStateRegistryFactory( SharedStateRegistryFactory sharedStateRegistryFactory) { this.sharedStateRegistryFactory = sharedStateRegistryFactory; @@ -764,7 +773,7 @@ public class CheckpointCoordinatorTestingUtils { } public CheckpointCoordinator build() { - return new CheckpointCoordinator( + final CheckpointCoordinator checkpointCoordinator = new CheckpointCoordinator( jobId, checkpointCoordinatorConfiguration, tasksToTrigger, @@ -778,6 +787,11 @@ public class CheckpointCoordinatorTestingUtils { timer, sharedStateRegistryFactory, failureManager); + checkpointCoordinator.start( + new ComponentMainThreadExecutorServiceAdapter( + mainThreadExecutor, + Thread.currentThread())); + return checkpointCoordinator; } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTriggeringTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTriggeringTest.java index 70fa6bfb32..52a4baa1af 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTriggeringTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTriggeringTest.java @@ -65,11 +65,13 @@ import static org.mockito.Mockito.doAnswer; public class CheckpointCoordinatorTriggeringTest extends TestLogger { private static final String TASK_MANAGER_LOCATION_INFO = "Unknown location"; - private ManuallyTriggeredScheduledExecutor manuallyTriggeredScheduledExecutor; + private ManuallyTriggeredScheduledExecutor timer; + private ManuallyTriggeredScheduledExecutor mainThreadExecutor; @Before public void setUp() throws Exception { - manuallyTriggeredScheduledExecutor = new ManuallyTriggeredScheduledExecutor(); + timer = new ManuallyTriggeredScheduledExecutor(); + mainThreadExecutor = new ManuallyTriggeredScheduledExecutor(); } @Test @@ -126,14 +128,15 @@ public class CheckpointCoordinatorTriggeringTest extends TestLogger { .setTasksToWaitFor(new ExecutionVertex[] { ackVertex }) .setTasksToCommitTo(new ExecutionVertex[] { commitVertex }) .setCompletedCheckpointStore(new StandaloneCompletedCheckpointStore(2)) - .setTimer(manuallyTriggeredScheduledExecutor) + .setTimer(timer) + .setMainThreadExecutor(mainThreadExecutor) .build(); checkpointCoordinator.startCheckpointScheduler(); do { - manuallyTriggeredScheduledExecutor.triggerPeriodicScheduledTasks(); - manuallyTriggeredScheduledExecutor.triggerAll(); + timer.triggerPeriodicScheduledTasks(); + mainThreadExecutor.triggerAll(); } while (numCalls.get() < 5); assertEquals(5, numCalls.get()); @@ -141,8 +144,8 @@ public class CheckpointCoordinatorTriggeringTest extends TestLogger { checkpointCoordinator.stopCheckpointScheduler(); // no further calls may come. - manuallyTriggeredScheduledExecutor.triggerPeriodicScheduledTasks(); - manuallyTriggeredScheduledExecutor.triggerAll(); + timer.triggerPeriodicScheduledTasks(); + mainThreadExecutor.triggerAll(); assertEquals(5, numCalls.get()); // start another sequence of periodic scheduling @@ -150,8 +153,8 @@ public class CheckpointCoordinatorTriggeringTest extends TestLogger { checkpointCoordinator.startCheckpointScheduler(); do { - manuallyTriggeredScheduledExecutor.triggerPeriodicScheduledTasks(); - manuallyTriggeredScheduledExecutor.triggerAll(); + timer.triggerPeriodicScheduledTasks(); + mainThreadExecutor.triggerAll(); } while (numCalls.get() < 5); assertEquals(5, numCalls.get()); @@ -159,8 +162,8 @@ public class CheckpointCoordinatorTriggeringTest extends TestLogger { checkpointCoordinator.stopCheckpointScheduler(); // no further calls may come - manuallyTriggeredScheduledExecutor.triggerPeriodicScheduledTasks(); - manuallyTriggeredScheduledExecutor.triggerAll(); + timer.triggerPeriodicScheduledTasks(); + mainThreadExecutor.triggerAll(); assertEquals(5, numCalls.get()); checkpointCoordinator.shutdown(JobStatus.FINISHED); @@ -207,13 +210,14 @@ public class CheckpointCoordinatorTriggeringTest extends TestLogger { .setCheckpointCoordinatorConfiguration(checkpointCoordinatorConfiguration) .setTasks(new ExecutionVertex[] { vertex }) .setCompletedCheckpointStore(new StandaloneCompletedCheckpointStore(2)) - .setTimer(manuallyTriggeredScheduledExecutor) + .setTimer(timer) + .setMainThreadExecutor(mainThreadExecutor) .build(); try { checkpointCoordinator.startCheckpointScheduler(); - manuallyTriggeredScheduledExecutor.triggerPeriodicScheduledTasks(); - manuallyTriggeredScheduledExecutor.triggerAll(); + timer.triggerPeriodicScheduledTasks(); + mainThreadExecutor.triggerAll(); // wait until the first checkpoint was triggered Long firstCallId = triggerCalls.take(); @@ -225,13 +229,13 @@ public class CheckpointCoordinatorTriggeringTest extends TestLogger { final long ackTime = System.nanoTime(); checkpointCoordinator.receiveAcknowledgeMessage(ackMsg, TASK_MANAGER_LOCATION_INFO); - manuallyTriggeredScheduledExecutor.triggerPeriodicScheduledTasks(); - manuallyTriggeredScheduledExecutor.triggerAll(); + timer.triggerPeriodicScheduledTasks(); + mainThreadExecutor.triggerAll(); while (triggerCalls.isEmpty()) { // sleeps for a while to simulate periodic scheduling Thread.sleep(checkpointInterval); - manuallyTriggeredScheduledExecutor.triggerPeriodicScheduledTasks(); - manuallyTriggeredScheduledExecutor.triggerAll(); + timer.triggerPeriodicScheduledTasks(); + mainThreadExecutor.triggerAll(); } // wait until the next checkpoint is triggered Long nextCallId = triggerCalls.take(); @@ -258,7 +262,7 @@ public class CheckpointCoordinatorTriggeringTest extends TestLogger { final CompletableFuture onCompletionPromise1 = triggerPeriodicCheckpoint(checkpointCoordinator); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); try { onCompletionPromise1.get(); fail("The triggerCheckpoint call expected an exception"); @@ -277,7 +281,7 @@ public class CheckpointCoordinatorTriggeringTest extends TestLogger { null, false, false); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertFalse(onCompletionPromise2.isCompletedExceptionally()); } @@ -290,7 +294,7 @@ checkpointCoordinator.startCheckpointScheduler(); triggerPeriodicCheckpoint(checkpointCoordinator); checkpointCoordinator.shutdown(JobStatus.FAILED); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); try { onCompletionPromise.get(); fail("Should not reach here"); @@ -332,7 +336,7 @@ checkpointCoordinator.startCheckpointScheduler(); assertTrue(checkpointCoordinator.isTriggering()); assertEquals(1, checkpointCoordinator.getTriggerRequestQueue().size()); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertFalse(onCompletionPromise1.isCompletedExceptionally()); assertFalse(onCompletionPromise2.isCompletedExceptionally()); assertFalse(checkpointCoordinator.isTriggering()); @@ -358,7 +362,7 @@ checkpointCoordinator.startCheckpointScheduler(); new CheckpointCoordinatorBuilder() .setTasks(new ExecutionVertex[] { vertex }) .setCheckpointIDCounter(new UnstableCheckpointIDCounter(id -> id == 0)) - .setTimer(manuallyTriggeredScheduledExecutor) + .setMainThreadExecutor(mainThreadExecutor) .build(); checkpointCoordinator.startCheckpointScheduler(); @@ -378,7 +382,7 @@ checkpointCoordinator.startCheckpointScheduler(); assertTrue(checkpointCoordinator.isTriggering()); assertEquals(2, checkpointCoordinator.getTriggerRequestQueue().size()); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); // the first triggered checkpoint fails by design through UnstableCheckpointIDCounter assertTrue(onCompletionPromise1.isCompletedExceptionally()); assertFalse(onCompletionPromise2.isCompletedExceptionally()); @@ -411,11 +415,11 @@ checkpointCoordinator.startCheckpointScheduler(); triggerPeriodicCheckpoint(checkpointCoordinator); // checkpoint trigger will not finish since master hook checkpoint is not finished yet - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertTrue(checkpointCoordinator.isTriggering()); // trigger cancellation - manuallyTriggeredScheduledExecutor.triggerNonPeriodicScheduledTasks(); + mainThreadExecutor.triggerNonPeriodicScheduledTasks(); assertTrue(checkpointCoordinator.isTriggering()); try { @@ -432,7 +436,7 @@ checkpointCoordinator.startCheckpointScheduler(); // continue triggering masterHookCheckpointFuture.complete("finish master hook"); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertFalse(checkpointCoordinator.isTriggering()); // it doesn't really trigger task manager to do checkpoint assertEquals(0, taskManagerCheckpointTriggeredTimes.get()); @@ -450,7 +454,7 @@ checkpointCoordinator.startCheckpointScheduler(); new CheckpointCoordinatorBuilder() .setTasks(new ExecutionVertex[] { vertex }) .setCheckpointIDCounter(new UnstableCheckpointIDCounter(id -> id == 0)) - .setTimer(manuallyTriggeredScheduledExecutor) + .setMainThreadExecutor(mainThreadExecutor) .build(); checkpointCoordinator.startCheckpointScheduler(); @@ -459,7 +463,7 @@ checkpointCoordinator.startCheckpointScheduler(); assertTrue(checkpointCoordinator.isTriggering()); assertEquals(0, checkpointCoordinator.getTriggerRequestQueue().size()); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); try { onCompletionPromise1.get(); fail("This checkpoint should fail through UnstableCheckpointIDCounter"); @@ -476,7 +480,7 @@ checkpointCoordinator.startCheckpointScheduler(); final CompletableFuture onCompletionPromise2 = triggerPeriodicCheckpoint(checkpointCoordinator); assertTrue(checkpointCoordinator.isTriggering()); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertFalse(onCompletionPromise2.isCompletedExceptionally()); assertFalse(checkpointCoordinator.isTriggering()); assertEquals(0, checkpointCoordinator.getTriggerRequestQueue().size()); @@ -505,13 +509,13 @@ checkpointCoordinator.startCheckpointScheduler(); triggerPeriodicCheckpoint(checkpointCoordinator); // checkpoint trigger will not finish since master hook checkpoint is not finished yet - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertTrue(checkpointCoordinator.isTriggering()); // continue triggering masterHookCheckpointFuture.completeExceptionally(new Exception("by design")); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertFalse(checkpointCoordinator.isTriggering()); try { @@ -531,14 +535,16 @@ checkpointCoordinator.startCheckpointScheduler(); private CheckpointCoordinator createCheckpointCoordinator() { return new CheckpointCoordinatorBuilder() - .setTimer(manuallyTriggeredScheduledExecutor) + .setTimer(timer) + .setMainThreadExecutor(mainThreadExecutor) .build(); } private CheckpointCoordinator createCheckpointCoordinator(ExecutionVertex executionVertex) { return new CheckpointCoordinatorBuilder() .setTasks(new ExecutionVertex[] { executionVertex }) - .setTimer(manuallyTriggeredScheduledExecutor) + .setTimer(timer) + .setMainThreadExecutor(mainThreadExecutor) .build(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java index 4bc1df7838..29fb4a2d5f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java @@ -98,7 +98,7 @@ public class CheckpointStateRestoreTest { tasks.add(stateful); tasks.add(stateless); - ManuallyTriggeredScheduledExecutor manuallyTriggeredScheduledExecutor = + ManuallyTriggeredScheduledExecutor mainThreadExecutor = new ManuallyTriggeredScheduledExecutor(); CheckpointCoordinator coord = @@ -107,13 +107,13 @@ public class CheckpointStateRestoreTest { .setTasksToTrigger(new ExecutionVertex[] { stateful1, stateful2, stateful3, stateless1, stateless2 }) .setTasksToWaitFor(new ExecutionVertex[] { stateful1, stateful2, stateful3, stateless1, stateless2 }) .setTasksToCommitTo(new ExecutionVertex[0]) - .setTimer(manuallyTriggeredScheduledExecutor) + .setMainThreadExecutor(mainThreadExecutor) .build(); // create ourselves a checkpoint with state final long timestamp = 34623786L; coord.triggerCheckpoint(timestamp, false); - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); PendingCheckpoint pending = coord.getPendingCheckpoints().values().iterator().next(); final long checkpointId = pending.getCheckpointId(); @@ -134,7 +134,7 @@ public class CheckpointStateRestoreTest { coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statelessExec1.getAttemptId(), checkpointId), TASK_MANAGER_LOCATION_INFO); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statelessExec2.getAttemptId(), checkpointId), TASK_MANAGER_LOCATION_INFO); // CheckpointCoordinator#completePendingCheckpoint is async, we have to finish the completion manually - manuallyTriggeredScheduledExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertEquals(1, coord.getNumberOfRetainedSuccessfulCheckpoints()); assertEquals(0, coord.getNumberOfPendingCheckpoints()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java index e28fa2c4a6..e9623be119 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java @@ -146,8 +146,6 @@ public class ExecutionGraphCheckpointCoordinatorTest extends TestLogger { .setAllocationTimeout(timeout) .build(); - executionGraph.start(ComponentMainThreadExecutorServiceAdapter.forMainThread()); - CheckpointCoordinatorConfiguration chkConfig = new CheckpointCoordinatorConfiguration( 100, 100, @@ -169,6 +167,8 @@ public class ExecutionGraphCheckpointCoordinatorTest extends TestLogger { new MemoryStateBackend(), CheckpointStatsTrackerTest.createTestTracker()); + executionGraph.start(ComponentMainThreadExecutorServiceAdapter.forMainThread()); + return executionGraph; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailoverStrategyCheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailoverStrategyCheckpointCoordinatorTest.java index 6d7b6cfb6d..379b68ca47 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailoverStrategyCheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailoverStrategyCheckpointCoordinatorTest.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor; import org.apache.flink.runtime.execution.ExecutionState; @@ -49,11 +50,15 @@ import static org.powermock.api.mockito.PowerMockito.when; * Tests for the interaction between the {@link FailoverStrategy} and the {@link CheckpointCoordinator}. */ public class FailoverStrategyCheckpointCoordinatorTest extends TestLogger { - private ManuallyTriggeredScheduledExecutor manualThreadExecutor; + + private ManuallyTriggeredScheduledExecutor timer; + + private ManuallyTriggeredScheduledExecutor mainThreadExecutor; @Before public void setUp() { - manualThreadExecutor = new ManuallyTriggeredScheduledExecutor(); + timer = new ManuallyTriggeredScheduledExecutor(); + mainThreadExecutor = new ManuallyTriggeredScheduledExecutor(); } /** @@ -85,24 +90,29 @@ public class FailoverStrategyCheckpointCoordinatorTest extends TestLogger { new StandaloneCompletedCheckpointStore(1), new MemoryStateBackend(), Executors.directExecutor(), - manualThreadExecutor, + timer, SharedStateRegistry.DEFAULT_FACTORY, mock(CheckpointFailureManager.class)); - // switch current execution's state to running to allow checkpoint could be triggered. + checkpointCoordinator.start( + new ComponentMainThreadExecutorServiceAdapter( + mainThreadExecutor, + Thread.currentThread())); + + // switch current execution's state to running to allow checkpoint could be triggered. mockExecutionRunning(executionVertex); checkpointCoordinator.startCheckpointScheduler(); assertTrue(checkpointCoordinator.isCurrentPeriodicTriggerAvailable()); // only trigger the periodic scheduling // we can't trigger all scheduled task, because there is also a cancellation scheduled - manualThreadExecutor.triggerPeriodicScheduledTasks(); - manualThreadExecutor.triggerAll(); + timer.triggerPeriodicScheduledTasks(); + mainThreadExecutor.triggerAll(); assertEquals(1, checkpointCoordinator.getNumberOfPendingCheckpoints()); for (int i = 1; i < maxConcurrentCheckpoints; i++) { checkpointCoordinator.triggerCheckpoint(System.currentTimeMillis(), false); - manualThreadExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertEquals(i + 1, checkpointCoordinator.getNumberOfPendingCheckpoints()); assertTrue(checkpointCoordinator.isCurrentPeriodicTriggerAvailable()); } @@ -110,7 +120,7 @@ public class FailoverStrategyCheckpointCoordinatorTest extends TestLogger { // as we only support limited concurrent checkpoints, after checkpoint triggered more than the limits, // the currentPeriodicTrigger would been assigned as null. checkpointCoordinator.triggerCheckpoint(System.currentTimeMillis(), false); - manualThreadExecutor.triggerAll(); + mainThreadExecutor.triggerAll(); assertFalse(checkpointCoordinator.isCurrentPeriodicTriggerAvailable()); assertEquals(maxConcurrentCheckpoints, checkpointCoordinator.getNumberOfPendingCheckpoints()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AdaptedRestartPipelinedRegionStrategyNGAbortPendingCheckpointsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AdaptedRestartPipelinedRegionStrategyNGAbortPendingCheckpointsTest.java index 3f75d90e3d..314b142e80 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AdaptedRestartPipelinedRegionStrategyNGAbortPendingCheckpointsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AdaptedRestartPipelinedRegionStrategyNGAbortPendingCheckpointsTest.java @@ -51,6 +51,7 @@ import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; import static org.apache.flink.util.Preconditions.checkState; import static org.hamcrest.Matchers.empty; @@ -94,8 +95,13 @@ public class AdaptedRestartPipelinedRegionStrategyNGAbortPendingCheckpointsTest final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator(); checkState(checkpointCoordinator != null); - checkpointCoordinator.triggerCheckpoint(System.currentTimeMillis(), false); - checkpointTriggeredLatch.await(); + checkpointCoordinator.triggerCheckpoint(System.currentTimeMillis(), false); + // there is an async call in IO thread + // here we need to retry to make sure the callback of the async call could be executed + manualMainThreadExecutor.triggerAll(); + while (!checkpointTriggeredLatch.await(10, TimeUnit.MILLISECONDS)) { + manualMainThreadExecutor.triggerAll(); + } assertEquals(1, checkpointCoordinator.getNumberOfPendingCheckpoints()); long checkpointId = checkpointCoordinator.getPendingCheckpoints().keySet().iterator().next(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java index a7f6ad940a..8e80683324 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java @@ -103,8 +103,6 @@ public class ArchivedExecutionGraphTest extends TestLogger { .setJobGraph(jobGraph) .build(); - runtimeGraph.start(ComponentMainThreadExecutorServiceAdapter.forMainThread()); - List jobVertices = new ArrayList<>(); jobVertices.add(runtimeGraph.getJobVertex(v1ID)); jobVertices.add(runtimeGraph.getJobVertex(v2ID)); @@ -138,6 +136,8 @@ public class ArchivedExecutionGraphTest extends TestLogger { runtimeGraph.setJsonPlan("{}"); + runtimeGraph.start(ComponentMainThreadExecutorServiceAdapter.forMainThread()); + runtimeGraph.getJobVertex(v2ID).getTaskVertices()[0].getCurrentExecutionAttempt().fail(new RuntimeException("This exception was thrown on purpose.")); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java index b4f39e9e47..2f20f40d43 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java @@ -72,6 +72,7 @@ import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executor; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -444,7 +445,11 @@ public class DefaultSchedulerTest extends TestLogger { final CountDownLatch checkpointTriggeredLatch = getCheckpointTriggeredLatch(); - final DefaultScheduler scheduler = createSchedulerAndStartScheduling(jobGraph); + // use a direct io executor for checkpoint relevant case + // otherwise the requirement of main thread executor could not be satisfied here + final DefaultScheduler scheduler = createSchedulerAndStartScheduling( + jobGraph, + org.apache.flink.runtime.concurrent.Executors.directExecutor()); final ArchivedExecutionVertex onlyExecutionVertex = Iterables.getOnlyElement(scheduler.requestJob().getAllExecutionVertices()); final ExecutionAttemptID attemptId = onlyExecutionVertex.getCurrentExecutionAttempt().getAttemptId(); @@ -468,7 +473,11 @@ public class DefaultSchedulerTest extends TestLogger { final CountDownLatch checkpointTriggeredLatch = getCheckpointTriggeredLatch(); - final DefaultScheduler scheduler = createSchedulerAndStartScheduling(jobGraph); + // use a direct io executor for checkpoint relevant case + // otherwise the requirement of main thread executor could not be satisfied here + final DefaultScheduler scheduler = createSchedulerAndStartScheduling( + jobGraph, + org.apache.flink.runtime.concurrent.Executors.directExecutor()); final ArchivedExecutionVertex onlyExecutionVertex = Iterables.getOnlyElement(scheduler.requestJob().getAllExecutionVertices()); final ExecutionAttemptID attemptId = onlyExecutionVertex.getCurrentExecutionAttempt().getAttemptId(); @@ -499,7 +508,11 @@ public class DefaultSchedulerTest extends TestLogger { final CountDownLatch checkpointTriggeredLatch = getCheckpointTriggeredLatch(); - final DefaultScheduler scheduler = createSchedulerAndStartScheduling(jobGraph); + // use a direct io executor for checkpoint relevant case + // otherwise the requirement of main thread executor could not be satisfied here + final DefaultScheduler scheduler = createSchedulerAndStartScheduling( + jobGraph, + org.apache.flink.runtime.concurrent.Executors.directExecutor()); final ArchivedExecutionVertex onlyExecutionVertex = Iterables.getOnlyElement(scheduler.requestJob().getAllExecutionVertices()); final ExecutionAttemptID attemptId = onlyExecutionVertex.getCurrentExecutionAttempt().getAttemptId(); @@ -513,7 +526,7 @@ public class DefaultSchedulerTest extends TestLogger { checkpointCoordinator.addMasterHook(masterHook); // complete one checkpoint for state restore - checkpointCoordinator.triggerCheckpoint(System.currentTimeMillis(), false); + checkpointCoordinator.triggerCheckpoint(System.currentTimeMillis(), false); checkpointTriggeredLatch.await(); final long checkpointId = checkpointCoordinator.getPendingCheckpoints().keySet().iterator().next(); acknowledgePendingCheckpoint(scheduler, checkpointId); @@ -754,13 +767,23 @@ public class DefaultSchedulerTest extends TestLogger { } private DefaultScheduler createSchedulerAndStartScheduling(final JobGraph jobGraph) { + return createSchedulerAndStartScheduling(jobGraph, executor); + } + + private DefaultScheduler createSchedulerAndStartScheduling( + final JobGraph jobGraph, + Executor ioExecutor) { + final SchedulingStrategyFactory schedulingStrategyFactory = jobGraph.getScheduleMode() == ScheduleMode.LAZY_FROM_SOURCES ? new LazyFromSourcesSchedulingStrategy.Factory() : new EagerSchedulingStrategy.Factory(); try { - final DefaultScheduler scheduler = createScheduler(jobGraph, schedulingStrategyFactory); + final DefaultScheduler scheduler = createScheduler( + jobGraph, + schedulingStrategyFactory, + ioExecutor != null ? ioExecutor : executor); startScheduling(scheduler); return scheduler; } catch (Exception e) { @@ -768,13 +791,21 @@ public class DefaultSchedulerTest extends TestLogger { } } + private DefaultScheduler createScheduler( + final JobGraph jobGraph, + final SchedulingStrategyFactory schedulingStrategyFactory) throws Exception { + + return createScheduler(jobGraph, schedulingStrategyFactory, executor); + } + private DefaultScheduler createScheduler( final JobGraph jobGraph, - final SchedulingStrategyFactory schedulingStrategyFactory) throws Exception { + final SchedulingStrategyFactory schedulingStrategyFactory, + final Executor ioExecutor) throws Exception { return SchedulerTestingUtils.newSchedulerBuilder(jobGraph) .setLogger(log) - .setIoExecutor(executor) + .setIoExecutor(ioExecutor) .setJobMasterConfiguration(configuration) .setFutureExecutor(scheduledExecutorService) .setDelayExecutor(taskRestartExecutor) -- Gitee From 300263e17a483a46c1bf4d2c0baee4a8ad31c26d Mon Sep 17 00:00:00 2001 From: ifndef-SleePy Date: Fri, 6 Mar 2020 00:12:35 +0800 Subject: [PATCH 246/885] [hotfix] Harden ResumeCheckpointManuallyITCase The way of detecting external checkpoint is done or not is not strict. The finalization of checkpoint might be interrupted by the cancallation. --- .../ResumeCheckpointManuallyITCase.java | 28 +++++++++++-------- 1 file changed, 16 insertions(+), 12 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java index ebd57a37a7..f9411372c7 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java @@ -29,6 +29,7 @@ import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.contrib.streaming.state.RocksDBStateBackend; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.flink.runtime.state.CheckpointListener; import org.apache.flink.runtime.state.StateBackend; import org.apache.flink.runtime.state.filesystem.FsStateBackend; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; @@ -297,13 +298,15 @@ public class ResumeCheckpointManuallyITCase extends TestLogger { private static String runJobAndGetExternalizedCheckpoint(StateBackend backend, File checkpointDir, @Nullable String externalCheckpoint, ClusterClient client) throws Exception { JobGraph initialJobGraph = getJobGraph(backend, externalCheckpoint); NotifyingInfiniteTupleSource.countDownLatch = new CountDownLatch(PARALLELISM); + NotifyingInfiniteTupleSource.checkpointCompletedLatch = new CountDownLatch(PARALLELISM); ClientUtils.submitJob(client, initialJobGraph); // wait until all sources have been started NotifyingInfiniteTupleSource.countDownLatch.await(); + // wait the checkpoint completing + NotifyingInfiniteTupleSource.checkpointCompletedLatch.await(); - waitUntilExternalizedCheckpointCreated(checkpointDir, initialJobGraph.getJobID()); client.cancel(initialJobGraph.getJobID()).get(); waitUntilCanceled(initialJobGraph.getJobID(), client); @@ -319,16 +322,6 @@ public class ResumeCheckpointManuallyITCase extends TestLogger { } } - private static void waitUntilExternalizedCheckpointCreated(File checkpointDir, JobID jobId) throws InterruptedException, IOException { - while (true) { - Thread.sleep(50); - Optional externalizedCheckpoint = findExternalizedCheckpoint(checkpointDir, jobId); - if (externalizedCheckpoint.isPresent()) { - break; - } - } - } - private static Optional findExternalizedCheckpoint(File checkpointDir, JobID jobId) throws IOException { try (Stream checkpoints = Files.list(checkpointDir.toPath().resolve(jobId.toString()))) { return checkpoints @@ -380,12 +373,16 @@ public class ResumeCheckpointManuallyITCase extends TestLogger { /** * Infinite source which notifies when all of its sub tasks have been started via the count down latch. */ - public static class NotifyingInfiniteTupleSource extends ManualWindowSpeedITCase.InfiniteTupleSource { + public static class NotifyingInfiniteTupleSource + extends ManualWindowSpeedITCase.InfiniteTupleSource + implements CheckpointListener { private static final long serialVersionUID = 8120981235081181746L; private static CountDownLatch countDownLatch; + private static CountDownLatch checkpointCompletedLatch; + public NotifyingInfiniteTupleSource(int numKeys) { super(numKeys); } @@ -398,5 +395,12 @@ public class ResumeCheckpointManuallyITCase extends TestLogger { super.run(out); } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + if (checkpointCompletedLatch != null) { + checkpointCompletedLatch.countDown(); + } + } } } -- Gitee From f248e6df7e79245091b569fa3afd107b78058fd9 Mon Sep 17 00:00:00 2001 From: ifndef-SleePy Date: Thu, 12 Mar 2020 02:16:49 +0800 Subject: [PATCH 247/885] [FLINK-14971][checkpointing] Make CompletedCheckpointStore thread-safe to avoid synchronization outside --- .../checkpoint/CheckpointCoordinator.java | 3 + .../checkpoint/CompletedCheckpointStore.java | 4 + .../StandaloneCompletedCheckpointStore.java | 51 +++-- .../ZooKeeperCompletedCheckpointStore.java | 195 ++++++++++-------- ...oKeeperCompletedCheckpointStoreITCase.java | 2 + 5 files changed, 153 insertions(+), 102 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java index 35cace779b..fa18a14342 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java @@ -1370,6 +1370,7 @@ public class CheckpointCoordinator { return this.pendingCheckpoints.size(); } + @VisibleForTesting public int getNumberOfRetainedSuccessfulCheckpoints() { synchronized (lock) { return completedCheckpointStore.getNumberOfRetainedCheckpoints(); @@ -1382,6 +1383,7 @@ public class CheckpointCoordinator { } } + @VisibleForTesting public List getSuccessfulCheckpoints() throws Exception { synchronized (lock) { return completedCheckpointStore.getAllCheckpoints(); @@ -1392,6 +1394,7 @@ public class CheckpointCoordinator { return checkpointStorage; } + @VisibleForTesting public CompletedCheckpointStore getCheckpointStore() { return completedCheckpointStore; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStore.java index 42f608b823..80ca9fbfb0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStore.java @@ -23,12 +23,16 @@ import org.apache.flink.api.common.JobStatus; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.concurrent.ThreadSafe; + import java.util.List; import java.util.ListIterator; /** * A bounded LIFO-queue of {@link CompletedCheckpoint} instances. + * Note that it might be visited by multiple threads. So implementation should keep it thread-safe. */ +@ThreadSafe public interface CompletedCheckpointStore { Logger LOG = LoggerFactory.getLogger(CompletedCheckpointStore.class); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStore.java index ddaf9095f8..4cc9931ee3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStore.java @@ -24,15 +24,19 @@ import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.concurrent.ThreadSafe; + import java.util.ArrayDeque; import java.util.ArrayList; import java.util.List; import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkState; /** * {@link CompletedCheckpointStore} for JobManagers running in {@link HighAvailabilityMode#NONE}. */ +@ThreadSafe public class StandaloneCompletedCheckpointStore implements CompletedCheckpointStore { private static final Logger LOG = LoggerFactory.getLogger(StandaloneCompletedCheckpointStore.class); @@ -43,6 +47,8 @@ public class StandaloneCompletedCheckpointStore implements CompletedCheckpointSt /** The completed checkpoints. */ private final ArrayDeque checkpoints; + private boolean shutdown = false; + /** * Creates {@link StandaloneCompletedCheckpointStore}. * @@ -64,26 +70,34 @@ public class StandaloneCompletedCheckpointStore implements CompletedCheckpointSt @Override public void addCheckpoint(CompletedCheckpoint checkpoint) throws Exception { - checkpoints.addLast(checkpoint); + synchronized (checkpoints) { + checkState(!shutdown, "StandaloneCompletedCheckpointStore has been shut down"); + + checkpoints.addLast(checkpoint); - if (checkpoints.size() > maxNumberOfCheckpointsToRetain) { - try { - CompletedCheckpoint checkpointToSubsume = checkpoints.removeFirst(); - checkpointToSubsume.discardOnSubsume(); - } catch (Exception e) { - LOG.warn("Fail to subsume the old checkpoint.", e); + if (checkpoints.size() > maxNumberOfCheckpointsToRetain) { + try { + CompletedCheckpoint checkpointToSubsume = checkpoints.removeFirst(); + checkpointToSubsume.discardOnSubsume(); + } catch (Exception e) { + LOG.warn("Fail to subsume the old checkpoint.", e); + } } } } @Override public List getAllCheckpoints() { - return new ArrayList<>(checkpoints); + synchronized (checkpoints) { + return new ArrayList<>(checkpoints); + } } @Override public int getNumberOfRetainedCheckpoints() { - return checkpoints.size(); + synchronized (checkpoints) { + return checkpoints.size(); + } } @Override @@ -93,14 +107,19 @@ public class StandaloneCompletedCheckpointStore implements CompletedCheckpointSt @Override public void shutdown(JobStatus jobStatus) throws Exception { - try { - LOG.info("Shutting down"); - - for (CompletedCheckpoint checkpoint : checkpoints) { - checkpoint.discardOnShutdown(jobStatus); + synchronized (checkpoints) { + if (!shutdown) { + shutdown = true; + try { + LOG.info("Shutting down"); + + for (CompletedCheckpoint checkpoint : checkpoints) { + checkpoint.discardOnShutdown(jobStatus); + } + } finally { + checkpoints.clear(); + } } - } finally { - checkpoints.clear(); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java index b4f72b7436..80b6d500ce 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java @@ -29,6 +29,8 @@ import org.apache.flink.util.function.ThrowingConsumer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.concurrent.ThreadSafe; + import java.io.IOException; import java.util.ArrayDeque; import java.util.ArrayList; @@ -40,6 +42,7 @@ import java.util.concurrent.Executor; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; /** * {@link CompletedCheckpointStore} for JobManagers running in {@link HighAvailabilityMode#ZOOKEEPER}. @@ -64,6 +67,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull; * checkpoints is consistent. Currently, after recovery we start out with only a single * checkpoint to circumvent those situations. */ +@ThreadSafe public class ZooKeeperCompletedCheckpointStore implements CompletedCheckpointStore { private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperCompletedCheckpointStore.class); @@ -85,6 +89,10 @@ public class ZooKeeperCompletedCheckpointStore implements CompletedCheckpointSto private final Executor executor; + private final Object lock = new Object(); + + private boolean shutdown = false; + /** * Creates a {@link ZooKeeperCompletedCheckpointStore} instance. * @@ -127,77 +135,79 @@ public class ZooKeeperCompletedCheckpointStore implements CompletedCheckpointSto public void recover() throws Exception { LOG.info("Recovering checkpoints from ZooKeeper."); - // Get all there is first - List, String>> initialCheckpoints; - while (true) { - try { - initialCheckpoints = checkpointsInZooKeeper.getAllAndLock(); - break; - } - catch (ConcurrentModificationException e) { - LOG.warn("Concurrent modification while reading from ZooKeeper. Retrying."); + synchronized (lock) { + checkState(!shutdown, "ZooKeeperCompletedCheckpointStore has been shut down"); + // Get all there is first + List, String>> initialCheckpoints; + while (true) { + try { + initialCheckpoints = checkpointsInZooKeeper.getAllAndLock(); + break; + } catch (ConcurrentModificationException e) { + LOG.warn("Concurrent modification while reading from ZooKeeper. Retrying."); + } } - } - Collections.sort(initialCheckpoints, STRING_COMPARATOR); + Collections.sort(initialCheckpoints, STRING_COMPARATOR); - int numberOfInitialCheckpoints = initialCheckpoints.size(); + int numberOfInitialCheckpoints = initialCheckpoints.size(); - LOG.info("Found {} checkpoints in ZooKeeper.", numberOfInitialCheckpoints); + LOG.info("Found {} checkpoints in ZooKeeper.", numberOfInitialCheckpoints); - // Try and read the state handles from storage. We try until we either successfully read - // all of them or when we reach a stable state, i.e. when we successfully read the same set - // of checkpoints in two tries. We do it like this to protect against transient outages - // of the checkpoint store (for example a DFS): if the DFS comes online midway through - // reading a set of checkpoints we would run the risk of reading only a partial set - // of checkpoints while we could in fact read the other checkpoints as well if we retried. - // Waiting until a stable state protects against this while also being resilient against - // checkpoints being actually unreadable. - // - // These considerations are also important in the scope of incremental checkpoints, where - // we use ref-counting for shared state handles and might accidentally delete shared state - // of checkpoints that we don't read due to transient storage outages. - List lastTryRetrievedCheckpoints = new ArrayList<>(numberOfInitialCheckpoints); - List retrievedCheckpoints = new ArrayList<>(numberOfInitialCheckpoints); - do { - LOG.info("Trying to fetch {} checkpoints from storage.", numberOfInitialCheckpoints); + // Try and read the state handles from storage. We try until we either successfully read + // all of them or when we reach a stable state, i.e. when we successfully read the same set + // of checkpoints in two tries. We do it like this to protect against transient outages + // of the checkpoint store (for example a DFS): if the DFS comes online midway through + // reading a set of checkpoints we would run the risk of reading only a partial set + // of checkpoints while we could in fact read the other checkpoints as well if we retried. + // Waiting until a stable state protects against this while also being resilient against + // checkpoints being actually unreadable. + // + // These considerations are also important in the scope of incremental checkpoints, where + // we use ref-counting for shared state handles and might accidentally delete shared state + // of checkpoints that we don't read due to transient storage outages. + List lastTryRetrievedCheckpoints = new ArrayList<>(numberOfInitialCheckpoints); + List retrievedCheckpoints = new ArrayList<>(numberOfInitialCheckpoints); + do { + LOG.info("Trying to fetch {} checkpoints from storage.", numberOfInitialCheckpoints); - lastTryRetrievedCheckpoints.clear(); - lastTryRetrievedCheckpoints.addAll(retrievedCheckpoints); + lastTryRetrievedCheckpoints.clear(); + lastTryRetrievedCheckpoints.addAll(retrievedCheckpoints); - retrievedCheckpoints.clear(); + retrievedCheckpoints.clear(); - for (Tuple2, String> checkpointStateHandle : initialCheckpoints) { + for (Tuple2, String> checkpointStateHandle : initialCheckpoints) { - CompletedCheckpoint completedCheckpoint; + CompletedCheckpoint completedCheckpoint; - try { - completedCheckpoint = retrieveCompletedCheckpoint(checkpointStateHandle); - if (completedCheckpoint != null) { - retrievedCheckpoints.add(completedCheckpoint); + try { + completedCheckpoint = retrieveCompletedCheckpoint(checkpointStateHandle); + if (completedCheckpoint != null) { + retrievedCheckpoints.add(completedCheckpoint); + } + } catch (Exception e) { + LOG.warn("Could not retrieve checkpoint, not adding to list of recovered checkpoints.", e); } - } catch (Exception e) { - LOG.warn("Could not retrieve checkpoint, not adding to list of recovered checkpoints.", e); } - } - } while (retrievedCheckpoints.size() != numberOfInitialCheckpoints && - !CompletedCheckpoint.checkpointsMatch(lastTryRetrievedCheckpoints, retrievedCheckpoints)); - - // Clear local handles in order to prevent duplicates on - // recovery. The local handles should reflect the state - // of ZooKeeper. - completedCheckpoints.clear(); - completedCheckpoints.addAll(retrievedCheckpoints); - - if (completedCheckpoints.isEmpty() && numberOfInitialCheckpoints > 0) { - throw new FlinkException( - "Could not read any of the " + numberOfInitialCheckpoints + " checkpoints from storage."); - } else if (completedCheckpoints.size() != numberOfInitialCheckpoints) { - LOG.warn( - "Could only fetch {} of {} checkpoints from storage.", - completedCheckpoints.size(), - numberOfInitialCheckpoints); + } while (retrievedCheckpoints.size() != numberOfInitialCheckpoints && + !CompletedCheckpoint.checkpointsMatch(lastTryRetrievedCheckpoints, retrievedCheckpoints)); + + // Clear local handles in order to prevent duplicates on + // recovery. The local handles should reflect the state + // of ZooKeeper. + completedCheckpoints.clear(); + completedCheckpoints.addAll(retrievedCheckpoints); + + if (completedCheckpoints.isEmpty() && numberOfInitialCheckpoints > 0) { + throw new FlinkException( + "Could not read any of the " + numberOfInitialCheckpoints + " checkpoints from storage."); + } else if (completedCheckpoints.size() != numberOfInitialCheckpoints) { + LOG.warn( + "Could only fetch {} of {} checkpoints from storage.", + completedCheckpoints.size(), + numberOfInitialCheckpoints); + } } } @@ -210,20 +220,24 @@ public class ZooKeeperCompletedCheckpointStore implements CompletedCheckpointSto public void addCheckpoint(final CompletedCheckpoint checkpoint) throws Exception { checkNotNull(checkpoint, "Checkpoint"); - final String path = checkpointIdToPath(checkpoint.getCheckpointID()); + synchronized (lock) { + checkState(!shutdown, "ZooKeeperCompletedCheckpointStore has been shut down"); - // Now add the new one. If it fails, we don't want to loose existing data. - checkpointsInZooKeeper.addAndLock(path, checkpoint); + final String path = checkpointIdToPath(checkpoint.getCheckpointID()); - completedCheckpoints.addLast(checkpoint); + // Now add the new one. If it fails, we don't want to loose existing data. + checkpointsInZooKeeper.addAndLock(path, checkpoint); - // Everything worked, let's remove a previous checkpoint if necessary. - while (completedCheckpoints.size() > maxNumberOfCheckpointsToRetain) { - final CompletedCheckpoint completedCheckpoint = completedCheckpoints.removeFirst(); - tryRemoveCompletedCheckpoint(completedCheckpoint, CompletedCheckpoint::discardOnSubsume); - } + completedCheckpoints.addLast(checkpoint); + + // Everything worked, let's remove a previous checkpoint if necessary. + while (completedCheckpoints.size() > maxNumberOfCheckpointsToRetain) { + final CompletedCheckpoint completedCheckpoint = completedCheckpoints.removeFirst(); + tryRemoveCompletedCheckpoint(completedCheckpoint, CompletedCheckpoint::discardOnSubsume); + } - LOG.debug("Added {} to {}.", checkpoint, path); + LOG.debug("Added {} to {}.", checkpoint, path); + } } private void tryRemoveCompletedCheckpoint(CompletedCheckpoint completedCheckpoint, ThrowingConsumer discardCallback) { @@ -245,12 +259,16 @@ public class ZooKeeperCompletedCheckpointStore implements CompletedCheckpointSto @Override public List getAllCheckpoints() throws Exception { - return new ArrayList<>(completedCheckpoints); + synchronized (lock) { + return new ArrayList<>(completedCheckpoints); + } } @Override public int getNumberOfRetainedCheckpoints() { - return completedCheckpoints.size(); + synchronized (lock) { + return completedCheckpoints.size(); + } } @Override @@ -260,25 +278,30 @@ public class ZooKeeperCompletedCheckpointStore implements CompletedCheckpointSto @Override public void shutdown(JobStatus jobStatus) throws Exception { - if (jobStatus.isGloballyTerminalState()) { - LOG.info("Shutting down"); - - for (CompletedCheckpoint checkpoint : completedCheckpoints) { - tryRemoveCompletedCheckpoint( - checkpoint, - completedCheckpoint -> completedCheckpoint.discardOnShutdown(jobStatus)); - } + synchronized (lock) { + if (!shutdown) { + shutdown = true; + if (jobStatus.isGloballyTerminalState()) { + LOG.info("Shutting down"); + + for (CompletedCheckpoint checkpoint : completedCheckpoints) { + tryRemoveCompletedCheckpoint( + checkpoint, + completedCheckpoint -> completedCheckpoint.discardOnShutdown(jobStatus)); + } - completedCheckpoints.clear(); - checkpointsInZooKeeper.deleteChildren(); - } else { - LOG.info("Suspending"); + completedCheckpoints.clear(); + checkpointsInZooKeeper.deleteChildren(); + } else { + LOG.info("Suspending"); - // Clear the local handles, but don't remove any state - completedCheckpoints.clear(); + // Clear the local handles, but don't remove any state + completedCheckpoints.clear(); - // Release the state handle locks in ZooKeeper such that they can be deleted - checkpointsInZooKeeper.releaseAll(); + // Release the state handle locks in ZooKeeper such that they can be deleted + checkpointsInZooKeeper.releaseAll(); + } + } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java index 859f1b9408..35465c2a0c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java @@ -158,6 +158,7 @@ public class ZooKeeperCompletedCheckpointStoreITCase extends CompletedCheckpoint assertNull(client.checkExists().forPath(CHECKPOINT_PATH + ZooKeeperCompletedCheckpointStore.checkpointIdToPath(checkpoint.getCheckpointID()))); sharedStateRegistry.close(); + store = createCompletedCheckpoints(1); store.recover(); assertEquals(0, store.getNumberOfRetainedCheckpoints()); @@ -192,6 +193,7 @@ public class ZooKeeperCompletedCheckpointStoreITCase extends CompletedCheckpoint // Recover again sharedStateRegistry.close(); + store = createCompletedCheckpoints(1); store.recover(); CompletedCheckpoint recovered = store.getLatestCheckpoint(false); -- Gitee From d8267be27ff397aaa3f0b7c2860f4bf8885fcd4b Mon Sep 17 00:00:00 2001 From: ifndef-SleePy Date: Thu, 5 Mar 2020 02:51:32 +0800 Subject: [PATCH 248/885] [FLINK-14971][checkpointing] Remove coordinator-wide lock of CheckpointCoordinator Since all non-IO operations are executed in main thread, the coordinator-wide lock could be avoided now. --- .../checkpoint/CheckpointCoordinator.java | 599 ++++++++---------- 1 file changed, 272 insertions(+), 327 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java index fa18a14342..6901506e30 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java @@ -93,9 +93,6 @@ public class CheckpointCoordinator { // ------------------------------------------------------------------------ - /** Coordinator-wide lock to safeguard the checkpoint updates. */ - private final Object lock = new Object(); - /** The job whose checkpoint this coordinator coordinates. */ private final JobID job; @@ -176,15 +173,16 @@ public class CheckpointCoordinator { private long lastCheckpointCompletionRelativeTime; /** Flag whether a triggered checkpoint should immediately schedule the next checkpoint. - * Non-volatile, because only accessed in synchronized scope */ + * Non-volatile, because only accessed in main thread */ private boolean periodicScheduling; /** Flag whether periodic triggering is suspended (too many concurrent pending checkpoint). - * Non-volatile, because only accessed in synchronized scope */ + * Non-volatile, because only accessed in main thread */ private boolean periodicTriggeringSuspended; - /** Flag marking the coordinator as shut down (not accepting any messages any more). */ - private volatile boolean shutdown; + /** Flag marking the coordinator as shut down (not accepting any messages any more). + * Non-volatile, because only accessed in main thread */ + private boolean shutdown; /** Optional tracker for checkpoint statistics. */ @Nullable @@ -357,14 +355,12 @@ public class CheckpointCoordinator { final String id = hook.getIdentifier(); checkArgument(!StringUtils.isNullOrWhitespaceOnly(id), "The hook has a null or empty id"); - synchronized (lock) { - if (!masterHooks.containsKey(id)) { - masterHooks.put(id, hook); - return true; - } - else { - return false; - } + if (!masterHooks.containsKey(id)) { + masterHooks.put(id, hook); + return true; + } + else { + return false; } } @@ -372,9 +368,7 @@ public class CheckpointCoordinator { * Gets the number of currently register master hooks. */ public int getNumberOfRegisteredMasterHooks() { - synchronized (lock) { - return masterHooks.size(); - } + return masterHooks.size(); } /** @@ -397,27 +391,26 @@ public class CheckpointCoordinator { * and further messages and cannot trigger any further checkpoints. */ public void shutdown(JobStatus jobStatus) throws Exception { - synchronized (lock) { - assertRunningMainThread(); + assertRunningMainThread(); + + if (!shutdown) { + shutdown = true; + LOG.info("Stopping checkpoint coordinator for job {}.", job); - if (!shutdown) { - shutdown = true; - LOG.info("Stopping checkpoint coordinator for job {}.", job); - periodicScheduling = false; - periodicTriggeringSuspended = false; + periodicScheduling = false; + periodicTriggeringSuspended = false; - // shut down the hooks - MasterHooks.close(masterHooks.values(), LOG); - masterHooks.clear(); + // shut down the hooks + MasterHooks.close(masterHooks.values(), LOG); + masterHooks.clear(); - final CheckpointException reason = new CheckpointException( - CheckpointFailureReason.CHECKPOINT_COORDINATOR_SHUTDOWN); - // clear queued requests and in-flight checkpoints - abortPendingAndQueuedCheckpoints(reason); + final CheckpointException reason = new CheckpointException( + CheckpointFailureReason.CHECKPOINT_COORDINATOR_SHUTDOWN); + // clear queued requests and in-flight checkpoints + abortPendingAndQueuedCheckpoints(reason); - completedCheckpointStore.shutdown(jobStatus); - checkpointIdCounter.shutdown(jobStatus); - } + completedCheckpointStore.shutdown(jobStatus); + checkpointIdCounter.shutdown(jobStatus); } } @@ -528,19 +521,17 @@ public class CheckpointCoordinator { final CompletableFuture onCompletionPromise = new CompletableFuture<>(); - synchronized (lock) { - if (isTriggering || !triggerRequestQueue.isEmpty()) { - // we can't trigger checkpoint directly if there is a trigger request being processed - // or queued - triggerRequestQueue.add(new CheckpointTriggerRequest( - timestamp, - props, - externalSavepointLocation, - isPeriodic, - advanceToEndOfTime, - onCompletionPromise)); - return onCompletionPromise; - } + if (isTriggering || !triggerRequestQueue.isEmpty()) { + // we can't trigger checkpoint directly if there is a trigger request being processed + // or queued + triggerRequestQueue.add(new CheckpointTriggerRequest( + timestamp, + props, + externalSavepointLocation, + isPeriodic, + advanceToEndOfTime, + onCompletionPromise)); + return onCompletionPromise; } startTriggeringCheckpoint( timestamp, @@ -562,9 +553,7 @@ public class CheckpointCoordinator { try { // make some eager pre-checks - synchronized (lock) { - preCheckBeforeTriggeringCheckpoint(isPeriodic, props.forceCheckpoint()); - } + preCheckBeforeTriggeringCheckpoint(isPeriodic, props.forceCheckpoint()); final Execution[] executions = getTriggerExecutions(); final Map ackTasks = getAckTasks(); @@ -612,12 +601,12 @@ public class CheckpointCoordinator { advanceToEndOfTime); onTriggerSuccess(); } else { - // the initialization might not be finished yet - if (checkpoint == null) { - onTriggerFailure(onCompletionPromise, throwable); - } else { - onTriggerFailure(checkpoint, throwable); - } + // the initialization might not be finished yet + if (checkpoint == null) { + onTriggerFailure(onCompletionPromise, throwable); + } else { + onTriggerFailure(checkpoint, throwable); + } } }, mainThreadExecutor); @@ -667,14 +656,12 @@ public class CheckpointCoordinator { assertRunningMainThread(); - synchronized (lock) { - try { - // since we haven't created the PendingCheckpoint yet, we need to check the - // global state here. - preCheckGlobalState(isPeriodic); - } catch (Throwable t) { - throw new CompletionException(t); - } + try { + // since we haven't created the PendingCheckpoint yet, we need to check the + // global state here. + preCheckGlobalState(isPeriodic); + } catch (Throwable t) { + throw new CompletionException(t); } final PendingCheckpoint checkpoint = new PendingCheckpoint( @@ -700,17 +687,15 @@ public class CheckpointCoordinator { checkpoint.setStatsCallback(callback); } - synchronized (lock) { - pendingCheckpoints.put(checkpointID, checkpoint); + pendingCheckpoints.put(checkpointID, checkpoint); - ScheduledFuture cancellerHandle = mainThreadExecutor.schedule( - new CheckpointCanceller(checkpoint), - checkpointTimeout, TimeUnit.MILLISECONDS); + ScheduledFuture cancellerHandle = mainThreadExecutor.schedule( + new CheckpointCanceller(checkpoint), + checkpointTimeout, TimeUnit.MILLISECONDS); - if (!checkpoint.setCancellerHandle(cancellerHandle)) { - // checkpoint is already disposed! - cancellerHandle.cancel(false); - } + if (!checkpoint.setCancellerHandle(cancellerHandle)) { + // checkpoint is already disposed! + cancellerHandle.cancel(false); } LOG.info("Triggering checkpoint {} @ {} for job {}.", checkpointID, timestamp, job); @@ -738,23 +723,21 @@ public class CheckpointCoordinator { .whenCompleteAsync( (masterState, throwable) -> { try { - synchronized (lock) { - if (masterStateCompletableFuture.isDone()) { - return; - } - if (checkpoint.isDiscarded()) { - throw new IllegalStateException( - "Checkpoint " + checkpointID + " has been discarded"); - } - if (throwable == null) { - checkpoint.acknowledgeMasterState( - masterHook.getIdentifier(), masterState); - if (checkpoint.areMasterStatesFullyAcknowledged()) { - masterStateCompletableFuture.complete(null); - } - } else { - masterStateCompletableFuture.completeExceptionally(throwable); + if (masterStateCompletableFuture.isDone()) { + return; + } + if (checkpoint.isDiscarded()) { + throw new IllegalStateException( + "Checkpoint " + checkpointID + " has been discarded"); + } + if (throwable == null) { + checkpoint.acknowledgeMasterState( + masterHook.getIdentifier(), masterState); + if (checkpoint.areMasterStatesFullyAcknowledged()) { + masterStateCompletableFuture.complete(null); } + } else { + masterStateCompletableFuture.completeExceptionally(throwable); } } catch (Throwable t) { masterStateCompletableFuture.completeExceptionally(t); @@ -844,9 +827,7 @@ public class CheckpointCoordinator { final CheckpointException cause = getCheckpointException( CheckpointFailureReason.TRIGGER_CHECKPOINT_FAILURE, throwable); - synchronized (lock) { - abortPendingCheckpoint(checkpoint, cause); - } + abortPendingCheckpoint(checkpoint, cause); } } finally { isTriggering = false; @@ -859,15 +840,10 @@ public class CheckpointCoordinator { * NOTE: this must be called after each triggering */ private void checkQueuedCheckpointTriggerRequest() { - synchronized (lock) { - if (triggerRequestQueue.isEmpty()) { - return; - } - } - final CheckpointTriggerRequest request; - synchronized (lock) { - request = triggerRequestQueue.poll(); + if (triggerRequestQueue.isEmpty()) { + return; } + final CheckpointTriggerRequest request = triggerRequestQueue.poll(); if (request != null) { startTriggeringCheckpoint( request.timestamp, @@ -904,48 +880,44 @@ public class CheckpointCoordinator { final long checkpointId = message.getCheckpointId(); final String reason = (message.getReason() != null ? message.getReason().getMessage() : ""); - PendingCheckpoint checkpoint; + // we need to check inside the lock for being shutdown as well, otherwise we + // get races and invalid error log messages + if (shutdown) { + return; + } - synchronized (lock) { - // we need to check inside the lock for being shutdown as well, otherwise we - // get races and invalid error log messages - if (shutdown) { - return; + PendingCheckpoint checkpoint = pendingCheckpoints.get(checkpointId); + + if (checkpoint != null) { + Preconditions.checkState( + !checkpoint.isDiscarded(), + "Received message for discarded but non-removed checkpoint " + checkpointId); + LOG.info("Decline checkpoint {} by task {} of job {} at {}.", + checkpointId, + message.getTaskExecutionId(), + job, + taskManagerLocationInfo); + final CheckpointException checkpointException; + if (message.getReason() == null) { + checkpointException = + new CheckpointException(CheckpointFailureReason.CHECKPOINT_DECLINED); + } else { + checkpointException = getCheckpointException( + CheckpointFailureReason.JOB_FAILURE, message.getReason()); } - - checkpoint = pendingCheckpoints.get(checkpointId); - - if (checkpoint != null) { - Preconditions.checkState( - !checkpoint.isDiscarded(), - "Received message for discarded but non-removed checkpoint " + checkpointId); - LOG.info("Decline checkpoint {} by task {} of job {} at {}.", - checkpointId, - message.getTaskExecutionId(), - job, - taskManagerLocationInfo); - final CheckpointException checkpointException; - if (message.getReason() == null) { - checkpointException = - new CheckpointException(CheckpointFailureReason.CHECKPOINT_DECLINED); - } else { - checkpointException = getCheckpointException( - CheckpointFailureReason.JOB_FAILURE, message.getReason()); - } - abortPendingCheckpoint( - checkpoint, - checkpointException, - message.getTaskExecutionId()); - } else if (LOG.isDebugEnabled()) { - if (recentPendingCheckpoints.contains(checkpointId)) { - // message is for an unknown checkpoint, or comes too late (checkpoint disposed) - LOG.debug("Received another decline message for now expired checkpoint attempt {} from task {} of job {} at {} : {}", - checkpointId, message.getTaskExecutionId(), job, taskManagerLocationInfo, reason); - } else { - // message is for an unknown checkpoint. might be so old that we don't even remember it any more - LOG.debug("Received decline message for unknown (too old?) checkpoint attempt {} from task {} of job {} at {} : {}", - checkpointId, message.getTaskExecutionId(), job, taskManagerLocationInfo, reason); - } + abortPendingCheckpoint( + checkpoint, + checkpointException, + message.getTaskExecutionId()); + } else if (LOG.isDebugEnabled()) { + if (recentPendingCheckpoints.contains(checkpointId)) { + // message is for an unknown checkpoint, or comes too late (checkpoint disposed) + LOG.debug("Received another decline message for now expired checkpoint attempt {} from task {} of job {} at {} : {}", + checkpointId, message.getTaskExecutionId(), job, taskManagerLocationInfo, reason); + } else { + // message is for an unknown checkpoint. might be so old that we don't even remember it any more + LOG.debug("Received decline message for unknown (too old?) checkpoint attempt {} from task {} of job {} at {} : {}", + checkpointId, message.getTaskExecutionId(), job, taskManagerLocationInfo, reason); } } } @@ -976,75 +948,73 @@ public class CheckpointCoordinator { final long checkpointId = message.getCheckpointId(); - synchronized (lock) { - // we need to check inside the lock for being shutdown as well, otherwise we - // get races and invalid error log messages - if (shutdown) { - return false; - } - - final PendingCheckpoint checkpoint = pendingCheckpoints.get(checkpointId); + // we need to check inside the lock for being shutdown as well, otherwise we + // get races and invalid error log messages + if (shutdown) { + return false; + } - if (checkpoint != null && !checkpoint.isDiscarded()) { + final PendingCheckpoint checkpoint = pendingCheckpoints.get(checkpointId); - switch (checkpoint.acknowledgeTask(message.getTaskExecutionId(), message.getSubtaskState(), message.getCheckpointMetrics())) { - case SUCCESS: - LOG.debug("Received acknowledge message for checkpoint {} from task {} of job {} at {}.", - checkpointId, message.getTaskExecutionId(), message.getJob(), taskManagerLocationInfo); + if (checkpoint != null && !checkpoint.isDiscarded()) { - if (checkpoint.areTasksFullyAcknowledged()) { - completePendingCheckpoint(checkpoint); - } - break; - case DUPLICATE: - LOG.debug("Received a duplicate acknowledge message for checkpoint {}, task {}, job {}, location {}.", - message.getCheckpointId(), message.getTaskExecutionId(), message.getJob(), taskManagerLocationInfo); - break; - case UNKNOWN: - LOG.warn("Could not acknowledge the checkpoint {} for task {} of job {} at {}, " + - "because the task's execution attempt id was unknown. Discarding " + - "the state handle to avoid lingering state.", message.getCheckpointId(), - message.getTaskExecutionId(), message.getJob(), taskManagerLocationInfo); - - discardSubtaskState(message.getJob(), message.getTaskExecutionId(), message.getCheckpointId(), message.getSubtaskState()); - - break; - case DISCARDED: - LOG.warn("Could not acknowledge the checkpoint {} for task {} of job {} at {}, " + - "because the pending checkpoint had been discarded. Discarding the " + - "state handle tp avoid lingering state.", - message.getCheckpointId(), message.getTaskExecutionId(), message.getJob(), taskManagerLocationInfo); - - discardSubtaskState(message.getJob(), message.getTaskExecutionId(), message.getCheckpointId(), message.getSubtaskState()); - } + switch (checkpoint.acknowledgeTask(message.getTaskExecutionId(), message.getSubtaskState(), message.getCheckpointMetrics())) { + case SUCCESS: + LOG.debug("Received acknowledge message for checkpoint {} from task {} of job {} at {}.", + checkpointId, message.getTaskExecutionId(), message.getJob(), taskManagerLocationInfo); - return true; + if (checkpoint.areTasksFullyAcknowledged()) { + completePendingCheckpoint(checkpoint); + } + break; + case DUPLICATE: + LOG.debug("Received a duplicate acknowledge message for checkpoint {}, task {}, job {}, location {}.", + message.getCheckpointId(), message.getTaskExecutionId(), message.getJob(), taskManagerLocationInfo); + break; + case UNKNOWN: + LOG.warn("Could not acknowledge the checkpoint {} for task {} of job {} at {}, " + + "because the task's execution attempt id was unknown. Discarding " + + "the state handle to avoid lingering state.", message.getCheckpointId(), + message.getTaskExecutionId(), message.getJob(), taskManagerLocationInfo); + + discardSubtaskState(message.getJob(), message.getTaskExecutionId(), message.getCheckpointId(), message.getSubtaskState()); + + break; + case DISCARDED: + LOG.warn("Could not acknowledge the checkpoint {} for task {} of job {} at {}, " + + "because the pending checkpoint had been discarded. Discarding the " + + "state handle tp avoid lingering state.", + message.getCheckpointId(), message.getTaskExecutionId(), message.getJob(), taskManagerLocationInfo); + + discardSubtaskState(message.getJob(), message.getTaskExecutionId(), message.getCheckpointId(), message.getSubtaskState()); } - else if (checkpoint != null) { - // this should not happen - throw new IllegalStateException( - "Received message for discarded but non-removed checkpoint " + checkpointId); + + return true; + } + else if (checkpoint != null) { + // this should not happen + throw new IllegalStateException( + "Received message for discarded but non-removed checkpoint " + checkpointId); + } + else { + boolean wasPendingCheckpoint; + + // message is for an unknown checkpoint, or comes too late (checkpoint disposed) + if (recentPendingCheckpoints.contains(checkpointId)) { + wasPendingCheckpoint = true; + LOG.warn("Received late message for now expired checkpoint attempt {} from task " + + "{} of job {} at {}.", checkpointId, message.getTaskExecutionId(), message.getJob(), taskManagerLocationInfo); } else { - boolean wasPendingCheckpoint; - - // message is for an unknown checkpoint, or comes too late (checkpoint disposed) - if (recentPendingCheckpoints.contains(checkpointId)) { - wasPendingCheckpoint = true; - LOG.warn("Received late message for now expired checkpoint attempt {} from task " + - "{} of job {} at {}.", checkpointId, message.getTaskExecutionId(), message.getJob(), taskManagerLocationInfo); - } - else { - LOG.debug("Received message for an unknown checkpoint {} from task {} of job {} at {}.", - checkpointId, message.getTaskExecutionId(), message.getJob(), taskManagerLocationInfo); - wasPendingCheckpoint = false; - } + LOG.debug("Received message for an unknown checkpoint {} from task {} of job {} at {}.", + checkpointId, message.getTaskExecutionId(), message.getJob(), taskManagerLocationInfo); + wasPendingCheckpoint = false; + } - // try to discard the state so that we don't have lingering state lying around - discardSubtaskState(message.getJob(), message.getTaskExecutionId(), message.getCheckpointId(), message.getSubtaskState()); + // try to discard the state so that we don't have lingering state lying around + discardSubtaskState(message.getJob(), message.getTaskExecutionId(), message.getCheckpointId(), message.getSubtaskState()); - return wasPendingCheckpoint; - } + return wasPendingCheckpoint; } } @@ -1062,20 +1032,18 @@ public class CheckpointCoordinator { pendingCheckpoint.finalizeCheckpoint() .whenCompleteAsync((completedCheckpoint, throwable) -> { - synchronized (lock) { - if (shutdown) { - return; - } - if (throwable != null) { - if (!pendingCheckpoint.isDiscarded()) { - abortPendingCheckpoint( - pendingCheckpoint, - new CheckpointException( - CheckpointFailureReason.FINALIZE_CHECKPOINT_FAILURE, throwable)); - } - } else { - onCheckpointSuccess(completedCheckpoint); + if (shutdown) { + return; + } + if (throwable != null) { + if (!pendingCheckpoint.isDiscarded()) { + abortPendingCheckpoint( + pendingCheckpoint, + new CheckpointException( + CheckpointFailureReason.FINALIZE_CHECKPOINT_FAILURE, throwable)); } + } else { + onCheckpointSuccess(completedCheckpoint); } }, mainThreadExecutor); } @@ -1146,11 +1114,9 @@ public class CheckpointCoordinator { public void failUnacknowledgedPendingCheckpointsFor(ExecutionAttemptID executionAttemptId, Throwable cause) { assertRunningMainThread(); - synchronized (lock) { - abortPendingCheckpoints( - checkpoint -> !checkpoint.isAcknowledgedBy(executionAttemptId), - new CheckpointException(CheckpointFailureReason.TASK_FAILURE, cause)); - } + abortPendingCheckpoints( + checkpoint -> !checkpoint.isAcknowledgedBy(executionAttemptId), + new CheckpointException(CheckpointFailureReason.TASK_FAILURE, cause)); } private void rememberRecentCheckpointId(long id) { @@ -1251,75 +1217,73 @@ public class CheckpointCoordinator { assertRunningMainThread(); - synchronized (lock) { - if (shutdown) { - throw new IllegalStateException("CheckpointCoordinator is shut down"); - } + if (shutdown) { + throw new IllegalStateException("CheckpointCoordinator is shut down"); + } - // We create a new shared state registry object, so that all pending async disposal requests from previous - // runs will go against the old object (were they can do no harm). - // This must happen under the checkpoint lock. - sharedStateRegistry.close(); - sharedStateRegistry = sharedStateRegistryFactory.create(executor); + // We create a new shared state registry object, so that all pending async disposal requests from previous + // runs will go against the old object (were they can do no harm). + // This must happen under the checkpoint lock. + sharedStateRegistry.close(); + sharedStateRegistry = sharedStateRegistryFactory.create(executor); - // Recover the checkpoints, TODO this could be done only when there is a new leader, not on each recovery - completedCheckpointStore.recover(); + // Recover the checkpoints, TODO this could be done only when there is a new leader, not on each recovery + completedCheckpointStore.recover(); - // Now, we re-register all (shared) states from the checkpoint store with the new registry - for (CompletedCheckpoint completedCheckpoint : completedCheckpointStore.getAllCheckpoints()) { - completedCheckpoint.registerSharedStatesAfterRestored(sharedStateRegistry); - } + // Now, we re-register all (shared) states from the checkpoint store with the new registry + for (CompletedCheckpoint completedCheckpoint : completedCheckpointStore.getAllCheckpoints()) { + completedCheckpoint.registerSharedStatesAfterRestored(sharedStateRegistry); + } - LOG.debug("Status of the shared state registry of job {} after restore: {}.", job, sharedStateRegistry); + LOG.debug("Status of the shared state registry of job {} after restore: {}.", job, sharedStateRegistry); - // Restore from the latest checkpoint - CompletedCheckpoint latest = completedCheckpointStore.getLatestCheckpoint(isPreferCheckpointForRecovery); + // Restore from the latest checkpoint + CompletedCheckpoint latest = completedCheckpointStore.getLatestCheckpoint(isPreferCheckpointForRecovery); - if (latest == null) { - if (errorIfNoCheckpoint) { - throw new IllegalStateException("No completed checkpoint available"); - } else { - LOG.debug("Resetting the master hooks."); - MasterHooks.reset(masterHooks.values(), LOG); + if (latest == null) { + if (errorIfNoCheckpoint) { + throw new IllegalStateException("No completed checkpoint available"); + } else { + LOG.debug("Resetting the master hooks."); + MasterHooks.reset(masterHooks.values(), LOG); - return false; - } + return false; } + } - LOG.info("Restoring job {} from latest valid checkpoint: {}.", job, latest); - - // re-assign the task states - final Map operatorStates = latest.getOperatorStates(); - - StateAssignmentOperation stateAssignmentOperation = - new StateAssignmentOperation(latest.getCheckpointID(), tasks, operatorStates, allowNonRestoredState); + LOG.info("Restoring job {} from latest valid checkpoint: {}.", job, latest); - stateAssignmentOperation.assignStates(); + // re-assign the task states + final Map operatorStates = latest.getOperatorStates(); - // call master hooks for restore + StateAssignmentOperation stateAssignmentOperation = + new StateAssignmentOperation(latest.getCheckpointID(), tasks, operatorStates, allowNonRestoredState); - MasterHooks.restoreMasterHooks( - masterHooks, - latest.getMasterHookStates(), - latest.getCheckpointID(), - allowNonRestoredState, - LOG); + stateAssignmentOperation.assignStates(); - // update metrics + // call master hooks for restore - if (statsTracker != null) { - long restoreTimestamp = System.currentTimeMillis(); - RestoredCheckpointStats restored = new RestoredCheckpointStats( - latest.getCheckpointID(), - latest.getProperties(), - restoreTimestamp, - latest.getExternalPointer()); + MasterHooks.restoreMasterHooks( + masterHooks, + latest.getMasterHookStates(), + latest.getCheckpointID(), + allowNonRestoredState, + LOG); - statsTracker.reportRestoredCheckpoint(restored); - } + // update metrics - return true; + if (statsTracker != null) { + long restoreTimestamp = System.currentTimeMillis(); + RestoredCheckpointStats restored = new RestoredCheckpointStats( + latest.getCheckpointID(), + latest.getProperties(), + restoreTimestamp, + latest.getExternalPointer()); + + statsTracker.reportRestoredCheckpoint(restored); } + + return true; } /** @@ -1372,22 +1336,16 @@ public class CheckpointCoordinator { @VisibleForTesting public int getNumberOfRetainedSuccessfulCheckpoints() { - synchronized (lock) { - return completedCheckpointStore.getNumberOfRetainedCheckpoints(); - } + return completedCheckpointStore.getNumberOfRetainedCheckpoints(); } public Map getPendingCheckpoints() { - synchronized (lock) { - return new HashMap<>(this.pendingCheckpoints); - } + return new HashMap<>(this.pendingCheckpoints); } @VisibleForTesting public List getSuccessfulCheckpoints() throws Exception { - synchronized (lock) { - return completedCheckpointStore.getAllCheckpoints(); - } + return completedCheckpointStore.getAllCheckpoints(); } public CheckpointStorageCoordinatorView getCheckpointStorage() { @@ -1432,37 +1390,33 @@ public class CheckpointCoordinator { public void startCheckpointScheduler() { assertRunningMainThread(); - synchronized (lock) { - if (shutdown) { - throw new IllegalArgumentException("Checkpoint coordinator is shut down"); - } + if (shutdown) { + throw new IllegalArgumentException("Checkpoint coordinator is shut down"); + } - // make sure all prior timers are cancelled - stopCheckpointScheduler(); + // make sure all prior timers are cancelled + stopCheckpointScheduler(); - periodicScheduling = true; - currentPeriodicTrigger = scheduleTriggerWithDelay(getRandomInitDelay()); - } + periodicScheduling = true; + currentPeriodicTrigger = scheduleTriggerWithDelay(getRandomInitDelay()); } public void stopCheckpointScheduler() { assertRunningMainThread(); - synchronized (lock) { - periodicTriggeringSuspended = false; - periodicScheduling = false; + periodicTriggeringSuspended = false; + periodicScheduling = false; - if (currentPeriodicTrigger != null) { - currentPeriodicTrigger.cancel(false); - currentPeriodicTrigger = null; - } + if (currentPeriodicTrigger != null) { + currentPeriodicTrigger.cancel(false); + currentPeriodicTrigger = null; + } - final CheckpointException reason = - new CheckpointException(CheckpointFailureReason.CHECKPOINT_COORDINATOR_SUSPEND); - abortPendingAndQueuedCheckpoints(reason); + final CheckpointException reason = + new CheckpointException(CheckpointFailureReason.CHECKPOINT_COORDINATOR_SUSPEND); + abortPendingAndQueuedCheckpoints(reason); - numUnsuccessfulCheckpointsTriggers.set(0); - } + numUnsuccessfulCheckpointsTriggers.set(0); } /** @@ -1470,16 +1424,13 @@ public class CheckpointCoordinator { * @param exception The exception. */ public void abortPendingCheckpoints(CheckpointException exception) { - synchronized (lock) { - abortPendingCheckpoints(ignored -> true, exception); - } + abortPendingCheckpoints(ignored -> true, exception); } private void abortPendingCheckpoints( Predicate checkpointToFailPredicate, CheckpointException exception) { - assert Thread.holdsLock(lock); assertRunningMainThread(); final PendingCheckpoint[] pendingCheckpointsToFail = pendingCheckpoints @@ -1548,17 +1499,15 @@ public class CheckpointCoordinator { // ------------------------------------------------------------------------ public JobStatusListener createActivatorDeactivator() { - synchronized (lock) { - if (shutdown) { - throw new IllegalArgumentException("Checkpoint coordinator is shut down"); - } - - if (jobStatusListener == null) { - jobStatusListener = new CheckpointCoordinatorDeActivator(this); - } + if (shutdown) { + throw new IllegalArgumentException("Checkpoint coordinator is shut down"); + } - return jobStatusListener; + if (jobStatusListener == null) { + jobStatusListener = new CheckpointCoordinatorDeActivator(this); } + + return jobStatusListener; } // ------------------------------------------------------------------------ @@ -1614,7 +1563,6 @@ public class CheckpointCoordinator { CheckpointException exception, @Nullable final ExecutionAttemptID executionAttemptID) { - assert(Thread.holdsLock(lock)); assertRunningMainThread(); if (!pendingCheckpoint.isDiscarded()) { @@ -1723,7 +1671,6 @@ public class CheckpointCoordinator { } private void abortPendingAndQueuedCheckpoints(CheckpointException exception) { - assert(Thread.holdsLock(lock)); CheckpointTriggerRequest request; while ((request = triggerRequestQueue.poll()) != null) { request.onCompletionPromise.completeExceptionally(exception); @@ -1752,17 +1699,15 @@ public class CheckpointCoordinator { @Override public void run() { assertRunningMainThread(); - synchronized (lock) { - // only do the work if the checkpoint is not discarded anyways - // note that checkpoint completion discards the pending checkpoint object - if (!pendingCheckpoint.isDiscarded()) { - LOG.info("Checkpoint {} of job {} expired before completing.", - pendingCheckpoint.getCheckpointId(), job); - - abortPendingCheckpoint( - pendingCheckpoint, - new CheckpointException(CheckpointFailureReason.CHECKPOINT_EXPIRED)); - } + // only do the work if the checkpoint is not discarded anyways + // note that checkpoint completion discards the pending checkpoint object + if (!pendingCheckpoint.isDiscarded()) { + LOG.info("Checkpoint {} of job {} expired before completing.", + pendingCheckpoint.getCheckpointId(), job); + + abortPendingCheckpoint( + pendingCheckpoint, + new CheckpointException(CheckpointFailureReason.CHECKPOINT_EXPIRED)); } } } -- Gitee From 6936713b3b9b0caea8ce080d9b9d3de16a4db46f Mon Sep 17 00:00:00 2001 From: ifndef-SleePy Date: Thu, 5 Mar 2020 02:56:49 +0800 Subject: [PATCH 249/885] [FLINK-14971][checkpointing] Remove lock of PendingCheckpoint and introduce IO lock for PendingCheckpoint Since all non-IO operations are executed in main thread, the lock of PendingCheckpoint could be avoided now. --- .../runtime/checkpoint/PendingCheckpoint.java | 297 +++++++++--------- 1 file changed, 145 insertions(+), 152 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java index 27395ea5be..da89f7059f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java @@ -78,8 +78,6 @@ public class PendingCheckpoint { /** The PendingCheckpoint logs to the same logger as the CheckpointCoordinator. */ private static final Logger LOG = LoggerFactory.getLogger(CheckpointCoordinator.class); - private final Object lock = new Object(); - private final JobID jobId; private final long checkpointId; @@ -116,6 +114,9 @@ public class PendingCheckpoint { private final CompletedCheckpointStore completedCheckpointStore; + /** The lock for avoiding conflict between I/O operations. */ + private final Object operationLock = new Object(); + private int numAcknowledgedTasks; private boolean discarded; @@ -266,19 +267,17 @@ public class PendingCheckpoint { * @return true, if the handle was set, false, if the checkpoint is already disposed; */ public boolean setCancellerHandle(ScheduledFuture cancellerHandle) { - synchronized (lock) { - if (this.cancellerHandle == null) { - if (!discarded) { - this.cancellerHandle = cancellerHandle; - return true; - } else { - return false; - } - } - else { - throw new IllegalStateException("A canceller handle was already set"); + if (this.cancellerHandle == null) { + if (!discarded) { + this.cancellerHandle = cancellerHandle; + return true; + } else { + return false; } } + else { + throw new IllegalStateException("A canceller handle was already set"); + } } public CheckpointException getFailureCause() { @@ -300,24 +299,24 @@ public class PendingCheckpoint { public CompletableFuture finalizeCheckpoint() { - synchronized (lock) { - if (isDiscarded()) { - return FutureUtils.completedExceptionally(new IllegalStateException( - "checkpoint is discarded")); - } - if (!isFullyAcknowledged()) { - return FutureUtils.completedExceptionally(new IllegalStateException( - "Pending checkpoint has not been fully acknowledged yet")); - } + if (isDiscarded()) { + return FutureUtils.completedExceptionally(new IllegalStateException( + "checkpoint is discarded")); + } + if (!isFullyAcknowledged()) { + return FutureUtils.completedExceptionally(new IllegalStateException( + "Pending checkpoint has not been fully acknowledged yet")); + } - // now we stop the canceller before finalization - // it simplifies the concurrent conflict issue here - cancelCanceller(); + // now we stop the canceller before finalization + // it simplifies the concurrent conflict issue here + cancelCanceller(); - // make sure we fulfill the promise with an exception if something fails - final CompletableFuture finalizingFuture = - CompletableFuture.supplyAsync(() -> { - try { + // make sure we fulfill the promise with an exception if something fails + final CompletableFuture finalizingFuture = + CompletableFuture.supplyAsync(() -> { + try { + synchronized (operationLock) { checkState(!isDiscarded(), "The checkpoint has been discarded"); // write out the metadata final CheckpointMetadata savepoint = new CheckpointMetadata(checkpointId, operatorStates.values(), masterStates); @@ -344,37 +343,38 @@ public class PendingCheckpoint { completed.discardOnFailedStoring(); } return completed; - } catch (Throwable t) { - LOG.warn("Could not finalize checkpoint {}.", checkpointId, t); - onCompletionPromise.completeExceptionally(t); - throw new CompletionException(t); } - }, executor); + } catch (Throwable t) { + LOG.warn("Could not finalize checkpoint {}.", checkpointId, t); + onCompletionPromise.completeExceptionally(t); + throw new CompletionException(t); + } + }, executor); - return finalizingFuture.thenApplyAsync((completed) -> { + return finalizingFuture.thenApplyAsync((completed) -> { - // since canceller has been already cancelled, discarding means the coordinator must be shut down - // all the resources should be released properly when it's shutting down the coordinator - checkState(!isDiscarded(), "The checkpoint has been discarded"); + // since canceller has been already cancelled, discarding means the coordinator must be shut down + // all the resources should be released properly when it's shutting down the coordinator + checkState(!isDiscarded(), "The checkpoint has been discarded"); - onCompletionPromise.complete(completed); + onCompletionPromise.complete(completed); - // to prevent null-pointers from concurrent modification, copy reference onto stack - PendingCheckpointStats statsCallback = this.statsCallback; - if (statsCallback != null) { - // Finalize the statsCallback and give the completed checkpoint a - // callback for discards. - CompletedCheckpointStats.DiscardCallback discardCallback = - statsCallback.reportCompletedCheckpoint(completed.getExternalPointer()); - completed.setDiscardCallback(discardCallback); - } + // to prevent null-pointers from concurrent modification, copy reference onto stack + PendingCheckpointStats statsCallback = this.statsCallback; + if (statsCallback != null) { + // Finalize the statsCallback and give the completed checkpoint a + // callback for discards. + CompletedCheckpointStats.DiscardCallback discardCallback = + statsCallback.reportCompletedCheckpoint(completed.getExternalPointer()); + completed.setDiscardCallback(discardCallback); - // mark this pending checkpoint as disposed, but do NOT drop the state - dispose(false); + } - return completed; - }, mainThreadExecutor); - } + // mark this pending checkpoint as disposed, but do NOT drop the state + dispose(false); + + return completed; + }, mainThreadExecutor); } /** @@ -390,112 +390,108 @@ public class PendingCheckpoint { TaskStateSnapshot operatorSubtaskStates, CheckpointMetrics metrics) { - synchronized (lock) { - if (discarded) { - return TaskAcknowledgeResult.DISCARDED; - } + if (discarded) { + return TaskAcknowledgeResult.DISCARDED; + } - final ExecutionVertex vertex = notYetAcknowledgedTasks.remove(executionAttemptId); + final ExecutionVertex vertex = notYetAcknowledgedTasks.remove(executionAttemptId); - if (vertex == null) { - if (acknowledgedTasks.contains(executionAttemptId)) { - return TaskAcknowledgeResult.DUPLICATE; - } else { - return TaskAcknowledgeResult.UNKNOWN; - } + if (vertex == null) { + if (acknowledgedTasks.contains(executionAttemptId)) { + return TaskAcknowledgeResult.DUPLICATE; } else { - acknowledgedTasks.add(executionAttemptId); + return TaskAcknowledgeResult.UNKNOWN; } + } else { + acknowledgedTasks.add(executionAttemptId); + } - List operatorIDs = vertex.getJobVertex().getOperatorIDs(); - int subtaskIndex = vertex.getParallelSubtaskIndex(); - long ackTimestamp = System.currentTimeMillis(); - - long stateSize = 0L; + List operatorIDs = vertex.getJobVertex().getOperatorIDs(); + int subtaskIndex = vertex.getParallelSubtaskIndex(); + long ackTimestamp = System.currentTimeMillis(); - if (operatorSubtaskStates != null) { - for (OperatorID operatorID : operatorIDs) { + long stateSize = 0L; - OperatorSubtaskState operatorSubtaskState = - operatorSubtaskStates.getSubtaskStateByOperatorID(operatorID); + if (operatorSubtaskStates != null) { + for (OperatorID operatorID : operatorIDs) { - // if no real operatorSubtaskState was reported, we insert an empty state - if (operatorSubtaskState == null) { - operatorSubtaskState = new OperatorSubtaskState(); - } + OperatorSubtaskState operatorSubtaskState = + operatorSubtaskStates.getSubtaskStateByOperatorID(operatorID); - OperatorState operatorState = operatorStates.get(operatorID); + // if no real operatorSubtaskState was reported, we insert an empty state + if (operatorSubtaskState == null) { + operatorSubtaskState = new OperatorSubtaskState(); + } - if (operatorState == null) { - operatorState = new OperatorState( - operatorID, - vertex.getTotalNumberOfParallelSubtasks(), - vertex.getMaxParallelism()); - operatorStates.put(operatorID, operatorState); - } + OperatorState operatorState = operatorStates.get(operatorID); - operatorState.putState(subtaskIndex, operatorSubtaskState); - stateSize += operatorSubtaskState.getStateSize(); + if (operatorState == null) { + operatorState = new OperatorState( + operatorID, + vertex.getTotalNumberOfParallelSubtasks(), + vertex.getMaxParallelism()); + operatorStates.put(operatorID, operatorState); } - } - ++numAcknowledgedTasks; - - // publish the checkpoint statistics - // to prevent null-pointers from concurrent modification, copy reference onto stack - final PendingCheckpointStats statsCallback = this.statsCallback; - if (statsCallback != null) { - // Do this in millis because the web frontend works with them - long alignmentDurationMillis = metrics.getAlignmentDurationNanos() / 1_000_000; - long checkpointStartDelayMillis = metrics.getCheckpointStartDelayNanos() / 1_000_000; - - SubtaskStateStats subtaskStateStats = new SubtaskStateStats( - subtaskIndex, - ackTimestamp, - stateSize, - metrics.getSyncDurationMillis(), - metrics.getAsyncDurationMillis(), - metrics.getBytesBufferedInAlignment(), - alignmentDurationMillis, - checkpointStartDelayMillis); - - statsCallback.reportSubtaskStats(vertex.getJobvertexId(), subtaskStateStats); + operatorState.putState(subtaskIndex, operatorSubtaskState); + stateSize += operatorSubtaskState.getStateSize(); } + } - return TaskAcknowledgeResult.SUCCESS; + ++numAcknowledgedTasks; + + // publish the checkpoint statistics + // to prevent null-pointers from concurrent modification, copy reference onto stack + final PendingCheckpointStats statsCallback = this.statsCallback; + if (statsCallback != null) { + // Do this in millis because the web frontend works with them + long alignmentDurationMillis = metrics.getAlignmentDurationNanos() / 1_000_000; + long checkpointStartDelayMillis = metrics.getCheckpointStartDelayNanos() / 1_000_000; + + SubtaskStateStats subtaskStateStats = new SubtaskStateStats( + subtaskIndex, + ackTimestamp, + stateSize, + metrics.getSyncDurationMillis(), + metrics.getAsyncDurationMillis(), + metrics.getBytesBufferedInAlignment(), + alignmentDurationMillis, + checkpointStartDelayMillis); + + statsCallback.reportSubtaskStats(vertex.getJobvertexId(), subtaskStateStats); } + + return TaskAcknowledgeResult.SUCCESS; } public TaskAcknowledgeResult acknowledgeCoordinatorState( OperatorCoordinatorCheckpointContext coordinatorInfo, @Nullable StreamStateHandle stateHandle) { - synchronized (lock) { - if (discarded) { - return TaskAcknowledgeResult.DISCARDED; - } + if (discarded) { + return TaskAcknowledgeResult.DISCARDED; + } - final OperatorID operatorId = coordinatorInfo.operatorId(); - OperatorState operatorState = operatorStates.get(operatorId); + final OperatorID operatorId = coordinatorInfo.operatorId(); + OperatorState operatorState = operatorStates.get(operatorId); - // sanity check for better error reporting - if (!notYetAcknowledgedOperatorCoordinators.remove(operatorId)) { - return operatorState != null && operatorState.getCoordinatorState() != null - ? TaskAcknowledgeResult.DUPLICATE - : TaskAcknowledgeResult.UNKNOWN; - } + // sanity check for better error reporting + if (!notYetAcknowledgedOperatorCoordinators.remove(operatorId)) { + return operatorState != null && operatorState.getCoordinatorState() != null + ? TaskAcknowledgeResult.DUPLICATE + : TaskAcknowledgeResult.UNKNOWN; + } - if (stateHandle != null) { - if (operatorState == null) { - operatorState = new OperatorState( - operatorId, coordinatorInfo.currentParallelism(), coordinatorInfo.maxParallelism()); - operatorStates.put(operatorId, operatorState); - } - operatorState.setCoordinatorState(stateHandle); + if (stateHandle != null) { + if (operatorState == null) { + operatorState = new OperatorState( + operatorId, coordinatorInfo.currentParallelism(), coordinatorInfo.maxParallelism()); + operatorStates.put(operatorId, operatorState); } - - return TaskAcknowledgeResult.SUCCESS; + operatorState.setCoordinatorState(stateHandle); } + + return TaskAcknowledgeResult.SUCCESS; } /** @@ -506,12 +502,9 @@ public class PendingCheckpoint { * @param state The state to acknowledge */ public void acknowledgeMasterState(String identifier, @Nullable MasterState state) { - - synchronized (lock) { - if (!discarded) { - if (notYetAcknowledgedMasterStates.remove(identifier) && state != null) { - masterStates.add(state); - } + if (!discarded) { + if (notYetAcknowledgedMasterStates.remove(identifier) && state != null) { + masterStates.add(state); } } } @@ -550,14 +543,14 @@ public class PendingCheckpoint { private void dispose(boolean releaseState) { - synchronized (lock) { - try { - numAcknowledgedTasks = -1; - if (!discarded && releaseState) { - executor.execute(new Runnable() { - @Override - public void run() { + try { + numAcknowledgedTasks = -1; + if (!discarded && releaseState) { + executor.execute(new Runnable() { + @Override + public void run() { + synchronized (operationLock) { // discard the private states. // unregistered shared states are still considered private at this point. try { @@ -570,15 +563,15 @@ public class PendingCheckpoint { operatorStates.clear(); } } - }); + } + }); - } - } finally { - discarded = true; - notYetAcknowledgedTasks.clear(); - acknowledgedTasks.clear(); - cancelCanceller(); } + } finally { + discarded = true; + notYetAcknowledgedTasks.clear(); + acknowledgedTasks.clear(); + cancelCanceller(); } } -- Gitee From e08606cf750c64acfa45506cd9bf8c05cd64c32d Mon Sep 17 00:00:00 2001 From: zhule Date: Fri, 20 Mar 2020 17:06:47 +0800 Subject: [PATCH 250/885] [FLINK-16092][docs-zh] Translate dev/table/functions/index.zh.md into Chinese This closes #11461 --- docs/dev/table/functions/index.md | 8 +-- docs/dev/table/functions/index.zh.md | 86 ++++++++++++++-------------- 2 files changed, 47 insertions(+), 47 deletions(-) diff --git a/docs/dev/table/functions/index.md b/docs/dev/table/functions/index.md index 0acb06039f..fd3b777873 100644 --- a/docs/dev/table/functions/index.md +++ b/docs/dev/table/functions/index.md @@ -55,14 +55,14 @@ Referencing Functions There are two ways users can reference a function in Flink - referencing function precisely or ambiguously. -## Precise Function Reference +### Precise Function Reference Precise function reference empowers users to use catalog functions specifically, and across catalog and across database, e.g. `select mycatalog.mydb.myfunc(x) from mytable` and `select mydb.myfunc(x) from mytable`. This is only supported starting from Flink 1.10. -## Ambiguous Function Reference +### Ambiguous Function Reference In ambiguous function reference, users just specify the function's name in SQL query, e.g. `select myfunc(x) from mytable`. @@ -74,7 +74,7 @@ The resolution order only matters when there are functions of different types bu e.g. when there’re three functions all named “myfunc” but are of temporary catalog, catalog, and system function respectively. If there’s no function name collision, functions will just be resolved to the sole one. -## Precise Function Reference +### Precise Function Reference Because system functions don’t have namespaces, a precise function reference in Flink must be pointing to either a temporary catalog function or a catalog function. @@ -84,7 +84,7 @@ The resolution order is: 1. Temporary catalog function 2. Catalog function -## Ambiguous Function Reference +### Ambiguous Function Reference The resolution order is: diff --git a/docs/dev/table/functions/index.zh.md b/docs/dev/table/functions/index.zh.md index ac3d9923f2..321e770320 100644 --- a/docs/dev/table/functions/index.zh.md +++ b/docs/dev/table/functions/index.zh.md @@ -1,5 +1,5 @@ --- -title: "Functions" +title: "函数" nav-id: table_functions nav-parent_id: tableapi nav-pos: 60 @@ -24,74 +24,74 @@ specific language governing permissions and limitations under the License. --> -Flink Table API & SQL empowers users to do data transformations with functions. +Flink 允许用户在 Table API 和 SQL 中使用函数进行数据的转换。 * This will be replaced by the TOC {:toc} -Types of Functions +函数类型 ------------------ -There are two dimensions to classify functions in Flink. +Flink 中的函数有两个划分标准。 -One dimension is system (or built-in) functions v.s. catalog functions. System functions have no namespace and can be -referenced with just their names. Catalog functions belong to a catalog and database therefore they have catalog and database -namespaces, they can be referenced by either fully/partially qualified name (`catalog.db.func` or `db.func`) or just the -function name. +一个划分标准是:系统(内置)函数和 Catalog 函数。系统函数没有名称空间,只能通过其名称来进行引用。 +Catalog 函数属于 Catalog 和数据库,因此它们拥有 Catalog 和数据库命名空间。 +用户可以通过全/部分限定名(`catalog.db.func` 或 `db.func`)或者函数名 +来对 Catalog 函数进行引用。 -The other dimension is temporary functions v.s. persistent functions. Temporary functions are volatile and only live up to - lifespan of a session, they are always created by users. Persistent functions live across lifespan of sessions, they are either - provided by the system or persisted in catalogs. - -The two dimensions give Flink users 4 categories of functions: +另一个划分标准是:临时函数和持久化函数。 +临时函数始终由用户创建,它容易改变并且仅在会话的生命周期内有效。 +持久化函数不是由系统提供,就是存储在 Catalog 中,它在会话的整个生命周期内都有效。 -1. Temporary system functions -2. System functions -3. Temporary catalog functions -4. Catalog functions +这两个划分标准给 Flink 用户提供了 4 种函数: -Note that system functions always precede catalog's, and temporary functions always precede persistent on their own dimension -in function resolution order explained below. +1. 临时性系统函数 +2. 系统函数 +3. 临时性 Catalog 函数 +4. Catalog 函数 -Referencing Functions +请注意,系统函数始终优先于 Catalog 函数解析,临时函数始终优先于持久化函数解析, +函数解析优先级如下所述。 + +函数引用 --------------------- -There are two ways users can reference a function in Flink - referencing function precisely or ambiguously. +用户在 Flink 中可以通过精确、模糊两种引用方式引用函数。 -## Precise Function Reference +### 精确函数引用 -Precise function reference empowers users to use catalog functions specifically, and across catalog and across database, -e.g. `select mycatalog.mydb.myfunc(x) from mytable` and `select mydb.myfunc(x) from mytable`. +精确函数引用允许用户跨 Catalog,跨数据库调用 Catalog 函数。 +例如:`select mycatalog.mydb.myfunc(x) from mytable` 和 `select mydb.myfunc(x) from mytable`。 -This is only supported starting from Flink 1.10. +仅 Flink 1.10 以上版本支持。 -## Ambiguous Function Reference +### 模糊函数引用 -In ambiguous function reference, users just specify the function's name in SQL query, e.g. `select myfunc(x) from mytable`. +在模糊函数引用中,用户只需在 SQL 查询中指定函数名,例如: `select myfunc(x) from mytable`。 -Function Resolution Order +函数解析顺序 ------------------------- -The resolution order only matters when there are functions of different types but the same name, -e.g. when there’re three functions all named “myfunc” but are of temporary catalog, catalog, and system function respectively. -If there’s no function name collision, functions will just be resolved to the sole one. +当函数名相同,函数类型不同时,函数解析顺序才有意义。 +例如:当有三个都名为 "myfunc" 的临时性 Catalog 函数,Catalog 函数,和系统函数时, +如果没有命名冲突,三个函数将会被解析为一个函数。 -## Precise Function Reference +### 精确函数引用 -Because system functions don’t have namespaces, a precise function reference in Flink must be pointing to either a temporary catalog -function or a catalog function. +由于系统函数没有命名空间,Flink 中的精确函数引用必须 +指向临时性 Catalog 函数或 Catalog 函数。 -The resolution order is: +解析顺序如下: -1. Temporary catalog function -2. Catalog function +1. 临时性 catalog 函数 +2. Catalog 函数 -## Ambiguous Function Reference +### 模糊函数引用 -The resolution order is: +解析顺序如下: -1. Temporary system function -2. System function -3. Temporary catalog function, in the current catalog and current database of the session -4. Catalog function, in the current catalog and current database of the session +1. 临时性系统函数 +2. 系统函数 +3. 临时性 Catalog 函数, 在会话的当前 Catalog 和当前数据库中 +4. Catalog 函数, 在会话的当前 Catalog 和当前数据库中 -- Gitee From 9c351b79af07a13ffc964da388309adaedc94a89 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=BC=AB=E6=AD=A5=E4=BA=91=E7=AB=AF?= Date: Tue, 24 Mar 2020 12:05:21 +0800 Subject: [PATCH 251/885] [FLINK-16567][table][doc] Update to TableConfig in query_configuration document This closes #11410 --- .../table/streaming/query_configuration.md | 34 +++++++++---------- .../table/streaming/query_configuration.zh.md | 34 +++++++++---------- 2 files changed, 34 insertions(+), 34 deletions(-) diff --git a/docs/dev/table/streaming/query_configuration.md b/docs/dev/table/streaming/query_configuration.md index 2a2f50cead..3bf0c45d9d 100644 --- a/docs/dev/table/streaming/query_configuration.md +++ b/docs/dev/table/streaming/query_configuration.md @@ -24,7 +24,7 @@ under the License. Table API and SQL queries have the same semantics regardless whether their input is bounded batch input or unbounded stream input. In many cases, continuous queries on streaming input are capable of computing accurate results that are identical to offline computed results. However, this is not possible in general case because continuous queries have to restrict the size of the state they are maintaining in order to avoid to run out of storage and to be able to process unbounded streaming data over a long period of time. As a result, a continuous query might only be able to provide approximated results depending on the characteristics of the input data and the query itself. -Flink's Table API and SQL interface provide parameters to tune the accuracy and resource consumption of continuous queries. The parameters are specified via a `QueryConfig` object. The `QueryConfig` can be obtained from the `TableEnvironment` and is passed back when a `Table` is translated, i.e., when it is [transformed into a DataStream]({{ site.baseurl }}/dev/table/common.html#convert-a-table-into-a-datastream-or-dataset) or [emitted via a TableSink](../common.html#emit-a-table). +Flink's Table API and SQL interface provide parameters to tune the accuracy and resource consumption of continuous queries. The parameters are specified via a `TableConfig` object. The `TableConfig` can be obtained from the `TableEnvironment` and is passed back when a `Table` is translated, i.e., when it is [transformed into a DataStream]({{ site.baseurl }}/dev/table/common.html#convert-a-table-into-a-datastream-or-dataset) or [emitted via a TableSink](../common.html#emit-a-table).

    @@ -33,9 +33,9 @@ StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironm StreamTableEnvironment tableEnv = StreamTableEnvironment.create(env); // obtain query configuration from TableEnvironment -StreamQueryConfig qConfig = tableEnv.queryConfig(); +TableConfig tConfig = tableEnv.getConfig(); // set query parameters -qConfig.withIdleStateRetentionTime(Time.hours(12), Time.hours(24)); +tConfig.setIdleStateRetentionTime(Time.hours(12), Time.hours(24)); // define query Table result = ... @@ -51,10 +51,10 @@ tableEnv.registerTableSink( sink); // table sink // emit result Table via a TableSink -result.insertInto("outputTable", qConfig); +result.insertInto("outputTable"); // convert result Table into a DataStream -DataStream stream = tableEnv.toAppendStream(result, Row.class, qConfig); +DataStream stream = tableEnv.toAppendStream(result, Row.class); {% endhighlight %}
    @@ -64,9 +64,9 @@ val env = StreamExecutionEnvironment.getExecutionEnvironment val tableEnv = StreamTableEnvironment.create(env) // obtain query configuration from TableEnvironment -val qConfig: StreamQueryConfig = tableEnv.queryConfig +val tConfig: TableConfig = tableEnv.getConfig // set query parameters -qConfig.withIdleStateRetentionTime(Time.hours(12), Time.hours(24)) +tConfig.setIdleStateRetentionTime(Time.hours(12), Time.hours(24)) // define query val result: Table = ??? @@ -82,16 +82,16 @@ tableEnv.registerTableSink( sink) // table sink // emit result Table via a TableSink -result.insertInto("outputTable", qConfig) +result.insertInto("outputTable") // convert result Table into a DataStream[Row] -val stream: DataStream[Row] = result.toAppendStream[Row](qConfig) +val stream: DataStream[Row] = result.toAppendStream[Row] {% endhighlight %}
    {% highlight python %} -# use TableConfig instead of QueryConfig in python API +# use TableConfig in python API t_config = TableConfig() # set query parameters t_config.set_idle_state_retention_time(timedelta(hours=12), timedelta(hours=24)) @@ -116,7 +116,7 @@ result.insert_into("outputTable")
    -In the following we describe the parameters of the `QueryConfig` and how they affect the accuracy and resource consumption of a query. +In the following we describe the parameters of the `TableConfig` and how they affect the accuracy and resource consumption of a query. Idle State Retention Time ------------------------- @@ -145,30 +145,30 @@ The parameters are specified as follows:
    {% highlight java %} -StreamQueryConfig qConfig = ... +TableConfig tConfig = ... // set idle state retention time: min = 12 hours, max = 24 hours -qConfig.withIdleStateRetentionTime(Time.hours(12), Time.hours(24)); +tConfig.setIdleStateRetentionTime(Time.hours(12), Time.hours(24)); {% endhighlight %}
    {% highlight scala %} -val qConfig: StreamQueryConfig = ??? +val tConfig: TableConfig = ??? // set idle state retention time: min = 12 hours, max = 24 hours -qConfig.withIdleStateRetentionTime(Time.hours(12), Time.hours(24)) +tConfig.setIdleStateRetentionTime(Time.hours(12), Time.hours(24)) {% endhighlight %}
    {% highlight python %} -q_config = ... # type: StreamQueryConfig +t_config = ... # type: TableConfig # set idle state retention time: min = 12 hours, max = 24 hours -q_config.with_idle_state_retention_time(timedelta(hours=12), timedelta(hours=24)) +t_config.set_idle_state_retention_time(timedelta(hours=12), timedelta(hours=24)) {% endhighlight %}
    diff --git a/docs/dev/table/streaming/query_configuration.zh.md b/docs/dev/table/streaming/query_configuration.zh.md index 2a2f50cead..3bf0c45d9d 100644 --- a/docs/dev/table/streaming/query_configuration.zh.md +++ b/docs/dev/table/streaming/query_configuration.zh.md @@ -24,7 +24,7 @@ under the License. Table API and SQL queries have the same semantics regardless whether their input is bounded batch input or unbounded stream input. In many cases, continuous queries on streaming input are capable of computing accurate results that are identical to offline computed results. However, this is not possible in general case because continuous queries have to restrict the size of the state they are maintaining in order to avoid to run out of storage and to be able to process unbounded streaming data over a long period of time. As a result, a continuous query might only be able to provide approximated results depending on the characteristics of the input data and the query itself. -Flink's Table API and SQL interface provide parameters to tune the accuracy and resource consumption of continuous queries. The parameters are specified via a `QueryConfig` object. The `QueryConfig` can be obtained from the `TableEnvironment` and is passed back when a `Table` is translated, i.e., when it is [transformed into a DataStream]({{ site.baseurl }}/dev/table/common.html#convert-a-table-into-a-datastream-or-dataset) or [emitted via a TableSink](../common.html#emit-a-table). +Flink's Table API and SQL interface provide parameters to tune the accuracy and resource consumption of continuous queries. The parameters are specified via a `TableConfig` object. The `TableConfig` can be obtained from the `TableEnvironment` and is passed back when a `Table` is translated, i.e., when it is [transformed into a DataStream]({{ site.baseurl }}/dev/table/common.html#convert-a-table-into-a-datastream-or-dataset) or [emitted via a TableSink](../common.html#emit-a-table).
    @@ -33,9 +33,9 @@ StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironm StreamTableEnvironment tableEnv = StreamTableEnvironment.create(env); // obtain query configuration from TableEnvironment -StreamQueryConfig qConfig = tableEnv.queryConfig(); +TableConfig tConfig = tableEnv.getConfig(); // set query parameters -qConfig.withIdleStateRetentionTime(Time.hours(12), Time.hours(24)); +tConfig.setIdleStateRetentionTime(Time.hours(12), Time.hours(24)); // define query Table result = ... @@ -51,10 +51,10 @@ tableEnv.registerTableSink( sink); // table sink // emit result Table via a TableSink -result.insertInto("outputTable", qConfig); +result.insertInto("outputTable"); // convert result Table into a DataStream -DataStream stream = tableEnv.toAppendStream(result, Row.class, qConfig); +DataStream stream = tableEnv.toAppendStream(result, Row.class); {% endhighlight %}
    @@ -64,9 +64,9 @@ val env = StreamExecutionEnvironment.getExecutionEnvironment val tableEnv = StreamTableEnvironment.create(env) // obtain query configuration from TableEnvironment -val qConfig: StreamQueryConfig = tableEnv.queryConfig +val tConfig: TableConfig = tableEnv.getConfig // set query parameters -qConfig.withIdleStateRetentionTime(Time.hours(12), Time.hours(24)) +tConfig.setIdleStateRetentionTime(Time.hours(12), Time.hours(24)) // define query val result: Table = ??? @@ -82,16 +82,16 @@ tableEnv.registerTableSink( sink) // table sink // emit result Table via a TableSink -result.insertInto("outputTable", qConfig) +result.insertInto("outputTable") // convert result Table into a DataStream[Row] -val stream: DataStream[Row] = result.toAppendStream[Row](qConfig) +val stream: DataStream[Row] = result.toAppendStream[Row] {% endhighlight %}
    {% highlight python %} -# use TableConfig instead of QueryConfig in python API +# use TableConfig in python API t_config = TableConfig() # set query parameters t_config.set_idle_state_retention_time(timedelta(hours=12), timedelta(hours=24)) @@ -116,7 +116,7 @@ result.insert_into("outputTable")
    -In the following we describe the parameters of the `QueryConfig` and how they affect the accuracy and resource consumption of a query. +In the following we describe the parameters of the `TableConfig` and how they affect the accuracy and resource consumption of a query. Idle State Retention Time ------------------------- @@ -145,30 +145,30 @@ The parameters are specified as follows:
    {% highlight java %} -StreamQueryConfig qConfig = ... +TableConfig tConfig = ... // set idle state retention time: min = 12 hours, max = 24 hours -qConfig.withIdleStateRetentionTime(Time.hours(12), Time.hours(24)); +tConfig.setIdleStateRetentionTime(Time.hours(12), Time.hours(24)); {% endhighlight %}
    {% highlight scala %} -val qConfig: StreamQueryConfig = ??? +val tConfig: TableConfig = ??? // set idle state retention time: min = 12 hours, max = 24 hours -qConfig.withIdleStateRetentionTime(Time.hours(12), Time.hours(24)) +tConfig.setIdleStateRetentionTime(Time.hours(12), Time.hours(24)) {% endhighlight %}
    {% highlight python %} -q_config = ... # type: StreamQueryConfig +t_config = ... # type: TableConfig # set idle state retention time: min = 12 hours, max = 24 hours -q_config.with_idle_state_retention_time(timedelta(hours=12), timedelta(hours=24)) +t_config.set_idle_state_retention_time(timedelta(hours=12), timedelta(hours=24)) {% endhighlight %}
    -- Gitee From f0777fe29dee40e3af8d949894955a7e21b7ef15 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Fri, 13 Mar 2020 20:40:50 +0800 Subject: [PATCH 252/885] [FLINK-16608][python] Remove the method which derives the LogicalType from the Arrow ValueVector --- .../flink/table/runtime/arrow/ArrowUtils.java | 77 +++++-------------- .../ArrowPythonScalarFunctionOperator.java | 2 +- ...eRowArrowPythonScalarFunctionOperator.java | 2 +- .../ArrowPythonScalarFunctionRunner.java | 2 +- ...aseRowArrowPythonScalarFunctionRunner.java | 2 +- .../table/runtime/arrow/ArrowUtilsTest.java | 10 +-- .../arrow/BaseRowArrowReaderWriterTest.java | 4 +- .../arrow/RowArrowReaderWriterTest.java | 4 +- ...ArrowPythonScalarFunctionOperatorTest.java | 2 +- ...ArrowPythonScalarFunctionOperatorTest.java | 2 +- .../ArrowPythonScalarFunctionRunnerTest.java | 2 +- 11 files changed, 35 insertions(+), 74 deletions(-) diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java index 72212277d7..5e4c9c2b92 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java @@ -41,11 +41,9 @@ import org.apache.flink.table.runtime.arrow.writers.BigIntWriter; import org.apache.flink.table.runtime.arrow.writers.IntWriter; import org.apache.flink.table.runtime.arrow.writers.SmallIntWriter; import org.apache.flink.table.runtime.arrow.writers.TinyIntWriter; -import org.apache.flink.table.types.logical.ArrayType; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.MapType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.SmallIntType; import org.apache.flink.table.types.logical.TinyIntType; @@ -98,19 +96,19 @@ public final class ArrowUtils { /** * Creates an {@link ArrowWriter} for the specified {@link VectorSchemaRoot}. */ - public static ArrowWriter createRowArrowWriter(VectorSchemaRoot root) { + public static ArrowWriter createRowArrowWriter(VectorSchemaRoot root, RowType rowType) { ArrowFieldWriter[] fieldWriters = new ArrowFieldWriter[root.getFieldVectors().size()]; List vectors = root.getFieldVectors(); for (int i = 0; i < vectors.size(); i++) { FieldVector vector = vectors.get(i); vector.allocateNew(); - fieldWriters[i] = createRowArrowFieldWriter(vector); + fieldWriters[i] = createRowArrowFieldWriter(vector, rowType.getTypeAt(i)); } return new ArrowWriter<>(root, fieldWriters); } - private static ArrowFieldWriter createRowArrowFieldWriter(FieldVector vector) { + private static ArrowFieldWriter createRowArrowFieldWriter(FieldVector vector, LogicalType fieldType) { if (vector instanceof TinyIntVector) { return new TinyIntWriter((TinyIntVector) vector); } else if (vector instanceof SmallIntVector) { @@ -121,26 +119,26 @@ public final class ArrowUtils { return new BigIntWriter((BigIntVector) vector); } else { throw new UnsupportedOperationException(String.format( - "Unsupported type %s.", fromArrowFieldToLogicalType(vector.getField()))); + "Unsupported type %s.", fieldType)); } } /** * Creates an {@link ArrowWriter} for blink planner for the specified {@link VectorSchemaRoot}. */ - public static ArrowWriter createBaseRowArrowWriter(VectorSchemaRoot root) { + public static ArrowWriter createBaseRowArrowWriter(VectorSchemaRoot root, RowType rowType) { ArrowFieldWriter[] fieldWriters = new ArrowFieldWriter[root.getFieldVectors().size()]; List vectors = root.getFieldVectors(); for (int i = 0; i < vectors.size(); i++) { FieldVector vector = vectors.get(i); vector.allocateNew(); - fieldWriters[i] = createBaseRowArrowFieldWriter(vector); + fieldWriters[i] = createBaseRowArrowFieldWriter(vector, rowType.getTypeAt(i)); } return new ArrowWriter<>(root, fieldWriters); } - private static ArrowFieldWriter createBaseRowArrowFieldWriter(FieldVector vector) { + private static ArrowFieldWriter createBaseRowArrowFieldWriter(FieldVector vector, LogicalType fieldType) { if (vector instanceof TinyIntVector) { return new BaseRowTinyIntWriter((TinyIntVector) vector); } else if (vector instanceof SmallIntVector) { @@ -151,23 +149,24 @@ public final class ArrowUtils { return new BaseRowBigIntWriter((BigIntVector) vector); } else { throw new UnsupportedOperationException(String.format( - "Unsupported type %s.", fromArrowFieldToLogicalType(vector.getField()))); + "Unsupported type %s.", fieldType)); } } /** * Creates an {@link ArrowReader} for the specified {@link VectorSchemaRoot}. */ - public static RowArrowReader createRowArrowReader(VectorSchemaRoot root) { + public static RowArrowReader createRowArrowReader(VectorSchemaRoot root, RowType rowType) { List fieldReaders = new ArrayList<>(); - for (FieldVector vector : root.getFieldVectors()) { - fieldReaders.add(createRowArrowFieldReader(vector)); + List fieldVectors = root.getFieldVectors(); + for (int i = 0; i < fieldVectors.size(); i++) { + fieldReaders.add(createRowArrowFieldReader(fieldVectors.get(i), rowType.getTypeAt(i))); } return new RowArrowReader(fieldReaders.toArray(new ArrowFieldReader[0])); } - private static ArrowFieldReader createRowArrowFieldReader(FieldVector vector) { + private static ArrowFieldReader createRowArrowFieldReader(FieldVector vector, LogicalType fieldType) { if (vector instanceof TinyIntVector) { return new TinyIntFieldReader((TinyIntVector) vector); } else if (vector instanceof SmallIntVector) { @@ -178,23 +177,24 @@ public final class ArrowUtils { return new BigIntFieldReader((BigIntVector) vector); } else { throw new UnsupportedOperationException(String.format( - "Unsupported type %s.", fromArrowFieldToLogicalType(vector.getField()))); + "Unsupported type %s.", fieldType)); } } /** * Creates an {@link ArrowReader} for blink planner for the specified {@link VectorSchemaRoot}. */ - public static BaseRowArrowReader createBaseRowArrowReader(VectorSchemaRoot root) { + public static BaseRowArrowReader createBaseRowArrowReader(VectorSchemaRoot root, RowType rowType) { List columnVectors = new ArrayList<>(); - for (FieldVector vector : root.getFieldVectors()) { - columnVectors.add(createColumnVector(vector)); + List fieldVectors = root.getFieldVectors(); + for (int i = 0; i < fieldVectors.size(); i++) { + columnVectors.add(createColumnVector(fieldVectors.get(i), rowType.getTypeAt(i))); } return new BaseRowArrowReader(columnVectors.toArray(new ColumnVector[0])); } - private static ColumnVector createColumnVector(FieldVector vector) { + private static ColumnVector createColumnVector(FieldVector vector, LogicalType fieldType) { if (vector instanceof TinyIntVector) { return new ArrowTinyIntColumnVector((TinyIntVector) vector); } else if (vector instanceof SmallIntVector) { @@ -205,47 +205,10 @@ public final class ArrowUtils { return new ArrowBigIntColumnVector((BigIntVector) vector); } else { throw new UnsupportedOperationException(String.format( - "Unsupported type %s.", fromArrowFieldToLogicalType(vector.getField()))); + "Unsupported type %s.", fieldType)); } } - public static LogicalType fromArrowFieldToLogicalType(Field field) { - if (field.getType() == ArrowType.List.INSTANCE) { - LogicalType elementType = fromArrowFieldToLogicalType(field.getChildren().get(0)); - return new ArrayType(field.isNullable(), elementType); - } else if (field.getType() == ArrowType.Struct.INSTANCE) { - List fields = field.getChildren().stream().map(child -> { - LogicalType type = fromArrowFieldToLogicalType(child); - return new RowType.RowField(child.getName(), type, null); - }).collect(Collectors.toList()); - return new RowType(field.isNullable(), fields); - } else if (field.getType() instanceof ArrowType.Map) { - Field elementField = field.getChildren().get(0); - LogicalType keyType = fromArrowFieldToLogicalType(elementField.getChildren().get(0)); - LogicalType valueType = fromArrowFieldToLogicalType(elementField.getChildren().get(1)); - return new MapType(field.isNullable(), keyType, valueType); - } else { - return fromArrowTypeToLogicalType(field.isNullable(), field.getType()); - } - } - - private static LogicalType fromArrowTypeToLogicalType(boolean isNullable, ArrowType arrowType) { - if (arrowType instanceof ArrowType.Int && ((ArrowType.Int) arrowType).getIsSigned()) { - ArrowType.Int intType = (ArrowType.Int) arrowType; - if (intType.getBitWidth() == 8) { - return new TinyIntType(isNullable); - } else if (intType.getBitWidth() == 8 * 2) { - return new SmallIntType(isNullable); - } else if (intType.getBitWidth() == 8 * 4) { - return new IntType(isNullable); - } else if (intType.getBitWidth() == 8 * 8) { - return new BigIntType(isNullable); - } - } - throw new UnsupportedOperationException( - String.format("Unexpected arrow type: %s.", arrowType.toString())); - } - private static class LogicalTypeToArrowTypeConverter extends LogicalTypeDefaultVisitor { private static final LogicalTypeToArrowTypeConverter INSTANCE = new LogicalTypeToArrowTypeConverter(); diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperator.java index 4a686e8442..aac9294e94 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperator.java @@ -116,7 +116,7 @@ public class ArrowPythonScalarFunctionOperator extends AbstractRowPythonScalarFu reader.loadNextBatch(); VectorSchemaRoot root = reader.getVectorSchemaRoot(); if (arrowReader == null) { - arrowReader = ArrowUtils.createRowArrowReader(root); + arrowReader = ArrowUtils.createRowArrowReader(root, outputType); } for (int i = 0; i < root.getRowCount(); i++) { CRow input = forwardedInputQueue.poll(); diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/BaseRowArrowPythonScalarFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/BaseRowArrowPythonScalarFunctionOperator.java index 6a20fe69ba..bab87e6e1f 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/BaseRowArrowPythonScalarFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/BaseRowArrowPythonScalarFunctionOperator.java @@ -115,7 +115,7 @@ public class BaseRowArrowPythonScalarFunctionOperator extends AbstractBaseRowPyt reader.loadNextBatch(); VectorSchemaRoot root = reader.getVectorSchemaRoot(); if (arrowReader == null) { - arrowReader = ArrowUtils.createBaseRowArrowReader(root); + arrowReader = ArrowUtils.createBaseRowArrowReader(root, outputType); } for (int i = 0; i < root.getRowCount(); i++) { BaseRow input = forwardedInputQueue.poll(); diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/ArrowPythonScalarFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/ArrowPythonScalarFunctionRunner.java index df3d291aac..09f8a12da4 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/ArrowPythonScalarFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/ArrowPythonScalarFunctionRunner.java @@ -53,6 +53,6 @@ public class ArrowPythonScalarFunctionRunner extends AbstractArrowPythonScalarFu @Override public ArrowWriter createArrowWriter() { - return ArrowUtils.createRowArrowWriter(root); + return ArrowUtils.createRowArrowWriter(root, getInputType()); } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/BaseRowArrowPythonScalarFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/BaseRowArrowPythonScalarFunctionRunner.java index 60e9609334..848660b9d2 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/BaseRowArrowPythonScalarFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/BaseRowArrowPythonScalarFunctionRunner.java @@ -53,6 +53,6 @@ public class BaseRowArrowPythonScalarFunctionRunner extends AbstractArrowPythonS @Override public ArrowWriter createArrowWriter() { - return ArrowUtils.createBaseRowArrowWriter(root); + return ArrowUtils.createBaseRowArrowWriter(root, getInputType()); } } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java index 1e5bcbd16c..9de7b2efc1 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java @@ -103,15 +103,13 @@ public class ArrowUtilsTest { // verify convert from RowType to ArrowType assertEquals(testFields.get(i).f0, fields.get(i).getName()); assertEquals(testFields.get(i).f2, fields.get(i).getType()); - // verify convert from ArrowType to LogicalType - assertEquals(testFields.get(i).f1, ArrowUtils.fromArrowFieldToLogicalType(fields.get(i))); } } @Test public void testCreateRowArrowReader() { VectorSchemaRoot root = VectorSchemaRoot.create(ArrowUtils.toArrowSchema(rowType), allocator); - RowArrowReader reader = ArrowUtils.createRowArrowReader(root); + RowArrowReader reader = ArrowUtils.createRowArrowReader(root, rowType); ArrowFieldReader[] fieldReaders = reader.getFieldReaders(); for (int i = 0; i < fieldReaders.length; i++) { assertEquals(testFields.get(i).f5, fieldReaders[i].getClass()); @@ -121,7 +119,7 @@ public class ArrowUtilsTest { @Test public void testCreateBaseRowArrowReader() { VectorSchemaRoot root = VectorSchemaRoot.create(ArrowUtils.toArrowSchema(rowType), allocator); - BaseRowArrowReader reader = ArrowUtils.createBaseRowArrowReader(root); + BaseRowArrowReader reader = ArrowUtils.createBaseRowArrowReader(root, rowType); ColumnVector[] columnVectors = reader.getColumnVectors(); for (int i = 0; i < columnVectors.length; i++) { assertEquals(testFields.get(i).f6, columnVectors[i].getClass()); @@ -131,7 +129,7 @@ public class ArrowUtilsTest { @Test public void testCreateRowArrowWriter() { VectorSchemaRoot root = VectorSchemaRoot.create(ArrowUtils.toArrowSchema(rowType), allocator); - ArrowWriter writer = ArrowUtils.createRowArrowWriter(root); + ArrowWriter writer = ArrowUtils.createRowArrowWriter(root, rowType); ArrowFieldWriter[] fieldWriters = writer.getFieldWriters(); for (int i = 0; i < fieldWriters.length; i++) { assertEquals(testFields.get(i).f3, fieldWriters[i].getClass()); @@ -141,7 +139,7 @@ public class ArrowUtilsTest { @Test public void testCreateBaseRowArrowWriter() { VectorSchemaRoot root = VectorSchemaRoot.create(ArrowUtils.toArrowSchema(rowType), allocator); - ArrowWriter writer = ArrowUtils.createBaseRowArrowWriter(root); + ArrowWriter writer = ArrowUtils.createBaseRowArrowWriter(root, rowType); ArrowFieldWriter[] fieldWriters = writer.getFieldWriters(); for (int i = 0; i < fieldWriters.length; i++) { assertEquals(testFields.get(i).f4, fieldWriters[i].getClass()); diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java index 13f211b31b..ee936cd86c 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java @@ -99,13 +99,13 @@ public class BaseRowArrowReaderWriterTest extends ArrowReaderWriterTestBase createArrowReader(InputStream inputStream) throws IOException { ArrowStreamReader reader = new ArrowStreamReader(inputStream, allocator); reader.loadNextBatch(); - return ArrowUtils.createBaseRowArrowReader(reader.getVectorSchemaRoot()); + return ArrowUtils.createBaseRowArrowReader(reader.getVectorSchemaRoot(), rowType); } @Override public Tuple2, ArrowStreamWriter> createArrowWriter(OutputStream outputStream) throws IOException { VectorSchemaRoot root = VectorSchemaRoot.create(ArrowUtils.toArrowSchema(rowType), allocator); - ArrowWriter arrowWriter = ArrowUtils.createBaseRowArrowWriter(root); + ArrowWriter arrowWriter = ArrowUtils.createBaseRowArrowWriter(root, rowType); ArrowStreamWriter arrowStreamWriter = new ArrowStreamWriter(root, null, outputStream); arrowStreamWriter.start(); return Tuple2.of(arrowWriter, arrowStreamWriter); diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/RowArrowReaderWriterTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/RowArrowReaderWriterTest.java index d943b2be68..2276526e2f 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/RowArrowReaderWriterTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/RowArrowReaderWriterTest.java @@ -66,13 +66,13 @@ public class RowArrowReaderWriterTest extends ArrowReaderWriterTestBase { public ArrowReader createArrowReader(InputStream inputStream) throws IOException { ArrowStreamReader reader = new ArrowStreamReader(inputStream, allocator); reader.loadNextBatch(); - return ArrowUtils.createRowArrowReader(reader.getVectorSchemaRoot()); + return ArrowUtils.createRowArrowReader(reader.getVectorSchemaRoot(), rowType); } @Override public Tuple2, ArrowStreamWriter> createArrowWriter(OutputStream outputStream) throws IOException { VectorSchemaRoot root = VectorSchemaRoot.create(ArrowUtils.toArrowSchema(rowType), allocator); - ArrowWriter arrowWriter = ArrowUtils.createRowArrowWriter(root); + ArrowWriter arrowWriter = ArrowUtils.createRowArrowWriter(root, rowType); ArrowStreamWriter arrowStreamWriter = new ArrowStreamWriter(root, null, outputStream); arrowStreamWriter.start(); return Tuple2.of(arrowWriter, arrowStreamWriter); diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperatorTest.java index affca785e5..f4e6b2e624 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperatorTest.java @@ -103,7 +103,7 @@ public class ArrowPythonScalarFunctionOperatorTest extends PythonScalarFunctionO jobOptions) { @Override public ArrowWriter createArrowWriter() { - return ArrowUtils.createRowArrowWriter(root); + return ArrowUtils.createRowArrowWriter(root, getInputType()); } }; } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/BaseRowArrowPythonScalarFunctionOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/BaseRowArrowPythonScalarFunctionOperatorTest.java index f32bf32d16..6fae4c953e 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/BaseRowArrowPythonScalarFunctionOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/BaseRowArrowPythonScalarFunctionOperatorTest.java @@ -125,7 +125,7 @@ public class BaseRowArrowPythonScalarFunctionOperatorTest jobOptions) { @Override public ArrowWriter createArrowWriter() { - return ArrowUtils.createBaseRowArrowWriter(root); + return ArrowUtils.createBaseRowArrowWriter(root, getInputType()); } }; } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/ArrowPythonScalarFunctionRunnerTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/ArrowPythonScalarFunctionRunnerTest.java index 208b5bf22d..9d802e1497 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/ArrowPythonScalarFunctionRunnerTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/ArrowPythonScalarFunctionRunnerTest.java @@ -177,7 +177,7 @@ public class ArrowPythonScalarFunctionRunnerTest extends AbstractPythonScalarFun jobBundleFactory) { @Override public ArrowWriter createArrowWriter() { - return ArrowUtils.createRowArrowWriter(root); + return ArrowUtils.createRowArrowWriter(root, getInputType()); } }; } -- Gitee From 09fcfab27ade7035b8d2d3453863e401ad760f43 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Thu, 12 Mar 2020 14:48:20 +0800 Subject: [PATCH 253/885] [FLINK-16608][python] Support BooleanType in vectorized Python UDF --- flink-python/pyflink/fn_execution/coders.py | 2 + .../pyflink/table/tests/test_pandas_udf.py | 27 +++++++--- .../flink/table/runtime/arrow/ArrowUtils.java | 19 +++++++ .../arrow/readers/BooleanFieldReader.java | 39 ++++++++++++++ .../vectors/ArrowBooleanColumnVector.java | 51 +++++++++++++++++++ .../arrow/writers/BaseRowBooleanWriter.java | 46 +++++++++++++++++ .../runtime/arrow/writers/BooleanWriter.java | 46 +++++++++++++++++ .../table/runtime/arrow/ArrowUtilsTest.java | 7 +++ .../arrow/BaseRowArrowReaderWriterTest.java | 14 ++--- .../arrow/RowArrowReaderWriterTest.java | 14 ++--- 10 files changed, 246 insertions(+), 19 deletions(-) create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/BooleanFieldReader.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowBooleanColumnVector.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowBooleanWriter.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BooleanWriter.java diff --git a/flink-python/pyflink/fn_execution/coders.py b/flink-python/pyflink/fn_execution/coders.py index da4f03b7a0..fd10b921a7 100644 --- a/flink-python/pyflink/fn_execution/coders.py +++ b/flink-python/pyflink/fn_execution/coders.py @@ -422,6 +422,8 @@ class ArrowCoder(DeterministicCoder): return pa.field(field.name, pa.int32(), field.type.nullable) elif field.type.type_name == flink_fn_execution_pb2.Schema.TypeName.BIGINT: return pa.field(field.name, pa.int64(), field.type.nullable) + elif field.type.type_name == flink_fn_execution_pb2.Schema.TypeName.BOOLEAN: + return pa.field(field.name, pa.bool_(), field.type.nullable) else: raise ValueError("field_type %s is not supported." % field.type) diff --git a/flink-python/pyflink/table/tests/test_pandas_udf.py b/flink-python/pyflink/table/tests/test_pandas_udf.py index 188285d5be..e8c398f76e 100644 --- a/flink-python/pyflink/table/tests/test_pandas_udf.py +++ b/flink-python/pyflink/table/tests/test_pandas_udf.py @@ -91,6 +91,12 @@ class PandasUDFITTests(object): 'bigint_param of wrong type %s !' % type(bigint_param[0]) return bigint_param + def boolean_func(boolean_param): + assert isinstance(boolean_param, pd.Series) + assert isinstance(boolean_param[0], np.bool_), \ + 'boolean_param of wrong type %s !' % type(boolean_param[0]) + return boolean_param + self.t_env.register_function( "tinyint_func", udf(tinyint_func, [DataTypes.TINYINT()], DataTypes.TINYINT(), udf_type="pandas")) @@ -107,28 +113,37 @@ class PandasUDFITTests(object): "bigint_func", udf(bigint_func, [DataTypes.BIGINT()], DataTypes.BIGINT(), udf_type="pandas")) + self.t_env.register_function( + "boolean_func", + udf(boolean_func, [DataTypes.BOOLEAN()], DataTypes.BOOLEAN(), udf_type="pandas")) + table_sink = source_sink_utils.TestAppendSink( - ['a', 'b', 'c', 'd'], - [DataTypes.TINYINT(), DataTypes.SMALLINT(), DataTypes.INT(), DataTypes.BIGINT()]) + ['a', 'b', 'c', 'd', 'e', 'f'], + [DataTypes.TINYINT(), DataTypes.SMALLINT(), DataTypes.INT(), DataTypes.BIGINT(), + DataTypes.BOOLEAN(), DataTypes.BOOLEAN()]) self.t_env.register_table_sink("Results", table_sink) t = self.t_env.from_elements( - [(1, 32767, -2147483648, 1)], + [(1, 32767, -2147483648, 1, True, False)], DataTypes.ROW( [DataTypes.FIELD("a", DataTypes.TINYINT()), DataTypes.FIELD("b", DataTypes.SMALLINT()), DataTypes.FIELD("c", DataTypes.INT()), - DataTypes.FIELD("d", DataTypes.BIGINT())])) + DataTypes.FIELD("d", DataTypes.BIGINT()), + DataTypes.FIELD("e", DataTypes.BOOLEAN()), + DataTypes.FIELD("f", DataTypes.BOOLEAN())])) t.select("tinyint_func(a)," "smallint_func(b)," "int_func(c)," - "bigint_func(d)") \ + "bigint_func(d)," + "boolean_func(e)," + "boolean_func(f)") \ .insert_into("Results") self.t_env.execute("test") actual = source_sink_utils.results() self.assert_equals(actual, - ["1,32767,-2147483648,1"]) + ["1,32767,-2147483648,1,true,false"]) class StreamPandasUDFITTests(PandasUDFITTests, diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java index 5e4c9c2b92..3f8d661f5c 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java @@ -23,25 +23,30 @@ import org.apache.flink.table.dataformat.BaseRow; import org.apache.flink.table.dataformat.vector.ColumnVector; import org.apache.flink.table.runtime.arrow.readers.ArrowFieldReader; import org.apache.flink.table.runtime.arrow.readers.BigIntFieldReader; +import org.apache.flink.table.runtime.arrow.readers.BooleanFieldReader; import org.apache.flink.table.runtime.arrow.readers.IntFieldReader; import org.apache.flink.table.runtime.arrow.readers.RowArrowReader; import org.apache.flink.table.runtime.arrow.readers.SmallIntFieldReader; import org.apache.flink.table.runtime.arrow.readers.TinyIntFieldReader; import org.apache.flink.table.runtime.arrow.vectors.ArrowBigIntColumnVector; +import org.apache.flink.table.runtime.arrow.vectors.ArrowBooleanColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowSmallIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowTinyIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.BaseRowArrowReader; import org.apache.flink.table.runtime.arrow.writers.ArrowFieldWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowBigIntWriter; +import org.apache.flink.table.runtime.arrow.writers.BaseRowBooleanWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowSmallIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowTinyIntWriter; import org.apache.flink.table.runtime.arrow.writers.BigIntWriter; +import org.apache.flink.table.runtime.arrow.writers.BooleanWriter; import org.apache.flink.table.runtime.arrow.writers.IntWriter; import org.apache.flink.table.runtime.arrow.writers.SmallIntWriter; import org.apache.flink.table.runtime.arrow.writers.TinyIntWriter; import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.BooleanType; import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; @@ -52,6 +57,7 @@ import org.apache.flink.types.Row; import org.apache.arrow.memory.RootAllocator; import org.apache.arrow.vector.BigIntVector; +import org.apache.arrow.vector.BitVector; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.IntVector; import org.apache.arrow.vector.SmallIntVector; @@ -117,6 +123,8 @@ public final class ArrowUtils { return new IntWriter((IntVector) vector); } else if (vector instanceof BigIntVector) { return new BigIntWriter((BigIntVector) vector); + } else if (vector instanceof BitVector) { + return new BooleanWriter((BitVector) vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -147,6 +155,8 @@ public final class ArrowUtils { return new BaseRowIntWriter((IntVector) vector); } else if (vector instanceof BigIntVector) { return new BaseRowBigIntWriter((BigIntVector) vector); + } else if (vector instanceof BitVector) { + return new BaseRowBooleanWriter((BitVector) vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -175,6 +185,8 @@ public final class ArrowUtils { return new IntFieldReader((IntVector) vector); } else if (vector instanceof BigIntVector) { return new BigIntFieldReader((BigIntVector) vector); + } else if (vector instanceof BitVector) { + return new BooleanFieldReader((BitVector) vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -203,6 +215,8 @@ public final class ArrowUtils { return new ArrowIntColumnVector((IntVector) vector); } else if (vector instanceof BigIntVector) { return new ArrowBigIntColumnVector((BigIntVector) vector); + } else if (vector instanceof BitVector) { + return new ArrowBooleanColumnVector((BitVector) vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -233,6 +247,11 @@ public final class ArrowUtils { return new ArrowType.Int(8 * 8, true); } + @Override + public ArrowType visit(BooleanType booleanType) { + return ArrowType.Bool.INSTANCE; + } + @Override protected ArrowType defaultMethod(LogicalType logicalType) { throw new UnsupportedOperationException(String.format( diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/BooleanFieldReader.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/BooleanFieldReader.java new file mode 100644 index 0000000000..a397b8a453 --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/BooleanFieldReader.java @@ -0,0 +1,39 @@ +/* + * 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.table.runtime.arrow.readers; + +import org.apache.flink.annotation.Internal; + +import org.apache.arrow.vector.BitVector; + +/** + * {@link ArrowFieldReader} for Boolean. + */ +@Internal +public final class BooleanFieldReader extends ArrowFieldReader { + + public BooleanFieldReader(BitVector bitVector) { + super(bitVector); + } + + @Override + public Boolean read(int index) { + return ((BitVector) getValueVector()).getObject(index); + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowBooleanColumnVector.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowBooleanColumnVector.java new file mode 100644 index 0000000000..f97eaa8c7f --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowBooleanColumnVector.java @@ -0,0 +1,51 @@ +/* + * 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.table.runtime.arrow.vectors; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.dataformat.vector.BooleanColumnVector; +import org.apache.flink.util.Preconditions; + +import org.apache.arrow.vector.BitVector; + +/** + * Arrow column vector for Boolean. + */ +@Internal +public final class ArrowBooleanColumnVector implements BooleanColumnVector { + + /** + * Container which is used to store the sequence of boolean values of a column to read. + */ + private final BitVector bitVector; + + public ArrowBooleanColumnVector(BitVector bitVector) { + this.bitVector = Preconditions.checkNotNull(bitVector); + } + + @Override + public boolean getBoolean(int i) { + return bitVector.get(i) != 0; + } + + @Override + public boolean isNullAt(int i) { + return bitVector.isNull(i); + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowBooleanWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowBooleanWriter.java new file mode 100644 index 0000000000..ac338f50df --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowBooleanWriter.java @@ -0,0 +1,46 @@ +/* + * 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.table.runtime.arrow.writers; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.dataformat.BaseRow; + +import org.apache.arrow.vector.BitVector; + +/** + * {@link ArrowFieldWriter} for Boolean. + */ +@Internal +public final class BaseRowBooleanWriter extends ArrowFieldWriter { + + public BaseRowBooleanWriter(BitVector bitVector) { + super(bitVector); + } + + @Override + public void doWrite(BaseRow row, int ordinal) { + if (row.isNullAt(ordinal)) { + ((BitVector) getValueVector()).setNull(getCount()); + } else if (row.getBoolean(ordinal)) { + ((BitVector) getValueVector()).setSafe(getCount(), 1); + } else { + ((BitVector) getValueVector()).setSafe(getCount(), 0); + } + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BooleanWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BooleanWriter.java new file mode 100644 index 0000000000..37d9fddd50 --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BooleanWriter.java @@ -0,0 +1,46 @@ +/* + * 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.table.runtime.arrow.writers; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.types.Row; + +import org.apache.arrow.vector.BitVector; + +/** + * {@link ArrowFieldWriter} for Boolean. + */ +@Internal +public final class BooleanWriter extends ArrowFieldWriter { + + public BooleanWriter(BitVector bitVector) { + super(bitVector); + } + + @Override + public void doWrite(Row value, int ordinal) { + if (value.getField(ordinal) == null) { + ((BitVector) getValueVector()).setNull(getCount()); + } else if ((boolean) value.getField(ordinal)) { + ((BitVector) getValueVector()).setSafe(getCount(), 1); + } else { + ((BitVector) getValueVector()).setSafe(getCount(), 0); + } + } +} diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java index 9de7b2efc1..59f7eda8bc 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java @@ -23,25 +23,30 @@ import org.apache.flink.table.dataformat.BaseRow; import org.apache.flink.table.dataformat.vector.ColumnVector; import org.apache.flink.table.runtime.arrow.readers.ArrowFieldReader; import org.apache.flink.table.runtime.arrow.readers.BigIntFieldReader; +import org.apache.flink.table.runtime.arrow.readers.BooleanFieldReader; import org.apache.flink.table.runtime.arrow.readers.IntFieldReader; import org.apache.flink.table.runtime.arrow.readers.RowArrowReader; import org.apache.flink.table.runtime.arrow.readers.SmallIntFieldReader; import org.apache.flink.table.runtime.arrow.readers.TinyIntFieldReader; import org.apache.flink.table.runtime.arrow.vectors.ArrowBigIntColumnVector; +import org.apache.flink.table.runtime.arrow.vectors.ArrowBooleanColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowSmallIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowTinyIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.BaseRowArrowReader; import org.apache.flink.table.runtime.arrow.writers.ArrowFieldWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowBigIntWriter; +import org.apache.flink.table.runtime.arrow.writers.BaseRowBooleanWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowSmallIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowTinyIntWriter; import org.apache.flink.table.runtime.arrow.writers.BigIntWriter; +import org.apache.flink.table.runtime.arrow.writers.BooleanWriter; import org.apache.flink.table.runtime.arrow.writers.IntWriter; import org.apache.flink.table.runtime.arrow.writers.SmallIntWriter; import org.apache.flink.table.runtime.arrow.writers.TinyIntWriter; import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.BooleanType; import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; @@ -83,6 +88,8 @@ public class ArrowUtilsTest { IntWriter.class, BaseRowIntWriter.class, IntFieldReader.class, ArrowIntColumnVector.class)); testFields.add(Tuple7.of("f4", new BigIntType(), new ArrowType.Int(8 * 8, true), BigIntWriter.class, BaseRowBigIntWriter.class, BigIntFieldReader.class, ArrowBigIntColumnVector.class)); + testFields.add(Tuple7.of("f5", new BooleanType(), new ArrowType.Bool(), + BooleanWriter.class, BaseRowBooleanWriter.class, BooleanFieldReader.class, ArrowBooleanColumnVector.class)); List rowFields = new ArrayList<>(); for (Tuple7, Class, Class, Class> field : testFields) { diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java index ee936cd86c..346910606f 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java @@ -25,6 +25,7 @@ import org.apache.flink.table.dataformat.BinaryRow; import org.apache.flink.table.runtime.typeutils.BaseRowSerializer; import org.apache.flink.table.runtime.util.StreamRecordUtils; import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.BooleanType; import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; @@ -86,6 +87,7 @@ public class BaseRowArrowReaderWriterTest extends ArrowReaderWriterTestBase rowFields = new ArrayList<>(); for (int i = 0; i < fieldTypes.size(); i++) { @@ -113,12 +115,12 @@ public class BaseRowArrowReaderWriterTest extends ArrowReaderWriterTestBase { fieldTypes.add(new SmallIntType()); fieldTypes.add(new IntType()); fieldTypes.add(new BigIntType()); + fieldTypes.add(new BooleanType()); List rowFields = new ArrayList<>(); for (int i = 0; i < fieldTypes.size(); i++) { @@ -80,12 +82,10 @@ public class RowArrowReaderWriterTest extends ArrowReaderWriterTestBase { @Override public Row[] getTestData() { - Row row1 = Row.of((byte) 1, (short) 2, 3, 4L); - Row row2 = Row.of((byte) 1, (short) 2, 3, 4L); - Row row3 = Row.of(null, (short) 2, 3, 4L); - Row row4 = Row.of((byte) 1, null, 3, 4L); - Row row5 = Row.of(null, null, null, null); - Row row6 = Row.of(null, null, null, null); - return new Row[]{row1, row2, row3, row4, row5, row6}; + Row row1 = Row.of((byte) 1, (short) 2, 3, 4L, true); + Row row2 = Row.of(null, (short) 2, 3, 4L, false); + Row row3 = Row.of((byte) 1, null, 3, 4L, true); + Row row4 = Row.of(null, null, null, null, null); + return new Row[]{row1, row2, row3, row4}; } } -- Gitee From 63ba2017915c59e782b3f1d13215ea91ee2a8315 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Thu, 12 Mar 2020 15:10:58 +0800 Subject: [PATCH 254/885] [FLINK-16608][python] Support FloatType in vectorized Python UDF --- flink-python/pyflink/fn_execution/coders.py | 2 + .../pyflink/table/tests/test_pandas_udf.py | 24 ++++++--- .../flink/table/runtime/arrow/ArrowUtils.java | 20 ++++++++ .../arrow/readers/FloatFieldReader.java | 39 ++++++++++++++ .../arrow/vectors/ArrowFloatColumnVector.java | 51 +++++++++++++++++++ .../arrow/writers/BaseRowFloatWriter.java | 44 ++++++++++++++++ .../runtime/arrow/writers/FloatWriter.java | 44 ++++++++++++++++ .../table/runtime/arrow/ArrowUtilsTest.java | 8 +++ .../arrow/BaseRowArrowReaderWriterTest.java | 14 ++--- .../arrow/RowArrowReaderWriterTest.java | 10 ++-- 10 files changed, 240 insertions(+), 16 deletions(-) create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/FloatFieldReader.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowFloatColumnVector.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowFloatWriter.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/FloatWriter.java diff --git a/flink-python/pyflink/fn_execution/coders.py b/flink-python/pyflink/fn_execution/coders.py index fd10b921a7..73cc1d8765 100644 --- a/flink-python/pyflink/fn_execution/coders.py +++ b/flink-python/pyflink/fn_execution/coders.py @@ -424,6 +424,8 @@ class ArrowCoder(DeterministicCoder): return pa.field(field.name, pa.int64(), field.type.nullable) elif field.type.type_name == flink_fn_execution_pb2.Schema.TypeName.BOOLEAN: return pa.field(field.name, pa.bool_(), field.type.nullable) + elif field.type.type_name == flink_fn_execution_pb2.Schema.TypeName.FLOAT: + return pa.field(field.name, pa.float32(), field.type.nullable) else: raise ValueError("field_type %s is not supported." % field.type) diff --git a/flink-python/pyflink/table/tests/test_pandas_udf.py b/flink-python/pyflink/table/tests/test_pandas_udf.py index e8c398f76e..ab3d2f157a 100644 --- a/flink-python/pyflink/table/tests/test_pandas_udf.py +++ b/flink-python/pyflink/table/tests/test_pandas_udf.py @@ -97,6 +97,12 @@ class PandasUDFITTests(object): 'boolean_param of wrong type %s !' % type(boolean_param[0]) return boolean_param + def float_func(float_param): + assert isinstance(float_param, pd.Series) + assert isinstance(float_param[0], np.float32), \ + 'float_param of wrong type %s !' % type(float_param[0]) + return float_param + self.t_env.register_function( "tinyint_func", udf(tinyint_func, [DataTypes.TINYINT()], DataTypes.TINYINT(), udf_type="pandas")) @@ -117,33 +123,39 @@ class PandasUDFITTests(object): "boolean_func", udf(boolean_func, [DataTypes.BOOLEAN()], DataTypes.BOOLEAN(), udf_type="pandas")) + self.t_env.register_function( + "float_func", + udf(float_func, [DataTypes.FLOAT()], DataTypes.FLOAT(), udf_type="pandas")) + table_sink = source_sink_utils.TestAppendSink( - ['a', 'b', 'c', 'd', 'e', 'f'], + ['a', 'b', 'c', 'd', 'e', 'f', 'g'], [DataTypes.TINYINT(), DataTypes.SMALLINT(), DataTypes.INT(), DataTypes.BIGINT(), - DataTypes.BOOLEAN(), DataTypes.BOOLEAN()]) + DataTypes.BOOLEAN(), DataTypes.BOOLEAN(), DataTypes.FLOAT()]) self.t_env.register_table_sink("Results", table_sink) t = self.t_env.from_elements( - [(1, 32767, -2147483648, 1, True, False)], + [(1, 32767, -2147483648, 1, True, False, 1.0)], DataTypes.ROW( [DataTypes.FIELD("a", DataTypes.TINYINT()), DataTypes.FIELD("b", DataTypes.SMALLINT()), DataTypes.FIELD("c", DataTypes.INT()), DataTypes.FIELD("d", DataTypes.BIGINT()), DataTypes.FIELD("e", DataTypes.BOOLEAN()), - DataTypes.FIELD("f", DataTypes.BOOLEAN())])) + DataTypes.FIELD("f", DataTypes.BOOLEAN()), + DataTypes.FIELD("g", DataTypes.FLOAT())])) t.select("tinyint_func(a)," "smallint_func(b)," "int_func(c)," "bigint_func(d)," "boolean_func(e)," - "boolean_func(f)") \ + "boolean_func(f)," + "float_func(g)") \ .insert_into("Results") self.t_env.execute("test") actual = source_sink_utils.results() self.assert_equals(actual, - ["1,32767,-2147483648,1,true,false"]) + ["1,32767,-2147483648,1,true,false,1.0"]) class StreamPandasUDFITTests(PandasUDFITTests, diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java index 3f8d661f5c..0b29f3a59f 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java @@ -24,12 +24,14 @@ import org.apache.flink.table.dataformat.vector.ColumnVector; import org.apache.flink.table.runtime.arrow.readers.ArrowFieldReader; import org.apache.flink.table.runtime.arrow.readers.BigIntFieldReader; import org.apache.flink.table.runtime.arrow.readers.BooleanFieldReader; +import org.apache.flink.table.runtime.arrow.readers.FloatFieldReader; import org.apache.flink.table.runtime.arrow.readers.IntFieldReader; import org.apache.flink.table.runtime.arrow.readers.RowArrowReader; import org.apache.flink.table.runtime.arrow.readers.SmallIntFieldReader; import org.apache.flink.table.runtime.arrow.readers.TinyIntFieldReader; import org.apache.flink.table.runtime.arrow.vectors.ArrowBigIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowBooleanColumnVector; +import org.apache.flink.table.runtime.arrow.vectors.ArrowFloatColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowSmallIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowTinyIntColumnVector; @@ -37,16 +39,19 @@ import org.apache.flink.table.runtime.arrow.vectors.BaseRowArrowReader; import org.apache.flink.table.runtime.arrow.writers.ArrowFieldWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowBigIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowBooleanWriter; +import org.apache.flink.table.runtime.arrow.writers.BaseRowFloatWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowSmallIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowTinyIntWriter; import org.apache.flink.table.runtime.arrow.writers.BigIntWriter; import org.apache.flink.table.runtime.arrow.writers.BooleanWriter; +import org.apache.flink.table.runtime.arrow.writers.FloatWriter; import org.apache.flink.table.runtime.arrow.writers.IntWriter; import org.apache.flink.table.runtime.arrow.writers.SmallIntWriter; import org.apache.flink.table.runtime.arrow.writers.TinyIntWriter; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.FloatType; import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; @@ -59,10 +64,12 @@ import org.apache.arrow.memory.RootAllocator; import org.apache.arrow.vector.BigIntVector; import org.apache.arrow.vector.BitVector; import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.Float4Vector; import org.apache.arrow.vector.IntVector; import org.apache.arrow.vector.SmallIntVector; import org.apache.arrow.vector.TinyIntVector; import org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.arrow.vector.types.FloatingPointPrecision; import org.apache.arrow.vector.types.pojo.ArrowType; import org.apache.arrow.vector.types.pojo.Field; import org.apache.arrow.vector.types.pojo.FieldType; @@ -125,6 +132,8 @@ public final class ArrowUtils { return new BigIntWriter((BigIntVector) vector); } else if (vector instanceof BitVector) { return new BooleanWriter((BitVector) vector); + } else if (vector instanceof Float4Vector) { + return new FloatWriter((Float4Vector) vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -157,6 +166,8 @@ public final class ArrowUtils { return new BaseRowBigIntWriter((BigIntVector) vector); } else if (vector instanceof BitVector) { return new BaseRowBooleanWriter((BitVector) vector); + } else if (vector instanceof Float4Vector) { + return new BaseRowFloatWriter((Float4Vector) vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -187,6 +198,8 @@ public final class ArrowUtils { return new BigIntFieldReader((BigIntVector) vector); } else if (vector instanceof BitVector) { return new BooleanFieldReader((BitVector) vector); + } else if (vector instanceof Float4Vector) { + return new FloatFieldReader((Float4Vector) vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -217,6 +230,8 @@ public final class ArrowUtils { return new ArrowBigIntColumnVector((BigIntVector) vector); } else if (vector instanceof BitVector) { return new ArrowBooleanColumnVector((BitVector) vector); + } else if (vector instanceof Float4Vector) { + return new ArrowFloatColumnVector((Float4Vector) vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -252,6 +267,11 @@ public final class ArrowUtils { return ArrowType.Bool.INSTANCE; } + @Override + public ArrowType visit(FloatType floatType) { + return new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE); + } + @Override protected ArrowType defaultMethod(LogicalType logicalType) { throw new UnsupportedOperationException(String.format( diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/FloatFieldReader.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/FloatFieldReader.java new file mode 100644 index 0000000000..1c3a3ba95b --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/FloatFieldReader.java @@ -0,0 +1,39 @@ +/* + * 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.table.runtime.arrow.readers; + +import org.apache.flink.annotation.Internal; + +import org.apache.arrow.vector.Float4Vector; + +/** + * {@link ArrowFieldReader} for Float. + */ +@Internal +public final class FloatFieldReader extends ArrowFieldReader { + + public FloatFieldReader(Float4Vector floatVector) { + super(floatVector); + } + + @Override + public Float read(int index) { + return ((Float4Vector) getValueVector()).getObject(index); + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowFloatColumnVector.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowFloatColumnVector.java new file mode 100644 index 0000000000..4169145f66 --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowFloatColumnVector.java @@ -0,0 +1,51 @@ +/* + * 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.table.runtime.arrow.vectors; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.dataformat.vector.FloatColumnVector; +import org.apache.flink.util.Preconditions; + +import org.apache.arrow.vector.Float4Vector; + +/** + * Arrow column vector for Float. + */ +@Internal +public final class ArrowFloatColumnVector implements FloatColumnVector { + + /** + * Container which is used to store the sequence of float values of a column to read. + */ + private final Float4Vector floatVector; + + public ArrowFloatColumnVector(Float4Vector floatVector) { + this.floatVector = Preconditions.checkNotNull(floatVector); + } + + @Override + public float getFloat(int i) { + return floatVector.get(i); + } + + @Override + public boolean isNullAt(int i) { + return floatVector.isNull(i); + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowFloatWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowFloatWriter.java new file mode 100644 index 0000000000..7561176345 --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowFloatWriter.java @@ -0,0 +1,44 @@ +/* + * 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.table.runtime.arrow.writers; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.dataformat.BaseRow; + +import org.apache.arrow.vector.Float4Vector; + +/** + * {@link ArrowFieldWriter} for Float. + */ +@Internal +public final class BaseRowFloatWriter extends ArrowFieldWriter { + + public BaseRowFloatWriter(Float4Vector floatVector) { + super(floatVector); + } + + @Override + public void doWrite(BaseRow row, int ordinal) { + if (row.isNullAt(ordinal)) { + ((Float4Vector) getValueVector()).setNull(getCount()); + } else { + ((Float4Vector) getValueVector()).setSafe(getCount(), row.getFloat(ordinal)); + } + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/FloatWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/FloatWriter.java new file mode 100644 index 0000000000..5d08f01d40 --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/FloatWriter.java @@ -0,0 +1,44 @@ +/* + * 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.table.runtime.arrow.writers; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.types.Row; + +import org.apache.arrow.vector.Float4Vector; + +/** + * {@link ArrowFieldWriter} for Float. + */ +@Internal +public final class FloatWriter extends ArrowFieldWriter { + + public FloatWriter(Float4Vector floatVector) { + super(floatVector); + } + + @Override + public void doWrite(Row value, int ordinal) { + if (value.getField(ordinal) == null) { + ((Float4Vector) getValueVector()).setNull(getCount()); + } else { + ((Float4Vector) getValueVector()).setSafe(getCount(), (float) value.getField(ordinal)); + } + } +} diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java index 59f7eda8bc..14f943cab5 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java @@ -24,12 +24,14 @@ import org.apache.flink.table.dataformat.vector.ColumnVector; import org.apache.flink.table.runtime.arrow.readers.ArrowFieldReader; import org.apache.flink.table.runtime.arrow.readers.BigIntFieldReader; import org.apache.flink.table.runtime.arrow.readers.BooleanFieldReader; +import org.apache.flink.table.runtime.arrow.readers.FloatFieldReader; import org.apache.flink.table.runtime.arrow.readers.IntFieldReader; import org.apache.flink.table.runtime.arrow.readers.RowArrowReader; import org.apache.flink.table.runtime.arrow.readers.SmallIntFieldReader; import org.apache.flink.table.runtime.arrow.readers.TinyIntFieldReader; import org.apache.flink.table.runtime.arrow.vectors.ArrowBigIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowBooleanColumnVector; +import org.apache.flink.table.runtime.arrow.vectors.ArrowFloatColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowSmallIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowTinyIntColumnVector; @@ -37,16 +39,19 @@ import org.apache.flink.table.runtime.arrow.vectors.BaseRowArrowReader; import org.apache.flink.table.runtime.arrow.writers.ArrowFieldWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowBigIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowBooleanWriter; +import org.apache.flink.table.runtime.arrow.writers.BaseRowFloatWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowSmallIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowTinyIntWriter; import org.apache.flink.table.runtime.arrow.writers.BigIntWriter; import org.apache.flink.table.runtime.arrow.writers.BooleanWriter; +import org.apache.flink.table.runtime.arrow.writers.FloatWriter; import org.apache.flink.table.runtime.arrow.writers.IntWriter; import org.apache.flink.table.runtime.arrow.writers.SmallIntWriter; import org.apache.flink.table.runtime.arrow.writers.TinyIntWriter; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.FloatType; import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; @@ -56,6 +61,7 @@ import org.apache.flink.types.Row; import org.apache.arrow.memory.BufferAllocator; import org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.arrow.vector.types.FloatingPointPrecision; import org.apache.arrow.vector.types.pojo.ArrowType; import org.apache.arrow.vector.types.pojo.Field; import org.apache.arrow.vector.types.pojo.Schema; @@ -90,6 +96,8 @@ public class ArrowUtilsTest { BigIntWriter.class, BaseRowBigIntWriter.class, BigIntFieldReader.class, ArrowBigIntColumnVector.class)); testFields.add(Tuple7.of("f5", new BooleanType(), new ArrowType.Bool(), BooleanWriter.class, BaseRowBooleanWriter.class, BooleanFieldReader.class, ArrowBooleanColumnVector.class)); + testFields.add(Tuple7.of("f6", new FloatType(), new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE), + FloatWriter.class, BaseRowFloatWriter.class, FloatFieldReader.class, ArrowFloatColumnVector.class)); List rowFields = new ArrayList<>(); for (Tuple7, Class, Class, Class> field : testFields) { diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java index 346910606f..f6f736bed5 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java @@ -26,6 +26,7 @@ import org.apache.flink.table.runtime.typeutils.BaseRowSerializer; import org.apache.flink.table.runtime.util.StreamRecordUtils; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.FloatType; import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; @@ -88,6 +89,7 @@ public class BaseRowArrowReaderWriterTest extends ArrowReaderWriterTestBase rowFields = new ArrayList<>(); for (int i = 0; i < fieldTypes.size(); i++) { @@ -115,12 +117,12 @@ public class BaseRowArrowReaderWriterTest extends ArrowReaderWriterTestBase { fieldTypes.add(new IntType()); fieldTypes.add(new BigIntType()); fieldTypes.add(new BooleanType()); + fieldTypes.add(new FloatType()); List rowFields = new ArrayList<>(); for (int i = 0; i < fieldTypes.size(); i++) { @@ -82,10 +84,10 @@ public class RowArrowReaderWriterTest extends ArrowReaderWriterTestBase { @Override public Row[] getTestData() { - Row row1 = Row.of((byte) 1, (short) 2, 3, 4L, true); - Row row2 = Row.of(null, (short) 2, 3, 4L, false); - Row row3 = Row.of((byte) 1, null, 3, 4L, true); - Row row4 = Row.of(null, null, null, null, null); + Row row1 = Row.of((byte) 1, (short) 2, 3, 4L, true, 1.0f); + Row row2 = Row.of(null, (short) 2, 3, 4L, false, 1.0f); + Row row3 = Row.of((byte) 1, null, 3, 4L, true, 1.0f); + Row row4 = Row.of(null, null, null, null, null, null); return new Row[]{row1, row2, row3, row4}; } } -- Gitee From 180c2c9b8d163badbce8753478ee01a8d7bc9e0e Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Thu, 12 Mar 2020 15:23:38 +0800 Subject: [PATCH 255/885] [FLINK-16608][python] Support DoubleType in vectorized Python UDF --- flink-python/pyflink/fn_execution/coders.py | 2 + .../pyflink/table/tests/test_pandas_udf.py | 24 ++++++--- .../flink/table/runtime/arrow/ArrowUtils.java | 19 +++++++ .../arrow/readers/DoubleFieldReader.java | 39 ++++++++++++++ .../vectors/ArrowDoubleColumnVector.java | 51 +++++++++++++++++++ .../arrow/writers/BaseRowDoubleWriter.java | 44 ++++++++++++++++ .../runtime/arrow/writers/DoubleWriter.java | 44 ++++++++++++++++ .../table/runtime/arrow/ArrowUtilsTest.java | 7 +++ .../arrow/BaseRowArrowReaderWriterTest.java | 14 ++--- .../arrow/RowArrowReaderWriterTest.java | 10 ++-- 10 files changed, 238 insertions(+), 16 deletions(-) create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/DoubleFieldReader.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowDoubleColumnVector.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowDoubleWriter.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/DoubleWriter.java diff --git a/flink-python/pyflink/fn_execution/coders.py b/flink-python/pyflink/fn_execution/coders.py index 73cc1d8765..99a2622265 100644 --- a/flink-python/pyflink/fn_execution/coders.py +++ b/flink-python/pyflink/fn_execution/coders.py @@ -426,6 +426,8 @@ class ArrowCoder(DeterministicCoder): return pa.field(field.name, pa.bool_(), field.type.nullable) elif field.type.type_name == flink_fn_execution_pb2.Schema.TypeName.FLOAT: return pa.field(field.name, pa.float32(), field.type.nullable) + elif field.type.type_name == flink_fn_execution_pb2.Schema.TypeName.DOUBLE: + return pa.field(field.name, pa.float64(), field.type.nullable) else: raise ValueError("field_type %s is not supported." % field.type) diff --git a/flink-python/pyflink/table/tests/test_pandas_udf.py b/flink-python/pyflink/table/tests/test_pandas_udf.py index ab3d2f157a..5ecf2b7027 100644 --- a/flink-python/pyflink/table/tests/test_pandas_udf.py +++ b/flink-python/pyflink/table/tests/test_pandas_udf.py @@ -103,6 +103,12 @@ class PandasUDFITTests(object): 'float_param of wrong type %s !' % type(float_param[0]) return float_param + def double_func(double_param): + assert isinstance(double_param, pd.Series) + assert isinstance(double_param[0], np.float64), \ + 'double_param of wrong type %s !' % type(double_param[0]) + return double_param + self.t_env.register_function( "tinyint_func", udf(tinyint_func, [DataTypes.TINYINT()], DataTypes.TINYINT(), udf_type="pandas")) @@ -127,14 +133,18 @@ class PandasUDFITTests(object): "float_func", udf(float_func, [DataTypes.FLOAT()], DataTypes.FLOAT(), udf_type="pandas")) + self.t_env.register_function( + "double_func", + udf(double_func, [DataTypes.DOUBLE()], DataTypes.DOUBLE(), udf_type="pandas")) + table_sink = source_sink_utils.TestAppendSink( - ['a', 'b', 'c', 'd', 'e', 'f', 'g'], + ['a', 'b', 'c', 'd', 'e', 'f', 'g', 'h'], [DataTypes.TINYINT(), DataTypes.SMALLINT(), DataTypes.INT(), DataTypes.BIGINT(), - DataTypes.BOOLEAN(), DataTypes.BOOLEAN(), DataTypes.FLOAT()]) + DataTypes.BOOLEAN(), DataTypes.BOOLEAN(), DataTypes.FLOAT(), DataTypes.DOUBLE()]) self.t_env.register_table_sink("Results", table_sink) t = self.t_env.from_elements( - [(1, 32767, -2147483648, 1, True, False, 1.0)], + [(1, 32767, -2147483648, 1, True, False, 1.0, 1.0)], DataTypes.ROW( [DataTypes.FIELD("a", DataTypes.TINYINT()), DataTypes.FIELD("b", DataTypes.SMALLINT()), @@ -142,7 +152,8 @@ class PandasUDFITTests(object): DataTypes.FIELD("d", DataTypes.BIGINT()), DataTypes.FIELD("e", DataTypes.BOOLEAN()), DataTypes.FIELD("f", DataTypes.BOOLEAN()), - DataTypes.FIELD("g", DataTypes.FLOAT())])) + DataTypes.FIELD("g", DataTypes.FLOAT()), + DataTypes.FIELD("h", DataTypes.DOUBLE())])) t.select("tinyint_func(a)," "smallint_func(b)," @@ -150,12 +161,13 @@ class PandasUDFITTests(object): "bigint_func(d)," "boolean_func(e)," "boolean_func(f)," - "float_func(g)") \ + "float_func(g)," + "double_func(h)") \ .insert_into("Results") self.t_env.execute("test") actual = source_sink_utils.results() self.assert_equals(actual, - ["1,32767,-2147483648,1,true,false,1.0"]) + ["1,32767,-2147483648,1,true,false,1.0,1.0"]) class StreamPandasUDFITTests(PandasUDFITTests, diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java index 0b29f3a59f..a5c64df033 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java @@ -24,6 +24,7 @@ import org.apache.flink.table.dataformat.vector.ColumnVector; import org.apache.flink.table.runtime.arrow.readers.ArrowFieldReader; import org.apache.flink.table.runtime.arrow.readers.BigIntFieldReader; import org.apache.flink.table.runtime.arrow.readers.BooleanFieldReader; +import org.apache.flink.table.runtime.arrow.readers.DoubleFieldReader; import org.apache.flink.table.runtime.arrow.readers.FloatFieldReader; import org.apache.flink.table.runtime.arrow.readers.IntFieldReader; import org.apache.flink.table.runtime.arrow.readers.RowArrowReader; @@ -31,6 +32,7 @@ import org.apache.flink.table.runtime.arrow.readers.SmallIntFieldReader; import org.apache.flink.table.runtime.arrow.readers.TinyIntFieldReader; import org.apache.flink.table.runtime.arrow.vectors.ArrowBigIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowBooleanColumnVector; +import org.apache.flink.table.runtime.arrow.vectors.ArrowDoubleColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowFloatColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowSmallIntColumnVector; @@ -39,18 +41,21 @@ import org.apache.flink.table.runtime.arrow.vectors.BaseRowArrowReader; import org.apache.flink.table.runtime.arrow.writers.ArrowFieldWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowBigIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowBooleanWriter; +import org.apache.flink.table.runtime.arrow.writers.BaseRowDoubleWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowFloatWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowSmallIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowTinyIntWriter; import org.apache.flink.table.runtime.arrow.writers.BigIntWriter; import org.apache.flink.table.runtime.arrow.writers.BooleanWriter; +import org.apache.flink.table.runtime.arrow.writers.DoubleWriter; import org.apache.flink.table.runtime.arrow.writers.FloatWriter; import org.apache.flink.table.runtime.arrow.writers.IntWriter; import org.apache.flink.table.runtime.arrow.writers.SmallIntWriter; import org.apache.flink.table.runtime.arrow.writers.TinyIntWriter; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.DoubleType; import org.apache.flink.table.types.logical.FloatType; import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.LogicalType; @@ -65,6 +70,7 @@ import org.apache.arrow.vector.BigIntVector; import org.apache.arrow.vector.BitVector; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.Float4Vector; +import org.apache.arrow.vector.Float8Vector; import org.apache.arrow.vector.IntVector; import org.apache.arrow.vector.SmallIntVector; import org.apache.arrow.vector.TinyIntVector; @@ -134,6 +140,8 @@ public final class ArrowUtils { return new BooleanWriter((BitVector) vector); } else if (vector instanceof Float4Vector) { return new FloatWriter((Float4Vector) vector); + } else if (vector instanceof Float8Vector) { + return new DoubleWriter((Float8Vector) vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -168,6 +176,8 @@ public final class ArrowUtils { return new BaseRowBooleanWriter((BitVector) vector); } else if (vector instanceof Float4Vector) { return new BaseRowFloatWriter((Float4Vector) vector); + } else if (vector instanceof Float8Vector) { + return new BaseRowDoubleWriter((Float8Vector) vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -200,6 +210,8 @@ public final class ArrowUtils { return new BooleanFieldReader((BitVector) vector); } else if (vector instanceof Float4Vector) { return new FloatFieldReader((Float4Vector) vector); + } else if (vector instanceof Float8Vector) { + return new DoubleFieldReader((Float8Vector) vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -232,6 +244,8 @@ public final class ArrowUtils { return new ArrowBooleanColumnVector((BitVector) vector); } else if (vector instanceof Float4Vector) { return new ArrowFloatColumnVector((Float4Vector) vector); + } else if (vector instanceof Float8Vector) { + return new ArrowDoubleColumnVector((Float8Vector) vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -272,6 +286,11 @@ public final class ArrowUtils { return new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE); } + @Override + public ArrowType visit(DoubleType doubleType) { + return new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE); + } + @Override protected ArrowType defaultMethod(LogicalType logicalType) { throw new UnsupportedOperationException(String.format( diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/DoubleFieldReader.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/DoubleFieldReader.java new file mode 100644 index 0000000000..0d2fdc981f --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/DoubleFieldReader.java @@ -0,0 +1,39 @@ +/* + * 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.table.runtime.arrow.readers; + +import org.apache.flink.annotation.Internal; + +import org.apache.arrow.vector.Float8Vector; + +/** + * {@link ArrowFieldReader} for Double. + */ +@Internal +public final class DoubleFieldReader extends ArrowFieldReader { + + public DoubleFieldReader(Float8Vector doubleVector) { + super(doubleVector); + } + + @Override + public Double read(int index) { + return ((Float8Vector) getValueVector()).getObject(index); + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowDoubleColumnVector.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowDoubleColumnVector.java new file mode 100644 index 0000000000..e918d81a4b --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowDoubleColumnVector.java @@ -0,0 +1,51 @@ +/* + * 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.table.runtime.arrow.vectors; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.dataformat.vector.DoubleColumnVector; +import org.apache.flink.util.Preconditions; + +import org.apache.arrow.vector.Float8Vector; + +/** + * Arrow column vector for Double. + */ +@Internal +public final class ArrowDoubleColumnVector implements DoubleColumnVector { + + /** + * Container which is used to store the sequence of double values of a column to read. + */ + private final Float8Vector doubleVector; + + public ArrowDoubleColumnVector(Float8Vector doubleVector) { + this.doubleVector = Preconditions.checkNotNull(doubleVector); + } + + @Override + public double getDouble(int i) { + return doubleVector.get(i); + } + + @Override + public boolean isNullAt(int i) { + return doubleVector.isNull(i); + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowDoubleWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowDoubleWriter.java new file mode 100644 index 0000000000..4ead3f7758 --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowDoubleWriter.java @@ -0,0 +1,44 @@ +/* + * 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.table.runtime.arrow.writers; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.dataformat.BaseRow; + +import org.apache.arrow.vector.Float8Vector; + +/** + * {@link ArrowFieldWriter} for Double. + */ +@Internal +public final class BaseRowDoubleWriter extends ArrowFieldWriter { + + public BaseRowDoubleWriter(Float8Vector doubleVector) { + super(doubleVector); + } + + @Override + public void doWrite(BaseRow row, int ordinal) { + if (row.isNullAt(ordinal)) { + ((Float8Vector) getValueVector()).setNull(getCount()); + } else { + ((Float8Vector) getValueVector()).setSafe(getCount(), row.getDouble(ordinal)); + } + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/DoubleWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/DoubleWriter.java new file mode 100644 index 0000000000..020b173b1b --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/DoubleWriter.java @@ -0,0 +1,44 @@ +/* + * 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.table.runtime.arrow.writers; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.types.Row; + +import org.apache.arrow.vector.Float8Vector; + +/** + * {@link ArrowFieldWriter} for Double. + */ +@Internal +public final class DoubleWriter extends ArrowFieldWriter { + + public DoubleWriter(Float8Vector doubleVector) { + super(doubleVector); + } + + @Override + public void doWrite(Row value, int ordinal) { + if (value.getField(ordinal) == null) { + ((Float8Vector) getValueVector()).setNull(getCount()); + } else { + ((Float8Vector) getValueVector()).setSafe(getCount(), (double) value.getField(ordinal)); + } + } +} diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java index 14f943cab5..9166dc6cc6 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java @@ -24,6 +24,7 @@ import org.apache.flink.table.dataformat.vector.ColumnVector; import org.apache.flink.table.runtime.arrow.readers.ArrowFieldReader; import org.apache.flink.table.runtime.arrow.readers.BigIntFieldReader; import org.apache.flink.table.runtime.arrow.readers.BooleanFieldReader; +import org.apache.flink.table.runtime.arrow.readers.DoubleFieldReader; import org.apache.flink.table.runtime.arrow.readers.FloatFieldReader; import org.apache.flink.table.runtime.arrow.readers.IntFieldReader; import org.apache.flink.table.runtime.arrow.readers.RowArrowReader; @@ -31,6 +32,7 @@ import org.apache.flink.table.runtime.arrow.readers.SmallIntFieldReader; import org.apache.flink.table.runtime.arrow.readers.TinyIntFieldReader; import org.apache.flink.table.runtime.arrow.vectors.ArrowBigIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowBooleanColumnVector; +import org.apache.flink.table.runtime.arrow.vectors.ArrowDoubleColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowFloatColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowSmallIntColumnVector; @@ -39,18 +41,21 @@ import org.apache.flink.table.runtime.arrow.vectors.BaseRowArrowReader; import org.apache.flink.table.runtime.arrow.writers.ArrowFieldWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowBigIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowBooleanWriter; +import org.apache.flink.table.runtime.arrow.writers.BaseRowDoubleWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowFloatWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowSmallIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowTinyIntWriter; import org.apache.flink.table.runtime.arrow.writers.BigIntWriter; import org.apache.flink.table.runtime.arrow.writers.BooleanWriter; +import org.apache.flink.table.runtime.arrow.writers.DoubleWriter; import org.apache.flink.table.runtime.arrow.writers.FloatWriter; import org.apache.flink.table.runtime.arrow.writers.IntWriter; import org.apache.flink.table.runtime.arrow.writers.SmallIntWriter; import org.apache.flink.table.runtime.arrow.writers.TinyIntWriter; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.DoubleType; import org.apache.flink.table.types.logical.FloatType; import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.LogicalType; @@ -98,6 +103,8 @@ public class ArrowUtilsTest { BooleanWriter.class, BaseRowBooleanWriter.class, BooleanFieldReader.class, ArrowBooleanColumnVector.class)); testFields.add(Tuple7.of("f6", new FloatType(), new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE), FloatWriter.class, BaseRowFloatWriter.class, FloatFieldReader.class, ArrowFloatColumnVector.class)); + testFields.add(Tuple7.of("f7", new DoubleType(), new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE), + DoubleWriter.class, BaseRowDoubleWriter.class, DoubleFieldReader.class, ArrowDoubleColumnVector.class)); List rowFields = new ArrayList<>(); for (Tuple7, Class, Class, Class> field : testFields) { diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java index f6f736bed5..fbf962d613 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java @@ -26,6 +26,7 @@ import org.apache.flink.table.runtime.typeutils.BaseRowSerializer; import org.apache.flink.table.runtime.util.StreamRecordUtils; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.DoubleType; import org.apache.flink.table.types.logical.FloatType; import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.LogicalType; @@ -90,6 +91,7 @@ public class BaseRowArrowReaderWriterTest extends ArrowReaderWriterTestBase rowFields = new ArrayList<>(); for (int i = 0; i < fieldTypes.size(); i++) { @@ -117,12 +119,12 @@ public class BaseRowArrowReaderWriterTest extends ArrowReaderWriterTestBase { fieldTypes.add(new BigIntType()); fieldTypes.add(new BooleanType()); fieldTypes.add(new FloatType()); + fieldTypes.add(new DoubleType()); List rowFields = new ArrayList<>(); for (int i = 0; i < fieldTypes.size(); i++) { @@ -84,10 +86,10 @@ public class RowArrowReaderWriterTest extends ArrowReaderWriterTestBase { @Override public Row[] getTestData() { - Row row1 = Row.of((byte) 1, (short) 2, 3, 4L, true, 1.0f); - Row row2 = Row.of(null, (short) 2, 3, 4L, false, 1.0f); - Row row3 = Row.of((byte) 1, null, 3, 4L, true, 1.0f); - Row row4 = Row.of(null, null, null, null, null, null); + Row row1 = Row.of((byte) 1, (short) 2, 3, 4L, true, 1.0f, 1.0); + Row row2 = Row.of(null, (short) 2, 3, 4L, false, 1.0f, 1.0); + Row row3 = Row.of((byte) 1, null, 3, 4L, true, 1.0f, 1.0); + Row row4 = Row.of(null, null, null, null, null, null, null); return new Row[]{row1, row2, row3, row4}; } } -- Gitee From cf1d8c63c039c5ec715573982d1d45d3dd2cf5a6 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Fri, 13 Mar 2020 14:34:12 +0800 Subject: [PATCH 256/885] [FLINK-16608][python] Support VarCharType in vectorized Python UDF --- flink-python/pyflink/fn_execution/coders.py | 2 + .../pyflink/table/tests/test_pandas_udf.py | 27 +++++++--- .../flink/table/runtime/arrow/ArrowUtils.java | 19 +++++++ .../arrow/readers/VarCharFieldReader.java | 45 ++++++++++++++++ .../vectors/ArrowVarCharColumnVector.java | 52 +++++++++++++++++++ .../arrow/writers/BaseRowVarCharWriter.java | 44 ++++++++++++++++ .../runtime/arrow/writers/VarCharWriter.java | 46 ++++++++++++++++ .../table/runtime/arrow/ArrowUtilsTest.java | 7 +++ .../arrow/BaseRowArrowReaderWriterTest.java | 14 ++--- .../arrow/RowArrowReaderWriterTest.java | 10 ++-- 10 files changed, 250 insertions(+), 16 deletions(-) create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/VarCharFieldReader.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowVarCharColumnVector.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowVarCharWriter.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/VarCharWriter.java diff --git a/flink-python/pyflink/fn_execution/coders.py b/flink-python/pyflink/fn_execution/coders.py index 99a2622265..0b79ef1249 100644 --- a/flink-python/pyflink/fn_execution/coders.py +++ b/flink-python/pyflink/fn_execution/coders.py @@ -428,6 +428,8 @@ class ArrowCoder(DeterministicCoder): return pa.field(field.name, pa.float32(), field.type.nullable) elif field.type.type_name == flink_fn_execution_pb2.Schema.TypeName.DOUBLE: return pa.field(field.name, pa.float64(), field.type.nullable) + elif field.type.type_name == flink_fn_execution_pb2.Schema.TypeName.VARCHAR: + return pa.field(field.name, pa.utf8(), field.type.nullable) else: raise ValueError("field_type %s is not supported." % field.type) diff --git a/flink-python/pyflink/table/tests/test_pandas_udf.py b/flink-python/pyflink/table/tests/test_pandas_udf.py index 5ecf2b7027..6daa831ef0 100644 --- a/flink-python/pyflink/table/tests/test_pandas_udf.py +++ b/flink-python/pyflink/table/tests/test_pandas_udf.py @@ -109,6 +109,12 @@ class PandasUDFITTests(object): 'double_param of wrong type %s !' % type(double_param[0]) return double_param + def varchar_func(varchar_param): + assert isinstance(varchar_param, pd.Series) + assert isinstance(varchar_param[0], str), \ + 'varchar_param of wrong type %s !' % type(varchar_param[0]) + return varchar_param + self.t_env.register_function( "tinyint_func", udf(tinyint_func, [DataTypes.TINYINT()], DataTypes.TINYINT(), udf_type="pandas")) @@ -137,14 +143,19 @@ class PandasUDFITTests(object): "double_func", udf(double_func, [DataTypes.DOUBLE()], DataTypes.DOUBLE(), udf_type="pandas")) + self.t_env.register_function( + "varchar_func", + udf(varchar_func, [DataTypes.STRING()], DataTypes.STRING(), udf_type="pandas")) + table_sink = source_sink_utils.TestAppendSink( - ['a', 'b', 'c', 'd', 'e', 'f', 'g', 'h'], + ['a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j'], [DataTypes.TINYINT(), DataTypes.SMALLINT(), DataTypes.INT(), DataTypes.BIGINT(), - DataTypes.BOOLEAN(), DataTypes.BOOLEAN(), DataTypes.FLOAT(), DataTypes.DOUBLE()]) + DataTypes.BOOLEAN(), DataTypes.BOOLEAN(), DataTypes.FLOAT(), DataTypes.DOUBLE(), + DataTypes.STRING(), DataTypes.STRING()]) self.t_env.register_table_sink("Results", table_sink) t = self.t_env.from_elements( - [(1, 32767, -2147483648, 1, True, False, 1.0, 1.0)], + [(1, 32767, -2147483648, 1, True, False, 1.0, 1.0, 'hello', '中文')], DataTypes.ROW( [DataTypes.FIELD("a", DataTypes.TINYINT()), DataTypes.FIELD("b", DataTypes.SMALLINT()), @@ -153,7 +164,9 @@ class PandasUDFITTests(object): DataTypes.FIELD("e", DataTypes.BOOLEAN()), DataTypes.FIELD("f", DataTypes.BOOLEAN()), DataTypes.FIELD("g", DataTypes.FLOAT()), - DataTypes.FIELD("h", DataTypes.DOUBLE())])) + DataTypes.FIELD("h", DataTypes.DOUBLE()), + DataTypes.FIELD("i", DataTypes.STRING()), + DataTypes.FIELD("j", DataTypes.STRING())])) t.select("tinyint_func(a)," "smallint_func(b)," @@ -162,12 +175,14 @@ class PandasUDFITTests(object): "boolean_func(e)," "boolean_func(f)," "float_func(g)," - "double_func(h)") \ + "double_func(h)," + "varchar_func(i)," + "varchar_func(j)") \ .insert_into("Results") self.t_env.execute("test") actual = source_sink_utils.results() self.assert_equals(actual, - ["1,32767,-2147483648,1,true,false,1.0,1.0"]) + ["1,32767,-2147483648,1,true,false,1.0,1.0,hello,中文"]) class StreamPandasUDFITTests(PandasUDFITTests, diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java index a5c64df033..8444551b35 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java @@ -30,6 +30,7 @@ import org.apache.flink.table.runtime.arrow.readers.IntFieldReader; import org.apache.flink.table.runtime.arrow.readers.RowArrowReader; import org.apache.flink.table.runtime.arrow.readers.SmallIntFieldReader; import org.apache.flink.table.runtime.arrow.readers.TinyIntFieldReader; +import org.apache.flink.table.runtime.arrow.readers.VarCharFieldReader; import org.apache.flink.table.runtime.arrow.vectors.ArrowBigIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowBooleanColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowDoubleColumnVector; @@ -37,6 +38,7 @@ import org.apache.flink.table.runtime.arrow.vectors.ArrowFloatColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowSmallIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowTinyIntColumnVector; +import org.apache.flink.table.runtime.arrow.vectors.ArrowVarCharColumnVector; import org.apache.flink.table.runtime.arrow.vectors.BaseRowArrowReader; import org.apache.flink.table.runtime.arrow.writers.ArrowFieldWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowBigIntWriter; @@ -46,6 +48,7 @@ import org.apache.flink.table.runtime.arrow.writers.BaseRowFloatWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowSmallIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowTinyIntWriter; +import org.apache.flink.table.runtime.arrow.writers.BaseRowVarCharWriter; import org.apache.flink.table.runtime.arrow.writers.BigIntWriter; import org.apache.flink.table.runtime.arrow.writers.BooleanWriter; import org.apache.flink.table.runtime.arrow.writers.DoubleWriter; @@ -53,6 +56,7 @@ import org.apache.flink.table.runtime.arrow.writers.FloatWriter; import org.apache.flink.table.runtime.arrow.writers.IntWriter; import org.apache.flink.table.runtime.arrow.writers.SmallIntWriter; import org.apache.flink.table.runtime.arrow.writers.TinyIntWriter; +import org.apache.flink.table.runtime.arrow.writers.VarCharWriter; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BooleanType; import org.apache.flink.table.types.logical.DoubleType; @@ -62,6 +66,7 @@ import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.SmallIntType; import org.apache.flink.table.types.logical.TinyIntType; +import org.apache.flink.table.types.logical.VarCharType; import org.apache.flink.table.types.logical.utils.LogicalTypeDefaultVisitor; import org.apache.flink.types.Row; @@ -74,6 +79,7 @@ import org.apache.arrow.vector.Float8Vector; import org.apache.arrow.vector.IntVector; import org.apache.arrow.vector.SmallIntVector; import org.apache.arrow.vector.TinyIntVector; +import org.apache.arrow.vector.VarCharVector; import org.apache.arrow.vector.VectorSchemaRoot; import org.apache.arrow.vector.types.FloatingPointPrecision; import org.apache.arrow.vector.types.pojo.ArrowType; @@ -142,6 +148,8 @@ public final class ArrowUtils { return new FloatWriter((Float4Vector) vector); } else if (vector instanceof Float8Vector) { return new DoubleWriter((Float8Vector) vector); + } else if (vector instanceof VarCharVector) { + return new VarCharWriter((VarCharVector) vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -178,6 +186,8 @@ public final class ArrowUtils { return new BaseRowFloatWriter((Float4Vector) vector); } else if (vector instanceof Float8Vector) { return new BaseRowDoubleWriter((Float8Vector) vector); + } else if (vector instanceof VarCharVector) { + return new BaseRowVarCharWriter((VarCharVector) vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -212,6 +222,8 @@ public final class ArrowUtils { return new FloatFieldReader((Float4Vector) vector); } else if (vector instanceof Float8Vector) { return new DoubleFieldReader((Float8Vector) vector); + } else if (vector instanceof VarCharVector) { + return new VarCharFieldReader((VarCharVector) vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -246,6 +258,8 @@ public final class ArrowUtils { return new ArrowFloatColumnVector((Float4Vector) vector); } else if (vector instanceof Float8Vector) { return new ArrowDoubleColumnVector((Float8Vector) vector); + } else if (vector instanceof VarCharVector) { + return new ArrowVarCharColumnVector((VarCharVector) vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -291,6 +305,11 @@ public final class ArrowUtils { return new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE); } + @Override + public ArrowType visit(VarCharType varCharType) { + return ArrowType.Utf8.INSTANCE; + } + @Override protected ArrowType defaultMethod(LogicalType logicalType) { throw new UnsupportedOperationException(String.format( diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/VarCharFieldReader.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/VarCharFieldReader.java new file mode 100644 index 0000000000..01b0a6fce9 --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/VarCharFieldReader.java @@ -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.flink.table.runtime.arrow.readers; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.runtime.util.StringUtf8Utils; + +import org.apache.arrow.vector.VarCharVector; + +/** + * {@link ArrowFieldReader} for VarChar. + */ +@Internal +public final class VarCharFieldReader extends ArrowFieldReader { + + public VarCharFieldReader(VarCharVector varCharVector) { + super(varCharVector); + } + + @Override + public String read(int index) { + if (getValueVector().isNull(index)) { + return null; + } else { + byte[] bytes = ((VarCharVector) getValueVector()).get(index); + return StringUtf8Utils.decodeUTF8(bytes, 0, bytes.length); + } + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowVarCharColumnVector.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowVarCharColumnVector.java new file mode 100644 index 0000000000..04188bd775 --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowVarCharColumnVector.java @@ -0,0 +1,52 @@ +/* + * 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.table.runtime.arrow.vectors; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.dataformat.vector.BytesColumnVector; +import org.apache.flink.util.Preconditions; + +import org.apache.arrow.vector.VarCharVector; + +/** + * Arrow column vector for VarChar. + */ +@Internal +public final class ArrowVarCharColumnVector implements BytesColumnVector { + + /** + * Container which is used to store the sequence of varchar values of a column to read. + */ + private final VarCharVector varCharVector; + + public ArrowVarCharColumnVector(VarCharVector varCharVector) { + this.varCharVector = Preconditions.checkNotNull(varCharVector); + } + + @Override + public Bytes getBytes(int i) { + byte[] bytes = varCharVector.get(i); + return new Bytes(bytes, 0, bytes.length); + } + + @Override + public boolean isNullAt(int i) { + return varCharVector.isNull(i); + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowVarCharWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowVarCharWriter.java new file mode 100644 index 0000000000..915e1869d8 --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowVarCharWriter.java @@ -0,0 +1,44 @@ +/* + * 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.table.runtime.arrow.writers; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.dataformat.BaseRow; + +import org.apache.arrow.vector.VarCharVector; + +/** + * {@link ArrowFieldWriter} for VarChar. + */ +@Internal +public final class BaseRowVarCharWriter extends ArrowFieldWriter { + + public BaseRowVarCharWriter(VarCharVector varCharVector) { + super(varCharVector); + } + + @Override + public void doWrite(BaseRow row, int ordinal) { + if (row.isNullAt(ordinal)) { + ((VarCharVector) getValueVector()).setNull(getCount()); + } else { + ((VarCharVector) getValueVector()).setSafe(getCount(), row.getString(ordinal).getBytes()); + } + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/VarCharWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/VarCharWriter.java new file mode 100644 index 0000000000..fcd3457d8b --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/VarCharWriter.java @@ -0,0 +1,46 @@ +/* + * 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.table.runtime.arrow.writers; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.runtime.util.StringUtf8Utils; +import org.apache.flink.types.Row; + +import org.apache.arrow.vector.VarCharVector; + +/** + * {@link ArrowFieldWriter} for VarChar. + */ +@Internal +public final class VarCharWriter extends ArrowFieldWriter { + + public VarCharWriter(VarCharVector varCharVector) { + super(varCharVector); + } + + @Override + public void doWrite(Row value, int ordinal) { + if (value.getField(ordinal) == null) { + ((VarCharVector) getValueVector()).setNull(getCount()); + } else { + ((VarCharVector) getValueVector()).setSafe( + getCount(), StringUtf8Utils.encodeUTF8(((String) value.getField(ordinal)))); + } + } +} diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java index 9166dc6cc6..ee24b06e9d 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java @@ -30,6 +30,7 @@ import org.apache.flink.table.runtime.arrow.readers.IntFieldReader; import org.apache.flink.table.runtime.arrow.readers.RowArrowReader; import org.apache.flink.table.runtime.arrow.readers.SmallIntFieldReader; import org.apache.flink.table.runtime.arrow.readers.TinyIntFieldReader; +import org.apache.flink.table.runtime.arrow.readers.VarCharFieldReader; import org.apache.flink.table.runtime.arrow.vectors.ArrowBigIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowBooleanColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowDoubleColumnVector; @@ -37,6 +38,7 @@ import org.apache.flink.table.runtime.arrow.vectors.ArrowFloatColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowSmallIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowTinyIntColumnVector; +import org.apache.flink.table.runtime.arrow.vectors.ArrowVarCharColumnVector; import org.apache.flink.table.runtime.arrow.vectors.BaseRowArrowReader; import org.apache.flink.table.runtime.arrow.writers.ArrowFieldWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowBigIntWriter; @@ -46,6 +48,7 @@ import org.apache.flink.table.runtime.arrow.writers.BaseRowFloatWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowSmallIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowTinyIntWriter; +import org.apache.flink.table.runtime.arrow.writers.BaseRowVarCharWriter; import org.apache.flink.table.runtime.arrow.writers.BigIntWriter; import org.apache.flink.table.runtime.arrow.writers.BooleanWriter; import org.apache.flink.table.runtime.arrow.writers.DoubleWriter; @@ -53,6 +56,7 @@ import org.apache.flink.table.runtime.arrow.writers.FloatWriter; import org.apache.flink.table.runtime.arrow.writers.IntWriter; import org.apache.flink.table.runtime.arrow.writers.SmallIntWriter; import org.apache.flink.table.runtime.arrow.writers.TinyIntWriter; +import org.apache.flink.table.runtime.arrow.writers.VarCharWriter; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BooleanType; import org.apache.flink.table.types.logical.DoubleType; @@ -62,6 +66,7 @@ import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.SmallIntType; import org.apache.flink.table.types.logical.TinyIntType; +import org.apache.flink.table.types.logical.VarCharType; import org.apache.flink.types.Row; import org.apache.arrow.memory.BufferAllocator; @@ -105,6 +110,8 @@ public class ArrowUtilsTest { FloatWriter.class, BaseRowFloatWriter.class, FloatFieldReader.class, ArrowFloatColumnVector.class)); testFields.add(Tuple7.of("f7", new DoubleType(), new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE), DoubleWriter.class, BaseRowDoubleWriter.class, DoubleFieldReader.class, ArrowDoubleColumnVector.class)); + testFields.add(Tuple7.of("f8", new VarCharType(), ArrowType.Utf8.INSTANCE, + VarCharWriter.class, BaseRowVarCharWriter.class, VarCharFieldReader.class, ArrowVarCharColumnVector.class)); List rowFields = new ArrayList<>(); for (Tuple7, Class, Class, Class> field : testFields) { diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java index fbf962d613..be9efeae7e 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java @@ -33,6 +33,7 @@ import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.SmallIntType; import org.apache.flink.table.types.logical.TinyIntType; +import org.apache.flink.table.types.logical.VarCharType; import org.apache.flink.testutils.DeeplyEqualsChecker; import org.apache.arrow.memory.BufferAllocator; @@ -92,6 +93,7 @@ public class BaseRowArrowReaderWriterTest extends ArrowReaderWriterTestBase rowFields = new ArrayList<>(); for (int i = 0; i < fieldTypes.size(); i++) { @@ -119,12 +121,12 @@ public class BaseRowArrowReaderWriterTest extends ArrowReaderWriterTestBase { fieldTypes.add(new BooleanType()); fieldTypes.add(new FloatType()); fieldTypes.add(new DoubleType()); + fieldTypes.add(new VarCharType()); List rowFields = new ArrayList<>(); for (int i = 0; i < fieldTypes.size(); i++) { @@ -86,10 +88,10 @@ public class RowArrowReaderWriterTest extends ArrowReaderWriterTestBase { @Override public Row[] getTestData() { - Row row1 = Row.of((byte) 1, (short) 2, 3, 4L, true, 1.0f, 1.0); - Row row2 = Row.of(null, (short) 2, 3, 4L, false, 1.0f, 1.0); - Row row3 = Row.of((byte) 1, null, 3, 4L, true, 1.0f, 1.0); - Row row4 = Row.of(null, null, null, null, null, null, null); + Row row1 = Row.of((byte) 1, (short) 2, 3, 4L, true, 1.0f, 1.0, "hello"); + Row row2 = Row.of(null, (short) 2, 3, 4L, false, 1.0f, 1.0, "中文"); + Row row3 = Row.of((byte) 1, null, 3, 4L, true, 1.0f, 1.0, "hello"); + Row row4 = Row.of(null, null, null, null, null, null, null, null); return new Row[]{row1, row2, row3, row4}; } } -- Gitee From 6918ae75f61995daa1a9a7f4d86256f2944f8c4d Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Fri, 13 Mar 2020 14:37:03 +0800 Subject: [PATCH 257/885] [FLINK-16608][python] Support VarBinaryType in vectorized Python UDF --- flink-python/pyflink/fn_execution/coders.py | 2 + .../pyflink/table/tests/test_pandas_udf.py | 26 +++++++--- .../flink/table/runtime/arrow/ArrowUtils.java | 19 +++++++ .../arrow/readers/VarBinaryFieldReader.java | 39 ++++++++++++++ .../vectors/ArrowVarBinaryColumnVector.java | 52 +++++++++++++++++++ .../arrow/writers/BaseRowVarBinaryWriter.java | 44 ++++++++++++++++ .../arrow/writers/VarBinaryWriter.java | 44 ++++++++++++++++ .../table/runtime/arrow/ArrowUtilsTest.java | 7 +++ .../arrow/BaseRowArrowReaderWriterTest.java | 14 ++--- .../arrow/RowArrowReaderWriterTest.java | 10 ++-- .../table/runtime/util/StreamRecordUtils.java | 2 + 11 files changed, 243 insertions(+), 16 deletions(-) create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/VarBinaryFieldReader.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowVarBinaryColumnVector.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowVarBinaryWriter.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/VarBinaryWriter.java diff --git a/flink-python/pyflink/fn_execution/coders.py b/flink-python/pyflink/fn_execution/coders.py index 0b79ef1249..d20a6d1f7a 100644 --- a/flink-python/pyflink/fn_execution/coders.py +++ b/flink-python/pyflink/fn_execution/coders.py @@ -430,6 +430,8 @@ class ArrowCoder(DeterministicCoder): return pa.field(field.name, pa.float64(), field.type.nullable) elif field.type.type_name == flink_fn_execution_pb2.Schema.TypeName.VARCHAR: return pa.field(field.name, pa.utf8(), field.type.nullable) + elif field.type.type_name == flink_fn_execution_pb2.Schema.TypeName.VARBINARY: + return pa.field(field.name, pa.binary(), field.type.nullable) else: raise ValueError("field_type %s is not supported." % field.type) diff --git a/flink-python/pyflink/table/tests/test_pandas_udf.py b/flink-python/pyflink/table/tests/test_pandas_udf.py index 6daa831ef0..fcf8a686b1 100644 --- a/flink-python/pyflink/table/tests/test_pandas_udf.py +++ b/flink-python/pyflink/table/tests/test_pandas_udf.py @@ -115,6 +115,12 @@ class PandasUDFITTests(object): 'varchar_param of wrong type %s !' % type(varchar_param[0]) return varchar_param + def varbinary_func(varbinary_param): + assert isinstance(varbinary_param, pd.Series) + assert isinstance(varbinary_param[0], bytes), \ + 'varbinary_param of wrong type %s !' % type(varbinary_param[0]) + return varbinary_param + self.t_env.register_function( "tinyint_func", udf(tinyint_func, [DataTypes.TINYINT()], DataTypes.TINYINT(), udf_type="pandas")) @@ -147,15 +153,20 @@ class PandasUDFITTests(object): "varchar_func", udf(varchar_func, [DataTypes.STRING()], DataTypes.STRING(), udf_type="pandas")) + self.t_env.register_function( + "varbinary_func", + udf(varbinary_func, [DataTypes.BYTES()], DataTypes.BYTES(), udf_type="pandas")) + table_sink = source_sink_utils.TestAppendSink( - ['a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j'], + ['a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j', 'k'], [DataTypes.TINYINT(), DataTypes.SMALLINT(), DataTypes.INT(), DataTypes.BIGINT(), DataTypes.BOOLEAN(), DataTypes.BOOLEAN(), DataTypes.FLOAT(), DataTypes.DOUBLE(), - DataTypes.STRING(), DataTypes.STRING()]) + DataTypes.STRING(), DataTypes.STRING(), DataTypes.BYTES()]) self.t_env.register_table_sink("Results", table_sink) t = self.t_env.from_elements( - [(1, 32767, -2147483648, 1, True, False, 1.0, 1.0, 'hello', '中文')], + [(1, 32767, -2147483648, 1, True, False, 1.0, 1.0, 'hello', '中文', + bytearray(b'flink'))], DataTypes.ROW( [DataTypes.FIELD("a", DataTypes.TINYINT()), DataTypes.FIELD("b", DataTypes.SMALLINT()), @@ -166,7 +177,8 @@ class PandasUDFITTests(object): DataTypes.FIELD("g", DataTypes.FLOAT()), DataTypes.FIELD("h", DataTypes.DOUBLE()), DataTypes.FIELD("i", DataTypes.STRING()), - DataTypes.FIELD("j", DataTypes.STRING())])) + DataTypes.FIELD("j", DataTypes.STRING()), + DataTypes.FIELD("k", DataTypes.BYTES())])) t.select("tinyint_func(a)," "smallint_func(b)," @@ -177,12 +189,14 @@ class PandasUDFITTests(object): "float_func(g)," "double_func(h)," "varchar_func(i)," - "varchar_func(j)") \ + "varchar_func(j)," + "varbinary_func(k)") \ .insert_into("Results") self.t_env.execute("test") actual = source_sink_utils.results() self.assert_equals(actual, - ["1,32767,-2147483648,1,true,false,1.0,1.0,hello,中文"]) + ["1,32767,-2147483648,1,true,false,1.0,1.0,hello,中文," + "[102, 108, 105, 110, 107]"]) class StreamPandasUDFITTests(PandasUDFITTests, diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java index 8444551b35..b9c3426fc1 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java @@ -30,6 +30,7 @@ import org.apache.flink.table.runtime.arrow.readers.IntFieldReader; import org.apache.flink.table.runtime.arrow.readers.RowArrowReader; import org.apache.flink.table.runtime.arrow.readers.SmallIntFieldReader; import org.apache.flink.table.runtime.arrow.readers.TinyIntFieldReader; +import org.apache.flink.table.runtime.arrow.readers.VarBinaryFieldReader; import org.apache.flink.table.runtime.arrow.readers.VarCharFieldReader; import org.apache.flink.table.runtime.arrow.vectors.ArrowBigIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowBooleanColumnVector; @@ -38,6 +39,7 @@ import org.apache.flink.table.runtime.arrow.vectors.ArrowFloatColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowSmallIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowTinyIntColumnVector; +import org.apache.flink.table.runtime.arrow.vectors.ArrowVarBinaryColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowVarCharColumnVector; import org.apache.flink.table.runtime.arrow.vectors.BaseRowArrowReader; import org.apache.flink.table.runtime.arrow.writers.ArrowFieldWriter; @@ -48,6 +50,7 @@ import org.apache.flink.table.runtime.arrow.writers.BaseRowFloatWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowSmallIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowTinyIntWriter; +import org.apache.flink.table.runtime.arrow.writers.BaseRowVarBinaryWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowVarCharWriter; import org.apache.flink.table.runtime.arrow.writers.BigIntWriter; import org.apache.flink.table.runtime.arrow.writers.BooleanWriter; @@ -56,6 +59,7 @@ import org.apache.flink.table.runtime.arrow.writers.FloatWriter; import org.apache.flink.table.runtime.arrow.writers.IntWriter; import org.apache.flink.table.runtime.arrow.writers.SmallIntWriter; import org.apache.flink.table.runtime.arrow.writers.TinyIntWriter; +import org.apache.flink.table.runtime.arrow.writers.VarBinaryWriter; import org.apache.flink.table.runtime.arrow.writers.VarCharWriter; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BooleanType; @@ -66,6 +70,7 @@ import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.SmallIntType; import org.apache.flink.table.types.logical.TinyIntType; +import org.apache.flink.table.types.logical.VarBinaryType; import org.apache.flink.table.types.logical.VarCharType; import org.apache.flink.table.types.logical.utils.LogicalTypeDefaultVisitor; import org.apache.flink.types.Row; @@ -79,6 +84,7 @@ import org.apache.arrow.vector.Float8Vector; import org.apache.arrow.vector.IntVector; import org.apache.arrow.vector.SmallIntVector; import org.apache.arrow.vector.TinyIntVector; +import org.apache.arrow.vector.VarBinaryVector; import org.apache.arrow.vector.VarCharVector; import org.apache.arrow.vector.VectorSchemaRoot; import org.apache.arrow.vector.types.FloatingPointPrecision; @@ -150,6 +156,8 @@ public final class ArrowUtils { return new DoubleWriter((Float8Vector) vector); } else if (vector instanceof VarCharVector) { return new VarCharWriter((VarCharVector) vector); + } else if (vector instanceof VarBinaryVector) { + return new VarBinaryWriter((VarBinaryVector) vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -188,6 +196,8 @@ public final class ArrowUtils { return new BaseRowDoubleWriter((Float8Vector) vector); } else if (vector instanceof VarCharVector) { return new BaseRowVarCharWriter((VarCharVector) vector); + } else if (vector instanceof VarBinaryVector) { + return new BaseRowVarBinaryWriter((VarBinaryVector) vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -224,6 +234,8 @@ public final class ArrowUtils { return new DoubleFieldReader((Float8Vector) vector); } else if (vector instanceof VarCharVector) { return new VarCharFieldReader((VarCharVector) vector); + } else if (vector instanceof VarBinaryVector) { + return new VarBinaryFieldReader((VarBinaryVector) vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -260,6 +272,8 @@ public final class ArrowUtils { return new ArrowDoubleColumnVector((Float8Vector) vector); } else if (vector instanceof VarCharVector) { return new ArrowVarCharColumnVector((VarCharVector) vector); + } else if (vector instanceof VarBinaryVector) { + return new ArrowVarBinaryColumnVector((VarBinaryVector) vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -310,6 +324,11 @@ public final class ArrowUtils { return ArrowType.Utf8.INSTANCE; } + @Override + public ArrowType visit(VarBinaryType varCharType) { + return ArrowType.Binary.INSTANCE; + } + @Override protected ArrowType defaultMethod(LogicalType logicalType) { throw new UnsupportedOperationException(String.format( diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/VarBinaryFieldReader.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/VarBinaryFieldReader.java new file mode 100644 index 0000000000..9f9995ec91 --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/VarBinaryFieldReader.java @@ -0,0 +1,39 @@ +/* + * 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.table.runtime.arrow.readers; + +import org.apache.flink.annotation.Internal; + +import org.apache.arrow.vector.VarBinaryVector; + +/** + * {@link ArrowFieldReader} for VarBinary. + */ +@Internal +public final class VarBinaryFieldReader extends ArrowFieldReader { + + public VarBinaryFieldReader(VarBinaryVector varBinaryVector) { + super(varBinaryVector); + } + + @Override + public byte[] read(int index) { + return ((VarBinaryVector) getValueVector()).get(index); + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowVarBinaryColumnVector.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowVarBinaryColumnVector.java new file mode 100644 index 0000000000..434e776e77 --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowVarBinaryColumnVector.java @@ -0,0 +1,52 @@ +/* + * 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.table.runtime.arrow.vectors; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.dataformat.vector.BytesColumnVector; +import org.apache.flink.util.Preconditions; + +import org.apache.arrow.vector.VarBinaryVector; + +/** + * Arrow column vector for VarBinary. + */ +@Internal +public final class ArrowVarBinaryColumnVector implements BytesColumnVector { + + /** + * Container which is used to store the sequence of varbinary values of a column to read. + */ + private final VarBinaryVector varBinaryVector; + + public ArrowVarBinaryColumnVector(VarBinaryVector varBinaryVector) { + this.varBinaryVector = Preconditions.checkNotNull(varBinaryVector); + } + + @Override + public Bytes getBytes(int i) { + byte[] bytes = varBinaryVector.get(i); + return new Bytes(bytes, 0, bytes.length); + } + + @Override + public boolean isNullAt(int i) { + return varBinaryVector.isNull(i); + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowVarBinaryWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowVarBinaryWriter.java new file mode 100644 index 0000000000..2c33ac6ffa --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowVarBinaryWriter.java @@ -0,0 +1,44 @@ +/* + * 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.table.runtime.arrow.writers; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.dataformat.BaseRow; + +import org.apache.arrow.vector.VarBinaryVector; + +/** + * {@link ArrowFieldWriter} for VarBinary. + */ +@Internal +public final class BaseRowVarBinaryWriter extends ArrowFieldWriter { + + public BaseRowVarBinaryWriter(VarBinaryVector varBinaryVector) { + super(varBinaryVector); + } + + @Override + public void doWrite(BaseRow row, int ordinal) { + if (row.isNullAt(ordinal)) { + ((VarBinaryVector) getValueVector()).setNull(getCount()); + } else { + ((VarBinaryVector) getValueVector()).setSafe(getCount(), row.getBinary(ordinal)); + } + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/VarBinaryWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/VarBinaryWriter.java new file mode 100644 index 0000000000..72042e533c --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/VarBinaryWriter.java @@ -0,0 +1,44 @@ +/* + * 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.table.runtime.arrow.writers; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.types.Row; + +import org.apache.arrow.vector.VarBinaryVector; + +/** + * {@link ArrowFieldWriter} for VarBinary. + */ +@Internal +public final class VarBinaryWriter extends ArrowFieldWriter { + + public VarBinaryWriter(VarBinaryVector varBinaryVector) { + super(varBinaryVector); + } + + @Override + public void doWrite(Row value, int ordinal) { + if (value.getField(ordinal) == null) { + ((VarBinaryVector) getValueVector()).setNull(getCount()); + } else { + ((VarBinaryVector) getValueVector()).setSafe(getCount(), (byte[]) value.getField(ordinal)); + } + } +} diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java index ee24b06e9d..31057eefd4 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java @@ -30,6 +30,7 @@ import org.apache.flink.table.runtime.arrow.readers.IntFieldReader; import org.apache.flink.table.runtime.arrow.readers.RowArrowReader; import org.apache.flink.table.runtime.arrow.readers.SmallIntFieldReader; import org.apache.flink.table.runtime.arrow.readers.TinyIntFieldReader; +import org.apache.flink.table.runtime.arrow.readers.VarBinaryFieldReader; import org.apache.flink.table.runtime.arrow.readers.VarCharFieldReader; import org.apache.flink.table.runtime.arrow.vectors.ArrowBigIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowBooleanColumnVector; @@ -38,6 +39,7 @@ import org.apache.flink.table.runtime.arrow.vectors.ArrowFloatColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowSmallIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowTinyIntColumnVector; +import org.apache.flink.table.runtime.arrow.vectors.ArrowVarBinaryColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowVarCharColumnVector; import org.apache.flink.table.runtime.arrow.vectors.BaseRowArrowReader; import org.apache.flink.table.runtime.arrow.writers.ArrowFieldWriter; @@ -48,6 +50,7 @@ import org.apache.flink.table.runtime.arrow.writers.BaseRowFloatWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowSmallIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowTinyIntWriter; +import org.apache.flink.table.runtime.arrow.writers.BaseRowVarBinaryWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowVarCharWriter; import org.apache.flink.table.runtime.arrow.writers.BigIntWriter; import org.apache.flink.table.runtime.arrow.writers.BooleanWriter; @@ -56,6 +59,7 @@ import org.apache.flink.table.runtime.arrow.writers.FloatWriter; import org.apache.flink.table.runtime.arrow.writers.IntWriter; import org.apache.flink.table.runtime.arrow.writers.SmallIntWriter; import org.apache.flink.table.runtime.arrow.writers.TinyIntWriter; +import org.apache.flink.table.runtime.arrow.writers.VarBinaryWriter; import org.apache.flink.table.runtime.arrow.writers.VarCharWriter; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BooleanType; @@ -66,6 +70,7 @@ import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.SmallIntType; import org.apache.flink.table.types.logical.TinyIntType; +import org.apache.flink.table.types.logical.VarBinaryType; import org.apache.flink.table.types.logical.VarCharType; import org.apache.flink.types.Row; @@ -112,6 +117,8 @@ public class ArrowUtilsTest { DoubleWriter.class, BaseRowDoubleWriter.class, DoubleFieldReader.class, ArrowDoubleColumnVector.class)); testFields.add(Tuple7.of("f8", new VarCharType(), ArrowType.Utf8.INSTANCE, VarCharWriter.class, BaseRowVarCharWriter.class, VarCharFieldReader.class, ArrowVarCharColumnVector.class)); + testFields.add(Tuple7.of("f9", new VarBinaryType(), ArrowType.Binary.INSTANCE, + VarBinaryWriter.class, BaseRowVarBinaryWriter.class, VarBinaryFieldReader.class, ArrowVarBinaryColumnVector.class)); List rowFields = new ArrayList<>(); for (Tuple7, Class, Class, Class> field : testFields) { diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java index be9efeae7e..5fd0da47fc 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java @@ -33,6 +33,7 @@ import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.SmallIntType; import org.apache.flink.table.types.logical.TinyIntType; +import org.apache.flink.table.types.logical.VarBinaryType; import org.apache.flink.table.types.logical.VarCharType; import org.apache.flink.testutils.DeeplyEqualsChecker; @@ -94,6 +95,7 @@ public class BaseRowArrowReaderWriterTest extends ArrowReaderWriterTestBase rowFields = new ArrayList<>(); for (int i = 0; i < fieldTypes.size(); i++) { @@ -121,12 +123,12 @@ public class BaseRowArrowReaderWriterTest extends ArrowReaderWriterTestBase { fieldTypes.add(new FloatType()); fieldTypes.add(new DoubleType()); fieldTypes.add(new VarCharType()); + fieldTypes.add(new VarBinaryType()); List rowFields = new ArrayList<>(); for (int i = 0; i < fieldTypes.size(); i++) { @@ -88,10 +90,10 @@ public class RowArrowReaderWriterTest extends ArrowReaderWriterTestBase { @Override public Row[] getTestData() { - Row row1 = Row.of((byte) 1, (short) 2, 3, 4L, true, 1.0f, 1.0, "hello"); - Row row2 = Row.of(null, (short) 2, 3, 4L, false, 1.0f, 1.0, "中文"); - Row row3 = Row.of((byte) 1, null, 3, 4L, true, 1.0f, 1.0, "hello"); - Row row4 = Row.of(null, null, null, null, null, null, null, null); + Row row1 = Row.of((byte) 1, (short) 2, 3, 4L, true, 1.0f, 1.0, "hello", "hello".getBytes()); + Row row2 = Row.of(null, (short) 2, 3, 4L, false, 1.0f, 1.0, "中文", "中文".getBytes()); + Row row3 = Row.of((byte) 1, null, 3, 4L, true, 1.0f, 1.0, "hello", "hello".getBytes()); + Row row4 = Row.of(null, null, null, null, null, null, null, null, null); return new Row[]{row1, row2, row3, row4}; } } diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/util/StreamRecordUtils.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/util/StreamRecordUtils.java index 4a811186f4..0abb52722f 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/util/StreamRecordUtils.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/util/StreamRecordUtils.java @@ -115,6 +115,8 @@ public class StreamRecordUtils { writer.writeLong(j, (Long) value); } else if (value instanceof Boolean) { writer.writeBoolean(j, (Boolean) value); + } else if (value instanceof byte[]) { + writer.writeBinary(j, (byte[]) value); } else { throw new RuntimeException("Not support yet!"); } -- Gitee From 27d854726bcf124f1c9448b60586c6e31fa16479 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Fri, 13 Mar 2020 14:37:36 +0800 Subject: [PATCH 258/885] [FLINK-16608][python] Support DecimalType in vectorized Python UDF --- flink-python/pyflink/fn_execution/coders.py | 5 ++ .../pyflink/table/tests/test_pandas_udf.py | 32 +++++++--- .../flink/table/runtime/arrow/ArrowUtils.java | 43 ++++++++++++++ .../arrow/readers/DecimalFieldReader.java | 41 +++++++++++++ .../vectors/ArrowDecimalColumnVector.java | 52 ++++++++++++++++ .../arrow/writers/BaseRowDecimalWriter.java | 59 +++++++++++++++++++ .../runtime/arrow/writers/DecimalWriter.java | 59 +++++++++++++++++++ .../runtime/typeutils/PythonTypeUtils.java | 20 +++++++ .../table/runtime/arrow/ArrowUtilsTest.java | 7 +++ .../arrow/BaseRowArrowReaderWriterTest.java | 15 +++-- .../arrow/RowArrowReaderWriterTest.java | 11 ++-- .../table/runtime/util/StreamRecordUtils.java | 4 ++ 12 files changed, 331 insertions(+), 17 deletions(-) create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/DecimalFieldReader.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowDecimalColumnVector.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowDecimalWriter.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/DecimalWriter.java diff --git a/flink-python/pyflink/fn_execution/coders.py b/flink-python/pyflink/fn_execution/coders.py index d20a6d1f7a..2b068674d6 100644 --- a/flink-python/pyflink/fn_execution/coders.py +++ b/flink-python/pyflink/fn_execution/coders.py @@ -432,6 +432,11 @@ class ArrowCoder(DeterministicCoder): return pa.field(field.name, pa.utf8(), field.type.nullable) elif field.type.type_name == flink_fn_execution_pb2.Schema.TypeName.VARBINARY: return pa.field(field.name, pa.binary(), field.type.nullable) + elif field.type.type_name == flink_fn_execution_pb2.Schema.TypeName.DECIMAL: + return pa.field(field.name, + pa.decimal128(field.type.decimal_info.precision, + field.type.decimal_info.scale), + field.type.nullable) else: raise ValueError("field_type %s is not supported." % field.type) diff --git a/flink-python/pyflink/table/tests/test_pandas_udf.py b/flink-python/pyflink/table/tests/test_pandas_udf.py index fcf8a686b1..837f0bed0d 100644 --- a/flink-python/pyflink/table/tests/test_pandas_udf.py +++ b/flink-python/pyflink/table/tests/test_pandas_udf.py @@ -15,7 +15,7 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ - +import decimal import unittest from pyflink.table import DataTypes @@ -121,6 +121,12 @@ class PandasUDFITTests(object): 'varbinary_param of wrong type %s !' % type(varbinary_param[0]) return varbinary_param + def decimal_func(decimal_param): + assert isinstance(decimal_param, pd.Series) + assert isinstance(decimal_param[0], decimal.Decimal), \ + 'decimal_param of wrong type %s !' % type(decimal_param[0]) + return decimal_param + self.t_env.register_function( "tinyint_func", udf(tinyint_func, [DataTypes.TINYINT()], DataTypes.TINYINT(), udf_type="pandas")) @@ -157,16 +163,23 @@ class PandasUDFITTests(object): "varbinary_func", udf(varbinary_func, [DataTypes.BYTES()], DataTypes.BYTES(), udf_type="pandas")) + self.t_env.register_function( + "decimal_func", + udf(decimal_func, [DataTypes.DECIMAL(38, 18)], DataTypes.DECIMAL(38, 18), + udf_type="pandas")) + table_sink = source_sink_utils.TestAppendSink( - ['a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j', 'k'], + ['a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j', 'k', 'l', 'm'], [DataTypes.TINYINT(), DataTypes.SMALLINT(), DataTypes.INT(), DataTypes.BIGINT(), DataTypes.BOOLEAN(), DataTypes.BOOLEAN(), DataTypes.FLOAT(), DataTypes.DOUBLE(), - DataTypes.STRING(), DataTypes.STRING(), DataTypes.BYTES()]) + DataTypes.STRING(), DataTypes.STRING(), DataTypes.BYTES(), DataTypes.DECIMAL(38, 18), + DataTypes.DECIMAL(38, 18)]) self.t_env.register_table_sink("Results", table_sink) t = self.t_env.from_elements( [(1, 32767, -2147483648, 1, True, False, 1.0, 1.0, 'hello', '中文', - bytearray(b'flink'))], + bytearray(b'flink'), decimal.Decimal('1000000000000000000.05'), + decimal.Decimal('1000000000000000000.05999999999999999899999999999'))], DataTypes.ROW( [DataTypes.FIELD("a", DataTypes.TINYINT()), DataTypes.FIELD("b", DataTypes.SMALLINT()), @@ -178,7 +191,9 @@ class PandasUDFITTests(object): DataTypes.FIELD("h", DataTypes.DOUBLE()), DataTypes.FIELD("i", DataTypes.STRING()), DataTypes.FIELD("j", DataTypes.STRING()), - DataTypes.FIELD("k", DataTypes.BYTES())])) + DataTypes.FIELD("k", DataTypes.BYTES()), + DataTypes.FIELD("l", DataTypes.DECIMAL(38, 18)), + DataTypes.FIELD("m", DataTypes.DECIMAL(38, 18))])) t.select("tinyint_func(a)," "smallint_func(b)," @@ -190,13 +205,16 @@ class PandasUDFITTests(object): "double_func(h)," "varchar_func(i)," "varchar_func(j)," - "varbinary_func(k)") \ + "varbinary_func(k)," + "decimal_func(l)," + "decimal_func(m)") \ .insert_into("Results") self.t_env.execute("test") actual = source_sink_utils.results() self.assert_equals(actual, ["1,32767,-2147483648,1,true,false,1.0,1.0,hello,中文," - "[102, 108, 105, 110, 107]"]) + "[102, 108, 105, 110, 107],1000000000000000000.050000000000000000," + "1000000000000000000.059999999999999999"]) class StreamPandasUDFITTests(PandasUDFITTests, diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java index b9c3426fc1..9f795d1890 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java @@ -24,6 +24,7 @@ import org.apache.flink.table.dataformat.vector.ColumnVector; import org.apache.flink.table.runtime.arrow.readers.ArrowFieldReader; import org.apache.flink.table.runtime.arrow.readers.BigIntFieldReader; import org.apache.flink.table.runtime.arrow.readers.BooleanFieldReader; +import org.apache.flink.table.runtime.arrow.readers.DecimalFieldReader; import org.apache.flink.table.runtime.arrow.readers.DoubleFieldReader; import org.apache.flink.table.runtime.arrow.readers.FloatFieldReader; import org.apache.flink.table.runtime.arrow.readers.IntFieldReader; @@ -34,6 +35,7 @@ import org.apache.flink.table.runtime.arrow.readers.VarBinaryFieldReader; import org.apache.flink.table.runtime.arrow.readers.VarCharFieldReader; import org.apache.flink.table.runtime.arrow.vectors.ArrowBigIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowBooleanColumnVector; +import org.apache.flink.table.runtime.arrow.vectors.ArrowDecimalColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowDoubleColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowFloatColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowIntColumnVector; @@ -45,6 +47,7 @@ import org.apache.flink.table.runtime.arrow.vectors.BaseRowArrowReader; import org.apache.flink.table.runtime.arrow.writers.ArrowFieldWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowBigIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowBooleanWriter; +import org.apache.flink.table.runtime.arrow.writers.BaseRowDecimalWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowDoubleWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowFloatWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowIntWriter; @@ -54,6 +57,7 @@ import org.apache.flink.table.runtime.arrow.writers.BaseRowVarBinaryWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowVarCharWriter; import org.apache.flink.table.runtime.arrow.writers.BigIntWriter; import org.apache.flink.table.runtime.arrow.writers.BooleanWriter; +import org.apache.flink.table.runtime.arrow.writers.DecimalWriter; import org.apache.flink.table.runtime.arrow.writers.DoubleWriter; import org.apache.flink.table.runtime.arrow.writers.FloatWriter; import org.apache.flink.table.runtime.arrow.writers.IntWriter; @@ -63,9 +67,11 @@ import org.apache.flink.table.runtime.arrow.writers.VarBinaryWriter; import org.apache.flink.table.runtime.arrow.writers.VarCharWriter; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.DecimalType; import org.apache.flink.table.types.logical.DoubleType; import org.apache.flink.table.types.logical.FloatType; import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LegacyTypeInformationType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.SmallIntType; @@ -78,6 +84,7 @@ import org.apache.flink.types.Row; import org.apache.arrow.memory.RootAllocator; import org.apache.arrow.vector.BigIntVector; import org.apache.arrow.vector.BitVector; +import org.apache.arrow.vector.DecimalVector; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.Float4Vector; import org.apache.arrow.vector.Float8Vector; @@ -93,6 +100,7 @@ import org.apache.arrow.vector.types.pojo.Field; import org.apache.arrow.vector.types.pojo.FieldType; import org.apache.arrow.vector.types.pojo.Schema; +import java.math.BigDecimal; import java.util.ArrayList; import java.util.Collection; import java.util.List; @@ -158,6 +166,9 @@ public final class ArrowUtils { return new VarCharWriter((VarCharVector) vector); } else if (vector instanceof VarBinaryVector) { return new VarBinaryWriter((VarBinaryVector) vector); + } else if (vector instanceof DecimalVector) { + DecimalVector decimalVector = (DecimalVector) vector; + return new DecimalWriter(decimalVector, getPrecision(decimalVector), decimalVector.getScale()); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -198,6 +209,9 @@ public final class ArrowUtils { return new BaseRowVarCharWriter((VarCharVector) vector); } else if (vector instanceof VarBinaryVector) { return new BaseRowVarBinaryWriter((VarBinaryVector) vector); + } else if (vector instanceof DecimalVector) { + DecimalVector decimalVector = (DecimalVector) vector; + return new BaseRowDecimalWriter(decimalVector, getPrecision(decimalVector), decimalVector.getScale()); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -236,6 +250,8 @@ public final class ArrowUtils { return new VarCharFieldReader((VarCharVector) vector); } else if (vector instanceof VarBinaryVector) { return new VarBinaryFieldReader((VarBinaryVector) vector); + } else if (vector instanceof DecimalVector) { + return new DecimalFieldReader((DecimalVector) vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -274,6 +290,8 @@ public final class ArrowUtils { return new ArrowVarCharColumnVector((VarCharVector) vector); } else if (vector instanceof VarBinaryVector) { return new ArrowVarBinaryColumnVector((VarBinaryVector) vector); + } else if (vector instanceof DecimalVector) { + return new ArrowDecimalColumnVector((DecimalVector) vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -329,10 +347,35 @@ public final class ArrowUtils { return ArrowType.Binary.INSTANCE; } + @Override + public ArrowType visit(DecimalType decimalType) { + return new ArrowType.Decimal(decimalType.getPrecision(), decimalType.getScale()); + } + @Override protected ArrowType defaultMethod(LogicalType logicalType) { + if (logicalType instanceof LegacyTypeInformationType) { + Class typeClass = ((LegacyTypeInformationType) logicalType).getTypeInformation().getTypeClass(); + if (typeClass == BigDecimal.class) { + // Because we can't get precision and scale from legacy BIG_DEC_TYPE_INFO, + // we set the precision and scale to default value compatible with python. + return new ArrowType.Decimal(38, 18); + } + } throw new UnsupportedOperationException(String.format( "Python vectorized UDF doesn't support logical type %s currently.", logicalType.asSummaryString())); } } + + private static int getPrecision(DecimalVector decimalVector) { + int precision = -1; + try { + java.lang.reflect.Field precisionField = decimalVector.getClass().getDeclaredField("precision"); + precisionField.setAccessible(true); + precision = (int) precisionField.get(decimalVector); + } catch (NoSuchFieldException | IllegalAccessException e) { + // should not happen, ignore + } + return precision; + } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/DecimalFieldReader.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/DecimalFieldReader.java new file mode 100644 index 0000000000..17b5e1c2f5 --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/DecimalFieldReader.java @@ -0,0 +1,41 @@ +/* + * 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.table.runtime.arrow.readers; + +import org.apache.flink.annotation.Internal; + +import org.apache.arrow.vector.DecimalVector; + +import java.math.BigDecimal; + +/** + * {@link ArrowFieldReader} for Decimal. + */ +@Internal +public final class DecimalFieldReader extends ArrowFieldReader { + + public DecimalFieldReader(DecimalVector decimalVector) { + super(decimalVector); + } + + @Override + public BigDecimal read(int index) { + return ((DecimalVector) getValueVector()).getObject(index); + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowDecimalColumnVector.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowDecimalColumnVector.java new file mode 100644 index 0000000000..b842d41fcf --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowDecimalColumnVector.java @@ -0,0 +1,52 @@ +/* + * 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.table.runtime.arrow.vectors; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.dataformat.Decimal; +import org.apache.flink.table.dataformat.vector.DecimalColumnVector; +import org.apache.flink.util.Preconditions; + +import org.apache.arrow.vector.DecimalVector; + +/** + * Arrow column vector for Decimal. + */ +@Internal +public final class ArrowDecimalColumnVector implements DecimalColumnVector { + + /** + * Container which is used to store the sequence of Decimal values of a column to read. + */ + private final DecimalVector decimalVector; + + public ArrowDecimalColumnVector(DecimalVector decimalVector) { + this.decimalVector = Preconditions.checkNotNull(decimalVector); + } + + @Override + public Decimal getDecimal(int i, int precision, int scale) { + return Decimal.fromBigDecimal(decimalVector.getObject(i), precision, scale); + } + + @Override + public boolean isNullAt(int i) { + return decimalVector.isNull(i); + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowDecimalWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowDecimalWriter.java new file mode 100644 index 0000000000..ee0ddad851 --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowDecimalWriter.java @@ -0,0 +1,59 @@ +/* + * 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.table.runtime.arrow.writers; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.dataformat.BaseRow; + +import org.apache.arrow.vector.DecimalVector; + +import java.math.BigDecimal; + +import static org.apache.flink.table.runtime.typeutils.PythonTypeUtils.fromBigDecimal; + +/** + * {@link ArrowFieldWriter} for Decimal. + */ +@Internal +public final class BaseRowDecimalWriter extends ArrowFieldWriter { + + private final int precision; + private final int scale; + + public BaseRowDecimalWriter(DecimalVector decimalVector, int precision, int scale) { + super(decimalVector); + this.precision = precision; + this.scale = scale; + } + + @Override + public void doWrite(BaseRow row, int ordinal) { + if (row.isNullAt(ordinal)) { + ((DecimalVector) getValueVector()).setNull(getCount()); + } else { + BigDecimal bigDecimal = row.getDecimal(ordinal, precision, scale).toBigDecimal(); + bigDecimal = fromBigDecimal(bigDecimal, precision, scale); + if (bigDecimal == null) { + ((DecimalVector) getValueVector()).setNull(getCount()); + } else { + ((DecimalVector) getValueVector()).setSafe(getCount(), bigDecimal); + } + } + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/DecimalWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/DecimalWriter.java new file mode 100644 index 0000000000..85da42edfb --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/DecimalWriter.java @@ -0,0 +1,59 @@ +/* + * 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.table.runtime.arrow.writers; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.types.Row; + +import org.apache.arrow.vector.DecimalVector; + +import java.math.BigDecimal; + +import static org.apache.flink.table.runtime.typeutils.PythonTypeUtils.fromBigDecimal; + +/** + * {@link ArrowFieldWriter} for Decimal. + */ +@Internal +public final class DecimalWriter extends ArrowFieldWriter { + + private final int precision; + private final int scale; + + public DecimalWriter(DecimalVector decimalVector, int precision, int scale) { + super(decimalVector); + this.precision = precision; + this.scale = scale; + } + + @Override + public void doWrite(Row value, int ordinal) { + if (value.getField(ordinal) == null) { + ((DecimalVector) getValueVector()).setNull(getCount()); + } else { + BigDecimal bigDecimal = (BigDecimal) value.getField(ordinal); + bigDecimal = fromBigDecimal(bigDecimal, precision, scale); + if (bigDecimal == null) { + ((DecimalVector) getValueVector()).setNull(getCount()); + } else { + ((DecimalVector) getValueVector()).setSafe(getCount(), bigDecimal); + } + } + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java index 60f79a0a03..1f4731fff2 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java @@ -32,6 +32,7 @@ import org.apache.flink.api.common.typeutils.base.ShortSerializer; import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer; import org.apache.flink.api.java.typeutils.runtime.RowSerializer; import org.apache.flink.fnexecution.v1.FlinkFnApi; +import org.apache.flink.table.dataformat.Decimal; import org.apache.flink.table.runtime.typeutils.serializers.python.BaseArraySerializer; import org.apache.flink.table.runtime.typeutils.serializers.python.BaseMapSerializer; import org.apache.flink.table.runtime.typeutils.serializers.python.BaseRowSerializer; @@ -66,6 +67,7 @@ import org.apache.flink.table.types.logical.utils.LogicalTypeDefaultVisitor; import java.lang.reflect.Array; import java.math.BigDecimal; +import java.math.RoundingMode; import java.sql.Date; import java.sql.Time; import java.sql.Timestamp; @@ -87,6 +89,24 @@ public final class PythonTypeUtils { return logicalType.accept(new LogicalTypeToBlinkTypeSerializerConverter()); } + /** + * Convert the specified bigDecimal according to the specified precision and scale. + * The specified bigDecimal may be rounded to have the specified scale and then + * the specified precision is checked. If precision overflow, it will return `null`. + * + *

    Note: The implementation refers to {@link Decimal#fromBigDecimal}. + */ + public static BigDecimal fromBigDecimal(BigDecimal bigDecimal, int precision, int scale) { + if (bigDecimal.scale() != scale || bigDecimal.precision() > precision) { + // need adjust the precision and scale + bigDecimal = bigDecimal.setScale(scale, RoundingMode.HALF_UP); + if (bigDecimal.precision() > precision) { + return null; + } + } + return bigDecimal; + } + /** * Convert LogicalType to conversion class for flink planner. */ diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java index 31057eefd4..14f9f65d3a 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java @@ -24,6 +24,7 @@ import org.apache.flink.table.dataformat.vector.ColumnVector; import org.apache.flink.table.runtime.arrow.readers.ArrowFieldReader; import org.apache.flink.table.runtime.arrow.readers.BigIntFieldReader; import org.apache.flink.table.runtime.arrow.readers.BooleanFieldReader; +import org.apache.flink.table.runtime.arrow.readers.DecimalFieldReader; import org.apache.flink.table.runtime.arrow.readers.DoubleFieldReader; import org.apache.flink.table.runtime.arrow.readers.FloatFieldReader; import org.apache.flink.table.runtime.arrow.readers.IntFieldReader; @@ -34,6 +35,7 @@ import org.apache.flink.table.runtime.arrow.readers.VarBinaryFieldReader; import org.apache.flink.table.runtime.arrow.readers.VarCharFieldReader; import org.apache.flink.table.runtime.arrow.vectors.ArrowBigIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowBooleanColumnVector; +import org.apache.flink.table.runtime.arrow.vectors.ArrowDecimalColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowDoubleColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowFloatColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowIntColumnVector; @@ -45,6 +47,7 @@ import org.apache.flink.table.runtime.arrow.vectors.BaseRowArrowReader; import org.apache.flink.table.runtime.arrow.writers.ArrowFieldWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowBigIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowBooleanWriter; +import org.apache.flink.table.runtime.arrow.writers.BaseRowDecimalWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowDoubleWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowFloatWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowIntWriter; @@ -54,6 +57,7 @@ import org.apache.flink.table.runtime.arrow.writers.BaseRowVarBinaryWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowVarCharWriter; import org.apache.flink.table.runtime.arrow.writers.BigIntWriter; import org.apache.flink.table.runtime.arrow.writers.BooleanWriter; +import org.apache.flink.table.runtime.arrow.writers.DecimalWriter; import org.apache.flink.table.runtime.arrow.writers.DoubleWriter; import org.apache.flink.table.runtime.arrow.writers.FloatWriter; import org.apache.flink.table.runtime.arrow.writers.IntWriter; @@ -63,6 +67,7 @@ import org.apache.flink.table.runtime.arrow.writers.VarBinaryWriter; import org.apache.flink.table.runtime.arrow.writers.VarCharWriter; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.DecimalType; import org.apache.flink.table.types.logical.DoubleType; import org.apache.flink.table.types.logical.FloatType; import org.apache.flink.table.types.logical.IntType; @@ -119,6 +124,8 @@ public class ArrowUtilsTest { VarCharWriter.class, BaseRowVarCharWriter.class, VarCharFieldReader.class, ArrowVarCharColumnVector.class)); testFields.add(Tuple7.of("f9", new VarBinaryType(), ArrowType.Binary.INSTANCE, VarBinaryWriter.class, BaseRowVarBinaryWriter.class, VarBinaryFieldReader.class, ArrowVarBinaryColumnVector.class)); + testFields.add(Tuple7.of("f10", new DecimalType(10, 3), new ArrowType.Decimal(10, 3), + DecimalWriter.class, BaseRowDecimalWriter.class, DecimalFieldReader.class, ArrowDecimalColumnVector.class)); List rowFields = new ArrayList<>(); for (Tuple7, Class, Class, Class> field : testFields) { diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java index 5fd0da47fc..7b6460a14e 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java @@ -22,10 +22,12 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.table.dataformat.BaseRow; import org.apache.flink.table.dataformat.BinaryRow; +import org.apache.flink.table.dataformat.Decimal; import org.apache.flink.table.runtime.typeutils.BaseRowSerializer; import org.apache.flink.table.runtime.util.StreamRecordUtils; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.DecimalType; import org.apache.flink.table.types.logical.DoubleType; import org.apache.flink.table.types.logical.FloatType; import org.apache.flink.table.types.logical.IntType; @@ -96,6 +98,7 @@ public class BaseRowArrowReaderWriterTest extends ArrowReaderWriterTestBase rowFields = new ArrayList<>(); for (int i = 0; i < fieldTypes.size(); i++) { @@ -123,12 +126,12 @@ public class BaseRowArrowReaderWriterTest extends ArrowReaderWriterTestBase { fieldTypes.add(new DoubleType()); fieldTypes.add(new VarCharType()); fieldTypes.add(new VarBinaryType()); + fieldTypes.add(new DecimalType(10, 0)); List rowFields = new ArrayList<>(); for (int i = 0; i < fieldTypes.size(); i++) { @@ -90,10 +93,10 @@ public class RowArrowReaderWriterTest extends ArrowReaderWriterTestBase { @Override public Row[] getTestData() { - Row row1 = Row.of((byte) 1, (short) 2, 3, 4L, true, 1.0f, 1.0, "hello", "hello".getBytes()); - Row row2 = Row.of(null, (short) 2, 3, 4L, false, 1.0f, 1.0, "中文", "中文".getBytes()); - Row row3 = Row.of((byte) 1, null, 3, 4L, true, 1.0f, 1.0, "hello", "hello".getBytes()); - Row row4 = Row.of(null, null, null, null, null, null, null, null, null); + Row row1 = Row.of((byte) 1, (short) 2, 3, 4L, true, 1.0f, 1.0, "hello", "hello".getBytes(), new BigDecimal(1)); + Row row2 = Row.of(null, (short) 2, 3, 4L, false, 1.0f, 1.0, "中文", "中文".getBytes(), new BigDecimal(1)); + Row row3 = Row.of((byte) 1, null, 3, 4L, true, 1.0f, 1.0, "hello", "hello".getBytes(), new BigDecimal(1)); + Row row4 = Row.of(null, null, null, null, null, null, null, null, null, null); return new Row[]{row1, row2, row3, row4}; } } diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/util/StreamRecordUtils.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/util/StreamRecordUtils.java index 0abb52722f..fa1ea1f3e9 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/util/StreamRecordUtils.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/util/StreamRecordUtils.java @@ -23,6 +23,7 @@ import org.apache.flink.table.dataformat.BaseRow; import org.apache.flink.table.dataformat.BinaryRow; import org.apache.flink.table.dataformat.BinaryRowWriter; import org.apache.flink.table.dataformat.BinaryString; +import org.apache.flink.table.dataformat.Decimal; import org.apache.flink.table.dataformat.GenericRow; import org.apache.flink.table.dataformat.util.BaseRowUtil; @@ -117,6 +118,9 @@ public class StreamRecordUtils { writer.writeBoolean(j, (Boolean) value); } else if (value instanceof byte[]) { writer.writeBinary(j, (byte[]) value); + } else if (value instanceof Decimal) { + Decimal decimal = (Decimal) value; + writer.writeDecimal(j, decimal, decimal.getPrecision()); } else { throw new RuntimeException("Not support yet!"); } -- Gitee From 9992b3774f48e570193f567a05e91b0fd35f80c5 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Fri, 13 Mar 2020 14:37:56 +0800 Subject: [PATCH 259/885] [FLINK-16608][python] Support DateType in vectorized Python UDF --- flink-python/pyflink/fn_execution/coders.py | 2 + .../pyflink/table/tests/test_pandas_udf.py | 26 +++++++--- .../flink/table/runtime/arrow/ArrowUtils.java | 20 ++++++++ .../arrow/readers/DateFieldReader.java | 46 +++++++++++++++++ .../arrow/vectors/ArrowDateColumnVector.java | 51 +++++++++++++++++++ .../arrow/writers/BaseRowDateWriter.java | 44 ++++++++++++++++ .../runtime/arrow/writers/DateWriter.java | 48 +++++++++++++++++ .../runtime/typeutils/PythonTypeUtils.java | 35 +++++++++++++ .../serializers/python/DateSerializer.java | 23 ++------- .../table/runtime/arrow/ArrowUtilsTest.java | 8 +++ .../arrow/BaseRowArrowReaderWriterTest.java | 14 ++--- .../arrow/RowArrowReaderWriterTest.java | 11 ++-- 12 files changed, 292 insertions(+), 36 deletions(-) create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/DateFieldReader.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowDateColumnVector.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowDateWriter.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/DateWriter.java diff --git a/flink-python/pyflink/fn_execution/coders.py b/flink-python/pyflink/fn_execution/coders.py index 2b068674d6..fbe8b7c33a 100644 --- a/flink-python/pyflink/fn_execution/coders.py +++ b/flink-python/pyflink/fn_execution/coders.py @@ -437,6 +437,8 @@ class ArrowCoder(DeterministicCoder): pa.decimal128(field.type.decimal_info.precision, field.type.decimal_info.scale), field.type.nullable) + elif field.type.type_name == flink_fn_execution_pb2.Schema.TypeName.DATE: + return pa.field(field.name, pa.date32(), field.type.nullable) else: raise ValueError("field_type %s is not supported." % field.type) diff --git a/flink-python/pyflink/table/tests/test_pandas_udf.py b/flink-python/pyflink/table/tests/test_pandas_udf.py index 837f0bed0d..56ede1af87 100644 --- a/flink-python/pyflink/table/tests/test_pandas_udf.py +++ b/flink-python/pyflink/table/tests/test_pandas_udf.py @@ -15,6 +15,7 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ +import datetime import decimal import unittest @@ -127,6 +128,12 @@ class PandasUDFITTests(object): 'decimal_param of wrong type %s !' % type(decimal_param[0]) return decimal_param + def date_func(date_param): + assert isinstance(date_param, pd.Series) + assert isinstance(date_param[0], datetime.date), \ + 'date_param of wrong type %s !' % type(date_param[0]) + return date_param + self.t_env.register_function( "tinyint_func", udf(tinyint_func, [DataTypes.TINYINT()], DataTypes.TINYINT(), udf_type="pandas")) @@ -168,18 +175,23 @@ class PandasUDFITTests(object): udf(decimal_func, [DataTypes.DECIMAL(38, 18)], DataTypes.DECIMAL(38, 18), udf_type="pandas")) + self.t_env.register_function( + "date_func", + udf(date_func, [DataTypes.DATE()], DataTypes.DATE(), udf_type="pandas")) + table_sink = source_sink_utils.TestAppendSink( - ['a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j', 'k', 'l', 'm'], + ['a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j', 'k', 'l', 'm', 'n'], [DataTypes.TINYINT(), DataTypes.SMALLINT(), DataTypes.INT(), DataTypes.BIGINT(), DataTypes.BOOLEAN(), DataTypes.BOOLEAN(), DataTypes.FLOAT(), DataTypes.DOUBLE(), DataTypes.STRING(), DataTypes.STRING(), DataTypes.BYTES(), DataTypes.DECIMAL(38, 18), - DataTypes.DECIMAL(38, 18)]) + DataTypes.DECIMAL(38, 18), DataTypes.DATE()]) self.t_env.register_table_sink("Results", table_sink) t = self.t_env.from_elements( [(1, 32767, -2147483648, 1, True, False, 1.0, 1.0, 'hello', '中文', bytearray(b'flink'), decimal.Decimal('1000000000000000000.05'), - decimal.Decimal('1000000000000000000.05999999999999999899999999999'))], + decimal.Decimal('1000000000000000000.05999999999999999899999999999'), + datetime.date(2014, 9, 13))], DataTypes.ROW( [DataTypes.FIELD("a", DataTypes.TINYINT()), DataTypes.FIELD("b", DataTypes.SMALLINT()), @@ -193,7 +205,8 @@ class PandasUDFITTests(object): DataTypes.FIELD("j", DataTypes.STRING()), DataTypes.FIELD("k", DataTypes.BYTES()), DataTypes.FIELD("l", DataTypes.DECIMAL(38, 18)), - DataTypes.FIELD("m", DataTypes.DECIMAL(38, 18))])) + DataTypes.FIELD("m", DataTypes.DECIMAL(38, 18)), + DataTypes.FIELD("n", DataTypes.DATE())])) t.select("tinyint_func(a)," "smallint_func(b)," @@ -207,14 +220,15 @@ class PandasUDFITTests(object): "varchar_func(j)," "varbinary_func(k)," "decimal_func(l)," - "decimal_func(m)") \ + "decimal_func(m)," + "date_func(n)") \ .insert_into("Results") self.t_env.execute("test") actual = source_sink_utils.results() self.assert_equals(actual, ["1,32767,-2147483648,1,true,false,1.0,1.0,hello,中文," "[102, 108, 105, 110, 107],1000000000000000000.050000000000000000," - "1000000000000000000.059999999999999999"]) + "1000000000000000000.059999999999999999,2014-09-13"]) class StreamPandasUDFITTests(PandasUDFITTests, diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java index 9f795d1890..91a331c613 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java @@ -24,6 +24,7 @@ import org.apache.flink.table.dataformat.vector.ColumnVector; import org.apache.flink.table.runtime.arrow.readers.ArrowFieldReader; import org.apache.flink.table.runtime.arrow.readers.BigIntFieldReader; import org.apache.flink.table.runtime.arrow.readers.BooleanFieldReader; +import org.apache.flink.table.runtime.arrow.readers.DateFieldReader; import org.apache.flink.table.runtime.arrow.readers.DecimalFieldReader; import org.apache.flink.table.runtime.arrow.readers.DoubleFieldReader; import org.apache.flink.table.runtime.arrow.readers.FloatFieldReader; @@ -35,6 +36,7 @@ import org.apache.flink.table.runtime.arrow.readers.VarBinaryFieldReader; import org.apache.flink.table.runtime.arrow.readers.VarCharFieldReader; import org.apache.flink.table.runtime.arrow.vectors.ArrowBigIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowBooleanColumnVector; +import org.apache.flink.table.runtime.arrow.vectors.ArrowDateColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowDecimalColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowDoubleColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowFloatColumnVector; @@ -47,6 +49,7 @@ import org.apache.flink.table.runtime.arrow.vectors.BaseRowArrowReader; import org.apache.flink.table.runtime.arrow.writers.ArrowFieldWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowBigIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowBooleanWriter; +import org.apache.flink.table.runtime.arrow.writers.BaseRowDateWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowDecimalWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowDoubleWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowFloatWriter; @@ -57,6 +60,7 @@ import org.apache.flink.table.runtime.arrow.writers.BaseRowVarBinaryWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowVarCharWriter; import org.apache.flink.table.runtime.arrow.writers.BigIntWriter; import org.apache.flink.table.runtime.arrow.writers.BooleanWriter; +import org.apache.flink.table.runtime.arrow.writers.DateWriter; import org.apache.flink.table.runtime.arrow.writers.DecimalWriter; import org.apache.flink.table.runtime.arrow.writers.DoubleWriter; import org.apache.flink.table.runtime.arrow.writers.FloatWriter; @@ -67,6 +71,7 @@ import org.apache.flink.table.runtime.arrow.writers.VarBinaryWriter; import org.apache.flink.table.runtime.arrow.writers.VarCharWriter; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.DateType; import org.apache.flink.table.types.logical.DecimalType; import org.apache.flink.table.types.logical.DoubleType; import org.apache.flink.table.types.logical.FloatType; @@ -84,6 +89,7 @@ import org.apache.flink.types.Row; import org.apache.arrow.memory.RootAllocator; import org.apache.arrow.vector.BigIntVector; import org.apache.arrow.vector.BitVector; +import org.apache.arrow.vector.DateDayVector; import org.apache.arrow.vector.DecimalVector; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.Float4Vector; @@ -94,6 +100,7 @@ import org.apache.arrow.vector.TinyIntVector; import org.apache.arrow.vector.VarBinaryVector; import org.apache.arrow.vector.VarCharVector; import org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.arrow.vector.types.DateUnit; import org.apache.arrow.vector.types.FloatingPointPrecision; import org.apache.arrow.vector.types.pojo.ArrowType; import org.apache.arrow.vector.types.pojo.Field; @@ -169,6 +176,8 @@ public final class ArrowUtils { } else if (vector instanceof DecimalVector) { DecimalVector decimalVector = (DecimalVector) vector; return new DecimalWriter(decimalVector, getPrecision(decimalVector), decimalVector.getScale()); + } else if (vector instanceof DateDayVector) { + return new DateWriter((DateDayVector) vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -212,6 +221,8 @@ public final class ArrowUtils { } else if (vector instanceof DecimalVector) { DecimalVector decimalVector = (DecimalVector) vector; return new BaseRowDecimalWriter(decimalVector, getPrecision(decimalVector), decimalVector.getScale()); + } else if (vector instanceof DateDayVector) { + return new BaseRowDateWriter((DateDayVector) vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -252,6 +263,8 @@ public final class ArrowUtils { return new VarBinaryFieldReader((VarBinaryVector) vector); } else if (vector instanceof DecimalVector) { return new DecimalFieldReader((DecimalVector) vector); + } else if (vector instanceof DateDayVector) { + return new DateFieldReader((DateDayVector) vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -292,6 +305,8 @@ public final class ArrowUtils { return new ArrowVarBinaryColumnVector((VarBinaryVector) vector); } else if (vector instanceof DecimalVector) { return new ArrowDecimalColumnVector((DecimalVector) vector); + } else if (vector instanceof DateDayVector) { + return new ArrowDateColumnVector((DateDayVector) vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -352,6 +367,11 @@ public final class ArrowUtils { return new ArrowType.Decimal(decimalType.getPrecision(), decimalType.getScale()); } + @Override + public ArrowType visit(DateType dateType) { + return new ArrowType.Date(DateUnit.DAY); + } + @Override protected ArrowType defaultMethod(LogicalType logicalType) { if (logicalType instanceof LegacyTypeInformationType) { diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/DateFieldReader.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/DateFieldReader.java new file mode 100644 index 0000000000..76db46bdc2 --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/DateFieldReader.java @@ -0,0 +1,46 @@ +/* + * 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.table.runtime.arrow.readers; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.runtime.functions.SqlDateTimeUtils; + +import org.apache.arrow.vector.DateDayVector; + +import java.sql.Date; + +/** + * {@link ArrowFieldReader} for Date. + */ +@Internal +public final class DateFieldReader extends ArrowFieldReader { + + public DateFieldReader(DateDayVector dateDayVector) { + super(dateDayVector); + } + + @Override + public Date read(int index) { + if (getValueVector().isNull(index)) { + return null; + } else { + return SqlDateTimeUtils.internalToDate(((DateDayVector) getValueVector()).get(index)); + } + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowDateColumnVector.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowDateColumnVector.java new file mode 100644 index 0000000000..16f566f182 --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowDateColumnVector.java @@ -0,0 +1,51 @@ +/* + * 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.table.runtime.arrow.vectors; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.dataformat.vector.IntColumnVector; +import org.apache.flink.util.Preconditions; + +import org.apache.arrow.vector.DateDayVector; + +/** + * Arrow column vector for Date. + */ +@Internal +public final class ArrowDateColumnVector implements IntColumnVector { + + /** + * Container which is used to store the sequence of date values of a column to read. + */ + private final DateDayVector dateDayVector; + + public ArrowDateColumnVector(DateDayVector dateDayVector) { + this.dateDayVector = Preconditions.checkNotNull(dateDayVector); + } + + @Override + public int getInt(int i) { + return dateDayVector.get(i); + } + + @Override + public boolean isNullAt(int i) { + return dateDayVector.isNull(i); + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowDateWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowDateWriter.java new file mode 100644 index 0000000000..1ce87590eb --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowDateWriter.java @@ -0,0 +1,44 @@ +/* + * 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.table.runtime.arrow.writers; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.dataformat.BaseRow; + +import org.apache.arrow.vector.DateDayVector; + +/** + * {@link ArrowFieldWriter} for Date. + */ +@Internal +public final class BaseRowDateWriter extends ArrowFieldWriter { + + public BaseRowDateWriter(DateDayVector dateDayVector) { + super(dateDayVector); + } + + @Override + public void doWrite(BaseRow row, int ordinal) { + if (row.isNullAt(ordinal)) { + ((DateDayVector) getValueVector()).setNull(getCount()); + } else { + ((DateDayVector) getValueVector()).setSafe(getCount(), row.getInt(ordinal)); + } + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/DateWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/DateWriter.java new file mode 100644 index 0000000000..27a6a91a18 --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/DateWriter.java @@ -0,0 +1,48 @@ +/* + * 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.table.runtime.arrow.writers; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.runtime.typeutils.PythonTypeUtils; +import org.apache.flink.types.Row; + +import org.apache.arrow.vector.DateDayVector; + +import java.sql.Date; + +/** + * {@link ArrowFieldWriter} for Date. + */ +@Internal +public final class DateWriter extends ArrowFieldWriter { + + public DateWriter(DateDayVector dateDayVector) { + super(dateDayVector); + } + + @Override + public void doWrite(Row value, int ordinal) { + if (value.getField(ordinal) == null) { + ((DateDayVector) getValueVector()).setNull(getCount()); + } else { + ((DateDayVector) getValueVector()).setSafe( + getCount(), PythonTypeUtils.dateToInternal(((Date) value.getField(ordinal)))); + } + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java index 1f4731fff2..7569f1cfa3 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java @@ -33,6 +33,7 @@ import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerial import org.apache.flink.api.java.typeutils.runtime.RowSerializer; import org.apache.flink.fnexecution.v1.FlinkFnApi; import org.apache.flink.table.dataformat.Decimal; +import org.apache.flink.table.runtime.functions.SqlDateTimeUtils; import org.apache.flink.table.runtime.typeutils.serializers.python.BaseArraySerializer; import org.apache.flink.table.runtime.typeutils.serializers.python.BaseMapSerializer; import org.apache.flink.table.runtime.typeutils.serializers.python.BaseRowSerializer; @@ -71,6 +72,7 @@ import java.math.RoundingMode; import java.sql.Date; import java.sql.Time; import java.sql.Timestamp; +import java.util.TimeZone; /** * Utilities for converting Flink logical types, such as convert it to the related @@ -81,6 +83,16 @@ public final class PythonTypeUtils { private static final String EMPTY_STRING = ""; + /** + * The local time zone. + */ + private static final TimeZone LOCAL_TZ = TimeZone.getDefault(); + + /** + * The number of milliseconds in a day. + */ + private static final long MILLIS_PER_DAY = 86400000L; // = 24 * 60 * 60 * 1000 + public static TypeSerializer toFlinkTypeSerializer(LogicalType logicalType) { return logicalType.accept(new LogicalTypeToTypeSerializerConverter()); } @@ -107,6 +119,29 @@ public final class PythonTypeUtils { return bigDecimal; } + /** + * Converts the internal representation of a SQL DATE (int) to the Java + * type used for UDF parameters ({@link java.sql.Date}). + * + *

    Note: The implementation refers to {@link SqlDateTimeUtils#internalToDate}. + */ + public static java.sql.Date internalToDate(int v) { + // note that, in this case, can't handle Daylight Saving Time + final long t = v * MILLIS_PER_DAY; + return new java.sql.Date(t - LOCAL_TZ.getOffset(t)); + } + + /** + * Converts the Java type used for UDF parameters of SQL DATE type + * ({@link java.sql.Date}) to internal representation (int). + * + *

    Note: The implementation refers to {@link SqlDateTimeUtils#dateToInternal}. + */ + public static int dateToInternal(java.sql.Date date) { + long ts = date.getTime() + LOCAL_TZ.getOffset(date.getTime()); + return (int) (ts / MILLIS_PER_DAY); + } + /** * Convert LogicalType to conversion class for flink planner. */ diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/DateSerializer.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/DateSerializer.java index 5c279d95f8..d66c28ed02 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/DateSerializer.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/DateSerializer.java @@ -24,10 +24,10 @@ import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.table.runtime.typeutils.PythonTypeUtils; import java.io.IOException; import java.sql.Date; -import java.util.TimeZone; /** * Takes int instead of long as the serialized value. It not only reduces the length of @@ -39,13 +39,6 @@ public class DateSerializer extends TypeSerializerSingleton { private static final long serialVersionUID = 1L; - /** - * The local time zone. - */ - private static final TimeZone LOCAL_TZ = TimeZone.getDefault(); - - private static final long MILLIS_PER_DAY = 86400000L; // = 24 * 60 * 60 * 1000 - public static final DateSerializer INSTANCE = new DateSerializer(); @Override @@ -85,22 +78,12 @@ public class DateSerializer extends TypeSerializerSingleton { if (record == null) { throw new IllegalArgumentException("The Date record must not be null."); } - target.writeInt(dateToInternal(record)); + target.writeInt(PythonTypeUtils.dateToInternal(record)); } @Override public Date deserialize(DataInputView source) throws IOException { - return internalToDate(source.readInt()); - } - - private int dateToInternal(Date date) { - long ts = date.getTime() + LOCAL_TZ.getOffset(date.getTime()); - return (int) (ts / MILLIS_PER_DAY); - } - - private Date internalToDate(int v) { - final long t = v * MILLIS_PER_DAY; - return new Date(t - LOCAL_TZ.getOffset(t)); + return PythonTypeUtils.internalToDate(source.readInt()); } @Override diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java index 14f9f65d3a..052d59117a 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java @@ -24,6 +24,7 @@ import org.apache.flink.table.dataformat.vector.ColumnVector; import org.apache.flink.table.runtime.arrow.readers.ArrowFieldReader; import org.apache.flink.table.runtime.arrow.readers.BigIntFieldReader; import org.apache.flink.table.runtime.arrow.readers.BooleanFieldReader; +import org.apache.flink.table.runtime.arrow.readers.DateFieldReader; import org.apache.flink.table.runtime.arrow.readers.DecimalFieldReader; import org.apache.flink.table.runtime.arrow.readers.DoubleFieldReader; import org.apache.flink.table.runtime.arrow.readers.FloatFieldReader; @@ -35,6 +36,7 @@ import org.apache.flink.table.runtime.arrow.readers.VarBinaryFieldReader; import org.apache.flink.table.runtime.arrow.readers.VarCharFieldReader; import org.apache.flink.table.runtime.arrow.vectors.ArrowBigIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowBooleanColumnVector; +import org.apache.flink.table.runtime.arrow.vectors.ArrowDateColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowDecimalColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowDoubleColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowFloatColumnVector; @@ -47,6 +49,7 @@ import org.apache.flink.table.runtime.arrow.vectors.BaseRowArrowReader; import org.apache.flink.table.runtime.arrow.writers.ArrowFieldWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowBigIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowBooleanWriter; +import org.apache.flink.table.runtime.arrow.writers.BaseRowDateWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowDecimalWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowDoubleWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowFloatWriter; @@ -57,6 +60,7 @@ import org.apache.flink.table.runtime.arrow.writers.BaseRowVarBinaryWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowVarCharWriter; import org.apache.flink.table.runtime.arrow.writers.BigIntWriter; import org.apache.flink.table.runtime.arrow.writers.BooleanWriter; +import org.apache.flink.table.runtime.arrow.writers.DateWriter; import org.apache.flink.table.runtime.arrow.writers.DecimalWriter; import org.apache.flink.table.runtime.arrow.writers.DoubleWriter; import org.apache.flink.table.runtime.arrow.writers.FloatWriter; @@ -67,6 +71,7 @@ import org.apache.flink.table.runtime.arrow.writers.VarBinaryWriter; import org.apache.flink.table.runtime.arrow.writers.VarCharWriter; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.DateType; import org.apache.flink.table.types.logical.DecimalType; import org.apache.flink.table.types.logical.DoubleType; import org.apache.flink.table.types.logical.FloatType; @@ -81,6 +86,7 @@ import org.apache.flink.types.Row; import org.apache.arrow.memory.BufferAllocator; import org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.arrow.vector.types.DateUnit; import org.apache.arrow.vector.types.FloatingPointPrecision; import org.apache.arrow.vector.types.pojo.ArrowType; import org.apache.arrow.vector.types.pojo.Field; @@ -126,6 +132,8 @@ public class ArrowUtilsTest { VarBinaryWriter.class, BaseRowVarBinaryWriter.class, VarBinaryFieldReader.class, ArrowVarBinaryColumnVector.class)); testFields.add(Tuple7.of("f10", new DecimalType(10, 3), new ArrowType.Decimal(10, 3), DecimalWriter.class, BaseRowDecimalWriter.class, DecimalFieldReader.class, ArrowDecimalColumnVector.class)); + testFields.add(Tuple7.of("f11", new DateType(), new ArrowType.Date(DateUnit.DAY), + DateWriter.class, BaseRowDateWriter.class, DateFieldReader.class, ArrowDateColumnVector.class)); List rowFields = new ArrayList<>(); for (Tuple7, Class, Class, Class> field : testFields) { diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java index 7b6460a14e..22b346fecd 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java @@ -27,6 +27,7 @@ import org.apache.flink.table.runtime.typeutils.BaseRowSerializer; import org.apache.flink.table.runtime.util.StreamRecordUtils; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.DateType; import org.apache.flink.table.types.logical.DecimalType; import org.apache.flink.table.types.logical.DoubleType; import org.apache.flink.table.types.logical.FloatType; @@ -99,6 +100,7 @@ public class BaseRowArrowReaderWriterTest extends ArrowReaderWriterTestBase rowFields = new ArrayList<>(); for (int i = 0; i < fieldTypes.size(); i++) { @@ -126,12 +128,12 @@ public class BaseRowArrowReaderWriterTest extends ArrowReaderWriterTestBase { fieldTypes.add(new VarCharType()); fieldTypes.add(new VarBinaryType()); fieldTypes.add(new DecimalType(10, 0)); + fieldTypes.add(new DateType()); List rowFields = new ArrayList<>(); for (int i = 0; i < fieldTypes.size(); i++) { @@ -93,10 +96,10 @@ public class RowArrowReaderWriterTest extends ArrowReaderWriterTestBase { @Override public Row[] getTestData() { - Row row1 = Row.of((byte) 1, (short) 2, 3, 4L, true, 1.0f, 1.0, "hello", "hello".getBytes(), new BigDecimal(1)); - Row row2 = Row.of(null, (short) 2, 3, 4L, false, 1.0f, 1.0, "中文", "中文".getBytes(), new BigDecimal(1)); - Row row3 = Row.of((byte) 1, null, 3, 4L, true, 1.0f, 1.0, "hello", "hello".getBytes(), new BigDecimal(1)); - Row row4 = Row.of(null, null, null, null, null, null, null, null, null, null); + Row row1 = Row.of((byte) 1, (short) 2, 3, 4L, true, 1.0f, 1.0, "hello", "hello".getBytes(), new BigDecimal(1), SqlDateTimeUtils.internalToDate(100)); + Row row2 = Row.of(null, (short) 2, 3, 4L, false, 1.0f, 1.0, "中文", "中文".getBytes(), new BigDecimal(1), SqlDateTimeUtils.internalToDate(100)); + Row row3 = Row.of((byte) 1, null, 3, 4L, true, 1.0f, 1.0, "hello", "hello".getBytes(), new BigDecimal(1), SqlDateTimeUtils.internalToDate(100)); + Row row4 = Row.of(null, null, null, null, null, null, null, null, null, null, null); return new Row[]{row1, row2, row3, row4}; } } -- Gitee From 28bcbd0cf68a4e1820bc091a8be0417f060dd01c Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Sat, 14 Mar 2020 17:45:46 +0800 Subject: [PATCH 260/885] [FLINK-16608][python] Support TimeType in vectorized Python UDF --- flink-python/pyflink/fn_execution/coders.py | 9 +++ .../pyflink/table/tests/test_pandas_udf.py | 24 ++++-- .../flink/table/runtime/arrow/ArrowUtils.java | 35 +++++++++ .../arrow/readers/TimeFieldReader.java | 70 +++++++++++++++++ .../arrow/vectors/ArrowTimeColumnVector.java | 68 +++++++++++++++++ .../arrow/writers/BaseRowTimeWriter.java | 62 +++++++++++++++ .../runtime/arrow/writers/TimeWriter.java | 76 +++++++++++++++++++ .../table/runtime/arrow/ArrowUtilsTest.java | 14 ++++ .../arrow/BaseRowArrowReaderWriterTest.java | 17 +++-- .../arrow/RowArrowReaderWriterTest.java | 16 +++- 10 files changed, 375 insertions(+), 16 deletions(-) create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/TimeFieldReader.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowTimeColumnVector.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowTimeWriter.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/TimeWriter.java diff --git a/flink-python/pyflink/fn_execution/coders.py b/flink-python/pyflink/fn_execution/coders.py index fbe8b7c33a..759fed839e 100644 --- a/flink-python/pyflink/fn_execution/coders.py +++ b/flink-python/pyflink/fn_execution/coders.py @@ -439,6 +439,15 @@ class ArrowCoder(DeterministicCoder): field.type.nullable) elif field.type.type_name == flink_fn_execution_pb2.Schema.TypeName.DATE: return pa.field(field.name, pa.date32(), field.type.nullable) + elif field.type.type_name == flink_fn_execution_pb2.Schema.TypeName.TIME: + if field.type.time_info.precision == 0: + return pa.field(field.name, pa.time32('s'), field.type.nullable) + elif 1 <= field.type.time_type.precision <= 3: + return pa.field(field.name, pa.time32('ms'), field.type.nullable) + elif 4 <= field.type.time_type.precision <= 6: + return pa.field(field.name, pa.time64('us'), field.type.nullable) + else: + return pa.field(field.name, pa.time64('ns'), field.type.nullable) else: raise ValueError("field_type %s is not supported." % field.type) diff --git a/flink-python/pyflink/table/tests/test_pandas_udf.py b/flink-python/pyflink/table/tests/test_pandas_udf.py index 56ede1af87..95468e0c11 100644 --- a/flink-python/pyflink/table/tests/test_pandas_udf.py +++ b/flink-python/pyflink/table/tests/test_pandas_udf.py @@ -134,6 +134,12 @@ class PandasUDFITTests(object): 'date_param of wrong type %s !' % type(date_param[0]) return date_param + def time_func(time_param): + assert isinstance(time_param, pd.Series) + assert isinstance(time_param[0], datetime.time), \ + 'time_param of wrong type %s !' % type(time_param[0]) + return time_param + self.t_env.register_function( "tinyint_func", udf(tinyint_func, [DataTypes.TINYINT()], DataTypes.TINYINT(), udf_type="pandas")) @@ -179,19 +185,23 @@ class PandasUDFITTests(object): "date_func", udf(date_func, [DataTypes.DATE()], DataTypes.DATE(), udf_type="pandas")) + self.t_env.register_function( + "time_func", + udf(time_func, [DataTypes.TIME()], DataTypes.TIME(), udf_type="pandas")) + table_sink = source_sink_utils.TestAppendSink( - ['a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j', 'k', 'l', 'm', 'n'], + ['a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j', 'k', 'l', 'm', 'n', 'o'], [DataTypes.TINYINT(), DataTypes.SMALLINT(), DataTypes.INT(), DataTypes.BIGINT(), DataTypes.BOOLEAN(), DataTypes.BOOLEAN(), DataTypes.FLOAT(), DataTypes.DOUBLE(), DataTypes.STRING(), DataTypes.STRING(), DataTypes.BYTES(), DataTypes.DECIMAL(38, 18), - DataTypes.DECIMAL(38, 18), DataTypes.DATE()]) + DataTypes.DECIMAL(38, 18), DataTypes.DATE(), DataTypes.TIME()]) self.t_env.register_table_sink("Results", table_sink) t = self.t_env.from_elements( [(1, 32767, -2147483648, 1, True, False, 1.0, 1.0, 'hello', '中文', bytearray(b'flink'), decimal.Decimal('1000000000000000000.05'), decimal.Decimal('1000000000000000000.05999999999999999899999999999'), - datetime.date(2014, 9, 13))], + datetime.date(2014, 9, 13), datetime.time(hour=1, minute=0, second=1))], DataTypes.ROW( [DataTypes.FIELD("a", DataTypes.TINYINT()), DataTypes.FIELD("b", DataTypes.SMALLINT()), @@ -206,7 +216,8 @@ class PandasUDFITTests(object): DataTypes.FIELD("k", DataTypes.BYTES()), DataTypes.FIELD("l", DataTypes.DECIMAL(38, 18)), DataTypes.FIELD("m", DataTypes.DECIMAL(38, 18)), - DataTypes.FIELD("n", DataTypes.DATE())])) + DataTypes.FIELD("n", DataTypes.DATE()), + DataTypes.FIELD("o", DataTypes.TIME())])) t.select("tinyint_func(a)," "smallint_func(b)," @@ -221,14 +232,15 @@ class PandasUDFITTests(object): "varbinary_func(k)," "decimal_func(l)," "decimal_func(m)," - "date_func(n)") \ + "date_func(n)," + "time_func(o)") \ .insert_into("Results") self.t_env.execute("test") actual = source_sink_utils.results() self.assert_equals(actual, ["1,32767,-2147483648,1,true,false,1.0,1.0,hello,中文," "[102, 108, 105, 110, 107],1000000000000000000.050000000000000000," - "1000000000000000000.059999999999999999,2014-09-13"]) + "1000000000000000000.059999999999999999,2014-09-13,01:00:01"]) class StreamPandasUDFITTests(PandasUDFITTests, diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java index 91a331c613..4ad3af7e7c 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java @@ -31,6 +31,7 @@ import org.apache.flink.table.runtime.arrow.readers.FloatFieldReader; import org.apache.flink.table.runtime.arrow.readers.IntFieldReader; import org.apache.flink.table.runtime.arrow.readers.RowArrowReader; import org.apache.flink.table.runtime.arrow.readers.SmallIntFieldReader; +import org.apache.flink.table.runtime.arrow.readers.TimeFieldReader; import org.apache.flink.table.runtime.arrow.readers.TinyIntFieldReader; import org.apache.flink.table.runtime.arrow.readers.VarBinaryFieldReader; import org.apache.flink.table.runtime.arrow.readers.VarCharFieldReader; @@ -42,6 +43,7 @@ import org.apache.flink.table.runtime.arrow.vectors.ArrowDoubleColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowFloatColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowSmallIntColumnVector; +import org.apache.flink.table.runtime.arrow.vectors.ArrowTimeColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowTinyIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowVarBinaryColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowVarCharColumnVector; @@ -55,6 +57,7 @@ import org.apache.flink.table.runtime.arrow.writers.BaseRowDoubleWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowFloatWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowSmallIntWriter; +import org.apache.flink.table.runtime.arrow.writers.BaseRowTimeWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowTinyIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowVarBinaryWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowVarCharWriter; @@ -66,6 +69,7 @@ import org.apache.flink.table.runtime.arrow.writers.DoubleWriter; import org.apache.flink.table.runtime.arrow.writers.FloatWriter; import org.apache.flink.table.runtime.arrow.writers.IntWriter; import org.apache.flink.table.runtime.arrow.writers.SmallIntWriter; +import org.apache.flink.table.runtime.arrow.writers.TimeWriter; import org.apache.flink.table.runtime.arrow.writers.TinyIntWriter; import org.apache.flink.table.runtime.arrow.writers.VarBinaryWriter; import org.apache.flink.table.runtime.arrow.writers.VarCharWriter; @@ -80,6 +84,7 @@ import org.apache.flink.table.types.logical.LegacyTypeInformationType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.SmallIntType; +import org.apache.flink.table.types.logical.TimeType; import org.apache.flink.table.types.logical.TinyIntType; import org.apache.flink.table.types.logical.VarBinaryType; import org.apache.flink.table.types.logical.VarCharType; @@ -96,12 +101,17 @@ import org.apache.arrow.vector.Float4Vector; import org.apache.arrow.vector.Float8Vector; import org.apache.arrow.vector.IntVector; import org.apache.arrow.vector.SmallIntVector; +import org.apache.arrow.vector.TimeMicroVector; +import org.apache.arrow.vector.TimeMilliVector; +import org.apache.arrow.vector.TimeNanoVector; +import org.apache.arrow.vector.TimeSecVector; import org.apache.arrow.vector.TinyIntVector; import org.apache.arrow.vector.VarBinaryVector; import org.apache.arrow.vector.VarCharVector; import org.apache.arrow.vector.VectorSchemaRoot; import org.apache.arrow.vector.types.DateUnit; import org.apache.arrow.vector.types.FloatingPointPrecision; +import org.apache.arrow.vector.types.TimeUnit; import org.apache.arrow.vector.types.pojo.ArrowType; import org.apache.arrow.vector.types.pojo.Field; import org.apache.arrow.vector.types.pojo.FieldType; @@ -178,6 +188,9 @@ public final class ArrowUtils { return new DecimalWriter(decimalVector, getPrecision(decimalVector), decimalVector.getScale()); } else if (vector instanceof DateDayVector) { return new DateWriter((DateDayVector) vector); + } else if (vector instanceof TimeSecVector || vector instanceof TimeMilliVector || + vector instanceof TimeMicroVector || vector instanceof TimeNanoVector) { + return new TimeWriter(vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -223,6 +236,9 @@ public final class ArrowUtils { return new BaseRowDecimalWriter(decimalVector, getPrecision(decimalVector), decimalVector.getScale()); } else if (vector instanceof DateDayVector) { return new BaseRowDateWriter((DateDayVector) vector); + } else if (vector instanceof TimeSecVector || vector instanceof TimeMilliVector || + vector instanceof TimeMicroVector || vector instanceof TimeNanoVector) { + return new BaseRowTimeWriter(vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -265,6 +281,9 @@ public final class ArrowUtils { return new DecimalFieldReader((DecimalVector) vector); } else if (vector instanceof DateDayVector) { return new DateFieldReader((DateDayVector) vector); + } else if (vector instanceof TimeSecVector || vector instanceof TimeMilliVector || + vector instanceof TimeMicroVector || vector instanceof TimeNanoVector) { + return new TimeFieldReader(vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -307,6 +326,9 @@ public final class ArrowUtils { return new ArrowDecimalColumnVector((DecimalVector) vector); } else if (vector instanceof DateDayVector) { return new ArrowDateColumnVector((DateDayVector) vector); + } else if (vector instanceof TimeSecVector || vector instanceof TimeMilliVector || + vector instanceof TimeMicroVector || vector instanceof TimeNanoVector) { + return new ArrowTimeColumnVector(vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -372,6 +394,19 @@ public final class ArrowUtils { return new ArrowType.Date(DateUnit.DAY); } + @Override + public ArrowType visit(TimeType timeType) { + if (timeType.getPrecision() == 0) { + return new ArrowType.Time(TimeUnit.SECOND, 32); + } else if (timeType.getPrecision() >= 1 && timeType.getPrecision() <= 3) { + return new ArrowType.Time(TimeUnit.MILLISECOND, 32); + } else if (timeType.getPrecision() >= 4 && timeType.getPrecision() <= 6) { + return new ArrowType.Time(TimeUnit.MICROSECOND, 64); + } else { + return new ArrowType.Time(TimeUnit.NANOSECOND, 64); + } + } + @Override protected ArrowType defaultMethod(LogicalType logicalType) { if (logicalType instanceof LegacyTypeInformationType) { diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/TimeFieldReader.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/TimeFieldReader.java new file mode 100644 index 0000000000..723cc4f389 --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/TimeFieldReader.java @@ -0,0 +1,70 @@ +/* + * 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.table.runtime.arrow.readers; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.util.Preconditions; + +import org.apache.arrow.vector.TimeMicroVector; +import org.apache.arrow.vector.TimeMilliVector; +import org.apache.arrow.vector.TimeNanoVector; +import org.apache.arrow.vector.TimeSecVector; +import org.apache.arrow.vector.ValueVector; + +import java.sql.Time; +import java.util.TimeZone; + +/** + * {@link ArrowFieldReader} for Time. + */ +@Internal +public final class TimeFieldReader extends ArrowFieldReader

    + * 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.table.runtime.arrow.vectors; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.dataformat.vector.IntColumnVector; +import org.apache.flink.util.Preconditions; + +import org.apache.arrow.vector.TimeMicroVector; +import org.apache.arrow.vector.TimeMilliVector; +import org.apache.arrow.vector.TimeNanoVector; +import org.apache.arrow.vector.TimeSecVector; +import org.apache.arrow.vector.ValueVector; + +/** + * Arrow column vector for Time. + */ +@Internal +public final class ArrowTimeColumnVector implements IntColumnVector { + + /** + * Container which is used to store the sequence of time values of a column to read. + */ + private final ValueVector valueVector; + + public ArrowTimeColumnVector(ValueVector valueVector) { + this.valueVector = Preconditions.checkNotNull(valueVector); + Preconditions.checkState( + valueVector instanceof TimeSecVector || + valueVector instanceof TimeMilliVector || + valueVector instanceof TimeMicroVector || + valueVector instanceof TimeNanoVector); + } + + @Override + public int getInt(int i) { + if (valueVector instanceof TimeSecVector) { + return ((TimeSecVector) valueVector).get(i) * 1000; + } else if (valueVector instanceof TimeMilliVector) { + return ((TimeMilliVector) valueVector).get(i); + } else if (valueVector instanceof TimeMicroVector) { + return (int) (((TimeMicroVector) valueVector).get(i) / 1000); + } else { + return (int) (((TimeNanoVector) valueVector).get(i) / 1000000); + } + } + + @Override + public boolean isNullAt(int i) { + return valueVector.isNull(i); + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowTimeWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowTimeWriter.java new file mode 100644 index 0000000000..1ccdc4948a --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowTimeWriter.java @@ -0,0 +1,62 @@ +/* + * 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.table.runtime.arrow.writers; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.dataformat.BaseRow; +import org.apache.flink.util.Preconditions; + +import org.apache.arrow.vector.BaseFixedWidthVector; +import org.apache.arrow.vector.TimeMicroVector; +import org.apache.arrow.vector.TimeMilliVector; +import org.apache.arrow.vector.TimeNanoVector; +import org.apache.arrow.vector.TimeSecVector; +import org.apache.arrow.vector.ValueVector; + +/** + * {@link ArrowFieldWriter} for Time. + */ +@Internal +public final class BaseRowTimeWriter extends ArrowFieldWriter { + + public BaseRowTimeWriter(ValueVector valueVector) { + super(valueVector); + Preconditions.checkState( + valueVector instanceof TimeSecVector || + valueVector instanceof TimeMilliVector || + valueVector instanceof TimeMicroVector || + valueVector instanceof TimeNanoVector); + } + + @Override + public void doWrite(BaseRow row, int ordinal) { + ValueVector valueVector = getValueVector(); + if (row.isNullAt(ordinal)) { + ((BaseFixedWidthVector) valueVector).setNull(getCount()); + } else if (valueVector instanceof TimeSecVector) { + ((TimeSecVector) valueVector).setSafe(getCount(), row.getInt(ordinal) / 1000); + } else if (valueVector instanceof TimeMilliVector) { + ((TimeMilliVector) valueVector).setSafe(getCount(), row.getInt(ordinal)); + } else if (valueVector instanceof TimeMicroVector) { + ((TimeMicroVector) valueVector).setSafe(getCount(), row.getInt(ordinal) * 1000L); + } else { + ((TimeNanoVector) valueVector).setSafe(getCount(), row.getInt(ordinal) * 1000000L); + } + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/TimeWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/TimeWriter.java new file mode 100644 index 0000000000..0356d2076d --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/TimeWriter.java @@ -0,0 +1,76 @@ +/* + * 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.table.runtime.arrow.writers; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + +import org.apache.arrow.vector.BaseFixedWidthVector; +import org.apache.arrow.vector.TimeMicroVector; +import org.apache.arrow.vector.TimeMilliVector; +import org.apache.arrow.vector.TimeNanoVector; +import org.apache.arrow.vector.TimeSecVector; +import org.apache.arrow.vector.ValueVector; + +import java.sql.Time; +import java.util.TimeZone; + +/** + * {@link ArrowFieldWriter} for Time. + */ +@Internal +public final class TimeWriter extends ArrowFieldWriter { + + // The local time zone. + private static final TimeZone LOCAL_TZ = TimeZone.getDefault(); + + private static final long MILLIS_PER_DAY = 86400000L; // = 24 * 60 * 60 * 1000 + + public TimeWriter(ValueVector valueVector) { + super(valueVector); + Preconditions.checkState( + valueVector instanceof TimeSecVector || + valueVector instanceof TimeMilliVector || + valueVector instanceof TimeMicroVector || + valueVector instanceof TimeNanoVector); + } + + @Override + public void doWrite(Row row, int ordinal) { + ValueVector valueVector = getValueVector(); + if (row.getField(ordinal) == null) { + ((BaseFixedWidthVector) getValueVector()).setNull(getCount()); + } else { + Time time = (Time) row.getField(ordinal); + long ts = time.getTime() + LOCAL_TZ.getOffset(time.getTime()); + int timeMilli = (int) (ts % MILLIS_PER_DAY); + + if (valueVector instanceof TimeSecVector) { + ((TimeSecVector) valueVector).setSafe(getCount(), timeMilli / 1000); + } else if (valueVector instanceof TimeMilliVector) { + ((TimeMilliVector) valueVector).setSafe(getCount(), timeMilli); + } else if (valueVector instanceof TimeMicroVector) { + ((TimeMicroVector) valueVector).setSafe(getCount(), timeMilli * 1000L); + } else { + ((TimeNanoVector) valueVector).setSafe(getCount(), timeMilli * 1000000L); + } + } + } +} diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java index 052d59117a..7fb5f2fc1d 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java @@ -31,6 +31,7 @@ import org.apache.flink.table.runtime.arrow.readers.FloatFieldReader; import org.apache.flink.table.runtime.arrow.readers.IntFieldReader; import org.apache.flink.table.runtime.arrow.readers.RowArrowReader; import org.apache.flink.table.runtime.arrow.readers.SmallIntFieldReader; +import org.apache.flink.table.runtime.arrow.readers.TimeFieldReader; import org.apache.flink.table.runtime.arrow.readers.TinyIntFieldReader; import org.apache.flink.table.runtime.arrow.readers.VarBinaryFieldReader; import org.apache.flink.table.runtime.arrow.readers.VarCharFieldReader; @@ -42,6 +43,7 @@ import org.apache.flink.table.runtime.arrow.vectors.ArrowDoubleColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowFloatColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowSmallIntColumnVector; +import org.apache.flink.table.runtime.arrow.vectors.ArrowTimeColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowTinyIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowVarBinaryColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowVarCharColumnVector; @@ -55,6 +57,7 @@ import org.apache.flink.table.runtime.arrow.writers.BaseRowDoubleWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowFloatWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowSmallIntWriter; +import org.apache.flink.table.runtime.arrow.writers.BaseRowTimeWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowTinyIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowVarBinaryWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowVarCharWriter; @@ -66,6 +69,7 @@ import org.apache.flink.table.runtime.arrow.writers.DoubleWriter; import org.apache.flink.table.runtime.arrow.writers.FloatWriter; import org.apache.flink.table.runtime.arrow.writers.IntWriter; import org.apache.flink.table.runtime.arrow.writers.SmallIntWriter; +import org.apache.flink.table.runtime.arrow.writers.TimeWriter; import org.apache.flink.table.runtime.arrow.writers.TinyIntWriter; import org.apache.flink.table.runtime.arrow.writers.VarBinaryWriter; import org.apache.flink.table.runtime.arrow.writers.VarCharWriter; @@ -79,6 +83,7 @@ import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.SmallIntType; +import org.apache.flink.table.types.logical.TimeType; import org.apache.flink.table.types.logical.TinyIntType; import org.apache.flink.table.types.logical.VarBinaryType; import org.apache.flink.table.types.logical.VarCharType; @@ -88,6 +93,7 @@ import org.apache.arrow.memory.BufferAllocator; import org.apache.arrow.vector.VectorSchemaRoot; import org.apache.arrow.vector.types.DateUnit; import org.apache.arrow.vector.types.FloatingPointPrecision; +import org.apache.arrow.vector.types.TimeUnit; import org.apache.arrow.vector.types.pojo.ArrowType; import org.apache.arrow.vector.types.pojo.Field; import org.apache.arrow.vector.types.pojo.Schema; @@ -134,6 +140,14 @@ public class ArrowUtilsTest { DecimalWriter.class, BaseRowDecimalWriter.class, DecimalFieldReader.class, ArrowDecimalColumnVector.class)); testFields.add(Tuple7.of("f11", new DateType(), new ArrowType.Date(DateUnit.DAY), DateWriter.class, BaseRowDateWriter.class, DateFieldReader.class, ArrowDateColumnVector.class)); + testFields.add(Tuple7.of("f13", new TimeType(0), new ArrowType.Time(TimeUnit.SECOND, 32), + TimeWriter.class, BaseRowTimeWriter.class, TimeFieldReader.class, ArrowTimeColumnVector.class)); + testFields.add(Tuple7.of("f14", new TimeType(2), new ArrowType.Time(TimeUnit.MILLISECOND, 32), + TimeWriter.class, BaseRowTimeWriter.class, TimeFieldReader.class, ArrowTimeColumnVector.class)); + testFields.add(Tuple7.of("f15", new TimeType(4), new ArrowType.Time(TimeUnit.MICROSECOND, 64), + TimeWriter.class, BaseRowTimeWriter.class, TimeFieldReader.class, ArrowTimeColumnVector.class)); + testFields.add(Tuple7.of("f16", new TimeType(8), new ArrowType.Time(TimeUnit.NANOSECOND, 64), + TimeWriter.class, BaseRowTimeWriter.class, TimeFieldReader.class, ArrowTimeColumnVector.class)); List rowFields = new ArrayList<>(); for (Tuple7, Class, Class, Class> field : testFields) { diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java index 22b346fecd..db4bbbea47 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java @@ -35,6 +35,7 @@ import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.SmallIntType; +import org.apache.flink.table.types.logical.TimeType; import org.apache.flink.table.types.logical.TinyIntType; import org.apache.flink.table.types.logical.VarBinaryType; import org.apache.flink.table.types.logical.VarCharType; @@ -101,6 +102,10 @@ public class BaseRowArrowReaderWriterTest extends ArrowReaderWriterTestBase rowFields = new ArrayList<>(); for (int i = 0; i < fieldTypes.size(); i++) { @@ -128,12 +133,12 @@ public class BaseRowArrowReaderWriterTest extends ArrowReaderWriterTestBase { fieldTypes.add(new VarBinaryType()); fieldTypes.add(new DecimalType(10, 0)); fieldTypes.add(new DateType()); + fieldTypes.add(new TimeType(0)); + fieldTypes.add(new TimeType(2)); + fieldTypes.add(new TimeType(4)); + fieldTypes.add(new TimeType(8)); List rowFields = new ArrayList<>(); for (int i = 0; i < fieldTypes.size(); i++) { @@ -96,10 +101,13 @@ public class RowArrowReaderWriterTest extends ArrowReaderWriterTestBase { @Override public Row[] getTestData() { - Row row1 = Row.of((byte) 1, (short) 2, 3, 4L, true, 1.0f, 1.0, "hello", "hello".getBytes(), new BigDecimal(1), SqlDateTimeUtils.internalToDate(100)); - Row row2 = Row.of(null, (short) 2, 3, 4L, false, 1.0f, 1.0, "中文", "中文".getBytes(), new BigDecimal(1), SqlDateTimeUtils.internalToDate(100)); - Row row3 = Row.of((byte) 1, null, 3, 4L, true, 1.0f, 1.0, "hello", "hello".getBytes(), new BigDecimal(1), SqlDateTimeUtils.internalToDate(100)); - Row row4 = Row.of(null, null, null, null, null, null, null, null, null, null, null); + Row row1 = Row.of((byte) 1, (short) 2, 3, 4L, true, 1.0f, 1.0, "hello", "hello".getBytes(), new BigDecimal(1), SqlDateTimeUtils.internalToDate(100), + SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000)); + Row row2 = Row.of(null, (short) 2, 3, 4L, false, 1.0f, 1.0, "中文", "中文".getBytes(), new BigDecimal(1), SqlDateTimeUtils.internalToDate(100), + SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000)); + Row row3 = Row.of((byte) 1, null, 3, 4L, true, 1.0f, 1.0, "hello", "hello".getBytes(), new BigDecimal(1), SqlDateTimeUtils.internalToDate(100), + SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000)); + Row row4 = Row.of(null, null, null, null, null, null, null, null, null, null, null, null, null, null, null); return new Row[]{row1, row2, row3, row4}; } } -- Gitee From 813f590f9d92d154ec286377dfc12401aeae1b04 Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Sun, 15 Mar 2020 14:26:05 +0300 Subject: [PATCH 261/885] [FLINK-16225] Improve metaspace out-of-memory error handling thrown in user code Improve error message, explaining the possible reasons and ways to resolve. In case of metaspace OOM error, try a graceful TM shutdown. This closes #11408. --- .../org/apache/flink/util/ExceptionUtils.java | 52 +++++++++++++++++++ .../taskexecutor/TaskManagerRunner.java | 8 ++- .../flink/runtime/taskmanager/Task.java | 2 + 3 files changed, 60 insertions(+), 2 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java b/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java index ddd0276a05..5fc1bfe288 100644 --- a/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java +++ b/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java @@ -25,6 +25,7 @@ package org.apache.flink.util; import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.util.function.RunnableWithException; import javax.annotation.Nullable; @@ -48,6 +49,14 @@ public final class ExceptionUtils { /** The stringified representation of a null exception reference. */ public static final String STRINGIFIED_NULL_EXCEPTION = "(null)"; + private static final String TM_METASPACE_OOM_ERROR_MESSAGE = String.format( + "Metaspace. The metaspace out-of-memory error has occurred. This can mean two things: either the job requires " + + "a larger size of JVM metaspace to load classes or there is a class loading leak. In the first case " + + "'%s' configuration option should be increased. If the error persists (usually in cluster after " + + "several job (re-)submissions) then there is probably a class loading leak which has to be " + + "investigated and fixed. The task executor has to be shutdown...", + TaskManagerOptions.JVM_METASPACE.key()); + /** * Makes a string representation of the exception's stack trace, or "(null)", if the * exception is null. @@ -109,6 +118,49 @@ public final class ExceptionUtils { return isJvmFatalError(t) || t instanceof OutOfMemoryError; } + /** + * Generates new {@link OutOfMemoryError} with more detailed message. + * + *

    This method improves error message for metaspace {@link OutOfMemoryError}. + * It adds description of possible causes and ways of resolution. + * + * @param exception The exception to enrich. + * @return either enriched exception if needed or the original one. + */ + public static Throwable enrichTaskManagerOutOfMemoryError(Throwable exception) { + if (isMetaspaceOutOfMemoryError(exception)) { + return changeOutOfMemoryErrorMessage(exception, TM_METASPACE_OOM_ERROR_MESSAGE); + } + return exception; + } + + private static OutOfMemoryError changeOutOfMemoryErrorMessage(Throwable exception, String newMessage) { + Preconditions.checkArgument(exception instanceof OutOfMemoryError); + if (exception.getMessage().equals(newMessage)) { + return (OutOfMemoryError) exception; + } + OutOfMemoryError newError = new OutOfMemoryError(newMessage); + newError.initCause(exception.getCause()); + newError.setStackTrace(exception.getStackTrace()); + return newError; + } + + /** + * Checks whether the given exception indicates a JVM metaspace out-of-memory error. + * + * @param t The exception to check. + * @return True, if the exception is the metaspace {@link OutOfMemoryError}, false otherwise. + */ + public static boolean isMetaspaceOutOfMemoryError(Throwable t) { + return isOutOfMemoryErrorWithMessageStartingWith(t, "Metaspace"); + } + + private static boolean isOutOfMemoryErrorWithMessageStartingWith(Throwable t, String prefix) { + // the exact matching of the class is checked to avoid matching any custom subclasses of OutOfMemoryError + // as we are interested in the original exceptions, generated by JVM. + return t.getClass() == OutOfMemoryError.class && t.getMessage() != null && t.getMessage().startsWith(prefix); + } + /** * Rethrows the given {@code Throwable}, if it represents an error that is fatal to the JVM. * See {@link ExceptionUtils#isJvmFatalError(Throwable)} for a definition of fatal errors. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java index ca93e327d4..8ed4fe478e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java @@ -246,9 +246,13 @@ public class TaskManagerRunner implements FatalErrorHandler, AutoCloseableAsync @Override public void onFatalError(Throwable exception) { - LOG.error("Fatal error occurred while executing the TaskManager. Shutting it down...", exception); + Throwable enrichedException = ExceptionUtils.enrichTaskManagerOutOfMemoryError(exception); + LOG.error("Fatal error occurred while executing the TaskManager. Shutting it down...", enrichedException); - if (ExceptionUtils.isJvmFatalOrOutOfMemoryError(exception)) { + // In case of the Metaspace OutOfMemoryError, we expect that the graceful shutdown is possible, + // as it does not usually require more class loading to fail again with the Metaspace OutOfMemoryError. + if (ExceptionUtils.isJvmFatalOrOutOfMemoryError(enrichedException) && + !ExceptionUtils.isMetaspaceOutOfMemoryError(enrichedException)) { terminateJVM(); } else { closeAsync(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java index 30e7c8a1a3..25d5f8eda3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java @@ -752,6 +752,8 @@ public class Task implements Runnable, TaskSlotPayload, TaskActions, PartitionPr // an exception was thrown as a side effect of cancelling // ---------------------------------------------------------------- + t = ExceptionUtils.enrichTaskManagerOutOfMemoryError(t); + try { // check if the exception is unrecoverable if (ExceptionUtils.isJvmFatalError(t) || -- Gitee From 874a4af736c62e2be20b0b73d7b370ebeef36e73 Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Thu, 19 Mar 2020 11:46:46 +0300 Subject: [PATCH 262/885] [FLINK-16225] Add JVM Metaspace Error assumption test --- .../flink/test/util/TestProcessBuilder.java | 21 ++- .../runtime/util/ExceptionUtilsITCases.java | 121 ++++++++++++++++++ ...TaskManagerProcessFailureRecoveryTest.java | 2 +- .../ProcessFailureCancelingITCase.java | 4 +- 4 files changed, 141 insertions(+), 7 deletions(-) create mode 100644 flink-tests/src/test/java/org/apache/flink/runtime/util/ExceptionUtilsITCases.java diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestProcessBuilder.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestProcessBuilder.java index b98f2b0993..2e2d2045c9 100644 --- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestProcessBuilder.java +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestProcessBuilder.java @@ -70,10 +70,12 @@ public class TestProcessBuilder { commands.addAll(mainClassArgs); StringWriter processOutput = new StringWriter(); + StringWriter errorOutput = new StringWriter(); Process process = new ProcessBuilder(commands).start(); - new PipeForwarder(process.getErrorStream(), processOutput); + new PipeForwarder(process.getInputStream(), processOutput); + new PipeForwarder(process.getErrorStream(), errorOutput); - return new TestProcess(process, processOutput); + return new TestProcess(process, processOutput, errorOutput); } public TestProcessBuilder setJvmMemory(MemorySize jvmMemory) { @@ -81,6 +83,11 @@ public class TestProcessBuilder { return this; } + public TestProcessBuilder addJvmArg(String arg) { + jvmArgs.add(arg); + return this; + } + public TestProcessBuilder addMainClassArg(String arg) { mainClassArgs.add(arg); return this; @@ -100,20 +107,26 @@ public class TestProcessBuilder { public static class TestProcess { private final Process process; private final StringWriter processOutput; + private final StringWriter errorOutput; - public TestProcess(Process process, StringWriter processOutput) { + public TestProcess(Process process, StringWriter processOutput, StringWriter errorOutput) { this.process = process; this.processOutput = processOutput; + this.errorOutput = errorOutput; } public Process getProcess() { return process; } - public StringWriter getOutput() { + public StringWriter getProcessOutput() { return processOutput; } + public StringWriter getErrorOutput() { + return errorOutput; + } + public void destroy() { process.destroy(); } diff --git a/flink-tests/src/test/java/org/apache/flink/runtime/util/ExceptionUtilsITCases.java b/flink-tests/src/test/java/org/apache/flink/runtime/util/ExceptionUtilsITCases.java new file mode 100644 index 0000000000..045babb06c --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/runtime/util/ExceptionUtilsITCases.java @@ -0,0 +1,121 @@ +/* + * 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.runtime.util; + +import org.apache.flink.test.util.TestProcessBuilder; +import org.apache.flink.test.util.TestProcessBuilder.TestProcess; +import org.apache.flink.testutils.ClassLoaderUtils; +import org.apache.flink.testutils.ClassLoaderUtils.ClassLoaderBuilder; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.TestLogger; + +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; +import java.lang.management.ManagementFactory; +import java.lang.management.MemoryPoolMXBean; +import java.util.ArrayList; +import java.util.Collection; + +import static org.hamcrest.CoreMatchers.is; +import static org.junit.Assert.assertThat; + +/** + * Tests for {@link ExceptionUtils} which require to spawn JVM process and set JVM memory args. + */ +public class ExceptionUtilsITCases extends TestLogger { + private static final long INITIAL_BIG_METASPACE_SIZE = 32 * (1 << 20); // 32Mb + + @ClassRule + public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Test + public void testIsMetaspaceOutOfMemoryError() throws IOException, InterruptedException { + // load only one class and record required Metaspace + long okMetaspace = Long.parseLong(run(1, INITIAL_BIG_METASPACE_SIZE)); + // load more classes to cause 'OutOfMemoryError: Metaspace' + assertThat(run(1000, okMetaspace), is("")); + } + + private static String run(int numberOfLoadedClasses, long metaspaceSize) throws InterruptedException, IOException { + TestProcessBuilder taskManagerProcessBuilder = new TestProcessBuilder(DummyClassLoadingProgram.class.getName()); + taskManagerProcessBuilder.addJvmArg("-XX:-UseCompressedOops"); + taskManagerProcessBuilder.addJvmArg(String.format("-XX:MaxMetaspaceSize=%d", metaspaceSize)); + taskManagerProcessBuilder.addMainClassArg(Integer.toString(numberOfLoadedClasses)); + taskManagerProcessBuilder.addMainClassArg(TEMPORARY_FOLDER.getRoot().getAbsolutePath()); + TestProcess p = taskManagerProcessBuilder.start(); + p.getProcess().waitFor(); + assertThat(p.getErrorOutput().toString().trim(), is("")); + return p.getProcessOutput().toString().trim(); + } + + /** + * Dummy java program to generate Metaspace OOM. + */ + public static class DummyClassLoadingProgram { + private DummyClassLoadingProgram() { + } + + public static void main(String[] args) { + // trigger needed classes loaded + output(""); + ExceptionUtils.isMetaspaceOutOfMemoryError(new Exception()); + + Collection> classes = new ArrayList<>(); + int numberOfLoadedClasses = Integer.parseInt(args[0]); + try { + for (int index = 0; index < numberOfLoadedClasses; index++) { + classes.add(loadDummyClass(index, args[1])); + } + String out = classes.size() > 1 ? "Exception is not thrown, metaspace usage: " : ""; + output(out + getMetaspaceUsage()); + } catch (Throwable t) { + if (ExceptionUtils.isMetaspaceOutOfMemoryError(t)) { + return; + } + output("Wrong exception: " + t); + } + } + + private static Class loadDummyClass(int index, String folderToSaveSource) throws ClassNotFoundException, IOException { + String className = "DummyClass" + index; + String sourcePattern = "public class %s { @Override public String toString() { return \"%s\"; } }"; + ClassLoaderBuilder classLoaderBuilder = ClassLoaderUtils.withRoot(new File(folderToSaveSource)); + classLoaderBuilder.addClass(className, String.format(sourcePattern, className, "dummy")); + ClassLoader classLoader = classLoaderBuilder.build(); + return Class.forName(className, true, classLoader); + } + + private static long getMetaspaceUsage() { + for (MemoryPoolMXBean memoryMXBean : ManagementFactory.getMemoryPoolMXBeans()) { + if ("Metaspace".equals(memoryMXBean.getName())) { + return memoryMXBean.getUsage().getUsed(); + } + } + throw new RuntimeException("Metaspace usage is not found"); + } + + private static void output(String text) { + System.out.println(text); + } + } +} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java index f69545b886..d9c82303c0 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java @@ -258,7 +258,7 @@ public abstract class AbstractTaskManagerProcessFailureRecoveryTest extends Test System.out.println("-----------------------------------------"); System.out.println(" BEGIN SPAWNED PROCESS LOG FOR " + processName); System.out.println("-----------------------------------------"); - System.out.println(process.getOutput().toString()); + System.out.println(process.getErrorOutput().toString()); System.out.println("-----------------------------------------"); System.out.println(" END SPAWNED PROCESS LOG"); System.out.println("-----------------------------------------"); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java index 966b2fea84..9d1839511e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java @@ -237,11 +237,11 @@ public class ProcessFailureCancelingITCase extends TestLogger { // all seems well :-) } catch (Exception e) { - printProcessLog("TaskManager", taskManagerProcess.getOutput().toString()); + printProcessLog("TaskManager", taskManagerProcess.getErrorOutput().toString()); throw e; } catch (Error e) { - printProcessLog("TaskManager 1", taskManagerProcess.getOutput().toString()); + printProcessLog("TaskManager 1", taskManagerProcess.getErrorOutput().toString()); throw e; } finally { -- Gitee From a7521844d7a120b2e36dd7d57cea0fe5d819dbba Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Thu, 19 Mar 2020 16:16:58 +0300 Subject: [PATCH 263/885] [FLINK-15989] Improve direct out-of-memory error handling in MemorySegmentFactory --- .../core/memory/MemorySegmentFactory.java | 25 ++++++++++++++++- .../org/apache/flink/util/ExceptionUtils.java | 27 ++++++++++++++++++- 2 files changed, 50 insertions(+), 2 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentFactory.java b/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentFactory.java index c297a26480..760d2aca94 100644 --- a/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentFactory.java +++ b/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentFactory.java @@ -19,6 +19,10 @@ package org.apache.flink.core.memory; import org.apache.flink.annotation.Internal; +import org.apache.flink.util.ExceptionUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.nio.ByteBuffer; @@ -32,6 +36,7 @@ import java.nio.ByteBuffer; */ @Internal public final class MemorySegmentFactory { + private static final Logger LOG = LoggerFactory.getLogger(MemorySegmentFactory.class); /** * Creates a new memory segment that targets the given heap memory region. @@ -94,10 +99,28 @@ public final class MemorySegmentFactory { * @return A new memory segment, backed by unpooled off-heap memory. */ public static MemorySegment allocateUnpooledOffHeapMemory(int size, Object owner) { - ByteBuffer memory = ByteBuffer.allocateDirect(size); + ByteBuffer memory = allocateDirectMemory(size); return new HybridMemorySegment(memory, owner, null); } + private static ByteBuffer allocateDirectMemory(int size) { + //noinspection ErrorNotRethrown + try { + return ByteBuffer.allocateDirect(size); + } catch (OutOfMemoryError outOfMemoryError) { + // TODO: this error handling can be removed in future, + // once we find a common way to handle OOM errors in netty threads. + // Here we enrich it to propagate better OOM message to the receiver + // if it happens in a netty thread. + OutOfMemoryError enrichedOutOfMemoryError = (OutOfMemoryError) ExceptionUtils + .enrichTaskManagerOutOfMemoryError(outOfMemoryError); + if (ExceptionUtils.isDirectOutOfMemoryError(outOfMemoryError)) { + LOG.error("Cannot allocate direct memory segment", enrichedOutOfMemoryError); + } + throw enrichedOutOfMemoryError; + } + } + /** * Allocates an off-heap unsafe memory and creates a new memory segment to represent that memory. * diff --git a/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java b/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java index 5fc1bfe288..0a63ae5b4f 100644 --- a/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java +++ b/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java @@ -49,6 +49,19 @@ public final class ExceptionUtils { /** The stringified representation of a null exception reference. */ public static final String STRINGIFIED_NULL_EXCEPTION = "(null)"; + private static final String TM_DIRECT_OOM_ERROR_MESSAGE = String.format( + "Direct buffer memory. The direct out-of-memory error has occurred. This can mean two things: either job(s) require(s) " + + "a larger size of JVM direct memory or there is a direct memory leak. The direct memory can be " + + "allocated by user code or some of its dependencies. In this case '%s' configuration option should be " + + "increased. Flink framework and its dependencies also consume the direct memory, mostly for network " + + "communication. The most of network memory is managed by Flink and should not result in out-of-memory " + + "error. In certain special cases, in particular for jobs with high parallelism, the framework may " + + "require more direct memory which is not managed by Flink. In this case '%s' configuration option " + + "should be increased. If the error persists then there is probably a direct memory leak which has to " + + "be investigated and fixed. The task executor has to be shutdown...", + TaskManagerOptions.TASK_OFF_HEAP_MEMORY.key(), + TaskManagerOptions.FRAMEWORK_OFF_HEAP_MEMORY.key()); + private static final String TM_METASPACE_OOM_ERROR_MESSAGE = String.format( "Metaspace. The metaspace out-of-memory error has occurred. This can mean two things: either the job requires " + "a larger size of JVM metaspace to load classes or there is a class loading leak. In the first case " + @@ -121,7 +134,7 @@ public final class ExceptionUtils { /** * Generates new {@link OutOfMemoryError} with more detailed message. * - *

    This method improves error message for metaspace {@link OutOfMemoryError}. + *

    This method improves error message for direct and metaspace {@link OutOfMemoryError}. * It adds description of possible causes and ways of resolution. * * @param exception The exception to enrich. @@ -130,6 +143,8 @@ public final class ExceptionUtils { public static Throwable enrichTaskManagerOutOfMemoryError(Throwable exception) { if (isMetaspaceOutOfMemoryError(exception)) { return changeOutOfMemoryErrorMessage(exception, TM_METASPACE_OOM_ERROR_MESSAGE); + } else if (isDirectOutOfMemoryError(exception)) { + return changeOutOfMemoryErrorMessage(exception, TM_DIRECT_OOM_ERROR_MESSAGE); } return exception; } @@ -155,6 +170,16 @@ public final class ExceptionUtils { return isOutOfMemoryErrorWithMessageStartingWith(t, "Metaspace"); } + /** + * Checks whether the given exception indicates a JVM direct out-of-memory error. + * + * @param t The exception to check. + * @return True, if the exception is the direct {@link OutOfMemoryError}, false otherwise. + */ + public static boolean isDirectOutOfMemoryError(Throwable t) { + return isOutOfMemoryErrorWithMessageStartingWith(t, "Direct buffer memory"); + } + private static boolean isOutOfMemoryErrorWithMessageStartingWith(Throwable t, String prefix) { // the exact matching of the class is checked to avoid matching any custom subclasses of OutOfMemoryError // as we are interested in the original exceptions, generated by JVM. -- Gitee From f22756c02d2f33640f70cbc1870ce3928132cbe1 Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Thu, 19 Mar 2020 12:33:57 +0300 Subject: [PATCH 264/885] [FLINK-15989] Add JVM Direct Error assumption test --- .../runtime/util/ExceptionUtilsITCases.java | 78 ++++++++++++++++--- 1 file changed, 66 insertions(+), 12 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/runtime/util/ExceptionUtilsITCases.java b/flink-tests/src/test/java/org/apache/flink/runtime/util/ExceptionUtilsITCases.java index 045babb06c..fd42f59fab 100644 --- a/flink-tests/src/test/java/org/apache/flink/runtime/util/ExceptionUtilsITCases.java +++ b/flink-tests/src/test/java/org/apache/flink/runtime/util/ExceptionUtilsITCases.java @@ -33,8 +33,11 @@ import java.io.File; import java.io.IOException; import java.lang.management.ManagementFactory; import java.lang.management.MemoryPoolMXBean; +import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import static org.hamcrest.CoreMatchers.is; import static org.junit.Assert.assertThat; @@ -43,31 +46,82 @@ import static org.junit.Assert.assertThat; * Tests for {@link ExceptionUtils} which require to spawn JVM process and set JVM memory args. */ public class ExceptionUtilsITCases extends TestLogger { - private static final long INITIAL_BIG_METASPACE_SIZE = 32 * (1 << 20); // 32Mb + private static final int DIRECT_MEMORY_SIZE = 10 * 1024; // 10Kb + private static final int DIRECT_MEMORY_ALLOCATION_PAGE_SIZE = 1024; // 1Kb + private static final int DIRECT_MEMORY_PAGE_NUMBER = DIRECT_MEMORY_SIZE / DIRECT_MEMORY_ALLOCATION_PAGE_SIZE; + private static final long INITIAL_BIG_METASPACE_SIZE = 128 * (1 << 20); // 128Mb @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @Test + public void testIsDirectOutOfMemoryError() throws IOException, InterruptedException { + String className = DummyDirectAllocatingProgram.class.getName(); + String out = run(className, Collections.emptyList(), DIRECT_MEMORY_SIZE, -1); + assertThat(out, is("")); + } + @Test public void testIsMetaspaceOutOfMemoryError() throws IOException, InterruptedException { + String className = DummyClassLoadingProgram.class.getName(); // load only one class and record required Metaspace - long okMetaspace = Long.parseLong(run(1, INITIAL_BIG_METASPACE_SIZE)); + String normalOut = run(className, getDummyClassLoadingProgramArgs(1), -1, INITIAL_BIG_METASPACE_SIZE); + long okMetaspace = Long.parseLong(normalOut); // load more classes to cause 'OutOfMemoryError: Metaspace' - assertThat(run(1000, okMetaspace), is("")); + String oomOut = run(className, getDummyClassLoadingProgramArgs(1000), -1, okMetaspace); + assertThat(oomOut, is("")); } - private static String run(int numberOfLoadedClasses, long metaspaceSize) throws InterruptedException, IOException { - TestProcessBuilder taskManagerProcessBuilder = new TestProcessBuilder(DummyClassLoadingProgram.class.getName()); - taskManagerProcessBuilder.addJvmArg("-XX:-UseCompressedOops"); - taskManagerProcessBuilder.addJvmArg(String.format("-XX:MaxMetaspaceSize=%d", metaspaceSize)); - taskManagerProcessBuilder.addMainClassArg(Integer.toString(numberOfLoadedClasses)); - taskManagerProcessBuilder.addMainClassArg(TEMPORARY_FOLDER.getRoot().getAbsolutePath()); + private static String run( + String className, + Iterable args, + long directMemorySize, + long metaspaceSize) throws InterruptedException, IOException { + TestProcessBuilder taskManagerProcessBuilder = new TestProcessBuilder(className); + if (directMemorySize > 0) { + taskManagerProcessBuilder.addJvmArg(String.format("-XX:MaxDirectMemorySize=%d", directMemorySize)); + } + if (metaspaceSize > 0) { + taskManagerProcessBuilder.addJvmArg("-XX:-UseCompressedOops"); + taskManagerProcessBuilder.addJvmArg(String.format("-XX:MaxMetaspaceSize=%d", metaspaceSize)); + } + for (String arg : args) { + taskManagerProcessBuilder.addMainClassArg(arg); + } TestProcess p = taskManagerProcessBuilder.start(); p.getProcess().waitFor(); assertThat(p.getErrorOutput().toString().trim(), is("")); return p.getProcessOutput().toString().trim(); } + private static Collection getDummyClassLoadingProgramArgs(int numberOfLoadedClasses) { + return Arrays.asList( + Integer.toString(numberOfLoadedClasses), + TEMPORARY_FOLDER.getRoot().getAbsolutePath()); + } + + /** + * Dummy java program to generate Direct OOM. + */ + public static class DummyDirectAllocatingProgram { + private DummyDirectAllocatingProgram() { + } + + public static void main(String[] args) { + try { + Collection buffers = new ArrayList<>(); + for (int page = 0; page < 2 * DIRECT_MEMORY_PAGE_NUMBER; page++) { + buffers.add(ByteBuffer.allocateDirect(DIRECT_MEMORY_ALLOCATION_PAGE_SIZE)); + } + output("buffers: " + buffers); + } catch (Throwable t) { + if (!ExceptionUtils.isDirectOutOfMemoryError(t)) { + output("Wrong exception: " + t); + } + } + } + } + /** * Dummy java program to generate Metaspace OOM. */ @@ -113,9 +167,9 @@ public class ExceptionUtilsITCases extends TestLogger { } throw new RuntimeException("Metaspace usage is not found"); } + } - private static void output(String text) { - System.out.println(text); - } + private static void output(String text) { + System.out.println(text); } } -- Gitee From d6a0e4a6f390a22f8a54db4469d0ce64a8e5d045 Mon Sep 17 00:00:00 2001 From: Rui Li Date: Tue, 24 Mar 2020 20:38:39 +0800 Subject: [PATCH 265/885] [FLINK-16740][orc] Orc logicalTypeToOrcType fails to create decimal type with precision < 10 This closes #11492 --- .../main/java/org/apache/flink/orc/OrcSplitReaderUtil.java | 4 ++-- .../java/org/apache/flink/orc/OrcSplitReaderUtilTest.java | 1 + 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/OrcSplitReaderUtil.java b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/OrcSplitReaderUtil.java index aafc1bef1c..28e9496ca7 100644 --- a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/OrcSplitReaderUtil.java +++ b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/OrcSplitReaderUtil.java @@ -158,8 +158,8 @@ public class OrcSplitReaderUtil { case DECIMAL: DecimalType decimalType = (DecimalType) type; return TypeDescription.createDecimal() - .withPrecision(decimalType.getPrecision()) - .withScale(decimalType.getScale()); + .withScale(decimalType.getScale()) + .withPrecision(decimalType.getPrecision()); case TINYINT: return TypeDescription.createByte(); case SMALLINT: diff --git a/flink-formats/flink-orc/src/test/java/org/apache/flink/orc/OrcSplitReaderUtilTest.java b/flink-formats/flink-orc/src/test/java/org/apache/flink/orc/OrcSplitReaderUtilTest.java index 7b84f1e9cc..7f05a8ed99 100644 --- a/flink-formats/flink-orc/src/test/java/org/apache/flink/orc/OrcSplitReaderUtilTest.java +++ b/flink-formats/flink-orc/src/test/java/org/apache/flink/orc/OrcSplitReaderUtilTest.java @@ -59,6 +59,7 @@ public class OrcSplitReaderUtilTest { DataTypes.FIELD("int0", DataTypes.INT()), DataTypes.FIELD("int1", DataTypes.INT())) ))); + test("decimal(4,2)", DataTypes.DECIMAL(4, 2)); } private void test(String expected, DataType type) { -- Gitee From b7a83ff45ef0bf29c03616f542ff6d1abf22e2e2 Mon Sep 17 00:00:00 2001 From: Rui Li Date: Tue, 24 Mar 2020 20:40:03 +0800 Subject: [PATCH 266/885] [FLINK-16732][hive] Failed to call Hive UDF with constant return value This closes #11494 --- .../flink/table/functions/hive/HiveGenericUDF.java | 6 +++++- .../flink/table/module/hive/HiveModuleTest.java | 12 ++++++++++++ 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveGenericUDF.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveGenericUDF.java index 9adf5ffffa..1c3649aa35 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveGenericUDF.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveGenericUDF.java @@ -27,6 +27,7 @@ import org.apache.flink.table.types.DataType; import org.apache.hadoop.hive.ql.exec.UDFArgumentException; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.udf.generic.GenericUDF; +import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; import org.slf4j.Logger; @@ -83,7 +84,10 @@ public class HiveGenericUDF extends HiveScalarFunction { } try { - return HiveInspectors.toFlinkObject(returnInspector, function.evaluate(deferredObjects), hiveShim); + Object result = returnInspector instanceof ConstantObjectInspector ? + ((ConstantObjectInspector) returnInspector).getWritableConstantValue() : + function.evaluate(deferredObjects); + return HiveInspectors.toFlinkObject(returnInspector, result, hiveShim); } catch (HiveException e) { throw new FlinkHiveUDFException(e); } diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/module/hive/HiveModuleTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/module/hive/HiveModuleTest.java index 50a5494dbb..28231ecbc8 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/module/hive/HiveModuleTest.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/module/hive/HiveModuleTest.java @@ -149,4 +149,16 @@ public class HiveModuleTest { assertFalse(hiveModule.getFunctionDefinition(banned).isPresent()); } } + + @Test + public void testConstantReturnValue() throws Exception { + TableEnvironment tableEnv = HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(); + + tableEnv.unloadModule("core"); + tableEnv.loadModule("hive", new HiveModule()); + + List results = TableUtils.collectToList(tableEnv.sqlQuery("select str_to_map('a:1,b:2,c:3',',',':')")); + + assertEquals("[{a=1, b=2, c=3}]", results.toString()); + } } -- Gitee From c4aa57022c54703cd89f250032ebd3ea166152cc Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Mon, 2 Mar 2020 16:55:16 +0800 Subject: [PATCH 267/885] [hotfix][runtime] Minor code clean-ups. --- .../org/apache/flink/runtime/minicluster/MiniCluster.java | 8 ++++---- .../runtime/minicluster/MiniClusterConfiguration.java | 5 ----- .../flink/runtime/rpc/akka/AkkaRpcServiceUtils.java | 4 ++-- 3 files changed, 6 insertions(+), 11 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java index 4551428912..d760f572cb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java @@ -904,16 +904,16 @@ public class MiniCluster implements JobExecutorService, AutoCloseableAsync { protected class DedicatedRpcServiceFactory implements RpcServiceFactory { private final AkkaRpcServiceConfiguration akkaRpcServiceConfig; - private final String jobManagerBindAddress; + private final String bindAddress; - DedicatedRpcServiceFactory(AkkaRpcServiceConfiguration akkaRpcServiceConfig, String jobManagerBindAddress) { + DedicatedRpcServiceFactory(AkkaRpcServiceConfiguration akkaRpcServiceConfig, String bindAddress) { this.akkaRpcServiceConfig = akkaRpcServiceConfig; - this.jobManagerBindAddress = jobManagerBindAddress; + this.bindAddress = bindAddress; } @Override public RpcService createRpcService() { - final RpcService rpcService = MiniCluster.this.createRpcService(akkaRpcServiceConfig, true, jobManagerBindAddress); + final RpcService rpcService = MiniCluster.this.createRpcService(akkaRpcServiceConfig, true, bindAddress); synchronized (lock) { rpcServices.add(rpcService); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java index 2bfeeed20e..b1341c5f31 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java @@ -29,9 +29,6 @@ import org.apache.flink.runtime.taskexecutor.TaskExecutorResourceUtils; import org.apache.flink.util.Preconditions; import org.apache.flink.util.StringUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import javax.annotation.Nullable; import static org.apache.flink.runtime.minicluster.RpcServiceSharing.SHARED; @@ -41,8 +38,6 @@ import static org.apache.flink.runtime.minicluster.RpcServiceSharing.SHARED; */ public class MiniClusterConfiguration { - private static final Logger LOG = LoggerFactory.getLogger(MiniClusterConfiguration.class); - static final String SCHEDULER_TYPE_KEY = JobManagerOptions.SCHEDULER.key(); private final UnmodifiableConfiguration configuration; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceUtils.java index 0c4b877dd9..7d9d37f58c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceUtils.java @@ -18,8 +18,6 @@ package org.apache.flink.runtime.rpc.akka; -import com.typesafe.config.Config; -import com.typesafe.config.ConfigFactory; import org.apache.flink.configuration.AkkaOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.clusterframework.BootstrapTools; @@ -31,6 +29,8 @@ import org.apache.flink.util.NetUtils; import org.apache.flink.util.Preconditions; import akka.actor.ActorSystem; +import com.typesafe.config.Config; +import com.typesafe.config.ConfigFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -- Gitee From 61086f4565d57ee02113ddbe1495cfab8e83cdb2 Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Tue, 3 Mar 2020 14:31:14 +0800 Subject: [PATCH 268/885] [hotfix][e2e] Deduplicate codes for building docker image with custom jar. --- flink-end-to-end-tests/test-scripts/common_docker.sh | 6 ++++++ flink-end-to-end-tests/test-scripts/common_kubernetes.sh | 1 + .../test-scripts/test_docker_embedded_job.sh | 3 ++- .../test-scripts/test_kubernetes_embedded_job.sh | 2 +- .../test-scripts/test_kubernetes_session.sh | 2 +- 5 files changed, 11 insertions(+), 3 deletions(-) diff --git a/flink-end-to-end-tests/test-scripts/common_docker.sh b/flink-end-to-end-tests/test-scripts/common_docker.sh index cf3a798028..502610c12c 100644 --- a/flink-end-to-end-tests/test-scripts/common_docker.sh +++ b/flink-end-to-end-tests/test-scripts/common_docker.sh @@ -32,3 +32,9 @@ function containers_health_check() { fi done } + +function build_image_with_jar() { + local job_artifacts=$1 + local image_name=${2:-flink-job} + ./build.sh --from-local-dist --job-artifacts ${job_artifacts} --image-name ${image_name} +} diff --git a/flink-end-to-end-tests/test-scripts/common_kubernetes.sh b/flink-end-to-end-tests/test-scripts/common_kubernetes.sh index de8940bda9..261c10cd2d 100755 --- a/flink-end-to-end-tests/test-scripts/common_kubernetes.sh +++ b/flink-end-to-end-tests/test-scripts/common_kubernetes.sh @@ -18,6 +18,7 @@ ################################################################################ source "$(dirname "$0")"/common.sh +source "$(dirname "$0")"/common_docker.sh DOCKER_MODULE_DIR=${END_TO_END_DIR}/../flink-container/docker KUBERNETES_MODULE_DIR=${END_TO_END_DIR}/../flink-container/kubernetes diff --git a/flink-end-to-end-tests/test-scripts/test_docker_embedded_job.sh b/flink-end-to-end-tests/test-scripts/test_docker_embedded_job.sh index 277781c253..664e8d85b5 100755 --- a/flink-end-to-end-tests/test-scripts/test_docker_embedded_job.sh +++ b/flink-end-to-end-tests/test-scripts/test_docker_embedded_job.sh @@ -18,6 +18,7 @@ ################################################################################ source "$(dirname "$0")"/common.sh +source "$(dirname "$0")"/common_docker.sh DOCKER_MODULE_DIR=${END_TO_END_DIR}/../flink-container/docker DOCKER_SCRIPTS=${END_TO_END_DIR}/test-scripts/container-scripts @@ -52,7 +53,7 @@ esac export FLINK_JOB_ARGUMENTS="${INPUT_ARGS} --output ${OUTPUT_PATH}/docker_wc_out" build_image() { - ./build.sh --from-local-dist --job-artifacts ${FLINK_DIR}/examples/batch/WordCount.jar --image-name ${FLINK_DOCKER_IMAGE_NAME} + build_image_with_jar ${FLINK_DIR}/examples/batch/WordCount.jar ${FLINK_DOCKER_IMAGE_NAME} } # user inside the container must be able to create files, this is a workaround in-container permissions diff --git a/flink-end-to-end-tests/test-scripts/test_kubernetes_embedded_job.sh b/flink-end-to-end-tests/test-scripts/test_kubernetes_embedded_job.sh index 52464652ee..eb051840c6 100755 --- a/flink-end-to-end-tests/test-scripts/test_kubernetes_embedded_job.sh +++ b/flink-end-to-end-tests/test-scripts/test_kubernetes_embedded_job.sh @@ -38,7 +38,7 @@ start_kubernetes mkdir -p $OUTPUT_VOLUME cd "$DOCKER_MODULE_DIR" -./build.sh --from-local-dist --job-artifacts ${FLINK_DIR}/examples/batch/WordCount.jar --image-name ${FLINK_IMAGE_NAME} +build_image_with_jar ${FLINK_DIR}/examples/batch/WordCount.jar ${FLINK_IMAGE_NAME} cd "$END_TO_END_DIR" diff --git a/flink-end-to-end-tests/test-scripts/test_kubernetes_session.sh b/flink-end-to-end-tests/test-scripts/test_kubernetes_session.sh index 007cac3a0f..3dc3bae84d 100755 --- a/flink-end-to-end-tests/test-scripts/test_kubernetes_session.sh +++ b/flink-end-to-end-tests/test-scripts/test_kubernetes_session.sh @@ -36,7 +36,7 @@ start_kubernetes cd "$DOCKER_MODULE_DIR" # Build a Flink image without any user jars -./build.sh --from-local-dist --job-artifacts ${TEST_INFRA_DIR}/test-data/words --image-name ${FLINK_IMAGE_NAME} +build_image_with_jar ${TEST_INFRA_DIR}/test-data/words ${FLINK_IMAGE_NAME} kubectl create clusterrolebinding ${CLUSTER_ROLE_BINDING} --clusterrole=edit --serviceaccount=default:default --namespace=default -- Gitee From 03b6e763d5578c3b503246a2a40e5fd1fb990570 Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Mon, 2 Mar 2020 17:17:13 +0800 Subject: [PATCH 269/885] [FLINK-15911][runtime] Refactor to create AkkaRpsService with builder class, in order to reduce the number of nested creating methods. --- .../clusterframework/BootstrapTools.java | 228 +++++++++--------- .../runtime/entrypoint/ClusterEntrypoint.java | 2 +- .../runtime/metrics/util/MetricUtils.java | 10 +- .../runtime/minicluster/MiniCluster.java | 84 +++---- .../minicluster/MiniClusterConfiguration.java | 8 + .../runtime/rpc/akka/AkkaRpcService.java | 2 + .../runtime/rpc/akka/AkkaRpcServiceUtils.java | 186 ++++++++------ .../taskexecutor/TaskManagerRunner.java | 2 +- .../apache/flink/runtime/akka/AkkaUtils.scala | 31 ++- .../clusterframework/BootstrapToolsTest.java | 4 +- ...aRpcActorOversizedResponseMessageTest.java | 4 +- .../rpc/akka/MessageSerializationTest.java | 4 +- ...ManagerHAProcessFailureRecoveryITCase.java | 2 +- .../ProcessFailureCancelingITCase.java | 2 +- 14 files changed, 321 insertions(+), 248 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java index 88f2676b4b..494d1509b5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.clusterframework; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.configuration.AkkaOptions; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.ConfigOption; @@ -41,7 +42,6 @@ import org.apache.commons.cli.Option; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.io.File; @@ -52,6 +52,7 @@ import java.net.BindException; import java.util.HashMap; import java.util.Iterator; import java.util.Map; +import java.util.Optional; import java.util.stream.Stream; import scala.Some; @@ -82,92 +83,80 @@ public class BootstrapTools { .build(); /** - * Starts an ActorSystem with the given configuration listening at the address/ports. + * Starts a remote ActorSystem at given address and specific port range. * @param configuration The Flink configuration - * @param listeningAddress The address to listen at. - * @param portRangeDefinition The port range to choose a port from. + * @param externalAddress The external address to access the ActorSystem. + * @param externalPortRange The choosing range of the external port to access the ActorSystem. * @param logger The logger to output log information. * @return The ActorSystem which has been started * @throws Exception Thrown when actor system cannot be started in specified port range */ - public static ActorSystem startActorSystem( + @VisibleForTesting + public static ActorSystem startRemoteActorSystem( Configuration configuration, - String listeningAddress, - String portRangeDefinition, + String externalAddress, + String externalPortRange, Logger logger) throws Exception { - return startActorSystem( - configuration, - listeningAddress, - portRangeDefinition, - logger, - ForkJoinExecutorConfiguration.fromConfiguration(configuration)); - } - - /** - * Starts an ActorSystem with the given configuration listening at the address/ports. - * - * @param configuration The Flink configuration - * @param listeningAddress The address to listen at. - * @param portRangeDefinition The port range to choose a port from. - * @param logger The logger to output log information. - * @param actorSystemExecutorConfiguration configuration for the ActorSystem's underlying executor - * @return The ActorSystem which has been started - * @throws Exception Thrown when actor system cannot be started in specified port range - */ - public static ActorSystem startActorSystem( - Configuration configuration, - String listeningAddress, - String portRangeDefinition, - Logger logger, - @Nonnull ActorSystemExecutorConfiguration actorSystemExecutorConfiguration) throws Exception { - return startActorSystem( + return startRemoteActorSystem( configuration, AkkaUtils.getFlinkActorSystemName(), - listeningAddress, - portRangeDefinition, + externalAddress, + externalPortRange, + NetUtils.getWildcardIPAddress(), + Optional.empty(), logger, - actorSystemExecutorConfiguration); + ForkJoinExecutorConfiguration.fromConfiguration(configuration), + null); } /** - * Starts an ActorSystem with the given configuration listening at the address/ports. + * Starts a remote ActorSystem at given address and specific port range. * * @param configuration The Flink configuration * @param actorSystemName Name of the started {@link ActorSystem} - * @param listeningAddress The address to listen at. - * @param portRangeDefinition The port range to choose a port from. + * @param externalAddress The external address to access the ActorSystem. + * @param externalPortRange The choosing range of the external port to access the ActorSystem. + * @param bindAddress The local address to bind to. + * @param bindPort The local port to bind to. If not present, then the external port will be used. * @param logger The logger to output log information. * @param actorSystemExecutorConfiguration configuration for the ActorSystem's underlying executor + * @param customConfig Custom Akka config to be combined with the config derived from Flink configuration. * @return The ActorSystem which has been started * @throws Exception Thrown when actor system cannot be started in specified port range */ - public static ActorSystem startActorSystem( + public static ActorSystem startRemoteActorSystem( Configuration configuration, String actorSystemName, - String listeningAddress, - String portRangeDefinition, + String externalAddress, + String externalPortRange, + String bindAddress, + @SuppressWarnings("OptionalUsedAsFieldOrParameterType") Optional bindPort, Logger logger, - @Nonnull ActorSystemExecutorConfiguration actorSystemExecutorConfiguration) throws Exception { + ActorSystemExecutorConfiguration actorSystemExecutorConfiguration, + Config customConfig) throws Exception { // parse port range definition and create port iterator Iterator portsIterator; try { - portsIterator = NetUtils.getPortRangeFromString(portRangeDefinition); + portsIterator = NetUtils.getPortRangeFromString(externalPortRange); } catch (Exception e) { - throw new IllegalArgumentException("Invalid port range definition: " + portRangeDefinition); + throw new IllegalArgumentException("Invalid port range definition: " + externalPortRange); } while (portsIterator.hasNext()) { - final int port = portsIterator.next(); + final int externalPort = portsIterator.next(); try { - return startActorSystem( + return startRemoteActorSystem( configuration, actorSystemName, - listeningAddress, - port, + externalAddress, + externalPort, + bindAddress, + bindPort.orElse(externalPort), logger, - actorSystemExecutorConfiguration); + actorSystemExecutorConfiguration, + customConfig); } catch (Exception e) { // we can continue to try if this contains a netty channel exception @@ -181,104 +170,115 @@ public class BootstrapTools { // if we come here, we have exhausted the port range throw new BindException("Could not start actor system on any port in port range " - + portRangeDefinition); + + externalPortRange); } /** - * Starts an Actor System at a specific port. - * + * Starts a remote Actor System at given address and specific port. * @param configuration The Flink configuration. - * @param listeningAddress The address to listen at. - * @param listeningPort The port to listen at. + * @param actorSystemName Name of the started {@link ActorSystem} + * @param externalAddress The external address to access the ActorSystem. + * @param externalPort The external port to access the ActorSystem. + * @param bindAddress The local address to bind to. + * @param bindPort The local port to bind to. * @param logger the logger to output log information. + * @param actorSystemExecutorConfiguration configuration for the ActorSystem's underlying executor + * @param customConfig Custom Akka config to be combined with the config derived from Flink configuration. * @return The ActorSystem which has been started. * @throws Exception */ - public static ActorSystem startActorSystem( + private static ActorSystem startRemoteActorSystem( Configuration configuration, - String listeningAddress, - int listeningPort, - Logger logger) throws Exception { - return startActorSystem( - configuration, - listeningAddress, - listeningPort, - logger, - ForkJoinExecutorConfiguration.fromConfiguration(configuration)); - } + String actorSystemName, + String externalAddress, + int externalPort, + String bindAddress, + int bindPort, + Logger logger, + ActorSystemExecutorConfiguration actorSystemExecutorConfiguration, + Config customConfig) throws Exception { - /** - * Starts an Actor System at a specific port. - * @param configuration The Flink configuration. - * @param listeningAddress The address to listen at. - * @param listeningPort The port to listen at. - * @param logger the logger to output log information. - * @param actorSystemExecutorConfiguration configuration for the ActorSystem's underlying executor - * @return The ActorSystem which has been started. - * @throws Exception - */ - public static ActorSystem startActorSystem( - Configuration configuration, - String listeningAddress, - int listeningPort, - Logger logger, - ActorSystemExecutorConfiguration actorSystemExecutorConfiguration) throws Exception { - return startActorSystem( - configuration, - AkkaUtils.getFlinkActorSystemName(), - listeningAddress, - listeningPort, - logger, - actorSystemExecutorConfiguration); + String externalHostPortUrl = NetUtils.unresolvedHostAndPortToNormalizedString(externalAddress, externalPort); + String bindHostPortUrl = NetUtils.unresolvedHostAndPortToNormalizedString(bindAddress, bindPort); + logger.info("Trying to start actor system, external address {}, bind address {}.", externalHostPortUrl, bindHostPortUrl); + + try { + Config akkaConfig = AkkaUtils.getAkkaConfig( + configuration, + new Some<>(new Tuple2<>(externalAddress, externalPort)), + new Some<>(new Tuple2<>(bindAddress, bindPort)), + actorSystemExecutorConfiguration.getAkkaConfig()); + + if (customConfig != null) { + akkaConfig = customConfig.withFallback(akkaConfig); + } + + return startActorSystem(akkaConfig, actorSystemName, logger); + } + catch (Throwable t) { + if (t instanceof ChannelException) { + Throwable cause = t.getCause(); + if (cause != null && t.getCause() instanceof BindException) { + throw new IOException("Unable to create ActorSystem at address " + bindHostPortUrl + + " : " + cause.getMessage(), t); + } + } + throw new Exception("Could not create actor system", t); + } } /** - * Starts an Actor System at a specific port. + * Starts a local Actor System. * @param configuration The Flink configuration. - * @param actorSystemName Name of the started {@link ActorSystem} - * @param listeningAddress The address to listen at. - * @param listeningPort The port to listen at. - * @param logger the logger to output log information. - * @param actorSystemExecutorConfiguration configuration for the ActorSystem's underlying executor + * @param actorSystemName Name of the started ActorSystem. + * @param logger The logger to output log information. + * @param actorSystemExecutorConfiguration Configuration for the ActorSystem's underlying executor. + * @param customConfig Custom Akka config to be combined with the config derived from Flink configuration. * @return The ActorSystem which has been started. * @throws Exception */ - public static ActorSystem startActorSystem( + public static ActorSystem startLocalActorSystem( Configuration configuration, String actorSystemName, - String listeningAddress, - int listeningPort, Logger logger, - ActorSystemExecutorConfiguration actorSystemExecutorConfiguration) throws Exception { + ActorSystemExecutorConfiguration actorSystemExecutorConfiguration, + Config customConfig) throws Exception { - String hostPortUrl = NetUtils.unresolvedHostAndPortToNormalizedString(listeningAddress, listeningPort); - logger.info("Trying to start actor system at {}", hostPortUrl); + logger.info("Trying to start local actor system"); try { Config akkaConfig = AkkaUtils.getAkkaConfig( configuration, - new Some<>(new Tuple2<>(listeningAddress, listeningPort)), + scala.Option.empty(), + scala.Option.empty(), actorSystemExecutorConfiguration.getAkkaConfig()); - logger.debug("Using akka configuration\n {}", akkaConfig); - - ActorSystem actorSystem = AkkaUtils.createActorSystem(actorSystemName, akkaConfig); + if (customConfig != null) { + akkaConfig = customConfig.withFallback(akkaConfig); + } - logger.info("Actor system started at {}", AkkaUtils.getAddress(actorSystem)); - return actorSystem; + return startActorSystem(akkaConfig, actorSystemName, logger); } catch (Throwable t) { - if (t instanceof ChannelException) { - Throwable cause = t.getCause(); - if (cause != null && t.getCause() instanceof BindException) { - throw new IOException("Unable to create ActorSystem at address " + hostPortUrl + - " : " + cause.getMessage(), t); - } - } throw new Exception("Could not create actor system", t); } } + /** + * Starts an Actor System with given Akka config. + * @param akkaConfig Config of the started ActorSystem. + * @param actorSystemName Name of the started ActorSystem. + * @param logger The logger to output log information. + * @return The ActorSystem which has been started. + */ + private static ActorSystem startActorSystem(Config akkaConfig, String actorSystemName, Logger logger) { + logger.debug("Using akka configuration\n {}", akkaConfig); + ActorSystem actorSystem = AkkaUtils.createActorSystem(actorSystemName, akkaConfig); + + logger.info("Actor system started at {}", AkkaUtils.getAddress(actorSystem)); + return actorSystem; + } + /** * Writes a Flink YAML config file from a Flink Configuration object. * @param cfg The Flink config diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java index f66eda823c..faa5a98fb9 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java @@ -282,7 +282,7 @@ public abstract class ClusterEntrypoint implements AutoCloseableAsync, FatalErro @Nonnull private RpcService createRpcService(Configuration configuration, String bindAddress, String portRange) throws Exception { - return AkkaRpcServiceUtils.createRpcService(bindAddress, portRange, configuration); + return AkkaRpcServiceUtils.remoteServiceBuilder(configuration, bindAddress, portRange).createAndStart(); } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/util/MetricUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/util/MetricUtils.java index cda453d469..73e33ae8b5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/util/MetricUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/util/MetricUtils.java @@ -136,12 +136,10 @@ public class MetricUtils { final String portRange = configuration.getString(MetricOptions.QUERY_SERVICE_PORT); final int threadPriority = configuration.getInteger(MetricOptions.QUERY_SERVICE_THREAD_PRIORITY); - return AkkaRpcServiceUtils.createRpcService( - hostname, - portRange, - configuration, - METRICS_ACTOR_SYSTEM_NAME, - new BootstrapTools.FixedThreadPoolExecutorConfiguration(1, 1, threadPriority)); + return AkkaRpcServiceUtils.remoteServiceBuilder(configuration, hostname, portRange) + .withActorSystemName(METRICS_ACTOR_SYSTEM_NAME) + .withActorSystemExecutorConfiguration(new BootstrapTools.FixedThreadPoolExecutorConfiguration(1, 1, threadPriority)) + .createAndStart(); } private static void instantiateClassLoaderMetrics(MetricGroup metrics) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java index d760f572cb..3d00163873 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java @@ -66,8 +66,7 @@ import org.apache.flink.runtime.resourcemanager.StandaloneResourceManagerFactory import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.RpcUtils; -import org.apache.flink.runtime.rpc.akka.AkkaRpcService; -import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceConfiguration; +import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils; import org.apache.flink.runtime.taskexecutor.TaskExecutor; import org.apache.flink.runtime.taskexecutor.TaskManagerRunner; import org.apache.flink.runtime.util.ExecutorThreadFactory; @@ -82,8 +81,6 @@ import org.apache.flink.util.ExecutorUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.function.FunctionUtils; -import akka.actor.ActorSystem; -import com.typesafe.config.Config; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -261,26 +258,30 @@ public class MiniCluster implements JobExecutorService, AutoCloseableAsync { // bring up all the RPC services LOG.info("Starting RPC Service(s)"); - AkkaRpcServiceConfiguration akkaRpcServiceConfig = AkkaRpcServiceConfiguration.fromConfiguration(configuration); - final RpcServiceFactory dispatcherResourceManagreComponentRpcServiceFactory; if (useSingleRpcService) { // we always need the 'commonRpcService' for auxiliary calls - commonRpcService = createRpcService(akkaRpcServiceConfig, false, null); + commonRpcService = createLocalRpcService(configuration); final CommonRpcServiceFactory commonRpcServiceFactory = new CommonRpcServiceFactory(commonRpcService); taskManagerRpcServiceFactory = commonRpcServiceFactory; dispatcherResourceManagreComponentRpcServiceFactory = commonRpcServiceFactory; } else { - // we always need the 'commonRpcService' for auxiliary calls - commonRpcService = createRpcService(akkaRpcServiceConfig, true, null); // start a new service per component, possibly with custom bind addresses final String jobManagerBindAddress = miniClusterConfiguration.getJobManagerBindAddress(); final String taskManagerBindAddress = miniClusterConfiguration.getTaskManagerBindAddress(); + final String jobManagerBindPort = miniClusterConfiguration.getJobManagerBindPortRange(); + final String taskManagerBindPort = miniClusterConfiguration.getTaskManagerBindPortRange(); + + dispatcherResourceManagreComponentRpcServiceFactory = + new DedicatedRpcServiceFactory(configuration, jobManagerBindAddress, jobManagerBindPort); + taskManagerRpcServiceFactory = + new DedicatedRpcServiceFactory(configuration, taskManagerBindAddress, taskManagerBindPort); - dispatcherResourceManagreComponentRpcServiceFactory = new DedicatedRpcServiceFactory(akkaRpcServiceConfig, jobManagerBindAddress); - taskManagerRpcServiceFactory = new DedicatedRpcServiceFactory(akkaRpcServiceConfig, taskManagerBindAddress); + // we always need the 'commonRpcService' for auxiliary calls + // bind to the JobManager address with port 0 + commonRpcService = createRemoteRpcService(configuration, jobManagerBindAddress, "0"); } RpcService metricQueryServiceRpcService = MetricUtils.startMetricsRpcService( @@ -714,35 +715,32 @@ public class MiniCluster implements JobExecutorService, AutoCloseableAsync { } /** - * Factory method to instantiate the RPC service. - * - * @param akkaRpcServiceConfig - * The default RPC timeout for asynchronous "ask" requests. - * @param remoteEnabled - * True, if the RPC service should be reachable from other (remote) RPC services. - * @param bindAddress - * The address to bind the RPC service to. Only relevant when "remoteEnabled" is true. + * Factory method to instantiate the remote RPC service. * + * @param configuration Flink configuration. + * @param bindAddress The address to bind the RPC service to. + * @param bindPortRange The port range to bind the RPC service to. * @return The instantiated RPC service */ - protected RpcService createRpcService( - AkkaRpcServiceConfiguration akkaRpcServiceConfig, - boolean remoteEnabled, - String bindAddress) { - - final Config akkaConfig; - - if (remoteEnabled) { - akkaConfig = AkkaUtils.getAkkaConfig(akkaRpcServiceConfig.getConfiguration(), bindAddress, 0); - } else { - akkaConfig = AkkaUtils.getAkkaConfig(akkaRpcServiceConfig.getConfiguration()); - } - - final Config effectiveAkkaConfig = AkkaUtils.testDispatcherConfig().withFallback(akkaConfig); - - final ActorSystem actorSystem = AkkaUtils.createActorSystem(effectiveAkkaConfig); + protected RpcService createRemoteRpcService( + Configuration configuration, + String bindAddress, + String bindPortRange) throws Exception { + return AkkaRpcServiceUtils.remoteServiceBuilder(configuration, bindAddress, bindPortRange) + .withCustomConfig(AkkaUtils.testDispatcherConfig()) + .createAndStart(); + } - return new AkkaRpcService(actorSystem, akkaRpcServiceConfig); + /** + * Factory method to instantiate the local RPC service. + * + * @param configuration Flink configuration. + * @return The instantiated RPC service + */ + protected RpcService createLocalRpcService(Configuration configuration) throws Exception { + return AkkaRpcServiceUtils.localServiceBuilder(configuration) + .withCustomConfig(AkkaUtils.testDispatcherConfig()) + .createAndStart(); } // ------------------------------------------------------------------------ @@ -878,7 +876,7 @@ public class MiniCluster implements JobExecutorService, AutoCloseableAsync { * Internal factory for {@link RpcService}. */ protected interface RpcServiceFactory { - RpcService createRpcService(); + RpcService createRpcService() throws Exception; } /** @@ -903,17 +901,19 @@ public class MiniCluster implements JobExecutorService, AutoCloseableAsync { */ protected class DedicatedRpcServiceFactory implements RpcServiceFactory { - private final AkkaRpcServiceConfiguration akkaRpcServiceConfig; + private final Configuration configuration; private final String bindAddress; + private final String bindPortRange; - DedicatedRpcServiceFactory(AkkaRpcServiceConfiguration akkaRpcServiceConfig, String bindAddress) { - this.akkaRpcServiceConfig = akkaRpcServiceConfig; + DedicatedRpcServiceFactory(Configuration configuration, String bindAddress, String bindPortRange) { + this.configuration = configuration; this.bindAddress = bindAddress; + this.bindPortRange = bindPortRange; } @Override - public RpcService createRpcService() { - final RpcService rpcService = MiniCluster.this.createRpcService(akkaRpcServiceConfig, true, bindAddress); + public RpcService createRpcService() throws Exception { + final RpcService rpcService = MiniCluster.this.createRemoteRpcService(configuration, bindAddress, bindPortRange); synchronized (lock) { rpcServices.add(rpcService); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java index b1341c5f31..f4d6d49e16 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java @@ -106,6 +106,14 @@ public class MiniClusterConfiguration { configuration.getString(TaskManagerOptions.HOST, "localhost"); } + public String getJobManagerBindPortRange() { + return String.valueOf(configuration.getInteger(JobManagerOptions.PORT, 0)); + } + + public String getTaskManagerBindPortRange() { + return configuration.getString(TaskManagerOptions.RPC_PORT); + } + public Time getRpcTimeout() { return AkkaUtils.getTimeoutAsTime(configuration); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java index 0c41f053e6..f629ffd31d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.rpc.akka; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.concurrent.FutureUtils; @@ -106,6 +107,7 @@ public class AkkaRpcService implements RpcService { private volatile boolean stopped; + @VisibleForTesting public AkkaRpcService(final ActorSystem actorSystem, final AkkaRpcServiceConfiguration configuration) { this.actorSystem = checkNotNull(actorSystem, "actor system"); this.configuration = checkNotNull(configuration, "akka rpc service configuration"); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceUtils.java index 7d9d37f58c..181c0af74a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceUtils.java @@ -18,13 +18,14 @@ package org.apache.flink.runtime.rpc.akka; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.configuration.AkkaOptions; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.clusterframework.BootstrapTools; import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils.AddressResolution; import org.apache.flink.runtime.net.SSLUtils; -import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.util.NetUtils; import org.apache.flink.util.Preconditions; @@ -34,11 +35,11 @@ import com.typesafe.config.ConfigFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.annotation.Nonnull; +import javax.annotation.Nullable; -import java.io.IOException; import java.net.InetAddress; import java.net.UnknownHostException; +import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; import static org.apache.flink.util.NetUtils.isValidClientPort; @@ -68,75 +69,17 @@ public class AkkaRpcServiceUtils { // RPC instantiation // ------------------------------------------------------------------------ - /** - * Utility method to create RPC service from configuration and hostname, port. - * - * @param hostname The hostname/address that describes the TaskManager's data location. - * @param portRangeDefinition The port range to start TaskManager on. - * @param configuration The configuration for the TaskManager. - * @return The rpc service which is used to start and connect to the TaskManager RpcEndpoint . - * @throws IOException Thrown, if the actor system can not bind to the address - * @throws Exception Thrown is some other error occurs while creating akka actor system - */ - public static RpcService createRpcService( - String hostname, - String portRangeDefinition, - Configuration configuration) throws Exception { - final ActorSystem actorSystem = BootstrapTools.startActorSystem(configuration, hostname, portRangeDefinition, LOG); - return instantiateAkkaRpcService(configuration, actorSystem); - } - - /** - * Utility method to create RPC service from configuration and hostname, port. - * - * @param hostname The hostname/address that describes the TaskManager's data location. - * @param port If true, the TaskManager will not initiate the TCP network stack. - * @param configuration The configuration for the TaskManager. - * @return The rpc service which is used to start and connect to the TaskManager RpcEndpoint . - * @throws IOException Thrown, if the actor system can not bind to the address - * @throws Exception Thrown is some other error occurs while creating akka actor system - */ - public static RpcService createRpcService( - String hostname, - int port, - Configuration configuration) throws Exception { - final ActorSystem actorSystem = BootstrapTools.startActorSystem(configuration, hostname, port, LOG); - return instantiateAkkaRpcService(configuration, actorSystem); + public static AkkaRpcServiceBuilder remoteServiceBuilder(Configuration configuration, @Nullable String externalAddress, String externalPortRange) { + return new AkkaRpcServiceBuilder(configuration, LOG, externalAddress, externalPortRange); } - /** - * Utility method to create RPC service from configuration and hostname, port. - * - * @param hostname The hostname/address that describes the TaskManager's data location. - * @param portRangeDefinition The port range to start TaskManager on. - * @param configuration The configuration for the TaskManager. - * @param actorSystemName The actor system name of the RpcService. - * @param actorSystemExecutorConfiguration The configuration of the executor of the actor system. - * @return The rpc service which is used to start and connect to the TaskManager RpcEndpoint . - * @throws IOException Thrown, if the actor system can not bind to the address - * @throws Exception Thrown is some other error occurs while creating akka actor system - */ - public static RpcService createRpcService( - String hostname, - String portRangeDefinition, - Configuration configuration, - String actorSystemName, - @Nonnull BootstrapTools.ActorSystemExecutorConfiguration actorSystemExecutorConfiguration) throws Exception { - - final ActorSystem actorSystem = BootstrapTools.startActorSystem( - configuration, - actorSystemName, - hostname, - portRangeDefinition, - LOG, - actorSystemExecutorConfiguration); - - return instantiateAkkaRpcService(configuration, actorSystem); + @VisibleForTesting + public static AkkaRpcServiceBuilder remoteServiceBuilder(Configuration configuration, @Nullable String externalAddress, int externalPort) { + return remoteServiceBuilder(configuration, externalAddress, String.valueOf(externalPort)); } - @Nonnull - private static RpcService instantiateAkkaRpcService(Configuration configuration, ActorSystem actorSystem) { - return new AkkaRpcService(actorSystem, AkkaRpcServiceConfiguration.fromConfiguration(configuration)); + public static AkkaRpcServiceBuilder localServiceBuilder(Configuration configuration) { + return new AkkaRpcServiceBuilder(configuration, LOG); } // ------------------------------------------------------------------------ @@ -246,6 +189,113 @@ public class AkkaRpcServiceUtils { return akkaConfig.getBytes(MAXIMUM_FRAME_SIZE_PATH); } + // ------------------------------------------------------------------------ + // RPC service builder + // ------------------------------------------------------------------------ + + /** + * Builder for {@link AkkaRpcService}. + */ + public static class AkkaRpcServiceBuilder { + + private final Configuration configuration; + private final Logger logger; + @Nullable private final String externalAddress; + @Nullable private final String externalPortRange; + + private String actorSystemName = AkkaUtils.getFlinkActorSystemName(); + @Nullable private BootstrapTools.ActorSystemExecutorConfiguration actorSystemExecutorConfiguration = null; + @Nullable private Config customConfig = null; + private String bindAddress = NetUtils.getWildcardIPAddress(); + @Nullable private Integer bindPort = null; + + /** + * Builder for creating a remote RPC service. + */ + private AkkaRpcServiceBuilder( + final Configuration configuration, + final Logger logger, + @Nullable final String externalAddress, + final String externalPortRange) { + this.configuration = Preconditions.checkNotNull(configuration); + this.logger = Preconditions.checkNotNull(logger); + this.externalAddress = externalAddress == null ? InetAddress.getLoopbackAddress().getHostAddress() : externalAddress; + this.externalPortRange = Preconditions.checkNotNull(externalPortRange); + } + + /** + * Builder for creating a local RPC service. + */ + private AkkaRpcServiceBuilder( + final Configuration configuration, + final Logger logger) { + this.configuration = Preconditions.checkNotNull(configuration); + this.logger = Preconditions.checkNotNull(logger); + this.externalAddress = null; + this.externalPortRange = null; + } + + public AkkaRpcServiceBuilder withActorSystemName(final String actorSystemName) { + this.actorSystemName = Preconditions.checkNotNull(actorSystemName); + return this; + } + + public AkkaRpcServiceBuilder withActorSystemExecutorConfiguration( + final BootstrapTools.ActorSystemExecutorConfiguration actorSystemExecutorConfiguration) { + this.actorSystemExecutorConfiguration = actorSystemExecutorConfiguration; + return this; + } + + public AkkaRpcServiceBuilder withCustomConfig(final Config customConfig) { + this.customConfig = customConfig; + return this; + } + + public AkkaRpcServiceBuilder withBindAddress(final String bindAddress) { + this.bindAddress = Preconditions.checkNotNull(bindAddress); + return this; + } + + public AkkaRpcServiceBuilder withBindPort(int bindPort) { + Preconditions.checkArgument(NetUtils.isValidHostPort(bindPort), "Invalid port number: " + bindPort); + this.bindPort = bindPort; + return this; + } + + public AkkaRpcService createAndStart() throws Exception { + if (actorSystemExecutorConfiguration == null) { + actorSystemExecutorConfiguration = BootstrapTools.ForkJoinExecutorConfiguration.fromConfiguration(configuration); + } + + final ActorSystem actorSystem; + + if (externalAddress == null) { + // create local actor system + actorSystem = BootstrapTools.startLocalActorSystem( + configuration, + actorSystemName, + logger, + actorSystemExecutorConfiguration, + customConfig); + } else { + // create remote actor system + actorSystem = BootstrapTools.startRemoteActorSystem( + configuration, + actorSystemName, + externalAddress, + externalPortRange, + bindAddress, + Optional.ofNullable(bindPort), + logger, + actorSystemExecutorConfiguration, + customConfig); + } + + return new AkkaRpcService(actorSystem, AkkaRpcServiceConfiguration.fromConfiguration(configuration)); + } + + } + // ------------------------------------------------------------------------ /** This class is not meant to be instantiated. */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java index 8ed4fe478e..a62db016d5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java @@ -422,7 +422,7 @@ public class TaskManagerRunner implements FatalErrorHandler, AutoCloseableAsync final String taskManagerAddress = determineTaskManagerBindAddress(configuration, haServices); final String portRangeDefinition = configuration.getString(TaskManagerOptions.RPC_PORT); - return AkkaRpcServiceUtils.createRpcService(taskManagerAddress, portRangeDefinition, configuration); + return AkkaRpcServiceUtils.remoteServiceBuilder(configuration, taskManagerAddress, portRangeDefinition).createAndStart(); } private static String determineTaskManagerBindAddress( diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala index 0e20d43f66..efa15c2dbd 100755 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala @@ -148,7 +148,7 @@ object AkkaUtils { hostname: String, port: Int, executorConfig: Config): Config = { - getAkkaConfig(configuration, Some((hostname, port)), executorConfig) + getAkkaConfig(configuration, Some((hostname, port)), None, executorConfig) } /** @@ -191,6 +191,7 @@ object AkkaUtils { getAkkaConfig( configuration, externalAddress, + None, getForkJoinExecutorConfig(ForkJoinExecutorConfiguration.fromConfiguration(configuration))) } @@ -199,28 +200,42 @@ object AkkaUtils { * specified, then the actor system will listen on the respective address. * * @param configuration instance containing the user provided configuration values - * @param externalAddress optional tuple of bindAddress and port to be reachable at. + * @param externalAddress optional tuple of external address and port to be reachable at. * If None is given, then an Akka config for local actor system * will be returned + * @param bindAddress optional tuple of bind address and port to be used locally. + * If None is given, wildcard IP address and the external port wil be used. + * Take effects only if externalAddress is not None. * @param executorConfig config defining the used executor by the default dispatcher * @return Akka config */ @throws(classOf[UnknownHostException]) def getAkkaConfig(configuration: Configuration, externalAddress: Option[(String, Int)], + bindAddress: Option[(String, Int)], executorConfig: Config): Config = { val defaultConfig = getBasicAkkaConfig(configuration).withFallback(executorConfig) externalAddress match { - case Some((hostname, port)) => + case Some((externalHostname, externalPort)) => - val remoteConfig = getRemoteAkkaConfig(configuration, - // the wildcard IP lets us bind to all network interfaces - NetUtils.getWildcardIPAddress, port, - hostname, port) + bindAddress match { - remoteConfig.withFallback(defaultConfig) + case Some((bindHostname, bindPort)) => + + val remoteConfig = getRemoteAkkaConfig( + configuration, bindHostname, bindPort, externalHostname, externalPort) + + remoteConfig.withFallback(defaultConfig) + + case None => + val remoteConfig = getRemoteAkkaConfig(configuration, + // the wildcard IP lets us bind to all network interfaces + NetUtils.getWildcardIPAddress, externalPort, externalHostname, externalPort) + + remoteConfig.withFallback(defaultConfig) + } case None => defaultConfig diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/BootstrapToolsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/BootstrapToolsTest.java index 9338ac2d92..cee17cac44 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/BootstrapToolsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/BootstrapToolsTest.java @@ -392,7 +392,7 @@ public class BootstrapToolsTest extends TestLogger { CheckedSupplier.unchecked(() -> { cyclicBarrier.await(); - return BootstrapTools.startActorSystem( + return BootstrapTools.startRemoteActorSystem( new Configuration(), "localhost", "0", @@ -420,7 +420,7 @@ public class BootstrapToolsTest extends TestLogger { try { final int port = portOccupier.getLocalPort(); - BootstrapTools.startActorSystem(new Configuration(), "0.0.0.0", port, LOG); + BootstrapTools.startRemoteActorSystem(new Configuration(), "0.0.0.0", String.valueOf(port), LOG); fail("Expected to fail with a BindException"); } catch (Exception e) { assertThat(ExceptionUtils.findThrowable(e, BindException.class).isPresent(), is(true)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorOversizedResponseMessageTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorOversizedResponseMessageTest.java index 5e607bc8ca..4d0e1f1919 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorOversizedResponseMessageTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorOversizedResponseMessageTest.java @@ -67,8 +67,8 @@ public class AkkaRpcActorOversizedResponseMessageTest extends TestLogger { final Configuration configuration = new Configuration(); configuration.setString(AkkaOptions.FRAMESIZE, FRAMESIZE + " b"); - rpcService1 = AkkaRpcServiceUtils.createRpcService("localhost", 0, configuration); - rpcService2 = AkkaRpcServiceUtils.createRpcService("localhost", 0, configuration); + rpcService1 = AkkaRpcServiceUtils.remoteServiceBuilder(configuration, "localhost", 0).createAndStart(); + rpcService2 = AkkaRpcServiceUtils.remoteServiceBuilder(configuration, "localhost", 0).createAndStart(); } @AfterClass diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java index 0b9d4da2c4..60c8293c53 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java @@ -59,8 +59,8 @@ public class MessageSerializationTest extends TestLogger { Configuration configuration = new Configuration(); configuration.setString(AkkaOptions.FRAMESIZE, maxFrameSize + "b"); - akkaRpcService1 = AkkaRpcServiceUtils.createRpcService("localhost", 0, configuration); - akkaRpcService2 = AkkaRpcServiceUtils.createRpcService("localhost", 0, configuration); + akkaRpcService1 = AkkaRpcServiceUtils.remoteServiceBuilder(configuration, "localhost", 0).createAndStart(); + akkaRpcService2 = AkkaRpcServiceUtils.remoteServiceBuilder(configuration, "localhost", 0).createAndStart(); } @AfterClass diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureRecoveryITCase.java index 1262182023..8400bb4c6a 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureRecoveryITCase.java @@ -253,7 +253,7 @@ public class JobManagerHAProcessFailureRecoveryITCase extends TestLogger { config.set(TaskManagerOptions.TASK_HEAP_MEMORY, MemorySize.parse("128m")); config.set(TaskManagerOptions.CPU_CORES, 1.0); - final RpcService rpcService = AkkaRpcServiceUtils.createRpcService("localhost", 0, config); + final RpcService rpcService = AkkaRpcServiceUtils.remoteServiceBuilder(config, "localhost", 0).createAndStart(); try { final Deadline deadline = Deadline.fromNow(TEST_TIMEOUT); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java index 9d1839511e..4f78c2f830 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java @@ -123,7 +123,7 @@ public class ProcessFailureCancelingITCase extends TestLogger { config.set(TaskManagerOptions.CPU_CORES, 1.0); config.setInteger(RestOptions.PORT, 0); - final RpcService rpcService = AkkaRpcServiceUtils.createRpcService("localhost", 0, config); + final RpcService rpcService = AkkaRpcServiceUtils.remoteServiceBuilder(config, "localhost", 0).createAndStart(); final int jobManagerPort = rpcService.getPort(); config.setInteger(JobManagerOptions.PORT, jobManagerPort); -- Gitee From 8a1dbcc07d6b4ed7702741cd8b1c73d95f0d2583 Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Mon, 2 Mar 2020 17:49:09 +0800 Subject: [PATCH 270/885] [FLINK-15911][runtime] Support configure address/port and bind-address/bind-port separately for JM/TM RPC services. --- .../generated/all_taskmanager_section.html | 4 +- .../generated/common_host_port_section.html | 4 +- .../generated/job_manager_configuration.html | 12 ++++ .../generated/task_manager_configuration.html | 16 ++++- .../configuration/JobManagerOptions.java | 20 ++++++ .../configuration/TaskManagerOptions.java | 34 +++++++--- .../runtime/entrypoint/ClusterEntrypoint.java | 18 +++--- .../runtime/minicluster/MiniCluster.java | 63 +++++++++++++++---- .../minicluster/MiniClusterConfiguration.java | 28 ++++++--- .../runtime/rpc/akka/AkkaRpcServiceUtils.java | 18 ++++++ .../taskexecutor/TaskManagerRunner.java | 10 +-- 11 files changed, 179 insertions(+), 48 deletions(-) diff --git a/docs/_includes/generated/all_taskmanager_section.html b/docs/_includes/generated/all_taskmanager_section.html index 692c38b6e2..fadb54d2fd 100644 --- a/docs/_includes/generated/all_taskmanager_section.html +++ b/docs/_includes/generated/all_taskmanager_section.html @@ -54,7 +54,7 @@

    - + @@ -109,7 +109,7 @@ - +
    taskmanager.host
    (none) StringThe address of the network interface that the TaskManager binds to. This option can be used to define explicitly a binding address. Because different TaskManagers need different values for this option, usually it is specified in an additional non-shared TaskManager-specific config file.The external address of the network interface where the TaskManager is exposed. Because different TaskManagers need different values for this option, usually it is specified in an additional non-shared TaskManager-specific config file.
    taskmanager.jvm-exit-on-oom
    taskmanager.rpc.port
    "0" StringThe task manager’s IPC port. Accepts a list of ports (“50100,50101”), ranges (“50100-50200”) or a combination of both. It is recommended to set a range of ports to avoid collisions when multiple TaskManagers are running on the same machine.The external RPC port where the TaskManager is exposed. Accepts a list of ports (“50100,50101”), ranges (“50100-50200”) or a combination of both. It is recommended to set a range of ports to avoid collisions when multiple TaskManagers are running on the same machine.
    diff --git a/docs/_includes/generated/common_host_port_section.html b/docs/_includes/generated/common_host_port_section.html index 5deb71edc4..1841073155 100644 --- a/docs/_includes/generated/common_host_port_section.html +++ b/docs/_includes/generated/common_host_port_section.html @@ -60,13 +60,13 @@
    taskmanager.host
    (none) StringThe address of the network interface that the TaskManager binds to. This option can be used to define explicitly a binding address. Because different TaskManagers need different values for this option, usually it is specified in an additional non-shared TaskManager-specific config file.The external address of the network interface where the TaskManager is exposed. Because different TaskManagers need different values for this option, usually it is specified in an additional non-shared TaskManager-specific config file.
    taskmanager.rpc.port
    "0" StringThe task manager’s IPC port. Accepts a list of ports (“50100,50101”), ranges (“50100-50200”) or a combination of both. It is recommended to set a range of ports to avoid collisions when multiple TaskManagers are running on the same machine.The external RPC port where the TaskManager is exposed. Accepts a list of ports (“50100,50101”), ranges (“50100-50200”) or a combination of both. It is recommended to set a range of ports to avoid collisions when multiple TaskManagers are running on the same machine.
    diff --git a/docs/_includes/generated/job_manager_configuration.html b/docs/_includes/generated/job_manager_configuration.html index ef9c5652a6..3839890635 100644 --- a/docs/_includes/generated/job_manager_configuration.html +++ b/docs/_includes/generated/job_manager_configuration.html @@ -14,6 +14,12 @@ String Dictionary for JobManager to store the archives of completed jobs. + +
    jobmanager.bind-host
    + (none) + String + The local address of the network interface that the job manager binds to. If not configured, '0.0.0.0' will be used. +
    jobmanager.execution.attempts-history-size
    16 @@ -38,6 +44,12 @@ String The config parameter defining the network address to connect to for communication with the job manager. This value is only interpreted in setups where a single JobManager with static name or address exists (simple standalone setups, or container setups with dynamic service name resolution). It is not used in many high-availability setups, when a leader-election service (like ZooKeeper) is used to elect and discover the JobManager leader from potentially multiple standby JobManagers. + +
    jobmanager.rpc.bind-port
    + (none) + Integer + The local RPC port that the JobManager binds to. If not configured, the external port (configured by 'jobmanager.rpc.port') will be used. +
    jobmanager.rpc.port
    6123 diff --git a/docs/_includes/generated/task_manager_configuration.html b/docs/_includes/generated/task_manager_configuration.html index db02deee1f..6425a3e0e3 100644 --- a/docs/_includes/generated/task_manager_configuration.html +++ b/docs/_includes/generated/task_manager_configuration.html @@ -26,6 +26,12 @@ Long Time we wait for the timers in milliseconds to finish all pending timer threads when the stream task is cancelled. + +
    taskmanager.bind-host
    + (none) + String + The local address of the network interface that the task manager binds to. If not configured, '0.0.0.0' will be used. +
    taskmanager.debug.memory.log
    false @@ -42,7 +48,7 @@
    taskmanager.host
    (none) String - The address of the network interface that the TaskManager binds to. This option can be used to define explicitly a binding address. Because different TaskManagers need different values for this option, usually it is specified in an additional non-shared TaskManager-specific config file. + The external address of the network interface where the TaskManager is exposed. Because different TaskManagers need different values for this option, usually it is specified in an additional non-shared TaskManager-specific config file.
    taskmanager.jvm-exit-on-oom
    @@ -87,11 +93,17 @@ Duration Defines the timeout for the TaskManager registration. If the duration is exceeded without a successful registration, then the TaskManager terminates. + +
    taskmanager.rpc.bind-port
    + (none) + Integer + The local RPC port that the TaskManager binds to. If not configured, the external port (configured by 'taskmanager.rpc.port') will be used. +
    taskmanager.rpc.port
    "0" String - The task manager’s IPC port. Accepts a list of ports (“50100,50101”), ranges (“50100-50200”) or a combination of both. It is recommended to set a range of ports to avoid collisions when multiple TaskManagers are running on the same machine. + The external RPC port where the TaskManager is exposed. Accepts a list of ports (“50100,50101”), ranges (“50100-50200”) or a combination of both. It is recommended to set a range of ports to avoid collisions when multiple TaskManagers are running on the same machine. diff --git a/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java index efeb340972..1de85c6f51 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java @@ -54,6 +54,16 @@ public class JobManagerOptions { " leader-election service (like ZooKeeper) is used to elect and discover the JobManager" + " leader from potentially multiple standby JobManagers."); + /** + * The local address of the network interface that the job manager binds to. + */ + public static final ConfigOption BIND_HOST = + key("jobmanager.bind-host") + .stringType() + .noDefaultValue() + .withDescription("The local address of the network interface that the job manager binds to. If not" + + " configured, '0.0.0.0' will be used."); + /** * The config parameter defining the network port to connect to * for communication with the job manager. @@ -78,6 +88,16 @@ public class JobManagerOptions { " leader-election service (like ZooKeeper) is used to elect and discover the JobManager" + " leader from potentially multiple standby JobManagers."); + /** + * The local port that the job manager binds to. + */ + public static final ConfigOption RPC_BIND_PORT = + key("jobmanager.rpc.bind-port") + .intType() + .noDefaultValue() + .withDescription("The local RPC port that the JobManager binds to. If not configured, the external port" + + " (configured by '" + PORT.key() + "') will be used."); + /** * JVM heap size for the JobManager with memory size. */ diff --git a/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java index e2477fc472..b17efed81b 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java @@ -94,7 +94,7 @@ public class TaskManagerOptions { " or if it has been quarantined by another actor system."); /** - * The config parameter defining the task manager's hostname. + * The external address of the network interface where the TaskManager is exposed. * Overrides {@link #HOST_BIND_POLICY} automatic address binding. */ @Documentation.Section({Documentation.Sections.COMMON_HOST_PORT, Documentation.Sections.ALL_TASK_MANAGER}) @@ -102,11 +102,20 @@ public class TaskManagerOptions { key("taskmanager.host") .stringType() .noDefaultValue() - .withDescription("The address of the network interface that the TaskManager binds to." + - " This option can be used to define explicitly a binding address. Because" + - " different TaskManagers need different values for this option, usually it is specified in an" + + .withDescription("The external address of the network interface where the TaskManager is exposed." + + " Because different TaskManagers need different values for this option, usually it is specified in an" + " additional non-shared TaskManager-specific config file."); + /** + * The local address of the network interface that the task manager binds to. + */ + public static final ConfigOption BIND_HOST = + key("taskmanager.bind-host") + .stringType() + .noDefaultValue() + .withDescription("The local address of the network interface that the task manager binds to. If not" + + " configured, '0.0.0.0' will be used."); + /** * The default network port range the task manager expects incoming IPC connections. The {@code "0"} means that * the TaskManager searches for a free port. @@ -116,9 +125,20 @@ public class TaskManagerOptions { key("taskmanager.rpc.port") .stringType() .defaultValue("0") - .withDescription("The task manager’s IPC port. Accepts a list of ports (“50100,50101”), ranges" + - " (“50100-50200”) or a combination of both. It is recommended to set a range of ports to avoid" + - " collisions when multiple TaskManagers are running on the same machine."); + .withDescription("The external RPC port where the TaskManager is exposed. Accepts a list of ports" + + " (“50100,50101”), ranges (“50100-50200”) or a combination of both. It is recommended to set a" + + " range of ports to avoid collisions when multiple TaskManagers are running on the same machine."); + + + /** + * The local port that the task manager binds to. + */ + public static final ConfigOption RPC_BIND_PORT = + key("taskmanager.rpc.bind-port") + .intType() + .noDefaultValue() + .withDescription("The local RPC port that the TaskManager binds to. If not configured, the external port" + + " (configured by '" + RPC_PORT.key() + "') will be used."); /** * The initial registration backoff between two consecutive registration attempts. The backoff diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java index faa5a98fb9..59d1eb6141 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java @@ -70,7 +70,6 @@ import org.apache.flink.util.ShutdownHookUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.annotation.Nonnull; import javax.annotation.Nullable; import javax.annotation.concurrent.GuardedBy; @@ -248,10 +247,12 @@ public abstract class ClusterEntrypoint implements AutoCloseableAsync, FatalErro LOG.info("Initializing cluster services."); synchronized (lock) { - final String bindAddress = configuration.getString(JobManagerOptions.ADDRESS); - final String portRange = getRPCPortRange(configuration); - - commonRpcService = createRpcService(configuration, bindAddress, portRange); + commonRpcService = AkkaRpcServiceUtils.createRemoteRpcService( + configuration, + configuration.getString(JobManagerOptions.ADDRESS), + getRPCPortRange(configuration), + configuration.getString(JobManagerOptions.BIND_HOST), + configuration.getOptional(JobManagerOptions.RPC_BIND_PORT)); // update the configuration used to create the high availability services configuration.setString(JobManagerOptions.ADDRESS, commonRpcService.getAddress()); @@ -266,7 +267,7 @@ public abstract class ClusterEntrypoint implements AutoCloseableAsync, FatalErro heartbeatServices = createHeartbeatServices(configuration); metricRegistry = createMetricRegistry(configuration); - final RpcService metricQueryServiceRpcService = MetricUtils.startMetricsRpcService(configuration, bindAddress); + final RpcService metricQueryServiceRpcService = MetricUtils.startMetricsRpcService(configuration, commonRpcService.getAddress()); metricRegistry.startQueryService(metricQueryServiceRpcService, null); final String hostname = RpcUtils.getHostname(commonRpcService); @@ -280,11 +281,6 @@ public abstract class ClusterEntrypoint implements AutoCloseableAsync, FatalErro } } - @Nonnull - private RpcService createRpcService(Configuration configuration, String bindAddress, String portRange) throws Exception { - return AkkaRpcServiceUtils.remoteServiceBuilder(configuration, bindAddress, portRange).createAndStart(); - } - /** * Returns the port range for the common {@link RpcService}. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java index 3d00163873..41e3bdf461 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java @@ -269,19 +269,29 @@ public class MiniCluster implements JobExecutorService, AutoCloseableAsync { } else { // start a new service per component, possibly with custom bind addresses + final String jobManagerExternalAddress = miniClusterConfiguration.getJobManagerExternalAddress(); + final String taskManagerExternalAddress = miniClusterConfiguration.getTaskManagerExternalAddress(); + final String jobManagerExternalPortRange = miniClusterConfiguration.getJobManagerExternalPortRange(); + final String taskManagerExternalPortRange = miniClusterConfiguration.getTaskManagerExternalPortRange(); final String jobManagerBindAddress = miniClusterConfiguration.getJobManagerBindAddress(); final String taskManagerBindAddress = miniClusterConfiguration.getTaskManagerBindAddress(); - final String jobManagerBindPort = miniClusterConfiguration.getJobManagerBindPortRange(); - final String taskManagerBindPort = miniClusterConfiguration.getTaskManagerBindPortRange(); dispatcherResourceManagreComponentRpcServiceFactory = - new DedicatedRpcServiceFactory(configuration, jobManagerBindAddress, jobManagerBindPort); + new DedicatedRpcServiceFactory( + configuration, + jobManagerExternalAddress, + jobManagerExternalPortRange, + jobManagerBindAddress); taskManagerRpcServiceFactory = - new DedicatedRpcServiceFactory(configuration, taskManagerBindAddress, taskManagerBindPort); + new DedicatedRpcServiceFactory( + configuration, + taskManagerExternalAddress, + taskManagerExternalPortRange, + taskManagerBindAddress); // we always need the 'commonRpcService' for auxiliary calls // bind to the JobManager address with port 0 - commonRpcService = createRemoteRpcService(configuration, jobManagerBindAddress, "0"); + commonRpcService = createRemoteRpcService(configuration, jobManagerBindAddress, 0); } RpcService metricQueryServiceRpcService = MetricUtils.startMetricsRpcService( @@ -719,14 +729,36 @@ public class MiniCluster implements JobExecutorService, AutoCloseableAsync { * * @param configuration Flink configuration. * @param bindAddress The address to bind the RPC service to. - * @param bindPortRange The port range to bind the RPC service to. + * @param bindPort The port range to bind the RPC service to. * @return The instantiated RPC service */ protected RpcService createRemoteRpcService( Configuration configuration, String bindAddress, - String bindPortRange) throws Exception { - return AkkaRpcServiceUtils.remoteServiceBuilder(configuration, bindAddress, bindPortRange) + int bindPort) throws Exception { + return AkkaRpcServiceUtils.remoteServiceBuilder(configuration, bindAddress, String.valueOf(bindPort)) + .withBindAddress(bindAddress) + .withBindPort(bindPort) + .withCustomConfig(AkkaUtils.testDispatcherConfig()) + .createAndStart(); + } + + /** + * Factory method to instantiate the remote RPC service. + * + * @param configuration Flink configuration. + * @param externalAddress The external address to access the RPC service. + * @param externalPortRange The external port range to access the RPC service. + * @param bindAddress The address to bind the RPC service to. + * @return The instantiated RPC service + */ + protected RpcService createRemoteRpcService( + Configuration configuration, + String externalAddress, + String externalPortRange, + String bindAddress) throws Exception { + return AkkaRpcServiceUtils.remoteServiceBuilder(configuration, externalAddress, externalPortRange) + .withBindAddress(bindAddress) .withCustomConfig(AkkaUtils.testDispatcherConfig()) .createAndStart(); } @@ -902,18 +934,25 @@ public class MiniCluster implements JobExecutorService, AutoCloseableAsync { protected class DedicatedRpcServiceFactory implements RpcServiceFactory { private final Configuration configuration; + private final String externalAddress; + private final String externalPortRange; private final String bindAddress; - private final String bindPortRange; - DedicatedRpcServiceFactory(Configuration configuration, String bindAddress, String bindPortRange) { + DedicatedRpcServiceFactory( + Configuration configuration, + String externalAddress, + String externalPortRange, + String bindAddress) { this.configuration = configuration; + this.externalAddress = externalAddress; + this.externalPortRange = externalPortRange; this.bindAddress = bindAddress; - this.bindPortRange = bindPortRange; } @Override public RpcService createRpcService() throws Exception { - final RpcService rpcService = MiniCluster.this.createRemoteRpcService(configuration, bindAddress, bindPortRange); + final RpcService rpcService = MiniCluster.this.createRemoteRpcService( + configuration, externalAddress, externalPortRange, bindAddress); synchronized (lock) { rpcServices.add(rpcService); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java index f4d6d49e16..abe02704ca 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java @@ -94,26 +94,38 @@ public class MiniClusterConfiguration { return numTaskManagers; } - public String getJobManagerBindAddress() { + public String getJobManagerExternalAddress() { return commonBindAddress != null ? - commonBindAddress : - configuration.getString(JobManagerOptions.ADDRESS, "localhost"); + commonBindAddress : + configuration.getString(JobManagerOptions.ADDRESS, "localhost"); } - public String getTaskManagerBindAddress() { + public String getTaskManagerExternalAddress() { return commonBindAddress != null ? - commonBindAddress : - configuration.getString(TaskManagerOptions.HOST, "localhost"); + commonBindAddress : + configuration.getString(TaskManagerOptions.HOST, "localhost"); } - public String getJobManagerBindPortRange() { + public String getJobManagerExternalPortRange() { return String.valueOf(configuration.getInteger(JobManagerOptions.PORT, 0)); } - public String getTaskManagerBindPortRange() { + public String getTaskManagerExternalPortRange() { return configuration.getString(TaskManagerOptions.RPC_PORT); } + public String getJobManagerBindAddress() { + return commonBindAddress != null ? + commonBindAddress : + configuration.getString(JobManagerOptions.BIND_HOST, "localhost"); + } + + public String getTaskManagerBindAddress() { + return commonBindAddress != null ? + commonBindAddress : + configuration.getString(TaskManagerOptions.BIND_HOST, "localhost"); + } + public Time getRpcTimeout() { return AkkaUtils.getTimeoutAsTime(configuration); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceUtils.java index 181c0af74a..1b50e451ef 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceUtils.java @@ -69,6 +69,24 @@ public class AkkaRpcServiceUtils { // RPC instantiation // ------------------------------------------------------------------------ + public static AkkaRpcService createRemoteRpcService( + Configuration configuration, + @Nullable String externalAddress, + String externalPortRange, + @Nullable String bindAddress, + @SuppressWarnings("OptionalUsedAsFieldOrParameterType") Optional bindPort) throws Exception { + final AkkaRpcServiceBuilder akkaRpcServiceBuilder = + AkkaRpcServiceUtils.remoteServiceBuilder(configuration, externalAddress, externalPortRange); + + if (bindAddress != null) { + akkaRpcServiceBuilder.withBindAddress(bindAddress); + } + + bindPort.ifPresent(akkaRpcServiceBuilder::withBindPort); + + return akkaRpcServiceBuilder.createAndStart(); + } + public static AkkaRpcServiceBuilder remoteServiceBuilder(Configuration configuration, @Nullable String externalAddress, String externalPortRange) { return new AkkaRpcServiceBuilder(configuration, LOG, externalAddress, externalPortRange); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java index a62db016d5..b7c6d74be8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java @@ -419,10 +419,12 @@ public class TaskManagerRunner implements FatalErrorHandler, AutoCloseableAsync checkNotNull(configuration); checkNotNull(haServices); - final String taskManagerAddress = determineTaskManagerBindAddress(configuration, haServices); - final String portRangeDefinition = configuration.getString(TaskManagerOptions.RPC_PORT); - - return AkkaRpcServiceUtils.remoteServiceBuilder(configuration, taskManagerAddress, portRangeDefinition).createAndStart(); + return AkkaRpcServiceUtils.createRemoteRpcService( + configuration, + determineTaskManagerBindAddress(configuration, haServices), + configuration.getString(TaskManagerOptions.RPC_PORT), + configuration.getString(TaskManagerOptions.BIND_HOST), + configuration.getOptional(TaskManagerOptions.RPC_BIND_PORT)); } private static String determineTaskManagerBindAddress( -- Gitee From 3c997796e3aa6c4498849bcb029ffe6f5db515fc Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Mon, 2 Mar 2020 18:08:43 +0800 Subject: [PATCH 271/885] [FLINK-15911][runtime] Support configure address/port and bind-address/bind-port separately for netty shuffle service. --- .../generated/all_taskmanager_section.html | 2 +- .../generated/common_host_port_section.html | 2 +- ...tty_shuffle_environment_configuration.html | 8 +++- .../NettyShuffleEnvironmentOptions.java | 12 ++++- .../runtime/taskexecutor/KvStateService.java | 4 +- .../taskexecutor/TaskManagerRunner.java | 6 +-- .../taskexecutor/TaskManagerServices.java | 12 +++-- .../TaskManagerServicesConfiguration.java | 44 +++++++++++++++---- .../NettyShuffleEnvironmentConfiguration.java | 25 +++++++---- ...skExecutorLocalStateStoresManagerTest.java | 3 +- 10 files changed, 86 insertions(+), 32 deletions(-) diff --git a/docs/_includes/generated/all_taskmanager_section.html b/docs/_includes/generated/all_taskmanager_section.html index fadb54d2fd..3e7a8f94d0 100644 --- a/docs/_includes/generated/all_taskmanager_section.html +++ b/docs/_includes/generated/all_taskmanager_section.html @@ -30,7 +30,7 @@
    taskmanager.data.port
    0 Integer - The task manager’s port used for data exchange operations. + The task manager’s external port used for data exchange operations.
    taskmanager.data.ssl.enabled
    diff --git a/docs/_includes/generated/common_host_port_section.html b/docs/_includes/generated/common_host_port_section.html index 1841073155..fb5dd6e179 100644 --- a/docs/_includes/generated/common_host_port_section.html +++ b/docs/_includes/generated/common_host_port_section.html @@ -54,7 +54,7 @@
    taskmanager.data.port
    0 Integer - The task manager’s port used for data exchange operations. + The task manager’s external port used for data exchange operations.
    taskmanager.host
    diff --git a/docs/_includes/generated/netty_shuffle_environment_configuration.html b/docs/_includes/generated/netty_shuffle_environment_configuration.html index 901a40b3bb..18a4028e1f 100644 --- a/docs/_includes/generated/netty_shuffle_environment_configuration.html +++ b/docs/_includes/generated/netty_shuffle_environment_configuration.html @@ -8,11 +8,17 @@ + +
    taskmanager.data.bind-port
    + (none) + Integer + The task manager's bind port used for data exchange operations. If not configured, 'taskmanager.data.port' will be used. +
    taskmanager.data.port
    0 Integer - The task manager’s port used for data exchange operations. + The task manager’s external port used for data exchange operations.
    taskmanager.data.ssl.enabled
    diff --git a/flink-core/src/main/java/org/apache/flink/configuration/NettyShuffleEnvironmentOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/NettyShuffleEnvironmentOptions.java index 8670183486..c84aa34b18 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/NettyShuffleEnvironmentOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/NettyShuffleEnvironmentOptions.java @@ -41,7 +41,17 @@ public class NettyShuffleEnvironmentOptions { public static final ConfigOption DATA_PORT = key("taskmanager.data.port") .defaultValue(0) - .withDescription("The task manager’s port used for data exchange operations."); + .withDescription("The task manager’s external port used for data exchange operations."); + + /** + * The local network port that the task manager listen at for data exchange. + */ + public static final ConfigOption DATA_BIND_PORT = + key("taskmanager.data.bind-port") + .intType() + .noDefaultValue() + .withDescription("The task manager's bind port used for data exchange operations. If not configured, '" + + DATA_PORT.key() + "' will be used."); /** * Config parameter to override SSL support for taskmanager's data transport. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/KvStateService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/KvStateService.java index e283dc6546..2ebe4970b8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/KvStateService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/KvStateService.java @@ -170,7 +170,7 @@ public class KvStateService { int numProxyServerQueryThreads = qsConfig.numProxyQueryThreads() == 0 ? taskManagerServicesConfiguration.getNumberOfSlots() : qsConfig.numProxyQueryThreads(); kvClientProxy = QueryableStateUtils.createKvStateClientProxy( - taskManagerServicesConfiguration.getTaskManagerAddress(), + taskManagerServicesConfiguration.getExternalAddress(), qsConfig.getProxyPortRange(), numProxyServerNetworkThreads, numProxyServerQueryThreads, @@ -181,7 +181,7 @@ public class KvStateService { int numStateServerQueryThreads = qsConfig.numStateQueryThreads() == 0 ? taskManagerServicesConfiguration.getNumberOfSlots() : qsConfig.numStateQueryThreads(); kvStateServer = QueryableStateUtils.createKvStateServer( - taskManagerServicesConfiguration.getTaskManagerAddress(), + taskManagerServicesConfiguration.getExternalAddress(), qsConfig.getStateServerPortRange(), numStateServerNetworkThreads, numStateServerQueryThreads, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java index b7c6d74be8..9b6ec4ecf2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java @@ -355,7 +355,7 @@ public class TaskManagerRunner implements FatalErrorHandler, AutoCloseableAsync LOG.info("Starting TaskManager with ResourceID: {}", resourceID); - InetAddress remoteAddress = InetAddress.getByName(rpcService.getAddress()); + InetAddress externalAddress = InetAddress.getByName(rpcService.getAddress()); final TaskExecutorResourceSpec taskExecutorResourceSpec = TaskExecutorResourceUtils.resourceSpecFromConfig(configuration); @@ -363,13 +363,13 @@ public class TaskManagerRunner implements FatalErrorHandler, AutoCloseableAsync TaskManagerServicesConfiguration.fromConfiguration( configuration, resourceID, - remoteAddress, + externalAddress, localCommunicationOnly, taskExecutorResourceSpec); Tuple2 taskManagerMetricGroup = MetricUtils.instantiateTaskManagerMetricGroup( metricRegistry, - TaskManagerLocation.getHostName(remoteAddress), + TaskManagerLocation.getHostName(externalAddress), resourceID, taskManagerServicesConfiguration.getSystemResourceMetricsProbingInterval()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java index 04c3b29524..a45bec2c26 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java @@ -229,15 +229,19 @@ public class TaskManagerServices { taskManagerServicesConfiguration, taskEventDispatcher, taskManagerMetricGroup); - final int dataPort = shuffleEnvironment.start(); + final int listeningDataPort = shuffleEnvironment.start(); final KvStateService kvStateService = KvStateService.fromConfiguration(taskManagerServicesConfiguration); kvStateService.start(); final TaskManagerLocation taskManagerLocation = new TaskManagerLocation( taskManagerServicesConfiguration.getResourceID(), - taskManagerServicesConfiguration.getTaskManagerAddress(), - dataPort); + taskManagerServicesConfiguration.getExternalAddress(), + // we expose the task manager location with the listening port + // iff the external data port is not explicitly defined + taskManagerServicesConfiguration.getExternalDataPort() > 0 ? + taskManagerServicesConfiguration.getExternalDataPort() : + listeningDataPort); final BroadcastVariableManager broadcastVariableManager = new BroadcastVariableManager(); @@ -304,7 +308,7 @@ public class TaskManagerServices { taskManagerServicesConfiguration.getResourceID(), taskManagerServicesConfiguration.getNetworkMemorySize(), taskManagerServicesConfiguration.isLocalCommunicationOnly(), - taskManagerServicesConfiguration.getTaskManagerAddress(), + taskManagerServicesConfiguration.getBindAddress(), taskEventDispatcher, taskManagerMetricGroup); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java index 5ec28e0050..7c1c0932c5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java @@ -23,10 +23,13 @@ import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ConfigurationUtils; import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; +import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.registration.RetryingRegistrationConfiguration; import org.apache.flink.runtime.util.ConfigurationParserUtils; +import org.apache.flink.util.NetUtils; import javax.annotation.Nullable; @@ -46,7 +49,11 @@ public class TaskManagerServicesConfiguration { private final ResourceID resourceID; - private final InetAddress taskManagerAddress; + private final InetAddress externalAddress; + + private final InetAddress bindAddress; + + private final int externalDataPort; private final boolean localCommunicationOnly; @@ -74,7 +81,9 @@ public class TaskManagerServicesConfiguration { public TaskManagerServicesConfiguration( Configuration configuration, ResourceID resourceID, - InetAddress taskManagerAddress, + InetAddress externalAddress, + InetAddress bindAddress, + int externalDataPort, boolean localCommunicationOnly, String[] tmpDirPaths, String[] localRecoveryStateRootDirectories, @@ -89,7 +98,9 @@ public class TaskManagerServicesConfiguration { this.configuration = checkNotNull(configuration); this.resourceID = checkNotNull(resourceID); - this.taskManagerAddress = checkNotNull(taskManagerAddress); + this.externalAddress = checkNotNull(externalAddress); + this.bindAddress = checkNotNull(bindAddress); + this.externalDataPort = externalDataPort; this.localCommunicationOnly = localCommunicationOnly; this.tmpDirPaths = checkNotNull(tmpDirPaths); this.localRecoveryStateRootDirectories = checkNotNull(localRecoveryStateRootDirectories); @@ -121,8 +132,16 @@ public class TaskManagerServicesConfiguration { return resourceID; } - InetAddress getTaskManagerAddress() { - return taskManagerAddress; + InetAddress getExternalAddress() { + return externalAddress; + } + + InetAddress getBindAddress() { + return bindAddress; + } + + int getExternalDataPort() { + return externalDataPort; } boolean isLocalCommunicationOnly() { @@ -188,7 +207,7 @@ public class TaskManagerServicesConfiguration { * * @param configuration The configuration. * @param resourceID resource ID of the task manager - * @param remoteAddress identifying the IP address under which the TaskManager will be accessible + * @param externalAddress identifying the IP address under which the TaskManager will be accessible * @param localCommunicationOnly True if only local communication is possible. * Use only in cases where only one task manager runs. * @@ -197,9 +216,9 @@ public class TaskManagerServicesConfiguration { public static TaskManagerServicesConfiguration fromConfiguration( Configuration configuration, ResourceID resourceID, - InetAddress remoteAddress, + InetAddress externalAddress, boolean localCommunicationOnly, - TaskExecutorResourceSpec taskExecutorResourceSpec) { + TaskExecutorResourceSpec taskExecutorResourceSpec) throws Exception { final String[] tmpDirs = ConfigurationUtils.parseTempDirectories(configuration); String[] localStateRootDir = ConfigurationUtils.parseLocalStateDirectories(configuration); if (localStateRootDir.length == 0) { @@ -215,10 +234,17 @@ public class TaskManagerServicesConfiguration { final RetryingRegistrationConfiguration retryingRegistrationConfiguration = RetryingRegistrationConfiguration.fromConfiguration(configuration); + final int externalDataPort = configuration.getInteger(NettyShuffleEnvironmentOptions.DATA_PORT); + + String bindAddr = configuration.getString(TaskManagerOptions.BIND_HOST, NetUtils.getWildcardIPAddress()); + InetAddress bindAddress = InetAddress.getByName(bindAddr); + return new TaskManagerServicesConfiguration( configuration, resourceID, - remoteAddress, + externalAddress, + bindAddress, + externalDataPort, localCommunicationOnly, tmpDirs, localStateRootDir, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/NettyShuffleEnvironmentConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/NettyShuffleEnvironmentConfiguration.java index 05bf123d02..c1c3056c2e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/NettyShuffleEnvironmentConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/NettyShuffleEnvironmentConfiguration.java @@ -182,11 +182,11 @@ public class NettyShuffleEnvironmentConfiguration { boolean localTaskManagerCommunication, InetAddress taskManagerAddress) { - final int dataport = getDataport(configuration); + final int dataBindPort = getDataBindPort(configuration); final int pageSize = ConfigurationParserUtils.getPageSize(configuration); - final NettyConfig nettyConfig = createNettyConfig(configuration, localTaskManagerCommunication, taskManagerAddress, dataport); + final NettyConfig nettyConfig = createNettyConfig(configuration, localTaskManagerCommunication, taskManagerAddress, dataBindPort); final int numberOfNetworkBuffers = calculateNumberOfNetworkBuffers( configuration, @@ -238,12 +238,21 @@ public class NettyShuffleEnvironmentConfiguration { * @param configuration configuration object * @return the data port */ - private static int getDataport(Configuration configuration) { - final int dataport = configuration.getInteger(NettyShuffleEnvironmentOptions.DATA_PORT); - ConfigurationParserUtils.checkConfigParameter(dataport >= 0, dataport, NettyShuffleEnvironmentOptions.DATA_PORT.key(), - "Leave config parameter empty or use 0 to let the system choose a port automatically."); - - return dataport; + private static int getDataBindPort(Configuration configuration) { + final int dataBindPort; + if (configuration.contains(NettyShuffleEnvironmentOptions.DATA_BIND_PORT)) { + dataBindPort = configuration.getInteger(NettyShuffleEnvironmentOptions.DATA_BIND_PORT); + ConfigurationParserUtils.checkConfigParameter( + dataBindPort >= 0, dataBindPort, NettyShuffleEnvironmentOptions.DATA_BIND_PORT.key(), + "Leave config parameter empty to fallback to '" + + NettyShuffleEnvironmentOptions.DATA_PORT.key() + "' automatically."); + } else { + dataBindPort = configuration.getInteger(NettyShuffleEnvironmentOptions.DATA_PORT); + ConfigurationParserUtils.checkConfigParameter( + dataBindPort >= 0, dataBindPort, NettyShuffleEnvironmentOptions.DATA_PORT.key(), + "Leave config parameter empty or use 0 to let the system choose a port automatically."); + } + return dataBindPort; } /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManagerTest.java index 13e5e696c8..009a2be7f3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManagerTest.java @@ -38,7 +38,6 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; import java.io.File; -import java.io.IOException; import java.net.InetAddress; public class TaskExecutorLocalStateStoresManagerTest extends TestLogger { @@ -204,7 +203,7 @@ public class TaskExecutorLocalStateStoresManagerTest extends TestLogger { } private TaskManagerServicesConfiguration createTaskManagerServiceConfiguration( - Configuration config) throws IOException { + Configuration config) throws Exception { return TaskManagerServicesConfiguration.fromConfiguration( config, ResourceID.generate(), -- Gitee From 9dc7a4a47f85153b98e1f50eb8917dd6c8e15c85 Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Fri, 20 Mar 2020 19:27:43 +0800 Subject: [PATCH 272/885] [FLINK-15911][runtime] Support unresolveable external hostname. --- .../network/AbstractServerBase.java | 7 +- .../client/proxy/KvStateClientProxyImpl.java | 5 +- .../server/KvStateServerImpl.java | 5 +- .../proxy/KvStateClientProxyImplTest.java | 2 +- .../network/AbstractServerTest.java | 2 +- .../queryablestate/network/ClientTest.java | 2 +- .../network/KvStateServerHandlerTest.java | 4 +- .../network/KvStateServerTest.java | 2 +- .../flink/runtime/jobmaster/JobMaster.java | 15 ++- .../runtime/jobmaster/JobMasterGateway.java | 6 +- .../runtime/query/QueryableStateUtils.java | 9 +- .../taskexecutor/JobLeaderService.java | 14 +-- .../runtime/taskexecutor/TaskExecutor.java | 12 +-- .../taskexecutor/TaskManagerRunner.java | 7 +- .../taskexecutor/TaskManagerServices.java | 18 ++-- .../TaskManagerServicesConfiguration.java | 8 +- .../taskmanager/TaskManagerLocation.java | 11 ++ .../UnresolvedTaskManagerLocation.java | 60 +++++++++++ .../JobMasterPartitionReleaseTest.java | 10 +- .../runtime/jobmaster/JobMasterTest.java | 41 +++---- .../utils/TestingJobMasterGateway.java | 10 +- .../utils/TestingJobMasterGatewayBuilder.java | 6 +- ...skExecutorLocalStateStoresManagerTest.java | 2 +- .../taskexecutor/JobLeaderServiceTest.java | 4 +- .../taskexecutor/TaskExecutorTest.java | 102 +++++++++--------- .../TaskManagerServicesBuilder.java | 16 +-- .../LocalUnresolvedTaskManagerLocation.java | 32 ++++++ 27 files changed, 262 insertions(+), 150 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/UnresolvedTaskManagerLocation.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/LocalUnresolvedTaskManagerLocation.java diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerBase.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerBase.java index 2e67b98592..9e605ce2ce 100644 --- a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerBase.java +++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/network/AbstractServerBase.java @@ -41,7 +41,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.net.BindException; -import java.net.InetAddress; import java.net.InetSocketAddress; import java.util.HashSet; import java.util.Iterator; @@ -74,8 +73,8 @@ public abstract class AbstractServerBase bindPortRange; @@ -113,7 +112,7 @@ public abstract class AbstractServerBase bindPortIterator, final Integer numEventLoopThreads, final Integer numQueryThreads) { diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImpl.java b/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImpl.java index 41e7f4e112..2e5ea583ea 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImpl.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImpl.java @@ -33,7 +33,6 @@ import org.apache.flink.util.Preconditions; import javax.annotation.Nullable; -import java.net.InetAddress; import java.net.InetSocketAddress; import java.util.Iterator; import java.util.concurrent.ConcurrentHashMap; @@ -57,7 +56,7 @@ public class KvStateClientProxyImpl extends AbstractServerBaseThe server is instantiated using reflection by the - * {@link org.apache.flink.runtime.query.QueryableStateUtils#createKvStateClientProxy(InetAddress, Iterator, int, int, KvStateRequestStats) + * {@link org.apache.flink.runtime.query.QueryableStateUtils#createKvStateClientProxy(String, Iterator, int, int, KvStateRequestStats) * QueryableStateUtils.createKvStateClientProxy(InetAddress, Iterator, int, int, KvStateRequestStats)}. * *

    The server needs to be started via {@link #start()} in order to bind @@ -70,7 +69,7 @@ public class KvStateClientProxyImpl extends AbstractServerBase bindPortIterator, final Integer numEventLoopThreads, final Integer numQueryThreads, diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/server/KvStateServerImpl.java b/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/server/KvStateServerImpl.java index 072026888a..729b7bf835 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/server/KvStateServerImpl.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/server/KvStateServerImpl.java @@ -29,7 +29,6 @@ import org.apache.flink.runtime.query.KvStateRegistry; import org.apache.flink.runtime.query.KvStateServer; import org.apache.flink.util.Preconditions; -import java.net.InetAddress; import java.net.InetSocketAddress; import java.util.Iterator; import java.util.concurrent.TimeUnit; @@ -51,7 +50,7 @@ public class KvStateServerImpl extends AbstractServerBaseThe server is instantiated using reflection by the - * {@link org.apache.flink.runtime.query.QueryableStateUtils#createKvStateServer(InetAddress, Iterator, int, int, KvStateRegistry, KvStateRequestStats) + * {@link org.apache.flink.runtime.query.QueryableStateUtils#createKvStateServer(String, Iterator, int, int, KvStateRegistry, KvStateRequestStats) * QueryableStateUtils.createKvStateServer(InetAddress, Iterator, int, int, KvStateRegistry, KvStateRequestStats)}. * *

    The server needs to be started via {@link #start()} in order to bind @@ -65,7 +64,7 @@ public class KvStateServerImpl extends AbstractServerBase bindPortIterator, final Integer numEventLoopThreads, final Integer numQueryThreads, diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImplTest.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImplTest.java index 7e1cc24edb..db0b51e831 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImplTest.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImplTest.java @@ -47,7 +47,7 @@ public class KvStateClientProxyImplTest extends TestLogger { @Before public void setup() { kvStateClientProxy = new KvStateClientProxyImpl( - InetAddress.getLoopbackAddress(), + InetAddress.getLoopbackAddress().getHostName(), Collections.singleton(0).iterator(), 1, 1, diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/AbstractServerTest.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/AbstractServerTest.java index 02bfa7cf09..990cab32f6 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/AbstractServerTest.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/AbstractServerTest.java @@ -160,7 +160,7 @@ public class AbstractServerTest extends TestLogger { private final KvStateRequestStats requestStats; TestServer(String name, KvStateRequestStats stats, Iterator bindPort) throws UnknownHostException { - super(name, InetAddress.getLocalHost(), bindPort, 1, 1); + super(name, InetAddress.getLocalHost().getHostName(), bindPort, 1, 1); this.requestStats = stats; } diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java index 85b29fab70..b2ebdaf839 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java @@ -673,7 +673,7 @@ public class ClientTest { registry[i] = new KvStateRegistry(); serverStats[i] = new AtomicKvStateRequestStats(); server[i] = new KvStateServerImpl( - InetAddress.getLocalHost(), + InetAddress.getLocalHost().getHostName(), Collections.singletonList(0).iterator(), numServerEventLoopThreads, numServerQueryThreads, diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerHandlerTest.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerHandlerTest.java index f6f396eeea..e605dc9760 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerHandlerTest.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerHandlerTest.java @@ -89,7 +89,7 @@ public class KvStateServerHandlerTest extends TestLogger { public static void setup() { try { testServer = new KvStateServerImpl( - InetAddress.getLocalHost(), + InetAddress.getLocalHost().getHostName(), Collections.singletonList(0).iterator(), 1, 1, @@ -413,7 +413,7 @@ public class KvStateServerHandlerTest extends TestLogger { AtomicKvStateRequestStats stats = new AtomicKvStateRequestStats(); KvStateServerImpl localTestServer = new KvStateServerImpl( - InetAddress.getLocalHost(), + InetAddress.getLocalHost().getHostName(), Collections.singletonList(0).iterator(), 1, 1, diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java index 6aace48343..b6b2e30a3c 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java @@ -99,7 +99,7 @@ public class KvStateServerTest { KvStateRequestStats stats = new AtomicKvStateRequestStats(); server = new KvStateServerImpl( - InetAddress.getLocalHost(), + InetAddress.getLocalHost().getHostName(), Collections.singletonList(0).iterator(), 1, 1, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java index 474ea9387a..543ef1295d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java @@ -87,6 +87,7 @@ import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; import org.apache.flink.runtime.taskexecutor.slot.SlotOffer; import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import org.apache.flink.runtime.taskmanager.UnresolvedTaskManagerLocation; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.InstantiationUtil; @@ -571,9 +572,21 @@ public class JobMaster extends FencedRpcEndpoint implements JobMast @Override public CompletableFuture registerTaskManager( final String taskManagerRpcAddress, - final TaskManagerLocation taskManagerLocation, + final UnresolvedTaskManagerLocation unresolvedTaskManagerLocation, final Time timeout) { + final TaskManagerLocation taskManagerLocation; + try { + taskManagerLocation = TaskManagerLocation.fromUnresolvedLocation(unresolvedTaskManagerLocation); + } catch (Throwable throwable) { + final String errMsg = String.format( + "Could not accept TaskManager registration. TaskManager address %s cannot be resolved. %s", + unresolvedTaskManagerLocation.getExternalAddress(), + throwable.getMessage()); + log.error(errMsg); + return CompletableFuture.completedFuture(new RegistrationResponse.Decline(errMsg)); + } + final ResourceID taskManagerId = taskManagerLocation.getResourceID(); if (registeredTaskManagers.containsKey(taskManagerId)) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java index 506564dbb8..1006dadf0a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java @@ -43,7 +43,7 @@ import org.apache.flink.runtime.rpc.RpcTimeout; import org.apache.flink.runtime.taskexecutor.AccumulatorReport; import org.apache.flink.runtime.taskexecutor.slot.SlotOffer; import org.apache.flink.runtime.taskmanager.TaskExecutionState; -import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import org.apache.flink.runtime.taskmanager.UnresolvedTaskManagerLocation; import javax.annotation.Nullable; @@ -167,13 +167,13 @@ public interface JobMasterGateway extends * Registers the task manager at the job manager. * * @param taskManagerRpcAddress the rpc address of the task manager - * @param taskManagerLocation location of the task manager + * @param unresolvedTaskManagerLocation unresolved location of the task manager * @param timeout for the rpc call * @return Future registration response indicating whether the registration was successful or not */ CompletableFuture registerTaskManager( final String taskManagerRpcAddress, - final TaskManagerLocation taskManagerLocation, + final UnresolvedTaskManagerLocation unresolvedTaskManagerLocation, @RpcTimeout final Time timeout); /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateUtils.java index 0b4d448280..8c8ff19cd2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/QueryableStateUtils.java @@ -26,7 +26,6 @@ import org.slf4j.LoggerFactory; import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; -import java.net.InetAddress; import java.util.Iterator; /** @@ -54,7 +53,7 @@ public final class QueryableStateUtils { * @return the {@link KvStateClientProxy client proxy}. */ public static KvStateClientProxy createKvStateClientProxy( - final InetAddress address, + final String address, final Iterator ports, final int eventLoopThreads, final int queryThreads, @@ -70,7 +69,7 @@ public final class QueryableStateUtils { String classname = "org.apache.flink.queryablestate.client.proxy.KvStateClientProxyImpl"; Class clazz = Class.forName(classname).asSubclass(KvStateClientProxy.class); Constructor constructor = clazz.getConstructor( - InetAddress.class, + String.class, Iterator.class, Integer.class, Integer.class, @@ -108,7 +107,7 @@ public final class QueryableStateUtils { * @return the {@link KvStateServer state server}. */ public static KvStateServer createKvStateServer( - final InetAddress address, + final String address, final Iterator ports, final int eventLoopThreads, final int queryThreads, @@ -126,7 +125,7 @@ public final class QueryableStateUtils { String classname = "org.apache.flink.queryablestate.server.KvStateServerImpl"; Class clazz = Class.forName(classname).asSubclass(KvStateServer.class); Constructor constructor = clazz.getConstructor( - InetAddress.class, + String.class, Iterator.class, Integer.class, Integer.class, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java index b444cd09b7..fe9463a381 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java @@ -33,7 +33,7 @@ import org.apache.flink.runtime.registration.RegistrationResponse; import org.apache.flink.runtime.registration.RetryingRegistration; import org.apache.flink.runtime.registration.RetryingRegistrationConfiguration; import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import org.apache.flink.runtime.taskmanager.UnresolvedTaskManagerLocation; import org.apache.flink.util.Preconditions; import org.slf4j.Logger; @@ -64,7 +64,7 @@ public class JobLeaderService { private static final Logger LOG = LoggerFactory.getLogger(JobLeaderService.class); /** Self's location, used for the job manager connection. */ - private final TaskManagerLocation ownLocation; + private final UnresolvedTaskManagerLocation ownLocation; /** The leader retrieval service and listener for each registered job. */ private final Map> jobLeaderServices; @@ -87,7 +87,7 @@ public class JobLeaderService { private JobLeaderListener jobLeaderListener; public JobLeaderService( - TaskManagerLocation location, + UnresolvedTaskManagerLocation location, RetryingRegistrationConfiguration retryingRegistrationConfiguration) { this.ownLocation = Preconditions.checkNotNull(location); this.retryingRegistrationConfiguration = Preconditions.checkNotNull(retryingRegistrationConfiguration); @@ -418,7 +418,7 @@ public class JobLeaderService { private final String taskManagerRpcAddress; - private final TaskManagerLocation taskManagerLocation; + private final UnresolvedTaskManagerLocation unresolvedTaskManagerLocation; JobManagerRetryingRegistration( Logger log, @@ -429,7 +429,7 @@ public class JobLeaderService { JobMasterId jobMasterId, RetryingRegistrationConfiguration retryingRegistrationConfiguration, String taskManagerRpcAddress, - TaskManagerLocation taskManagerLocation) { + UnresolvedTaskManagerLocation unresolvedTaskManagerLocation) { super( log, rpcService, @@ -440,7 +440,7 @@ public class JobLeaderService { retryingRegistrationConfiguration); this.taskManagerRpcAddress = taskManagerRpcAddress; - this.taskManagerLocation = Preconditions.checkNotNull(taskManagerLocation); + this.unresolvedTaskManagerLocation = Preconditions.checkNotNull(unresolvedTaskManagerLocation); } @Override @@ -448,7 +448,7 @@ public class JobLeaderService { JobMasterGateway gateway, JobMasterId fencingToken, long timeoutMillis) { - return gateway.registerTaskManager(taskManagerRpcAddress, taskManagerLocation, Time.milliseconds(timeoutMillis)); + return gateway.registerTaskManager(taskManagerRpcAddress, unresolvedTaskManagerLocation, Time.milliseconds(timeoutMillis)); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java index 70931eed07..ec9faec420 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java @@ -118,7 +118,7 @@ import org.apache.flink.runtime.taskmanager.CheckpointResponder; import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.runtime.taskmanager.TaskManagerActions; -import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import org.apache.flink.runtime.taskmanager.UnresolvedTaskManagerLocation; import org.apache.flink.types.SerializableOptional; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; @@ -183,7 +183,7 @@ public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway { // --------- TaskManager services -------- /** The connection information of this task manager. */ - private final TaskManagerLocation taskManagerLocation; + private final UnresolvedTaskManagerLocation unresolvedTaskManagerLocation; private final TaskManagerMetricGroup taskManagerMetricGroup; @@ -272,7 +272,7 @@ public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway { this.taskSlotTable = taskExecutorServices.getTaskSlotTable(); this.jobManagerTable = taskExecutorServices.getJobManagerTable(); this.jobLeaderService = taskExecutorServices.getJobLeaderService(); - this.taskManagerLocation = taskExecutorServices.getTaskManagerLocation(); + this.unresolvedTaskManagerLocation = taskExecutorServices.getUnresolvedTaskManagerLocation(); this.localStateStoresManager = taskExecutorServices.getTaskManagerStateStore(); this.shuffleEnvironment = taskExecutorServices.getShuffleEnvironment(); this.kvStateService = taskExecutorServices.getKvStateService(); @@ -286,7 +286,7 @@ public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway { this.resourceManagerConnection = null; this.currentRegistrationTimeoutId = null; - final ResourceID resourceId = taskExecutorServices.getTaskManagerLocation().getResourceID(); + final ResourceID resourceId = taskExecutorServices.getUnresolvedTaskManagerLocation().getResourceID(); this.jobManagerHeartbeatManager = createJobManagerHeartbeatManager(heartbeatServices, resourceId); this.resourceManagerHeartbeatManager = createResourceManagerHeartbeatManager(heartbeatServices, resourceId); } @@ -1044,7 +1044,7 @@ public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway { final TaskExecutorRegistration taskExecutorRegistration = new TaskExecutorRegistration( getAddress(), getResourceID(), - taskManagerLocation.dataPort(), + unresolvedTaskManagerLocation.getDataPort(), hardwareDescription, taskManagerConfiguration.getDefaultSlotResourceProfile(), taskManagerConfiguration.getTotalResourceProfile() @@ -1686,7 +1686,7 @@ public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway { // ------------------------------------------------------------------------ public ResourceID getResourceID() { - return taskManagerLocation.getResourceID(); + return unresolvedTaskManagerLocation.getResourceID(); } // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java index 9b6ec4ecf2..0478953e8e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java @@ -54,7 +54,6 @@ import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils; import org.apache.flink.runtime.security.SecurityConfiguration; import org.apache.flink.runtime.security.SecurityUtils; import org.apache.flink.runtime.taskmanager.MemoryLogger; -import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.util.EnvironmentInformation; import org.apache.flink.runtime.util.ExecutorThreadFactory; import org.apache.flink.runtime.util.Hardware; @@ -132,7 +131,7 @@ public class TaskManagerRunner implements FatalErrorHandler, AutoCloseableAsync highAvailabilityServices = HighAvailabilityServicesUtils.createHighAvailabilityServices( configuration, executor, - HighAvailabilityServicesUtils.AddressResolution.TRY_ADDRESS_RESOLUTION); + HighAvailabilityServicesUtils.AddressResolution.NO_ADDRESS_RESOLUTION); rpcService = createRpcService(configuration, highAvailabilityServices); @@ -355,7 +354,7 @@ public class TaskManagerRunner implements FatalErrorHandler, AutoCloseableAsync LOG.info("Starting TaskManager with ResourceID: {}", resourceID); - InetAddress externalAddress = InetAddress.getByName(rpcService.getAddress()); + String externalAddress = rpcService.getAddress(); final TaskExecutorResourceSpec taskExecutorResourceSpec = TaskExecutorResourceUtils.resourceSpecFromConfig(configuration); @@ -369,7 +368,7 @@ public class TaskManagerRunner implements FatalErrorHandler, AutoCloseableAsync Tuple2 taskManagerMetricGroup = MetricUtils.instantiateTaskManagerMetricGroup( metricRegistry, - TaskManagerLocation.getHostName(externalAddress), + externalAddress, resourceID, taskManagerServicesConfiguration.getSystemResourceMetricsProbingInterval()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java index a45bec2c26..6d1161a437 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java @@ -34,7 +34,7 @@ import org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable; import org.apache.flink.runtime.taskexecutor.slot.TaskSlotTableImpl; import org.apache.flink.runtime.taskexecutor.slot.TimerService; import org.apache.flink.runtime.taskmanager.Task; -import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import org.apache.flink.runtime.taskmanager.UnresolvedTaskManagerLocation; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; @@ -59,7 +59,7 @@ public class TaskManagerServices { public static final String LOCAL_STATE_SUB_DIRECTORY_ROOT = "localState"; /** TaskManager services. */ - private final TaskManagerLocation taskManagerLocation; + private final UnresolvedTaskManagerLocation unresolvedTaskManagerLocation; private final long managedMemorySize; private final IOManager ioManager; private final ShuffleEnvironment shuffleEnvironment; @@ -72,7 +72,7 @@ public class TaskManagerServices { private final TaskEventDispatcher taskEventDispatcher; TaskManagerServices( - TaskManagerLocation taskManagerLocation, + UnresolvedTaskManagerLocation unresolvedTaskManagerLocation, long managedMemorySize, IOManager ioManager, ShuffleEnvironment shuffleEnvironment, @@ -84,7 +84,7 @@ public class TaskManagerServices { TaskExecutorLocalStateStoresManager taskManagerStateStore, TaskEventDispatcher taskEventDispatcher) { - this.taskManagerLocation = Preconditions.checkNotNull(taskManagerLocation); + this.unresolvedTaskManagerLocation = Preconditions.checkNotNull(unresolvedTaskManagerLocation); this.managedMemorySize = managedMemorySize; this.ioManager = Preconditions.checkNotNull(ioManager); this.shuffleEnvironment = Preconditions.checkNotNull(shuffleEnvironment); @@ -117,8 +117,8 @@ public class TaskManagerServices { return kvStateService; } - public TaskManagerLocation getTaskManagerLocation() { - return taskManagerLocation; + public UnresolvedTaskManagerLocation getUnresolvedTaskManagerLocation() { + return unresolvedTaskManagerLocation; } public BroadcastVariableManager getBroadcastVariableManager() { @@ -234,7 +234,7 @@ public class TaskManagerServices { final KvStateService kvStateService = KvStateService.fromConfiguration(taskManagerServicesConfiguration); kvStateService.start(); - final TaskManagerLocation taskManagerLocation = new TaskManagerLocation( + final UnresolvedTaskManagerLocation unresolvedTaskManagerLocation = new UnresolvedTaskManagerLocation( taskManagerServicesConfiguration.getResourceID(), taskManagerServicesConfiguration.getExternalAddress(), // we expose the task manager location with the listening port @@ -253,7 +253,7 @@ public class TaskManagerServices { final JobManagerTable jobManagerTable = new JobManagerTable(); - final JobLeaderService jobLeaderService = new JobLeaderService(taskManagerLocation, taskManagerServicesConfiguration.getRetryingRegistrationConfiguration()); + final JobLeaderService jobLeaderService = new JobLeaderService(unresolvedTaskManagerLocation, taskManagerServicesConfiguration.getRetryingRegistrationConfiguration()); final String[] stateRootDirectoryStrings = taskManagerServicesConfiguration.getLocalRecoveryStateRootDirectories(); @@ -269,7 +269,7 @@ public class TaskManagerServices { taskIOExecutor); return new TaskManagerServices( - taskManagerLocation, + unresolvedTaskManagerLocation, taskManagerServicesConfiguration.getManagedMemorySize().getBytes(), ioManager, shuffleEnvironment, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java index 7c1c0932c5..1336636ba8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java @@ -49,7 +49,7 @@ public class TaskManagerServicesConfiguration { private final ResourceID resourceID; - private final InetAddress externalAddress; + private final String externalAddress; private final InetAddress bindAddress; @@ -81,7 +81,7 @@ public class TaskManagerServicesConfiguration { public TaskManagerServicesConfiguration( Configuration configuration, ResourceID resourceID, - InetAddress externalAddress, + String externalAddress, InetAddress bindAddress, int externalDataPort, boolean localCommunicationOnly, @@ -132,7 +132,7 @@ public class TaskManagerServicesConfiguration { return resourceID; } - InetAddress getExternalAddress() { + String getExternalAddress() { return externalAddress; } @@ -216,7 +216,7 @@ public class TaskManagerServicesConfiguration { public static TaskManagerServicesConfiguration fromConfiguration( Configuration configuration, ResourceID resourceID, - InetAddress externalAddress, + String externalAddress, boolean localCommunicationOnly, TaskExecutorResourceSpec taskExecutorResourceSpec) throws Exception { final String[] tmpDirs = ConfigurationUtils.parseTempDirectories(configuration); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerLocation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerLocation.java index b3dca55770..fd9571242d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerLocation.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerLocation.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.taskmanager; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.util.NetUtils; @@ -27,6 +28,7 @@ import org.slf4j.LoggerFactory; import javax.annotation.Nonnull; import java.net.InetAddress; +import java.net.UnknownHostException; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -73,6 +75,7 @@ public class TaskManagerLocation implements Comparable, jav * @param dataPort * the port instance's task manager expects to receive transfer envelopes on */ + @VisibleForTesting public TaskManagerLocation(ResourceID resourceID, InetAddress inetAddress, int dataPort) { // -1 indicates a local instance connection info checkArgument(dataPort > 0 || dataPort == -1, "dataPort must be > 0, or -1 (local)"); @@ -90,6 +93,14 @@ public class TaskManagerLocation implements Comparable, jav "%s @ %s (dataPort=%d)", resourceID, fqdnHostName, dataPort); } + public static TaskManagerLocation fromUnresolvedLocation(final UnresolvedTaskManagerLocation unresolvedLocation) + throws UnknownHostException { + return new TaskManagerLocation( + unresolvedLocation.getResourceID(), + InetAddress.getByName(unresolvedLocation.getExternalAddress()), + unresolvedLocation.getDataPort()); + } + // ------------------------------------------------------------------------ // Getters // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/UnresolvedTaskManagerLocation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/UnresolvedTaskManagerLocation.java new file mode 100644 index 0000000000..78c3abe125 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/UnresolvedTaskManagerLocation.java @@ -0,0 +1,60 @@ +/* + * 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.runtime.taskmanager; + +import org.apache.flink.runtime.clusterframework.types.ResourceID; + +import java.io.Serializable; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * This class encapsulates the connection information of a TaskManager, without resolving the hostname. + * See also {@link TaskManagerLocation}. + */ +public class UnresolvedTaskManagerLocation implements Serializable { + + private static final long serialVersionUID = 1L; + + private final ResourceID resourceID; + private final String externalAddress; + private final int dataPort; + + public UnresolvedTaskManagerLocation(final ResourceID resourceID, final String externalAddress, final int dataPort) { + // -1 indicates a local instance connection info + checkArgument(dataPort > 0 || dataPort == -1, "dataPort must be > 0, or -1 (local)"); + + this.resourceID = checkNotNull(resourceID); + this.externalAddress = checkNotNull(externalAddress); + this.dataPort = dataPort; + } + + public ResourceID getResourceID() { + return resourceID; + } + + public String getExternalAddress() { + return externalAddress; + } + + public int getDataPort() { + return dataPort; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterPartitionReleaseTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterPartitionReleaseTest.java index be114f7c78..e850af8cac 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterPartitionReleaseTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterPartitionReleaseTest.java @@ -43,7 +43,7 @@ import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGateway; import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder; import org.apache.flink.runtime.taskexecutor.slot.SlotOffer; -import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; +import org.apache.flink.runtime.taskmanager.LocalUnresolvedTaskManagerLocation; import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.runtime.util.TestingFatalErrorHandler; import org.apache.flink.util.ExceptionUtils; @@ -186,7 +186,7 @@ public class JobMasterPartitionReleaseTest extends TestLogger { private final TemporaryFolder temporaryFolder = new TemporaryFolder(); - private final LocalTaskManagerLocation taskManagerLocation = new LocalTaskManagerLocation(); + private final LocalUnresolvedTaskManagerLocation localTaskManagerUnresolvedLocation = new LocalUnresolvedTaskManagerLocation(); private final CompletableFuture taskExecutorIdForStopTracking = new CompletableFuture<>(); private final CompletableFuture taskExecutorIdForPartitionRelease = new CompletableFuture<>(); @@ -248,12 +248,12 @@ public class JobMasterPartitionReleaseTest extends TestLogger { rpcService.registerGateway(taskExecutorGateway.getAddress(), taskExecutorGateway); - jobMasterGateway.registerTaskManager(taskExecutorGateway.getAddress(), taskManagerLocation, testingTimeout).get(); + jobMasterGateway.registerTaskManager(taskExecutorGateway.getAddress(), localTaskManagerUnresolvedLocation, testingTimeout).get(); final AllocationID allocationId = resourceManagerGateway.takeAllocationId(); Collection slotOffers = Collections.singleton(new SlotOffer(allocationId, 0, ResourceProfile.UNKNOWN)); - jobMasterGateway.offerSlots(taskManagerLocation.getResourceID(), slotOffers, testingTimeout).get(); + jobMasterGateway.offerSlots(localTaskManagerUnresolvedLocation.getResourceID(), slotOffers, testingTimeout).get(); } public JobMasterGateway getJobMasterGateway() { @@ -261,7 +261,7 @@ public class JobMasterPartitionReleaseTest extends TestLogger { } public ResourceID getTaskExecutorResourceID() { - return taskManagerLocation.getResourceID(); + return localTaskManagerUnresolvedLocation.getResourceID(); } public CompletableFuture getStopTrackingPartitionsTargetResourceId() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java index 34ab17ceb1..6ec0d8403a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java @@ -122,9 +122,10 @@ import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGateway; import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder; import org.apache.flink.runtime.taskexecutor.rpc.RpcCheckpointResponder; import org.apache.flink.runtime.taskexecutor.slot.SlotOffer; -import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; +import org.apache.flink.runtime.taskmanager.LocalUnresolvedTaskManagerLocation; import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import org.apache.flink.runtime.taskmanager.UnresolvedTaskManagerLocation; import org.apache.flink.runtime.testtasks.BlockingNoOpInvokable; import org.apache.flink.runtime.testtasks.NoOpInvokable; import org.apache.flink.runtime.testutils.CommonTestUtils; @@ -357,7 +358,7 @@ public class JobMasterTest extends TestLogger { public void testHeartbeatTimeoutWithTaskManager() throws Exception { final CompletableFuture heartbeatResourceIdFuture = new CompletableFuture<>(); final CompletableFuture disconnectedJobManagerFuture = new CompletableFuture<>(); - final TaskManagerLocation taskManagerLocation = new LocalTaskManagerLocation(); + final UnresolvedTaskManagerLocation unresolvedTaskManagerLocation = new LocalUnresolvedTaskManagerLocation(); final TestingTaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder() .setHeartbeatJobManagerConsumer((taskManagerId, ignored) -> heartbeatResourceIdFuture.complete(taskManagerId)) .setDisconnectJobManagerConsumer((jobId, throwable) -> disconnectedJobManagerFuture.complete(jobId)) @@ -384,7 +385,7 @@ public class JobMasterTest extends TestLogger { // register task manager will trigger monitor heartbeat target, schedule heartbeat request at interval time CompletableFuture registrationResponse = jobMasterGateway.registerTaskManager( taskExecutorGateway.getAddress(), - taskManagerLocation, + unresolvedTaskManagerLocation, testingTimeout); // wait for the completion of the registration @@ -413,7 +414,7 @@ public class JobMasterTest extends TestLogger { @Test public void testAllocatedSlotReportDoesNotContainStaleInformation() throws Exception { final CompletableFuture assertionFuture = new CompletableFuture<>(); - final TaskManagerLocation taskManagerLocation = new LocalTaskManagerLocation(); + final UnresolvedTaskManagerLocation unresolvedTaskManagerLocation = new LocalUnresolvedTaskManagerLocation(); final AtomicBoolean terminateHeartbeatVerification = new AtomicBoolean(false); final OneShotLatch hasReceivedSlotOffers = new OneShotLatch(); final TestingTaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder() @@ -454,7 +455,7 @@ public class JobMasterTest extends TestLogger { // register task manager will trigger monitor heartbeat target, schedule heartbeat request at interval time CompletableFuture registrationResponse = jobMasterGateway.registerTaskManager( taskExecutorGateway.getAddress(), - taskManagerLocation, + unresolvedTaskManagerLocation, testingTimeout); // wait for the completion of the registration @@ -462,7 +463,7 @@ public class JobMasterTest extends TestLogger { final SlotOffer slotOffer = new SlotOffer(new AllocationID(), 0, ResourceProfile.ANY); - final CompletableFuture> slotOfferFuture = jobMasterGateway.offerSlots(taskManagerLocation.getResourceID(), Collections.singleton(slotOffer), testingTimeout); + final CompletableFuture> slotOfferFuture = jobMasterGateway.offerSlots(unresolvedTaskManagerLocation.getResourceID(), Collections.singleton(slotOffer), testingTimeout); assertThat(slotOfferFuture.get(), containsInAnyOrder(slotOffer)); @@ -868,7 +869,7 @@ public class JobMasterTest extends TestLogger { blockingQueue.take(); final CompletableFuture submittedTaskFuture = new CompletableFuture<>(); - final LocalTaskManagerLocation taskManagerLocation = new LocalTaskManagerLocation(); + final LocalUnresolvedTaskManagerLocation taskManagerUnresolvedLocation = new LocalUnresolvedTaskManagerLocation(); final TestingTaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder() .setSubmitTaskConsumer((tdd, ignored) -> { submittedTaskFuture.complete(tdd); @@ -878,7 +879,7 @@ public class JobMasterTest extends TestLogger { rpcService.registerGateway(taskExecutorGateway.getAddress(), taskExecutorGateway); - jobMasterGateway.registerTaskManager(taskExecutorGateway.getAddress(), taskManagerLocation, testingTimeout).get(); + jobMasterGateway.registerTaskManager(taskExecutorGateway.getAddress(), taskManagerUnresolvedLocation, testingTimeout).get(); // wait for the slot request timeout final SlotRequest slotRequest = blockingQueue.take(); @@ -891,7 +892,7 @@ public class JobMasterTest extends TestLogger { final SlotOffer slotOffer = new SlotOffer(slotRequest.getAllocationId(), 0, ResourceProfile.ANY); - final CompletableFuture> acceptedSlotsFuture = jobMasterGateway.offerSlots(taskManagerLocation.getResourceID(), Collections.singleton(slotOffer), testingTimeout); + final CompletableFuture> acceptedSlotsFuture = jobMasterGateway.offerSlots(taskManagerUnresolvedLocation.getResourceID(), Collections.singleton(slotOffer), testingTimeout); final Collection acceptedSlots = acceptedSlotsFuture.get(); @@ -1731,7 +1732,7 @@ public class JobMasterTest extends TestLogger { final JobGraph jobGraph = JobGraphTestUtils.createSingleVertexJobGraph(); - final LocalTaskManagerLocation taskManagerLocation = new LocalTaskManagerLocation(); + final LocalUnresolvedTaskManagerLocation taskManagerUnresolvedLocation = new LocalUnresolvedTaskManagerLocation(); final AtomicBoolean isTrackingPartitions = new AtomicBoolean(true); final TestingJobMasterPartitionTracker partitionTracker = new TestingJobMasterPartitionTracker(); @@ -1761,7 +1762,7 @@ public class JobMasterTest extends TestLogger { final JobMasterGateway jobMasterGateway = jobMaster.getSelfGateway(JobMasterGateway.class); - final Collection slotOffers = registerSlotsAtJobMaster(1, jobMasterGateway, testingTaskExecutorGateway, taskManagerLocation); + final Collection slotOffers = registerSlotsAtJobMaster(1, jobMasterGateway, testingTaskExecutorGateway, taskManagerUnresolvedLocation); // check that we accepted the offered slot assertThat(slotOffers, hasSize(1)); @@ -1893,7 +1894,7 @@ public class JobMasterTest extends TestLogger { private void runJobFailureWhenTaskExecutorTerminatesTest( HeartbeatServices heartbeatServices, - BiConsumer jobReachedRunningState, + BiConsumer jobReachedRunningState, BiFunction> heartbeatConsumerFunction) throws Exception { final JobGraph jobGraph = JobGraphTestUtils.createSingleVertexJobGraph(); final JobMasterBuilder.TestingOnCompletionActions onCompletionActions = new JobMasterBuilder.TestingOnCompletionActions(); @@ -1910,24 +1911,24 @@ public class JobMasterTest extends TestLogger { final JobMasterGateway jobMasterGateway = jobMaster.getSelfGateway(JobMasterGateway.class); - final LocalTaskManagerLocation taskManagerLocation = new LocalTaskManagerLocation(); + final LocalUnresolvedTaskManagerLocation taskManagerUnresolvedLocation = new LocalUnresolvedTaskManagerLocation(); final CompletableFuture taskDeploymentFuture = new CompletableFuture<>(); final TestingTaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder() .setSubmitTaskConsumer((taskDeploymentDescriptor, jobMasterId) -> { taskDeploymentFuture.complete(taskDeploymentDescriptor.getExecutionAttemptId()); return CompletableFuture.completedFuture(Acknowledge.get()); }) - .setHeartbeatJobManagerConsumer(heartbeatConsumerFunction.apply(jobMasterGateway, taskManagerLocation.getResourceID())) + .setHeartbeatJobManagerConsumer(heartbeatConsumerFunction.apply(jobMasterGateway, taskManagerUnresolvedLocation.getResourceID())) .createTestingTaskExecutorGateway(); - final Collection slotOffers = registerSlotsAtJobMaster(1, jobMasterGateway, taskExecutorGateway, taskManagerLocation); + final Collection slotOffers = registerSlotsAtJobMaster(1, jobMasterGateway, taskExecutorGateway, taskManagerUnresolvedLocation); assertThat(slotOffers, hasSize(1)); final ExecutionAttemptID executionAttemptId = taskDeploymentFuture.get(); jobMasterGateway.updateTaskExecutionState(new TaskExecutionState(jobGraph.getJobID(), executionAttemptId, ExecutionState.RUNNING)).get(); - jobReachedRunningState.accept(taskManagerLocation, jobMasterGateway); + jobReachedRunningState.accept(taskManagerUnresolvedLocation, jobMasterGateway); final ArchivedExecutionGraph archivedExecutionGraph = onCompletionActions.getJobReachedGloballyTerminalStateFuture().get(); @@ -1945,21 +1946,21 @@ public class JobMasterTest extends TestLogger { numberSlots, jobMasterGateway, taskExecutorGateway, - new LocalTaskManagerLocation()); + new LocalUnresolvedTaskManagerLocation()); } private Collection registerSlotsAtJobMaster( int numberSlots, JobMasterGateway jobMasterGateway, TaskExecutorGateway taskExecutorGateway, - TaskManagerLocation taskManagerLocation) throws ExecutionException, InterruptedException { + UnresolvedTaskManagerLocation unresolvedTaskManagerLocation) throws ExecutionException, InterruptedException { final AllocationIdsResourceManagerGateway allocationIdsResourceManagerGateway = new AllocationIdsResourceManagerGateway(); rpcService.registerGateway(allocationIdsResourceManagerGateway.getAddress(), allocationIdsResourceManagerGateway); notifyResourceManagerLeaderListeners(allocationIdsResourceManagerGateway); rpcService.registerGateway(taskExecutorGateway.getAddress(), taskExecutorGateway); - jobMasterGateway.registerTaskManager(taskExecutorGateway.getAddress(), taskManagerLocation, testingTimeout).get(); + jobMasterGateway.registerTaskManager(taskExecutorGateway.getAddress(), unresolvedTaskManagerLocation, testingTimeout).get(); Collection slotOffers = IntStream .range(0, numberSlots) @@ -1970,7 +1971,7 @@ public class JobMasterTest extends TestLogger { }) .collect(Collectors.toList()); - return jobMasterGateway.offerSlots(taskManagerLocation.getResourceID(), slotOffers, testingTimeout).get(); + return jobMasterGateway.offerSlots(unresolvedTaskManagerLocation.getResourceID(), slotOffers, testingTimeout).get(); } private static final class AllocationIdsResourceManagerGateway extends TestingResourceManagerGateway { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/TestingJobMasterGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/TestingJobMasterGateway.java index 13d96300e7..47b92065a7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/TestingJobMasterGateway.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/TestingJobMasterGateway.java @@ -51,7 +51,7 @@ import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.taskexecutor.AccumulatorReport; import org.apache.flink.runtime.taskexecutor.slot.SlotOffer; import org.apache.flink.runtime.taskmanager.TaskExecutionState; -import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import org.apache.flink.runtime.taskmanager.UnresolvedTaskManagerLocation; import org.apache.flink.util.SerializedValue; import org.apache.flink.util.function.TriConsumer; import org.apache.flink.util.function.TriFunction; @@ -107,7 +107,7 @@ public class TestingJobMasterGateway implements JobMasterGateway { private final TriConsumer failSlotConsumer; @Nonnull - private final BiFunction> registerTaskManagerFunction; + private final BiFunction> registerTaskManagerFunction; @Nonnull private final BiConsumer taskManagerHeartbeatConsumer; @@ -169,7 +169,7 @@ public class TestingJobMasterGateway implements JobMasterGateway { @Nonnull Consumer disconnectResourceManagerConsumer, @Nonnull BiFunction, CompletableFuture>> offerSlotsFunction, @Nonnull TriConsumer failSlotConsumer, - @Nonnull BiFunction> registerTaskManagerFunction, + @Nonnull BiFunction> registerTaskManagerFunction, @Nonnull BiConsumer taskManagerHeartbeatConsumer, @Nonnull Consumer resourceManagerHeartbeatConsumer, @Nonnull Supplier> requestJobDetailsSupplier, @@ -262,8 +262,8 @@ public class TestingJobMasterGateway implements JobMasterGateway { } @Override - public CompletableFuture registerTaskManager(String taskManagerRpcAddress, TaskManagerLocation taskManagerLocation, Time timeout) { - return registerTaskManagerFunction.apply(taskManagerRpcAddress, taskManagerLocation); + public CompletableFuture registerTaskManager(String taskManagerRpcAddress, UnresolvedTaskManagerLocation unresolvedTaskManagerLocation, Time timeout) { + return registerTaskManagerFunction.apply(taskManagerRpcAddress, unresolvedTaskManagerLocation); } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/TestingJobMasterGatewayBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/TestingJobMasterGatewayBuilder.java index f2a47e9306..a960448be1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/TestingJobMasterGatewayBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/TestingJobMasterGatewayBuilder.java @@ -51,7 +51,7 @@ import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.taskexecutor.AccumulatorReport; import org.apache.flink.runtime.taskexecutor.slot.SlotOffer; import org.apache.flink.runtime.taskmanager.TaskExecutionState; -import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import org.apache.flink.runtime.taskmanager.UnresolvedTaskManagerLocation; import org.apache.flink.util.SerializedValue; import org.apache.flink.util.function.TriConsumer; import org.apache.flink.util.function.TriFunction; @@ -85,7 +85,7 @@ public class TestingJobMasterGatewayBuilder { private Consumer disconnectResourceManagerConsumer = ignored -> {}; private BiFunction, CompletableFuture>> offerSlotsFunction = (ignoredA, ignoredB) -> CompletableFuture.completedFuture(Collections.emptyList()); private TriConsumer failSlotConsumer = (ignoredA, ignoredB, ignoredC) -> {}; - private BiFunction> registerTaskManagerFunction = (ignoredA, ignoredB) -> CompletableFuture.completedFuture(new JMTMRegistrationSuccess(RESOURCE_MANAGER_ID)); + private BiFunction> registerTaskManagerFunction = (ignoredA, ignoredB) -> CompletableFuture.completedFuture(new JMTMRegistrationSuccess(RESOURCE_MANAGER_ID)); private BiConsumer taskManagerHeartbeatConsumer = (ignoredA, ignoredB) -> {}; private Consumer resourceManagerHeartbeatConsumer = ignored -> {}; private Supplier> requestJobDetailsSupplier = () -> FutureUtils.completedExceptionally(new UnsupportedOperationException()); @@ -158,7 +158,7 @@ public class TestingJobMasterGatewayBuilder { return this; } - public TestingJobMasterGatewayBuilder setRegisterTaskManagerFunction(BiFunction> registerTaskManagerFunction) { + public TestingJobMasterGatewayBuilder setRegisterTaskManagerFunction(BiFunction> registerTaskManagerFunction) { this.registerTaskManagerFunction = registerTaskManagerFunction; return this; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManagerTest.java index 009a2be7f3..d3a945254a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManagerTest.java @@ -207,7 +207,7 @@ public class TaskExecutorLocalStateStoresManagerTest extends TestLogger { return TaskManagerServicesConfiguration.fromConfiguration( config, ResourceID.generate(), - InetAddress.getLocalHost(), + InetAddress.getLocalHost().getHostName(), true, TaskExecutorResourceUtils.resourceSpecFromConfigForLocalExecution(config)); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/JobLeaderServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/JobLeaderServiceTest.java index 51595e85b4..a27e7d0632 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/JobLeaderServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/JobLeaderServiceTest.java @@ -28,7 +28,7 @@ import org.apache.flink.runtime.jobmaster.JobMasterId; import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService; import org.apache.flink.runtime.registration.RetryingRegistrationConfiguration; import org.apache.flink.runtime.rpc.TestingRpcServiceResource; -import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; +import org.apache.flink.runtime.taskmanager.LocalUnresolvedTaskManagerLocation; import org.apache.flink.util.TestLogger; import org.junit.ClassRule; @@ -53,7 +53,7 @@ public class JobLeaderServiceTest extends TestLogger { @Test public void handlesConcurrentJobAdditionsAndLeaderChanges() throws Exception { final JobLeaderService jobLeaderService = new JobLeaderService( - new LocalTaskManagerLocation(), + new LocalUnresolvedTaskManagerLocation(), RetryingRegistrationConfiguration.defaultConfiguration()); final TestingJobLeaderListener jobLeaderListener = new TestingJobLeaderListener(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java index 87ba525346..df42ca65bb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java @@ -95,9 +95,9 @@ import org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable; import org.apache.flink.runtime.taskexecutor.slot.TaskSlotTableImpl; import org.apache.flink.runtime.taskexecutor.slot.TaskSlotUtils; import org.apache.flink.runtime.taskexecutor.slot.TestingTaskSlotTable; -import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; +import org.apache.flink.runtime.taskmanager.LocalUnresolvedTaskManagerLocation; import org.apache.flink.runtime.taskmanager.Task; -import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import org.apache.flink.runtime.taskmanager.UnresolvedTaskManagerLocation; import org.apache.flink.runtime.testtasks.NoOpInvokable; import org.apache.flink.runtime.util.TestingFatalErrorHandler; import org.apache.flink.util.ExceptionUtils; @@ -201,7 +201,7 @@ public class TaskExecutorTest extends TestLogger { private Configuration configuration; - private TaskManagerLocation taskManagerLocation; + private UnresolvedTaskManagerLocation unresolvedTaskManagerLocation; private JobID jobId; @@ -228,7 +228,7 @@ public class TaskExecutorTest extends TestLogger { configuration = new Configuration(); - taskManagerLocation = new LocalTaskManagerLocation(); + unresolvedTaskManagerLocation = new LocalUnresolvedTaskManagerLocation(); jobId = new JobID(); testingFatalErrorHandler = new TestingFatalErrorHandler(); @@ -272,7 +272,7 @@ public class TaskExecutorTest extends TestLogger { public void testShouldShutDownTaskManagerServicesInPostStop() throws Exception { final TaskSlotTableImpl taskSlotTable = TaskSlotUtils.createTaskSlotTable(1); - final JobLeaderService jobLeaderService = new JobLeaderService(taskManagerLocation, RetryingRegistrationConfiguration.defaultConfiguration()); + final JobLeaderService jobLeaderService = new JobLeaderService(unresolvedTaskManagerLocation, RetryingRegistrationConfiguration.defaultConfiguration()); final IOManager ioManager = new IOManagerAsync(tmp.newFolder().getAbsolutePath()); @@ -287,7 +287,7 @@ public class TaskExecutorTest extends TestLogger { kvStateService.start(); final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() - .setTaskManagerLocation(taskManagerLocation) + .setUnresolvedTaskManagerLocation(unresolvedTaskManagerLocation) .setIoManager(ioManager) .setShuffleEnvironment(nettyShuffleEnvironment) .setKvStateService(kvStateService) @@ -313,7 +313,7 @@ public class TaskExecutorTest extends TestLogger { public void testHeartbeatTimeoutWithJobManager() throws Exception { final TaskSlotTable taskSlotTable = TaskSlotUtils.createTaskSlotTable(1); - final JobLeaderService jobLeaderService = new JobLeaderService(taskManagerLocation, RetryingRegistrationConfiguration.defaultConfiguration()); + final JobLeaderService jobLeaderService = new JobLeaderService(unresolvedTaskManagerLocation, RetryingRegistrationConfiguration.defaultConfiguration()); final long heartbeatInterval = 1L; final long heartbeatTimeout = 3L; @@ -325,12 +325,12 @@ public class TaskExecutorTest extends TestLogger { final ResourceID jmResourceId = ResourceID.generate(); final CountDownLatch registrationAttempts = new CountDownLatch(2); - final CompletableFuture taskManagerLocationFuture = new CompletableFuture<>(); + final CompletableFuture taskManagerUnresolvedLocationFuture = new CompletableFuture<>(); final CompletableFuture disconnectTaskManagerFuture = new CompletableFuture<>(); final TestingJobMasterGateway jobMasterGateway = new TestingJobMasterGatewayBuilder() - .setRegisterTaskManagerFunction((s, taskManagerLocation) -> { + .setRegisterTaskManagerFunction((s, taskManagerUnresolvedLocation) -> { registrationAttempts.countDown(); - taskManagerLocationFuture.complete(taskManagerLocation); + taskManagerUnresolvedLocationFuture.complete(taskManagerUnresolvedLocation); return CompletableFuture.completedFuture(new JMTMRegistrationSuccess(jmResourceId)); }) .setDisconnectTaskManagerFunction( @@ -344,7 +344,7 @@ public class TaskExecutorTest extends TestLogger { final TaskExecutorLocalStateStoresManager localStateStoresManager = createTaskExecutorLocalStateStoresManager(); final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() - .setTaskManagerLocation(taskManagerLocation) + .setUnresolvedTaskManagerLocation(unresolvedTaskManagerLocation) .setTaskSlotTable(taskSlotTable) .setJobLeaderService(jobLeaderService) .setTaskStateManager(localStateStoresManager) @@ -366,12 +366,12 @@ public class TaskExecutorTest extends TestLogger { jobManagerLeaderRetriever.notifyListener(jobMasterAddress, jmLeaderId); // register task manager success will trigger monitoring heartbeat target between tm and jm - final TaskManagerLocation taskManagerLocation1 = taskManagerLocationFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); - assertThat(taskManagerLocation1, equalTo(taskManagerLocation)); + final UnresolvedTaskManagerLocation unresolvedTaskManagerLocation1 = taskManagerUnresolvedLocationFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + assertThat(unresolvedTaskManagerLocation1, equalTo(unresolvedTaskManagerLocation)); // the timeout should trigger disconnecting from the JobManager final ResourceID resourceID = disconnectTaskManagerFuture.get(heartbeatTimeout * 50L, TimeUnit.MILLISECONDS); - assertThat(resourceID, equalTo(taskManagerLocation.getResourceID())); + assertThat(resourceID, equalTo(unresolvedTaskManagerLocation.getResourceID())); assertTrue( "The TaskExecutor should try to reconnect to the JM", @@ -428,7 +428,7 @@ public class TaskExecutorTest extends TestLogger { final TaskExecutorLocalStateStoresManager localStateStoresManager = createTaskExecutorLocalStateStoresManager(); final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() - .setTaskManagerLocation(taskManagerLocation) + .setUnresolvedTaskManagerLocation(unresolvedTaskManagerLocation) .setTaskSlotTable(taskSlotTable) .setTaskStateManager(localStateStoresManager) .build(); @@ -442,10 +442,10 @@ public class TaskExecutorTest extends TestLogger { resourceManagerLeaderRetriever.notifyListener(rmAddress, rmLeaderId.toUUID()); // register resource manager success will trigger monitoring heartbeat target between tm and rm - assertThat(taskExecutorRegistrationFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS), equalTo(taskManagerLocation.getResourceID())); + assertThat(taskExecutorRegistrationFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS), equalTo(unresolvedTaskManagerLocation.getResourceID())); // heartbeat timeout should trigger disconnect TaskManager from ResourceManager - assertThat(taskExecutorDisconnectFuture.get(heartbeatTimeout * 50L, TimeUnit.MILLISECONDS), equalTo(taskManagerLocation.getResourceID())); + assertThat(taskExecutorDisconnectFuture.get(heartbeatTimeout * 50L, TimeUnit.MILLISECONDS), equalTo(unresolvedTaskManagerLocation.getResourceID())); assertTrue( "The TaskExecutor should try to reconnect to the RM", @@ -489,7 +489,7 @@ public class TaskExecutorTest extends TestLogger { rpc.registerGateway(rmAddress, rmGateway); - final SlotID slotId = new SlotID(taskManagerLocation.getResourceID(), 0); + final SlotID slotId = new SlotID(unresolvedTaskManagerLocation.getResourceID(), 0); final ResourceProfile resourceProfile = ResourceProfile.fromResources(1.0, 1); final SlotReport slotReport1 = new SlotReport( new SlotStatus( @@ -512,7 +512,7 @@ public class TaskExecutorTest extends TestLogger { final TaskExecutorLocalStateStoresManager localStateStoresManager = createTaskExecutorLocalStateStoresManager(); final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() - .setTaskManagerLocation(taskManagerLocation) + .setUnresolvedTaskManagerLocation(unresolvedTaskManagerLocation) .setTaskSlotTable(taskSlotTable) .setTaskStateManager(localStateStoresManager) .build(); @@ -528,7 +528,7 @@ public class TaskExecutorTest extends TestLogger { resourceManagerLeaderRetriever.notifyListener(rmAddress, rmLeaderId); // register resource manager success will trigger monitoring heartbeat target between tm and rm - assertThat(taskExecutorRegistrationFuture.get(), equalTo(taskManagerLocation.getResourceID())); + assertThat(taskExecutorRegistrationFuture.get(), equalTo(unresolvedTaskManagerLocation.getResourceID())); assertThat(initialSlotReportFuture.get(), equalTo(slotReport1)); TaskExecutorGateway taskExecutorGateway = taskManager.getSelfGateway(TaskExecutorGateway.class); @@ -585,7 +585,7 @@ public class TaskExecutorTest extends TestLogger { final TaskSlotTable taskSlotTable = TaskSlotUtils.createTaskSlotTable(1); final TaskExecutorLocalStateStoresManager localStateStoresManager = createTaskExecutorLocalStateStoresManager(); final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() - .setTaskManagerLocation(taskManagerLocation) + .setUnresolvedTaskManagerLocation(unresolvedTaskManagerLocation) .setTaskSlotTable(taskSlotTable) .setTaskStateManager(localStateStoresManager) .build(); @@ -636,7 +636,7 @@ public class TaskExecutorTest extends TestLogger { final TaskExecutorLocalStateStoresManager localStateStoresManager = createTaskExecutorLocalStateStoresManager(); final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() - .setTaskManagerLocation(taskManagerLocation) + .setUnresolvedTaskManagerLocation(unresolvedTaskManagerLocation) .setTaskSlotTable(taskSlotTable) .setTaskStateManager(localStateStoresManager) .build(); @@ -655,7 +655,7 @@ public class TaskExecutorTest extends TestLogger { verify(rmGateway1, Mockito.timeout(timeout.toMilliseconds())).registerTaskExecutor( argThat(taskExecutorRegistration -> taskExecutorRegistration.getTaskExecutorAddress().equals(taskManagerAddress) && - taskExecutorRegistration.getResourceId().equals(taskManagerLocation.getResourceID())), + taskExecutorRegistration.getResourceId().equals(unresolvedTaskManagerLocation.getResourceID())), any(Time.class)); assertNotNull(taskManager.getResourceManagerConnection()); @@ -668,7 +668,7 @@ public class TaskExecutorTest extends TestLogger { verify(rmGateway2, Mockito.timeout(timeout.toMilliseconds())).registerTaskExecutor( argThat(taskExecutorRegistration -> taskExecutorRegistration.getTaskExecutorAddress().equals(taskManagerAddress) && - taskExecutorRegistration.getResourceId().equals(taskManagerLocation.getResourceID())), + taskExecutorRegistration.getResourceId().equals(unresolvedTaskManagerLocation.getResourceID())), any(Time.class)); assertNotNull(taskManager.getResourceManagerConnection()); } @@ -722,7 +722,7 @@ public class TaskExecutorTest extends TestLogger { public void testJobLeaderDetection() throws Exception { final TaskSlotTable taskSlotTable = TaskSlotUtils.createTaskSlotTable(1); final JobManagerTable jobManagerTable = new JobManagerTable(); - final JobLeaderService jobLeaderService = new JobLeaderService(taskManagerLocation, RetryingRegistrationConfiguration.defaultConfiguration()); + final JobLeaderService jobLeaderService = new JobLeaderService(unresolvedTaskManagerLocation, RetryingRegistrationConfiguration.defaultConfiguration()); final TestingResourceManagerGateway resourceManagerGateway = new TestingResourceManagerGateway(); CompletableFuture initialSlotReportFuture = new CompletableFuture<>(); @@ -744,12 +744,12 @@ public class TaskExecutorTest extends TestLogger { rpc.registerGateway(jobMasterGateway.getAddress(), jobMasterGateway); final AllocationID allocationId = new AllocationID(); - final SlotID slotId = new SlotID(taskManagerLocation.getResourceID(), 0); + final SlotID slotId = new SlotID(unresolvedTaskManagerLocation.getResourceID(), 0); final TaskExecutorLocalStateStoresManager localStateStoresManager = createTaskExecutorLocalStateStoresManager(); final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() - .setTaskManagerLocation(taskManagerLocation) + .setUnresolvedTaskManagerLocation(unresolvedTaskManagerLocation) .setTaskSlotTable(taskSlotTable) .setJobManagerTable(jobManagerTable) .setJobLeaderService(jobLeaderService) @@ -800,7 +800,7 @@ public class TaskExecutorTest extends TestLogger { public void testSlotAcceptance() throws Exception { final TaskSlotTable taskSlotTable = TaskSlotUtils.createTaskSlotTable(2); final JobManagerTable jobManagerTable = new JobManagerTable(); - final JobLeaderService jobLeaderService = new JobLeaderService(taskManagerLocation, RetryingRegistrationConfiguration.defaultConfiguration()); + final JobLeaderService jobLeaderService = new JobLeaderService(unresolvedTaskManagerLocation, RetryingRegistrationConfiguration.defaultConfiguration()); final String resourceManagerAddress = "rm"; final UUID resourceManagerLeaderId = UUID.randomUUID(); @@ -836,7 +836,7 @@ public class TaskExecutorTest extends TestLogger { when(jobMasterGateway.registerTaskManager( any(String.class), - eq(taskManagerLocation), + eq(unresolvedTaskManagerLocation), any(Time.class) )).thenReturn(CompletableFuture.completedFuture(new JMTMRegistrationSuccess(jmResourceId))); when(jobMasterGateway.getHostname()).thenReturn(jobManagerAddress); @@ -851,7 +851,7 @@ public class TaskExecutorTest extends TestLogger { final TaskExecutorLocalStateStoresManager localStateStoresManager = createTaskExecutorLocalStateStoresManager(); final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() - .setTaskManagerLocation(taskManagerLocation) + .setUnresolvedTaskManagerLocation(unresolvedTaskManagerLocation) .setTaskSlotTable(taskSlotTable) .setJobManagerTable(jobManagerTable) .setJobLeaderService(jobLeaderService) @@ -863,7 +863,7 @@ public class TaskExecutorTest extends TestLogger { try { taskManager.start(); - assertThat(registrationFuture.get(), equalTo(taskManagerLocation.getResourceID())); + assertThat(registrationFuture.get(), equalTo(unresolvedTaskManagerLocation.getResourceID())); taskSlotTable.allocateSlot(0, jobId, allocationId1, Time.milliseconds(10000L)); taskSlotTable.allocateSlot(1, jobId, allocationId2, Time.milliseconds(10000L)); @@ -874,7 +874,7 @@ public class TaskExecutorTest extends TestLogger { final Tuple3 instanceIDSlotIDAllocationIDTuple3 = availableSlotFuture.get(); - final Tuple3 expectedResult = Tuple3.of(registrationId, new SlotID(taskManagerLocation.getResourceID(), 1), allocationId2); + final Tuple3 expectedResult = Tuple3.of(registrationId, new SlotID(unresolvedTaskManagerLocation.getResourceID(), 1), allocationId2); assertThat(instanceIDSlotIDAllocationIDTuple3, equalTo(expectedResult)); @@ -893,7 +893,7 @@ public class TaskExecutorTest extends TestLogger { public void testSubmitTaskBeforeAcceptSlot() throws Exception { final TaskSlotTable taskSlotTable = TaskSlotUtils.createTaskSlotTable(2); final JobManagerTable jobManagerTable = new JobManagerTable(); - final JobLeaderService jobLeaderService = new JobLeaderService(taskManagerLocation, RetryingRegistrationConfiguration.defaultConfiguration()); + final JobLeaderService jobLeaderService = new JobLeaderService(unresolvedTaskManagerLocation, RetryingRegistrationConfiguration.defaultConfiguration()); final TestingResourceManagerGateway resourceManagerGateway = new TestingResourceManagerGateway(); resourceManagerLeaderRetriever.notifyListener(resourceManagerGateway.getAddress(), resourceManagerGateway.getFencingToken().toUUID()); @@ -934,7 +934,7 @@ public class TaskExecutorTest extends TestLogger { final TaskExecutorLocalStateStoresManager localStateStoresManager = createTaskExecutorLocalStateStoresManager(); final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() - .setTaskManagerLocation(taskManagerLocation) + .setUnresolvedTaskManagerLocation(unresolvedTaskManagerLocation) .setShuffleEnvironment(nettyShuffleEnvironment) .setTaskSlotTable(taskSlotTable) .setJobLeaderService(jobLeaderService) @@ -991,7 +991,7 @@ public class TaskExecutorTest extends TestLogger { private void requestSlots(TaskExecutorGateway tmGateway, Iterable allocationIds, ResourceManagerId resourceManagerId, String jobMasterGatewayAddress) { int slotIndex = 0; for (AllocationID allocationId : allocationIds) { - final SlotID slotId1 = new SlotID(taskManagerLocation.getResourceID(), slotIndex); + final SlotID slotId1 = new SlotID(unresolvedTaskManagerLocation.getResourceID(), slotIndex); tmGateway.requestSlot( slotId1, jobId, @@ -1022,7 +1022,7 @@ public class TaskExecutorTest extends TestLogger { final TaskExecutorLocalStateStoresManager localStateStoresManager = createTaskExecutorLocalStateStoresManager(); final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() - .setTaskManagerLocation(taskManagerLocation) + .setUnresolvedTaskManagerLocation(unresolvedTaskManagerLocation) .setJobManagerTable(jobManagerTableMock) .setJobLeaderService(jobLeaderService) .setTaskStateManager(localStateStoresManager) @@ -1084,7 +1084,7 @@ public class TaskExecutorTest extends TestLogger { final TaskExecutorLocalStateStoresManager localStateStoresManager = createTaskExecutorLocalStateStoresManager(); final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() - .setTaskManagerLocation(taskManagerLocation) + .setUnresolvedTaskManagerLocation(unresolvedTaskManagerLocation) .setTaskSlotTable(taskSlotTable) .setTaskStateManager(localStateStoresManager) .build(); @@ -1127,7 +1127,7 @@ public class TaskExecutorTest extends TestLogger { final TaskExecutorLocalStateStoresManager localStateStoresManager = createTaskExecutorLocalStateStoresManager(); final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() - .setTaskManagerLocation(taskManagerLocation) + .setUnresolvedTaskManagerLocation(unresolvedTaskManagerLocation) .setTaskSlotTable(taskSlotTable) .setTaskStateManager(localStateStoresManager) .build(); @@ -1159,7 +1159,7 @@ public class TaskExecutorTest extends TestLogger { final TaskExecutorGateway taskExecutorGateway = taskExecutor.getSelfGateway(TaskExecutorGateway.class); - final SlotID slotId = new SlotID(taskManagerLocation.getResourceID(), 0); + final SlotID slotId = new SlotID(unresolvedTaskManagerLocation.getResourceID(), 0); final AllocationID allocationId = new AllocationID(); assertThat(startFuture.isDone(), is(false)); @@ -1245,7 +1245,7 @@ public class TaskExecutorTest extends TestLogger { final ResourceID registrationResourceId = registrationFuture.get(); - assertThat(registrationResourceId, equalTo(taskManagerServices.getTaskManagerLocation().getResourceID())); + assertThat(registrationResourceId, equalTo(taskManagerServices.getUnresolvedTaskManagerLocation().getResourceID())); secondRegistration.await(); @@ -1308,10 +1308,10 @@ public class TaskExecutorTest extends TestLogger { @Test public void testReconnectionAttemptIfExplicitlyDisconnected() throws Exception { final TaskSlotTable taskSlotTable = TaskSlotUtils.createTaskSlotTable(1); - final TaskManagerLocation taskManagerLocation = new LocalTaskManagerLocation(); + final UnresolvedTaskManagerLocation unresolvedTaskManagerLocation = new LocalUnresolvedTaskManagerLocation(); final TaskExecutor taskExecutor = createTaskExecutor(new TaskManagerServicesBuilder() .setTaskSlotTable(taskSlotTable) - .setTaskManagerLocation(taskManagerLocation) + .setUnresolvedTaskManagerLocation(unresolvedTaskManagerLocation) .build()); taskExecutor.start(); @@ -1332,7 +1332,7 @@ public class TaskExecutorTest extends TestLogger { final ResourceID firstRegistrationAttempt = registrationQueue.take(); - assertThat(firstRegistrationAttempt, equalTo(taskManagerLocation.getResourceID())); + assertThat(firstRegistrationAttempt, equalTo(unresolvedTaskManagerLocation.getResourceID())); final TaskExecutorGateway taskExecutorGateway = taskExecutor.getSelfGateway(TaskExecutorGateway.class); @@ -1342,7 +1342,7 @@ public class TaskExecutorTest extends TestLogger { final ResourceID secondRegistrationAttempt = registrationQueue.take(); - assertThat(secondRegistrationAttempt, equalTo(taskManagerLocation.getResourceID())); + assertThat(secondRegistrationAttempt, equalTo(unresolvedTaskManagerLocation.getResourceID())); } finally { RpcUtils.terminateRpcEndpoint(taskExecutor, timeout); @@ -1416,10 +1416,10 @@ public class TaskExecutorTest extends TestLogger { @Test public void testInitialSlotReportFailure() throws Exception { final TaskSlotTable taskSlotTable = TaskSlotUtils.createTaskSlotTable(1); - final TaskManagerLocation taskManagerLocation = new LocalTaskManagerLocation(); + final UnresolvedTaskManagerLocation unresolvedTaskManagerLocation = new LocalUnresolvedTaskManagerLocation(); final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() .setTaskSlotTable(taskSlotTable) - .setTaskManagerLocation(taskManagerLocation) + .setUnresolvedTaskManagerLocation(unresolvedTaskManagerLocation) .build(); final TaskExecutor taskExecutor = createTaskExecutor(taskManagerServices); @@ -1579,7 +1579,7 @@ public class TaskExecutorTest extends TestLogger { initialSlotReportFuture.get(); - ResourceID resourceID = taskManagerServices.getTaskManagerLocation().getResourceID(); + ResourceID resourceID = taskManagerServices.getUnresolvedTaskManagerLocation().getResourceID(); taskExecutorGateway.requestSlot( new SlotID(resourceID, 0), jobId, @@ -1769,7 +1769,7 @@ public class TaskExecutorTest extends TestLogger { .setTaskSlotTable(new AllocateSlotNotifyingTaskSlotTable(receivedSlotRequest)) .build(); final TaskExecutor taskExecutor = createTaskExecutor(taskManagerServices); - final ResourceID taskExecutorResourceId = taskManagerServices.getTaskManagerLocation().getResourceID(); + final ResourceID taskExecutorResourceId = taskManagerServices.getUnresolvedTaskManagerLocation().getResourceID(); taskExecutor.start(); @@ -1857,10 +1857,10 @@ public class TaskExecutorTest extends TestLogger { private TaskExecutor createTaskExecutor(int numberOFSlots) { final TaskSlotTable taskSlotTable = TaskSlotUtils.createTaskSlotTable(numberOFSlots); - final TaskManagerLocation taskManagerLocation = new LocalTaskManagerLocation(); + final UnresolvedTaskManagerLocation unresolvedTaskManagerLocation = new LocalUnresolvedTaskManagerLocation(); final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() .setTaskSlotTable(taskSlotTable) - .setTaskManagerLocation(taskManagerLocation) + .setUnresolvedTaskManagerLocation(unresolvedTaskManagerLocation) .build(); configuration.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, numberOFSlots); return createTaskExecutor(taskManagerServices); @@ -1923,7 +1923,7 @@ public class TaskExecutorTest extends TestLogger { rpc.registerGateway(jobMasterGateway.getAddress(), jobMasterGateway); final JobLeaderService jobLeaderService = new JobLeaderService( - taskManagerLocation, + unresolvedTaskManagerLocation, RetryingRegistrationConfiguration.defaultConfiguration()); TaskExecutorLocalStateStoresManager stateStoresManager = createTaskExecutorLocalStateStoresManager(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java index c494ffdee1..543b26e01c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java @@ -28,9 +28,9 @@ import org.apache.flink.runtime.registration.RetryingRegistrationConfiguration; import org.apache.flink.runtime.state.TaskExecutorLocalStateStoresManager; import org.apache.flink.runtime.taskexecutor.slot.TestingTaskSlotTable; import org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable; -import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; +import org.apache.flink.runtime.taskmanager.LocalUnresolvedTaskManagerLocation; import org.apache.flink.runtime.taskmanager.Task; -import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import org.apache.flink.runtime.taskmanager.UnresolvedTaskManagerLocation; import java.util.concurrent.CompletableFuture; @@ -42,7 +42,7 @@ import static org.mockito.Mockito.mock; public class TaskManagerServicesBuilder { /** TaskManager services. */ - private TaskManagerLocation taskManagerLocation; + private UnresolvedTaskManagerLocation unresolvedTaskManagerLocation; private IOManager ioManager; private ShuffleEnvironment shuffleEnvironment; private KvStateService kvStateService; @@ -54,7 +54,7 @@ public class TaskManagerServicesBuilder { private TaskEventDispatcher taskEventDispatcher; public TaskManagerServicesBuilder() { - taskManagerLocation = new LocalTaskManagerLocation(); + unresolvedTaskManagerLocation = new LocalUnresolvedTaskManagerLocation(); ioManager = mock(IOManager.class); shuffleEnvironment = mock(ShuffleEnvironment.class); kvStateService = new KvStateService(new KvStateRegistry(), null, null); @@ -62,12 +62,12 @@ public class TaskManagerServicesBuilder { taskEventDispatcher = new TaskEventDispatcher(); taskSlotTable = TestingTaskSlotTable.newBuilder().closeAsyncReturns(CompletableFuture.completedFuture(null)).build(); jobManagerTable = new JobManagerTable(); - jobLeaderService = new JobLeaderService(taskManagerLocation, RetryingRegistrationConfiguration.defaultConfiguration()); + jobLeaderService = new JobLeaderService(unresolvedTaskManagerLocation, RetryingRegistrationConfiguration.defaultConfiguration()); taskStateManager = mock(TaskExecutorLocalStateStoresManager.class); } - public TaskManagerServicesBuilder setTaskManagerLocation(TaskManagerLocation taskManagerLocation) { - this.taskManagerLocation = taskManagerLocation; + public TaskManagerServicesBuilder setUnresolvedTaskManagerLocation(UnresolvedTaskManagerLocation unresolvedTaskManagerLocation) { + this.unresolvedTaskManagerLocation = unresolvedTaskManagerLocation; return this; } @@ -113,7 +113,7 @@ public class TaskManagerServicesBuilder { public TaskManagerServices build() { return new TaskManagerServices( - taskManagerLocation, + unresolvedTaskManagerLocation, MemoryManager.MIN_PAGE_SIZE, ioManager, shuffleEnvironment, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/LocalUnresolvedTaskManagerLocation.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/LocalUnresolvedTaskManagerLocation.java new file mode 100644 index 0000000000..24155ccb1d --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/LocalUnresolvedTaskManagerLocation.java @@ -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.flink.runtime.taskmanager; + +import org.apache.flink.runtime.clusterframework.types.ResourceID; + +/** + * Dummy local task manager unresolved location for testing purposes. + */ +public class LocalUnresolvedTaskManagerLocation extends UnresolvedTaskManagerLocation { + private static final long serialVersionUID = 1L; + + public LocalUnresolvedTaskManagerLocation() { + super(ResourceID.generate(), "localhost", 42); + } +} -- Gitee From 5c35d9d95144593193244a07b965e310707488a9 Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Mon, 2 Mar 2020 19:53:55 +0800 Subject: [PATCH 273/885] [FLINK-15911][e2e] Add e2e test for Flink over NAT. This closes #11284. --- flink-end-to-end-tests/run-nightly-tests.sh | 2 + .../container-scripts/docker-compose.nat.yml | 90 +++++++++++++++++++ .../test-scripts/test_nat.sh | 77 ++++++++++++++++ tools/travis/splits/split_container.sh | 1 + 4 files changed, 170 insertions(+) create mode 100644 flink-end-to-end-tests/test-scripts/container-scripts/docker-compose.nat.yml create mode 100755 flink-end-to-end-tests/test-scripts/test_nat.sh diff --git a/flink-end-to-end-tests/run-nightly-tests.sh b/flink-end-to-end-tests/run-nightly-tests.sh index a895740747..d6a691f44d 100755 --- a/flink-end-to-end-tests/run-nightly-tests.sh +++ b/flink-end-to-end-tests/run-nightly-tests.sh @@ -113,6 +113,8 @@ if [[ ${PROFILE} != *"jdk11"* ]]; then run_test "Run Kubernetes test" "$END_TO_END_DIR/test-scripts/test_kubernetes_embedded_job.sh" run_test "Run kubernetes session test" "$END_TO_END_DIR/test-scripts/test_kubernetes_session.sh" + run_test "Running Flink over NAT end-to-end test" "$END_TO_END_DIR/test-scripts/test_nat.sh" "skip_check_exceptions" + if [[ $PROFILE == *"include-hadoop"* ]]; then run_test "Run Mesos WordCount test" "$END_TO_END_DIR/test-scripts/test_mesos_wordcount.sh" run_test "Run Mesos multiple submission test" "$END_TO_END_DIR/test-scripts/test_mesos_multiple_submissions.sh" diff --git a/flink-end-to-end-tests/test-scripts/container-scripts/docker-compose.nat.yml b/flink-end-to-end-tests/test-scripts/container-scripts/docker-compose.nat.yml new file mode 100644 index 0000000000..943291c26d --- /dev/null +++ b/flink-end-to-end-tests/test-scripts/container-scripts/docker-compose.nat.yml @@ -0,0 +1,90 @@ +################################################################################ +# 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. +################################################################################ + +# Docker compose file for a Flink job cluster deployment with NAT network. +# +# All the network traffics are intended to go through the Docker host. NAT port mapping is simulated by publishing +# internal container ports to different external Docker host ports. A container's external hostname is simulated +# resolvable to all other containers except itself, by adding extra hosts to other containers that maps the hostname to +# the IP address of the Docker host. +# +# Parameters: +# * FLINK_DOCKER_IMAGE_NAME - Image name to use for the deployment (default: flink-job:latest) +# * FLINK_JOB - Name of the Flink job to execute (default: none) +# * FLINK_JOB_ARGUMENTS - Additional arguments which will be passed to the job cluster (default: none) +# +# * INPUT_VOLUME - Volume to be mounted for input. +# * OUTPUT_VOLUME - Volume to be mounted for output. +# * INPUT_PATH - Path inside container for the input. +# * OUTPUT_PATH - Path outside container for the input. +# +# * HOST_IP - IP address of the Docker host. This will be used for resolving JM/TM external addresses. +# * JM_EX_HOSTNAME - External hostname for JM. +# * TM_1_EX_HOSTNAME - External hostname for TM 1. +# * TM_2_EX_HOSTNAME - External hostname for TM 2. +# +# * JM_RPC_EX_PORT - External RPC port for JM. +# * JM_RPC_IN_PORT - Internal RPC port for JM. +# * TM_1_RPC_EX_PORT - External RPC port for TM 1. +# * TM_2_RPC_EX_PORT - External RPC port for TM 2. +# * TM_RPC_IN_PORT - Internal RPC port for both TMs. +# +# * TM_1_DATA_EX_PORT - External data port for TM 1. +# * TM_2_DATA_EX_PORT - External data port for TM 2. +# * TM_DATA_IN_PORT - Internal data port for both TMs. + +version: "2.2" +services: + job-cluster: + image: ${FLINK_DOCKER_IMAGE_NAME:-flink-job} + ports: + - "8081:8081" + - ${JM_RPC_EX_PORT}:${JM_RPC_IN_PORT} + volumes: + - ${INPUT_VOLUME}:${INPUT_PATH} + - ${OUTPUT_VOLUME}:${OUTPUT_PATH} + command: job-cluster --job-classname ${FLINK_JOB} -Dparallelism.default=2 -Djobmanager.rpc.address=${JM_EX_HOSTNAME} -Djobmanager.bind-host=0.0.0.0 -Djobmanager.rpc.port=${JM_RPC_EX_PORT} -Djobmanager.rpc.bind-port=${JM_RPC_IN_PORT} ${FLINK_JOB_ARGUMENTS} + extra_hosts: + - ${TM_1_EX_HOSTNAME}:${HOST_IP} + - ${TM_2_EX_HOSTNAME}:${HOST_IP} + + taskmanager1: + image: ${FLINK_DOCKER_IMAGE_NAME:-flink-job} + ports: + - ${TM_1_RPC_EX_PORT}:${TM_RPC_IN_PORT} + - ${TM_1_DATA_EX_PORT}:${TM_DATA_IN_PORT} + volumes: + - ${INPUT_VOLUME}:${INPUT_PATH} + - ${OUTPUT_VOLUME}:${OUTPUT_PATH} + command: task-manager -Djobmanager.rpc.address=${JM_EX_HOSTNAME} -Djobmanager.rpc.port=${JM_RPC_EX_PORT} -Dtaskmanager.host=${TM_1_EX_HOSTNAME} -Dtaskmanager.bind-host=0.0.0.0 -Dtaskmanager.rpc.port=${TM_1_RPC_EX_PORT} -Dtaskmanager.rpc.bind-port=${TM_RPC_IN_PORT} -Dtaskmanager.data.port=${TM_1_DATA_EX_PORT} -Dtaskmanager.data.bind-port=${TM_DATA_IN_PORT} + extra_hosts: + - ${JM_EX_HOSTNAME}:${HOST_IP} + - ${TM_2_EX_HOSTNAME}:${HOST_IP} + + taskmanager2: + image: ${FLINK_DOCKER_IMAGE_NAME:-flink-job} + ports: + - ${TM_2_RPC_EX_PORT}:${TM_RPC_IN_PORT} + - ${TM_2_DATA_EX_PORT}:${TM_DATA_IN_PORT} + volumes: + - ${INPUT_VOLUME}:${INPUT_PATH} + - ${OUTPUT_VOLUME}:${OUTPUT_PATH} + command: task-manager -Djobmanager.rpc.address=${JM_EX_HOSTNAME} -Djobmanager.rpc.port=${JM_RPC_EX_PORT} -Dtaskmanager.host=${TM_2_EX_HOSTNAME} -Dtaskmanager.bind-host=0.0.0.0 -Dtaskmanager.rpc.port=${TM_2_RPC_EX_PORT} -Dtaskmanager.rpc.bind-port=${TM_RPC_IN_PORT} -Dtaskmanager.data.port=${TM_2_DATA_EX_PORT} -Dtaskmanager.data.bind-port=${TM_DATA_IN_PORT} + extra_hosts: + - ${JM_EX_HOSTNAME}:${HOST_IP} + - ${TM_1_EX_HOSTNAME}:${HOST_IP} diff --git a/flink-end-to-end-tests/test-scripts/test_nat.sh b/flink-end-to-end-tests/test-scripts/test_nat.sh new file mode 100755 index 0000000000..ace023f809 --- /dev/null +++ b/flink-end-to-end-tests/test-scripts/test_nat.sh @@ -0,0 +1,77 @@ +#!/usr/bin/env bash +################################################################################ +# 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. +################################################################################ + +source "$(dirname "$0")"/common.sh +source "$(dirname "$0")"/common_docker.sh + +DOCKER_MODULE_DIR=${END_TO_END_DIR}/../flink-container/docker +DOCKER_SCRIPTS=${END_TO_END_DIR}/test-scripts/container-scripts +DOCKER_IMAGE_BUILD_RETRIES=3 +BUILD_BACKOFF_TIME=5 + +export FLINK_JOB=org.apache.flink.examples.java.wordcount.WordCount +export FLINK_DOCKER_IMAGE_NAME=test_nat +export INPUT_VOLUME=${END_TO_END_DIR}/test-scripts/test-data +export OUTPUT_VOLUME=${TEST_DATA_DIR}/out +export INPUT_PATH=/data/test/input +export OUTPUT_PATH=/data/test/output + +export HOST_IP=$(get_node_ip | awk '{print $1}') +export JM_EX_HOSTNAME=jm.flink.test +export TM_1_EX_HOSTNAME=tm1.flink.test +export TM_2_EX_HOSTNAME=tm2.flink.test + +export JM_RPC_EX_PORT=10000 +export JM_RPC_IN_PORT=10000 + +export TM_1_RPC_EX_PORT=10001 +export TM_2_RPC_EX_PORT=10002 +export TM_RPC_IN_PORT=10000 + +export TM_1_DATA_EX_PORT=11001 +export TM_2_DATA_EX_PORT=11002 +export TM_DATA_IN_PORT=11000 + +RESULT_HASH="72a690412be8928ba239c2da967328a5" +INPUT_ARGS="--input ${INPUT_PATH}/words" +OUTPUT_PREFIX="docker_wc_out" + +export FLINK_JOB_ARGUMENTS="${INPUT_ARGS} --output ${OUTPUT_PATH}/${OUTPUT_PREFIX}" + +build_image() { + build_image_with_jar ${FLINK_DIR}/examples/batch/WordCount.jar ${FLINK_DOCKER_IMAGE_NAME} +} + +# user inside the container must be able to create files, this is a workaround in-container permissions +mkdir -p $OUTPUT_VOLUME +chmod 777 $OUTPUT_VOLUME + +pushd "$DOCKER_MODULE_DIR" +if ! retry_times $DOCKER_IMAGE_BUILD_RETRIES ${BUILD_BACKOFF_TIME} build_image; then + echo "Failed to build docker image. Aborting..." + exit 1 +fi +popd + +docker-compose -f ${DOCKER_SCRIPTS}/docker-compose.nat.yml up --abort-on-container-exit --exit-code-from job-cluster &> /dev/null +docker-compose -f ${DOCKER_SCRIPTS}/docker-compose.nat.yml logs job-cluster > ${FLINK_DIR}/log/jobmanager.log +docker-compose -f ${DOCKER_SCRIPTS}/docker-compose.nat.yml logs taskmanager1 > ${FLINK_DIR}/log/taskmanager1.log +docker-compose -f ${DOCKER_SCRIPTS}/docker-compose.nat.yml logs taskmanager2 > ${FLINK_DIR}/log/taskmanager2.log + +check_result_hash "WordCount" ${OUTPUT_VOLUME}/${OUTPUT_PREFIX}/ "${RESULT_HASH}" diff --git a/tools/travis/splits/split_container.sh b/tools/travis/splits/split_container.sh index 4d2ba156fb..d57917245d 100755 --- a/tools/travis/splits/split_container.sh +++ b/tools/travis/splits/split_container.sh @@ -53,6 +53,7 @@ if [[ "${HADOOP_INTEGRATION}" = "with-hadoop" ]]; then run_test "Run Mesos WordCount test" "$END_TO_END_DIR/test-scripts/test_mesos_wordcount.sh" run_test "Run Mesos multiple submission test" "$END_TO_END_DIR/test-scripts/test_mesos_multiple_submissions.sh" fi +run_test "Running Flink over NAT end-to-end test" "$END_TO_END_DIR/test-scripts/test_nat.sh" "skip_check_exceptions" printf "\n[PASS] All tests passed\n" exit 0 -- Gitee From 3df0eb78ad73db1a428fc50ed3d0804e7114d68b Mon Sep 17 00:00:00 2001 From: felixzheng Date: Sat, 21 Mar 2020 15:16:24 +0800 Subject: [PATCH 274/885] [FLINK-15647][k8s] Support user-specified annotations for the JM/TM pods --- .../kubernetes_config_configuration.html | 12 +++++++++ .../KubernetesConfigOptions.java | 15 +++++++++++ .../decorators/InitJobManagerDecorator.java | 1 + .../decorators/InitTaskManagerDecorator.java | 1 + .../factory/KubernetesJobManagerFactory.java | 4 +-- .../KubernetesJobManagerParameters.java | 8 ++++++ .../parameters/KubernetesParameters.java | 7 ++++++ .../KubernetesTaskManagerParameters.java | 9 +++++++ .../InitJobManagerDecoratorTest.java | 16 ++++++++++++ .../InitTaskManagerDecoratorTest.java | 16 ++++++++++++ .../KubernetesJobManagerParametersTest.java | 21 ++++++++++++++++ .../KubernetesTaskManagerParametersTest.java | 25 ++++++++++++++++++- 12 files changed, 131 insertions(+), 4 deletions(-) diff --git a/docs/_includes/generated/kubernetes_config_configuration.html b/docs/_includes/generated/kubernetes_config_configuration.html index 6fa57f217f..3894c5780b 100644 --- a/docs/_includes/generated/kubernetes_config_configuration.html +++ b/docs/_includes/generated/kubernetes_config_configuration.html @@ -74,6 +74,12 @@ String Specify the name of an existing ConfigMap that contains custom Hadoop configuration to be mounted on the JobManager(s) and TaskManagers. + +

    kubernetes.jobmanager.annotations
    + (none) + Map + The user-specified annotations that are set to the JobManager pod. The value could be in the form of a1:v1,a2:v2 +
    kubernetes.jobmanager.cpu
    1.0 @@ -104,6 +110,12 @@ String Timeout used for creating the service. The timeout value requires a time-unit specifier (ms/s/min/h/d). + +
    kubernetes.taskmanager.annotations
    + (none) + Map + The user-specified annotations that are set to the TaskManager pod. The value could be in the form of a1:v1,a2:v2 +
    kubernetes.taskmanager.cpu
    -1.0 diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java index e91f0632d9..0006eed525 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java @@ -22,6 +22,7 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.configuration.ConfigOption; import java.util.List; +import java.util.Map; import static org.apache.flink.configuration.ConfigOptions.key; @@ -154,6 +155,20 @@ public class KubernetesConfigOptions { .withDescription("Specify the name of an existing ConfigMap that contains custom Hadoop configuration " + "to be mounted on the JobManager(s) and TaskManagers."); + public static final ConfigOption> JOB_MANAGER_ANNOTATIONS = + key("kubernetes.jobmanager.annotations") + .mapType() + .noDefaultValue() + .withDescription("The user-specified annotations that are set to the JobManager pod. The value could be " + + "in the form of a1:v1,a2:v2"); + + public static final ConfigOption> TASK_MANAGER_ANNOTATIONS = + key("kubernetes.taskmanager.annotations") + .mapType() + .noDefaultValue() + .withDescription("The user-specified annotations that are set to the TaskManager pod. The value could be " + + "in the form of a1:v1,a2:v2"); + /** * The flink rest service exposed type. */ diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecorator.java index 2836dea4d0..eac4283ea6 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecorator.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecorator.java @@ -60,6 +60,7 @@ public class InitJobManagerDecorator extends AbstractKubernetesStepDecorator { .withApiVersion(API_VERSION) .editOrNewMetadata() .withLabels(kubernetesJobManagerParameters.getLabels()) + .withAnnotations(kubernetesJobManagerParameters.getAnnotations()) .endMetadata() .editOrNewSpec() .withServiceAccountName(kubernetesJobManagerParameters.getServiceAccount()) diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecorator.java index ef118c2db3..8817339270 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecorator.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecorator.java @@ -55,6 +55,7 @@ public class InitTaskManagerDecorator extends AbstractKubernetesStepDecorator { .editOrNewMetadata() .withName(kubernetesTaskManagerParameters.getPodName()) .withLabels(kubernetesTaskManagerParameters.getLabels()) + .withAnnotations(kubernetesTaskManagerParameters.getAnnotations()) .endMetadata() .editOrNewSpec() .withImagePullSecrets(kubernetesTaskManagerParameters.getImagePullSecrets()) diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactory.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactory.java index 32977a2f5b..c7b77c8779 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactory.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactory.java @@ -94,9 +94,7 @@ public class KubernetesJobManagerFactory { .editOrNewSpec() .withReplicas(1) .editOrNewTemplate() - .editOrNewMetadata() - .withLabels(labels) - .endMetadata() + .withMetadata(resolvedPod.getMetadata()) .withSpec(resolvedPod.getSpec()) .endTemplate() .editOrNewSelector() diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java index 3af42d55fe..7fb7f7adb9 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java @@ -29,6 +29,8 @@ import org.apache.flink.kubernetes.configuration.KubernetesConfigOptionsInternal import org.apache.flink.kubernetes.utils.Constants; import org.apache.flink.kubernetes.utils.KubernetesUtils; +import javax.annotation.Nullable; + import java.util.Map; import static org.apache.flink.util.Preconditions.checkArgument; @@ -63,6 +65,12 @@ public class KubernetesJobManagerParameters extends AbstractKubernetesParameters return getPrefixedEnvironments(ResourceManagerOptions.CONTAINERIZED_MASTER_ENV_PREFIX); } + @Nullable + @Override + public Map getAnnotations() { + return flinkConfig.get(KubernetesConfigOptions.JOB_MANAGER_ANNOTATIONS); + } + public String getJobManagerMainContainerName() { return JOB_MANAGER_MAIN_CONTAINER_NAME; } diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesParameters.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesParameters.java index f18f067733..a3758beeb7 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesParameters.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesParameters.java @@ -57,6 +57,13 @@ public interface KubernetesParameters { */ Map getEnvironments(); + /** + * A map of user-specified annotations that are set to the JobManager and TaskManager pods. + * + * @return a map of annotations or null if not set. + */ + Map getAnnotations(); + /** * Directory in Pod that stores the flink-conf.yaml, log4j.properties, and the logback.xml. */ diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParameters.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParameters.java index 4fcadd0714..64ebecf71c 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParameters.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParameters.java @@ -20,9 +20,12 @@ package org.apache.flink.kubernetes.kubeclient.parameters; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; import org.apache.flink.kubernetes.utils.KubernetesUtils; import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; +import javax.annotation.Nullable; + import java.util.Map; import static org.apache.flink.util.Preconditions.checkArgument; @@ -67,6 +70,12 @@ public class KubernetesTaskManagerParameters extends AbstractKubernetesParameter return this.containeredTaskManagerParameters.taskManagerEnv(); } + @Nullable + @Override + public Map getAnnotations() { + return flinkConfig.get(KubernetesConfigOptions.TASK_MANAGER_ANNOTATIONS); + } + public String getTaskManagerMainContainerName() { return TASK_MANAGER_MAIN_CONTAINER_NAME; } diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecoratorTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecoratorTest.java index 8024792a0b..5e5a794be1 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecoratorTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecoratorTest.java @@ -41,7 +41,10 @@ import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; /** @@ -51,6 +54,12 @@ public class InitJobManagerDecoratorTest extends KubernetesJobManagerTestBase { private static final String SERVICE_ACCOUNT_NAME = "service-test"; private static final List IMAGE_PULL_SECRETS = Arrays.asList("s1", "s2", "s3"); + private static final Map ANNOTATIONS = new HashMap() { + { + put("a1", "v1"); + put("a2", "v2"); + } + }; private Pod resultPod; private Container resultMainContainer; @@ -60,6 +69,7 @@ public class InitJobManagerDecoratorTest extends KubernetesJobManagerTestBase { super.setup(); this.flinkConfig.set(KubernetesConfigOptions.JOB_MANAGER_SERVICE_ACCOUNT, SERVICE_ACCOUNT_NAME); this.flinkConfig.set(KubernetesConfigOptions.CONTAINER_IMAGE_PULL_SECRETS, IMAGE_PULL_SECRETS); + this.flinkConfig.set(KubernetesConfigOptions.JOB_MANAGER_ANNOTATIONS, ANNOTATIONS); final InitJobManagerDecorator initJobManagerDecorator = new InitJobManagerDecorator(this.kubernetesJobManagerParameters); @@ -138,6 +148,12 @@ public class InitJobManagerDecoratorTest extends KubernetesJobManagerTestBase { assertEquals(expectedLabels, this.resultPod.getMetadata().getLabels()); } + @Test + public void testPodAnnotations() { + final Map resultAnnotations = kubernetesJobManagerParameters.getAnnotations(); + assertThat(resultAnnotations, is(equalTo(ANNOTATIONS))); + } + @Test public void testPodServiceAccountName() { assertEquals(SERVICE_ACCOUNT_NAME, this.resultPod.getSpec().getServiceAccountName()); diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecoratorTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecoratorTest.java index 8ab07e008c..e69f30593b 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecoratorTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecoratorTest.java @@ -41,7 +41,10 @@ import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; /** * General tests for the {@link InitJobManagerDecorator}. @@ -49,6 +52,12 @@ import static org.junit.Assert.assertEquals; public class InitTaskManagerDecoratorTest extends KubernetesTaskManagerTestBase { private static final List IMAGE_PULL_SECRETS = Arrays.asList("s1", "s2", "s3"); + private static final Map ANNOTATIONS = new HashMap() { + { + put("a1", "v1"); + put("a2", "v2"); + } + }; private Pod resultPod; private Container resultMainContainer; @@ -57,6 +66,7 @@ public class InitTaskManagerDecoratorTest extends KubernetesTaskManagerTestBase public void setup() throws Exception { super.setup(); this.flinkConfig.set(KubernetesConfigOptions.CONTAINER_IMAGE_PULL_SECRETS, IMAGE_PULL_SECRETS); + this.flinkConfig.set(KubernetesConfigOptions.TASK_MANAGER_ANNOTATIONS, ANNOTATIONS); final InitTaskManagerDecorator initTaskManagerDecorator = new InitTaskManagerDecorator(kubernetesTaskManagerParameters); @@ -137,6 +147,12 @@ public class InitTaskManagerDecoratorTest extends KubernetesTaskManagerTestBase assertEquals(expectedLabels, this.resultPod.getMetadata().getLabels()); } + @Test + public void testPodAnnotations() { + final Map resultAnnotations = kubernetesTaskManagerParameters.getAnnotations(); + assertThat(resultAnnotations, is(equalTo(ANNOTATIONS))); + } + @Test public void testImagePullSecrets() { final List resultSecrets = this.resultPod.getSpec().getImagePullSecrets() diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParametersTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParametersTest.java index 2870c14ea9..c95e3ad651 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParametersTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParametersTest.java @@ -34,7 +34,10 @@ import java.util.HashMap; import java.util.Map; import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; @@ -71,6 +74,24 @@ public class KubernetesJobManagerParametersTest { assertEquals(expectedEnvironments, resultEnvironments); } + @Test + public void testGetNullAnnotations() { + assertNull(kubernetesJobManagerParameters.getAnnotations()); + } + + @Test + public void testGetAnnotations() { + final Map expectedAnnotations = new HashMap<>(); + expectedAnnotations.put("a1", "v1"); + expectedAnnotations.put("a2", "v2"); + + flinkConfig.set(KubernetesConfigOptions.JOB_MANAGER_ANNOTATIONS, expectedAnnotations); + + final Map resultAnnotations = kubernetesJobManagerParameters.getAnnotations(); + + assertThat(resultAnnotations, is(equalTo(expectedAnnotations))); + } + @Test public void testGetJobManagerMemoryMB() { assertEquals(JOB_MANAGER_MEMORY, kubernetesJobManagerParameters.getJobManagerMemoryMB()); diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParametersTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParametersTest.java index f3784c6caa..d11d28511a 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParametersTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParametersTest.java @@ -22,6 +22,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.MemorySize; import org.apache.flink.configuration.ResourceManagerOptions; import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec; import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils; @@ -32,7 +33,11 @@ import org.junit.Test; import java.util.HashMap; import java.util.Map; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThat; /** * General tests for the {@link KubernetesTaskManagerParameters}. @@ -53,11 +58,11 @@ public class KubernetesTaskManagerParametersTest { } }; + private Configuration flinkConfig = new Configuration(); private KubernetesTaskManagerParameters kubernetesTaskManagerParameters; @Before public void setup() { - final Configuration flinkConfig = new Configuration(); flinkConfig.set(TaskManagerOptions.CPU_CORES, TASK_MANAGER_CPU); flinkConfig.set(TaskManagerOptions.TOTAL_PROCESS_MEMORY, MemorySize.parse(TASK_MANAGER_MEMORY + "m")); flinkConfig.set(TaskManagerOptions.RPC_PORT, String.valueOf(RPC_PORT)); @@ -83,6 +88,24 @@ public class KubernetesTaskManagerParametersTest { assertEquals(customizedEnvs, kubernetesTaskManagerParameters.getEnvironments()); } + @Test + public void testGetNullAnnotations() { + assertNull(kubernetesTaskManagerParameters.getAnnotations()); + } + + @Test + public void testGetAnnotations() { + final Map expectedAnnotations = new HashMap<>(); + expectedAnnotations.put("a1", "v1"); + expectedAnnotations.put("a2", "v2"); + + flinkConfig.set(KubernetesConfigOptions.TASK_MANAGER_ANNOTATIONS, expectedAnnotations); + + final Map resultAnnotations = kubernetesTaskManagerParameters.getAnnotations(); + + assertThat(resultAnnotations, is(equalTo(expectedAnnotations))); + } + @Test public void testGetPodName() { assertEquals(POD_NAME, kubernetesTaskManagerParameters.getPodName()); -- Gitee From f6312b84e025bad6812fd62d47f4c98381f1c36a Mon Sep 17 00:00:00 2001 From: felixzheng Date: Mon, 23 Mar 2020 19:34:56 +0800 Subject: [PATCH 275/885] [FLINK-15647][k8s] Return an empty Map instead of null when annotations is not set This closes #10973. --- .../parameters/KubernetesJobManagerParameters.java | 6 ++---- .../kubeclient/parameters/KubernetesParameters.java | 2 -- .../parameters/KubernetesTaskManagerParameters.java | 6 ++---- .../parameters/KubernetesJobManagerParametersTest.java | 6 +++--- .../parameters/KubernetesTaskManagerParametersTest.java | 6 +++--- 5 files changed, 10 insertions(+), 16 deletions(-) diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java index 7fb7f7adb9..1c8d5908a8 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java @@ -29,8 +29,7 @@ import org.apache.flink.kubernetes.configuration.KubernetesConfigOptionsInternal import org.apache.flink.kubernetes.utils.Constants; import org.apache.flink.kubernetes.utils.KubernetesUtils; -import javax.annotation.Nullable; - +import java.util.Collections; import java.util.Map; import static org.apache.flink.util.Preconditions.checkArgument; @@ -65,10 +64,9 @@ public class KubernetesJobManagerParameters extends AbstractKubernetesParameters return getPrefixedEnvironments(ResourceManagerOptions.CONTAINERIZED_MASTER_ENV_PREFIX); } - @Nullable @Override public Map getAnnotations() { - return flinkConfig.get(KubernetesConfigOptions.JOB_MANAGER_ANNOTATIONS); + return flinkConfig.getOptional(KubernetesConfigOptions.JOB_MANAGER_ANNOTATIONS).orElse(Collections.emptyMap()); } public String getJobManagerMainContainerName() { diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesParameters.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesParameters.java index a3758beeb7..45069fbce0 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesParameters.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesParameters.java @@ -59,8 +59,6 @@ public interface KubernetesParameters { /** * A map of user-specified annotations that are set to the JobManager and TaskManager pods. - * - * @return a map of annotations or null if not set. */ Map getAnnotations(); diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParameters.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParameters.java index 64ebecf71c..dbfddcc606 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParameters.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParameters.java @@ -24,8 +24,7 @@ import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; import org.apache.flink.kubernetes.utils.KubernetesUtils; import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; -import javax.annotation.Nullable; - +import java.util.Collections; import java.util.Map; import static org.apache.flink.util.Preconditions.checkArgument; @@ -70,10 +69,9 @@ public class KubernetesTaskManagerParameters extends AbstractKubernetesParameter return this.containeredTaskManagerParameters.taskManagerEnv(); } - @Nullable @Override public Map getAnnotations() { - return flinkConfig.get(KubernetesConfigOptions.TASK_MANAGER_ANNOTATIONS); + return flinkConfig.getOptional(KubernetesConfigOptions.TASK_MANAGER_ANNOTATIONS).orElse(Collections.emptyMap()); } public String getTaskManagerMainContainerName() { diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParametersTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParametersTest.java index c95e3ad651..81fba4b7b1 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParametersTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParametersTest.java @@ -37,8 +37,8 @@ import static org.hamcrest.CoreMatchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; /** @@ -75,8 +75,8 @@ public class KubernetesJobManagerParametersTest { } @Test - public void testGetNullAnnotations() { - assertNull(kubernetesJobManagerParameters.getAnnotations()); + public void testGetEmptyAnnotations() { + assertTrue(kubernetesJobManagerParameters.getAnnotations().isEmpty()); } @Test diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParametersTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParametersTest.java index d11d28511a..a9033b5640 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParametersTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParametersTest.java @@ -36,8 +36,8 @@ import java.util.Map; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; /** * General tests for the {@link KubernetesTaskManagerParameters}. @@ -89,8 +89,8 @@ public class KubernetesTaskManagerParametersTest { } @Test - public void testGetNullAnnotations() { - assertNull(kubernetesTaskManagerParameters.getAnnotations()); + public void testGetEmptyAnnotations() { + assertTrue(kubernetesTaskManagerParameters.getAnnotations().isEmpty()); } @Test -- Gitee From 1827e4dddfbac75a533ff2aea2f3e690777a3e5e Mon Sep 17 00:00:00 2001 From: Leonard Xu Date: Wed, 25 Mar 2020 10:13:41 +0800 Subject: [PATCH 276/885] [FLINK-16170][elasticsearch] Fix SearchTemplateRequest ClassNotFoundException when using flink-sql-connector-elasticsearch7 We shouldn't `exclude org.elasticsearch:elasticsearch-geo` and `org.elasticsearch.plugin:lang-mustache-client` when shading. This closes #11396 --- .../pom.xml | 16 ++--- flink-connectors/pom.xml | 1 + .../flink-sql-client-test/pom.xml | 13 ++++ flink-end-to-end-tests/run-nightly-tests.sh | 6 +- .../test-scripts/elasticsearch-common.sh | 2 +- .../test-scripts/test_sql_client.sh | 61 +++++++++++++------ tools/travis/splits/split_misc.sh | 6 +- 7 files changed, 74 insertions(+), 31 deletions(-) diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml index 0f2da3f87b..d1e289dca6 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml @@ -63,14 +63,10 @@ under the License. - com.carrotsearch:hppc com.tdunning:t-digest joda-time:joda-time net.sf.jopt-simple:jopt-simple org.elasticsearch:jna - org.elasticsearch:elasticsearch-geo - org.elasticsearch.plugin:lang-mustache-client - com.github.spullara.mustache.java:compiler org.hdrhistogram:HdrHistogram org.yaml:snakeyaml @@ -135,14 +131,18 @@ under the License. org.elasticsearch org.apache.flink.elasticsearch7.shaded.org.elasticsearch - - org.apache.logging - org.apache.flink.elasticsearch7.shaded.org.apache.logging - com.fasterxml.jackson org.apache.flink.elasticsearch7.shaded.com.fasterxml.jackson + + com.carrotsearch.hppc + org.apache.flink.elasticsearch7.shaded.com.carrotsearch.hppc + + + com.github.mustachejava + org.apache.flink.elasticsearch7.shaded.com.github.mustachejava + diff --git a/flink-connectors/pom.xml b/flink-connectors/pom.xml index 1e92a10541..296bf899e4 100644 --- a/flink-connectors/pom.xml +++ b/flink-connectors/pom.xml @@ -94,6 +94,7 @@ under the License. flink-sql-connector-elasticsearch6 + flink-sql-connector-elasticsearch7 flink-sql-connector-kafka-0.10 flink-sql-connector-kafka-0.11 flink-sql-connector-kafka diff --git a/flink-end-to-end-tests/flink-sql-client-test/pom.xml b/flink-end-to-end-tests/flink-sql-client-test/pom.xml index c321c459ed..69dd2b2db4 100644 --- a/flink-end-to-end-tests/flink-sql-client-test/pom.xml +++ b/flink-end-to-end-tests/flink-sql-client-test/pom.xml @@ -96,6 +96,13 @@ under the License. ${project.version} provided + + + org.apache.flink + flink-sql-connector-elasticsearch7_${scala.binary.version} + ${project.version} + provided + @@ -191,6 +198,12 @@ under the License. ${project.version} jar + + org.apache.flink + flink-sql-connector-elasticsearch7_${scala.binary.version} + ${project.version} + jar + diff --git a/flink-end-to-end-tests/run-nightly-tests.sh b/flink-end-to-end-tests/run-nightly-tests.sh index d6a691f44d..fc854b6dfe 100755 --- a/flink-end-to-end-tests/run-nightly-tests.sh +++ b/flink-end-to-end-tests/run-nightly-tests.sh @@ -177,8 +177,10 @@ fi run_test "State TTL Heap backend end-to-end test" "$END_TO_END_DIR/test-scripts/test_stream_state_ttl.sh file" "skip_check_exceptions" run_test "State TTL RocksDb backend end-to-end test" "$END_TO_END_DIR/test-scripts/test_stream_state_ttl.sh rocks" "skip_check_exceptions" -run_test "SQL Client end-to-end test (Old planner)" "$END_TO_END_DIR/test-scripts/test_sql_client.sh old" -run_test "SQL Client end-to-end test (Blink planner)" "$END_TO_END_DIR/test-scripts/test_sql_client.sh blink" +run_test "SQL Client end-to-end test (Old planner) Elasticsearch (v6.3.1)" "$END_TO_END_DIR/test-scripts/test_sql_client.sh old 6" +run_test "SQL Client end-to-end test (Old planner) Elasticsearch (v7.5.1)" "$END_TO_END_DIR/test-scripts/test_sql_client.sh old 7" +run_test "SQL Client end-to-end test (Blink planner) Elasticsearch (v6.3.1)" "$END_TO_END_DIR/test-scripts/test_sql_client.sh blink 6" +run_test "SQL Client end-to-end test (Blink planner) Elasticsearch (v7.5.1)" "$END_TO_END_DIR/test-scripts/test_sql_client.sh blink 7" run_test "TPC-H end-to-end test (Blink planner)" "$END_TO_END_DIR/test-scripts/test_tpch.sh" run_test "TPC-DS end-to-end test (Blink planner)" "$END_TO_END_DIR/test-scripts/test_tpcds.sh" diff --git a/flink-end-to-end-tests/test-scripts/elasticsearch-common.sh b/flink-end-to-end-tests/test-scripts/elasticsearch-common.sh index 0ef278abf9..8a2afcb4ff 100644 --- a/flink-end-to-end-tests/test-scripts/elasticsearch-common.sh +++ b/flink-end-to-end-tests/test-scripts/elasticsearch-common.sh @@ -74,7 +74,7 @@ function verify_result_line_number { while : ; do curl "localhost:9200/${index}/_search?q=*&pretty&size=21" > $TEST_DATA_DIR/output || true - if [ -n "$(grep "\"total\" : $numRecords" $TEST_DATA_DIR/output)" ]; then + if [ -n "$(grep "\"total\" : $numRecords" $TEST_DATA_DIR/output)" ] || [ -n "$(grep "\"value\" : $numRecords" $TEST_DATA_DIR/output)" ]; then echo "Elasticsearch end to end test pass." break else diff --git a/flink-end-to-end-tests/test-scripts/test_sql_client.sh b/flink-end-to-end-tests/test-scripts/test_sql_client.sh index 8eec6eaf6e..54209249b0 100755 --- a/flink-end-to-end-tests/test-scripts/test_sql_client.sh +++ b/flink-end-to-end-tests/test-scripts/test_sql_client.sh @@ -20,12 +20,17 @@ set -Eeuo pipefail PLANNER="${1:-old}" +ELASTICSEARCH_VERSION=${2:-6} KAFKA_VERSION="2.2.0" CONFLUENT_VERSION="5.0.0" CONFLUENT_MAJOR_VERSION="5.0" KAFKA_SQL_VERSION="universal" +ELASTICSEARCH6_DOWNLOAD_URL='https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-6.3.1.tar.gz' +ELASTICSEARCH7_MAC_DOWNLOAD_URL='https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-7.5.1-darwin-x86_64.tar.gz' +ELASTICSEARCH7_LINUX_DOWNLOAD_URL='https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-7.5.1-linux-x86_64.tar.gz' + source "$(dirname "$0")"/common.sh source "$(dirname "$0")"/kafka_sql_common.sh \ $KAFKA_VERSION \ @@ -85,6 +90,30 @@ function sql_cleanup() { } on_exit sql_cleanup +function prepare_elasticsearch { + echo "Preparing Elasticsearch(version=$ELASTICSEARCH_VERSION)..." + # elastcisearch offers different release binary file for corresponding system since version 7. + case "$(uname -s)" in + Linux*) OS_TYPE=linux;; + Darwin*) OS_TYPE=mac;; + *) OS_TYPE="UNKNOWN:${unameOut}" + esac + + if [[ "$ELASTICSEARCH_VERSION" == 6 ]]; then + DOWNLOAD_URL=$ELASTICSEARCH6_DOWNLOAD_URL + elif [[ "$ELASTICSEARCH_VERSION" == 7 ]] && [[ "$OS_TYPE" == "mac" ]]; then + DOWNLOAD_URL=$ELASTICSEARCH7_MAC_DOWNLOAD_URL + elif [[ "$ELASTICSEARCH_VERSION" == 7 ]] && [[ "$OS_TYPE" == "linux" ]]; then + DOWNLOAD_URL=$ELASTICSEARCH7_LINUX_DOWNLOAD_URL + else + echo "[ERROR] Unsupported elasticsearch version($ELASTICSEARCH_VERSION) for OS: $OS_TYPE" + exit 1 + fi + + setup_elasticsearch $DOWNLOAD_URL + wait_elasticsearch_working +} + # prepare Kafka echo "Preparing Kafka..." @@ -96,13 +125,7 @@ create_kafka_json_source test-json create_kafka_topic 1 1 test-avro # prepare Elasticsearch -echo "Preparing Elasticsearch..." - -ELASTICSEARCH_VERSION=6 -DOWNLOAD_URL='https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-6.3.1.tar.gz' - -setup_elasticsearch $DOWNLOAD_URL $ELASTICSEARCH_VERSION -wait_elasticsearch_working +prepare_elasticsearch ################################################################################ # Prepare Flink @@ -121,7 +144,7 @@ echo "Testing SQL statements..." JSON_SQL_JAR=$(find "$SQL_JARS_DIR" | grep "json" ) KAFKA_SQL_JAR=$(find "$SQL_JARS_DIR" | grep "kafka_" ) -ELASTICSEARCH_SQL_JAR=$(find "$SQL_JARS_DIR" | grep "elasticsearch6" ) +ELASTICSEARCH_SQL_JAR=$(find "$SQL_JARS_DIR" | grep "elasticsearch$ELASTICSEARCH_VERSION" ) # create session environment file RESULT=$TEST_DATA_DIR/result @@ -146,7 +169,7 @@ cat >> $SQL_CONF << EOF data-type: BIGINT connector: type: elasticsearch - version: 6 + version: "$ELASTICSEARCH_VERSION" hosts: "http://localhost:9200" index: "$ELASTICSEARCH_INDEX" document-type: "user" @@ -167,7 +190,7 @@ cat >> $SQL_CONF << EOF data-type: BIGINT connector: type: elasticsearch - version: 6 + version: "$ELASTICSEARCH_VERSION" hosts: "http://localhost:9200" index: "$ELASTICSEARCH_INDEX" document-type: "user" @@ -190,7 +213,7 @@ EOF echo "Executing SQL: Values -> Elasticsearch (upsert)" -SQL_STATEMENT_3=$(cat << EOF +SQL_STATEMENT_1=$(cat << EOF INSERT INTO ElasticsearchUpsertSinkTable SELECT user_id, user_name, COUNT(*) AS user_count FROM (VALUES (1, 'Bob'), (22, 'Tom'), (42, 'Kim'), (42, 'Kim'), (42, 'Kim'), (1, 'Bob')) @@ -200,18 +223,20 @@ EOF ) JOB_ID=$($FLINK_DIR/bin/sql-client.sh embedded \ - --library $SQL_JARS_DIR \ + --jar $KAFKA_SQL_JAR \ + --jar $JSON_SQL_JAR \ + --jar $ELASTICSEARCH_SQL_JAR \ --jar $SQL_TOOLBOX_JAR \ --environment $SQL_CONF \ - --update "$SQL_STATEMENT_3" | grep "Job ID:" | sed 's/.* //g') + --update "$SQL_STATEMENT_1" | grep "Job ID:" | sed 's/.* //g') wait_job_terminal_state "$JOB_ID" "FINISHED" -verify_result_hash "SQL Client Elasticsearch Upsert" "$ELASTICSEARCH_INDEX" 3 "21a76360e2a40f442816d940e7071ccf" +verify_result_line_number 3 "$ELASTICSEARCH_INDEX" echo "Executing SQL: Values -> Elasticsearch (append, no key)" -SQL_STATEMENT_4=$(cat << EOF +SQL_STATEMENT_2=$(cat << EOF INSERT INTO ElasticsearchAppendSinkTable SELECT * FROM ( @@ -232,7 +257,7 @@ JOB_ID=$($FLINK_DIR/bin/sql-client.sh embedded \ --jar $ELASTICSEARCH_SQL_JAR \ --jar $SQL_TOOLBOX_JAR \ --environment $SQL_CONF \ - --update "$SQL_STATEMENT_4" | grep "Job ID:" | sed 's/.* //g') + --update "$SQL_STATEMENT_2" | grep "Job ID:" | sed 's/.* //g') wait_job_terminal_state "$JOB_ID" "FINISHED" @@ -241,7 +266,7 @@ verify_result_line_number 9 "$ELASTICSEARCH_INDEX" echo "Executing SQL: Match recognize -> Elasticsearch" -SQL_STATEMENT_5=$(cat << EOF +SQL_STATEMENT_3=$(cat << EOF INSERT INTO ElasticsearchAppendSinkTable SELECT 1 as user_id, T.userName as user_name, cast(1 as BIGINT) as user_count FROM ( @@ -265,7 +290,7 @@ JOB_ID=$($FLINK_DIR/bin/sql-client.sh embedded \ --jar $ELASTICSEARCH_SQL_JAR \ --jar $SQL_TOOLBOX_JAR \ --environment $SQL_CONF \ - --update "$SQL_STATEMENT_5" | grep "Job ID:" | sed 's/.* //g') + --update "$SQL_STATEMENT_3" | grep "Job ID:" | sed 's/.* //g') # 3 upsert results and 6 append results and 3 match_recognize results verify_result_line_number 12 "$ELASTICSEARCH_INDEX" diff --git a/tools/travis/splits/split_misc.sh b/tools/travis/splits/split_misc.sh index dfc2494965..c671a82c3a 100755 --- a/tools/travis/splits/split_misc.sh +++ b/tools/travis/splits/split_misc.sh @@ -72,8 +72,10 @@ fi run_test "State TTL Heap backend end-to-end test" "$END_TO_END_DIR/test-scripts/test_stream_state_ttl.sh file" "skip_check_exceptions" run_test "State TTL RocksDb backend end-to-end test" "$END_TO_END_DIR/test-scripts/test_stream_state_ttl.sh rocks" "skip_check_exceptions" -run_test "SQL Client end-to-end test (Old planner)" "$END_TO_END_DIR/test-scripts/test_sql_client.sh old" -run_test "SQL Client end-to-end test (Blink planner)" "$END_TO_END_DIR/test-scripts/test_sql_client.sh blink" +run_test "SQL Client end-to-end test (Old planner) Elasticsearch (v6.3.1)" "$END_TO_END_DIR/test-scripts/test_sql_client.sh old 6" +run_test "SQL Client end-to-end test (Old planner) Elasticsearch (v7.5.1)" "$END_TO_END_DIR/test-scripts/test_sql_client.sh old 7" +run_test "SQL Client end-to-end test (Blink planner) Elasticsearch (v6.3.1)" "$END_TO_END_DIR/test-scripts/test_sql_client.sh blink 6" +run_test "SQL Client end-to-end test (Blink planner) Elasticsearch (v7.5.1)" "$END_TO_END_DIR/test-scripts/test_sql_client.sh blink 7" run_test "Dependency shading of table modules test" "$END_TO_END_DIR/test-scripts/test_table_shaded_dependencies.sh" -- Gitee From 56424794c291b9115080005220a17963048c3621 Mon Sep 17 00:00:00 2001 From: zhengshuli Date: Mon, 10 Feb 2020 21:11:53 +0800 Subject: [PATCH 277/885] [FLINK-15579][table-planner-blink] UpsertStreamTableSink should work on batch mode --- .../io/jdbc/JDBCUpsertTableSinkITCase.java | 90 +++++++++++++++++++ .../nodes/physical/batch/BatchExecSink.scala | 45 +++++++--- .../plan/utils/UpdatingPlanChecker.scala | 5 +- 3 files changed, 127 insertions(+), 13 deletions(-) diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSinkITCase.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSinkITCase.java index a9848b3f1e..e8b3b79d63 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSinkITCase.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSinkITCase.java @@ -18,13 +18,24 @@ package org.apache.flink.api.java.io.jdbc; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.io.InputFormat; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.io.CollectionInputFormat; import org.apache.flink.api.java.tuple.Tuple4; +import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.TimeCharacteristic; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.timestamps.AscendingTimestampExtractor; +import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.java.StreamTableEnvironment; +import org.apache.flink.table.sources.InputFormatTableSource; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.utils.TypeConversions; import org.apache.flink.test.util.AbstractTestBase; import org.apache.flink.types.Row; @@ -38,6 +49,7 @@ import java.sql.SQLException; import java.sql.Statement; import java.sql.Timestamp; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.List; @@ -51,6 +63,7 @@ public class JDBCUpsertTableSinkITCase extends AbstractTestBase { public static final String DB_URL = "jdbc:derby:memory:upsert"; public static final String OUTPUT_TABLE1 = "upsertSink"; public static final String OUTPUT_TABLE2 = "appendSink"; + public static final String OUTPUT_TABLE3 = "batchSink"; @Before public void before() throws ClassNotFoundException, SQLException { @@ -72,6 +85,10 @@ public class JDBCUpsertTableSinkITCase extends AbstractTestBase { "num BIGINT NOT NULL DEFAULT 0," + "ts TIMESTAMP)"); + stat.executeUpdate("CREATE TABLE " + OUTPUT_TABLE3 + " (" + + "NAME VARCHAR(20) NOT NULL," + + "SCORE BIGINT NOT NULL DEFAULT 0)"); + stat.executeUpdate("CREATE TABLE REAL_TABLE (real_data REAL)"); } } @@ -84,6 +101,7 @@ public class JDBCUpsertTableSinkITCase extends AbstractTestBase { Statement stat = conn.createStatement()) { stat.execute("DROP TABLE " + OUTPUT_TABLE1); stat.execute("DROP TABLE " + OUTPUT_TABLE2); + stat.execute("DROP TABLE " + OUTPUT_TABLE3); stat.execute("DROP TABLE REAL_TABLE"); } } @@ -210,4 +228,76 @@ public class JDBCUpsertTableSinkITCase extends AbstractTestBase { Row.of(20, 6, Timestamp.valueOf("1970-01-01 00:00:00.02")) }, DB_URL, OUTPUT_TABLE2, new String[]{"id", "num", "ts"}); } + + @Test + public void testBatchUpsert() throws Exception { + StreamExecutionEnvironment bsEnv = StreamExecutionEnvironment.getExecutionEnvironment(); + EnvironmentSettings bsSettings = EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build(); + StreamTableEnvironment bsTableEnv = StreamTableEnvironment.create(bsEnv, bsSettings); + RowTypeInfo rt = (RowTypeInfo) Types.ROW_NAMED(new String[]{"NAME", "SCORE"}, Types.STRING, Types.LONG); + Table source = bsTableEnv.fromTableSource(new CollectionTableSource(generateRecords(2), rt)); + bsTableEnv.registerTable("sourceTable", source); + bsTableEnv.sqlUpdate( + "CREATE TABLE USER_RESULT(" + + "NAME VARCHAR," + + "SCORE BIGINT" + + ") WITH ( " + + "'connector.type' = 'jdbc'," + + "'connector.url'='" + DB_URL + "'," + + "'connector.table' = '" + OUTPUT_TABLE3 + "'" + + ")"); + + bsTableEnv.sqlUpdate("insert into USER_RESULT SELECT s.NAME, s.SCORE " + + "FROM sourceTable as s "); + bsTableEnv.execute("test"); + + check(new Row[] { + Row.of("a0", 0L), + Row.of("a1", 1L) + }, DB_URL, OUTPUT_TABLE3, new String[]{"NAME", "SCORE"}); + } + + private List generateRecords(int numRecords) { + int arity = 2; + List res = new ArrayList<>(numRecords); + for (long i = 0; i < numRecords; i++) { + Row row = new Row(arity); + row.setField(0, "a" + i); + row.setField(1, i); + res.add(row); + } + return res; + } + + private static class CollectionTableSource extends InputFormatTableSource { + + private final Collection data; + private final RowTypeInfo rowTypeInfo; + + CollectionTableSource(Collection data, RowTypeInfo rowTypeInfo) { + this.data = data; + this.rowTypeInfo = rowTypeInfo; + } + + @Override + public DataType getProducedDataType() { + return TypeConversions.fromLegacyInfoToDataType(rowTypeInfo); + } + + @Override + public TypeInformation getReturnType() { + return rowTypeInfo; + } + + @Override + public InputFormat getInputFormat() { + return new CollectionInputFormat<>(data, rowTypeInfo.createSerializer(new ExecutionConfig())); + } + + @Override + public TableSchema getTableSchema() { + return new TableSchema.Builder().fields(rowTypeInfo.getFieldNames(), + TypeConversions.fromLegacyInfoToDataType(rowTypeInfo.getFieldTypes())).build(); + } + } } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecSink.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecSink.scala index aeeba515d8..1d6c46ea90 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecSink.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecSink.scala @@ -31,15 +31,15 @@ import org.apache.flink.table.planner.plan.nodes.exec.{BatchExecNode, ExecNode} import org.apache.flink.table.planner.sinks.DataStreamTableSink import org.apache.flink.table.runtime.types.ClassLogicalTypeConverter import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo -import org.apache.flink.table.sinks.{RetractStreamTableSink, StreamTableSink, TableSink, UpsertStreamTableSink} +import org.apache.flink.table.sinks.{AppendStreamTableSink, RetractStreamTableSink, StreamTableSink, TableSink, UpsertStreamTableSink} import org.apache.flink.table.types.DataType - import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode - import java.lang.reflect.Modifier import java.util +import org.apache.flink.table.planner.plan.utils.UpdatingPlanChecker + import scala.collection.JavaConversions._ /** @@ -81,13 +81,39 @@ class BatchExecSink[T]( override protected def translateToPlanInternal( planner: BatchPlanner): Transformation[Any] = { val resultTransformation = sink match { - case _: RetractStreamTableSink[T] | _: UpsertStreamTableSink[T] => - throw new TableException("RetractStreamTableSink and UpsertStreamTableSink is not" + - " supported in Batch environment.") - case streamTableSink: StreamTableSink[T] => - // we can insert the bounded DataStream into a StreamTableSink - val transformation = translateToTransformation(withChangeFlag = false, planner) + val transformation = streamTableSink match { + case _: RetractStreamTableSink[T] => + translateToTransformation(withChangeFlag = true, planner) + + case upsertSink: UpsertStreamTableSink[T] => + // check for append only table + val isAppendOnlyTable = UpdatingPlanChecker.isAppendOnly(this) + upsertSink.setIsAppendOnly(isAppendOnlyTable) + val tableKeys = { + val sinkFieldNames = upsertSink.getTableSchema.getFieldNames + UpdatingPlanChecker.getUniqueKeyFields(getInput, planner, sinkFieldNames) match { + case Some(keys) => keys.sortBy(_.length).headOption + case None => None + } + } + + // check that we have keys if the table has changes (is not append-only) + tableKeys match { + case Some(keys) => upsertSink.setKeyFields(keys) + case None if isAppendOnlyTable => upsertSink.setKeyFields(null) + case None if !isAppendOnlyTable => throw new TableException( + "UpsertStreamTableSink requires that Table has" + + " a full primary keys if it is updated.") + } + + translateToTransformation(withChangeFlag = true, planner) + case _: AppendStreamTableSink[T] => + // we can insert the bounded DataStream into a StreamTableSink + translateToTransformation(withChangeFlag = false, planner) + case _ => + translateToTransformation(withChangeFlag = false, planner) + } val boundedStream = new DataStream(planner.getExecEnv, transformation) val dsSink = streamTableSink.consumeDataStream(boundedStream) if (dsSink == null) { @@ -96,7 +122,6 @@ class BatchExecSink[T]( s"However, ${sink.getClass.getCanonicalName} doesn't implement this method.") } dsSink.getTransformation - case dsTableSink: DataStreamTableSink[T] => // In case of table to bounded stream through Batchplannerironment#toBoundedStream, we // insert a DataStreamTableSink then wrap it as a LogicalSink, there is no real batch table diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/UpdatingPlanChecker.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/UpdatingPlanChecker.scala index 9fc9e44ba9..a2e78aba3e 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/UpdatingPlanChecker.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/UpdatingPlanChecker.scala @@ -17,10 +17,9 @@ */ package org.apache.flink.table.planner.plan.utils -import org.apache.flink.table.planner.delegation.StreamPlanner +import org.apache.flink.table.planner.delegation.PlannerBase import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery import org.apache.flink.table.planner.plan.nodes.physical.stream._ - import org.apache.calcite.plan.hep.HepRelVertex import org.apache.calcite.plan.volcano.RelSubset import org.apache.calcite.rel.{RelNode, RelVisitor} @@ -40,7 +39,7 @@ object UpdatingPlanChecker { /** Extracts the unique keys of the table produced by the plan. */ def getUniqueKeyFields( relNode: RelNode, - planner: StreamPlanner, + planner: PlannerBase, sinkFieldNames: Array[String]): Option[Array[Array[String]]] = { val fmq = FlinkRelMetadataQuery.reuseOrCreate(planner.getRelBuilder.getCluster.getMetadataQuery) val uniqueKeys = fmq.getUniqueKeys(relNode) -- Gitee From d38a010c55ad78f4e421d581ec72a96a79324dfe Mon Sep 17 00:00:00 2001 From: JingsongLi Date: Mon, 23 Mar 2020 18:54:47 +0800 Subject: [PATCH 278/885] [FLINK-15579][table-planner-blink] Fix UpsertStreamTableSink support and add tests --- .../io/jdbc/JDBCUpsertTableSinkITCase.java | 86 ++++-------------- .../nodes/physical/batch/BatchExecSink.scala | 34 ++------ .../physical/stream/StreamExecSink.scala | 13 +-- .../plan/utils/UpdatingPlanChecker.scala | 41 ++++++--- .../runtime/batch/table/TableSinkITCase.scala | 87 ++++++++++++++++++- 5 files changed, 140 insertions(+), 121 deletions(-) diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSinkITCase.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSinkITCase.java index e8b3b79d63..ecba1977fc 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSinkITCase.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSinkITCase.java @@ -18,24 +18,15 @@ package org.apache.flink.api.java.io.jdbc; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.io.InputFormat; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.api.java.io.CollectionInputFormat; import org.apache.flink.api.java.tuple.Tuple4; -import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.TimeCharacteristic; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.timestamps.AscendingTimestampExtractor; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.Table; -import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.java.StreamTableEnvironment; -import org.apache.flink.table.sources.InputFormatTableSource; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.utils.TypeConversions; import org.apache.flink.test.util.AbstractTestBase; import org.apache.flink.types.Row; @@ -49,7 +40,6 @@ import java.sql.SQLException; import java.sql.Statement; import java.sql.Timestamp; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.List; @@ -230,14 +220,12 @@ public class JDBCUpsertTableSinkITCase extends AbstractTestBase { } @Test - public void testBatchUpsert() throws Exception { - StreamExecutionEnvironment bsEnv = StreamExecutionEnvironment.getExecutionEnvironment(); - EnvironmentSettings bsSettings = EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build(); - StreamTableEnvironment bsTableEnv = StreamTableEnvironment.create(bsEnv, bsSettings); - RowTypeInfo rt = (RowTypeInfo) Types.ROW_NAMED(new String[]{"NAME", "SCORE"}, Types.STRING, Types.LONG); - Table source = bsTableEnv.fromTableSource(new CollectionTableSource(generateRecords(2), rt)); - bsTableEnv.registerTable("sourceTable", source); - bsTableEnv.sqlUpdate( + public void testBatchSink() throws Exception { + EnvironmentSettings bsSettings = EnvironmentSettings.newInstance() + .useBlinkPlanner().inBatchMode().build(); + TableEnvironment tEnv = TableEnvironment.create(bsSettings); + + tEnv.sqlUpdate( "CREATE TABLE USER_RESULT(" + "NAME VARCHAR," + "SCORE BIGINT" + @@ -247,57 +235,19 @@ public class JDBCUpsertTableSinkITCase extends AbstractTestBase { "'connector.table' = '" + OUTPUT_TABLE3 + "'" + ")"); - bsTableEnv.sqlUpdate("insert into USER_RESULT SELECT s.NAME, s.SCORE " + - "FROM sourceTable as s "); - bsTableEnv.execute("test"); + tEnv.sqlUpdate("INSERT INTO USER_RESULT\n" + + "SELECT user_name, score " + + "FROM (VALUES (1, 'Bob'), (22, 'Tom'), (42, 'Kim'), " + + "(42, 'Kim'), (1, 'Bob')) " + + "AS UserCountTable(score, user_name)"); + tEnv.execute("test"); check(new Row[] { - Row.of("a0", 0L), - Row.of("a1", 1L) + Row.of("Bob", 1), + Row.of("Tom", 22), + Row.of("Kim", 42), + Row.of("Kim", 42), + Row.of("Bob", 1) }, DB_URL, OUTPUT_TABLE3, new String[]{"NAME", "SCORE"}); } - - private List generateRecords(int numRecords) { - int arity = 2; - List res = new ArrayList<>(numRecords); - for (long i = 0; i < numRecords; i++) { - Row row = new Row(arity); - row.setField(0, "a" + i); - row.setField(1, i); - res.add(row); - } - return res; - } - - private static class CollectionTableSource extends InputFormatTableSource { - - private final Collection data; - private final RowTypeInfo rowTypeInfo; - - CollectionTableSource(Collection data, RowTypeInfo rowTypeInfo) { - this.data = data; - this.rowTypeInfo = rowTypeInfo; - } - - @Override - public DataType getProducedDataType() { - return TypeConversions.fromLegacyInfoToDataType(rowTypeInfo); - } - - @Override - public TypeInformation getReturnType() { - return rowTypeInfo; - } - - @Override - public InputFormat getInputFormat() { - return new CollectionInputFormat<>(data, rowTypeInfo.createSerializer(new ExecutionConfig())); - } - - @Override - public TableSchema getTableSchema() { - return new TableSchema.Builder().fields(rowTypeInfo.getFieldNames(), - TypeConversions.fromLegacyInfoToDataType(rowTypeInfo.getFieldTypes())).build(); - } - } } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecSink.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecSink.scala index 1d6c46ea90..6049882f78 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecSink.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecSink.scala @@ -28,18 +28,19 @@ import org.apache.flink.table.planner.codegen.{CodeGenUtils, CodeGeneratorContex import org.apache.flink.table.planner.delegation.BatchPlanner import org.apache.flink.table.planner.plan.nodes.calcite.Sink import org.apache.flink.table.planner.plan.nodes.exec.{BatchExecNode, ExecNode} +import org.apache.flink.table.planner.plan.utils.UpdatingPlanChecker import org.apache.flink.table.planner.sinks.DataStreamTableSink import org.apache.flink.table.runtime.types.ClassLogicalTypeConverter import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo -import org.apache.flink.table.sinks.{AppendStreamTableSink, RetractStreamTableSink, StreamTableSink, TableSink, UpsertStreamTableSink} +import org.apache.flink.table.sinks.{RetractStreamTableSink, StreamTableSink, TableSink, UpsertStreamTableSink} import org.apache.flink.table.types.DataType + import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode + import java.lang.reflect.Modifier import java.util -import org.apache.flink.table.planner.plan.utils.UpdatingPlanChecker - import scala.collection.JavaConversions._ /** @@ -87,30 +88,11 @@ class BatchExecSink[T]( translateToTransformation(withChangeFlag = true, planner) case upsertSink: UpsertStreamTableSink[T] => - // check for append only table - val isAppendOnlyTable = UpdatingPlanChecker.isAppendOnly(this) - upsertSink.setIsAppendOnly(isAppendOnlyTable) - val tableKeys = { - val sinkFieldNames = upsertSink.getTableSchema.getFieldNames - UpdatingPlanChecker.getUniqueKeyFields(getInput, planner, sinkFieldNames) match { - case Some(keys) => keys.sortBy(_.length).headOption - case None => None - } - } - - // check that we have keys if the table has changes (is not append-only) - tableKeys match { - case Some(keys) => upsertSink.setKeyFields(keys) - case None if isAppendOnlyTable => upsertSink.setKeyFields(null) - case None if !isAppendOnlyTable => throw new TableException( - "UpsertStreamTableSink requires that Table has" + - " a full primary keys if it is updated.") - } - + upsertSink.setIsAppendOnly(true) + upsertSink.setKeyFields( + UpdatingPlanChecker.getUniqueKeyForUpsertSink(this, planner, upsertSink).orNull) translateToTransformation(withChangeFlag = true, planner) - case _: AppendStreamTableSink[T] => - // we can insert the bounded DataStream into a StreamTableSink - translateToTransformation(withChangeFlag = false, planner) + case _ => translateToTransformation(withChangeFlag = false, planner) } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecSink.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecSink.scala index 6ccebb6088..67397702ae 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecSink.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecSink.scala @@ -95,19 +95,8 @@ class StreamExecSink[T]( val isAppendOnlyTable = UpdatingPlanChecker.isAppendOnly(this) upsertSink.setIsAppendOnly(isAppendOnlyTable) - // extract unique key fields - // Now we pick shortest one to sink - // TODO UpsertStreamTableSink setKeyFields interface should be Array[Array[String]] - val tableKeys = { - val sinkFieldNames = upsertSink.getTableSchema.getFieldNames - UpdatingPlanChecker.getUniqueKeyFields(getInput, planner, sinkFieldNames) match { - case Some(keys) => keys.sortBy(_.length).headOption - case None => None - } - } - // check that we have keys if the table has changes (is not append-only) - tableKeys match { + UpdatingPlanChecker.getUniqueKeyForUpsertSink(this, planner, upsertSink) match { case Some(keys) => upsertSink.setKeyFields(keys) case None if isAppendOnlyTable => upsertSink.setKeyFields(null) case None if !isAppendOnlyTable => throw new TableException( diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/UpdatingPlanChecker.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/UpdatingPlanChecker.scala index a2e78aba3e..d0b0cc5fe6 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/UpdatingPlanChecker.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/UpdatingPlanChecker.scala @@ -19,7 +19,10 @@ package org.apache.flink.table.planner.plan.utils import org.apache.flink.table.planner.delegation.PlannerBase import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery +import org.apache.flink.table.planner.plan.nodes.calcite.Sink import org.apache.flink.table.planner.plan.nodes.physical.stream._ +import org.apache.flink.table.sinks.UpsertStreamTableSink + import org.apache.calcite.plan.hep.HepRelVertex import org.apache.calcite.plan.volcano.RelSubset import org.apache.calcite.rel.{RelNode, RelVisitor} @@ -36,20 +39,6 @@ object UpdatingPlanChecker { appendOnlyValidator.isAppendOnly } - /** Extracts the unique keys of the table produced by the plan. */ - def getUniqueKeyFields( - relNode: RelNode, - planner: PlannerBase, - sinkFieldNames: Array[String]): Option[Array[Array[String]]] = { - val fmq = FlinkRelMetadataQuery.reuseOrCreate(planner.getRelBuilder.getCluster.getMetadataQuery) - val uniqueKeys = fmq.getUniqueKeys(relNode) - if (uniqueKeys != null && uniqueKeys.size() > 0) { - Some(uniqueKeys.filter(_.nonEmpty).map(_.toArray.map(sinkFieldNames)).toArray) - } else { - None - } - } - private class AppendOnlyValidator extends RelVisitor { var isAppendOnly = true @@ -67,4 +56,28 @@ object UpdatingPlanChecker { } } } + + def getUniqueKeyForUpsertSink( + sinkNode: Sink, + planner: PlannerBase, + sink: UpsertStreamTableSink[_]): Option[Array[String]] = { + // extract unique key fields + // Now we pick shortest one to sink + // TODO UpsertStreamTableSink setKeyFields interface should be Array[Array[String]] + val sinkFieldNames = sink.getTableSchema.getFieldNames + /** Extracts the unique keys of the table produced by the plan. */ + val fmq = FlinkRelMetadataQuery.reuseOrCreate( + planner.getRelBuilder.getCluster.getMetadataQuery) + val uniqueKeys = fmq.getUniqueKeys(sinkNode.getInput) + if (uniqueKeys != null && uniqueKeys.size() > 0) { + uniqueKeys + .filter(_.nonEmpty) + .map(_.toArray.map(sinkFieldNames)) + .toSeq + .sortBy(_.length) + .headOption + } else { + None + } + } } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/TableSinkITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/TableSinkITCase.scala index 3c9e6b4f36..8cd3051318 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/TableSinkITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/TableSinkITCase.scala @@ -20,14 +20,17 @@ package org.apache.flink.table.planner.runtime.batch.table import org.apache.flink.table.api.scala._ import org.apache.flink.table.api.{DataTypes, TableSchema} -import org.apache.flink.table.planner.runtime.utils.BatchTestBase +import org.apache.flink.table.planner.runtime.utils.{BatchTestBase, TestingRetractTableSink, TestingUpsertTableSink} import org.apache.flink.table.planner.runtime.utils.TestData._ import org.apache.flink.table.planner.utils.MemoryTableSourceSinkUtil import org.apache.flink.table.planner.utils.MemoryTableSourceSinkUtil.{DataTypeAppendStreamTableSink, DataTypeOutputFormatTableSink} import org.apache.flink.test.util.TestBaseUtils +import org.junit.Assert._ import org.junit._ +import java.util.TimeZone + import scala.collection.JavaConverters._ class TableSinkITCase extends BatchTestBase { @@ -115,5 +118,87 @@ class TableSinkITCase extends BatchTestBase { TestBaseUtils.compareResultAsText(results, expected) } + private def prepareForUpsertSink(): TestingUpsertTableSink = { + val schema = TableSchema.builder() + .field("a", DataTypes.INT()) + .field("b", DataTypes.DOUBLE()) + .build() + val sink = new TestingUpsertTableSink(Array(0), TimeZone.getDefault) + tEnv.registerTableSink("testSink", sink.configure(schema.getFieldNames, schema.getFieldTypes)) + registerCollection("MyTable", simpleData2, simpleType2, "a, b", nullableOfSimpleData2) + sink + } + + @Test + def testUpsertSink(): Unit = { + val sink = prepareForUpsertSink() + sink.expectedKeys = Some(Array("a")) + sink.expectedIsAppendOnly = Some(false) + + tEnv.from("MyTable") + .groupBy('a) + .select('a, 'b.sum()) + .insertInto("testSink") + tEnv.execute("") + + val result = sink.getUpsertResults.sorted + val expected = List( + "1,0.1", + "2,0.4", + "3,1.0", + "4,2.2", + "5,3.9").sorted + assertEquals(expected, result) + } + + @Test + def testUpsertSinkWithAppend(): Unit = { + val sink = prepareForUpsertSink() + sink.expectedKeys = None + sink.expectedIsAppendOnly = Some(true) + + tEnv.from("MyTable") + .select('a, 'b) + .where('a < 3) + .insertInto("testSink") + tEnv.execute("") + + val result = sink.getRawResults.sorted + val expected = List( + "(true,1,0.1)", + "(true,2,0.2)", + "(true,2,0.2)").sorted + assertEquals(expected, result) + } + + private def prepareForRetractSink(): TestingRetractTableSink = { + val schema = TableSchema.builder() + .field("a", DataTypes.INT()) + .field("b", DataTypes.DOUBLE()) + .build() + val sink = new TestingRetractTableSink(TimeZone.getDefault) + tEnv.registerTableSink("testSink", sink.configure(schema.getFieldNames, schema.getFieldTypes)) + registerCollection("MyTable", simpleData2, simpleType2, "a, b", nullableOfSimpleData2) + sink + } + + @Test + def testRetractSink(): Unit = { + val sink = prepareForRetractSink() + tEnv.from("MyTable") + .groupBy('a) + .select('a, 'b.sum()) + .insertInto("testSink") + tEnv.execute("") + + val result = sink.getRawResults.sorted + val expected = List( + "(true,1,0.1)", + "(true,2,0.4)", + "(true,3,1.0)", + "(true,4,2.2)", + "(true,5,3.9)").sorted + assertEquals(expected, result) + } } -- Gitee From bb4ec22a7d0e0d0831ec56b121eefb465bf8f939 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Mon, 23 Mar 2020 19:03:13 +0800 Subject: [PATCH 279/885] [FLINK-16712][task] Refactor StreamTask to construct final fields At the moment there are four fields initialized in the method of StreamTask#beforeInvoke, such as `stateBackend`, `checkpointStorage`, `timerService`, `asyncOperationsThreadPool`. In general it is suggested to use final fields to get known benefits. So we can refactor the StreamTask to initialize these fields in the constructor instead. This closes #11486 --- .../state/api/output/BoundedStreamTask.java | 2 +- .../tasks/AbstractTwoInputStreamTask.java | 2 +- .../tasks/MultipleInputStreamTask.java | 2 +- .../runtime/tasks/OneInputStreamTask.java | 4 +- .../runtime/tasks/SourceReaderStreamTask.java | 2 +- .../runtime/tasks/SourceStreamTask.java | 4 +- .../runtime/tasks/StreamIterationHead.java | 2 +- .../runtime/tasks/StreamIterationTail.java | 2 +- .../streaming/runtime/tasks/StreamTask.java | 68 ++++++++----------- .../runtime/tasks/TwoInputStreamTask.java | 2 +- .../tasks/LocalStateForwardingTest.java | 2 +- ...ipleInputStreamTaskTestHarnessBuilder.java | 4 +- .../tasks/OneInputStreamTaskTestHarness.java | 10 +-- .../StreamTaskExecutionDecorationTest.java | 2 +- .../StreamTaskMailboxTestHarnessBuilder.java | 6 +- .../tasks/StreamTaskSelectiveReadingTest.java | 2 +- .../tasks/StreamTaskTerminationTest.java | 2 +- .../runtime/tasks/StreamTaskTest.java | 19 ++++-- .../runtime/tasks/StreamTaskTestHarness.java | 18 ++--- .../tasks/SynchronousCheckpointITCase.java | 2 +- .../tasks/SynchronousCheckpointTest.java | 6 +- .../tasks/TaskCheckpointingBehaviourTest.java | 2 +- .../tasks/TwoInputStreamTaskTestHarness.java | 6 +- .../flink/streaming/util/MockStreamTask.java | 2 +- .../streaming/util/MockStreamTaskBuilder.java | 2 +- .../codegen/agg/batch/BatchAggTestBase.scala | 5 +- .../JobMasterStopWithSavepointIT.java | 6 +- 27 files changed, 91 insertions(+), 95 deletions(-) diff --git a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/output/BoundedStreamTask.java b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/output/BoundedStreamTask.java index 60f195882f..7a801a48c3 100644 --- a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/output/BoundedStreamTask.java +++ b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/output/BoundedStreamTask.java @@ -60,7 +60,7 @@ class BoundedStreamTask & Bo BoundedStreamTask( Environment environment, Iterable input, - Collector collector) { + Collector collector) throws Exception { super(environment, new NeverFireProcessingTimeService()); this.input = input.iterator(); this.collector = collector; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AbstractTwoInputStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AbstractTwoInputStreamTask.java index 7c3ddaac39..9b03e8bbec 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AbstractTwoInputStreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/AbstractTwoInputStreamTask.java @@ -47,7 +47,7 @@ public abstract class AbstractTwoInputStreamTask extends StreamTa * * @param env The task environment for this task. */ - public AbstractTwoInputStreamTask(Environment env) { + public AbstractTwoInputStreamTask(Environment env) throws Exception { super(env); input1WatermarkGauge = new WatermarkGauge(); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTask.java index 81980b1cf3..9cc361adb0 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTask.java @@ -46,7 +46,7 @@ import static org.apache.flink.util.Preconditions.checkState; */ @Internal public class MultipleInputStreamTask extends StreamTask> { - public MultipleInputStreamTask(Environment env) { + public MultipleInputStreamTask(Environment env) throws Exception { super(env); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java index 8de7654db0..8755bf03ad 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java @@ -59,7 +59,7 @@ public class OneInputStreamTask extends StreamTask extends StreamTask extends StreamTask> { - public SourceReaderStreamTask(Environment env) { + public SourceReaderStreamTask(Environment env) throws Exception { super(env); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java index a1c81aa8f1..c7450a96c2 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java @@ -64,11 +64,11 @@ public class SourceStreamTask, OP extends S */ private volatile boolean isFinished = false; - public SourceStreamTask(Environment env) { + public SourceStreamTask(Environment env) throws Exception { this(env, new Object()); } - private SourceStreamTask(Environment env, Object lock) { + private SourceStreamTask(Environment env, Object lock) throws Exception { super(env, null, FatalExitExceptionHandler.INSTANCE, StreamTaskActionExecutor.synchronizedExecutor(lock)); this.lock = Preconditions.checkNotNull(lock); this.sourceThread = new LegacySourceFunctionThread(); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java index 1bb9e2b979..cafbfa030e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java @@ -50,7 +50,7 @@ public class StreamIterationHead extends OneInputStreamTask { private final long iterationWaitTime; private final boolean shouldWait; - public StreamIterationHead(Environment env) { + public StreamIterationHead(Environment env) throws Exception { super(env); final String iterationId = getConfiguration().getIterationId(); if (iterationId == null || iterationId.length() == 0) { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationTail.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationTail.java index 26b273f429..eedf05ee6e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationTail.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationTail.java @@ -43,7 +43,7 @@ public class StreamIterationTail extends OneInputStreamTask { private static final Logger LOG = LoggerFactory.getLogger(StreamIterationTail.class); - public StreamIterationTail(Environment environment) { + public StreamIterationTail(Environment environment) throws Exception { super(environment); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java index 82651ac892..dd7cd4d1ab 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java @@ -168,19 +168,17 @@ public abstract class StreamTask> protected final StreamConfig configuration; /** Our state backend. We use this to create checkpoint streams and a keyed state backend. */ - protected StateBackend stateBackend; + protected final StateBackend stateBackend; /** The external storage where checkpoint data is persisted. */ - private CheckpointStorageWorkerView checkpointStorage; + private final CheckpointStorageWorkerView checkpointStorage; /** * The internal {@link TimerService} used to define the current * processing time (default = {@code System.currentTimeMillis()}) and * register timers for tasks to be executed in the future. */ - protected TimerService timerService; - - private final Thread.UncaughtExceptionHandler uncaughtExceptionHandler; + protected final TimerService timerService; /** The currently active background materialization threads. */ private final CloseableRegistry cancelables = new CloseableRegistry(); @@ -199,7 +197,7 @@ public abstract class StreamTask> private boolean disposedOperators; /** Thread pool for async snapshot workers. */ - private ExecutorService asyncOperationsThreadPool; + private final ExecutorService asyncOperationsThreadPool; private final RecordWriterDelegate>> recordWriter; @@ -214,7 +212,7 @@ public abstract class StreamTask> * * @param env The task environment for this task. */ - protected StreamTask(Environment env) { + protected StreamTask(Environment env) throws Exception { this(env, null); } @@ -224,14 +222,14 @@ public abstract class StreamTask> * @param env The task environment for this task. * @param timerService Optionally, a specific timer service to use. */ - protected StreamTask(Environment env, @Nullable TimerService timerService) { + protected StreamTask(Environment env, @Nullable TimerService timerService) throws Exception { this(env, timerService, FatalExitExceptionHandler.INSTANCE); } protected StreamTask( Environment environment, @Nullable TimerService timerService, - Thread.UncaughtExceptionHandler uncaughtExceptionHandler) { + Thread.UncaughtExceptionHandler uncaughtExceptionHandler) throws Exception { this(environment, timerService, uncaughtExceptionHandler, StreamTaskActionExecutor.IMMEDIATE); } @@ -251,7 +249,7 @@ public abstract class StreamTask> Environment environment, @Nullable TimerService timerService, Thread.UncaughtExceptionHandler uncaughtExceptionHandler, - StreamTaskActionExecutor actionExecutor) { + StreamTaskActionExecutor actionExecutor) throws Exception { this(environment, timerService, uncaughtExceptionHandler, actionExecutor, new TaskMailboxImpl(Thread.currentThread())); } @@ -260,17 +258,28 @@ public abstract class StreamTask> @Nullable TimerService timerService, Thread.UncaughtExceptionHandler uncaughtExceptionHandler, StreamTaskActionExecutor actionExecutor, - TaskMailbox mailbox) { + TaskMailbox mailbox) throws Exception { super(environment); - this.timerService = timerService; - this.uncaughtExceptionHandler = Preconditions.checkNotNull(uncaughtExceptionHandler); this.configuration = new StreamConfig(getTaskConfiguration()); this.recordWriter = createRecordWriterDelegate(configuration, environment); this.actionExecutor = Preconditions.checkNotNull(actionExecutor); this.mailboxProcessor = new MailboxProcessor(this::processInput, mailbox, actionExecutor); this.asyncExceptionHandler = new StreamTaskAsyncExceptionHandler(environment); + this.asyncOperationsThreadPool = Executors.newCachedThreadPool( + new ExecutorThreadFactory("AsyncOperations", uncaughtExceptionHandler)); + + this.stateBackend = createStateBackend(); + this.checkpointStorage = stateBackend.createCheckpointStorage(getEnvironment().getJobID()); + + // if the clock is not already set, then assign a default TimeServiceProvider + if (timerService == null) { + ThreadFactory timerThreadFactory = new DispatcherThreadFactory(TRIGGER_THREAD_GROUP, "Time Trigger for " + getName()); + this.timerService = new SystemProcessingTimeService(this::handleTimerException, timerThreadFactory); + } else { + this.timerService = timerService; + } } // ------------------------------------------------------------------------ @@ -404,21 +413,6 @@ public abstract class StreamTask> disposedOperators = false; LOG.debug("Initializing {}.", getName()); - asyncOperationsThreadPool = Executors.newCachedThreadPool(new ExecutorThreadFactory("AsyncOperations", uncaughtExceptionHandler)); - - stateBackend = createStateBackend(); - checkpointStorage = stateBackend.createCheckpointStorage(getEnvironment().getJobID()); - - // if the clock is not already set, then assign a default TimeServiceProvider - if (timerService == null) { - ThreadFactory timerThreadFactory = - new DispatcherThreadFactory(TRIGGER_THREAD_GROUP, "Time Trigger for " + getName()); - - timerService = new SystemProcessingTimeService( - this::handleTimerException, - timerThreadFactory); - } - operatorChain = new OperatorChain<>(this, recordWriter); headOperator = operatorChain.getHeadOperator(); @@ -636,11 +630,9 @@ public abstract class StreamTask> @Override protected void finalize() throws Throwable { super.finalize(); - if (timerService != null) { - if (!timerService.isTerminated()) { - LOG.info("Timer service is shutting down."); - timerService.shutdownService(); - } + if (!timerService.isTerminated()) { + LOG.info("Timer service is shutting down."); + timerService.shutdownService(); } cancelables.close(); @@ -879,7 +871,7 @@ public abstract class StreamTask> private void tryShutdownTimerService() { - if (timerService != null && !timerService.isTerminated()) { + if (!timerService.isTerminated()) { try { final long timeoutMs = getEnvironment().getTaskManagerInfo().getConfiguration(). @@ -959,7 +951,6 @@ public abstract class StreamTask> */ @VisibleForTesting TimerService getTimerService() { - Preconditions.checkState(timerService != null, "The timer service has not been initialized."); return timerService; } @@ -974,10 +965,9 @@ public abstract class StreamTask> } public ProcessingTimeServiceFactory getProcessingTimeServiceFactory() { - return mailboxExecutor -> { - Preconditions.checkState(timerService != null, "The timer service has not been initialized."); - return new ProcessingTimeServiceImpl(timerService, callback -> deferCallbackToMailbox(mailboxExecutor, callback)); - }; + return mailboxExecutor -> new ProcessingTimeServiceImpl( + timerService, + callback -> deferCallbackToMailbox(mailboxExecutor, callback)); } /** diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java index 9c8c484784..495081ff09 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java @@ -40,7 +40,7 @@ import static org.apache.flink.util.Preconditions.checkState; @Internal public class TwoInputStreamTask extends AbstractTwoInputStreamTask { - public TwoInputStreamTask(Environment env) { + public TwoInputStreamTask(Environment env) throws Exception { super(env); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/LocalStateForwardingTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/LocalStateForwardingTest.java index bd0e49e38c..021fb31f1a 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/LocalStateForwardingTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/LocalStateForwardingTest.java @@ -77,7 +77,7 @@ public class LocalStateForwardingTest extends TestLogger { * async checkpointing thread to the {@link org.apache.flink.runtime.state.TaskStateManager}. */ @Test - public void testReportingFromSnapshotToTaskStateManager() { + public void testReportingFromSnapshotToTaskStateManager() throws Exception { TestTaskStateManager taskStateManager = new TestTaskStateManager(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTaskTestHarnessBuilder.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTaskTestHarnessBuilder.java index 81e2bcb9e7..3dee5edae2 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTaskTestHarnessBuilder.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTaskTestHarnessBuilder.java @@ -27,11 +27,11 @@ import org.apache.flink.streaming.api.graph.StreamNode; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner; +import org.apache.flink.util.function.FunctionWithException; import java.util.ArrayList; import java.util.LinkedList; import java.util.List; -import java.util.function.Function; /** * Builder to create a {@link StreamTaskMailboxTestHarness} to test {@link MultipleInputStreamTask}. @@ -42,7 +42,7 @@ public class MultipleInputStreamTaskTestHarnessBuilder extends StreamTaskMa private final ArrayList inputChannelsPerGate = new ArrayList<>(); public MultipleInputStreamTaskTestHarnessBuilder( - Function> taskFactory, + FunctionWithException, Exception> taskFactory, TypeInformation outputType) { super(taskFactory, outputType); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java index 9ec4fe6102..c6f74d4d3b 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTestHarness.java @@ -28,9 +28,9 @@ import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.io.network.partition.consumer.StreamTestSingleInputGate; import org.apache.flink.runtime.state.LocalRecoveryConfig; import org.apache.flink.runtime.state.TestLocalRecoveryConfig; +import org.apache.flink.util.function.FunctionWithException; import java.io.File; -import java.util.function.Function; /** @@ -62,7 +62,7 @@ public class OneInputStreamTaskTestHarness extends StreamTaskTestHarnes * of channels per input gate and local recovery disabled. */ public OneInputStreamTaskTestHarness( - Function> taskFactory, + FunctionWithException, Exception> taskFactory, int numInputGates, int numInputChannelsPerGate, TypeInformation inputType, @@ -71,7 +71,7 @@ public class OneInputStreamTaskTestHarness extends StreamTaskTestHarnes } public OneInputStreamTaskTestHarness( - Function> taskFactory, + FunctionWithException, Exception> taskFactory, int numInputGates, int numInputChannelsPerGate, TypeInformation inputType, @@ -93,7 +93,7 @@ public class OneInputStreamTaskTestHarness extends StreamTaskTestHarnes * of channels per input gate and specified localRecoveryConfig. */ public OneInputStreamTaskTestHarness( - Function> taskFactory, + FunctionWithException, Exception> taskFactory, int numInputGates, int numInputChannelsPerGate, TypeInformation inputType, @@ -113,7 +113,7 @@ public class OneInputStreamTaskTestHarness extends StreamTaskTestHarnes * Creates a test harness with one input gate that has one input channel. */ public OneInputStreamTaskTestHarness( - Function> taskFactory, + FunctionWithException, Exception> taskFactory, TypeInformation inputType, TypeInformation outputType) { this(taskFactory, 1, 1, inputType, outputType, TestLocalRecoveryConfig.disabled()); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskExecutionDecorationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskExecutionDecorationTest.java index 1e43cc8dff..0f457638c4 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskExecutionDecorationTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskExecutionDecorationTest.java @@ -80,7 +80,7 @@ public class StreamTaskExecutionDecorationTest { } @Before - public void before() { + public void before() throws Exception { mailbox = new TaskMailboxImpl(); decorator = new CountingStreamTaskActionExecutor(); task = new StreamTask>(new StreamTaskTest.DeclineDummyEnvironment(), null, FatalExitExceptionHandler.INSTANCE, decorator, mailbox) { diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskMailboxTestHarnessBuilder.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskMailboxTestHarnessBuilder.java index b874a04e72..b8dbd78938 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskMailboxTestHarnessBuilder.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskMailboxTestHarnessBuilder.java @@ -42,6 +42,7 @@ import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.api.operators.StreamOperatorFactory; import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner; import org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer; +import org.apache.flink.util.function.FunctionWithException; import javax.annotation.Nullable; @@ -50,7 +51,6 @@ import java.util.Collections; import java.util.LinkedList; import java.util.List; import java.util.Queue; -import java.util.function.Function; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.flink.util.Preconditions.checkState; @@ -59,7 +59,7 @@ import static org.apache.flink.util.Preconditions.checkState; * Builder class for {@link StreamTaskMailboxTestHarness}. */ public abstract class StreamTaskMailboxTestHarnessBuilder { - protected final Function> taskFactory; + protected final FunctionWithException, Exception> taskFactory; protected final TypeSerializer outputSerializer; protected final ExecutionConfig executionConfig = new ExecutionConfig(); @@ -77,7 +77,7 @@ public abstract class StreamTaskMailboxTestHarnessBuilder { private boolean setupCalled = false; public StreamTaskMailboxTestHarnessBuilder( - Function> taskFactory, + FunctionWithException, Exception> taskFactory, TypeInformation outputType) { this.taskFactory = checkNotNull(taskFactory); outputSerializer = outputType.createSerializer(executionConfig); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskSelectiveReadingTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskSelectiveReadingTest.java index 94c8d849d2..9707aa417b 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskSelectiveReadingTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskSelectiveReadingTest.java @@ -188,7 +188,7 @@ public class StreamTaskSelectiveReadingTest { private volatile boolean started; - TestSelectiveReadingTask(Environment env) { + TestSelectiveReadingTask(Environment env) throws Exception { super(env); started = false; } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java index 06d394c9fa..7de33c7bdb 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java @@ -227,7 +227,7 @@ public class StreamTaskTerminationTest extends TestLogger { private boolean isRunning; - public BlockingStreamTask(Environment env) { + public BlockingStreamTask(Environment env) throws Exception { super(env); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java index 1628e4c055..62c54e596f 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java @@ -330,7 +330,7 @@ public class StreamTaskTest extends TestLogger { */ public static class CancelFailingTask extends StreamTask> { - public CancelFailingTask(Environment env) { + public CancelFailingTask(Environment env) throws Exception { super(env); } @@ -442,7 +442,7 @@ public class StreamTaskTest extends TestLogger { */ public static class CancelThrowingTask extends StreamTask> { - public CancelThrowingTask(Environment env) { + public CancelThrowingTask(Environment env) throws Exception { super(env); } @@ -1109,7 +1109,7 @@ public class StreamTaskTest extends TestLogger { */ public static class NoOpStreamTask> extends StreamTask { - public NoOpStreamTask(Environment environment) { + public NoOpStreamTask(Environment environment) throws Exception { super(environment); } @@ -1264,7 +1264,10 @@ public class StreamTaskTest extends TestLogger { private final OperatorChain> overrideOperatorChain; - MockStreamTask(Environment env, OperatorChain> operatorChain, Thread.UncaughtExceptionHandler uncaughtExceptionHandler) { + MockStreamTask( + Environment env, + OperatorChain> operatorChain, + Thread.UncaughtExceptionHandler uncaughtExceptionHandler) throws Exception { super(env, null, uncaughtExceptionHandler); this.overrideOperatorChain = operatorChain; } @@ -1315,7 +1318,9 @@ public class StreamTaskTest extends TestLogger { } } - private static MockStreamTask createMockStreamTask(Environment env, OperatorChain> operatorChain) { + private static MockStreamTask createMockStreamTask( + Environment env, + OperatorChain> operatorChain) throws Exception { return new MockStreamTask(env, operatorChain, FatalExitExceptionHandler.INSTANCE); } @@ -1328,7 +1333,7 @@ public class StreamTaskTest extends TestLogger { private static volatile boolean fail; - public StateBackendTestSource(Environment env) { + public StateBackendTestSource(Environment env) throws Exception { super(env); } @@ -1418,7 +1423,7 @@ public class StreamTaskTest extends TestLogger { /** Flag to wait until time trigger has been called. */ private transient boolean hasTimerTriggered; - ThreadInspectingTask(Environment env) { + ThreadInspectingTask(Environment env) throws Exception { super(env); Thread currentThread = Thread.currentThread(); taskThreadId = currentThread.getId(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java index cbb6fc6df7..3d3e034583 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java @@ -54,6 +54,8 @@ import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner; import org.apache.flink.streaming.runtime.streamrecord.StreamElement; import org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.function.FunctionWithException; +import org.apache.flink.util.function.SupplierWithException; import org.junit.Assert; @@ -64,8 +66,6 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.concurrent.LinkedBlockingQueue; -import java.util.function.Function; -import java.util.function.Supplier; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.flink.util.Preconditions.checkState; @@ -90,7 +90,7 @@ public class StreamTaskTestHarness { public static final int DEFAULT_NETWORK_BUFFER_SIZE = 1024; - private final Function> taskFactory; + private final FunctionWithException, Exception> taskFactory; public long memorySize; public int bufferSize; @@ -122,20 +122,20 @@ public class StreamTaskTestHarness { protected StreamTestSingleInputGate[] inputGates; public StreamTaskTestHarness( - Function> taskFactory, + FunctionWithException, Exception> taskFactory, TypeInformation outputType) { this(taskFactory, outputType, TestLocalRecoveryConfig.disabled()); } public StreamTaskTestHarness( - Function> taskFactory, + FunctionWithException, Exception> taskFactory, TypeInformation outputType, File localRootDir) { this(taskFactory, outputType, new LocalRecoveryConfig(true, new LocalRecoveryDirectoryProviderImpl(localRootDir, new JobID(), new JobVertexID(), 0))); } public StreamTaskTestHarness( - Function> taskFactory, + FunctionWithException, Exception> taskFactory, TypeInformation outputType, LocalRecoveryConfig localRecoveryConfig) { this.taskFactory = checkNotNull(taskFactory); @@ -441,20 +441,20 @@ public class StreamTaskTestHarness { private class TaskThread extends Thread { - private final Supplier> taskFactory; + private final SupplierWithException, Exception> taskFactory; private volatile StreamTask task; private volatile Throwable error; - TaskThread(Supplier> taskFactory) { + TaskThread(SupplierWithException, Exception> taskFactory) { super("Task Thread"); this.taskFactory = taskFactory; } @Override public void run() { - task = taskFactory.get(); try { + task = taskFactory.get(); task.invoke(); shutdownIOManager(); shutdownMemoryManager(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SynchronousCheckpointITCase.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SynchronousCheckpointITCase.java index 622ccc1a34..0d4cdf0717 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SynchronousCheckpointITCase.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SynchronousCheckpointITCase.java @@ -135,7 +135,7 @@ public class SynchronousCheckpointITCase { // Flag to emit the first event only once. private boolean isRunning; - public SynchronousCheckpointTestingTask(Environment environment) { + public SynchronousCheckpointTestingTask(Environment environment) throws Exception { super(environment); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SynchronousCheckpointTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SynchronousCheckpointTest.java index c5acee0e73..0efbe73549 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SynchronousCheckpointTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SynchronousCheckpointTest.java @@ -62,8 +62,8 @@ public class SynchronousCheckpointTest { taskInvocation = CompletableFuture.runAsync( () -> { - streamTaskUnderTest = createTask(eventQueue); try { + streamTaskUnderTest = createTask(eventQueue); streamTaskUnderTest.invoke(); } catch (RuntimeException e) { throw e; @@ -133,7 +133,7 @@ public class SynchronousCheckpointTest { } } - private static StreamTaskUnderTest createTask(Queue eventQueue) { + private static StreamTaskUnderTest createTask(Queue eventQueue) throws Exception { final DummyEnvironment environment = new DummyEnvironment("test", 1, 0); return new StreamTaskUnderTest(environment, eventQueue); } @@ -145,7 +145,7 @@ public class SynchronousCheckpointTest { StreamTaskUnderTest( final Environment env, - Queue eventQueue) { + Queue eventQueue) throws Exception { super(env); this.eventQueue = checkNotNull(eventQueue); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java index f87e0e493f..bdf3a52346 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java @@ -469,7 +469,7 @@ public class TaskCheckpointingBehaviourTest extends TestLogger { */ public static final class TestStreamTask extends OneInputStreamTask { - public TestStreamTask(Environment env) { + public TestStreamTask(Environment env) throws Exception { super(env); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java index 481cdd32ba..386813f86e 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTestHarness.java @@ -27,10 +27,10 @@ import org.apache.flink.streaming.api.graph.StreamNode; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner; +import org.apache.flink.util.function.FunctionWithException; import java.util.LinkedList; import java.util.List; -import java.util.function.Function; /** @@ -67,7 +67,7 @@ public class TwoInputStreamTaskTestHarness extends StreamTaskTest * it should be assigned to the first (1), or second (2) input of the task. */ public TwoInputStreamTaskTestHarness( - Function> taskFactory, + FunctionWithException, Exception> taskFactory, int numInputGates, int numInputChannelsPerGate, int[] inputGateAssignment, @@ -92,7 +92,7 @@ public class TwoInputStreamTaskTestHarness extends StreamTaskTest * second task input. */ public TwoInputStreamTaskTestHarness( - Function> taskFactory, + FunctionWithException, Exception> taskFactory, TypeInformation inputType1, TypeInformation inputType2, TypeInformation outputType) { diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTask.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTask.java index 77e29044af..7ca01d5f65 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTask.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTask.java @@ -67,7 +67,7 @@ public class MockStreamTask> extends StreamT BiConsumer handleAsyncException, TaskMailbox taskMailbox, StreamTaskActionExecutor.SynchronizedStreamTaskActionExecutor taskActionExecutor, - StreamInputProcessor inputProcessor) { + StreamInputProcessor inputProcessor) throws Exception { super(environment, timerService, FatalExitExceptionHandler.INSTANCE, taskActionExecutor, taskMailbox); this.name = name; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTaskBuilder.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTaskBuilder.java index 946809e5a5..b8fd659497 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTaskBuilder.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTaskBuilder.java @@ -136,7 +136,7 @@ public class MockStreamTaskBuilder { return this; } - public MockStreamTask build() { + public MockStreamTask build() throws Exception { return new MockStreamTask( environment, name, diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/codegen/agg/batch/BatchAggTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/codegen/agg/batch/BatchAggTestBase.scala index 541a849702..a21d6f34b4 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/codegen/agg/batch/BatchAggTestBase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/codegen/agg/batch/BatchAggTestBase.scala @@ -28,9 +28,10 @@ import org.apache.flink.table.planner.utils.BaseRowTestUtil import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo import org.apache.flink.table.types.logical.{BigIntType, DoubleType, LogicalType, RowType, VarCharType} +import org.apache.flink.util.function.FunctionWithException + import org.junit.Assert import java.util -import java.util.function import scala.collection.JavaConverters._ @@ -64,7 +65,7 @@ abstract class BatchAggTestBase extends AggTestBase(isBatchMode = true) { args: (CodeGenOperatorFactory[BaseRow], RowType, RowType), input: Array[BaseRow], expectedOutput: Array[GenericRow]): Unit = { val testHarness = new OneInputStreamTaskTestHarness[BaseRow, BaseRow]( - new function.Function[Environment, OneInputStreamTask[BaseRow, BaseRow]] { + new FunctionWithException[Environment, OneInputStreamTask[BaseRow, BaseRow], Exception] { override def apply(t: Environment) = new OneInputStreamTask(t) }, 1, 1, BaseRowTypeInfo.of(args._2), BaseRowTypeInfo.of(args._3)) testHarness.memorySize = 32 * 100 * 1024 diff --git a/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointIT.java b/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointIT.java index 7821afce4e..e576323599 100644 --- a/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointIT.java +++ b/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointIT.java @@ -316,7 +316,7 @@ public class JobMasterStopWithSavepointIT extends AbstractTestBase { private long synchronousSavepointId = Long.MIN_VALUE; - public ExceptionOnCallbackStreamTask(final Environment environment) { + public ExceptionOnCallbackStreamTask(final Environment environment) throws Exception { super(environment); } @@ -360,7 +360,7 @@ public class JobMasterStopWithSavepointIT extends AbstractTestBase { private final transient OneShotLatch finishLatch; - public NoOpBlockingStreamTask(final Environment environment) { + public NoOpBlockingStreamTask(final Environment environment) throws Exception { super(environment); this.finishLatch = new OneShotLatch(); } @@ -387,7 +387,7 @@ public class JobMasterStopWithSavepointIT extends AbstractTestBase { private final transient OneShotLatch finishLatch; - public CheckpointCountingTask(final Environment environment) { + public CheckpointCountingTask(final Environment environment) throws Exception { super(environment); this.finishLatch = new OneShotLatch(); } -- Gitee From f4b68c4440c0c0a8ab061a4eac366bbb19233455 Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Wed, 25 Mar 2020 14:31:22 +0800 Subject: [PATCH 280/885] [FLINK-15640][k8s] Support to set labels for jobmanager and taskmanager pod This closes #11489 . --- .../kubernetes_config_configuration.html | 12 ++++++++++ .../KubernetesConfigOptions.java | 14 +++++++++++ .../AbstractKubernetesParameters.java | 3 ++- .../KubernetesJobManagerParameters.java | 8 ++++--- .../KubernetesTaskManagerParameters.java | 6 ++++- .../kubernetes/utils/KubernetesUtils.java | 3 ++- .../flink/kubernetes/KubernetesTestBase.java | 6 +++++ .../KubernetesJobManagerTestBase.java | 8 +++++++ .../KubernetesTaskManagerTestBase.java | 9 ++++++++ .../ExternalServiceDecoratorTest.java | 1 + .../InitJobManagerDecoratorTest.java | 1 + .../InitTaskManagerDecoratorTest.java | 1 + .../InternalServiceDecoratorTest.java | 1 + .../KubernetesJobManagerFactoryTest.java | 6 +++-- .../KubernetesTaskManagerFactoryTest.java | 2 +- .../KubernetesJobManagerParametersTest.java | 21 +++++++++++++---- .../KubernetesTaskManagerParametersTest.java | 23 +++++++++++++++---- 17 files changed, 108 insertions(+), 17 deletions(-) diff --git a/docs/_includes/generated/kubernetes_config_configuration.html b/docs/_includes/generated/kubernetes_config_configuration.html index 3894c5780b..dbea103218 100644 --- a/docs/_includes/generated/kubernetes_config_configuration.html +++ b/docs/_includes/generated/kubernetes_config_configuration.html @@ -86,6 +86,12 @@ Double The number of cpu used by job manager + +
    kubernetes.jobmanager.labels
    + (none) + Map + The labels to be set for JobManager pod. Specified as key:value pairs separated by commas. For example, version:alphav1,deploy:test. +
    kubernetes.jobmanager.service-account
    "default" @@ -122,5 +128,11 @@ Double The number of cpu used by task manager. By default, the cpu is set to the number of slots per TaskManager + +
    kubernetes.taskmanager.labels
    + (none) + Map + The labels to be set for TaskManager pods. Specified as key:value pairs separated by commas. For example, version:alphav1,deploy:test. + diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java index 0006eed525..37ab3943bb 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java @@ -112,6 +112,20 @@ public class KubernetesConfigOptions { // The following config options could be overridden by KubernetesCliOptions. // --------------------------------------------------------------------------------- + public static final ConfigOption> JOB_MANAGER_LABELS = + key("kubernetes.jobmanager.labels") + .mapType() + .noDefaultValue() + .withDescription("The labels to be set for JobManager pod. Specified as key:value pairs separated by commas. " + + "For example, version:alphav1,deploy:test."); + + public static final ConfigOption> TASK_MANAGER_LABELS = + key("kubernetes.taskmanager.labels") + .mapType() + .noDefaultValue() + .withDescription("The labels to be set for TaskManager pods. Specified as key:value pairs separated by commas. " + + "For example, version:alphav1,deploy:test."); + public static final ConfigOption CLUSTER_ID = key("kubernetes.cluster-id") .stringType() diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/AbstractKubernetesParameters.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/AbstractKubernetesParameters.java index fc4ff48a95..8e7c443d5c 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/AbstractKubernetesParameters.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/AbstractKubernetesParameters.java @@ -28,6 +28,7 @@ import io.fabric8.kubernetes.api.model.LocalObjectReference; import org.apache.commons.lang3.StringUtils; import java.io.File; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -104,7 +105,7 @@ public abstract class AbstractKubernetesParameters implements KubernetesParamete commonLabels.put(Constants.LABEL_TYPE_KEY, Constants.LABEL_TYPE_NATIVE_TYPE); commonLabels.put(Constants.LABEL_APP_KEY, getClusterId()); - return commonLabels; + return Collections.unmodifiableMap(commonLabels); } @Override diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java index 1c8d5908a8..3f1460159c 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java @@ -30,6 +30,7 @@ import org.apache.flink.kubernetes.utils.Constants; import org.apache.flink.kubernetes.utils.KubernetesUtils; import java.util.Collections; +import java.util.HashMap; import java.util.Map; import static org.apache.flink.util.Preconditions.checkArgument; @@ -53,10 +54,11 @@ public class KubernetesJobManagerParameters extends AbstractKubernetesParameters @Override public Map getLabels() { - Map labels = getCommonLabels(); + final Map labels = new HashMap<>(); + labels.putAll(flinkConfig.getOptional(KubernetesConfigOptions.JOB_MANAGER_LABELS).orElse(Collections.emptyMap())); + labels.putAll(getCommonLabels()); labels.put(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_JOB_MANAGER); - - return labels; + return Collections.unmodifiableMap(labels); } @Override diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParameters.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParameters.java index dbfddcc606..28561ab88f 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParameters.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParameters.java @@ -25,6 +25,7 @@ import org.apache.flink.kubernetes.utils.KubernetesUtils; import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; import java.util.Collections; +import java.util.HashMap; import java.util.Map; import static org.apache.flink.util.Preconditions.checkArgument; @@ -61,7 +62,10 @@ public class KubernetesTaskManagerParameters extends AbstractKubernetesParameter @Override public Map getLabels() { - return KubernetesUtils.getTaskManagerLabels(getClusterId()); + final Map labels = new HashMap<>(); + labels.putAll(flinkConfig.getOptional(KubernetesConfigOptions.TASK_MANAGER_LABELS).orElse(Collections.emptyMap())); + labels.putAll(KubernetesUtils.getTaskManagerLabels(getClusterId())); + return Collections.unmodifiableMap(labels); } @Override diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java index a7286f7d79..f0c6f13879 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java @@ -39,6 +39,7 @@ import java.io.FileInputStream; import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStreamReader; +import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -143,7 +144,7 @@ public class KubernetesUtils { labels.put(Constants.LABEL_TYPE_KEY, Constants.LABEL_TYPE_NATIVE_TYPE); labels.put(Constants.LABEL_APP_KEY, clusterId); labels.put(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_TASK_MANAGER); - return labels; + return Collections.unmodifiableMap(labels); } /** diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesTestBase.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesTestBase.java index 3c3441253b..fa0f2611ad 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesTestBase.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesTestBase.java @@ -30,6 +30,7 @@ import org.apache.flink.test.util.TestBaseUtils; import org.apache.flink.util.TestLogger; import io.fabric8.kubernetes.client.KubernetesClient; +import org.junit.After; import org.junit.Before; import org.junit.Rule; import org.junit.rules.TemporaryFolder; @@ -86,6 +87,11 @@ public class KubernetesTestBase extends TestLogger { flinkKubeClient = new Fabric8FlinkKubeClient(flinkConfig, kubeClient); } + @After + public void tearDown() throws Exception { + flinkKubeClient.close(); + } + protected void writeFlinkConfiguration() throws IOException { BootstrapTools.writeConfiguration(this.flinkConfig, new File(flinkConfDir, "flink-conf.yaml")); } diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/KubernetesJobManagerTestBase.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/KubernetesJobManagerTestBase.java index 73ada490ac..719784b9ca 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/KubernetesJobManagerTestBase.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/KubernetesJobManagerTestBase.java @@ -51,6 +51,13 @@ public class KubernetesJobManagerTestBase extends KubernetesTestBase { } }; + protected final Map userLabels = new HashMap() { + { + put("label1", "value1"); + put("label2", "value2"); + } + }; + protected KubernetesJobManagerParameters kubernetesJobManagerParameters; protected FlinkPod baseFlinkPod; @@ -65,6 +72,7 @@ public class KubernetesJobManagerTestBase extends KubernetesTestBase { this.flinkConfig.set(KubernetesConfigOptions.JOB_MANAGER_CPU, JOB_MANAGER_CPU); this.customizedEnvs.forEach((k, v) -> this.flinkConfig.setString(ResourceManagerOptions.CONTAINERIZED_MASTER_ENV_PREFIX + k, v)); + this.flinkConfig.set(KubernetesConfigOptions.JOB_MANAGER_LABELS, userLabels); final ClusterSpecification clusterSpecification = new ClusterSpecification.ClusterSpecificationBuilder() .setMasterMemoryMB(JOB_MANAGER_MEMORY) diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/KubernetesTaskManagerTestBase.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/KubernetesTaskManagerTestBase.java index c4308a92f2..a4e3cac7ba 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/KubernetesTaskManagerTestBase.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/KubernetesTaskManagerTestBase.java @@ -22,6 +22,7 @@ import org.apache.flink.configuration.MemorySize; import org.apache.flink.configuration.ResourceManagerOptions; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.kubernetes.KubernetesTestBase; +import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesTaskManagerParameters; import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec; @@ -52,6 +53,13 @@ public class KubernetesTaskManagerTestBase extends KubernetesTestBase { } }; + protected final Map userLabels = new HashMap() { + { + put("label1", "value1"); + put("label2", "value2"); + } + }; + protected TaskExecutorProcessSpec taskExecutorProcessSpec; protected ContaineredTaskManagerParameters containeredTaskManagerParameters; @@ -69,6 +77,7 @@ public class KubernetesTaskManagerTestBase extends KubernetesTestBase { flinkConfig.set(TaskManagerOptions.TOTAL_PROCESS_MEMORY, MemorySize.parse(TOTAL_PROCESS_MEMORY + "m")); customizedEnvs.forEach((k, v) -> flinkConfig.setString(ResourceManagerOptions.CONTAINERIZED_TASK_MANAGER_ENV_PREFIX + k, v)); + this.flinkConfig.set(KubernetesConfigOptions.TASK_MANAGER_LABELS, userLabels); taskExecutorProcessSpec = TaskExecutorProcessUtils.processSpecFromConfig(flinkConfig); containeredTaskManagerParameters = ContaineredTaskManagerParameters.create(flinkConfig, taskExecutorProcessSpec, diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecoratorTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecoratorTest.java index d34cf9bb91..dc97770ecd 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecoratorTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecoratorTest.java @@ -75,6 +75,7 @@ public class ExternalServiceDecoratorTest extends KubernetesJobManagerTestBase { assertEquals(expectedServicePorts, restService.getSpec().getPorts()); expectedLabels.put(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_JOB_MANAGER); + expectedLabels.putAll(userLabels); assertEquals(expectedLabels, restService.getSpec().getSelector()); } diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecoratorTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecoratorTest.java index 5e5a794be1..9713cd62bf 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecoratorTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecoratorTest.java @@ -144,6 +144,7 @@ public class InitJobManagerDecoratorTest extends KubernetesJobManagerTestBase { public void testPodLabels() { final Map expectedLabels = new HashMap<>(getCommonLabels()); expectedLabels.put(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_JOB_MANAGER); + expectedLabels.putAll(userLabels); assertEquals(expectedLabels, this.resultPod.getMetadata().getLabels()); } diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecoratorTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecoratorTest.java index e69f30593b..5489611e07 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecoratorTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecoratorTest.java @@ -143,6 +143,7 @@ public class InitTaskManagerDecoratorTest extends KubernetesTaskManagerTestBase public void testPodLabels() { final Map expectedLabels = new HashMap<>(getCommonLabels()); expectedLabels.put(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_TASK_MANAGER); + expectedLabels.putAll(userLabels); assertEquals(expectedLabels, this.resultPod.getMetadata().getLabels()); } diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InternalServiceDecoratorTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InternalServiceDecoratorTest.java index 12f91ea824..cb591cb1fe 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InternalServiceDecoratorTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InternalServiceDecoratorTest.java @@ -86,6 +86,7 @@ public class InternalServiceDecoratorTest extends KubernetesJobManagerTestBase { assertEquals(expectedServicePorts, internalService.getSpec().getPorts()); expectedLabels.put(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_JOB_MANAGER); + expectedLabels.putAll(userLabels); assertEquals(expectedLabels, internalService.getSpec().getSelector()); } } diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactoryTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactoryTest.java index b179ddc61a..16c688a7df 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactoryTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactoryTest.java @@ -86,6 +86,7 @@ public class KubernetesJobManagerFactoryTest extends KubernetesJobManagerTestBas assertEquals(KubernetesUtils.getDeploymentName(CLUSTER_ID), resultDeployment.getMetadata().getName()); final Map expectedLabels = getCommonLabels(); expectedLabels.put(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_JOB_MANAGER); + expectedLabels.putAll(userLabels); assertEquals(expectedLabels, resultDeployment.getMetadata().getLabels()); } @@ -96,6 +97,7 @@ public class KubernetesJobManagerFactoryTest extends KubernetesJobManagerTestBas final Map expectedLabels = new HashMap<>(getCommonLabels()); expectedLabels.put(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_JOB_MANAGER); + expectedLabels.putAll(userLabels); assertEquals(expectedLabels, resultDeploymentSpec.getTemplate().getMetadata().getLabels()); assertEquals(expectedLabels, resultDeploymentSpec.getSelector().getMatchLabels()); @@ -179,14 +181,14 @@ public class KubernetesJobManagerFactoryTest extends KubernetesJobManagerTestBas assertEquals(resultInternalService.getSpec().getType(), "ClusterIP"); assertEquals(3, resultInternalService.getSpec().getPorts().size()); - assertEquals(3, resultInternalService.getSpec().getSelector().size()); + assertEquals(5, resultInternalService.getSpec().getSelector().size()); final Service resultRestService = restServiceCandidates.get(0); assertEquals(2, resultRestService.getMetadata().getLabels().size()); assertEquals(resultRestService.getSpec().getType(), "LoadBalancer"); assertEquals(1, resultRestService.getSpec().getPorts().size()); - assertEquals(3, resultRestService.getSpec().getSelector().size()); + assertEquals(5, resultRestService.getSpec().getSelector().size()); } @Test diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesTaskManagerFactoryTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesTaskManagerFactoryTest.java index 642b641100..4a360da91e 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesTaskManagerFactoryTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesTaskManagerFactoryTest.java @@ -56,7 +56,7 @@ public class KubernetesTaskManagerFactoryTest extends KubernetesTaskManagerTestB @Test public void testPod() { assertEquals(POD_NAME, this.resultPod.getMetadata().getName()); - assertEquals(3, this.resultPod.getMetadata().getLabels().size()); + assertEquals(5, this.resultPod.getMetadata().getLabels().size()); assertEquals(2, this.resultPod.getSpec().getVolumes().size()); } diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParametersTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParametersTest.java index 81fba4b7b1..00f3bae8bc 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParametersTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParametersTest.java @@ -20,12 +20,13 @@ package org.apache.flink.kubernetes.kubeclient.parameters; import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.configuration.BlobServerOptions; -import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.ResourceManagerOptions; import org.apache.flink.configuration.RestOptions; +import org.apache.flink.kubernetes.KubernetesTestBase; import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; import org.apache.flink.kubernetes.configuration.KubernetesConfigOptionsInternal; +import org.apache.flink.kubernetes.utils.Constants; import org.apache.flink.util.FlinkRuntimeException; import org.junit.Test; @@ -44,13 +45,11 @@ import static org.junit.Assert.fail; /** * General tests for the {@link KubernetesJobManagerParameters}. */ -public class KubernetesJobManagerParametersTest { +public class KubernetesJobManagerParametersTest extends KubernetesTestBase { private static final int JOB_MANAGER_MEMORY = 768; private static final double JOB_MANAGER_CPU = 2.0; - private final Configuration flinkConfig = new Configuration(); - private final ClusterSpecification clusterSpecification = new ClusterSpecification.ClusterSpecificationBuilder() .setMasterMemoryMB(JOB_MANAGER_MEMORY) .setTaskManagerMemoryMB(1024) @@ -171,4 +170,18 @@ public class KubernetesJobManagerParametersTest { assertEquals(KubernetesConfigOptions.ServiceExposedType.NodePort, kubernetesJobManagerParameters.getRestServiceExposedType()); } + + @Test + public void testPrioritizeBuiltInLabels() { + final Map userLabels = new HashMap<>(); + userLabels.put(Constants.LABEL_TYPE_KEY, "user-label-type"); + userLabels.put(Constants.LABEL_APP_KEY, "user-label-app"); + userLabels.put(Constants.LABEL_COMPONENT_KEY, "user-label-component-jm"); + + flinkConfig.set(KubernetesConfigOptions.JOB_MANAGER_LABELS, userLabels); + + final Map expectedLabels = new HashMap<>(getCommonLabels()); + expectedLabels.put(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_JOB_MANAGER); + assertThat(kubernetesJobManagerParameters.getLabels(), is(equalTo(expectedLabels))); + } } diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParametersTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParametersTest.java index a9033b5640..b91daa5acf 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParametersTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParametersTest.java @@ -18,11 +18,12 @@ package org.apache.flink.kubernetes.kubeclient.parameters; -import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.MemorySize; import org.apache.flink.configuration.ResourceManagerOptions; import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.kubernetes.KubernetesTestBase; import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; +import org.apache.flink.kubernetes.utils.Constants; import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec; import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils; @@ -42,7 +43,7 @@ import static org.junit.Assert.assertTrue; /** * General tests for the {@link KubernetesTaskManagerParameters}. */ -public class KubernetesTaskManagerParametersTest { +public class KubernetesTaskManagerParametersTest extends KubernetesTestBase { private static final int TASK_MANAGER_MEMORY = 1024; private static final double TASK_MANAGER_CPU = 1.2; @@ -58,11 +59,11 @@ public class KubernetesTaskManagerParametersTest { } }; - private Configuration flinkConfig = new Configuration(); private KubernetesTaskManagerParameters kubernetesTaskManagerParameters; @Before - public void setup() { + public void setup() throws Exception { + super.setup(); flinkConfig.set(TaskManagerOptions.CPU_CORES, TASK_MANAGER_CPU); flinkConfig.set(TaskManagerOptions.TOTAL_PROCESS_MEMORY, MemorySize.parse(TASK_MANAGER_MEMORY + "m")); flinkConfig.set(TaskManagerOptions.RPC_PORT, String.valueOf(RPC_PORT)); @@ -130,4 +131,18 @@ public class KubernetesTaskManagerParametersTest { public void testGetDynamicProperties() { assertEquals(DYNAMIC_PROPERTIES, kubernetesTaskManagerParameters.getDynamicProperties()); } + + @Test + public void testPrioritizeBuiltInLabels() { + final Map userLabels = new HashMap<>(); + userLabels.put(Constants.LABEL_TYPE_KEY, "user-label-type"); + userLabels.put(Constants.LABEL_APP_KEY, "user-label-app"); + userLabels.put(Constants.LABEL_COMPONENT_KEY, "user-label-component-tm"); + + flinkConfig.set(KubernetesConfigOptions.TASK_MANAGER_LABELS, userLabels); + + final Map expectedLabels = new HashMap<>(getCommonLabels()); + expectedLabels.put(Constants.LABEL_COMPONENT_KEY, Constants.LABEL_COMPONENT_TASK_MANAGER); + assertThat(kubernetesTaskManagerParameters.getLabels(), is(equalTo(expectedLabels))); + } } -- Gitee From f80c3847d854e0d6a62577cd1e998b57b67fc9f6 Mon Sep 17 00:00:00 2001 From: Canbin Zheng Date: Wed, 25 Mar 2020 14:35:48 +0800 Subject: [PATCH 281/885] [FLINK-16625][utils] Extract BootstrapTools#getEnvironmentVariables to ConfigurationUtils#getPrefixedKeyValuePairs This closes #11458 . --- .../configuration/ConfigurationUtils.java | 15 ++++++++++++++ .../configuration/ConfigurationUtilsTest.java | 20 ++++++++++++++++++- .../AbstractKubernetesParameters.java | 10 ---------- .../KubernetesJobManagerParameters.java | 3 ++- .../clusterframework/BootstrapTools.java | 18 ----------------- .../clusterframework/BootstrapToolsTest.java | 5 +++-- .../flink/yarn/YarnClusterDescriptor.java | 3 ++- 7 files changed, 41 insertions(+), 33 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigurationUtils.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigurationUtils.java index 745d474ee5..9eda0e467c 100755 --- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigurationUtils.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigurationUtils.java @@ -219,6 +219,21 @@ public class ConfigurationUtils { return configs; } + /** + * Extract and parse Flink configuration properties with a given name prefix and + * return the result as a Map. + */ + public static Map getPrefixedKeyValuePairs(String prefix, Configuration configuration) { + Map result = new HashMap<>(); + for (Map.Entry entry: configuration.toMap().entrySet()) { + if (entry.getKey().startsWith(prefix) && entry.getKey().length() > prefix.length()) { + String key = entry.getKey().substring(prefix.length()); + result.put(key, entry.getValue()); + } + } + return result; + } + // Make sure that we cannot instantiate this class private ConfigurationUtils() { } diff --git a/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationUtilsTest.java b/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationUtilsTest.java index fa9345a629..be5d340a30 100644 --- a/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationUtilsTest.java +++ b/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationUtilsTest.java @@ -55,7 +55,7 @@ public class ConfigurationUtilsTest extends TestLogger { } @Test - public void testHideSensitiveValues() { + public void testHideSensitiveValues() { final Map keyValuePairs = new HashMap<>(); keyValuePairs.put("foobar", "barfoo"); final String secretKey1 = "secret.key"; @@ -74,4 +74,22 @@ public class ConfigurationUtilsTest extends TestLogger { assertThat(hiddenSensitiveValues, is(equalTo(expectedKeyValuePairs))); } + @Test + public void testGetPrefixedKeyValuePairs() { + final String prefix = "test.prefix."; + final Map expectedKeyValuePairs = new HashMap() { + { + put("k1", "v1"); + put("k2", "v2"); + } + }; + + final Configuration configuration = new Configuration(); + expectedKeyValuePairs.forEach((k, v) -> configuration.setString(prefix + k, v)); + + final Map resultKeyValuePairs = ConfigurationUtils.getPrefixedKeyValuePairs(prefix, configuration); + + assertThat(resultKeyValuePairs, is(equalTo(expectedKeyValuePairs))); + } + } diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/AbstractKubernetesParameters.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/AbstractKubernetesParameters.java index 8e7c443d5c..4e0916a74a 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/AbstractKubernetesParameters.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/AbstractKubernetesParameters.java @@ -22,7 +22,6 @@ import org.apache.flink.client.cli.CliFrontend; import org.apache.flink.configuration.Configuration; import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; import org.apache.flink.kubernetes.utils.Constants; -import org.apache.flink.runtime.clusterframework.BootstrapTools; import io.fabric8.kubernetes.api.model.LocalObjectReference; import org.apache.commons.lang3.StringUtils; @@ -163,13 +162,4 @@ public abstract class AbstractKubernetesParameters implements KubernetesParamete return Optional.empty(); } - - /** - * Extract container customized environment variable properties with a given name prefix. - * @param envPrefix the given property name prefix - * @return a Map storing with customized environment variable key/value pairs. - */ - protected Map getPrefixedEnvironments(String envPrefix) { - return BootstrapTools.getEnvironmentVariables(envPrefix, flinkConfig); - } } diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java index 3f1460159c..cff893bce7 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java @@ -21,6 +21,7 @@ package org.apache.flink.kubernetes.kubeclient.parameters; import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ConfigurationUtils; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.ResourceManagerOptions; import org.apache.flink.configuration.RestOptions; @@ -63,7 +64,7 @@ public class KubernetesJobManagerParameters extends AbstractKubernetesParameters @Override public Map getEnvironments() { - return getPrefixedEnvironments(ResourceManagerOptions.CONTAINERIZED_MASTER_ENV_PREFIX); + return ConfigurationUtils.getPrefixedKeyValuePairs(ResourceManagerOptions.CONTAINERIZED_MASTER_ENV_PREFIX, flinkConfig); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java index 494d1509b5..cc294b213f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java @@ -713,22 +713,4 @@ public class BootstrapTools { } return memory - heapLimit; } - - /** - * Method to extract environment variables from the flinkConfiguration based on the given prefix String. - * - * @param envPrefix Prefix for the environment variables key - * @param flinkConfiguration The Flink config to get the environment variable definition from - */ - public static Map getEnvironmentVariables(String envPrefix, Configuration flinkConfiguration) { - Map result = new HashMap<>(); - for (Map.Entry entry: flinkConfiguration.toMap().entrySet()) { - if (entry.getKey().startsWith(envPrefix) && entry.getKey().length() > envPrefix.length()) { - // remove prefix - String key = entry.getKey().substring(envPrefix.length()); - result.put(key, entry.getValue()); - } - } - return result; - } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/BootstrapToolsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/BootstrapToolsTest.java index cee17cac44..0a9ccf2c24 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/BootstrapToolsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/BootstrapToolsTest.java @@ -23,6 +23,7 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ConfigurationUtils; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.configuration.MemorySize; @@ -531,7 +532,7 @@ public class BootstrapToolsTest extends TestLogger { Configuration testConf = new Configuration(); testConf.setString("containerized.master.env.LD_LIBRARY_PATH", "/usr/lib/native"); - Map res = BootstrapTools.getEnvironmentVariables("containerized.master.env.", testConf); + Map res = ConfigurationUtils.getPrefixedKeyValuePairs("containerized.master.env.", testConf); Assert.assertEquals(1, res.size()); Map.Entry entry = res.entrySet().iterator().next(); @@ -544,7 +545,7 @@ public class BootstrapToolsTest extends TestLogger { Configuration testConf = new Configuration(); testConf.setString("containerized.master.env.", "/usr/lib/native"); - Map res = BootstrapTools.getEnvironmentVariables("containerized.master.env.", testConf); + Map res = ConfigurationUtils.getPrefixedKeyValuePairs("containerized.master.env.", testConf); Assert.assertEquals(0, res.size()); } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java index 4d6f69140d..cde732b4f8 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java @@ -30,6 +30,7 @@ import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.ConfigUtils; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ConfigurationUtils; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.configuration.IllegalConfigurationException; @@ -947,7 +948,7 @@ public class YarnClusterDescriptor implements ClusterDescriptor { final Map appMasterEnv = new HashMap<>(); // set user specified app master environment variables appMasterEnv.putAll( - BootstrapTools.getEnvironmentVariables(ResourceManagerOptions.CONTAINERIZED_MASTER_ENV_PREFIX, configuration)); + ConfigurationUtils.getPrefixedKeyValuePairs(ResourceManagerOptions.CONTAINERIZED_MASTER_ENV_PREFIX, configuration)); // set Flink app class path appMasterEnv.put(YarnConfigKeys.ENV_FLINK_CLASSPATH, classPathBuilder.toString()); -- Gitee From 1ff2bf05214e350c116c0ab88eb61ed1689a94ae Mon Sep 17 00:00:00 2001 From: Yadong Xie Date: Wed, 25 Mar 2020 15:35:16 +0800 Subject: [PATCH 282/885] [FLINK-16604][web] JM config key column has 30% width --- .../configuration/job-manager-configuration.component.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime-web/web-dashboard/src/app/pages/job-manager/configuration/job-manager-configuration.component.html b/flink-runtime-web/web-dashboard/src/app/pages/job-manager/configuration/job-manager-configuration.component.html index e10be5d491..83fd7b9a06 100644 --- a/flink-runtime-web/web-dashboard/src/app/pages/job-manager/configuration/job-manager-configuration.component.html +++ b/flink-runtime-web/web-dashboard/src/app/pages/job-manager/configuration/job-manager-configuration.component.html @@ -22,7 +22,7 @@ [nzShowPagination]="false"> - Key + Key Value -- Gitee From 811c3be9beee6fd3d1c2482e70f1f57a67dc769c Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Thu, 19 Mar 2020 13:20:48 +0100 Subject: [PATCH 283/885] [FLINK-16629] Ignore Streaming bucketing end-to-end test with Hadoop 2.4.1 --- flink-end-to-end-tests/run-nightly-tests.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-end-to-end-tests/run-nightly-tests.sh b/flink-end-to-end-tests/run-nightly-tests.sh index fc854b6dfe..2f69f5df5c 100755 --- a/flink-end-to-end-tests/run-nightly-tests.sh +++ b/flink-end-to-end-tests/run-nightly-tests.sh @@ -152,7 +152,8 @@ run_test "DataSet allround end-to-end test" "$END_TO_END_DIR/test-scripts/test_b run_test "Batch SQL end-to-end test" "$END_TO_END_DIR/test-scripts/test_batch_sql.sh" run_test "Streaming SQL end-to-end test (Old planner)" "$END_TO_END_DIR/test-scripts/test_streaming_sql.sh old" "skip_check_exceptions" run_test "Streaming SQL end-to-end test (Blink planner)" "$END_TO_END_DIR/test-scripts/test_streaming_sql.sh blink" "skip_check_exceptions" -if [[ $PROFILE == *"include-hadoop"* ]]; then +# skip test if hadoop version is 2.4.1 (FLINK-16629) +if [[ $PROFILE == *"include-hadoop"* && $PROFILE != *"hadoop.version=2.4.1"* ]]; then run_test "Streaming bucketing end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_bucketing.sh" "skip_check_exceptions" fi run_test "Streaming File Sink end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_file_sink.sh" "skip_check_exceptions" -- Gitee From e5bca025adafad03d5d83fe7b3c6aa673f613fdb Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Wed, 25 Mar 2020 09:17:11 +0100 Subject: [PATCH 284/885] [FLINK-16663][docs] Interpret versions as strings --- docs/_config.yml | 22 +++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/docs/_config.yml b/docs/_config.yml index 7d4dc7b526..f998865591 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -64,17 +64,17 @@ is_stable: false show_outdated_warning: false previous_docs: - 1.10: http://ci.apache.org/projects/flink/flink-docs-release-1.10 - 1.9: http://ci.apache.org/projects/flink/flink-docs-release-1.9 - 1.8: http://ci.apache.org/projects/flink/flink-docs-release-1.8 - 1.7: http://ci.apache.org/projects/flink/flink-docs-release-1.7 - 1.6: http://ci.apache.org/projects/flink/flink-docs-release-1.6 - 1.5: http://ci.apache.org/projects/flink/flink-docs-release-1.5 - 1.4: http://ci.apache.org/projects/flink/flink-docs-release-1.4 - 1.3: http://ci.apache.org/projects/flink/flink-docs-release-1.3 - 1.2: http://ci.apache.org/projects/flink/flink-docs-release-1.2 - 1.1: http://ci.apache.org/projects/flink/flink-docs-release-1.1 - 1.0: http://ci.apache.org/projects/flink/flink-docs-release-1.0 + '1.10': http://ci.apache.org/projects/flink/flink-docs-release-1.10 + '1.9': http://ci.apache.org/projects/flink/flink-docs-release-1.9 + '1.8': http://ci.apache.org/projects/flink/flink-docs-release-1.8 + '1.7': http://ci.apache.org/projects/flink/flink-docs-release-1.7 + '1.6': http://ci.apache.org/projects/flink/flink-docs-release-1.6 + '1.5': http://ci.apache.org/projects/flink/flink-docs-release-1.5 + '1.4': http://ci.apache.org/projects/flink/flink-docs-release-1.4 + '1.3': http://ci.apache.org/projects/flink/flink-docs-release-1.3 + '1.2': http://ci.apache.org/projects/flink/flink-docs-release-1.2 + '1.1': http://ci.apache.org/projects/flink/flink-docs-release-1.1 + '1.0': http://ci.apache.org/projects/flink/flink-docs-release-1.0 #------------------------------------------------------------------------------ # BUILD CONFIG -- Gitee From 5aedb83db5dda09471cb36bb87e1933042c9385b Mon Sep 17 00:00:00 2001 From: liuyongvs Date: Sat, 21 Mar 2020 15:41:27 +0800 Subject: [PATCH 285/885] [FLINK-16692] Register custom JobListeners throug config. This closes #11471. --- .../generated/deployment_configuration.html | 6 ++ .../configuration/DeploymentOptions.java | 10 ++++ .../flink/api/java/ExecutionEnvironment.java | 17 ++++++ ...ream_execution_environment_completeness.py | 2 +- .../StreamExecutionEnvironment.java | 24 ++++++++ ...onEnvironmentComplexConfigurationTest.java | 59 +++++++++++++++++++ .../scala/StreamExecutionEnvironment.scala | 6 ++ 7 files changed, 123 insertions(+), 1 deletion(-) diff --git a/docs/_includes/generated/deployment_configuration.html b/docs/_includes/generated/deployment_configuration.html index 325dc98201..1f50f6fd3f 100644 --- a/docs/_includes/generated/deployment_configuration.html +++ b/docs/_includes/generated/deployment_configuration.html @@ -14,6 +14,12 @@ Boolean Specifies if the pipeline is submitted in attached or detached mode. + +
    execution.job-listeners
    + (none) + List<String> + Custom JobListeners to be registered with the execution environment. The registered listeners cannot have constructors with arguments. +
    execution.shutdown-on-attached-exit
    false diff --git a/flink-core/src/main/java/org/apache/flink/configuration/DeploymentOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/DeploymentOptions.java index 9ffa93c7e5..f3f7d6bc12 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/DeploymentOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/DeploymentOptions.java @@ -20,6 +20,8 @@ package org.apache.flink.configuration; import org.apache.flink.annotation.PublicEvolving; +import java.util.List; + import static org.apache.flink.configuration.ConfigOptions.key; /** @@ -46,4 +48,12 @@ public class DeploymentOptions { .defaultValue(false) .withDescription("If the job is submitted in attached mode, perform a best-effort cluster shutdown " + "when the CLI is terminated abruptly, e.g., in response to a user interrupt, such as typing Ctrl + C."); + + public static final ConfigOption> JOB_LISTENERS = + key("execution.job-listeners") + .stringType() + .asList() + .noDefaultValue() + .withDescription("Custom JobListeners to be registered with the execution environment." + + " The registered listeners cannot have constructors with arguments."); } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java index 3b19bca950..d439a2f646 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java @@ -65,10 +65,13 @@ import org.apache.flink.core.execution.PipelineExecutorServiceLoader; import org.apache.flink.core.fs.Path; import org.apache.flink.types.StringValue; import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.NumberSequenceIterator; import org.apache.flink.util.Preconditions; import org.apache.flink.util.SplittableIterator; import org.apache.flink.util.Visitor; +import org.apache.flink.util.WrappingRuntimeException; import com.esotericsoftware.kryo.Serializer; import org.slf4j.Logger; @@ -396,6 +399,8 @@ public class ExecutionEnvironment { */ @PublicEvolving public void configure(ReadableConfig configuration, ClassLoader classLoader) { + configuration.getOptional(DeploymentOptions.JOB_LISTENERS) + .ifPresent(listeners -> registerCustomListeners(classLoader, listeners)); configuration.getOptional(PipelineOptions.CACHED_FILES) .ifPresent(f -> { this.cacheFile.clear(); @@ -404,6 +409,18 @@ public class ExecutionEnvironment { config.configure(configuration, classLoader); } + private void registerCustomListeners(final ClassLoader classLoader, final List listeners) { + for (String listener : listeners) { + try { + final JobListener jobListener = InstantiationUtil.instantiate( + listener, JobListener.class, classLoader); + jobListeners.add(jobListener); + } catch (FlinkException e) { + throw new WrappingRuntimeException("Could not load JobListener : " + listener, e); + } + } + } + // -------------------------------------------------------------------------------------------- // Data set creations // -------------------------------------------------------------------------------------------- diff --git a/flink-python/pyflink/datastream/tests/test_stream_execution_environment_completeness.py b/flink-python/pyflink/datastream/tests/test_stream_execution_environment_completeness.py index a1a325fa09..b114ef4c38 100644 --- a/flink-python/pyflink/datastream/tests/test_stream_execution_environment_completeness.py +++ b/flink-python/pyflink/datastream/tests/test_stream_execution_environment_completeness.py @@ -49,7 +49,7 @@ class StreamExecutionEnvironmentCompletenessTests(PythonAPICompletenessTestCase, 'createInput', 'createLocalEnvironmentWithWebUI', 'fromCollection', 'socketTextStream', 'initializeContextEnvironment', 'readTextFile', 'addSource', 'setNumberOfExecutionRetries', 'configure', 'executeAsync', 'registerJobListener', - 'clearJobListeners'} + 'clearJobListeners', 'getJobListeners'} if __name__ == '__main__': diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java index 7a852eed92..cd684fcc3b 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java @@ -84,6 +84,8 @@ import org.apache.flink.streaming.api.graph.StreamGraphGenerator; import org.apache.flink.streaming.api.operators.StreamSource; import org.apache.flink.util.DynamicCodeLoadingException; import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.Preconditions; import org.apache.flink.util.SplittableIterator; import org.apache.flink.util.StringUtils; @@ -237,6 +239,14 @@ public class StreamExecutionEnvironment { return cacheFile; } + /** + * Gets the config JobListeners. + */ + @PublicEvolving + public List getJobListeners() { + return jobListeners; + } + /** * Sets the parallelism for operations executed through this environment. * Setting a parallelism of x here will cause all operators (such as map, @@ -754,6 +764,8 @@ public class StreamExecutionEnvironment { .ifPresent(c -> this.isChainingEnabled = c); configuration.getOptional(ExecutionOptions.BUFFER_TIMEOUT) .ifPresent(t -> this.setBufferTimeout(t.toMillis())); + configuration.getOptional(DeploymentOptions.JOB_LISTENERS) + .ifPresent(listeners -> registerCustomListeners(classLoader, listeners)); configuration.getOptional(PipelineOptions.CACHED_FILES) .ifPresent(f -> { this.cacheFile.clear(); @@ -763,6 +775,18 @@ public class StreamExecutionEnvironment { checkpointCfg.configure(configuration); } + private void registerCustomListeners(final ClassLoader classLoader, final List listeners) { + for (String listener : listeners) { + try { + final JobListener jobListener = InstantiationUtil.instantiate( + listener, JobListener.class, classLoader); + jobListeners.add(jobListener); + } catch (FlinkException e) { + throw new WrappingRuntimeException("Could not load JobListener : " + listener, e); + } + } + } + private StateBackend loadStateBackend(ReadableConfig configuration, ClassLoader classLoader) { try { return StateBackendLoader.loadStateBackendFromConfig( diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironmentComplexConfigurationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironmentComplexConfigurationTest.java index 67f89e95db..d966afcdc1 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironmentComplexConfigurationTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironmentComplexConfigurationTest.java @@ -18,19 +18,28 @@ package org.apache.flink.streaming.api.environment; +import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.cache.DistributedCache; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.ConfigUtils; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.DeploymentOptions; import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.core.execution.JobListener; import org.apache.flink.runtime.state.StateBackend; import org.apache.flink.runtime.state.memory.MemoryStateBackend; import org.junit.Test; +import javax.annotation.Nullable; + import java.util.Arrays; +import java.util.List; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.instanceOf; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; /** @@ -97,4 +106,54 @@ public class StreamExecutionEnvironmentComplexConfigurationTest { Tuple2.of("file3", new DistributedCache.DistributedCacheEntry("/tmp3", true)) ))); } + + @Test + public void testLoadingListenersFromConfiguration() { + StreamExecutionEnvironment envFromConfiguration = StreamExecutionEnvironment.getExecutionEnvironment(); + List listenersClass = Arrays.asList(BasicJobSubmittedCounter.class, BasicJobExecutedCounter.class); + + Configuration configuration = new Configuration(); + ConfigUtils.encodeCollectionToConfig(configuration, DeploymentOptions.JOB_LISTENERS, listenersClass, Class::getName); + + // mutate config according to configuration + envFromConfiguration.configure(configuration, Thread.currentThread().getContextClassLoader()); + + assertEquals(envFromConfiguration.getJobListeners().size(), 2); + assertThat(envFromConfiguration.getJobListeners().get(0), instanceOf(BasicJobSubmittedCounter.class)); + assertThat(envFromConfiguration.getJobListeners().get(1), instanceOf(BasicJobExecutedCounter.class)); + } + + /** + * JobSubmitted counter listener for unit test. + */ + public static class BasicJobSubmittedCounter implements JobListener { + private int count = 0; + + @Override + public void onJobSubmitted(@Nullable JobClient jobClient, @Nullable Throwable throwable) { + this.count = this.count + 1; + } + + @Override + public void onJobExecuted(@Nullable JobExecutionResult jobExecutionResult, @Nullable Throwable throwable) { + + } + } + + /** + * JobExecuted counter listener for unit test. + */ + public static class BasicJobExecutedCounter implements JobListener { + private int count = 0; + + @Override + public void onJobSubmitted(@Nullable JobClient jobClient, @Nullable Throwable throwable) { + this.count = this.count + 1; + } + + @Override + public void onJobExecuted(@Nullable JobExecutionResult jobExecutionResult, @Nullable Throwable throwable) { + + } + } } diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala index 4a4033deb3..0c7dd6c5d3 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala @@ -57,6 +57,12 @@ class StreamExecutionEnvironment(javaEnv: JavaEnv) { */ def getCachedFiles = javaEnv.getCachedFiles + /** + * Gets the config JobListeners. + */ + @PublicEvolving + def getJobListeners = javaEnv.getJobListeners + /** * Sets the parallelism for operations executed through this environment. * Setting a parallelism of x here will cause all operators (such as join, map, reduce) to run -- Gitee From 89ceb2a59c5b8bb1e5f421e4c2568b7603fe7fb5 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Wed, 4 Mar 2020 15:47:55 +0100 Subject: [PATCH 286/885] [FLINK-16316][task] Remove StreamTask dependency from AbstractStreamOperator#snapshotState --- .../api/operators/AbstractStreamOperator.java | 3 -- .../runtime/tasks/CheckpointingOperation.java | 33 ++++++++++++++++--- .../streaming/runtime/tasks/StreamTask.java | 3 +- 3 files changed, 30 insertions(+), 9 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java index 532c57f1be..2f8d0a12c1 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java @@ -431,9 +431,6 @@ public abstract class AbstractStreamOperator String snapshotFailMessage = "Could not complete snapshot " + checkpointId + " for operator " + getOperatorName() + "."; - if (!getContainingTask().isCanceled()) { - LOG.info(snapshotFailMessage, snapshotException); - } try { snapshotContext.closeExceptionally(); } catch (IOException e) { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/CheckpointingOperation.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/CheckpointingOperation.java index cb88ea0dbd..81a6fd8028 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/CheckpointingOperation.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/CheckpointingOperation.java @@ -33,6 +33,7 @@ import org.slf4j.LoggerFactory; import java.util.HashMap; import java.util.concurrent.ExecutorService; +import java.util.function.Supplier; final class CheckpointingOperation { @@ -48,7 +49,8 @@ final class CheckpointingOperation { CloseableRegistry closeableRegistry, ExecutorService threadPool, Environment environment, - AsyncExceptionHandler asyncExceptionHandler) throws Exception { + AsyncExceptionHandler asyncExceptionHandler, + Supplier isCanceled) throws Exception { Preconditions.checkNotNull(checkpointMetaData); Preconditions.checkNotNull(checkpointOptions); @@ -66,11 +68,12 @@ final class CheckpointingOperation { try { for (StreamOperatorWrapper operatorWrapper : operatorChain.getAllOperators(true)) { StreamOperator op = operatorWrapper.getStreamOperator(); - OperatorSnapshotFutures snapshotInProgress = op.snapshotState( - checkpointMetaData.getCheckpointId(), - checkpointMetaData.getTimestamp(), + OperatorSnapshotFutures snapshotInProgress = checkpointStreamOperator( + op, + checkpointMetaData, checkpointOptions, - storageLocation); + storageLocation, + isCanceled); operatorSnapshotsInProgress.put(op.getOperatorID(), snapshotInProgress); } @@ -133,4 +136,24 @@ final class CheckpointingOperation { } } + private static OperatorSnapshotFutures checkpointStreamOperator( + StreamOperator op, + CheckpointMetaData checkpointMetaData, + CheckpointOptions checkpointOptions, + CheckpointStreamFactory storageLocation, + Supplier isCanceled) throws Exception { + try { + return op.snapshotState( + checkpointMetaData.getCheckpointId(), + checkpointMetaData.getTimestamp(), + checkpointOptions, + storageLocation); + } + catch (Exception ex) { + if (!isCanceled.get()) { + LOG.info(ex.getMessage(), ex); + } + throw ex; + } + } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java index dd7cd4d1ab..f2972cb9c6 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java @@ -907,7 +907,8 @@ public abstract class StreamTask> getCancelables(), getAsyncOperationsThreadPool(), getEnvironment(), - this); + this, + this::isCanceled); } // ------------------------------------------------------------------------ -- Gitee From c5eb0e52938f1246dc9b5d07b2a2931743ece035 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Wed, 4 Mar 2020 16:23:50 +0100 Subject: [PATCH 287/885] [FLINK-16316][operators] Remove chaining strategy methods from the StreamOperator interface Those methods do not have any reason to be on the StreamOperator level since we introduced StreamOperatorFactory concept, so they should be moved to SetupableStreamOperator --- .../flink/state/api/output/SnapshotUtilsTest.java | 12 ------------ .../api/operators/SetupableStreamOperator.java | 4 ++++ .../api/operators/SimpleOperatorFactory.java | 8 ++++++-- .../streaming/api/operators/StreamOperator.java | 4 ---- .../AbstractUdfStreamOperatorLifecycleTest.java | 2 -- 5 files changed, 10 insertions(+), 20 deletions(-) diff --git a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/output/SnapshotUtilsTest.java b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/output/SnapshotUtilsTest.java index 01bcbbc776..de1f146e52 100644 --- a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/output/SnapshotUtilsTest.java +++ b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/output/SnapshotUtilsTest.java @@ -26,7 +26,6 @@ import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.state.CheckpointStorageWorkerView; import org.apache.flink.runtime.state.CheckpointStreamFactory; import org.apache.flink.runtime.state.ttl.mock.MockStateBackend; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures; import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -113,17 +112,6 @@ public class SnapshotUtilsTest { ACTUAL_ORDER_TRACKING.add("setKeyContextElement2"); } - @Override - public ChainingStrategy getChainingStrategy() { - ACTUAL_ORDER_TRACKING.add("getChainingStrategy"); - return null; - } - - @Override - public void setChainingStrategy(ChainingStrategy strategy) { - ACTUAL_ORDER_TRACKING.add("setChainingStrategy"); - } - @Override public MetricGroup getMetricGroup() { ACTUAL_ORDER_TRACKING.add("getMetricGroup"); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SetupableStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SetupableStreamOperator.java index 78fcf3077e..0fe8348ebe 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SetupableStreamOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SetupableStreamOperator.java @@ -38,4 +38,8 @@ public interface SetupableStreamOperator { * Initializes the operator. Sets access to the context and the output. */ void setup(StreamTask containingTask, StreamConfig config, Output> output); + + ChainingStrategy getChainingStrategy(); + + void setChainingStrategy(ChainingStrategy strategy); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SimpleOperatorFactory.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SimpleOperatorFactory.java index d9ee68b160..1f43c2e8f7 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SimpleOperatorFactory.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SimpleOperatorFactory.java @@ -61,7 +61,9 @@ public class SimpleOperatorFactory extends AbstractStreamOperatorFactory operator) { this.operator = checkNotNull(operator); - this.chainingStrategy = operator.getChainingStrategy(); + if (operator instanceof SetupableStreamOperator) { + this.chainingStrategy = ((SetupableStreamOperator) operator).getChainingStrategy(); + } } public StreamOperator getOperator() { @@ -84,7 +86,9 @@ public class SimpleOperatorFactory extends AbstractStreamOperatorFactory extends CheckpointListener, KeyContext, Dis void setKeyContextElement2(StreamRecord record) throws Exception; - ChainingStrategy getChainingStrategy(); - - void setChainingStrategy(ChainingStrategy strategy); - MetricGroup getMetricGroup(); OperatorID getOperatorID(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java index 1c2240e70e..ea91bd0f81 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java @@ -89,7 +89,6 @@ public class AbstractUdfStreamOperatorLifecycleTest { private static final String ALL_METHODS_STREAM_OPERATOR = "[" + "close[], " + "dispose[], " + - "getChainingStrategy[], " + "getCurrentKey[], " + "getMetricGroup[], " + "getOperatorID[], " + @@ -97,7 +96,6 @@ public class AbstractUdfStreamOperatorLifecycleTest { "notifyCheckpointComplete[long], " + "open[], " + "prepareSnapshotPreBarrier[long], " + - "setChainingStrategy[class org.apache.flink.streaming.api.operators.ChainingStrategy], " + "setCurrentKey[class java.lang.Object], " + "setKeyContextElement1[class org.apache.flink.streaming.runtime.streamrecord.StreamRecord], " + "setKeyContextElement2[class org.apache.flink.streaming.runtime.streamrecord.StreamRecord], " + -- Gitee From cb4c32f35987fdfc7e22a789f8888f13c0afb484 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Wed, 4 Mar 2020 17:34:30 +0100 Subject: [PATCH 288/885] [FLINK-16316][operators] Pass StreamTaskStateInitializer to operators from outside This removes another dependency on the StreamTask from AbstractStreamOperator --- .../flink/state/api/output/BoundedStreamTask.java | 2 +- .../flink/state/api/output/SnapshotUtilsTest.java | 3 ++- .../api/operators/AbstractStreamOperator.java | 4 +--- .../streaming/api/operators/StreamOperator.java | 2 +- .../streaming/runtime/tasks/OperatorChain.java | 14 +++++++------- .../flink/streaming/runtime/tasks/StreamTask.java | 2 +- .../AbstractUdfStreamOperatorLifecycleTest.java | 2 +- .../util/AbstractStreamOperatorTestHarness.java | 2 +- .../runtime/StreamTaskSelectiveReadingITCase.java | 3 +-- 9 files changed, 16 insertions(+), 18 deletions(-) diff --git a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/output/BoundedStreamTask.java b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/output/BoundedStreamTask.java index 7a801a48c3..2aacaa9f4a 100644 --- a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/output/BoundedStreamTask.java +++ b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/output/BoundedStreamTask.java @@ -81,7 +81,7 @@ class BoundedStreamTask & Bo configuration, new CollectorWrapper<>(collector)); headOperator = headOperatorAndTimeService.f0; - headOperator.initializeState(); + headOperator.initializeState(createStreamTaskStateInitializer()); headOperator.open(); } diff --git a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/output/SnapshotUtilsTest.java b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/output/SnapshotUtilsTest.java index de1f146e52..35266ef8c3 100644 --- a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/output/SnapshotUtilsTest.java +++ b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/output/SnapshotUtilsTest.java @@ -28,6 +28,7 @@ import org.apache.flink.runtime.state.CheckpointStreamFactory; import org.apache.flink.runtime.state.ttl.mock.MockStateBackend; import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures; import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamTaskStateInitializer; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.junit.Assert; @@ -98,7 +99,7 @@ public class SnapshotUtilsTest { } @Override - public void initializeState() throws Exception { + public void initializeState(StreamTaskStateInitializer streamTaskStateManager) throws Exception { ACTUAL_ORDER_TRACKING.add("initializeState"); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java index 2f8d0a12c1..98f743fe91 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java @@ -248,7 +248,7 @@ public abstract class AbstractStreamOperator } @Override - public final void initializeState() throws Exception { + public final void initializeState(StreamTaskStateInitializer streamTaskStateManager) throws Exception { final TypeSerializer keySerializer = config.getStateKeySerializer(getUserCodeClassloader()); @@ -256,8 +256,6 @@ public abstract class AbstractStreamOperator Preconditions.checkNotNull(getContainingTask()); final CloseableRegistry streamTaskCloseableRegistry = Preconditions.checkNotNull(containingTask.getCancelables()); - final StreamTaskStateInitializer streamTaskStateManager = - Preconditions.checkNotNull(containingTask.createStreamTaskStateInitializer()); final StreamOperatorStateContext context = streamTaskStateManager.streamOperatorStateContext( diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java index f5be3785b6..69a505661b 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java @@ -127,7 +127,7 @@ public interface StreamOperator extends CheckpointListener, KeyContext, Dis /** * Provides a context to initialize all state in the operator. */ - void initializeState() throws Exception; + void initializeState(StreamTaskStateInitializer streamTaskStateManager) throws Exception; // ------------------------------------------------------------------------ // miscellaneous diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java index 52c01747e6..3e1e3f0fda 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java @@ -47,6 +47,7 @@ import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.api.operators.StreamOperatorFactory; import org.apache.flink.streaming.api.operators.StreamOperatorFactoryUtil; +import org.apache.flink.streaming.api.operators.StreamTaskStateInitializer; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.io.RecordWriterOutput; import org.apache.flink.streaming.runtime.metrics.WatermarkGauge; @@ -282,15 +283,14 @@ public class OperatorChain> implements Strea } /** - * Executes {@link StreamOperator#initializeState()} followed by {@link StreamOperator#open()} - * of each operator in the chain of this {@link StreamTask}. State initialization and opening - * happens from tail to head operator in the chain, contrary to {@link StreamOperator#close()} - * which happens head to tail(see {@link #closeOperators(StreamTaskActionExecutor)}). + * Initialize state and open all operators in the chain from tail to head, + * contrary to {@link StreamOperator#close()} which happens head to tail + * (see {@link #closeOperators(StreamTaskActionExecutor)}). */ - protected void initializeStateAndOpenOperators() throws Exception { + protected void initializeStateAndOpenOperators(StreamTaskStateInitializer streamTaskStateInitializer) throws Exception { for (StreamOperatorWrapper operatorWrapper : getAllOperators(true)) { StreamOperator operator = operatorWrapper.getStreamOperator(); - operator.initializeState(); + operator.initializeState(streamTaskStateInitializer); operator.open(); } } @@ -298,7 +298,7 @@ public class OperatorChain> implements Strea /** * Closes all operators in a chain effect way. Closing happens from head to tail operator * in the chain, contrary to {@link StreamOperator#open()} which happens tail to head - * (see {@link #initializeStateAndOpenOperators()}). + * (see {@link #initializeStateAndOpenOperators(StreamTaskStateInitializer)}). */ protected void closeOperators(StreamTaskActionExecutor actionExecutor) throws Exception { if (headOperatorWrapper != null) { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java index f2972cb9c6..d3a79b6eb1 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java @@ -433,7 +433,7 @@ public abstract class StreamTask> // both the following operations are protected by the lock // so that we avoid race conditions in the case that initializeState() // registers a timer, that fires before the open() is called. - operatorChain.initializeStateAndOpenOperators(); + operatorChain.initializeStateAndOpenOperators(createStreamTaskStateInitializer()); }); isRunning = true; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java index ea91bd0f81..5fb8830ef8 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java @@ -92,7 +92,7 @@ public class AbstractUdfStreamOperatorLifecycleTest { "getCurrentKey[], " + "getMetricGroup[], " + "getOperatorID[], " + - "initializeState[], " + + "initializeState[interface org.apache.flink.streaming.api.operators.StreamTaskStateInitializer], " + "notifyCheckpointComplete[long], " + "open[], " + "prepareSnapshotPreBarrier[long], " + diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java index 7d0cd510e5..13624e69cb 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java @@ -495,7 +495,7 @@ public class AbstractStreamOperatorTestHarness implements AutoCloseable { } } - operator.initializeState(); + operator.initializeState(mockTask.createStreamTaskStateInitializer()); initializeCalled = true; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/StreamTaskSelectiveReadingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/StreamTaskSelectiveReadingITCase.java index 28840d9633..7fa49f8ae2 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/StreamTaskSelectiveReadingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/StreamTaskSelectiveReadingITCase.java @@ -25,7 +25,6 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; import org.apache.flink.streaming.api.operators.ChainingStrategy; -import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; import org.apache.flink.streaming.util.TestSequentialReadingStreamOperator; import org.apache.flink.test.streaming.runtime.util.TestListResultSink; @@ -60,7 +59,7 @@ public class StreamTaskSelectiveReadingITCase { .setParallelism(2); TestListResultSink resultSink = new TestListResultSink<>(); - TwoInputStreamOperator twoInputStreamOperator = new TestSequentialReadingStreamOperator("Operator0"); + TestSequentialReadingStreamOperator twoInputStreamOperator = new TestSequentialReadingStreamOperator("Operator0"); twoInputStreamOperator.setChainingStrategy(ChainingStrategy.NEVER); source0.connect(source1) -- Gitee From 63d6add140a3146fe4034515b458fe9bd6a83b97 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Wed, 11 Mar 2020 09:34:47 +0100 Subject: [PATCH 289/885] [hotfix][test] Fix formatting in AbstractStreamOperatorTest --- .../operators/AbstractStreamOperatorTest.java | 109 +++++++++--------- 1 file changed, 54 insertions(+), 55 deletions(-) diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java index f9d9aa5e50..93dca8fa10 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java @@ -84,13 +84,12 @@ import static org.powermock.api.mockito.PowerMockito.whenNew; @PrepareForTest(AbstractStreamOperator.class) @PowerMockIgnore({"java.*", "javax.*", "org.slf4j.*", "org.apache.log4j.*"}) public class AbstractStreamOperatorTest { - @Test public void testStateDoesNotInterfere() throws Exception { TestOperator testOperator = new TestOperator(); KeyedOneInputStreamOperatorTestHarness, String> testHarness = - new KeyedOneInputStreamOperatorTestHarness<>(testOperator, new TestKeySelector(), BasicTypeInfo.INT_TYPE_INFO); + new KeyedOneInputStreamOperatorTestHarness<>(testOperator, new TestKeySelector(), BasicTypeInfo.INT_TYPE_INFO); testHarness.open(); @@ -101,8 +100,8 @@ public class AbstractStreamOperatorTest { testHarness.processElement(new Tuple2<>(0, "EMIT_STATE"), 0); assertThat( - extractResult(testHarness), - contains("ON_ELEMENT:1:CIAO", "ON_ELEMENT:0:HELLO")); + extractResult(testHarness), + contains("ON_ELEMENT:1:CIAO", "ON_ELEMENT:0:HELLO")); } /** @@ -114,7 +113,7 @@ public class AbstractStreamOperatorTest { TestOperator testOperator = new TestOperator(); KeyedOneInputStreamOperatorTestHarness, String> testHarness = - new KeyedOneInputStreamOperatorTestHarness<>(testOperator, new TestKeySelector(), BasicTypeInfo.INT_TYPE_INFO); + new KeyedOneInputStreamOperatorTestHarness<>(testOperator, new TestKeySelector(), BasicTypeInfo.INT_TYPE_INFO); testHarness.open(); @@ -130,14 +129,14 @@ public class AbstractStreamOperatorTest { testHarness.processWatermark(10L); assertThat( - extractResult(testHarness), - contains("ON_EVENT_TIME:HELLO")); + extractResult(testHarness), + contains("ON_EVENT_TIME:HELLO")); testHarness.processWatermark(20L); assertThat( - extractResult(testHarness), - contains("ON_EVENT_TIME:CIAO")); + extractResult(testHarness), + contains("ON_EVENT_TIME:CIAO")); } /** @@ -149,7 +148,7 @@ public class AbstractStreamOperatorTest { TestOperator testOperator = new TestOperator(); KeyedOneInputStreamOperatorTestHarness, String> testHarness = - new KeyedOneInputStreamOperatorTestHarness<>(testOperator, new TestKeySelector(), BasicTypeInfo.INT_TYPE_INFO); + new KeyedOneInputStreamOperatorTestHarness<>(testOperator, new TestKeySelector(), BasicTypeInfo.INT_TYPE_INFO); testHarness.open(); @@ -165,14 +164,14 @@ public class AbstractStreamOperatorTest { testHarness.setProcessingTime(10L); assertThat( - extractResult(testHarness), - contains("ON_PROC_TIME:HELLO")); + extractResult(testHarness), + contains("ON_PROC_TIME:HELLO")); testHarness.setProcessingTime(20L); assertThat( - extractResult(testHarness), - contains("ON_PROC_TIME:CIAO")); + extractResult(testHarness), + contains("ON_PROC_TIME:CIAO")); } /** @@ -183,7 +182,7 @@ public class AbstractStreamOperatorTest { TestOperator testOperator = new TestOperator(); KeyedOneInputStreamOperatorTestHarness, String> testHarness = - new KeyedOneInputStreamOperatorTestHarness<>(testOperator, new TestKeySelector(), BasicTypeInfo.INT_TYPE_INFO); + new KeyedOneInputStreamOperatorTestHarness<>(testOperator, new TestKeySelector(), BasicTypeInfo.INT_TYPE_INFO); testHarness.open(); @@ -201,10 +200,10 @@ public class AbstractStreamOperatorTest { TestOperator testOperator1 = new TestOperator(); KeyedOneInputStreamOperatorTestHarness, String> testHarness1 = - new KeyedOneInputStreamOperatorTestHarness<>( - testOperator1, - new TestKeySelector(), - BasicTypeInfo.INT_TYPE_INFO); + new KeyedOneInputStreamOperatorTestHarness<>( + testOperator1, + new TestKeySelector(), + BasicTypeInfo.INT_TYPE_INFO); testHarness1.setProcessingTime(0L); @@ -215,14 +214,14 @@ public class AbstractStreamOperatorTest { testHarness1.setProcessingTime(10L); assertThat( - extractResult(testHarness1), - contains("ON_PROC_TIME:HELLO")); + extractResult(testHarness1), + contains("ON_PROC_TIME:HELLO")); testHarness1.setProcessingTime(20L); assertThat( - extractResult(testHarness1), - contains("ON_PROC_TIME:CIAO")); + extractResult(testHarness1), + contains("ON_PROC_TIME:CIAO")); } @@ -234,7 +233,7 @@ public class AbstractStreamOperatorTest { TestOperator testOperator = new TestOperator(); KeyedOneInputStreamOperatorTestHarness, String> testHarness = - new KeyedOneInputStreamOperatorTestHarness<>(testOperator, new TestKeySelector(), BasicTypeInfo.INT_TYPE_INFO); + new KeyedOneInputStreamOperatorTestHarness<>(testOperator, new TestKeySelector(), BasicTypeInfo.INT_TYPE_INFO); testHarness.open(); @@ -249,14 +248,14 @@ public class AbstractStreamOperatorTest { testHarness.processWatermark(20L); assertThat( - extractResult(testHarness), - contains("ON_EVENT_TIME:HELLO")); + extractResult(testHarness), + contains("ON_EVENT_TIME:HELLO")); testHarness.setProcessingTime(10L); assertThat( - extractResult(testHarness), - contains("ON_PROC_TIME:HELLO")); + extractResult(testHarness), + contains("ON_PROC_TIME:HELLO")); } /** @@ -281,13 +280,13 @@ public class AbstractStreamOperatorTest { TestOperator testOperator = new TestOperator(); KeyedOneInputStreamOperatorTestHarness, String> testHarness = - new KeyedOneInputStreamOperatorTestHarness<>( - testOperator, - new TestKeySelector(), - BasicTypeInfo.INT_TYPE_INFO, - maxParallelism, - 1, /* num subtasks */ - 0 /* subtask index */); + new KeyedOneInputStreamOperatorTestHarness<>( + testOperator, + new TestKeySelector(), + BasicTypeInfo.INT_TYPE_INFO, + maxParallelism, + 1, /* num subtasks */ + 0 /* subtask index */); testHarness.open(); @@ -314,13 +313,13 @@ public class AbstractStreamOperatorTest { TestOperator testOperator1 = new TestOperator(); KeyedOneInputStreamOperatorTestHarness, String> testHarness1 = - new KeyedOneInputStreamOperatorTestHarness<>( - testOperator1, - new TestKeySelector(), - BasicTypeInfo.INT_TYPE_INFO, - maxParallelism, - 2, /* num subtasks */ - 0 /* subtask index */); + new KeyedOneInputStreamOperatorTestHarness<>( + testOperator1, + new TestKeySelector(), + BasicTypeInfo.INT_TYPE_INFO, + maxParallelism, + 2, /* num subtasks */ + 0 /* subtask index */); testHarness1.setup(); testHarness1.initializeState(initState1); @@ -357,13 +356,13 @@ public class AbstractStreamOperatorTest { TestOperator testOperator2 = new TestOperator(); KeyedOneInputStreamOperatorTestHarness, String> testHarness2 = - new KeyedOneInputStreamOperatorTestHarness<>( - testOperator2, - new TestKeySelector(), - BasicTypeInfo.INT_TYPE_INFO, - maxParallelism, - 2, /* num subtasks */ - 1 /* subtask index */); + new KeyedOneInputStreamOperatorTestHarness<>( + testOperator2, + new TestKeySelector(), + BasicTypeInfo.INT_TYPE_INFO, + maxParallelism, + 2, /* num subtasks */ + 1 /* subtask index */); testHarness2.setup(); testHarness2.initializeState(initState2); @@ -687,24 +686,24 @@ public class AbstractStreamOperatorTest { * state or setting timers. */ private static class TestOperator - extends AbstractStreamOperator - implements OneInputStreamOperator, String>, Triggerable { + extends AbstractStreamOperator + implements OneInputStreamOperator, String>, Triggerable { private static final long serialVersionUID = 1L; private transient InternalTimerService timerService; private final ValueStateDescriptor stateDescriptor = - new ValueStateDescriptor<>("state", StringSerializer.INSTANCE); + new ValueStateDescriptor<>("state", StringSerializer.INSTANCE); @Override public void open() throws Exception { super.open(); this.timerService = getInternalTimerService( - "test-timers", - VoidNamespaceSerializer.INSTANCE, - this); + "test-timers", + VoidNamespaceSerializer.INSTANCE, + this); } @Override -- Gitee From a909af142cabe43aad74ed0d5848164c08e81a02 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Tue, 10 Mar 2020 14:09:08 +0100 Subject: [PATCH 290/885] [hotfix][test] Remove no-op tests for AbstractStreamOperator Those two tests were not performing any assertions since [FLINK-13326] (c75af84d44dfb9b883115bf4fd65b6a5989464e4) --- .../operators/AbstractStreamOperatorTest.java | 68 ------------------- 1 file changed, 68 deletions(-) diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java index 93dca8fa10..a4a0dc2c75 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java @@ -493,74 +493,6 @@ public class AbstractStreamOperatorTest { assertTrue(extractResult(testHarness3).isEmpty()); } - /** - * Checks that the state snapshot context is closed after a successful snapshot operation. - */ - @Test - public void testSnapshotMethod() throws Exception { - final long checkpointId = 42L; - final long timestamp = 1L; - - final CloseableRegistry closeableRegistry = new CloseableRegistry(); - - StateSnapshotContextSynchronousImpl context = spy(new StateSnapshotContextSynchronousImpl(0L, 0L)); - - whenNew(StateSnapshotContextSynchronousImpl.class).withAnyArguments().thenReturn(context); - - StreamTask> containingTask = mock(StreamTask.class); - when(containingTask.getCancelables()).thenReturn(closeableRegistry); - - AbstractStreamOperator operator = mock(AbstractStreamOperator.class); - when(operator.snapshotState(anyLong(), anyLong(), any(CheckpointOptions.class), any(CheckpointStreamFactory.class))).thenCallRealMethod(); - doReturn(containingTask).when(operator).getContainingTask(); - - operator.snapshotState( - checkpointId, - timestamp, - CheckpointOptions.forCheckpointWithDefaultLocation(), - new MemCheckpointStreamFactory(Integer.MAX_VALUE)); - - } - - /** - * Tests that the created StateSnapshotContextSynchronousImpl is closed in case of a failing - * Operator#snapshotState(StateSnapshotContextSynchronousImpl) call. - */ - @Test - public void testFailingSnapshotMethod() throws Exception { - final long checkpointId = 42L; - final long timestamp = 1L; - - final Exception failingException = new Exception("Test exception"); - - final CloseableRegistry closeableRegistry = new CloseableRegistry(); - - StateSnapshotContextSynchronousImpl context = mock(StateSnapshotContextSynchronousImpl.class); - - whenNew(StateSnapshotContextSynchronousImpl.class).withAnyArguments().thenReturn(context); - - StreamTask> containingTask = mock(StreamTask.class); - when(containingTask.getCancelables()).thenReturn(closeableRegistry); - - AbstractStreamOperator operator = mock(AbstractStreamOperator.class); - when(operator.snapshotState(anyLong(), anyLong(), any(CheckpointOptions.class), any(CheckpointStreamFactory.class))).thenCallRealMethod(); - doReturn(containingTask).when(operator).getContainingTask(); - - // lets fail when calling the actual snapshotState method - doThrow(failingException).when(operator).snapshotState(eq(context)); - - try { - operator.snapshotState( - checkpointId, - timestamp, - CheckpointOptions.forCheckpointWithDefaultLocation(), - new MemCheckpointStreamFactory(Integer.MAX_VALUE)); - fail("Exception expected."); - } catch (Exception e) { - assertEquals(failingException.getMessage(), e.getCause().getMessage()); - } - } - /** * Tests that a failing snapshot method call to the keyed state backend will trigger the closing * of the StateSnapshotContextSynchronousImpl and the cancellation of the -- Gitee From 7e7bcc49717822991f9f3f3c40902305667366e9 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Fri, 6 Mar 2020 17:32:21 +0100 Subject: [PATCH 291/885] [FLINK-16316][operators] Extract state handling code from AbstractStreamOperator Introduce StreamOperatorStateHandler class for that purpose and move more logic into IntrenalTimeServiceManager. This makes AbstractStreamOperator a simpler class and will allow to deduplicate code with new replacement of AbstractStreamOperator that will come soon. --- .../state/AbstractKeyedStateBackend.java | 5 + .../StateSnapshotContextSynchronousImpl.java | 4 +- .../testutils/ExpectedTestException.java | 4 +- .../api/operators/AbstractStreamOperator.java | 294 +++-------------- .../operators/InternalTimeServiceManager.java | 64 ++++ .../operators/StreamOperatorStateHandler.java | 310 ++++++++++++++++++ .../operators/AbstractStreamOperatorTest.java | 130 -------- .../StreamOperatorStateHandlerTest.java | 193 +++++++++++ ...dCoProcessOperatorWithWatermarkDelay.scala | 3 +- ...yedProcessOperatorWithWatermarkDelay.scala | 3 +- .../operators/CodeGenOperatorFactory.java | 17 +- ...edCoProcessOperatorWithWatermarkDelay.java | 8 +- 12 files changed, 641 insertions(+), 394 deletions(-) create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorStateHandler.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StreamOperatorStateHandlerTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java index a06030cfa7..27cd0586de 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java @@ -356,6 +356,11 @@ public abstract class AbstractKeyedStateBackend implements @VisibleForTesting public abstract int numKeyValueStateEntries(); + @VisibleForTesting + public int numKeyValueStatesByName() { + return keyValueStatesByName.size(); + } + // TODO remove this once heap-based timers are working with RocksDB incremental snapshots! public boolean requiresLegacySynchronousTimerSnapshots() { return false; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateSnapshotContextSynchronousImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateSnapshotContextSynchronousImpl.java index 1e52676938..1f19d32b84 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateSnapshotContextSynchronousImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateSnapshotContextSynchronousImpl.java @@ -59,8 +59,8 @@ public class StateSnapshotContextSynchronousImpl implements StateSnapshotContext /** Output stream for the raw operator state. */ private OperatorStateCheckpointOutputStream operatorStateCheckpointOutputStream; - private RunnableFuture> keyedStateCheckpointClosingFuture; - private RunnableFuture> operatorStateCheckpointClosingFuture; + protected RunnableFuture> keyedStateCheckpointClosingFuture; + protected RunnableFuture> operatorStateCheckpointClosingFuture; @VisibleForTesting public StateSnapshotContextSynchronousImpl(long checkpointId, long checkpointTimestamp) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/ExpectedTestException.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/ExpectedTestException.java index e4a714a8fb..5cb67e57a6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/ExpectedTestException.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/ExpectedTestException.java @@ -26,7 +26,9 @@ package org.apache.flink.runtime.operators.testutils; @SuppressWarnings("serial") public class ExpectedTestException extends RuntimeException { + public static final String MESSAGE = "Expected Test Exception"; + public ExpectedTestException() { - super("Expected Test Exception"); + super(MESSAGE); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java index 98f743fe91..e525c21b5e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java @@ -29,53 +29,38 @@ import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.MetricOptions; import org.apache.flink.core.fs.CloseableRegistry; -import org.apache.flink.core.memory.DataOutputViewStreamWrapper; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.runtime.checkpoint.CheckpointException; -import org.apache.flink.runtime.checkpoint.CheckpointFailureReason; import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup; import org.apache.flink.runtime.metrics.groups.TaskManagerJobMetricGroup; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; -import org.apache.flink.runtime.state.AbstractKeyedStateBackend; import org.apache.flink.runtime.state.CheckpointStreamFactory; -import org.apache.flink.runtime.state.DefaultKeyedStateStore; -import org.apache.flink.runtime.state.KeyGroupRange; -import org.apache.flink.runtime.state.KeyGroupStatePartitionStreamProvider; -import org.apache.flink.runtime.state.KeyGroupsList; import org.apache.flink.runtime.state.KeyedStateBackend; -import org.apache.flink.runtime.state.KeyedStateCheckpointOutputStream; import org.apache.flink.runtime.state.OperatorStateBackend; import org.apache.flink.runtime.state.StateInitializationContext; -import org.apache.flink.runtime.state.StateInitializationContextImpl; -import org.apache.flink.runtime.state.StatePartitionStreamProvider; import org.apache.flink.runtime.state.StateSnapshotContext; -import org.apache.flink.runtime.state.StateSnapshotContextSynchronousImpl; import org.apache.flink.runtime.state.VoidNamespace; import org.apache.flink.runtime.state.VoidNamespaceSerializer; import org.apache.flink.streaming.api.graph.StreamConfig; +import org.apache.flink.streaming.api.operators.StreamOperatorStateHandler.CheckpointedStreamOperator; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.apache.flink.streaming.util.LatencyStats; -import org.apache.flink.util.CloseableIterable; -import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.IOUtils; import org.apache.flink.util.OutputTag; import org.apache.flink.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.Closeable; -import java.io.IOException; import java.io.Serializable; import java.util.Locale; +import java.util.Optional; /** * Base class for all stream operators. Operators that contain a user function should extend the class @@ -93,7 +78,7 @@ import java.util.Locale; */ @PublicEvolving public abstract class AbstractStreamOperator - implements StreamOperator, SetupableStreamOperator, Serializable { + implements StreamOperator, SetupableStreamOperator, CheckpointedStreamOperator, Serializable { private static final long serialVersionUID = 1L; @@ -135,16 +120,9 @@ public abstract class AbstractStreamOperator */ private transient KeySelector stateKeySelector2; - /** Backend for keyed state. This might be empty if we're not on a keyed stream. */ - private transient AbstractKeyedStateBackend keyedStateBackend; + private transient StreamOperatorStateHandler stateHandler; - /** Keyed state store view on the keyed backend. */ - private transient DefaultKeyedStateStore keyedStateStore; - - // ---------------- operator state ------------------ - - /** Operator state backend / store. */ - private transient OperatorStateBackend operatorStateBackend; + private transient InternalTimeServiceManager timeServiceManager; // --------------- Metrics --------------------------- @@ -156,7 +134,6 @@ public abstract class AbstractStreamOperator // ---------------- time handler ------------------ protected transient ProcessingTimeService processingTimeService; - protected transient InternalTimeServiceManager timeServiceManager; // ---------------- two-input operator watermarks ------------------ @@ -267,37 +244,9 @@ public abstract class AbstractStreamOperator streamTaskCloseableRegistry, metrics); - this.operatorStateBackend = context.operatorStateBackend(); - this.keyedStateBackend = context.keyedStateBackend(); - - if (keyedStateBackend != null) { - this.keyedStateStore = new DefaultKeyedStateStore(keyedStateBackend, getExecutionConfig()); - } - + stateHandler = new StreamOperatorStateHandler(context, getExecutionConfig(), streamTaskCloseableRegistry); timeServiceManager = context.internalTimerServiceManager(); - - CloseableIterable keyedStateInputs = context.rawKeyedStateInputs(); - CloseableIterable operatorStateInputs = context.rawOperatorStateInputs(); - - try { - StateInitializationContext initializationContext = new StateInitializationContextImpl( - context.isRestored(), // information whether we restore or start for the first time - operatorStateBackend, // access to operator state backend - keyedStateStore, // access to keyed state backend - keyedStateInputs, // access to keyed state stream - operatorStateInputs); // access to operator state stream - - initializeState(initializationContext); - } finally { - closeFromRegistry(operatorStateInputs, streamTaskCloseableRegistry); - closeFromRegistry(keyedStateInputs, streamTaskCloseableRegistry); - } - } - - private static void closeFromRegistry(Closeable closeable, CloseableRegistry registry) { - if (registry.unregisterCloseable(closeable)) { - IOUtils.closeQuietly(closeable); - } + stateHandler.initializeOperatorState(this); } /** @@ -335,50 +284,8 @@ public abstract class AbstractStreamOperator */ @Override public void dispose() throws Exception { - - Exception exception = null; - - StreamTask containingTask = getContainingTask(); - CloseableRegistry taskCloseableRegistry = containingTask != null ? - containingTask.getCancelables() : - null; - - try { - if (taskCloseableRegistry == null || - taskCloseableRegistry.unregisterCloseable(operatorStateBackend)) { - operatorStateBackend.close(); - } - } catch (Exception e) { - exception = e; - } - - try { - if (taskCloseableRegistry == null || - taskCloseableRegistry.unregisterCloseable(keyedStateBackend)) { - keyedStateBackend.close(); - } - } catch (Exception e) { - exception = ExceptionUtils.firstOrSuppressed(e, exception); - } - - try { - if (operatorStateBackend != null) { - operatorStateBackend.dispose(); - } - } catch (Exception e) { - exception = ExceptionUtils.firstOrSuppressed(e, exception); - } - - try { - if (keyedStateBackend != null) { - keyedStateBackend.dispose(); - } - } catch (Exception e) { - exception = ExceptionUtils.firstOrSuppressed(e, exception); - } - - if (exception != null) { - throw exception; + if (stateHandler != null) { + stateHandler.dispose(); } } @@ -389,55 +296,19 @@ public abstract class AbstractStreamOperator } @Override - public final OperatorSnapshotFutures snapshotState(long checkpointId, long timestamp, CheckpointOptions checkpointOptions, + public final OperatorSnapshotFutures snapshotState( + long checkpointId, + long timestamp, + CheckpointOptions checkpointOptions, CheckpointStreamFactory factory) throws Exception { - - KeyGroupRange keyGroupRange = null != keyedStateBackend ? - keyedStateBackend.getKeyGroupRange() : KeyGroupRange.EMPTY_KEY_GROUP_RANGE; - - OperatorSnapshotFutures snapshotInProgress = new OperatorSnapshotFutures(); - - StateSnapshotContextSynchronousImpl snapshotContext = new StateSnapshotContextSynchronousImpl( + return stateHandler.snapshotState( + this, + Optional.ofNullable(timeServiceManager), + getOperatorName(), checkpointId, timestamp, - factory, - keyGroupRange, - getContainingTask().getCancelables()); - - try { - snapshotState(snapshotContext); - - snapshotInProgress.setKeyedStateRawFuture(snapshotContext.getKeyedStateStreamFuture()); - snapshotInProgress.setOperatorStateRawFuture(snapshotContext.getOperatorStateStreamFuture()); - - if (null != operatorStateBackend) { - snapshotInProgress.setOperatorStateManagedFuture( - operatorStateBackend.snapshot(checkpointId, timestamp, factory, checkpointOptions)); - } - - if (null != keyedStateBackend) { - snapshotInProgress.setKeyedStateManagedFuture( - keyedStateBackend.snapshot(checkpointId, timestamp, factory, checkpointOptions)); - } - } catch (Exception snapshotException) { - try { - snapshotInProgress.cancel(); - } catch (Exception e) { - snapshotException.addSuppressed(e); - } - - String snapshotFailMessage = "Could not complete snapshot " + checkpointId + " for operator " + - getOperatorName() + "."; - - try { - snapshotContext.closeExceptionally(); - } catch (IOException e) { - snapshotException.addSuppressed(e); - } - throw new CheckpointException(snapshotFailMessage, CheckpointFailureReason.CHECKPOINT_DECLINED, snapshotException); - } - - return snapshotInProgress; + checkpointOptions, + factory); } /** @@ -445,41 +316,8 @@ public abstract class AbstractStreamOperator * * @param context context that provides information and means required for taking a snapshot */ + @Override public void snapshotState(StateSnapshotContext context) throws Exception { - final KeyedStateBackend keyedStateBackend = getKeyedStateBackend(); - //TODO all of this can be removed once heap-based timers are integrated with RocksDB incremental snapshots - if (keyedStateBackend instanceof AbstractKeyedStateBackend && - ((AbstractKeyedStateBackend) keyedStateBackend).requiresLegacySynchronousTimerSnapshots()) { - - KeyedStateCheckpointOutputStream out; - - try { - out = context.getRawKeyedOperatorStateOutput(); - } catch (Exception exception) { - throw new Exception("Could not open raw keyed operator state stream for " + - getOperatorName() + '.', exception); - } - - try { - KeyGroupsList allKeyGroups = out.getKeyGroupList(); - for (int keyGroupIdx : allKeyGroups) { - out.startNewKeyGroup(keyGroupIdx); - - timeServiceManager.snapshotStateForKeyGroup( - new DataOutputViewStreamWrapper(out), keyGroupIdx); - } - } catch (Exception exception) { - throw new Exception("Could not write timer service of " + getOperatorName() + - " to checkpoint state stream.", exception); - } finally { - try { - out.close(); - } catch (Exception closeException) { - LOG.warn("Could not close raw keyed operator state stream for {}. This " + - "might have prevented deleting some state data.", getOperatorName(), closeException); - } - } - } } /** @@ -487,15 +325,13 @@ public abstract class AbstractStreamOperator * * @param context context that allows to register different states. */ + @Override public void initializeState(StateInitializationContext context) throws Exception { - } @Override public void notifyCheckpointComplete(long checkpointId) throws Exception { - if (keyedStateBackend != null) { - keyedStateBackend.notifyCheckpointComplete(checkpointId); - } + stateHandler.notifyCheckpointComplete(checkpointId); } // ------------------------------------------------------------------------ @@ -550,11 +386,11 @@ public abstract class AbstractStreamOperator @SuppressWarnings("unchecked") public KeyedStateBackend getKeyedStateBackend() { - return (KeyedStateBackend) keyedStateBackend; + return stateHandler.getKeyedStateBackend(); } public OperatorStateBackend getOperatorStateBackend() { - return operatorStateBackend; + return stateHandler.getOperatorStateBackend(); } /** @@ -578,15 +414,7 @@ public abstract class AbstractStreamOperator protected S getOrCreateKeyedState( TypeSerializer namespaceSerializer, StateDescriptor stateDescriptor) throws Exception { - - if (keyedStateStore != null) { - return keyedStateBackend.getOrCreateKeyedState(namespaceSerializer, stateDescriptor); - } - else { - throw new IllegalStateException("Cannot create partitioned state. " + - "The keyed state backend has not been set." + - "This indicates that the operator is not partitioned/keyed."); - } + return stateHandler.getOrCreateKeyedState(namespaceSerializer, stateDescriptor); } /** @@ -599,20 +427,7 @@ public abstract class AbstractStreamOperator N namespace, TypeSerializer namespaceSerializer, StateDescriptor stateDescriptor) throws Exception { - - /* - TODO: NOTE: This method does a lot of work caching / retrieving states just to update the namespace. - This method should be removed for the sake of namespaces being lazily fetched from the keyed - state backend, or being set on the state directly. - */ - - if (keyedStateStore != null) { - return keyedStateBackend.getPartitionedState(namespace, namespaceSerializer, stateDescriptor); - } else { - throw new RuntimeException("Cannot create partitioned state. The keyed state " + - "backend has not been set. This indicates that the operator is not " + - "partitioned/keyed."); - } + return stateHandler.getPartitionedState(namespace, namespaceSerializer, stateDescriptor); } @Override @@ -636,30 +451,19 @@ public abstract class AbstractStreamOperator @SuppressWarnings({"unchecked", "rawtypes"}) public void setCurrentKey(Object key) { - if (keyedStateBackend != null) { - try { - // need to work around type restrictions - @SuppressWarnings("unchecked,rawtypes") - AbstractKeyedStateBackend rawBackend = (AbstractKeyedStateBackend) keyedStateBackend; - - rawBackend.setCurrentKey(key); - } catch (Exception e) { - throw new RuntimeException("Exception occurred while setting the current key context.", e); - } - } + stateHandler.setCurrentKey(key); } @SuppressWarnings({"unchecked", "rawtypes"}) public Object getCurrentKey() { - if (keyedStateBackend != null) { - return keyedStateBackend.getCurrentKey(); - } else { - throw new UnsupportedOperationException("Key can only be retrieved on KeyedStream."); - } + return stateHandler.getCurrentKey(); } public KeyedStateStore getKeyedStateStore() { - return keyedStateStore; + if (stateHandler == null) { + return null; + } + return stateHandler.getKeyedStateStore().orElse(null); } // ------------------------------------------------------------------------ @@ -773,15 +577,15 @@ public abstract class AbstractStreamOperator String name, TypeSerializer namespaceSerializer, Triggerable triggerable) { - - checkTimerServiceInitialization(); - - // the following casting is to overcome type restrictions. - KeyedStateBackend keyedStateBackend = getKeyedStateBackend(); - TypeSerializer keySerializer = keyedStateBackend.getKeySerializer(); + if (timeServiceManager == null) { + throw new RuntimeException("The timer service has not been initialized."); + } InternalTimeServiceManager keyedTimeServiceHandler = (InternalTimeServiceManager) timeServiceManager; - TimerSerializer timerSerializer = new TimerSerializer<>(keySerializer, namespaceSerializer); - return keyedTimeServiceHandler.getInternalTimerService(name, timerSerializer, triggerable); + return keyedTimeServiceHandler.getInternalTimerService( + name, + namespaceSerializer, + triggerable, + stateHandler.getKeyedStateBackend()); } public void processWatermark(Watermark mark) throws Exception { @@ -791,14 +595,6 @@ public abstract class AbstractStreamOperator output.emitWatermark(mark); } - private void checkTimerServiceInitialization() { - if (getKeyedStateBackend() == null) { - throw new UnsupportedOperationException("Timers can only be used on keyed operators."); - } else if (timeServiceManager == null) { - throw new RuntimeException("The timer service has not been initialized."); - } - } - public void processWatermark1(Watermark mark) throws Exception { input1Watermark = mark.getTimestamp(); long newMin = Math.min(input1Watermark, input2Watermark); @@ -824,13 +620,15 @@ public abstract class AbstractStreamOperator @VisibleForTesting public int numProcessingTimeTimers() { - return timeServiceManager == null ? 0 : - timeServiceManager.numProcessingTimeTimers(); + return timeServiceManager == null ? 0 : timeServiceManager.numProcessingTimeTimers(); } @VisibleForTesting public int numEventTimeTimers() { - return timeServiceManager == null ? 0 : - timeServiceManager.numEventTimeTimers(); + return timeServiceManager == null ? 0 : timeServiceManager.numEventTimeTimers(); + } + + protected Optional> getTimeServiceManager() { + return Optional.ofNullable(timeServiceManager); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimeServiceManager.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimeServiceManager.java index ff48c3fae0..87274b95cb 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimeServiceManager.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimeServiceManager.java @@ -20,20 +20,32 @@ package org.apache.flink.streaming.api.operators; import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.runtime.state.AbstractKeyedStateBackend; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.KeyGroupedInternalPriorityQueue; +import org.apache.flink.runtime.state.KeyGroupsList; +import org.apache.flink.runtime.state.KeyedStateBackend; +import org.apache.flink.runtime.state.KeyedStateCheckpointOutputStream; import org.apache.flink.runtime.state.PriorityQueueSetFactory; +import org.apache.flink.runtime.state.StateSnapshotContext; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; import org.apache.flink.util.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.IOException; import java.io.InputStream; import java.util.Collections; import java.util.HashMap; import java.util.Map; +import static org.apache.flink.util.Preconditions.checkNotNull; + /** * An entity keeping all the time-related services available to all operators extending the * {@link AbstractStreamOperator}. Right now, this is only a @@ -45,6 +57,7 @@ import java.util.Map; */ @Internal public class InternalTimeServiceManager { + protected static final Logger LOG = LoggerFactory.getLogger(InternalTimeServiceManager.class); @VisibleForTesting static final String TIMER_STATE_PREFIX = "_timer_state"; @@ -78,6 +91,19 @@ public class InternalTimeServiceManager { this.timerServices = new HashMap<>(); } + public InternalTimerService getInternalTimerService( + String name, + TypeSerializer namespaceSerializer, + Triggerable triggerable, + KeyedStateBackend keyedStateBackend) { + checkNotNull(keyedStateBackend, "Timers can only be used on keyed operators."); + + TypeSerializer keySerializer = keyedStateBackend.getKeySerializer(); + // the following casting is to overcome type restrictions. + TimerSerializer timerSerializer = new TimerSerializer<>(keySerializer, namespaceSerializer); + return getInternalTimerService(name, timerSerializer, triggerable); + } + @SuppressWarnings("unchecked") public InternalTimerService getInternalTimerService( String name, @@ -131,6 +157,44 @@ public class InternalTimeServiceManager { ////////////////// Fault Tolerance Methods /////////////////// + public void snapshotState( + KeyedStateBackend keyedStateBackend, + StateSnapshotContext context, + String operatorName) throws Exception { + //TODO all of this can be removed once heap-based timers are integrated with RocksDB incremental snapshots + if (keyedStateBackend instanceof AbstractKeyedStateBackend && + ((AbstractKeyedStateBackend) keyedStateBackend).requiresLegacySynchronousTimerSnapshots()) { + + KeyedStateCheckpointOutputStream out; + try { + out = context.getRawKeyedOperatorStateOutput(); + } catch (Exception exception) { + throw new Exception("Could not open raw keyed operator state stream for " + + operatorName + '.', exception); + } + + try { + KeyGroupsList allKeyGroups = out.getKeyGroupList(); + for (int keyGroupIdx : allKeyGroups) { + out.startNewKeyGroup(keyGroupIdx); + + snapshotStateForKeyGroup( + new DataOutputViewStreamWrapper(out), keyGroupIdx); + } + } catch (Exception exception) { + throw new Exception("Could not write timer service of " + operatorName + + " to checkpoint state stream.", exception); + } finally { + try { + out.close(); + } catch (Exception closeException) { + LOG.warn("Could not close raw keyed operator state stream for {}. This " + + "might have prevented deleting some state data.", operatorName, closeException); + } + } + } + } + public void snapshotStateForKeyGroup(DataOutputView stream, int keyGroupIdx) throws IOException { Preconditions.checkState(useLegacySynchronousSnapshots); InternalTimerServiceSerializationProxy serializationProxy = diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorStateHandler.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorStateHandler.java new file mode 100644 index 0000000000..99123b1038 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorStateHandler.java @@ -0,0 +1,310 @@ +/* + * 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.operators; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.state.KeyedStateStore; +import org.apache.flink.api.common.state.State; +import org.apache.flink.api.common.state.StateDescriptor; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.runtime.checkpoint.CheckpointException; +import org.apache.flink.runtime.checkpoint.CheckpointFailureReason; +import org.apache.flink.runtime.checkpoint.CheckpointOptions; +import org.apache.flink.runtime.state.AbstractKeyedStateBackend; +import org.apache.flink.runtime.state.CheckpointStreamFactory; +import org.apache.flink.runtime.state.DefaultKeyedStateStore; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.KeyGroupStatePartitionStreamProvider; +import org.apache.flink.runtime.state.KeyedStateBackend; +import org.apache.flink.runtime.state.OperatorStateBackend; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateInitializationContextImpl; +import org.apache.flink.runtime.state.StatePartitionStreamProvider; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.runtime.state.StateSnapshotContextSynchronousImpl; +import org.apache.flink.util.CloseableIterable; +import org.apache.flink.util.IOUtils; + +import org.apache.flink.shaded.guava18.com.google.common.io.Closer; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Optional; + +import static org.apache.flink.util.Preconditions.checkState; + +/** + * Class encapsulating various state backend handling logic for {@link StreamOperator} implementations. + */ +@Internal +public class StreamOperatorStateHandler { + + protected static final Logger LOG = LoggerFactory.getLogger(StreamOperatorStateHandler.class); + + /** Backend for keyed state. This might be empty if we're not on a keyed stream. */ + @Nullable + private final AbstractKeyedStateBackend keyedStateBackend; + private final CloseableRegistry closeableRegistry; + @Nullable + private final DefaultKeyedStateStore keyedStateStore; + private final OperatorStateBackend operatorStateBackend; + private final StreamOperatorStateContext context; + + public StreamOperatorStateHandler( + StreamOperatorStateContext context, + ExecutionConfig executionConfig, + CloseableRegistry closeableRegistry) { + this.context = context; + operatorStateBackend = context.operatorStateBackend(); + keyedStateBackend = context.keyedStateBackend(); + this.closeableRegistry = closeableRegistry; + + if (keyedStateBackend != null) { + keyedStateStore = new DefaultKeyedStateStore(keyedStateBackend, executionConfig); + } + else { + keyedStateStore = null; + } + } + + public void initializeOperatorState(CheckpointedStreamOperator streamOperator) throws Exception { + CloseableIterable keyedStateInputs = context.rawKeyedStateInputs(); + CloseableIterable operatorStateInputs = context.rawOperatorStateInputs(); + + try { + StateInitializationContext initializationContext = new StateInitializationContextImpl( + context.isRestored(), // information whether we restore or start for the first time + operatorStateBackend, // access to operator state backend + keyedStateStore, // access to keyed state backend + keyedStateInputs, // access to keyed state stream + operatorStateInputs); // access to operator state stream + + streamOperator.initializeState(initializationContext); + } finally { + closeFromRegistry(operatorStateInputs, closeableRegistry); + closeFromRegistry(keyedStateInputs, closeableRegistry); + } + } + + private static void closeFromRegistry(Closeable closeable, CloseableRegistry registry) { + if (registry.unregisterCloseable(closeable)) { + IOUtils.closeQuietly(closeable); + } + } + + public void dispose() throws Exception { + try (Closer closer = Closer.create()) { + if (closeableRegistry.unregisterCloseable(operatorStateBackend)) { + closer.register(operatorStateBackend); + } + if (closeableRegistry.unregisterCloseable(keyedStateBackend)) { + closer.register(keyedStateBackend); + } + if (operatorStateBackend != null) { + closer.register(() -> operatorStateBackend.dispose()); + } + if (keyedStateBackend != null) { + closer.register(() -> keyedStateBackend.dispose()); + } + } + } + + public OperatorSnapshotFutures snapshotState( + CheckpointedStreamOperator streamOperator, + Optional> timeServiceManager, + String operatorName, + long checkpointId, + long timestamp, + CheckpointOptions checkpointOptions, + CheckpointStreamFactory factory) throws CheckpointException { + KeyGroupRange keyGroupRange = null != keyedStateBackend ? + keyedStateBackend.getKeyGroupRange() : KeyGroupRange.EMPTY_KEY_GROUP_RANGE; + + OperatorSnapshotFutures snapshotInProgress = new OperatorSnapshotFutures(); + + StateSnapshotContextSynchronousImpl snapshotContext = new StateSnapshotContextSynchronousImpl( + checkpointId, + timestamp, + factory, + keyGroupRange, + closeableRegistry); + + snapshotState( + streamOperator, + timeServiceManager, + operatorName, + checkpointId, + timestamp, + checkpointOptions, + factory, + snapshotInProgress, + snapshotContext); + + return snapshotInProgress; + } + + @VisibleForTesting + void snapshotState( + CheckpointedStreamOperator streamOperator, + Optional> timeServiceManager, + String operatorName, + long checkpointId, + long timestamp, + CheckpointOptions checkpointOptions, + CheckpointStreamFactory factory, + OperatorSnapshotFutures snapshotInProgress, + StateSnapshotContextSynchronousImpl snapshotContext) throws CheckpointException { + try { + if (timeServiceManager.isPresent()) { + checkState(keyedStateBackend != null, "keyedStateBackend should be available with timeServiceManager"); + timeServiceManager.get().snapshotState(keyedStateBackend, snapshotContext, operatorName); + } + streamOperator.snapshotState(snapshotContext); + + snapshotInProgress.setKeyedStateRawFuture(snapshotContext.getKeyedStateStreamFuture()); + snapshotInProgress.setOperatorStateRawFuture(snapshotContext.getOperatorStateStreamFuture()); + + if (null != operatorStateBackend) { + snapshotInProgress.setOperatorStateManagedFuture( + operatorStateBackend.snapshot(checkpointId, timestamp, factory, checkpointOptions)); + } + + if (null != keyedStateBackend) { + snapshotInProgress.setKeyedStateManagedFuture( + keyedStateBackend.snapshot(checkpointId, timestamp, factory, checkpointOptions)); + } + } catch (Exception snapshotException) { + try { + snapshotInProgress.cancel(); + } catch (Exception e) { + snapshotException.addSuppressed(e); + } + + String snapshotFailMessage = "Could not complete snapshot " + checkpointId + " for operator " + + operatorName + "."; + + try { + snapshotContext.closeExceptionally(); + } catch (IOException e) { + snapshotException.addSuppressed(e); + } + throw new CheckpointException(snapshotFailMessage, CheckpointFailureReason.CHECKPOINT_DECLINED, snapshotException); + } + } + + public void notifyCheckpointComplete(long checkpointId) throws Exception { + if (keyedStateBackend != null) { + keyedStateBackend.notifyCheckpointComplete(checkpointId); + } + } + + @SuppressWarnings("unchecked") + public KeyedStateBackend getKeyedStateBackend() { + return (KeyedStateBackend) keyedStateBackend; + } + + public OperatorStateBackend getOperatorStateBackend() { + return operatorStateBackend; + } + + public S getOrCreateKeyedState( + TypeSerializer namespaceSerializer, + StateDescriptor stateDescriptor) throws Exception { + + if (keyedStateStore != null) { + return keyedStateBackend.getOrCreateKeyedState(namespaceSerializer, stateDescriptor); + } + else { + throw new IllegalStateException("Cannot create partitioned state. " + + "The keyed state backend has not been set." + + "This indicates that the operator is not partitioned/keyed."); + } + } + + /** + * Creates a partitioned state handle, using the state backend configured for this task. + * + * @throws IllegalStateException Thrown, if the key/value state was already initialized. + * @throws Exception Thrown, if the state backend cannot create the key/value state. + */ + protected S getPartitionedState( + N namespace, + TypeSerializer namespaceSerializer, + StateDescriptor stateDescriptor) throws Exception { + + /* + TODO: NOTE: This method does a lot of work caching / retrieving states just to update the namespace. + This method should be removed for the sake of namespaces being lazily fetched from the keyed + state backend, or being set on the state directly. + */ + + if (keyedStateStore != null) { + return keyedStateBackend.getPartitionedState(namespace, namespaceSerializer, stateDescriptor); + } else { + throw new RuntimeException("Cannot create partitioned state. The keyed state " + + "backend has not been set. This indicates that the operator is not " + + "partitioned/keyed."); + } + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + public void setCurrentKey(Object key) { + if (keyedStateBackend != null) { + try { + // need to work around type restrictions + @SuppressWarnings("unchecked,rawtypes") + AbstractKeyedStateBackend rawBackend = (AbstractKeyedStateBackend) keyedStateBackend; + + rawBackend.setCurrentKey(key); + } catch (Exception e) { + throw new RuntimeException("Exception occurred while setting the current key context.", e); + } + } + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + public Object getCurrentKey() { + if (keyedStateBackend != null) { + return keyedStateBackend.getCurrentKey(); + } else { + throw new UnsupportedOperationException("Key can only be retrieved on KeyedStream."); + } + } + + public Optional getKeyedStateStore() { + return Optional.ofNullable(keyedStateStore); + } + + /** + * Custom state handling hooks to be invoked by {@link StreamOperatorStateHandler}. + */ + public interface CheckpointedStreamOperator { + void initializeState(StateInitializationContext context) throws Exception; + + void snapshotState(StateSnapshotContext context) throws Exception; + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java index a4a0dc2c75..ebd2ed10d9 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java @@ -23,66 +23,31 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.core.fs.CloseableRegistry; -import org.apache.flink.mock.Whitebox; -import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; -import org.apache.flink.runtime.state.AbstractKeyedStateBackend; -import org.apache.flink.runtime.state.CheckpointStreamFactory; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.KeyGroupRangeAssignment; -import org.apache.flink.runtime.state.KeyedStateHandle; -import org.apache.flink.runtime.state.OperatorStateBackend; -import org.apache.flink.runtime.state.OperatorStateHandle; -import org.apache.flink.runtime.state.SnapshotResult; -import org.apache.flink.runtime.state.StateSnapshotContextSynchronousImpl; import org.apache.flink.runtime.state.VoidNamespace; import org.apache.flink.runtime.state.VoidNamespaceSerializer; -import org.apache.flink.runtime.state.memory.MemCheckpointStreamFactory; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness; import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.junit.Test; -import org.junit.runner.RunWith; -import org.powermock.core.classloader.annotations.PowerMockIgnore; -import org.powermock.core.classloader.annotations.PrepareForTest; -import org.powermock.modules.junit4.PowerMockRunner; import java.util.ArrayList; import java.util.List; import java.util.Random; -import java.util.concurrent.RunnableFuture; import static junit.framework.TestCase.assertTrue; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; -import static org.mockito.ArgumentMatchers.nullable; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyBoolean; -import static org.mockito.Matchers.anyLong; -import static org.mockito.Matchers.eq; -import static org.mockito.Mockito.doCallRealMethod; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.verify; -import static org.powermock.api.mockito.PowerMockito.doReturn; -import static org.powermock.api.mockito.PowerMockito.mock; -import static org.powermock.api.mockito.PowerMockito.spy; -import static org.powermock.api.mockito.PowerMockito.when; -import static org.powermock.api.mockito.PowerMockito.whenNew; /** * Tests for the facilities provided by {@link AbstractStreamOperator}. This mostly * tests timers and state and whether they are correctly checkpointed/restored * with key-group reshuffling. */ -@RunWith(PowerMockRunner.class) -@PrepareForTest(AbstractStreamOperator.class) -@PowerMockIgnore({"java.*", "javax.*", "org.slf4j.*", "org.apache.log4j.*"}) public class AbstractStreamOperatorTest { @Test public void testStateDoesNotInterfere() throws Exception { @@ -493,101 +458,6 @@ public class AbstractStreamOperatorTest { assertTrue(extractResult(testHarness3).isEmpty()); } - /** - * Tests that a failing snapshot method call to the keyed state backend will trigger the closing - * of the StateSnapshotContextSynchronousImpl and the cancellation of the - * OperatorSnapshotResult. The latter is supposed to also cancel all assigned futures. - */ - @Test - public void testFailingBackendSnapshotMethod() throws Exception { - final long checkpointId = 42L; - final long timestamp = 1L; - - final Exception failingException = new Exception("Test exception"); - - final CloseableRegistry closeableRegistry = new CloseableRegistry(); - - RunnableFuture> futureKeyedStateHandle = mock(RunnableFuture.class); - RunnableFuture> futureOperatorStateHandle = mock(RunnableFuture.class); - - StateSnapshotContextSynchronousImpl context = spy(new StateSnapshotContextSynchronousImpl(checkpointId, timestamp)); - when(context.getKeyedStateStreamFuture()).thenReturn(futureKeyedStateHandle); - when(context.getOperatorStateStreamFuture()).thenReturn(futureOperatorStateHandle); - - OperatorSnapshotFutures operatorSnapshotResult = spy(new OperatorSnapshotFutures()); - - whenNew(StateSnapshotContextSynchronousImpl.class) - .withArguments( - anyLong(), - anyLong(), - any(CheckpointStreamFactory.class), - nullable(KeyGroupRange.class), - any(CloseableRegistry.class)) - .thenReturn(context); - whenNew(OperatorSnapshotFutures.class).withAnyArguments().thenReturn(operatorSnapshotResult); - - StreamTask> containingTask = mock(StreamTask.class); - when(containingTask.getCancelables()).thenReturn(closeableRegistry); - - AbstractStreamOperator operator = mock(AbstractStreamOperator.class); - when(operator.snapshotState(anyLong(), anyLong(), any(CheckpointOptions.class), any(CheckpointStreamFactory.class))).thenCallRealMethod(); - - doCallRealMethod().when(operator).close(); - doCallRealMethod().when(operator).dispose(); - - doReturn(containingTask).when(operator).getContainingTask(); - - RunnableFuture> futureManagedOperatorStateHandle = mock(RunnableFuture.class); - - OperatorStateBackend operatorStateBackend = mock(OperatorStateBackend.class); - when(operatorStateBackend.snapshot( - eq(checkpointId), - eq(timestamp), - any(CheckpointStreamFactory.class), - any(CheckpointOptions.class))).thenReturn(futureManagedOperatorStateHandle); - - AbstractKeyedStateBackend keyedStateBackend = mock(AbstractKeyedStateBackend.class); - when(keyedStateBackend.snapshot( - eq(checkpointId), - eq(timestamp), - any(CheckpointStreamFactory.class), - eq(CheckpointOptions.forCheckpointWithDefaultLocation()))).thenThrow(failingException); - - closeableRegistry.registerCloseable(operatorStateBackend); - closeableRegistry.registerCloseable(keyedStateBackend); - - Whitebox.setInternalState(operator, "operatorStateBackend", operatorStateBackend); - Whitebox.setInternalState(operator, "keyedStateBackend", keyedStateBackend); - - try { - operator.snapshotState( - checkpointId, - timestamp, - CheckpointOptions.forCheckpointWithDefaultLocation(), - new MemCheckpointStreamFactory(Integer.MAX_VALUE)); - fail("Exception expected."); - } catch (Exception e) { - assertEquals(failingException.getMessage(), e.getCause().getMessage()); - } - - // verify that the context has been closed, the operator snapshot result has been cancelled - // and that all futures have been cancelled. - verify(operatorSnapshotResult).cancel(); - - verify(futureKeyedStateHandle).cancel(anyBoolean()); - verify(futureOperatorStateHandle).cancel(anyBoolean()); - verify(futureKeyedStateHandle).cancel(anyBoolean()); - - operator.close(); - - operator.dispose(); - - verify(operatorStateBackend).close(); - verify(keyedStateBackend).close(); - verify(operatorStateBackend).dispose(); - verify(keyedStateBackend).dispose(); - } - /** * Extracts the result values form the test harness and clear the output queue. */ diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StreamOperatorStateHandlerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StreamOperatorStateHandlerTest.java new file mode 100644 index 0000000000..151dce1523 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StreamOperatorStateHandlerTest.java @@ -0,0 +1,193 @@ +/* + * 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.operators; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.runtime.checkpoint.CheckpointException; +import org.apache.flink.runtime.checkpoint.CheckpointOptions; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.metrics.util.InterceptingOperatorMetricGroup; +import org.apache.flink.runtime.operators.testutils.ExpectedTestException; +import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.KeyedStateHandle; +import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.SnapshotResult; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.runtime.state.StateSnapshotContextSynchronousImpl; +import org.apache.flink.runtime.state.memory.MemCheckpointStreamFactory; +import org.apache.flink.runtime.state.memory.MemoryStateBackend; +import org.apache.flink.streaming.api.operators.StreamOperatorStateHandler.CheckpointedStreamOperator; +import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; +import org.apache.flink.util.ExceptionUtils; + +import org.junit.Test; + +import java.util.Optional; +import java.util.concurrent.FutureTask; +import java.util.concurrent.RunnableFuture; + +import static junit.framework.TestCase.assertTrue; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.not; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.empty; +import static org.junit.Assert.fail; + +/** + * Tests for {@link StreamOperatorStateHandlerTest}. + */ +public class StreamOperatorStateHandlerTest { + /** + * Tests that a failing snapshot method call to the keyed state backend will trigger the closing + * of the StateSnapshotContextSynchronousImpl and the cancellation of the + * OperatorSnapshotResult. The latter is supposed to also cancel all assigned futures. + */ + @Test + public void testFailingBackendSnapshotMethod() throws Exception { + final long checkpointId = 42L; + final long timestamp = 1L; + + try (CloseableRegistry closeableRegistry = new CloseableRegistry()) { + RunnableFuture> keyedStateManagedFuture = new CancelableFuture<>(); + RunnableFuture> keyedStateRawFuture = new CancelableFuture<>(); + RunnableFuture> operatorStateManagedFuture = new CancelableFuture<>(); + RunnableFuture> operatorStateRawFuture = new CancelableFuture<>(); + + OperatorSnapshotFutures operatorSnapshotResult = new OperatorSnapshotFutures( + keyedStateManagedFuture, + keyedStateRawFuture, + operatorStateManagedFuture, + operatorStateRawFuture); + + StateSnapshotContextSynchronousImpl context = new TestStateSnapshotContextSynchronousImpl(checkpointId, timestamp, closeableRegistry); + context.getRawKeyedOperatorStateOutput(); + context.getRawOperatorStateOutput(); + + StreamTaskStateInitializerImpl stateInitializer = + new StreamTaskStateInitializerImpl(new MockEnvironmentBuilder().build(), new MemoryStateBackend()); + StreamOperatorStateContext stateContext = stateInitializer.streamOperatorStateContext( + new OperatorID(), + "whatever", + new TestProcessingTimeService(), + new UnUsedKeyContext(), + IntSerializer.INSTANCE, + closeableRegistry, + new InterceptingOperatorMetricGroup()); + StreamOperatorStateHandler stateHandler = new StreamOperatorStateHandler(stateContext, new ExecutionConfig(), closeableRegistry); + + final String keyedStateField = "keyedStateField"; + final String operatorStateField = "operatorStateField"; + + CheckpointedStreamOperator checkpointedStreamOperator = new CheckpointedStreamOperator() { + @Override + public void initializeState(StateInitializationContext context) throws Exception { + context.getKeyedStateStore() + .getState(new ValueStateDescriptor<>(keyedStateField, LongSerializer.INSTANCE)) + .update(42L); + context.getOperatorStateStore() + .getListState(new ListStateDescriptor<>(operatorStateField, LongSerializer.INSTANCE)) + .add(42L); + } + + @Override + public void snapshotState(StateSnapshotContext context) throws Exception { + throw new ExpectedTestException(); + } + }; + + stateHandler.setCurrentKey("44"); + stateHandler.initializeOperatorState(checkpointedStreamOperator); + + assertThat(stateContext.operatorStateBackend().getRegisteredStateNames(), is(not(empty()))); + assertThat(stateContext.keyedStateBackend().numKeyValueStatesByName(), equalTo(1)); + + try { + stateHandler.snapshotState( + checkpointedStreamOperator, + Optional.of(stateContext.internalTimerServiceManager()), + "42", + 42, + 42, + CheckpointOptions.forCheckpointWithDefaultLocation(), + new MemCheckpointStreamFactory(1024), + operatorSnapshotResult, + context); + fail("Exception expected."); + } catch (CheckpointException e) { + // We can not check for ExpectedTestException class directly, + // as CheckpointException is wrapping the cause with SerializedThrowable + if (!ExceptionUtils.findThrowableWithMessage(e, ExpectedTestException.MESSAGE).isPresent()) { + throw e; + } + } + + assertTrue(keyedStateManagedFuture.isCancelled()); + assertTrue(keyedStateRawFuture.isCancelled()); + assertTrue(context.getKeyedStateStreamFuture().isCancelled()); + assertTrue(operatorStateManagedFuture.isCancelled()); + assertTrue(operatorStateRawFuture.isCancelled()); + assertTrue(context.getOperatorStateStreamFuture().isCancelled()); + + stateHandler.dispose(); + + assertThat(stateContext.operatorStateBackend().getRegisteredBroadcastStateNames(), is(empty())); + assertThat(stateContext.operatorStateBackend().getRegisteredStateNames(), is(empty())); + assertThat(stateContext.keyedStateBackend().numKeyValueStatesByName(), is(0)); + } + } + + private static class TestStateSnapshotContextSynchronousImpl extends StateSnapshotContextSynchronousImpl { + public TestStateSnapshotContextSynchronousImpl( + long checkpointId, + long timestamp, + CloseableRegistry closeableRegistry) { + super(checkpointId, timestamp, new MemCheckpointStreamFactory(1024), new KeyGroupRange(0, 2), closeableRegistry); + this.keyedStateCheckpointClosingFuture = new CancelableFuture<>(); + this.operatorStateCheckpointClosingFuture = new CancelableFuture<>(); + } + } + + private static class CancelableFuture extends FutureTask { + public CancelableFuture() { + super(() -> { + throw new UnsupportedOperationException(); + }); + } + } + + private static class UnUsedKeyContext implements KeyContext { + @Override + public void setCurrentKey(Object key) { + throw new UnsupportedOperationException(); + } + + @Override + public Object getCurrentKey() { + throw new UnsupportedOperationException(); + } + } +} diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/runtime/operators/KeyedCoProcessOperatorWithWatermarkDelay.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/runtime/operators/KeyedCoProcessOperatorWithWatermarkDelay.scala index 72aec48249..3aaa7d22a0 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/runtime/operators/KeyedCoProcessOperatorWithWatermarkDelay.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/runtime/operators/KeyedCoProcessOperatorWithWatermarkDelay.scala @@ -50,7 +50,8 @@ class KeyedCoProcessOperatorWithWatermarkDelay[KEY, IN1, IN2, OUT]( @throws[Exception] override def processWatermark(mark: Watermark) { - if (timeServiceManager != null) timeServiceManager.advanceWatermark(mark) + val timeServiceManager = getTimeServiceManager + if (timeServiceManager.isPresent) timeServiceManager.get().advanceWatermark(mark) emitter(mark) } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/runtime/operators/KeyedProcessOperatorWithWatermarkDelay.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/runtime/operators/KeyedProcessOperatorWithWatermarkDelay.scala index f63bdb5acd..a47638ae30 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/runtime/operators/KeyedProcessOperatorWithWatermarkDelay.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/runtime/operators/KeyedProcessOperatorWithWatermarkDelay.scala @@ -51,7 +51,8 @@ class KeyedProcessOperatorWithWatermarkDelay[KEY, IN, OUT]( @throws[Exception] override def processWatermark(mark: Watermark) { - if (timeServiceManager != null) timeServiceManager.advanceWatermark(mark) + val timeServiceManager = getTimeServiceManager + if (timeServiceManager.isPresent) timeServiceManager.get().advanceWatermark(mark) emitter(mark) } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/CodeGenOperatorFactory.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/CodeGenOperatorFactory.java index 7eb8cf1f8d..56293e23b8 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/CodeGenOperatorFactory.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/CodeGenOperatorFactory.java @@ -18,12 +18,9 @@ package org.apache.flink.table.runtime.operators; -import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; -import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.operators.StreamOperator; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.tasks.StreamTask; +import org.apache.flink.streaming.api.operators.StreamOperatorInitializer; import org.apache.flink.table.runtime.generated.GeneratedClass; /** @@ -39,10 +36,14 @@ public class CodeGenOperatorFactory extends AbstractStreamOperatorFactory> T createStreamOperator(StreamTask containingTask, - StreamConfig config, Output> output) { - return (T) generatedClass.newInstance(containingTask.getUserCodeClassLoader(), - generatedClass.getReferences(), containingTask, config, output, processingTimeService); + public > T createStreamOperator(StreamOperatorInitializer initializer) { + return (T) generatedClass.newInstance( + initializer.getContainingTask().getUserCodeClassLoader(), + generatedClass.getReferences(), + initializer.getContainingTask(), + initializer.getStreamConfig(), + initializer.getOutput(), + processingTimeService); } @Override diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/KeyedCoProcessOperatorWithWatermarkDelay.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/KeyedCoProcessOperatorWithWatermarkDelay.java index 73e20052f8..fb2ce5035b 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/KeyedCoProcessOperatorWithWatermarkDelay.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/KeyedCoProcessOperatorWithWatermarkDelay.java @@ -19,11 +19,13 @@ package org.apache.flink.table.runtime.operators.join; import org.apache.flink.streaming.api.functions.co.KeyedCoProcessFunction; +import org.apache.flink.streaming.api.operators.InternalTimeServiceManager; import org.apache.flink.streaming.api.operators.co.KeyedCoProcessOperator; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.util.Preconditions; import java.io.Serializable; +import java.util.Optional; import java.util.function.Consumer; /** @@ -51,10 +53,10 @@ public class KeyedCoProcessOperatorWithWatermarkDelay @Override public void processWatermark(Watermark mark) throws Exception { - if (timeServiceManager != null) { - timeServiceManager.advanceWatermark(mark); + Optional> timeServiceManager = getTimeServiceManager(); + if (timeServiceManager.isPresent()) { + timeServiceManager.get().advanceWatermark(mark); } emitter.accept(mark); } - } -- Gitee From 239088e603baede4fbbd05f75655c54bb3750fcb Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Wed, 11 Mar 2020 17:53:38 +0100 Subject: [PATCH 292/885] [FLINK-16316][operators] Cut dependency between StreamingRuntimeContext and AbstractStreamOperator This simplifies dependencies between those two classes and will allow for StreamingRuntimeContext to be re-used in new replacement for AbstractStreamOperator. --- .../api/operators/AbstractStreamOperator.java | 9 +- .../operators/StreamingRuntimeContext.java | 63 ++++++--- .../StreamingRuntimeContextTest.java | 126 +++++++++++------- 3 files changed, 129 insertions(+), 69 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java index e525c21b5e..c2604b28c0 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java @@ -204,7 +204,13 @@ public abstract class AbstractStreamOperator LatencyStats.Granularity.SINGLE); } - this.runtimeContext = new StreamingRuntimeContext(this, environment, environment.getAccumulatorRegistry().getUserMap()); + this.runtimeContext = new StreamingRuntimeContext( + environment, + environment.getAccumulatorRegistry().getUserMap(), + getMetricGroup(), + getOperatorID(), + getProcessingTimeService(), + null); stateKeySelector1 = config.getStatePartitioner(0, getUserCodeClassloader()); stateKeySelector2 = config.getStatePartitioner(1, getUserCodeClassloader()); @@ -247,6 +253,7 @@ public abstract class AbstractStreamOperator stateHandler = new StreamOperatorStateHandler(context, getExecutionConfig(), streamTaskCloseableRegistry); timeServiceManager = context.internalTimerServiceManager(); stateHandler.initializeOperatorState(this); + runtimeContext.setKeyedStateStore(stateHandler.getKeyedStateStore().orElse(null)); } /** diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java index 23cb82e374..3fe540f0f5 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java @@ -19,6 +19,7 @@ package org.apache.flink.streaming.api.operators; import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.functions.BroadcastVariableInitializer; import org.apache.flink.api.common.functions.util.AbstractRuntimeUDFContext; @@ -36,17 +37,22 @@ import org.apache.flink.api.common.state.ReducingStateDescriptor; import org.apache.flink.api.common.state.StateDescriptor; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; import org.apache.flink.runtime.taskexecutor.GlobalAggregateManager; import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; -import org.apache.flink.util.Preconditions; + +import javax.annotation.Nullable; import java.util.List; import java.util.Map; +import static org.apache.flink.util.Preconditions.checkNotNull; + /** * Implementation of the {@link org.apache.flink.api.common.functions.RuntimeContext}, * for streaming operators. @@ -54,29 +60,49 @@ import java.util.Map; @Internal public class StreamingRuntimeContext extends AbstractRuntimeUDFContext { - /** The operator to which this function belongs. */ - private final AbstractStreamOperator operator; - /** The task environment running the operator. */ private final Environment taskEnvironment; - private final StreamConfig streamConfig; - private final String operatorUniqueID; - - public StreamingRuntimeContext(AbstractStreamOperator operator, - Environment env, Map> accumulators) { - super(env.getTaskInfo(), + private final ProcessingTimeService processingTimeService; + private @Nullable KeyedStateStore keyedStateStore; + + @VisibleForTesting + public StreamingRuntimeContext( + AbstractStreamOperator operator, + Environment env, + Map> accumulators) { + this( + env, + accumulators, + operator.getMetricGroup(), + operator.getOperatorID(), + operator.getProcessingTimeService(), + operator.getKeyedStateStore()); + } + + public StreamingRuntimeContext( + Environment env, + Map> accumulators, + MetricGroup operatorMetricGroup, + OperatorID operatorID, + ProcessingTimeService processingTimeService, + @Nullable KeyedStateStore keyedStateStore) { + super(checkNotNull(env).getTaskInfo(), env.getUserClassLoader(), - operator.getExecutionConfig(), + env.getExecutionConfig(), accumulators, env.getDistributedCacheEntries(), - operator.getMetricGroup()); - - this.operator = operator; + operatorMetricGroup); this.taskEnvironment = env; this.streamConfig = new StreamConfig(env.getTaskConfiguration()); - this.operatorUniqueID = operator.getOperatorID().toString(); + this.operatorUniqueID = checkNotNull(operatorID).toString(); + this.processingTimeService = processingTimeService; + this.keyedStateStore = keyedStateStore; + } + + public void setKeyedStateStore(@Nullable KeyedStateStore keyedStateStore) { + this.keyedStateStore = keyedStateStore; } // ------------------------------------------------------------------------ @@ -91,7 +117,7 @@ public class StreamingRuntimeContext extends AbstractRuntimeUDFContext { } public ProcessingTimeService getProcessingTimeService() { - return operator.getProcessingTimeService(); + return processingTimeService; } /** @@ -180,9 +206,8 @@ public class StreamingRuntimeContext extends AbstractRuntimeUDFContext { } private KeyedStateStore checkPreconditionsAndGetKeyedStateStore(StateDescriptor stateDescriptor) { - Preconditions.checkNotNull(stateDescriptor, "The state properties must not be null"); - KeyedStateStore keyedStateStore = operator.getKeyedStateStore(); - Preconditions.checkNotNull(keyedStateStore, "Keyed state can only be used on a 'keyed stream', i.e., after a 'keyBy()' operation."); + checkNotNull(stateDescriptor, "The state properties must not be null"); + checkNotNull(keyedStateStore, "Keyed state can only be used on a 'keyed stream', i.e., after a 'keyBy()' operation."); return keyedStateStore; } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContextTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContextTest.java index 7b678c32bb..1c2390c90e 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContextTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContextTest.java @@ -21,7 +21,6 @@ package org.apache.flink.streaming.api.operators; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.TaskInfo; -import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.functions.AggregateFunction; import org.apache.flink.api.common.functions.FoldFunction; import org.apache.flink.api.common.functions.ReduceFunction; @@ -38,6 +37,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.base.IntSerializer; import org.apache.flink.api.common.typeutils.base.ListSerializer; import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; +import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.CloseableRegistry; import org.apache.flink.core.fs.Path; import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; @@ -55,12 +55,19 @@ import org.apache.flink.runtime.state.VoidNamespace; import org.apache.flink.runtime.state.VoidNamespaceSerializer; import org.apache.flink.runtime.state.memory.MemoryStateBackend; import org.apache.flink.runtime.state.ttl.TtlTimeProvider; +import org.apache.flink.streaming.api.graph.StreamConfig; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.StreamTask; +import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; +import org.apache.flink.streaming.util.CollectorOutput; +import org.apache.flink.streaming.util.MockStreamTaskBuilder; import org.junit.Test; import org.mockito.Matchers; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; +import java.util.ArrayList; import java.util.Collections; import java.util.Map; import java.util.concurrent.atomic.AtomicReference; @@ -86,11 +93,7 @@ public class StreamingRuntimeContextTest { final AtomicReference descriptorCapture = new AtomicReference<>(); - StreamingRuntimeContext context = new StreamingRuntimeContext( - createDescriptorCapturingMockOp(descriptorCapture, config), - createMockEnvironment(), - Collections.>emptyMap()); - + StreamingRuntimeContext context = createRuntimeContext(descriptorCapture, config); ValueStateDescriptor descr = new ValueStateDescriptor<>("name", TaskInfo.class); context.getState(descr); @@ -110,10 +113,7 @@ public class StreamingRuntimeContextTest { final AtomicReference descriptorCapture = new AtomicReference<>(); - StreamingRuntimeContext context = new StreamingRuntimeContext( - createDescriptorCapturingMockOp(descriptorCapture, config), - createMockEnvironment(), - Collections.>emptyMap()); + StreamingRuntimeContext context = createRuntimeContext(descriptorCapture, config); @SuppressWarnings("unchecked") ReduceFunction reducer = (ReduceFunction) mock(ReduceFunction.class); @@ -133,16 +133,12 @@ public class StreamingRuntimeContextTest { @Test public void testAggregatingStateInstantiation() throws Exception { - final ExecutionConfig config = new ExecutionConfig(); config.registerKryoType(Path.class); final AtomicReference descriptorCapture = new AtomicReference<>(); - StreamingRuntimeContext context = new StreamingRuntimeContext( - createDescriptorCapturingMockOp(descriptorCapture, config), - createMockEnvironment(), - Collections.>emptyMap()); + StreamingRuntimeContext context = createRuntimeContext(descriptorCapture, config); @SuppressWarnings("unchecked") AggregateFunction aggregate = (AggregateFunction) mock(AggregateFunction.class); @@ -168,10 +164,7 @@ public class StreamingRuntimeContextTest { final AtomicReference descriptorCapture = new AtomicReference<>(); - StreamingRuntimeContext context = new StreamingRuntimeContext( - createDescriptorCapturingMockOp(descriptorCapture, config), - createMockEnvironment(), - Collections.>emptyMap()); + StreamingRuntimeContext context = createRuntimeContext(descriptorCapture, config); @SuppressWarnings("unchecked") FoldFunction folder = (FoldFunction) mock(FoldFunction.class); @@ -197,10 +190,7 @@ public class StreamingRuntimeContextTest { final AtomicReference descriptorCapture = new AtomicReference<>(); - StreamingRuntimeContext context = new StreamingRuntimeContext( - createDescriptorCapturingMockOp(descriptorCapture, config), - createMockEnvironment(), - Collections.>emptyMap()); + StreamingRuntimeContext context = createRuntimeContext(descriptorCapture, config); ListStateDescriptor descr = new ListStateDescriptor<>("name", TaskInfo.class); context.getListState(descr); @@ -218,11 +208,7 @@ public class StreamingRuntimeContextTest { @Test public void testListStateReturnsEmptyListByDefault() throws Exception { - - StreamingRuntimeContext context = new StreamingRuntimeContext( - createListPlainMockOp(), - createMockEnvironment(), - Collections.>emptyMap()); + StreamingRuntimeContext context = createRuntimeContext(); ListStateDescriptor descr = new ListStateDescriptor<>("name", String.class); ListState state = context.getListState(descr); @@ -240,10 +226,7 @@ public class StreamingRuntimeContextTest { final AtomicReference descriptorCapture = new AtomicReference<>(); - StreamingRuntimeContext context = new StreamingRuntimeContext( - createDescriptorCapturingMockOp(descriptorCapture, config), - createMockEnvironment(), - Collections.>emptyMap()); + StreamingRuntimeContext context = createRuntimeContext(descriptorCapture, config); MapStateDescriptor descr = new MapStateDescriptor<>("name", String.class, TaskInfo.class); @@ -261,10 +244,7 @@ public class StreamingRuntimeContextTest { @Test public void testMapStateReturnsEmptyMapByDefault() throws Exception { - StreamingRuntimeContext context = new StreamingRuntimeContext( - createMapPlainMockOp(), - createMockEnvironment(), - Collections.>emptyMap()); + StreamingRuntimeContext context = createMapOperatorRuntimeContext(); MapStateDescriptor descr = new MapStateDescriptor<>("name", Integer.class, String.class); MapState state = context.getMapState(descr); @@ -278,18 +258,71 @@ public class StreamingRuntimeContextTest { // // ------------------------------------------------------------------------ + private StreamingRuntimeContext createMapOperatorRuntimeContext() throws Exception { + AbstractStreamOperator mapPlainMockOp = createMapPlainMockOp(); + return createRuntimeContext(mapPlainMockOp); + } + + private StreamingRuntimeContext createRuntimeContext() throws Exception { + return new StreamingRuntimeContext( + createListPlainMockOp(), + MockEnvironment.builder() + .build(), + Collections.emptyMap()); + } + + private StreamingRuntimeContext createRuntimeContext( + AtomicReference descriptorCapture, + ExecutionConfig config) throws Exception { + return createDescriptorCapturingMockOp( + descriptorCapture, + config, + MockEnvironment.builder() + .setExecutionConfig(config) + .build()).getRuntimeContext(); + } + + private StreamingRuntimeContext createRuntimeContext(AbstractStreamOperator operator) { + return new StreamingRuntimeContext( + MockEnvironment.builder() + .build(), + Collections.emptyMap(), + operator.getMetricGroup(), + operator.getOperatorID(), + operator.getProcessingTimeService(), + operator.getKeyedStateStore()); + } + @SuppressWarnings("unchecked") private static AbstractStreamOperator createDescriptorCapturingMockOp( - final AtomicReference ref, final ExecutionConfig config) throws Exception { + final AtomicReference ref, + final ExecutionConfig config, + Environment environment) throws Exception { - AbstractStreamOperator operatorMock = mock(AbstractStreamOperator.class); + AbstractStreamOperator operator = new AbstractStreamOperator() { + @Override + public void setup( + StreamTask containingTask, + StreamConfig config, + Output> output) { + super.setup(containingTask, config, output); + } + }; + StreamConfig streamConfig = new StreamConfig(new Configuration()); + streamConfig.setOperatorID(new OperatorID()); + operator.setup( + new MockStreamTaskBuilder(environment).setExecutionConfig(config).build(), + streamConfig, + new CollectorOutput<>(new ArrayList<>())); + + StreamTaskStateInitializer streamTaskStateManager = new StreamTaskStateInitializerImpl( + environment, + new MemoryStateBackend()); KeyedStateBackend keyedStateBackend = mock(KeyedStateBackend.class); DefaultKeyedStateStore keyedStateStore = new DefaultKeyedStateStore(keyedStateBackend, config); - when(operatorMock.getExecutionConfig()).thenReturn(config); - doAnswer(new Answer() { @Override @@ -299,10 +332,10 @@ public class StreamingRuntimeContextTest { } }).when(keyedStateBackend).getPartitionedState(Matchers.any(), any(TypeSerializer.class), any(StateDescriptor.class)); - when(operatorMock.getKeyedStateStore()).thenReturn(keyedStateStore); - when(operatorMock.getOperatorID()).thenReturn(new OperatorID()); + operator.initializeState(streamTaskStateManager); + operator.getRuntimeContext().setKeyedStateStore(keyedStateStore); - return operatorMock; + return operator; } @SuppressWarnings("unchecked") @@ -384,12 +417,7 @@ public class StreamingRuntimeContextTest { when(operatorMock.getKeyedStateStore()).thenReturn(keyedStateStore); when(operatorMock.getOperatorID()).thenReturn(new OperatorID()); + when(operatorMock.getProcessingTimeService()).thenReturn(new TestProcessingTimeService()); return operatorMock; } - - private static Environment createMockEnvironment() { - return MockEnvironment.builder() - .setTaskName("test task") - .build(); - } } -- Gitee From c1398f057c452529e7d2e8f494bff1bf900b2f32 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Wed, 18 Mar 2020 17:07:15 +0100 Subject: [PATCH 293/885] [FLINK-16316][operators] Move inner CountingClass class out from AbstractStreamOperator --- .../api/operators/AbstractStreamOperator.java | 44 ------------- .../api/operators/CountingOutput.java | 65 +++++++++++++++++++ 2 files changed, 65 insertions(+), 44 deletions(-) create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/CountingOutput.java diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java index c2604b28c0..6507ce9ea3 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java @@ -29,7 +29,6 @@ import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.MetricOptions; import org.apache.flink.core.fs.CloseableRegistry; -import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.execution.Environment; @@ -52,7 +51,6 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.apache.flink.streaming.util.LatencyStats; -import org.apache.flink.util.OutputTag; import org.apache.flink.util.Preconditions; import org.slf4j.Logger; @@ -513,48 +511,6 @@ public abstract class AbstractStreamOperator this.output.emitLatencyMarker(marker); } - // ----------------------- Helper classes ----------------------- - - /** - * Wrapping {@link Output} that updates metrics on the number of emitted elements. - */ - public static class CountingOutput implements Output> { - private final Output> output; - private final Counter numRecordsOut; - - public CountingOutput(Output> output, Counter counter) { - this.output = output; - this.numRecordsOut = counter; - } - - @Override - public void emitWatermark(Watermark mark) { - output.emitWatermark(mark); - } - - @Override - public void emitLatencyMarker(LatencyMarker latencyMarker) { - output.emitLatencyMarker(latencyMarker); - } - - @Override - public void collect(StreamRecord record) { - numRecordsOut.inc(); - output.collect(record); - } - - @Override - public void collect(OutputTag outputTag, StreamRecord record) { - numRecordsOut.inc(); - output.collect(outputTag, record); - } - - @Override - public void close() { - output.close(); - } - } - // ------------------------------------------------------------------------ // Watermark handling // ------------------------------------------------------------------------ diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/CountingOutput.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/CountingOutput.java new file mode 100644 index 0000000000..79acabb646 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/CountingOutput.java @@ -0,0 +1,65 @@ +/* + * 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.operators; + +import org.apache.flink.metrics.Counter; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.util.OutputTag; + +/** + * Wrapping {@link Output} that updates metrics on the number of emitted elements. + */ +public class CountingOutput implements Output> { + private final Output> output; + private final Counter numRecordsOut; + + public CountingOutput(Output> output, Counter counter) { + this.output = output; + this.numRecordsOut = counter; + } + + @Override + public void emitWatermark(Watermark mark) { + output.emitWatermark(mark); + } + + @Override + public void emitLatencyMarker(LatencyMarker latencyMarker) { + output.emitLatencyMarker(latencyMarker); + } + + @Override + public void collect(StreamRecord record) { + numRecordsOut.inc(); + output.collect(record); + } + + @Override + public void collect(OutputTag outputTag, StreamRecord record) { + numRecordsOut.inc(); + output.collect(outputTag, record); + } + + @Override + public void close() { + output.close(); + } +} -- Gitee From f84cc1b78b1e3e2ac5077b4bf8efc45250b16af7 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Thu, 12 Mar 2020 15:36:24 +0100 Subject: [PATCH 294/885] [FLINK-16316][operators] Introduce StreamOperatorParameters class New POJO class will make Public and PublicEvolving interfaces more stable and easier to use. User will not have to pass n parameters, but just this one POJO. --- .../ContinuousFileReaderOperatorFactory.java | 10 ++- .../api/operators/SimpleOperatorFactory.java | 11 ++- .../api/operators/StreamOperatorFactory.java | 5 +- .../operators/StreamOperatorFactoryUtil.java | 8 ++- .../operators/StreamOperatorParameters.java | 67 +++++++++++++++++++ .../async/AsyncWaitOperatorFactory.java | 10 ++- .../api/graph/StreamGraphGeneratorTest.java | 6 +- .../operators/MailboxOperatorTest.java | 10 +-- .../StreamTaskOperatorTimerTest.java | 12 ++-- .../runtime/tasks/StreamTaskTest.java | 6 +- .../operators/CodeGenOperatorFactory.java | 12 ++-- .../WatermarkAssignerOperatorFactory.java | 14 ++-- .../runtime/MultipleInputITCase.java | 11 +-- 13 files changed, 116 insertions(+), 66 deletions(-) create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorParameters.java diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileReaderOperatorFactory.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileReaderOperatorFactory.java index 559fd9ef11..97a367321d 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileReaderOperatorFactory.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileReaderOperatorFactory.java @@ -20,15 +20,13 @@ package org.apache.flink.streaming.api.functions.source; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.io.FileInputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.MailboxExecutor; import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; -import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.api.operators.YieldingOperatorFactory; -import org.apache.flink.streaming.runtime.tasks.StreamTask; /** * {@link ContinuousFileReaderOperator} factory. @@ -58,11 +56,11 @@ public class ContinuousFileReaderOperatorFactory extends AbstractStreamOper } @Override - public StreamOperator createStreamOperator(StreamTask containingTask, StreamConfig config, Output output) { + public > T createStreamOperator(StreamOperatorParameters parameters) { ContinuousFileReaderOperator operator = new ContinuousFileReaderOperator<>(inputFormat, processingTimeService, mailboxExecutor); - operator.setup(containingTask, config, output); + operator.setup(parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); operator.setOutputType(type, executionConfig); - return operator; + return (T) operator; } @Override diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SimpleOperatorFactory.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SimpleOperatorFactory.java index 1f43c2e8f7..1e3de6eb48 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SimpleOperatorFactory.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SimpleOperatorFactory.java @@ -23,9 +23,6 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.InputTypeConfigurable; import org.apache.flink.streaming.api.functions.sink.OutputFormatSinkFunction; import org.apache.flink.streaming.api.functions.source.InputFormatSourceFunction; -import org.apache.flink.streaming.api.graph.StreamConfig; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.tasks.StreamTask; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -72,13 +69,15 @@ public class SimpleOperatorFactory extends AbstractStreamOperatorFactory> T createStreamOperator(StreamTask containingTask, - StreamConfig config, Output> output) { + public > T createStreamOperator(StreamOperatorParameters parameters) { if (operator instanceof AbstractStreamOperator) { ((AbstractStreamOperator) operator).setProcessingTimeService(processingTimeService); } if (operator instanceof SetupableStreamOperator) { - ((SetupableStreamOperator) operator).setup(containingTask, config, output); + ((SetupableStreamOperator) operator).setup( + parameters.getContainingTask(), + parameters.getStreamConfig(), + parameters.getOutput()); } return (T) operator; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorFactory.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorFactory.java index 1d5268ee6e..19df0011af 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorFactory.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorFactory.java @@ -20,9 +20,7 @@ package org.apache.flink.streaming.api.operators; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.graph.StreamGraph; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.StreamTask; import java.io.Serializable; @@ -38,8 +36,7 @@ public interface StreamOperatorFactory extends Serializable { /** * Create the operator. Sets access to the context and the output. */ - > T createStreamOperator( - StreamTask containingTask, StreamConfig config, Output> output); + > T createStreamOperator(StreamOperatorParameters parameters); /** * Set the chaining strategy for operator factory. diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorFactoryUtil.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorFactoryUtil.java index 6b8ab233b7..11090f6d08 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorFactoryUtil.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorFactoryUtil.java @@ -57,7 +57,13 @@ public class StreamOperatorFactoryUtil { ((ProcessingTimeServiceAware) operatorFactory).setProcessingTimeService(processingTimeService); } - OP op = operatorFactory.createStreamOperator(containingTask, configuration, output); + // TODO: what to do with ProcessingTimeServiceAware? + OP op = operatorFactory.createStreamOperator( + new StreamOperatorParameters<>( + containingTask, + configuration, + output, + processingTimeService)); return new Tuple2<>(op, Optional.ofNullable(processingTimeService)); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorParameters.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorParameters.java new file mode 100644 index 0000000000..1f6f1a03a6 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorParameters.java @@ -0,0 +1,67 @@ +/* + * 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.operators; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.streaming.api.graph.StreamConfig; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; +import org.apache.flink.streaming.runtime.tasks.StreamTask; + +/** + * Helper class to construct {@link StreamOperatorBase}. Wraps couple of internal parameters + * to simplify for users construction of classes extending {@link StreamOperatorBase} and to + * allow for backward compatible changes in the {@link StreamOperatorBase}'s constructor. + * + * @param The output type of an operator that will be constructed using {@link StreamOperatorParameters}. + */ +@Experimental +public class StreamOperatorParameters { + private final StreamTask containingTask; + private final StreamConfig config; + private final Output> output; + private final ProcessingTimeService processingTimeService; + + public StreamOperatorParameters( + StreamTask containingTask, + StreamConfig config, + Output> output, + ProcessingTimeService processingTimeService) { + this.containingTask = containingTask; + this.config = config; + this.output = output; + this.processingTimeService = processingTimeService; + } + + public StreamTask getContainingTask() { + return containingTask; + } + + public StreamConfig getStreamConfig() { + return config; + } + + public Output> getOutput() { + return output; + } + + public ProcessingTimeService getProcessingTimeService() { + return processingTimeService; + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorFactory.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorFactory.java index c30fb2fba5..c51c68aaca 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorFactory.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorFactory.java @@ -19,15 +19,13 @@ package org.apache.flink.streaming.api.operators.async; import org.apache.flink.streaming.api.datastream.AsyncDataStream; import org.apache.flink.streaming.api.functions.async.AsyncFunction; -import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.MailboxExecutor; import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; -import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.api.operators.YieldingOperatorFactory; -import org.apache.flink.streaming.runtime.tasks.StreamTask; /** * The factory of {@link AsyncWaitOperator}. @@ -61,7 +59,7 @@ public class AsyncWaitOperatorFactory extends AbstractStreamOperatorFac } @Override - public StreamOperator createStreamOperator(StreamTask containingTask, StreamConfig config, Output output) { + public > T createStreamOperator(StreamOperatorParameters parameters) { AsyncWaitOperator asyncWaitOperator = new AsyncWaitOperator( asyncFunction, timeout, @@ -69,8 +67,8 @@ public class AsyncWaitOperatorFactory extends AbstractStreamOperatorFac outputMode, processingTimeService, mailboxExecutor); - asyncWaitOperator.setup(containingTask, config, output); - return asyncWaitOperator; + asyncWaitOperator.setup(parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); + return (T) asyncWaitOperator; } @Override diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java index eafd410b82..91ea7dd625 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java @@ -41,6 +41,7 @@ import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.operators.OutputTypeConfigurable; import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.api.operators.StreamSource; import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; import org.apache.flink.streaming.api.transformations.MultipleInputTransformation; @@ -671,10 +672,7 @@ public class StreamGraphGeneratorTest extends TestLogger { private static class MultipleInputOperatorFactory implements StreamOperatorFactory { @Override - public > T createStreamOperator( - StreamTask containingTask, - StreamConfig config, - Output> output) { + public > T createStreamOperator(StreamOperatorParameters parameters) { throw new UnsupportedOperationException(); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/MailboxOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/MailboxOperatorTest.java index 850b6981bf..753ff8e429 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/MailboxOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/MailboxOperatorTest.java @@ -20,20 +20,18 @@ package org.apache.flink.streaming.runtime.operators; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeutils.base.IntSerializer; import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.MailboxExecutor; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; -import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.api.operators.YieldingOperatorFactory; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask; import org.apache.flink.streaming.runtime.tasks.OneInputStreamTaskTestHarness; -import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.apache.flink.util.TestLogger; import org.apache.flink.util.function.RunnableWithException; @@ -105,12 +103,10 @@ public class MailboxOperatorTest extends TestLogger { @Override public > Operator createStreamOperator( - StreamTask containingTask, - StreamConfig config, - Output> output) { + StreamOperatorParameters parameters) { ReplicatingMailOperator operator = new ReplicatingMailOperator(maxProcessingElements, mailboxExecutor); operator.setProcessingTimeService(processingTimeService); - operator.setup(containingTask, config, output); + operator.setup(parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); return (Operator) operator; } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskOperatorTimerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskOperatorTimerTest.java index 1d05ee7e08..bb3662cef9 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskOperatorTimerTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskOperatorTimerTest.java @@ -21,21 +21,19 @@ package org.apache.flink.streaming.runtime.operators; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.MailboxExecutor; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; -import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.api.operators.YieldingOperatorFactory; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask; import org.apache.flink.streaming.runtime.tasks.OneInputStreamTaskTestHarness; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; -import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -91,12 +89,10 @@ public class StreamTaskOperatorTimerTest extends TestLogger { @Override public > Operator createStreamOperator( - StreamTask containingTask, - StreamConfig config, - Output> output) { - TestOperator operator = new TestOperator(config.getChainIndex(), mailboxExecutor); + StreamOperatorParameters parameters) { + TestOperator operator = new TestOperator(parameters.getStreamConfig().getChainIndex(), mailboxExecutor); operator.setProcessingTimeService(processingTimeService); - operator.setup(containingTask, config, output); + operator.setup(parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); return (Operator) operator; } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java index 62c54e596f..29c22ba971 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java @@ -90,6 +90,7 @@ import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures; import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.operators.StreamMap; import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.api.operators.StreamOperatorStateContext; import org.apache.flink.streaming.api.operators.StreamSource; import org.apache.flink.streaming.api.operators.StreamTaskStateInitializer; @@ -1708,10 +1709,7 @@ public class StreamTaskTest extends TestLogger { private static class UnusedOperatorFactory extends AbstractStreamOperatorFactory { @Override - public > T createStreamOperator( - StreamTask containingTask, - StreamConfig config, - Output> output) { + public > T createStreamOperator(StreamOperatorParameters parameters) { throw new UnsupportedOperationException("This shouldn't be called"); } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/CodeGenOperatorFactory.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/CodeGenOperatorFactory.java index 56293e23b8..1648f6e5f0 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/CodeGenOperatorFactory.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/CodeGenOperatorFactory.java @@ -20,7 +20,7 @@ package org.apache.flink.table.runtime.operators; import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; import org.apache.flink.streaming.api.operators.StreamOperator; -import org.apache.flink.streaming.api.operators.StreamOperatorInitializer; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.table.runtime.generated.GeneratedClass; /** @@ -36,13 +36,13 @@ public class CodeGenOperatorFactory extends AbstractStreamOperatorFactory> T createStreamOperator(StreamOperatorInitializer initializer) { + public > T createStreamOperator(StreamOperatorParameters parameters) { return (T) generatedClass.newInstance( - initializer.getContainingTask().getUserCodeClassLoader(), + parameters.getContainingTask().getUserCodeClassLoader(), generatedClass.getReferences(), - initializer.getContainingTask(), - initializer.getStreamConfig(), - initializer.getOutput(), + parameters.getContainingTask(), + parameters.getStreamConfig(), + parameters.getOutput(), processingTimeService); } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperatorFactory.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperatorFactory.java index 75f4b796d0..e3dbe86a28 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperatorFactory.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperatorFactory.java @@ -18,12 +18,10 @@ package org.apache.flink.table.runtime.operators.wmassigners; -import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; -import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.operators.StreamOperator; -import org.apache.flink.streaming.runtime.tasks.StreamTask; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.table.dataformat.BaseRow; import org.apache.flink.table.runtime.generated.GeneratedWatermarkGenerator; import org.apache.flink.table.runtime.generated.WatermarkGenerator; @@ -53,14 +51,18 @@ public class WatermarkAssignerOperatorFactory extends AbstractStreamOperatorFact @SuppressWarnings("unchecked") @Override - public StreamOperator createStreamOperator(StreamTask containingTask, StreamConfig config, Output output) { - WatermarkGenerator watermarkGenerator = generatedWatermarkGenerator.newInstance(containingTask.getUserCodeClassLoader()); + public StreamOperator createStreamOperator(StreamOperatorParameters initializer) { + WatermarkGenerator watermarkGenerator = generatedWatermarkGenerator.newInstance( + initializer.getContainingTask().getUserCodeClassLoader()); WatermarkAssignerOperator operator = new WatermarkAssignerOperator( rowtimeFieldIndex, watermarkGenerator, idleTimeout, processingTimeService); - operator.setup(containingTask, config, output); + operator.setup( + initializer.getContainingTask(), + initializer.getStreamConfig(), + initializer.getOutput()); return operator; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/MultipleInputITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/MultipleInputITCase.java index f3356598a8..f81bac9d25 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/MultipleInputITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/MultipleInputITCase.java @@ -21,17 +21,15 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.MultipleConnectedStreams; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.Input; import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator; -import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.api.transformations.MultipleInputTransformation; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.apache.flink.test.streaming.runtime.util.TestListResultSink; import org.apache.flink.test.util.AbstractTestBase; @@ -120,12 +118,9 @@ public class MultipleInputITCase extends AbstractTestBase { private ChainingStrategy chainingStrategy; @Override - public > T createStreamOperator( - StreamTask containingTask, - StreamConfig config, - Output> output) { + public > T createStreamOperator(StreamOperatorParameters parameters) { SumAllInputOperator sumAllInputOperator = new SumAllInputOperator(); - sumAllInputOperator.setup(containingTask, config, output); + sumAllInputOperator.setup(parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); return (T) sumAllInputOperator; } -- Gitee From 16a0334879ac468b8cb5bda67116b5eab6549d1a Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Wed, 26 Feb 2020 09:39:53 +0100 Subject: [PATCH 295/885] [FLINK-16316][operators] Implement new AbstractStreamOperatorV2 as a replacement for AbstractStreamOperator The new base class for operators tries to address couple of limitations in the AbstractStreamOperator like: - lack of support for multiple inputs - setup(...) method --- .../api/operators/AbstractStreamOperator.java | 10 +- .../operators/AbstractStreamOperatorV2.java | 507 ++++++++++++++++++ .../operators/StreamOperatorParameters.java | 6 +- .../tasks/MultipleInputStreamTaskTest.java | 65 ++- .../TestBoundedMultipleInputOperator.java | 8 +- .../runtime/MultipleInputITCase.java | 34 +- 6 files changed, 590 insertions(+), 40 deletions(-) create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorV2.java diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java index 6507ce9ea3..61925f4138 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java @@ -72,7 +72,11 @@ import java.util.Optional; * the timer service, timer callbacks are also guaranteed not to be called concurrently with * methods on {@code StreamOperator}. * - * @param The output type of the operator + *

    Note, this class is going to be removed and replaced in the future by {@link AbstractStreamOperatorV2}. + * However as {@link AbstractStreamOperatorV2} is currently experimental, {@link AbstractStreamOperator} + * has not been deprecated just yet. + * + * @param The output type of the operator. */ @PublicEvolving public abstract class AbstractStreamOperator @@ -385,15 +389,18 @@ public abstract class AbstractStreamOperator * to interact with systems such as broadcast variables and managed state. This also allows * to register timers. */ + @VisibleForTesting public StreamingRuntimeContext getRuntimeContext() { return runtimeContext; } @SuppressWarnings("unchecked") + @VisibleForTesting public KeyedStateBackend getKeyedStateBackend() { return stateHandler.getKeyedStateBackend(); } + @VisibleForTesting public OperatorStateBackend getOperatorStateBackend() { return stateHandler.getOperatorStateBackend(); } @@ -402,6 +409,7 @@ public abstract class AbstractStreamOperator * Returns the {@link ProcessingTimeService} responsible for getting the current * processing time and registering timers. */ + @VisibleForTesting public ProcessingTimeService getProcessingTimeService() { return processingTimeService; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorV2.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorV2.java new file mode 100644 index 0000000000..a4ad021d24 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorV2.java @@ -0,0 +1,507 @@ +/* + * 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.operators; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.state.KeyedStateStore; +import org.apache.flink.api.common.state.State; +import org.apache.flink.api.common.state.StateDescriptor; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.MetricOptions; +import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.runtime.checkpoint.CheckpointOptions; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup; +import org.apache.flink.runtime.metrics.groups.TaskManagerJobMetricGroup; +import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; +import org.apache.flink.runtime.state.CheckpointStreamFactory; +import org.apache.flink.runtime.state.KeyedStateBackend; +import org.apache.flink.runtime.state.OperatorStateBackend; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.runtime.state.VoidNamespace; +import org.apache.flink.runtime.state.VoidNamespaceSerializer; +import org.apache.flink.streaming.api.graph.StreamConfig; +import org.apache.flink.streaming.api.operators.StreamOperatorStateHandler.CheckpointedStreamOperator; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; +import org.apache.flink.streaming.runtime.tasks.StreamTask; +import org.apache.flink.streaming.util.LatencyStats; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.Locale; +import java.util.Optional; + +/** + * New base class for all stream operators, intended to eventually replace {@link AbstractStreamOperator}. + * Currently intended to work smoothly just with {@link MultipleInputStreamOperator}. + * + *

    One note-able difference in comparison to {@link AbstractStreamOperator} is lack of + * {@link AbstractStreamOperator#setup(StreamTask, StreamConfig, Output)} in favor of initialisation + * in the constructor, and removed some tight coupling with classes like {@link StreamTask}. + * + *

    Methods are guaranteed not to be called concurrently. + * + * @param The output type of the operator + */ +@Experimental +public abstract class AbstractStreamOperatorV2 implements StreamOperator, CheckpointedStreamOperator { + /** The logger used by the operator class and its subclasses. */ + protected static final Logger LOG = LoggerFactory.getLogger(AbstractStreamOperatorV2.class); + + protected final StreamConfig config; + protected final Output> output; + private final StreamingRuntimeContext runtimeContext; + private final ExecutionConfig executionConfig; + private final ClassLoader userCodeClassLoader; + private final CloseableRegistry cancelables; + private final long[] inputWatermarks; + + /** Metric group for the operator. */ + protected final OperatorMetricGroup metrics; + protected final LatencyStats latencyStats; + protected final ProcessingTimeService processingTimeService; + + private StreamOperatorStateHandler stateHandler; + private InternalTimeServiceManager timeServiceManager; + + // We keep track of watermarks from both inputs, the combined input is the minimum + // Once the minimum advances we emit a new watermark for downstream operators + private long combinedWatermark = Long.MIN_VALUE; + + public AbstractStreamOperatorV2(StreamOperatorParameters parameters, int numberOfInputs) { + inputWatermarks = new long[numberOfInputs]; + Arrays.fill(inputWatermarks, Long.MIN_VALUE); + final Environment environment = parameters.getContainingTask().getEnvironment(); + config = parameters.getStreamConfig(); + CountingOutput countingOutput; + OperatorMetricGroup operatorMetricGroup; + try { + operatorMetricGroup = environment.getMetricGroup().getOrAddOperator(config.getOperatorID(), config.getOperatorName()); + countingOutput = new CountingOutput(parameters.getOutput(), operatorMetricGroup.getIOMetricGroup().getNumRecordsOutCounter()); + if (config.isChainStart()) { + operatorMetricGroup.getIOMetricGroup().reuseInputMetricsForTask(); + } + if (config.isChainEnd()) { + operatorMetricGroup.getIOMetricGroup().reuseOutputMetricsForTask(); + } + } catch (Exception e) { + LOG.warn("An error occurred while instantiating task metrics.", e); + countingOutput = null; + operatorMetricGroup = null; + } + + if (countingOutput == null || operatorMetricGroup == null) { + metrics = UnregisteredMetricGroups.createUnregisteredOperatorMetricGroup(); + output = parameters.getOutput(); + } + else { + metrics = operatorMetricGroup; + output = countingOutput; + } + + latencyStats = createLatencyStats( + environment.getTaskManagerInfo().getConfiguration(), + parameters.getContainingTask().getIndexInSubtaskGroup()); + + processingTimeService = Preconditions.checkNotNull(parameters.getProcessingTimeService()); + executionConfig = parameters.getContainingTask().getExecutionConfig(); + userCodeClassLoader = parameters.getContainingTask().getUserCodeClassLoader(); + cancelables = parameters.getContainingTask().getCancelables(); + + runtimeContext = new StreamingRuntimeContext( + environment, + environment.getAccumulatorRegistry().getUserMap(), + operatorMetricGroup, + getOperatorID(), + processingTimeService, + null); + } + + private LatencyStats createLatencyStats(Configuration taskManagerConfig, int indexInSubtaskGroup) { + try { + int historySize = taskManagerConfig.getInteger(MetricOptions.LATENCY_HISTORY_SIZE); + if (historySize <= 0) { + LOG.warn("{} has been set to a value equal or below 0: {}. Using default.", MetricOptions.LATENCY_HISTORY_SIZE, historySize); + historySize = MetricOptions.LATENCY_HISTORY_SIZE.defaultValue(); + } + + final String configuredGranularity = taskManagerConfig.getString(MetricOptions.LATENCY_SOURCE_GRANULARITY); + LatencyStats.Granularity granularity; + try { + granularity = LatencyStats.Granularity.valueOf(configuredGranularity.toUpperCase(Locale.ROOT)); + } catch (IllegalArgumentException iae) { + granularity = LatencyStats.Granularity.OPERATOR; + LOG.warn( + "Configured value {} option for {} is invalid. Defaulting to {}.", + configuredGranularity, + MetricOptions.LATENCY_SOURCE_GRANULARITY.key(), + granularity); + } + TaskManagerJobMetricGroup jobMetricGroup = this.metrics.parent().parent(); + return new LatencyStats(jobMetricGroup.addGroup("latency"), + historySize, + indexInSubtaskGroup, + getOperatorID(), + granularity); + } catch (Exception e) { + LOG.warn("An error occurred while instantiating latency metrics.", e); + return new LatencyStats( + UnregisteredMetricGroups.createUnregisteredTaskManagerJobMetricGroup().addGroup("latency"), + 1, + 0, + new OperatorID(), + LatencyStats.Granularity.SINGLE); + } + } + + @Override + public MetricGroup getMetricGroup() { + return metrics; + } + + @Override + public final void initializeState(StreamTaskStateInitializer streamTaskStateManager) throws Exception { + final TypeSerializer keySerializer = config.getStateKeySerializer(getUserCodeClassloader()); + + final StreamOperatorStateContext context = + streamTaskStateManager.streamOperatorStateContext( + getOperatorID(), + getClass().getSimpleName(), + getProcessingTimeService(), + this, + keySerializer, + cancelables, + metrics); + + stateHandler = new StreamOperatorStateHandler(context, getExecutionConfig(), cancelables); + timeServiceManager = context.internalTimerServiceManager(); + stateHandler.initializeOperatorState(this); + } + + /** + * This method is called immediately before any elements are processed, it should contain the + * operator's initialization logic, e.g. state initialization. + * + *

    The default implementation does nothing. + * + * @throws Exception An exception in this method causes the operator to fail. + */ + @Override + public void open() throws Exception {} + + /** + * This method is called after all records have been added to the operators via the methods + * {@link OneInputStreamOperator#processElement(StreamRecord)}, or + * {@link TwoInputStreamOperator#processElement1(StreamRecord)} and + * {@link TwoInputStreamOperator#processElement2(StreamRecord)}. + * + *

    The method is expected to flush all remaining buffered data. Exceptions during this flushing + * of buffered should be propagated, in order to cause the operation to be recognized asa failed, + * because the last data items are not processed properly. + * + * @throws Exception An exception in this method causes the operator to fail. + */ + @Override + public void close() throws Exception {} + + /** + * This method is called at the very end of the operator's life, both in the case of a successful + * completion of the operation, and in the case of a failure and canceling. + * + *

    This method is expected to make a thorough effort to release all resources + * that the operator has acquired. + */ + @Override + public void dispose() throws Exception { + if (stateHandler != null) { + stateHandler.dispose(); + } + } + + @Override + public void prepareSnapshotPreBarrier(long checkpointId) throws Exception { + // the default implementation does nothing and accepts the checkpoint + // this is purely for subclasses to override + } + + @Override + public final OperatorSnapshotFutures snapshotState(long checkpointId, long timestamp, CheckpointOptions checkpointOptions, + CheckpointStreamFactory factory) throws Exception { + return stateHandler.snapshotState( + this, + Optional.ofNullable(timeServiceManager), + getOperatorName(), + checkpointId, + timestamp, + checkpointOptions, + factory); + } + + /** + * Stream operators with state, which want to participate in a snapshot need to override this hook method. + * + * @param context context that provides information and means required for taking a snapshot + */ + @Override + public void snapshotState(StateSnapshotContext context) throws Exception { + } + + /** + * Stream operators with state which can be restored need to override this hook method. + * + * @param context context that allows to register different states. + */ + @Override + public void initializeState(StateInitializationContext context) throws Exception { + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + stateHandler.notifyCheckpointComplete(checkpointId); + } + + // ------------------------------------------------------------------------ + // Properties and Services + // ------------------------------------------------------------------------ + + /** + * Gets the execution config defined on the execution environment of the job to which this + * operator belongs. + * + * @return The job's execution config. + */ + public ExecutionConfig getExecutionConfig() { + return executionConfig; + } + + public StreamConfig getOperatorConfig() { + return config; + } + + public ClassLoader getUserCodeClassloader() { + return userCodeClassLoader; + } + + /** + * Return the operator name. If the runtime context has been set, then the task name with + * subtask index is returned. Otherwise, the simple class name is returned. + * + * @return If runtime context is set, then return task name with subtask index. Otherwise return + * simple class name. + */ + protected String getOperatorName() { + if (runtimeContext != null) { + return runtimeContext.getTaskNameWithSubtasks(); + } else { + return getClass().getSimpleName(); + } + } + + /** + * Returns a context that allows the operator to query information about the execution and also + * to interact with systems such as broadcast variables and managed state. This also allows + * to register timers. + */ + public StreamingRuntimeContext getRuntimeContext() { + return runtimeContext; + } + + @SuppressWarnings("unchecked") + @VisibleForTesting + public KeyedStateBackend getKeyedStateBackend() { + return (KeyedStateBackend) stateHandler.getKeyedStateBackend(); + } + + @VisibleForTesting + public OperatorStateBackend getOperatorStateBackend() { + return stateHandler.getOperatorStateBackend(); + } + + /** + * Returns the {@link ProcessingTimeService} responsible for getting the current + * processing time and registering timers. + */ + @VisibleForTesting + public ProcessingTimeService getProcessingTimeService() { + return processingTimeService; + } + + /** + * Creates a partitioned state handle, using the state backend configured for this task. + * + * @throws IllegalStateException Thrown, if the key/value state was already initialized. + * @throws Exception Thrown, if the state backend cannot create the key/value state. + */ + protected S getPartitionedState(StateDescriptor stateDescriptor) throws Exception { + return getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, stateDescriptor); + } + + protected S getOrCreateKeyedState( + TypeSerializer namespaceSerializer, + StateDescriptor stateDescriptor) throws Exception { + return stateHandler.getOrCreateKeyedState(namespaceSerializer, stateDescriptor); + } + + /** + * Creates a partitioned state handle, using the state backend configured for this task. + * + * @throws IllegalStateException Thrown, if the key/value state was already initialized. + * @throws Exception Thrown, if the state backend cannot create the key/value state. + */ + protected S getPartitionedState( + N namespace, + TypeSerializer namespaceSerializer, + StateDescriptor stateDescriptor) throws Exception { + return stateHandler.getPartitionedState(namespace, namespaceSerializer, stateDescriptor); + } + + protected void internalSetKeyContextElement(StreamRecord record, KeySelector selector) throws Exception { + if (selector != null) { + Object key = selector.getKey(record.getValue()); + setCurrentKey(key); + } + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + public void setCurrentKey(Object key) { + stateHandler.setCurrentKey(key); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + public Object getCurrentKey() { + return stateHandler.getCurrentKey(); + } + + public Optional getKeyedStateStore() { + if (stateHandler == null) { + return Optional.empty(); + } + return stateHandler.getKeyedStateStore(); + } + + protected void reportOrForwardLatencyMarker(LatencyMarker marker) { + // all operators are tracking latencies + this.latencyStats.reportLatency(marker); + + // everything except sinks forwards latency markers + this.output.emitLatencyMarker(marker); + } + + // ------------------------------------------------------------------------ + // Watermark handling + // ------------------------------------------------------------------------ + + /** + * Returns a {@link InternalTimerService} that can be used to query current processing time + * and event time and to set timers. An operator can have several timer services, where + * each has its own namespace serializer. Timer services are differentiated by the string + * key that is given when requesting them, if you call this method with the same key + * multiple times you will get the same timer service instance in subsequent requests. + * + *

    Timers are always scoped to a key, the currently active key of a keyed stream operation. + * When a timer fires, this key will also be set as the currently active key. + * + *

    Each timer has attached metadata, the namespace. Different timer services + * can have a different namespace type. If you don't need namespace differentiation you + * can use {@link VoidNamespaceSerializer} as the namespace serializer. + * + * @param name The name of the requested timer service. If no service exists under the given + * name a new one will be created and returned. + * @param namespaceSerializer {@code TypeSerializer} for the timer namespace. + * @param triggerable The {@link Triggerable} that should be invoked when timers fire + * + * @param The type of the timer namespace. + */ + @VisibleForTesting + public InternalTimerService getInternalTimerService( + String name, + TypeSerializer namespaceSerializer, + Triggerable triggerable) { + if (timeServiceManager == null) { + throw new RuntimeException("The timer service has not been initialized."); + } + InternalTimeServiceManager keyedTimeServiceHandler = (InternalTimeServiceManager) timeServiceManager; + return keyedTimeServiceHandler.getInternalTimerService( + name, + namespaceSerializer, + triggerable, + stateHandler.getKeyedStateBackend()); + } + + public void processWatermark(Watermark mark) throws Exception { + if (timeServiceManager != null) { + timeServiceManager.advanceWatermark(mark); + } + output.emitWatermark(mark); + } + + protected void reportWatermark(Watermark mark, int inputId) throws Exception { + inputWatermarks[inputId] = mark.getTimestamp(); + long newMin = mark.getTimestamp(); + for (long inputWatermark : inputWatermarks) { + newMin = Math.min(inputWatermark, newMin); + } + if (newMin > combinedWatermark) { + combinedWatermark = newMin; + processWatermark(new Watermark(combinedWatermark)); + } + } + + @Override + public OperatorID getOperatorID() { + return config.getOperatorID(); + } + + @VisibleForTesting + public int numProcessingTimeTimers() { + return timeServiceManager == null ? 0 : timeServiceManager.numProcessingTimeTimers(); + } + + @VisibleForTesting + public int numEventTimeTimers() { + return timeServiceManager == null ? 0 : timeServiceManager.numEventTimeTimers(); + } + + @Override + public void setKeyContextElement1(StreamRecord record) throws Exception { + throw new IllegalStateException("This method should never be called. Use Input class instead"); + } + + @Override + public void setKeyContextElement2(StreamRecord record) throws Exception { + throw new IllegalStateException("This method should never be called. Use Input class instead"); + } + + protected Optional> getTimeServiceManager() { + return Optional.ofNullable(timeServiceManager); + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorParameters.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorParameters.java index 1f6f1a03a6..5aaffcc46a 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorParameters.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorParameters.java @@ -25,9 +25,9 @@ import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; import org.apache.flink.streaming.runtime.tasks.StreamTask; /** - * Helper class to construct {@link StreamOperatorBase}. Wraps couple of internal parameters - * to simplify for users construction of classes extending {@link StreamOperatorBase} and to - * allow for backward compatible changes in the {@link StreamOperatorBase}'s constructor. + * Helper class to construct {@link AbstractStreamOperatorV2}. Wraps couple of internal parameters + * to simplify for users construction of classes extending {@link AbstractStreamOperatorV2} and to + * allow for backward compatible changes in the {@link AbstractStreamOperatorV2}'s constructor. * * @param The output type of an operator that will be constructed using {@link StreamOperatorParameters}. */ diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTaskTest.java index fc0144b732..88c4f9a66a 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTaskTest.java @@ -30,9 +30,12 @@ import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup; import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.streaming.api.functions.co.CoMapFunction; -import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorV2; import org.apache.flink.streaming.api.operators.Input; import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.api.operators.co.CoStreamMap; import org.apache.flink.streaming.runtime.io.InputStatus; import org.apache.flink.streaming.runtime.io.StreamMultipleInputProcessor; @@ -68,7 +71,7 @@ public class MultipleInputStreamTaskTest { .addInput(BasicTypeInfo.STRING_TYPE_INFO) .addInput(BasicTypeInfo.INT_TYPE_INFO) .addInput(BasicTypeInfo.DOUBLE_TYPE_INFO) - .setupOutputForSingletonOperatorChain(new MapToStringMultipleInputOperator()) + .setupOutputForSingletonOperatorChain(new MapToStringMultipleInputOperatorFactory()) .build()) { long initialTime = 0L; @@ -98,7 +101,7 @@ public class MultipleInputStreamTaskTest { .addInput(BasicTypeInfo.STRING_TYPE_INFO, 2) .addInput(BasicTypeInfo.INT_TYPE_INFO, 2) .addInput(BasicTypeInfo.DOUBLE_TYPE_INFO, 2) - .setupOutputForSingletonOperatorChain(new MapToStringMultipleInputOperator()) + .setupOutputForSingletonOperatorChain(new MapToStringMultipleInputOperatorFactory()) .build()) { ArrayDeque expectedOutput = new ArrayDeque<>(); long initialTime = 0L; @@ -148,7 +151,7 @@ public class MultipleInputStreamTaskTest { .addInput(BasicTypeInfo.STRING_TYPE_INFO, 2) .addInput(BasicTypeInfo.INT_TYPE_INFO, 2) .addInput(BasicTypeInfo.DOUBLE_TYPE_INFO, 2) - .setupOutputForSingletonOperatorChain(new MapToStringMultipleInputOperator()) + .setupOutputForSingletonOperatorChain(new MapToStringMultipleInputOperatorFactory()) .build()) { ArrayDeque expectedOutput = new ArrayDeque<>(); long initialTime = 0L; @@ -217,7 +220,7 @@ public class MultipleInputStreamTaskTest { .addInput(BasicTypeInfo.STRING_TYPE_INFO) .addInput(BasicTypeInfo.STRING_TYPE_INFO) .addInput(BasicTypeInfo.STRING_TYPE_INFO) - .setupOperatorChain(new DuplicatingOperator()) + .setupOperatorChain(new DuplicatingOperatorFactory()) .chain(new OneInputStreamTaskTest.DuplicatingOperator(), BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig())) .chain(new OneInputStreamTaskTest.DuplicatingOperator(), BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig())) .finish() @@ -246,9 +249,13 @@ public class MultipleInputStreamTaskTest { } } - static class DuplicatingOperator extends AbstractStreamOperator + static class DuplicatingOperator extends AbstractStreamOperatorV2 implements MultipleInputStreamOperator { + public DuplicatingOperator(StreamOperatorParameters parameters) { + super(parameters, 3); + } + @Override public List getInputs() { return Arrays.asList(new DuplicatingInput(), new DuplicatingInput(), new DuplicatingInput()); @@ -270,7 +277,7 @@ public class MultipleInputStreamTaskTest { .addInput(BasicTypeInfo.STRING_TYPE_INFO) .addInput(BasicTypeInfo.STRING_TYPE_INFO) .addInput(BasicTypeInfo.STRING_TYPE_INFO) - .setupOperatorChain(new TestBoundedMultipleInputOperator("Operator0")) + .setupOperatorChain(new TestBoundedMultipleInputOperatorFactory()) .chain(new TestBoundedOneInputStreamOperator("Operator1"), BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig())) .finish() .build(); @@ -317,7 +324,7 @@ public class MultipleInputStreamTaskTest { .addInput(BasicTypeInfo.STRING_TYPE_INFO) .addInput(BasicTypeInfo.STRING_TYPE_INFO) .addInput(BasicTypeInfo.STRING_TYPE_INFO) - .setupOutputForSingletonOperatorChain(new MapToStringMultipleInputOperator()) + .setupOutputForSingletonOperatorChain(new MapToStringMultipleInputOperatorFactory()) .build()) { ArrayDeque expectedOutput = new ArrayDeque<>(); @@ -352,12 +359,16 @@ public class MultipleInputStreamTaskTest { // This must only be used in one test, otherwise the static fields will be changed // by several tests concurrently private static class MapToStringMultipleInputOperator - extends AbstractStreamOperator implements MultipleInputStreamOperator { + extends AbstractStreamOperatorV2 implements MultipleInputStreamOperator { private static final long serialVersionUID = 1L; private boolean openCalled; private boolean closeCalled; + public MapToStringMultipleInputOperator(StreamOperatorParameters parameters) { + super(parameters, 3); + } + @Override public void open() throws Exception { super.open(); @@ -418,5 +429,41 @@ public class MultipleInputStreamTaskTest { return value.toString(); } } + + private static class TestBoundedMultipleInputOperatorFactory extends AbstractStreamOperatorFactory { + @Override + public > T createStreamOperator(StreamOperatorParameters parameters) { + return (T) new TestBoundedMultipleInputOperator("Operator0", parameters); + } + + @Override + public Class> getStreamOperatorClass(ClassLoader classLoader) { + return TestBoundedMultipleInputOperator.class; + } + } + + private static class DuplicatingOperatorFactory extends AbstractStreamOperatorFactory { + @Override + public > T createStreamOperator(StreamOperatorParameters parameters) { + return (T) new DuplicatingOperator(parameters); + } + + @Override + public Class> getStreamOperatorClass(ClassLoader classLoader) { + return DuplicatingOperator.class; + } + } + + private static class MapToStringMultipleInputOperatorFactory extends AbstractStreamOperatorFactory { + @Override + public > T createStreamOperator(StreamOperatorParameters parameters) { + return (T) new MapToStringMultipleInputOperator(parameters); + } + + @Override + public Class> getStreamOperatorClass(ClassLoader classLoader) { + return MapToStringMultipleInputOperator.class; + } + } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TestBoundedMultipleInputOperator.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TestBoundedMultipleInputOperator.java index 89d242a315..e6ebe84a8b 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TestBoundedMultipleInputOperator.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TestBoundedMultipleInputOperator.java @@ -18,10 +18,11 @@ package org.apache.flink.streaming.util; -import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorV2; import org.apache.flink.streaming.api.operators.BoundedMultiInput; import org.apache.flink.streaming.api.operators.Input; import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import java.util.Arrays; @@ -30,14 +31,15 @@ import java.util.List; /** * A test operator class implementing {@link BoundedMultiInput}. */ -public class TestBoundedMultipleInputOperator extends AbstractStreamOperator +public class TestBoundedMultipleInputOperator extends AbstractStreamOperatorV2 implements MultipleInputStreamOperator, BoundedMultiInput { private static final long serialVersionUID = 1L; private final String name; - public TestBoundedMultipleInputOperator(String name) { + public TestBoundedMultipleInputOperator(String name, StreamOperatorParameters parameters) { + super(parameters, 3); this.name = name; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/MultipleInputITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/MultipleInputITCase.java index f81bac9d25..32b64249ac 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/MultipleInputITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/MultipleInputITCase.java @@ -21,12 +21,11 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.MultipleConnectedStreams; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.operators.AbstractStreamOperator; -import org.apache.flink.streaming.api.operators.ChainingStrategy; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorV2; import org.apache.flink.streaming.api.operators.Input; import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator; import org.apache.flink.streaming.api.operators.StreamOperator; -import org.apache.flink.streaming.api.operators.StreamOperatorFactory; import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.api.transformations.MultipleInputTransformation; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -84,13 +83,14 @@ public class MultipleInputITCase extends AbstractTestBase { /** * 3 input operator that sums all of it inputs. - * TODO: provide non {@link SetupableStreamOperator} variant of {@link AbstractStreamOperator}? - * TODO: provide non {@link AbstractStreamOperator} seems to pre-override processWatermark1/2 and other - * methods that are not defined there? */ - public static class SumAllInputOperator extends AbstractStreamOperator implements MultipleInputStreamOperator { + public static class SumAllInputOperator extends AbstractStreamOperatorV2 implements MultipleInputStreamOperator { private long sum; + public SumAllInputOperator(StreamOperatorParameters parameters) { + super(parameters, 3); + } + @Override public List getInputs() { return Arrays.asList( @@ -114,29 +114,15 @@ public class MultipleInputITCase extends AbstractTestBase { /** * Factory for {@link SumAllInputOperator}. */ - public static class SumAllInputOperatorFactory implements StreamOperatorFactory { - private ChainingStrategy chainingStrategy; - + public static class SumAllInputOperatorFactory extends AbstractStreamOperatorFactory { @Override public > T createStreamOperator(StreamOperatorParameters parameters) { - SumAllInputOperator sumAllInputOperator = new SumAllInputOperator(); - sumAllInputOperator.setup(parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); - return (T) sumAllInputOperator; - } - - @Override - public void setChainingStrategy(ChainingStrategy chainingStrategy) { - this.chainingStrategy = chainingStrategy; - } - - @Override - public ChainingStrategy getChainingStrategy() { - return chainingStrategy; + return (T) new SumAllInputOperator(parameters); } @Override public Class getStreamOperatorClass(ClassLoader classLoader) { - throw new UnsupportedOperationException(); + return SumAllInputOperator.class; } } } -- Gitee From 73d410354d367e91ec6f68af87115cee0d86a0ac Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Fri, 13 Mar 2020 12:21:08 +0100 Subject: [PATCH 296/885] [FLINK-16316][operators] Make StreamOperatorFactory Experimental --- .../api/operators/AbstractStreamOperatorFactory.java | 2 ++ .../flink/streaming/api/operators/StreamOperatorFactory.java | 4 ++-- .../streaming/api/operators/YieldingOperatorFactory.java | 3 +++ 3 files changed, 7 insertions(+), 2 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorFactory.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorFactory.java index f29c860a57..23fa40d75e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorFactory.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorFactory.java @@ -17,6 +17,7 @@ package org.apache.flink.streaming.api.operators; +import org.apache.flink.annotation.Experimental; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeServiceAware; @@ -25,6 +26,7 @@ import org.apache.flink.streaming.runtime.tasks.ProcessingTimeServiceAware; * {@link ProcessingTimeServiceAware} interface which enables stream operators to access * {@link ProcessingTimeService}. */ +@Experimental public abstract class AbstractStreamOperatorFactory implements StreamOperatorFactory, ProcessingTimeServiceAware { protected ChainingStrategy chainingStrategy = ChainingStrategy.ALWAYS; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorFactory.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorFactory.java index 19df0011af..d1ea80c0ce 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorFactory.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorFactory.java @@ -17,7 +17,7 @@ package org.apache.flink.streaming.api.operators; -import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.Experimental; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.api.graph.StreamGraph; @@ -30,7 +30,7 @@ import java.io.Serializable; * * @param The output type of the operator */ -@Internal +@Experimental public interface StreamOperatorFactory extends Serializable { /** diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/YieldingOperatorFactory.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/YieldingOperatorFactory.java index 054ad8d51d..c3563c5e79 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/YieldingOperatorFactory.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/YieldingOperatorFactory.java @@ -17,10 +17,13 @@ package org.apache.flink.streaming.api.operators; +import org.apache.flink.annotation.Experimental; + /** * An operator that needs access to the {@link MailboxExecutor} to yield to downstream operators needs to be created * through a factory implementing this interface. */ +@Experimental public interface YieldingOperatorFactory extends StreamOperatorFactory { void setMailboxExecutor(MailboxExecutor mailboxExecutor); } -- Gitee From 66101d8781207326885f644fe2ad1ca4fd65f495 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Fri, 13 Mar 2020 12:57:57 +0100 Subject: [PATCH 297/885] [FLINK-16166][build] Append additional javadoc options --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 71f0265037..5c0cfb1dc4 100644 --- a/pom.xml +++ b/pom.xml @@ -1737,7 +1737,7 @@ under the License. true false - + -Xdoclint:none -- Gitee From 90a9b3edf299f1f259aa3416893ba0328c8dd9a1 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Tue, 11 Feb 2020 08:25:12 +0100 Subject: [PATCH 298/885] [FLINK-16377][table] Support calls to inline functions in the expressions DSL. --- .../apache/flink/table/api/Expressions.java | 15 +- .../rules/ResolveCallByArgumentsRule.java | 20 ++- .../operations/CalculatedQueryOperation.java | 46 +++-- .../operations/QueryOperationVisitor.java | 2 +- .../utils/OperationTreeBuilder.java | 11 +- .../factories/CalculatedTableFactory.java | 101 ++++++----- .../resolver/ExpressionResolverTest.java | 124 +++++++++++-- .../flink/table/api/expressionDsl.scala | 8 + .../PlannerTypeInferenceUtilImpl.java | 5 + .../converter/CallExpressionConvertRule.java | 3 + .../converter/ExpressionConverter.java | 14 +- .../FunctionDefinitionConvertRule.java | 65 +++++++ ...a => LegacyScalarFunctionConvertRule.java} | 2 +- .../bridging/BridgingSqlAggFunction.java | 13 +- .../bridging/BridgingSqlFunction.java | 15 +- .../functions/bridging/BridgingUtils.java | 59 ++++++- .../planner/plan/QueryOperationConverter.java | 87 ++++++--- .../codegen/CorrelateCodeGenerator.scala | 11 +- .../PlannerExpressionConverter.scala | 24 ++- .../planner/expressions/aggregations.scala | 19 ++ .../table/planner/expressions/call.scala | 15 ++ .../functions/utils/TableSqlFunction.scala | 3 +- .../runtime/stream/table/FunctionITCase.java | 165 ++++++++++++++++++ .../CorrelateStringExpressionTest.scala | 33 ++-- .../table/validation/CalcValidationTest.scala | 13 -- .../validation/CorrelateValidationTest.scala | 4 - .../table/plan/QueryOperationConverter.java | 68 +++++--- .../PlannerExpressionConverter.scala | 4 + .../runtime/stream/sql/FunctionITCase.java | 31 ++++ .../runtime/stream/table/FunctionITCase.java | 98 +++++++++++ .../validation/CorrelateValidationTest.scala | 4 - 31 files changed, 873 insertions(+), 209 deletions(-) create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/FunctionDefinitionConvertRule.java rename flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/{ScalarFunctionConvertRule.java => LegacyScalarFunctionConvertRule.java} (96%) create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/table/FunctionITCase.java create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/stream/table/FunctionITCase.java diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java index 9e0c10ed15..60717642bf 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java @@ -26,6 +26,7 @@ import org.apache.flink.table.expressions.ResolvedExpression; import org.apache.flink.table.expressions.TimePointUnit; import org.apache.flink.table.functions.BuiltInFunctionDefinitions; import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.flink.table.functions.UserDefinedFunction; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.utils.TypeConversions; import org.apache.flink.table.types.utils.ValueDataTypeConverter; @@ -509,10 +510,20 @@ public final class Expressions { * @see TableEnvironment#createTemporaryFunction * @see TableEnvironment#createTemporarySystemFunction */ - public static ApiExpression call(String path, Object... params) { + public static ApiExpression call(String path, Object... arguments) { return new ApiExpression(ApiExpressionUtils.lookupCall( path, - Arrays.stream(params).map(ApiExpressionUtils::objectToExpression).toArray(Expression[]::new))); + Arrays.stream(arguments).map(ApiExpressionUtils::objectToExpression).toArray(Expression[]::new))); + } + + /** + * A call to an unregistered, inline function. + * + *

    For functions that have been registered before and are identified by a name, use + * {@link #call(String, Object...)}. + */ + public static ApiExpression call(UserDefinedFunction function, Object... arguments) { + return apiCall(function, arguments); } private static ApiExpression apiCall(FunctionDefinition functionDefinition, Object... args) { diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/rules/ResolveCallByArgumentsRule.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/rules/ResolveCallByArgumentsRule.java index 1c2dedf7d6..3526f48aa7 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/rules/ResolveCallByArgumentsRule.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/rules/ResolveCallByArgumentsRule.java @@ -31,7 +31,6 @@ import org.apache.flink.table.expressions.ResolvedExpression; import org.apache.flink.table.expressions.UnresolvedCallExpression; import org.apache.flink.table.expressions.ValueLiteralExpression; import org.apache.flink.table.functions.AggregateFunctionDefinition; -import org.apache.flink.table.functions.BuiltInFunctionDefinition; import org.apache.flink.table.functions.BuiltInFunctionDefinitions; import org.apache.flink.table.functions.FunctionDefinition; import org.apache.flink.table.functions.FunctionIdentifier; @@ -174,14 +173,19 @@ final class ResolveCallByArgumentsRule implements ResolverRule { * Temporary method until all calls define a type inference. */ private Optional getOptionalTypeInference(FunctionDefinition definition) { - if (definition instanceof BuiltInFunctionDefinition) { - final BuiltInFunctionDefinition builtInDefinition = (BuiltInFunctionDefinition) definition; - final TypeInference inference = builtInDefinition.getTypeInference(resolutionContext.typeFactory()); - if (inference.getOutputTypeStrategy() != TypeStrategies.MISSING) { - return Optional.of(inference); - } + if (definition instanceof ScalarFunctionDefinition || + definition instanceof TableFunctionDefinition || + definition instanceof AggregateFunctionDefinition || + definition instanceof TableAggregateFunctionDefinition) { + return Optional.empty(); + } + + final TypeInference inference = definition.getTypeInference(resolutionContext.typeFactory()); + if (inference.getOutputTypeStrategy() != TypeStrategies.MISSING) { + return Optional.of(inference); + } else { + return Optional.empty(); } - return Optional.empty(); } private ResolvedExpression runTypeInference( diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/CalculatedQueryOperation.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/CalculatedQueryOperation.java index feff2fa8ee..b5e80ca8f4 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/CalculatedQueryOperation.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/CalculatedQueryOperation.java @@ -19,48 +19,52 @@ package org.apache.flink.table.operations; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.flink.table.functions.FunctionIdentifier; import org.apache.flink.table.functions.TableFunction; +import javax.annotation.Nullable; + import java.util.Collections; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Optional; /** * Describes a relational operation that was created from applying a {@link TableFunction}. */ @Internal -public class CalculatedQueryOperation implements QueryOperation { +public class CalculatedQueryOperation implements QueryOperation { - private final TableFunction tableFunction; - private final List parameters; - private final TypeInformation resultType; + private final FunctionDefinition functionDefinition; + private final @Nullable FunctionIdentifier functionIdentifier; + private final List arguments; private final TableSchema tableSchema; public CalculatedQueryOperation( - TableFunction tableFunction, - List parameters, - TypeInformation resultType, + FunctionDefinition functionDefinition, + @Nullable FunctionIdentifier functionIdentifier, + List arguments, TableSchema tableSchema) { - this.tableFunction = tableFunction; - this.parameters = parameters; - this.resultType = resultType; + this.functionDefinition = functionDefinition; + this.functionIdentifier = functionIdentifier; + this.arguments = arguments; this.tableSchema = tableSchema; } - public TableFunction getTableFunction() { - return tableFunction; + public FunctionDefinition getFunctionDefinition() { + return functionDefinition; } - public List getParameters() { - return parameters; + public Optional getFunctionIdentifier() { + return Optional.ofNullable(functionIdentifier); } - public TypeInformation getResultType() { - return resultType; + public List getArguments() { + return arguments; } @Override @@ -71,8 +75,12 @@ public class CalculatedQueryOperation implements QueryOperation { @Override public String asSummaryString() { Map args = new LinkedHashMap<>(); - args.put("function", tableFunction); - args.put("parameters", parameters); + if (functionIdentifier != null) { + args.put("function", functionIdentifier); + } else { + args.put("function", functionDefinition.toString()); + } + args.put("arguments", arguments); return OperationUtils.formatWithChildren("CalculatedTable", args, getChildren(), Operation::asSummaryString); } diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/QueryOperationVisitor.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/QueryOperationVisitor.java index 3673a9653b..fe8bd8339f 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/QueryOperationVisitor.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/QueryOperationVisitor.java @@ -43,7 +43,7 @@ public interface QueryOperationVisitor { T visit(SortQueryOperation sort); - T visit(CalculatedQueryOperation calculatedTable); + T visit(CalculatedQueryOperation calculatedTable); T visit(CatalogQueryOperation catalogTable); diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/utils/OperationTreeBuilder.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/utils/OperationTreeBuilder.java index 51242dc596..b9c6f1f1be 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/utils/OperationTreeBuilder.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/utils/OperationTreeBuilder.java @@ -471,9 +471,14 @@ public final class OperationTreeBuilder { throw new ValidationException("Only a table function can be used in the flatMap operator."); } - TypeInformation resultType = ((TableFunctionDefinition) ((UnresolvedCallExpression) resolvedTableFunction) - .getFunctionDefinition()) - .getResultType(); + FunctionDefinition functionDefinition = ((UnresolvedCallExpression) resolvedTableFunction) + .getFunctionDefinition(); + if (!(functionDefinition instanceof TableFunctionDefinition)) { + throw new ValidationException( + "The new type inference for functions is not supported in the flatMap yet."); + } + + TypeInformation resultType = ((TableFunctionDefinition) functionDefinition).getResultType(); List originFieldNames = Arrays.asList(FieldInfoUtils.getFieldNames(resultType)); List childFields = Arrays.asList(child.getTableSchema().getFieldNames()); diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/utils/factories/CalculatedTableFactory.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/utils/factories/CalculatedTableFactory.java index 0c0ecee215..cfd3bcad12 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/utils/factories/CalculatedTableFactory.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/utils/factories/CalculatedTableFactory.java @@ -19,7 +19,6 @@ package org.apache.flink.table.operations.utils.factories; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.expressions.CallExpression; @@ -28,19 +27,19 @@ import org.apache.flink.table.expressions.ExpressionUtils; import org.apache.flink.table.expressions.ResolvedExpression; import org.apache.flink.table.expressions.utils.ResolvedExpressionDefaultVisitor; import org.apache.flink.table.functions.FunctionDefinition; -import org.apache.flink.table.functions.TableFunctionDefinition; +import org.apache.flink.table.functions.FunctionIdentifier; import org.apache.flink.table.operations.CalculatedQueryOperation; import org.apache.flink.table.operations.QueryOperation; -import org.apache.flink.table.typeutils.FieldInfoUtils; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; +import org.apache.flink.table.types.utils.DataTypeUtils; import java.util.Arrays; import java.util.Collections; import java.util.List; import static java.util.stream.Collectors.toList; -import static org.apache.flink.table.expressions.ApiExpressionUtils.isFunctionOfKind; import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.AS; -import static org.apache.flink.table.functions.FunctionKind.TABLE; /** * Utility class for creating a valid {@link CalculatedQueryOperation} operation. @@ -59,30 +58,25 @@ public class CalculatedTableFactory { return callExpr.accept(calculatedTableCreator); } - private class FunctionTableCallVisitor extends ResolvedExpressionDefaultVisitor> { - - private String[] leftTableFieldNames; + private static class FunctionTableCallVisitor extends ResolvedExpressionDefaultVisitor { + private List leftTableFieldNames; + private static final String ATOMIC_FIELD_NAME = "f0"; public FunctionTableCallVisitor(String[] leftTableFieldNames) { - this.leftTableFieldNames = leftTableFieldNames; + this.leftTableFieldNames = Arrays.asList(leftTableFieldNames); } @Override - public CalculatedQueryOperation visit(CallExpression call) { + public CalculatedQueryOperation visit(CallExpression call) { FunctionDefinition definition = call.getFunctionDefinition(); if (definition.equals(AS)) { return unwrapFromAlias(call); - } else if (definition instanceof TableFunctionDefinition) { - return createFunctionCall( - (TableFunctionDefinition) definition, - Collections.emptyList(), - call.getResolvedChildren()); - } else { - return defaultMethod(call); } + + return createFunctionCall(call, Collections.emptyList(), call.getResolvedChildren()); } - private CalculatedQueryOperation unwrapFromAlias(CallExpression call) { + private CalculatedQueryOperation unwrapFromAlias(CallExpression call) { List children = call.getChildren(); List aliases = children.subList(1, children.size()) .stream() @@ -90,51 +84,76 @@ public class CalculatedTableFactory { .orElseThrow(() -> new ValidationException("Unexpected alias: " + alias))) .collect(toList()); - if (!isFunctionOfKind(children.get(0), TABLE)) { + if (!(children.get(0) instanceof CallExpression)) { throw fail(); } CallExpression tableCall = (CallExpression) children.get(0); - TableFunctionDefinition tableFunctionDefinition = - (TableFunctionDefinition) tableCall.getFunctionDefinition(); - return createFunctionCall(tableFunctionDefinition, aliases, tableCall.getResolvedChildren()); + return createFunctionCall(tableCall, aliases, tableCall.getResolvedChildren()); } - private CalculatedQueryOperation createFunctionCall( - TableFunctionDefinition tableFunctionDefinition, + private CalculatedQueryOperation createFunctionCall( + CallExpression callExpression, List aliases, List parameters) { - TypeInformation resultType = tableFunctionDefinition.getResultType(); - int callArity = resultType.getTotalFields(); - int aliasesSize = aliases.size(); + FunctionDefinition functionDefinition = callExpression.getFunctionDefinition(); + final TableSchema tableSchema = adjustNames( + extractSchema(callExpression.getOutputDataType()), + aliases, + callExpression.getFunctionIdentifier() + .map(FunctionIdentifier::asSummaryString) + .orElse(functionDefinition.toString())); + + return new CalculatedQueryOperation( + functionDefinition, + callExpression.getFunctionIdentifier().orElse(null), + parameters, + tableSchema); + } - String[] fieldNames; + private TableSchema extractSchema(DataType resultDataType) { + if (LogicalTypeChecks.isCompositeType(resultDataType.getLogicalType())) { + return DataTypeUtils.expandCompositeTypeToSchema(resultDataType); + } + + int i = 0; + String fieldName = ATOMIC_FIELD_NAME; + while (leftTableFieldNames.contains(fieldName)) { + fieldName = ATOMIC_FIELD_NAME + "_" + i++; + } + return TableSchema.builder() + .field(fieldName, resultDataType) + .build(); + } + + private TableSchema adjustNames( + TableSchema tableSchema, + List aliases, + String functionName) { + int aliasesSize = aliases.size(); if (aliasesSize == 0) { - fieldNames = FieldInfoUtils.getFieldNames(resultType, Arrays.asList(leftTableFieldNames)); - } else if (aliasesSize != callArity) { + return tableSchema; + } + + int callArity = tableSchema.getFieldCount(); + if (callArity != aliasesSize) { throw new ValidationException(String.format( "List of column aliases must have same degree as table; " + "the returned table of function '%s' has " + "%d columns, whereas alias list has %d columns", - tableFunctionDefinition.toString(), + functionName, callArity, aliasesSize)); - } else { - fieldNames = aliases.toArray(new String[aliasesSize]); } - TypeInformation[] fieldTypes = FieldInfoUtils.getFieldTypes(resultType); - - return new CalculatedQueryOperation( - tableFunctionDefinition.getTableFunction(), - parameters, - tableFunctionDefinition.getResultType(), - new TableSchema(fieldNames, fieldTypes)); + return TableSchema.builder() + .fields(aliases.toArray(new String[0]), tableSchema.getFieldDataTypes()) + .build(); } @Override - protected CalculatedQueryOperation defaultMethod(ResolvedExpression expression) { + protected CalculatedQueryOperation defaultMethod(ResolvedExpression expression) { throw fail(); } diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/expressions/resolver/ExpressionResolverTest.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/expressions/resolver/ExpressionResolverTest.java index 801ed185b4..4d4a809c3d 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/expressions/resolver/ExpressionResolverTest.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/expressions/resolver/ExpressionResolverTest.java @@ -20,6 +20,9 @@ package org.apache.flink.table.expressions.resolver; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.table.annotation.DataTypeHint; +import org.apache.flink.table.annotation.FunctionHint; +import org.apache.flink.table.annotation.InputGroup; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.TableSchema; @@ -50,6 +53,7 @@ import org.junit.runners.Parameterized; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -147,15 +151,32 @@ public class ExpressionResolverTest { .build() ) .select($("f0").isEqual($("f1"))) - .equalTo(new CallExpression( - FunctionIdentifier.of("equals"), - BuiltInFunctionDefinitions.EQUALS, - Arrays.asList( - new FieldReferenceExpression("f0", DataTypes.INT(), 0, 0), - new FieldReferenceExpression("f1", DataTypes.STRING(), 0, 1) - ), - DataTypes.BOOLEAN() - )), + .equalTo( + new CallExpression( + FunctionIdentifier.of("equals"), + BuiltInFunctionDefinitions.EQUALS, + Arrays.asList( + new FieldReferenceExpression("f0", DataTypes.INT(), 0, 0), + new FieldReferenceExpression("f1", DataTypes.STRING(), 0, 1) + ), + DataTypes.BOOLEAN() + )), + + TestSpec.test("Lookup legacy scalar function call") + .inputSchemas( + TableSchema.builder() + .field("f0", DataTypes.INT()) + .build() + ) + .lookupFunction("func", new ScalarFunctionDefinition("func", new LegacyScalarFunc())) + .select(call("func", 1, $("f0"))) + .equalTo( + new CallExpression( + FunctionIdentifier.of("func"), + new ScalarFunctionDefinition("func", new LegacyScalarFunc()), + Arrays.asList(valueLiteral(1), new FieldReferenceExpression("f0", DataTypes.INT(), 0, 0)), + DataTypes.INT().bridgedTo(Integer.class) + )), TestSpec.test("Lookup system function call") .inputSchemas( @@ -163,14 +184,60 @@ public class ExpressionResolverTest { .field("f0", DataTypes.INT()) .build() ) - .lookupFunction("func", new ScalarFunctionDefinition("func", new ScalarFunc())) + .lookupFunction("func", new ScalarFunc()) .select(call("func", 1, $("f0"))) - .equalTo(new CallExpression( - FunctionIdentifier.of("func"), - new ScalarFunctionDefinition("func", new ScalarFunc()), - Arrays.asList(valueLiteral(1), new FieldReferenceExpression("f0", DataTypes.INT(), 0, 0)), - DataTypes.INT().bridgedTo(Integer.class) - ))); + .equalTo( + new CallExpression( + FunctionIdentifier.of("func"), + new ScalarFunc(), + Arrays.asList(valueLiteral(1), new FieldReferenceExpression("f0", DataTypes.INT(), 0, 0)), + DataTypes.INT().notNull().bridgedTo(int.class) + )), + + TestSpec.test("Lookup catalog function call") + .inputSchemas( + TableSchema.builder() + .field("f0", DataTypes.INT()) + .build() + ) + .lookupFunction(ObjectIdentifier.of("cat", "db", "func"), new ScalarFunc()) + .select(call("cat.db.func", 1, $("f0"))) + .equalTo( + new CallExpression( + FunctionIdentifier.of(ObjectIdentifier.of("cat", "db", "func")), + new ScalarFunc(), + Arrays.asList(valueLiteral(1), new FieldReferenceExpression("f0", DataTypes.INT(), 0, 0)), + DataTypes.INT().notNull().bridgedTo(int.class) + )), + + TestSpec.test("Deeply nested user-defined inline calls") + .inputSchemas( + TableSchema.builder() + .field("f0", DataTypes.INT()) + .build() + ) + .lookupFunction("func", new ScalarFunc()) + .select(call("func", call(new ScalarFunc(), call("func", 1, $("f0"))))) + .equalTo( + new CallExpression( + FunctionIdentifier.of("func"), + new ScalarFunc(), + Collections.singletonList( + new CallExpression( + new ScalarFunc(), + Collections.singletonList(new CallExpression( + FunctionIdentifier.of("func"), + new ScalarFunc(), + Arrays.asList( + valueLiteral(1), + new FieldReferenceExpression("f0", DataTypes.INT(), 0, 0)), + DataTypes.INT().notNull().bridgedTo(int.class) + )), + DataTypes.INT().notNull().bridgedTo(int.class) + )), + DataTypes.INT().notNull().bridgedTo(int.class)) + ) + ); } @Parameterized.Parameter @@ -186,13 +253,36 @@ public class ExpressionResolverTest { } /** - * Test scalar function that uses legacy type inference logic. + * Test scalar function. */ + @FunctionHint( + input = @DataTypeHint(inputGroup = InputGroup.ANY), + isVarArgs = true, + output = @DataTypeHint(value = "INTEGER NOT NULL", bridgedTo = int.class)) public static class ScalarFunc extends ScalarFunction { public int eval(Object... any) { return 0; } + @Override + public int hashCode() { + return 0; + } + + @Override + public boolean equals(Object obj) { + return obj instanceof ScalarFunc; + } + } + + /** + * Legacy scalar function. + */ + public static class LegacyScalarFunc extends ScalarFunction { + public int eval(Object... any) { + return 0; + } + @Override public TypeInformation getResultType(Class[] signature) { return Types.INT; diff --git a/flink-table/flink-table-api-scala/src/main/scala/org/apache/flink/table/api/expressionDsl.scala b/flink-table/flink-table-api-scala/src/main/scala/org/apache/flink/table/api/expressionDsl.scala index cb71dc7d02..d42403109a 100644 --- a/flink-table/flink-table-api-scala/src/main/scala/org/apache/flink/table/api/expressionDsl.scala +++ b/flink-table/flink-table-api-scala/src/main/scala/org/apache/flink/table/api/expressionDsl.scala @@ -504,6 +504,14 @@ trait ImplicitExpressionConversions { */ def call(path: String, params: Expression*): Expression = Expressions.call(path, params: _*) + /** + * A call to an unregistered, inline function. For functions that have been registered before and + * are identified by a name, use [[call(String, Object...)]]. + */ + def call(function: UserDefinedFunction, params: Expression*): Expression = Expressions.call( + function, + params: _*) + // ---------------------------------------------------------------------------------------------- // Implicit expressions in prefix notation // ---------------------------------------------------------------------------------------------- diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/PlannerTypeInferenceUtilImpl.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/PlannerTypeInferenceUtilImpl.java index 0b0b934361..4640377a25 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/PlannerTypeInferenceUtilImpl.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/PlannerTypeInferenceUtilImpl.java @@ -31,6 +31,7 @@ import org.apache.flink.table.planner.validate.ValidationResult; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.inference.TypeInferenceUtil; +import java.util.ArrayList; import java.util.List; import java.util.Optional; import java.util.stream.Collectors; @@ -54,6 +55,10 @@ public final class PlannerTypeInferenceUtilImpl implements PlannerTypeInferenceU public TypeInferenceUtil.Result runTypeInference( UnresolvedCallExpression unresolvedCall, List resolvedArgs) { + // We should not try to resolve the children again with the old type stack + // The arguments might have been resolved with the new stack already. In that case the + // resolution will fail. + unresolvedCall = unresolvedCall.replaceArgs(new ArrayList<>(resolvedArgs)); final PlannerExpression plannerCall = unresolvedCall.accept(CONVERTER); if (plannerCall instanceof InputTypeSpec) { diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/CallExpressionConvertRule.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/CallExpressionConvertRule.java index bf6a5c4c2a..18ee37fbcd 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/CallExpressionConvertRule.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/CallExpressionConvertRule.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.expressions.converter; +import org.apache.flink.table.catalog.DataTypeFactory; import org.apache.flink.table.expressions.CallExpression; import org.apache.flink.table.expressions.Expression; import org.apache.flink.table.planner.calcite.FlinkTypeFactory; @@ -52,5 +53,7 @@ public interface CallExpressionConvertRule { RelBuilder getRelBuilder(); FlinkTypeFactory getTypeFactory(); + + DataTypeFactory getDataTypeFactory(); } } diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/ExpressionConverter.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/ExpressionConverter.java index 48bbafcd24..42013ea214 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/ExpressionConverter.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/ExpressionConverter.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.expressions.converter; import org.apache.flink.table.api.TableException; +import org.apache.flink.table.catalog.DataTypeFactory; import org.apache.flink.table.dataformat.Decimal; import org.apache.flink.table.expressions.CallExpression; import org.apache.flink.table.expressions.Expression; @@ -34,6 +35,7 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory; import org.apache.flink.table.planner.calcite.RexFieldVariable; import org.apache.flink.table.planner.expressions.RexNodeExpression; import org.apache.flink.table.planner.expressions.converter.CallExpressionConvertRule.ConvertContext; +import org.apache.flink.table.planner.utils.ShortcutUtils; import org.apache.flink.table.types.logical.DecimalType; import org.apache.flink.table.types.logical.LocalZonedTimestampType; import org.apache.flink.table.types.logical.LogicalType; @@ -71,7 +73,8 @@ import static org.apache.flink.table.util.TimestampStringUtils.fromLocalDateTime public class ExpressionConverter implements ExpressionVisitor { private static final List FUNCTION_CONVERT_CHAIN = Arrays.asList( - new ScalarFunctionConvertRule(), + new LegacyScalarFunctionConvertRule(), + new FunctionDefinitionConvertRule(), new OverConvertRule(), new DirectConvertRule(), new CustomizedConvertRule() @@ -79,10 +82,14 @@ public class ExpressionConverter implements ExpressionVisitor { private final RelBuilder relBuilder; private final FlinkTypeFactory typeFactory; + private final DataTypeFactory dataTypeFactory; public ExpressionConverter(RelBuilder relBuilder) { this.relBuilder = relBuilder; this.typeFactory = (FlinkTypeFactory) relBuilder.getRexBuilder().getTypeFactory(); + this.dataTypeFactory = ShortcutUtils.unwrapContext(relBuilder.getCluster()) + .getCatalogManager() + .getDataTypeFactory(); } @Override @@ -240,6 +247,11 @@ public class ExpressionConverter implements ExpressionVisitor { public FlinkTypeFactory getTypeFactory() { return typeFactory; } + + @Override + public DataTypeFactory getDataTypeFactory() { + return dataTypeFactory; + } }; } diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/FunctionDefinitionConvertRule.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/FunctionDefinitionConvertRule.java new file mode 100644 index 0000000000..9085d40c57 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/FunctionDefinitionConvertRule.java @@ -0,0 +1,65 @@ +/* + * 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.table.planner.expressions.converter; + +import org.apache.flink.table.expressions.CallExpression; +import org.apache.flink.table.planner.functions.bridging.BridgingSqlFunction; +import org.apache.flink.table.types.inference.TypeInference; +import org.apache.flink.table.types.inference.TypeStrategies; + +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.sql.SqlKind; + +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +/** + * A call expression converter rule that converts calls to user defined functions. + */ +public class FunctionDefinitionConvertRule implements CallExpressionConvertRule { + @Override + public Optional convert( + CallExpression call, + ConvertContext context) { + TypeInference typeInference = call.getFunctionDefinition().getTypeInference(context.getDataTypeFactory()); + + if (typeInference.getOutputTypeStrategy() == TypeStrategies.MISSING) { + return Optional.empty(); + } + + switch (call.getFunctionDefinition().getKind()) { + case SCALAR: + case TABLE: + List args = call.getChildren().stream().map(context::toRexNode).collect(Collectors.toList()); + + final BridgingSqlFunction sqlFunction = BridgingSqlFunction.of( + context.getDataTypeFactory(), + context.getTypeFactory(), + SqlKind.OTHER_FUNCTION, + call.getFunctionIdentifier().orElse(null), + call.getFunctionDefinition(), + typeInference); + + return Optional.of(context.getRelBuilder().call(sqlFunction, args)); + default: + return Optional.empty(); + } + } +} diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/ScalarFunctionConvertRule.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/LegacyScalarFunctionConvertRule.java similarity index 96% rename from flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/ScalarFunctionConvertRule.java rename to flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/LegacyScalarFunctionConvertRule.java index 9041e2f55d..82827a0fcd 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/ScalarFunctionConvertRule.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/LegacyScalarFunctionConvertRule.java @@ -35,7 +35,7 @@ import static org.apache.flink.table.planner.expressions.converter.ExpressionCon /** * {@link CallExpressionConvertRule} to convert {@link ScalarFunctionDefinition}. */ -public class ScalarFunctionConvertRule implements CallExpressionConvertRule { +public class LegacyScalarFunctionConvertRule implements CallExpressionConvertRule { @Override public Optional convert(CallExpression call, ConvertContext context) { diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/bridging/BridgingSqlAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/bridging/BridgingSqlAggFunction.java index 8ecade2edc..1ccd6a7067 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/bridging/BridgingSqlAggFunction.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/bridging/BridgingSqlAggFunction.java @@ -33,7 +33,10 @@ import org.apache.calcite.sql.SqlAggFunction; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.util.Optionality; +import javax.annotation.Nullable; + import java.util.List; +import java.util.Optional; import static org.apache.flink.table.planner.functions.bridging.BridgingUtils.createName; import static org.apache.flink.table.planner.functions.bridging.BridgingUtils.createParamTypes; @@ -55,7 +58,7 @@ public final class BridgingSqlAggFunction extends SqlAggFunction { private final FlinkTypeFactory typeFactory; - private final FunctionIdentifier identifier; + private final @Nullable FunctionIdentifier identifier; private final FunctionDefinition definition; @@ -67,11 +70,11 @@ public final class BridgingSqlAggFunction extends SqlAggFunction { DataTypeFactory dataTypeFactory, FlinkTypeFactory typeFactory, SqlKind kind, - FunctionIdentifier identifier, + @Nullable FunctionIdentifier identifier, FunctionDefinition definition, TypeInference typeInference) { super( - createName(identifier), + createName(identifier, definition), createSqlIdentifier(identifier), kind, createSqlReturnTypeInference(dataTypeFactory, definition, typeInference), @@ -130,8 +133,8 @@ public final class BridgingSqlAggFunction extends SqlAggFunction { return typeFactory; } - public FunctionIdentifier getIdentifier() { - return identifier; + public Optional getIdentifier() { + return Optional.ofNullable(identifier); } public FunctionDefinition getDefinition() { diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/bridging/BridgingSqlFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/bridging/BridgingSqlFunction.java index 8eec1e1c12..45cc0330e8 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/bridging/BridgingSqlFunction.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/bridging/BridgingSqlFunction.java @@ -31,7 +31,10 @@ import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.sql.SqlFunction; import org.apache.calcite.sql.SqlKind; +import javax.annotation.Nullable; + import java.util.List; +import java.util.Optional; import static org.apache.flink.table.planner.functions.bridging.BridgingUtils.createName; import static org.apache.flink.table.planner.functions.bridging.BridgingUtils.createParamTypes; @@ -53,7 +56,7 @@ public final class BridgingSqlFunction extends SqlFunction { private final FlinkTypeFactory typeFactory; - private final FunctionIdentifier identifier; + private final @Nullable FunctionIdentifier identifier; private final FunctionDefinition definition; @@ -63,11 +66,11 @@ public final class BridgingSqlFunction extends SqlFunction { DataTypeFactory dataTypeFactory, FlinkTypeFactory typeFactory, SqlKind kind, - FunctionIdentifier identifier, + @Nullable FunctionIdentifier identifier, FunctionDefinition definition, TypeInference typeInference) { super( - createName(identifier), + createName(identifier, definition), createSqlIdentifier(identifier), kind, createSqlReturnTypeInference(dataTypeFactory, definition, typeInference), @@ -98,7 +101,7 @@ public final class BridgingSqlFunction extends SqlFunction { DataTypeFactory dataTypeFactory, FlinkTypeFactory typeFactory, SqlKind kind, - FunctionIdentifier identifier, + @Nullable FunctionIdentifier identifier, FunctionDefinition definition, TypeInference typeInference) { @@ -123,8 +126,8 @@ public final class BridgingSqlFunction extends SqlFunction { return typeFactory; } - public FunctionIdentifier getIdentifier() { - return identifier; + public Optional getIdentifier() { + return Optional.ofNullable(identifier); } public FunctionDefinition getDefinition() { diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/bridging/BridgingUtils.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/bridging/BridgingUtils.java index 4b0d2898d2..6474c78d39 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/bridging/BridgingUtils.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/bridging/BridgingUtils.java @@ -18,10 +18,16 @@ package org.apache.flink.table.planner.functions.bridging; +import org.apache.flink.table.api.TableException; import org.apache.flink.table.catalog.DataTypeFactory; import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.functions.AggregateFunctionDefinition; import org.apache.flink.table.functions.FunctionDefinition; import org.apache.flink.table.functions.FunctionIdentifier; +import org.apache.flink.table.functions.ScalarFunctionDefinition; +import org.apache.flink.table.functions.TableAggregateFunctionDefinition; +import org.apache.flink.table.functions.TableFunctionDefinition; +import org.apache.flink.table.functions.UserDefinedFunction; import org.apache.flink.table.planner.calcite.FlinkTypeFactory; import org.apache.flink.table.planner.functions.inference.TypeInferenceOperandChecker; import org.apache.flink.table.planner.functions.inference.TypeInferenceOperandInference; @@ -39,14 +45,22 @@ import org.apache.calcite.sql.type.SqlReturnTypeInference; import javax.annotation.Nullable; import java.util.List; +import java.util.Optional; import java.util.stream.Collectors; /** * Utilities for bridging {@link FunctionDefinition} with Calcite's representation of functions. */ final class BridgingUtils { + static String createName(@Nullable FunctionIdentifier identifier, FunctionDefinition definition) { + if (identifier != null) { + return extractName(identifier); + } else { + return createInlineFunctionName(definition); + } + } - static String createName(FunctionIdentifier identifier) { + private static String extractName(FunctionIdentifier identifier) { if (identifier.getSimpleName().isPresent()) { return identifier.getSimpleName().get(); } @@ -55,10 +69,40 @@ final class BridgingUtils { .orElseThrow(IllegalStateException::new); } - static @Nullable SqlIdentifier createSqlIdentifier(FunctionIdentifier identifier) { + private static String createInlineFunctionName(FunctionDefinition functionDefinition) { + final Optional userDefinedFunction = extractUserDefinedFunction(functionDefinition); + + return userDefinedFunction.map(UserDefinedFunction::functionIdentifier) + .orElseThrow(() -> new TableException(String.format( + "Unsupported function definition: %s. Only user defined functions are supported as inline functions.", + functionDefinition))); + } + + private static Optional extractUserDefinedFunction(FunctionDefinition functionDefinition) { + if (functionDefinition instanceof UserDefinedFunction) { + return Optional.of((UserDefinedFunction) functionDefinition); + } else if (functionDefinition instanceof ScalarFunctionDefinition) { + return Optional.ofNullable(((ScalarFunctionDefinition) functionDefinition).getScalarFunction()); + } else if (functionDefinition instanceof AggregateFunctionDefinition) { + return Optional.ofNullable(((AggregateFunctionDefinition) functionDefinition).getAggregateFunction()); + } else if (functionDefinition instanceof TableFunctionDefinition) { + return Optional.ofNullable(((TableFunctionDefinition) functionDefinition).getTableFunction()); + } else if (functionDefinition instanceof TableAggregateFunctionDefinition) { + return Optional.ofNullable( + ((TableAggregateFunctionDefinition) functionDefinition).getTableAggregateFunction() + ); + } + return Optional.empty(); + } + + static @Nullable SqlIdentifier createSqlIdentifier(@Nullable FunctionIdentifier identifier) { + if (identifier == null) { + return null; + } + return identifier.getIdentifier() .map(i -> new SqlIdentifier(i.toList(), SqlParserPos.ZERO)) - .orElse(null); // null indicates a built-in system function + .orElseGet(() -> new SqlIdentifier(identifier.getSimpleName().get(), SqlParserPos.ZERO)); } static SqlReturnTypeInference createSqlReturnTypeInference( @@ -93,11 +137,12 @@ final class BridgingUtils { .orElse(null); } - static SqlFunctionCategory createSqlFunctionCategory(FunctionIdentifier identifier) { - if (identifier.getSimpleName().isPresent()) { - return SqlFunctionCategory.SYSTEM; + static SqlFunctionCategory createSqlFunctionCategory(@Nullable FunctionIdentifier identifier) { + if (identifier == null || identifier.getIdentifier().isPresent()) { + return SqlFunctionCategory.USER_DEFINED_FUNCTION; } - return SqlFunctionCategory.USER_DEFINED_FUNCTION; + + return SqlFunctionCategory.SYSTEM; } private BridgingUtils() { diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java index 39e2eaf2b3..f95b5bb91b 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java @@ -24,6 +24,7 @@ import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.catalog.CatalogManager; import org.apache.flink.table.catalog.ConnectorCatalogTable; +import org.apache.flink.table.catalog.DataTypeFactory; import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.catalog.UnresolvedIdentifier; import org.apache.flink.table.expressions.CallExpression; @@ -34,8 +35,8 @@ import org.apache.flink.table.expressions.ResolvedExpression; import org.apache.flink.table.expressions.ValueLiteralExpression; import org.apache.flink.table.functions.BuiltInFunctionDefinitions; import org.apache.flink.table.functions.FunctionDefinition; -import org.apache.flink.table.functions.FunctionIdentifier; import org.apache.flink.table.functions.TableFunction; +import org.apache.flink.table.functions.TableFunctionDefinition; import org.apache.flink.table.operations.AggregateQueryOperation; import org.apache.flink.table.operations.CalculatedQueryOperation; import org.apache.flink.table.operations.CatalogQueryOperation; @@ -65,6 +66,7 @@ import org.apache.flink.table.planner.expressions.PlannerWindowStart; import org.apache.flink.table.planner.expressions.RexNodeExpression; import org.apache.flink.table.planner.expressions.SqlAggFunctionVisitor; import org.apache.flink.table.planner.expressions.converter.ExpressionConverter; +import org.apache.flink.table.planner.functions.bridging.BridgingSqlFunction; import org.apache.flink.table.planner.functions.utils.TableSqlFunction; import org.apache.flink.table.planner.operations.DataStreamQueryOperation; import org.apache.flink.table.planner.operations.PlannerQueryOperation; @@ -79,6 +81,7 @@ import org.apache.flink.table.planner.plan.schema.TableSourceTable; import org.apache.flink.table.planner.plan.schema.TypedFlinkTableFunction; import org.apache.flink.table.planner.plan.stats.FlinkStatistic; import org.apache.flink.table.planner.sources.TableSourceUtil; +import org.apache.flink.table.planner.utils.ShortcutUtils; import org.apache.flink.table.sources.LookupableTableSource; import org.apache.flink.table.sources.StreamTableSource; import org.apache.flink.table.sources.TableSource; @@ -93,6 +96,7 @@ import org.apache.calcite.rel.logical.LogicalTableScan; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rex.RexNode; import org.apache.calcite.sql.SqlAggFunction; +import org.apache.calcite.sql.SqlKind; import org.apache.calcite.tools.RelBuilder; import org.apache.calcite.tools.RelBuilder.AggCall; import org.apache.calcite.tools.RelBuilder.GroupKey; @@ -270,36 +274,69 @@ public class QueryOperationConverter extends QueryOperationDefaultVisitor RelNode visit(CalculatedQueryOperation calculatedTable) { - DataType resultType = fromLegacyInfoToDataType(calculatedTable.getResultType()); - TableFunction tableFunction = calculatedTable.getTableFunction(); - String[] fieldNames = calculatedTable.getTableSchema().getFieldNames(); - - TypedFlinkTableFunction function = new TypedFlinkTableFunction( - tableFunction, fieldNames, resultType); - + public RelNode visit(CalculatedQueryOperation calculatedTable) { + FunctionDefinition functionDefinition = calculatedTable.getFunctionDefinition(); + List parameters = convertToRexNodes(calculatedTable.getArguments()); FlinkTypeFactory typeFactory = relBuilder.getTypeFactory(); + if (functionDefinition instanceof TableFunctionDefinition) { + return convertLegacyTableFunction( + calculatedTable, + (TableFunctionDefinition) functionDefinition, + parameters, + typeFactory); + } - TableSqlFunction sqlFunction = new TableSqlFunction( - FunctionIdentifier.of(tableFunction.functionIdentifier()), - tableFunction.toString(), - tableFunction, - resultType, + DataTypeFactory dataTypeFactory = ShortcutUtils.unwrapContext(relBuilder.getCluster()) + .getCatalogManager() + .getDataTypeFactory(); + + final BridgingSqlFunction sqlFunction = BridgingSqlFunction.of( + dataTypeFactory, typeFactory, - function, - scala.Option.empty()); + SqlKind.OTHER_FUNCTION, + calculatedTable.getFunctionIdentifier().orElse(null), + calculatedTable.getFunctionDefinition(), + calculatedTable.getFunctionDefinition().getTypeInference(dataTypeFactory)); + + return relBuilder.functionScan( + sqlFunction, + 0, + parameters) + .rename(Arrays.asList(calculatedTable.getTableSchema().getFieldNames())) + .build(); + } - List parameters = convertToRexNodes(calculatedTable.getParameters()); + private RelNode convertLegacyTableFunction( + CalculatedQueryOperation calculatedTable, + TableFunctionDefinition functionDefinition, + List parameters, + FlinkTypeFactory typeFactory) { + String[] fieldNames = calculatedTable.getTableSchema().getFieldNames(); - // TODO use relBuilder.functionScan() once we remove TableSqlFunction + TableFunction tableFunction = functionDefinition.getTableFunction(); + DataType resultType = fromLegacyInfoToDataType(functionDefinition.getResultType()); + TypedFlinkTableFunction function = new TypedFlinkTableFunction( + tableFunction, + fieldNames, + resultType + ); + + final TableSqlFunction sqlFunction = new TableSqlFunction( + calculatedTable.getFunctionIdentifier().orElse(null), + tableFunction.toString(), + tableFunction, + resultType, + typeFactory, + function, + scala.Option.empty()); return LogicalTableFunctionScan.create( - relBuilder.peek().getCluster(), - Collections.emptyList(), - relBuilder.getRexBuilder() - .makeCall(function.getRowType(typeFactory), sqlFunction, parameters), - function.getElementType(null), - function.getRowType(typeFactory), - null); + relBuilder.peek().getCluster(), + Collections.emptyList(), + relBuilder.getRexBuilder() + .makeCall(function.getRowType(typeFactory), sqlFunction, parameters), + function.getElementType(null), + function.getRowType(typeFactory), + null); } @Override diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CorrelateCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CorrelateCodeGenerator.scala index 812a6ad293..b2a0ace8c0 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CorrelateCodeGenerator.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CorrelateCodeGenerator.scala @@ -26,6 +26,7 @@ import org.apache.flink.api.dag.Transformation import org.apache.flink.streaming.api.functions.ProcessFunction import org.apache.flink.table.api.{TableConfig, TableException, ValidationException} import org.apache.flink.table.dataformat.{BaseRow, GenericRow, JoinedRow} +import org.apache.flink.table.functions.FunctionKind import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.codegen.CodeGenUtils._ import org.apache.flink.table.planner.functions.bridging.BridgingSqlFunction @@ -37,6 +38,8 @@ import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo import org.apache.flink.table.runtime.util.StreamRecordCollector import org.apache.flink.table.types.logical.RowType +import org.apache.calcite.sql.SqlKind + import scala.collection.JavaConversions._ object CorrelateCodeGenerator { @@ -65,9 +68,11 @@ object CorrelateCodeGenerator { // according to the SQL standard, every scalar function should also be a table function // but we don't allow that for now - if (!rexCall.getOperator.isInstanceOf[BridgingSqlFunction] && - !rexCall.getOperator.isInstanceOf[TableSqlFunction]) { - throw new ValidationException("Currently, only table functions can emit rows.") + rexCall.getOperator match { + case func: BridgingSqlFunction if func.getDefinition.getKind == FunctionKind.TABLE => // ok + case _: TableSqlFunction => // ok + case _ => + throw new ValidationException("Currently, only table functions can emit rows.") } val swallowInputOnly = if (projectProgram.isDefined) { diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala index fadf85439f..b160835d67 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala @@ -20,7 +20,7 @@ package org.apache.flink.table.planner.expressions import org.apache.flink.api.common.typeinfo.{TypeInformation, Types} import org.apache.flink.table.api.{TableException, ValidationException} -import org.apache.flink.table.expressions.{ApiExpressionVisitor, CallExpression, Expression, FieldReferenceExpression, LocalReferenceExpression, LookupCallExpression, TableReferenceExpression, TableSymbol, TimeIntervalUnit, TimePointUnit, TypeLiteralExpression, UnresolvedCallExpression, UnresolvedReferenceExpression, ValueLiteralExpression} +import org.apache.flink.table.expressions._ import org.apache.flink.table.functions.BuiltInFunctionDefinitions._ import org.apache.flink.table.functions._ import org.apache.flink.table.planner.expressions.{E => PlannerE, UUID => PlannerUUID} @@ -37,16 +37,30 @@ import _root_.scala.collection.JavaConverters._ class PlannerExpressionConverter private extends ApiExpressionVisitor[PlannerExpression] { override def visit(call: CallExpression): PlannerExpression = { - translateCall(call.getFunctionDefinition, call.getChildren.asScala) + val definition = call.getFunctionDefinition + translateCall( + definition, call.getChildren.asScala, + () => + if (definition.getKind == FunctionKind.AGGREGATE || + definition.getKind == FunctionKind.TABLE_AGGREGATE) { + ApiResolvedAggregateCallExpression(call) + } else { + ApiResolvedCallExpression(call) + }) } override def visit(unresolvedCall: UnresolvedCallExpression): PlannerExpression = { - translateCall(unresolvedCall.getFunctionDefinition, unresolvedCall.getChildren.asScala) + val definition = unresolvedCall.getFunctionDefinition + translateCall( + definition, + unresolvedCall.getChildren.asScala, + () => throw new TableException(s"Unsupported function definition: $definition")) } private def translateCall( func: FunctionDefinition, - children: Seq[Expression]) + children: Seq[Expression], + unknownFunctionHandler: () => PlannerExpression) : PlannerExpression = { // special case: requires individual handling of child expressions @@ -709,7 +723,7 @@ class PlannerExpressionConverter private extends ApiExpressionVisitor[PlannerExp StreamRecordTimestamp() case _ => - throw new TableException(s"Unsupported function definition: $fd") + unknownFunctionHandler() } } } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/aggregations.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/aggregations.scala index 08a2ff5d36..ffb4822855 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/aggregations.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/aggregations.scala @@ -19,12 +19,14 @@ package org.apache.flink.table.planner.expressions import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} import org.apache.flink.api.java.typeutils.MultisetTypeInfo +import org.apache.flink.table.expressions.CallExpression import org.apache.flink.table.functions.{AggregateFunction, TableAggregateFunction, UserDefinedAggregateFunction} import org.apache.flink.table.planner.calcite.FlinkTypeSystem import org.apache.flink.table.planner.functions.utils.UserDefinedFunctionUtils._ import org.apache.flink.table.planner.typeutils.TypeInfoCheckUtils import org.apache.flink.table.planner.validate.{ValidationFailure, ValidationResult, ValidationSuccess} import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.{fromLogicalTypeToTypeInfo, fromTypeInfoToLogicalType} +import org.apache.flink.table.types.utils.TypeConversions import org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType abstract sealed class Aggregation extends PlannerExpression { @@ -33,6 +35,23 @@ abstract sealed class Aggregation extends PlannerExpression { } +/** + * Wrapper for call expressions resolved already in the API with the new type inference stack. + * Separate from [[ApiResolvedCallExpression]] because others' expressions validation logic + * check for the [[Aggregation]] trait. + */ +case class ApiResolvedAggregateCallExpression( + resolvedCall: CallExpression) + extends Aggregation { + + private[flink] val children = Nil + + override private[flink] def resultType: TypeInformation[_] = TypeConversions + .fromDataTypeToLegacyInfo( + resolvedCall + .getOutputDataType) +} + case class DistinctAgg(child: PlannerExpression) extends Aggregation { def distinct: PlannerExpression = DistinctAgg(child) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/call.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/call.scala index b1bf10a2ed..f4caf9e6ec 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/call.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/call.scala @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.expressions import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation, Types} +import org.apache.flink.table.expressions.CallExpression import org.apache.flink.table.functions._ import org.apache.flink.table.planner.functions.utils.UserDefinedFunctionUtils._ import org.apache.flink.table.planner.validate.{ValidationFailure, ValidationResult, ValidationSuccess} @@ -25,9 +26,23 @@ import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromLog import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter.fromDataTypeToTypeInfo import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType import org.apache.flink.table.types.logical.LogicalType +import org.apache.flink.table.types.utils.TypeConversions import org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType import org.apache.flink.table.typeutils.TimeIntervalTypeInfo +/** + * Wrapper for call expressions resolved already in the API with the new type inference stack. + */ +case class ApiResolvedCallExpression( + resolvedCall: CallExpression) + extends LeafExpression { + + override private[flink] def resultType: TypeInformation[_] = TypeConversions + .fromDataTypeToLegacyInfo( + resolvedCall + .getOutputDataType) +} + /** * Over call with unresolved alias for over window. * diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/TableSqlFunction.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/TableSqlFunction.scala index df57754d8f..2533b35e8a 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/TableSqlFunction.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/TableSqlFunction.scala @@ -56,7 +56,8 @@ class TableSqlFunction( functionImpl: FlinkTableFunction, operandTypeInfer: Option[SqlOperandTypeChecker] = None) extends SqlUserDefinedTableFunction( - new SqlIdentifier(identifier.toList, SqlParserPos.ZERO), + Option(identifier).map(id => new SqlIdentifier(id.toList, SqlParserPos.ZERO)) + .getOrElse(new SqlIdentifier(udtf.functionIdentifier(), SqlParserPos.ZERO)), ReturnTypes.CURSOR, // type inference has the UNKNOWN operand types. createOperandTypeInference(displayName, udtf, typeFactory), diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/table/FunctionITCase.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/table/FunctionITCase.java new file mode 100644 index 0000000000..3e67bab807 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/table/FunctionITCase.java @@ -0,0 +1,165 @@ +/* + * 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.table.planner.runtime.stream.table; + +import org.apache.flink.table.annotation.DataTypeHint; +import org.apache.flink.table.annotation.FunctionHint; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.functions.ScalarFunction; +import org.apache.flink.table.functions.TableFunction; +import org.apache.flink.table.planner.factories.utils.TestCollectionTableFactory; +import org.apache.flink.table.planner.runtime.utils.StreamingTestBase; +import org.apache.flink.types.Row; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.Arrays; +import java.util.List; + +import static org.apache.flink.table.api.Expressions.$; +import static org.apache.flink.table.api.Expressions.call; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.junit.Assert.assertThat; + +/** + * Tests for user defined functions in the Table API. + */ +public class FunctionITCase extends StreamingTestBase { + + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Test + public void testScalarFunction() throws Exception { + final List sourceData = Arrays.asList( + Row.of(1, 1L, 1L), + Row.of(2, 2L, 1L), + Row.of(3, 3L, 1L) + ); + + final List sinkData = Arrays.asList( + Row.of(1, 2L, 1L), + Row.of(2, 4L, 1L), + Row.of(3, 6L, 1L) + ); + + TestCollectionTableFactory.reset(); + TestCollectionTableFactory.initData(sourceData); + + tEnv().sqlUpdate("CREATE TABLE TestTable(a INT, b BIGINT, c BIGINT) WITH ('connector' = 'COLLECTION')"); + + tEnv().from("TestTable") + .select( + $("a"), + call(new SimpleScalarFunction(), $("a"), $("b")), + call(new SimpleScalarFunction(), $("a"), $("b")) + .plus(1) + .minus(call(new SimpleScalarFunction(), $("a"), $("b"))) + ) + .insertInto("TestTable"); + tEnv().execute("Test Job"); + + assertThat(TestCollectionTableFactory.getResult(), equalTo(sinkData)); + } + + @Test + public void testJoinWithTableFunction() throws Exception { + final List sourceData = Arrays.asList( + Row.of("1,2,3"), + Row.of("2,3,4"), + Row.of("3,4,5"), + Row.of((String) null) + ); + + final List sinkData = Arrays.asList( + Row.of("1,2,3", new String[]{"1", "2", "3"}), + Row.of("2,3,4", new String[]{"2", "3", "4"}), + Row.of("3,4,5", new String[]{"3", "4", "5"}) + ); + + TestCollectionTableFactory.reset(); + TestCollectionTableFactory.initData(sourceData); + + tEnv().sqlUpdate("CREATE TABLE SourceTable(s STRING) WITH ('connector' = 'COLLECTION')"); + tEnv().sqlUpdate("CREATE TABLE SinkTable(s STRING, sa ARRAY) WITH ('connector' = 'COLLECTION')"); + + tEnv().from("SourceTable") + .joinLateral(call(new SimpleTableFunction(), $("s")).as("a", "b")) + .select($("a"), $("b")) + .insertInto("SinkTable"); + tEnv().execute("Test Job"); + + assertThat(TestCollectionTableFactory.getResult(), equalTo(sinkData)); + } + + @Test + public void testLateralJoinWithScalarFunction() throws Exception { + thrown.expect(ValidationException.class); + thrown.expectMessage("Currently, only table functions can emit rows."); + + TestCollectionTableFactory.reset(); + tEnv().sqlUpdate("CREATE TABLE SourceTable(s STRING) WITH ('connector' = 'COLLECTION')"); + tEnv().sqlUpdate("CREATE TABLE SinkTable(s STRING, sa ARRAY) WITH ('connector' = 'COLLECTION')"); + + tEnv().from("SourceTable") + .joinLateral(call(new RowScalarFunction(), $("s")).as("a", "b")) + .select($("a"), $("b")) + .insertInto("SinkTable"); + tEnv().execute("Test Job"); + } + + // -------------------------------------------------------------------------------------------- + // Test functions + // -------------------------------------------------------------------------------------------- + + /** + * Simple scalar function. + */ + public static class SimpleScalarFunction extends ScalarFunction { + public Long eval(Integer i, Long j) { + return i + j; + } + } + + /** + * Scalar function that returns a row. + */ + @FunctionHint(output = @DataTypeHint("ROW>")) + public static class RowScalarFunction extends ScalarFunction { + public Row eval(String s) { + return Row.of(s, s.split(",")); + } + } + + /** + * Table function that returns a row. + */ + @FunctionHint(output = @DataTypeHint("ROW>")) + public static class SimpleTableFunction extends TableFunction { + public void eval(String s) { + if (s == null) { + collect(null); + } else { + collect(Row.of(s, s.split(","))); + } + } + } +} diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/stringexpr/CorrelateStringExpressionTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/stringexpr/CorrelateStringExpressionTest.scala index 45dcd0bb53..7da7ead41c 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/stringexpr/CorrelateStringExpressionTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/stringexpr/CorrelateStringExpressionTest.scala @@ -24,7 +24,7 @@ import org.apache.flink.streaming.api.datastream.{DataStream => JDataStream} import org.apache.flink.streaming.api.scala.DataStream import org.apache.flink.table.api._ import org.apache.flink.table.api.scala._ -import org.apache.flink.table.planner.utils.{HierarchyTableFunction, PojoTableFunc, TableFunc1, TableFunc2, TableTestBase} +import org.apache.flink.table.planner.utils._ import org.apache.flink.types.Row import org.junit.Test @@ -52,17 +52,17 @@ class CorrelateStringExpressionTest extends TableTestBase { javaUtil.addFunction("func1", func1) scalaUtil.addFunction("func1", func1) - var scalaTable = sTab.joinLateral(func1('c) as 's).select('c, 's) + var scalaTable = sTab.joinLateral(call("func1", 'c) as 's).select('c, 's) var javaTable = jTab.joinLateral("func1(c).as(s)").select("c, s") verifyTableEquals(scalaTable, javaTable) // test left outer join - scalaTable = sTab.leftOuterJoinLateral(func1('c) as 's).select('c, 's) + scalaTable = sTab.leftOuterJoinLateral(call("func1", 'c) as 's).select('c, 's) javaTable = jTab.leftOuterJoinLateral("as(func1(c), s)").select("c, s") verifyTableEquals(scalaTable, javaTable) // test overloading - scalaTable = sTab.joinLateral(func1('c, "$") as 's).select('c, 's) + scalaTable = sTab.joinLateral(call("func1", 'c, "$") as 's).select('c, 's) javaTable = jTab.joinLateral("func1(c, '$') as (s)").select("c, s") verifyTableEquals(scalaTable, javaTable) @@ -70,7 +70,7 @@ class CorrelateStringExpressionTest extends TableTestBase { val func2 = new TableFunc2 javaUtil.addFunction("func2", func2) scalaUtil.addFunction("func2", func2) - scalaTable = sTab.joinLateral(func2('c) as ('name, 'len)).select('c, 'name, 'len) + scalaTable = sTab.joinLateral(call("func2", 'c) as ('name, 'len)).select('c, 'name, 'len) javaTable = jTab.joinLateral( "func2(c).as(name, len)").select("c, name, len") verifyTableEquals(scalaTable, javaTable) @@ -80,7 +80,7 @@ class CorrelateStringExpressionTest extends TableTestBase { javaUtil.addFunction("hierarchy", hierarchy) scalaUtil.addFunction("hierarchy", hierarchy) scalaTable = sTab.joinLateral( - hierarchy('c) as ('name, 'adult, 'len)).select('c, 'name, 'len, 'adult) + call("hierarchy", 'c) as ('name, 'adult, 'len)).select('c, 'name, 'len, 'adult) javaTable = jTab.joinLateral("AS(hierarchy(c), name, adult, len)") .select("c, name, len, adult") verifyTableEquals(scalaTable, javaTable) @@ -89,13 +89,13 @@ class CorrelateStringExpressionTest extends TableTestBase { val pojo = new PojoTableFunc javaUtil.addFunction("pojo", pojo) scalaUtil.addFunction("pojo", pojo) - scalaTable = sTab.joinLateral(pojo('c)).select('c, 'name, 'age) + scalaTable = sTab.joinLateral(call("pojo", 'c)).select('c, 'name, 'age) javaTable = jTab.joinLateral("pojo(c)").select("c, name, age") verifyTableEquals(scalaTable, javaTable) // test with filter scalaTable = sTab.joinLateral( - func2('c) as ('name, 'len)).select('c, 'name, 'len).filter('len > 2) + call("func2", 'c) as ('name, 'len)).select('c, 'name, 'len).filter('len > 2) javaTable = jTab.joinLateral("func2(c) as (name, len)") .select("c, name, len").filter("len > 2") verifyTableEquals(scalaTable, javaTable) @@ -119,7 +119,7 @@ class CorrelateStringExpressionTest extends TableTestBase { val func1 = new TableFunc1 scalaUtil.addFunction("func1", func1) javaUtil.addFunction("func1", func1) - var scalaTable = sTab.flatMap(func1('c)).as('s).select('s) + var scalaTable = sTab.flatMap(call("func1", 'c)).as('s).select('s) var javaTable = jTab.flatMap("func1(c)").as("s").select("s") verifyTableEquals(scalaTable, javaTable) @@ -127,7 +127,7 @@ class CorrelateStringExpressionTest extends TableTestBase { val func2 = new TableFunc2 scalaUtil.addFunction("func2", func2) javaUtil.addFunction("func2", func2) - scalaTable = sTab.flatMap(func2('c)).as('name, 'len).select('name, 'len) + scalaTable = sTab.flatMap(call("func2", 'c)).as('name, 'len).select('name, 'len) javaTable = jTab.flatMap("func2(c)").as("name, len").select("name, len") verifyTableEquals(scalaTable, javaTable) @@ -135,7 +135,9 @@ class CorrelateStringExpressionTest extends TableTestBase { val hierarchy = new HierarchyTableFunction scalaUtil.addFunction("hierarchy", hierarchy) javaUtil.addFunction("hierarchy", hierarchy) - scalaTable = sTab.flatMap(hierarchy('c)).as('name, 'adult, 'len).select('name, 'len, 'adult) + scalaTable = sTab.flatMap(call("hierarchy", 'c)) + .as('name, 'adult, 'len) + .select('name, 'len, 'adult) javaTable = jTab.flatMap("hierarchy(c)").as("name, adult, len").select("name, len, adult") verifyTableEquals(scalaTable, javaTable) @@ -143,17 +145,20 @@ class CorrelateStringExpressionTest extends TableTestBase { val pojo = new PojoTableFunc scalaUtil.addFunction("pojo", pojo) javaUtil.addFunction("pojo", pojo) - scalaTable = sTab.flatMap(pojo('c)).select('name, 'age) + scalaTable = sTab.flatMap(call("pojo", 'c)).select('name, 'age) javaTable = jTab.flatMap("pojo(c)").select("name, age") verifyTableEquals(scalaTable, javaTable) // test with filter - scalaTable = sTab.flatMap(func2('c)).as('name, 'len).select('name, 'len).filter('len > 2) + scalaTable = sTab.flatMap(call("func2", 'c)) + .as('name, 'len) + .select('name, 'len) + .filter('len > 2) javaTable = jTab.flatMap("func2(c)").as("name, len").select("name, len").filter("len > 2") verifyTableEquals(scalaTable, javaTable) // test with scalar function - scalaTable = sTab.flatMap(func1('c.substring(2))).as('s).select('s) + scalaTable = sTab.flatMap(call("func1", 'c.substring(2))).as('s).select('s) javaTable = jTab.flatMap("func1(substring(c, 2))").as("s").select("s") verifyTableEquals(scalaTable, javaTable) } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/CalcValidationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/CalcValidationTest.scala index 231fea6d13..dd0a3367c3 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/CalcValidationTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/CalcValidationTest.scala @@ -149,17 +149,4 @@ class CalcValidationTest extends TableTestBase { "MyTable", 'string) .map("func(string) as a") // do not support TableFunction as input } - - @Test - def testInvalidParameterTypes(): Unit = { - expectedException.expect(classOf[ValidationException]) - expectedException.expectMessage("log('long) fails on input type checking: " + - "[expecting Double on 0th input, get Long].\nOperand should be casted to proper type") - - val util = streamTestUtil() - - util.addFunction("func", new TableFunc0) - util.addTableSource[(Int, Long, String)]("MyTable", 'int, 'long, 'string) - .select('int, 'long.log as 'long, 'string) - } } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/CorrelateValidationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/CorrelateValidationTest.scala index 247225b60d..0813db9071 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/CorrelateValidationTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/CorrelateValidationTest.scala @@ -68,10 +68,6 @@ class CorrelateValidationTest extends TableTestBase { //========= throw exception when the called function is a scalar function ==== util.addFunction("func0", Func0) - // Java Table API call - expectExceptionThrown( - t.joinLateral("func0(a)"), - "only accepts a string expression which defines a table function call") // SQL API call // NOTE: it doesn't throw an exception but an AssertionError, maybe a Calcite bug expectExceptionThrown( diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/plan/QueryOperationConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/plan/QueryOperationConverter.java index f2992f8606..2b059f7582 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/plan/QueryOperationConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/plan/QueryOperationConverter.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.calcite.FlinkRelBuilder; import org.apache.flink.table.calcite.FlinkTypeFactory; import org.apache.flink.table.catalog.CatalogReader; @@ -40,6 +41,7 @@ import org.apache.flink.table.expressions.RexPlannerExpression; import org.apache.flink.table.expressions.UnresolvedCallExpression; import org.apache.flink.table.expressions.WindowReference; import org.apache.flink.table.functions.TableFunction; +import org.apache.flink.table.functions.TableFunctionDefinition; import org.apache.flink.table.functions.utils.TableSqlFunction; import org.apache.flink.table.operations.AggregateQueryOperation; import org.apache.flink.table.operations.CalculatedQueryOperation; @@ -231,35 +233,43 @@ public class QueryOperationConverter extends QueryOperationDefaultVisitor RelNode visit(CalculatedQueryOperation calculatedTable) { - String[] fieldNames = calculatedTable.getTableSchema().getFieldNames(); - int[] fieldIndices = IntStream.range(0, fieldNames.length).toArray(); - TypeInformation resultType = calculatedTable.getResultType(); - - FlinkTableFunctionImpl function = new FlinkTableFunctionImpl<>( - resultType, - fieldIndices, - fieldNames); - TableFunction tableFunction = calculatedTable.getTableFunction(); - + public RelNode visit(CalculatedQueryOperation calculatedTable) { FlinkTypeFactory typeFactory = relBuilder.getTypeFactory(); - TableSqlFunction sqlFunction = new TableSqlFunction( - tableFunction.functionIdentifier(), - tableFunction.toString(), - tableFunction, - resultType, - typeFactory, - function); - - List parameters = convertToRexNodes(calculatedTable.getParameters()); - - return LogicalTableFunctionScan.create( - relBuilder.peek().getCluster(), - Collections.emptyList(), - relBuilder.call(sqlFunction, parameters), - function.getElementType(null), - function.getRowType(typeFactory, null), - null); + if (calculatedTable.getFunctionDefinition() instanceof TableFunctionDefinition) { + TableFunctionDefinition functionDefinition = + (TableFunctionDefinition) calculatedTable.getFunctionDefinition(); + + String[] fieldNames = calculatedTable.getTableSchema().getFieldNames(); + int[] fieldIndices = IntStream.range(0, fieldNames.length).toArray(); + + TableFunction tableFunction = functionDefinition.getTableFunction(); + TypeInformation rowType = functionDefinition.getResultType(); + FlinkTableFunctionImpl function = new FlinkTableFunctionImpl<>( + rowType, + fieldIndices, + fieldNames + ); + + final TableSqlFunction sqlFunction = new TableSqlFunction( + tableFunction.functionIdentifier(), + tableFunction.toString(), + tableFunction, + rowType, + typeFactory, + function); + + List parameters = convertToRexNodes(calculatedTable.getArguments()); + return LogicalTableFunctionScan.create( + relBuilder.peek().getCluster(), + Collections.emptyList(), + relBuilder.call(sqlFunction, parameters), + function.getElementType(null), + function.getRowType(typeFactory, null), + null); + } + + throw new ValidationException( + "The new type inference for functions is only supported in the Blink planner."); } @Override @@ -285,7 +295,7 @@ public class QueryOperationConverter extends QueryOperationDefaultVisitor) other); } else if (other instanceof ScalaDataStreamQueryOperation) { - ScalaDataStreamQueryOperation dataStreamQueryOperation = + ScalaDataStreamQueryOperation dataStreamQueryOperation = (ScalaDataStreamQueryOperation) other; return convertToDataStreamScan( dataStreamQueryOperation.getDataStream(), diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/expressions/PlannerExpressionConverter.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/expressions/PlannerExpressionConverter.scala index 7b2c999254..88772a49b8 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/expressions/PlannerExpressionConverter.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/expressions/PlannerExpressionConverter.scala @@ -110,6 +110,10 @@ class PlannerExpressionConverter private extends ApiExpressionVisitor[PlannerExp tafd.getAccumulatorTypeInfo, args) + case _ : UserDefinedFunction => + throw new ValidationException( + "The new type inference for functions is only supported in the Blink planner.") + case fd: FunctionDefinition => fd match { diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/stream/sql/FunctionITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/stream/sql/FunctionITCase.java index 3e5bee6640..7d3aee29dc 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/stream/sql/FunctionITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/stream/sql/FunctionITCase.java @@ -19,6 +19,7 @@ package org.apache.flink.table.runtime.stream.sql; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.Table; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.api.java.StreamTableEnvironment; @@ -30,7 +31,9 @@ import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.test.util.AbstractTestBase; import org.apache.flink.types.Row; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; import java.util.ArrayList; import java.util.Arrays; @@ -49,6 +52,9 @@ public class FunctionITCase extends AbstractTestBase { private static final String TEST_FUNCTION = TestUDF.class.getName(); + @Rule + public ExpectedException thrown = ExpectedException.none(); + @Test public void testCreateCatalogFunctionInDefaultCatalog() { TableEnvironment tableEnv = getTableEnvironment(); @@ -444,6 +450,31 @@ public class FunctionITCase extends AbstractTestBase { tableEnv.sqlUpdate("drop table t2"); } + @Test + public void testDataTypeBasedTypeInferenceNotSupported() throws Exception { + thrown.expect(ValidationException.class); + thrown.expectMessage("The new type inference for functions is only supported in the Blink planner."); + + StreamExecutionEnvironment streamExecEnvironment = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnvironment = StreamTableEnvironment.create(streamExecEnvironment); + + tableEnvironment.createTemporarySystemFunction("func", SimpleScalarFunction.class); + Table table = tableEnvironment + .sqlQuery("SELECT func(1)"); + tableEnvironment.toAppendStream(table, Row.class).print(); + + streamExecEnvironment.execute(); + } + + /** + * Simple scalar function. + */ + public static class SimpleScalarFunction extends ScalarFunction { + public long eval(Integer i) { + return i; + } + } + private TableEnvironment getTableEnvironment() { StreamExecutionEnvironment streamExecEnvironment = StreamExecutionEnvironment.getExecutionEnvironment(); return StreamTableEnvironment.create(streamExecEnvironment); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/stream/table/FunctionITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/stream/table/FunctionITCase.java new file mode 100644 index 0000000000..52832dc53f --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/stream/table/FunctionITCase.java @@ -0,0 +1,98 @@ +/* + * 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.table.runtime.stream.table; + +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.annotation.DataTypeHint; +import org.apache.flink.table.annotation.FunctionHint; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.api.java.StreamTableEnvironment; +import org.apache.flink.table.functions.ScalarFunction; +import org.apache.flink.table.functions.TableFunction; +import org.apache.flink.test.util.AbstractTestBase; +import org.apache.flink.types.Row; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import static org.apache.flink.table.api.Expressions.$; +import static org.apache.flink.table.api.Expressions.call; + +/** + * Tests for user defined functions in the Table API. + */ +public class FunctionITCase extends AbstractTestBase { + + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Test + public void testDataTypeBasedTypeInferenceNotSupported() throws Exception { + thrown.expect(ValidationException.class); + thrown.expectMessage("The new type inference for functions is only supported in the Blink planner."); + + StreamExecutionEnvironment streamExecEnvironment = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnvironment = StreamTableEnvironment.create(streamExecEnvironment); + + Table table = tableEnvironment + .sqlQuery("SELECT * FROM (VALUES (1)) AS TableName(f0)") + .select(call(new SimpleScalarFunction(), $("f0"))); + tableEnvironment.toAppendStream(table, Row.class).print(); + + streamExecEnvironment.execute(); + } + + @Test + public void testDataTypeBasedTypeInferenceNotSupportedInLateralJoin() throws Exception { + thrown.expect(ValidationException.class); + thrown.expectMessage("The new type inference for functions is only supported in the Blink planner."); + + StreamExecutionEnvironment streamExecEnvironment = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tableEnvironment = StreamTableEnvironment.create(streamExecEnvironment); + + Table table = tableEnvironment + .sqlQuery("SELECT * FROM (VALUES ('A,B,C')) AS TableName(f0)") + .joinLateral(call(new SimpleTableFunction(), $("f0")).as("a", "b")) + .select($("a"), $("b")); + tableEnvironment.toAppendStream(table, Row.class).print(); + + streamExecEnvironment.execute(); + } + + /** + * Simple scalar function. + */ + public static class SimpleScalarFunction extends ScalarFunction { + public long eval(Integer i) { + return i; + } + } + + /** + * Simple table function. + */ + @FunctionHint(output = @DataTypeHint("ROW>")) + public static class SimpleTableFunction extends TableFunction { + public void eval(String s) { + // no-op + } + } +} diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/table/validation/CorrelateValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/table/validation/CorrelateValidationTest.scala index 4fa7b7fd24..f21c3fa392 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/table/validation/CorrelateValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/table/validation/CorrelateValidationTest.scala @@ -70,10 +70,6 @@ class CorrelateValidationTest extends TableTestBase { //========= throw exception when the called function is a scalar function ==== util.tableEnv.registerFunction("func0", Func0) - // Java Table API call - expectExceptionThrown( - t.joinLateral("func0(a)"), - "only accepts a string expression which defines a table function call") // SQL API call // NOTE: it doesn't throw an exception but an AssertionError, maybe a Calcite bug expectExceptionThrown( -- Gitee From 1c86b8b9dd05615a3b2600984db738a9bf388259 Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Fri, 6 Mar 2020 15:58:15 +0100 Subject: [PATCH 299/885] [FLINK-16480][AZP] fix log upload condition --- tools/azure-pipelines/jobs-template.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tools/azure-pipelines/jobs-template.yml b/tools/azure-pipelines/jobs-template.yml index 8a2a832117..ffcdd95ecb 100644 --- a/tools/azure-pipelines/jobs-template.yml +++ b/tools/azure-pipelines/jobs-template.yml @@ -180,7 +180,7 @@ jobs: IT_CASE_S3_SECRET_KEY: $(SECRET_S3_SECRET_KEY) # upload debug artifacts - task: PublishPipelineArtifact@1 - condition: and(succeededOrFailed(), not(eq('$(ARTIFACT_DIR)', ''))) + condition: and(succeededOrFailed(), not(eq(variables['ARTIFACT_DIR'], ''))) displayName: Upload Logs inputs: path: $(ARTIFACT_DIR) -- Gitee From b17d01ba8022398b40075174c9fc288c552910f6 Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Mon, 23 Mar 2020 13:46:40 +0100 Subject: [PATCH 300/885] [FLINK-16480][e2e] On failure, add e2e logs into artifact This closes #11501 --- flink-end-to-end-tests/run-nightly-tests.sh | 38 +++++++++---------- .../test-scripts/test-runner-common.sh | 9 ++++- 2 files changed, 26 insertions(+), 21 deletions(-) diff --git a/flink-end-to-end-tests/run-nightly-tests.sh b/flink-end-to-end-tests/run-nightly-tests.sh index 2f69f5df5c..1fb904435a 100755 --- a/flink-end-to-end-tests/run-nightly-tests.sh +++ b/flink-end-to-end-tests/run-nightly-tests.sh @@ -34,6 +34,22 @@ fi source "${END_TO_END_DIR}/test-scripts/test-runner-common.sh" +# On Azure CI, set artifacts dir +if [ ! -z "$TF_BUILD" ] ; then + export ARTIFACTS_DIR="${END_TO_END_DIR}/artifacts" + mkdir -p $ARTIFACTS_DIR || { echo "FAILURE: cannot create log directory '${ARTIFACTS_DIR}'." ; exit 1; } + + # compress and register logs for publication on exit + function compress_logs { + echo "COMPRESSING build artifacts." + COMPRESSED_ARCHIVE=${BUILD_BUILDNUMBER}.tgz + mkdir compressed-archive-dir + tar -zcvf compressed-archive-dir/${COMPRESSED_ARCHIVE} $ARTIFACTS_DIR + echo "##vso[task.setvariable variable=ARTIFACT_DIR]$(pwd)/compressed-archive-dir" + } + on_exit compress_logs +fi + FLINK_DIR="`( cd \"$FLINK_DIR\" && pwd -P)`" # absolutized and normalized echo "flink-end-to-end-test directory: $END_TO_END_DIR" @@ -211,19 +227,10 @@ printf "\n\n==================================================================== printf "Running Java end-to-end tests\n" printf "==============================================================================\n" -HERE="`dirname \"$0\"`" -HERE="`( cd \"${HERE}\" && pwd -P)`" -if [ -z "${HERE}" ] ; then - # error; for some reason, the path is not accessible - # to the script (e.g. permissions re-evaled after suid) - exit 1 # fail -fi -ARTIFACTS_DIR="${HERE}/artifacts" -mkdir -p $ARTIFACTS_DIR || { echo "FAILURE: cannot create log directory '${ARTIFACTS_DIR}'." ; exit 1; } -LOG4J_PROPERTIES=${HERE}/../tools/log4j-travis.properties +LOG4J_PROPERTIES=${END_TO_END_DIR}/../tools/log4j-travis.properties -MVN_LOGGING_OPTIONS="-Dlog.dir=${ARTIFACTS_DIR} -Dlog4j.configurationFile=file://$LOG4J_PROPERTIES -Dorg.slf4j.simpleLogger.log.org.apache.maven.cli.transfer.Slf4jMavenTransferListener=warn" +MVN_LOGGING_OPTIONS="-Dlog.dir=${ARTIFACTS_DIR} -DlogBackupDir=${ARTIFACTS_DIR} -Dlog4j.configurationFile=file://$LOG4J_PROPERTIES -Dorg.slf4j.simpleLogger.log.org.apache.maven.cli.transfer.Slf4jMavenTransferListener=warn" MVN_COMMON_OPTIONS="-nsu -B -Dflink.forkCount=2 -Dflink.forkCountTestPackage=2 -Dmaven.wagon.http.pool=false -Dfast -Pskip-webui-build" e2e_modules=$(find flink-end-to-end-tests -mindepth 2 -maxdepth 5 -name 'pom.xml' -printf '%h\n' | sort -u | tr '\n' ',') e2e_modules="${e2e_modules},$(find flink-walkthroughs -mindepth 2 -maxdepth 2 -name 'pom.xml' -printf '%h\n' | sort -u | tr '\n' ',')" @@ -233,14 +240,5 @@ mvn ${MVN_COMMON_OPTIONS} ${MVN_LOGGING_OPTIONS} ${PROFILE} verify -pl ${e2e_mod EXIT_CODE=$? -# On Azure, publish ARTIFACTS_FILE as a build artifact -if [ ! -z "$TF_BUILD" ] ; then - echo "COMPRESSING build artifacts." - ARTIFACTS_FILE=${BUILD_BUILDNUMBER}.tgz - tar -zcvf ${ARTIFACTS_FILE} $ARTIFACTS_DIR - mkdir artifact-dir - cp ${ARTIFACTS_FILE} artifact-dir/ - echo "##vso[task.setvariable variable=ARTIFACT_DIR]$(pwd)/artifact-dir" -fi exit $EXIT_CODE diff --git a/flink-end-to-end-tests/test-scripts/test-runner-common.sh b/flink-end-to-end-tests/test-scripts/test-runner-common.sh index 7e09e515e8..326043ec41 100644 --- a/flink-end-to-end-tests/test-scripts/test-runner-common.sh +++ b/flink-end-to-end-tests/test-scripts/test-runner-common.sh @@ -94,6 +94,13 @@ function post_test_validation { if [[ ${exit_code} == 0 ]]; then cleanup else + # make logs available if ARTIFACTS_DIR is set + if [[ ${ARTIFACTS_DIR} != "" ]]; then + mkdir ${ARTIFACTS_DIR}/e2e-flink-logs + cp $FLINK_DIR/log/* ${ARTIFACTS_DIR}/e2e-flink-logs/ + echo "Published e2e logs into debug logs artifact:" + ls ${ARTIFACTS_DIR}/e2e-flink-logs/ + fi exit "${exit_code}" fi } @@ -120,4 +127,4 @@ function cleanup { } trap cleanup SIGINT -trap cleanup_proc EXIT +on_exit cleanup_proc -- Gitee From 5b3d4eb30b7ebd16599195a78a3c7c46a4ada56e Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Fri, 20 Mar 2020 11:14:31 +0100 Subject: [PATCH 301/885] [hotfix][tests] Remove dead code --- .../tasks/MultipleInputStreamTaskTest.java | 16 ---------------- 1 file changed, 16 deletions(-) diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTaskTest.java index 88c4f9a66a..383721d6b0 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTaskTest.java @@ -29,7 +29,6 @@ import org.apache.flink.runtime.metrics.NoOpMetricRegistry; import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup; import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; -import org.apache.flink.streaming.api.functions.co.CoMapFunction; import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; import org.apache.flink.streaming.api.operators.AbstractStreamOperatorV2; import org.apache.flink.streaming.api.operators.Input; @@ -415,21 +414,6 @@ public class MultipleInputStreamTaskTest { } } - private static class IdentityMap implements CoMapFunction { - private static final long serialVersionUID = 1L; - - @Override - public String map1(String value) { - return value; - } - - @Override - public String map2(Integer value) { - - return value.toString(); - } - } - private static class TestBoundedMultipleInputOperatorFactory extends AbstractStreamOperatorFactory { @Override public > T createStreamOperator(StreamOperatorParameters parameters) { -- Gitee From 158e4478cb00f3a7dde32b41f366de87554d9743 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Fri, 20 Mar 2020 14:59:01 +0100 Subject: [PATCH 302/885] [FLINK-16317][operators] Provide support for watermarks in MultipleInputStreamOperator --- .../api/operators/AbstractInput.java | 48 +++ .../operators/AbstractStreamOperatorV2.java | 2 +- .../flink/streaming/api/operators/Input.java | 9 + .../io/StreamMultipleInputProcessor.java | 12 +- .../tasks/MultipleInputStreamTask.java | 2 +- .../tasks/MultipleInputStreamTaskTest.java | 279 +++++++++++++++++- ...ipleInputStreamTaskTestHarnessBuilder.java | 11 + .../StreamTaskMailboxTestHarnessBuilder.java | 5 + .../runtime/tasks/TwoInputStreamTaskTest.java | 2 +- .../TestBoundedMultipleInputOperator.java | 17 +- ...rocessOperatorWithWatermarkDelayTest.scala | 2 +- .../runtime/MultipleInputITCase.java | 13 +- 12 files changed, 371 insertions(+), 31 deletions(-) create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractInput.java diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractInput.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractInput.java new file mode 100644 index 0000000000..e9d227c672 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractInput.java @@ -0,0 +1,48 @@ +/* + * 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.operators; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * Base abstract implementation of {@link Input} interface intended to be used when extending + * {@link AbstractStreamOperatorV2}. + */ +@Experimental +public abstract class AbstractInput implements Input { + protected final AbstractStreamOperatorV2 owner; + protected final int inputId; + protected final Output> output; + + public AbstractInput(AbstractStreamOperatorV2 owner, int inputId) { + checkArgument(inputId > 0, "Inputs are index from 1"); + this.owner = owner; + this.inputId = inputId; + this.output = owner.output; + } + + @Override + public void processWatermark(Watermark mark) throws Exception { + owner.reportWatermark(mark, inputId); + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorV2.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorV2.java index a4ad021d24..9f1fa89c31 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorV2.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorV2.java @@ -465,7 +465,7 @@ public abstract class AbstractStreamOperatorV2 implements StreamOperator { * This method is guaranteed to not be called concurrently with other methods of the operator. */ void processElement(StreamRecord element) throws Exception; + + /** + * Processes a {@link Watermark} that arrived on the first input of this two-input operator. + * This method is guaranteed to not be called concurrently with other methods of the operator. + * + * @see org.apache.flink.streaming.api.watermark.Watermark + */ + void processWatermark(Watermark mark) throws Exception; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java index 6754549972..b93e517935 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java @@ -81,13 +81,14 @@ public final class StreamMultipleInputProcessor implements StreamInputProcessor this.inputSelectionHandler = checkNotNull(inputSelectionHandler); List inputs = streamOperator.getInputs(); - int operatorsCount = inputs.size(); + int inputsCount = inputs.size(); - this.inputProcessors = new InputProcessor[operatorsCount]; - this.streamStatuses = new StreamStatus[operatorsCount]; + this.inputProcessors = new InputProcessor[inputsCount]; + this.streamStatuses = new StreamStatus[inputsCount]; this.numRecordsIn = numRecordsIn; - for (int i = 0; i < operatorsCount; i++) { + for (int i = 0; i < inputsCount; i++) { + streamStatuses[i] = StreamStatus.ACTIVE; StreamTaskNetworkOutput dataOutput = new StreamTaskNetworkOutput<>( inputs.get(i), streamStatusMaintainer, @@ -282,7 +283,8 @@ public final class StreamMultipleInputProcessor implements StreamInputProcessor @Override public void emitWatermark(Watermark watermark) throws Exception { - throw new UnsupportedOperationException(); + inputWatermarkGauge.setCurrentWatermark(watermark.getTimestamp()); + input.processWatermark(watermark); } @Override diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTask.java index 9cc361adb0..7562be7e06 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTask.java @@ -63,7 +63,7 @@ public class MultipleInputStreamTask extends StreamTask(); watermarkGauges[i] = new WatermarkGauge(); - headOperator.getMetricGroup().gauge(MetricNames.currentInputWatermarkName(i), watermarkGauges[i]); + headOperator.getMetricGroup().gauge(MetricNames.currentInputWatermarkName(i + 1), watermarkGauges[i]); } MinWatermarkGauge minInputWatermarkGauge = new MinWatermarkGauge(watermarkGauges); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTaskTest.java index 383721d6b0..75b7e5bcd3 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTaskTest.java @@ -21,14 +21,21 @@ package org.apache.flink.streaming.runtime.tasks; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Metric; import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker; import org.apache.flink.runtime.io.network.api.CheckpointBarrier; import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.metrics.MetricNames; import org.apache.flink.runtime.metrics.NoOpMetricRegistry; import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup; +import org.apache.flink.runtime.metrics.groups.TaskIOMetricGroup; import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; +import org.apache.flink.runtime.metrics.util.InterceptingOperatorMetricGroup; +import org.apache.flink.runtime.metrics.util.InterceptingTaskMetricGroup; +import org.apache.flink.streaming.api.operators.AbstractInput; import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; import org.apache.flink.streaming.api.operators.AbstractStreamOperatorV2; import org.apache.flink.streaming.api.operators.Input; @@ -36,17 +43,25 @@ import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator; import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.api.operators.co.CoStreamMap; +import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.io.InputStatus; import org.apache.flink.streaming.runtime.io.StreamMultipleInputProcessor; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.streamstatus.StreamStatus; +import org.apache.flink.streaming.runtime.tasks.OneInputStreamTaskTest.WatermarkMetricOperator; import org.apache.flink.streaming.util.TestBoundedMultipleInputOperator; +import org.apache.flink.streaming.util.TestHarnessUtil; +import org.hamcrest.collection.IsEmptyCollection; +import org.hamcrest.collection.IsMapContaining; import org.junit.Assert; import org.junit.Test; import java.util.ArrayDeque; import java.util.Arrays; import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; @@ -57,7 +72,6 @@ import static org.junit.Assert.assertEquals; * {@link StreamMultipleInputProcessor}. */ public class MultipleInputStreamTaskTest { - /** * This test verifies that open() and close() are correctly called. This test also verifies * that timestamps of emitted elements are correct. {@link CoStreamMap} assigns the input @@ -257,10 +271,17 @@ public class MultipleInputStreamTaskTest { @Override public List getInputs() { - return Arrays.asList(new DuplicatingInput(), new DuplicatingInput(), new DuplicatingInput()); + return Arrays.asList( + new DuplicatingInput(this, 1), + new DuplicatingInput(this, 2), + new DuplicatingInput(this, 3)); } - class DuplicatingInput implements Input { + class DuplicatingInput extends AbstractInput { + public DuplicatingInput(AbstractStreamOperatorV2 owner, int inputId) { + super(owner, inputId); + } + @Override public void processElement(StreamRecord element) throws Exception { output.collect(element); @@ -355,8 +376,244 @@ public class MultipleInputStreamTaskTest { } } - // This must only be used in one test, otherwise the static fields will be changed - // by several tests concurrently + @Test + public void testWatermark() throws Exception { + try (StreamTaskMailboxTestHarness testHarness = + new MultipleInputStreamTaskTestHarnessBuilder<>(MultipleInputStreamTask::new, BasicTypeInfo.STRING_TYPE_INFO) + .addInput(BasicTypeInfo.STRING_TYPE_INFO, 2) + .addInput(BasicTypeInfo.INT_TYPE_INFO, 2) + .addInput(BasicTypeInfo.DOUBLE_TYPE_INFO, 2) + .setupOutputForSingletonOperatorChain(new MapToStringMultipleInputOperatorFactory()) + .build()) { + ArrayDeque expectedOutput = new ArrayDeque<>(); + + long initialTime = 0L; + + testHarness.processElement(new Watermark(initialTime), 0, 0); + testHarness.processElement(new Watermark(initialTime), 0, 1); + testHarness.processElement(new Watermark(initialTime), 1, 0); + testHarness.processElement(new Watermark(initialTime), 1, 1); + + testHarness.processElement(new Watermark(initialTime), 2, 0); + + assertThat(testHarness.getOutput(), IsEmptyCollection.empty()); + + testHarness.processElement(new Watermark(initialTime), 2, 1); + + // now the watermark should have propagated, Map simply forward Watermarks + expectedOutput.add(new Watermark(initialTime)); + assertThat(testHarness.getOutput(), contains(expectedOutput.toArray())); + + // contrary to checkpoint barriers these elements are not blocked by watermarks + testHarness.processElement(new StreamRecord<>("Hello", initialTime), 0, 0); + testHarness.processElement(new StreamRecord<>(42, initialTime), 1, 1); + expectedOutput.add(new StreamRecord<>("Hello", initialTime)); + expectedOutput.add(new StreamRecord<>("42", initialTime)); + + assertThat(testHarness.getOutput(), contains(expectedOutput.toArray())); + + testHarness.processElement(new Watermark(initialTime + 4), 0, 0); + testHarness.processElement(new Watermark(initialTime + 3), 0, 1); + testHarness.processElement(new Watermark(initialTime + 3), 1, 0); + testHarness.processElement(new Watermark(initialTime + 4), 1, 1); + testHarness.processElement(new Watermark(initialTime + 3), 2, 0); + testHarness.processElement(new Watermark(initialTime + 2), 2, 1); + + // check whether we get the minimum of all the watermarks, this must also only occur in + // the output after the two StreamRecords + expectedOutput.add(new Watermark(initialTime + 2)); + assertThat(testHarness.getOutput(), contains(expectedOutput.toArray())); + + // advance watermark from one of the inputs, now we should get a new one since the + // minimum increases + testHarness.processElement(new Watermark(initialTime + 4), 2, 1); + expectedOutput.add(new Watermark(initialTime + 3)); + assertThat(testHarness.getOutput(), contains(expectedOutput.toArray())); + + // advance the other two inputs, now we should get a new one since the + // minimum increases again + testHarness.processElement(new Watermark(initialTime + 4), 0, 1); + testHarness.processElement(new Watermark(initialTime + 4), 1, 0); + testHarness.processElement(new Watermark(initialTime + 4), 2, 0); + expectedOutput.add(new Watermark(initialTime + 4)); + assertThat(testHarness.getOutput(), contains(expectedOutput.toArray())); + + List resultElements = TestHarnessUtil.getRawElementsFromOutput(testHarness.getOutput()); + assertEquals(2, resultElements.size()); + } + } + + /** + * This test verifies that watermarks and stream statuses are correctly forwarded. This also checks whether + * watermarks are forwarded only when we have received watermarks from all inputs. The + * forwarded watermark must be the minimum of the watermarks of all active inputs. + */ + @Test + public void testWatermarkAndStreamStatusForwarding() throws Exception { + try (StreamTaskMailboxTestHarness testHarness = + new MultipleInputStreamTaskTestHarnessBuilder<>(MultipleInputStreamTask::new, BasicTypeInfo.STRING_TYPE_INFO) + .addInput(BasicTypeInfo.STRING_TYPE_INFO, 2) + .addInput(BasicTypeInfo.INT_TYPE_INFO, 2) + .addInput(BasicTypeInfo.DOUBLE_TYPE_INFO, 2) + .setupOutputForSingletonOperatorChain(new MapToStringMultipleInputOperatorFactory()) + .build()) { + ArrayDeque expectedOutput = new ArrayDeque<>(); + + long initialTime = 0L; + + // test whether idle input channels are acknowledged correctly when forwarding watermarks + testHarness.processElement(StreamStatus.IDLE, 0, 1); + testHarness.processElement(StreamStatus.IDLE, 1, 1); + testHarness.processElement(StreamStatus.IDLE, 2, 0); + testHarness.processElement(new Watermark(initialTime + 6), 0, 0); + testHarness.processElement(new Watermark(initialTime + 6), 1, 0); + testHarness.processElement(new Watermark(initialTime + 5), 2, 1); // this watermark should be advanced first + testHarness.processElement(StreamStatus.IDLE, 2, 1); // once this is acknowledged, + + expectedOutput.add(new Watermark(initialTime + 5)); + // We don't expect to see Watermark(6) here because the idle status of one + // input doesn't propagate to the other input. That is, if input 1 is at WM 6 and input + // two was at WM 5 before going to IDLE then the output watermark will not jump to WM 6. + assertThat(testHarness.getOutput(), contains(expectedOutput.toArray())); + + // make all input channels idle and check that the operator's idle status is forwarded + testHarness.processElement(StreamStatus.IDLE, 0, 0); + testHarness.processElement(StreamStatus.IDLE, 1, 0); + expectedOutput.add(StreamStatus.IDLE); + assertThat(testHarness.getOutput(), contains(expectedOutput.toArray())); + + // make some input channels active again and check that the operator's active status is forwarded only once + testHarness.processElement(StreamStatus.ACTIVE, 1, 0); + testHarness.processElement(StreamStatus.ACTIVE, 0, 1); + expectedOutput.add(StreamStatus.ACTIVE); + assertThat(testHarness.getOutput(), contains(expectedOutput.toArray())); + } + } + + @Test + @SuppressWarnings("unchecked") + public void testWatermarkMetrics() throws Exception { + OperatorID headOperatorId = new OperatorID(); + OperatorID chainedOperatorId = new OperatorID(); + + InterceptingOperatorMetricGroup headOperatorMetricGroup = new InterceptingOperatorMetricGroup(); + InterceptingOperatorMetricGroup chainedOperatorMetricGroup = new InterceptingOperatorMetricGroup(); + InterceptingTaskMetricGroup taskMetricGroup = new InterceptingTaskMetricGroup() { + @Override + public OperatorMetricGroup getOrAddOperator(OperatorID id, String name) { + if (id.equals(headOperatorId)) { + return headOperatorMetricGroup; + } else if (id.equals(chainedOperatorId)) { + return chainedOperatorMetricGroup; + } else { + return super.getOrAddOperator(id, name); + } + } + }; + + try (StreamTaskMailboxTestHarness testHarness = + new MultipleInputStreamTaskTestHarnessBuilder<>(MultipleInputStreamTask::new, BasicTypeInfo.STRING_TYPE_INFO) + .addInput(BasicTypeInfo.STRING_TYPE_INFO) + .addInput(BasicTypeInfo.INT_TYPE_INFO) + .addInput(BasicTypeInfo.DOUBLE_TYPE_INFO) + .setupOperatorChain(headOperatorId, new MapToStringMultipleInputOperatorFactory()) + .chain( + chainedOperatorId, + new WatermarkMetricOperator(), + BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig())) + .finish() + .setTaskMetricGroup(taskMetricGroup) + .build()) { + Gauge taskInputWatermarkGauge = (Gauge) taskMetricGroup.get(MetricNames.IO_CURRENT_INPUT_WATERMARK); + Gauge headInput1WatermarkGauge = (Gauge) headOperatorMetricGroup.get(MetricNames.currentInputWatermarkName(1)); + Gauge headInput2WatermarkGauge = (Gauge) headOperatorMetricGroup.get(MetricNames.currentInputWatermarkName(2)); + Gauge headInput3WatermarkGauge = (Gauge) headOperatorMetricGroup.get(MetricNames.currentInputWatermarkName(3)); + Gauge headInputWatermarkGauge = (Gauge) headOperatorMetricGroup.get(MetricNames.IO_CURRENT_INPUT_WATERMARK); + Gauge headOutputWatermarkGauge = (Gauge) headOperatorMetricGroup.get(MetricNames.IO_CURRENT_OUTPUT_WATERMARK); + Gauge chainedInputWatermarkGauge = (Gauge) chainedOperatorMetricGroup.get(MetricNames.IO_CURRENT_INPUT_WATERMARK); + Gauge chainedOutputWatermarkGauge = (Gauge) chainedOperatorMetricGroup.get(MetricNames.IO_CURRENT_OUTPUT_WATERMARK); + + assertEquals(Long.MIN_VALUE, taskInputWatermarkGauge.getValue().longValue()); + assertEquals(Long.MIN_VALUE, headInputWatermarkGauge.getValue().longValue()); + assertEquals(Long.MIN_VALUE, headInput1WatermarkGauge.getValue().longValue()); + assertEquals(Long.MIN_VALUE, headInput2WatermarkGauge.getValue().longValue()); + assertEquals(Long.MIN_VALUE, headInput3WatermarkGauge.getValue().longValue()); + assertEquals(Long.MIN_VALUE, headOutputWatermarkGauge.getValue().longValue()); + assertEquals(Long.MIN_VALUE, chainedInputWatermarkGauge.getValue().longValue()); + assertEquals(Long.MIN_VALUE, chainedOutputWatermarkGauge.getValue().longValue()); + + testHarness.processElement(new Watermark(1L), 0); + assertEquals(Long.MIN_VALUE, taskInputWatermarkGauge.getValue().longValue()); + assertEquals(Long.MIN_VALUE, headInputWatermarkGauge.getValue().longValue()); + assertEquals(1L, headInput1WatermarkGauge.getValue().longValue()); + assertEquals(Long.MIN_VALUE, headInput2WatermarkGauge.getValue().longValue()); + assertEquals(Long.MIN_VALUE, headInput3WatermarkGauge.getValue().longValue()); + assertEquals(Long.MIN_VALUE, headOutputWatermarkGauge.getValue().longValue()); + assertEquals(Long.MIN_VALUE, chainedInputWatermarkGauge.getValue().longValue()); + assertEquals(Long.MIN_VALUE, chainedOutputWatermarkGauge.getValue().longValue()); + + testHarness.processElement(new Watermark(2L), 1); + assertEquals(Long.MIN_VALUE, taskInputWatermarkGauge.getValue().longValue()); + assertEquals(Long.MIN_VALUE, headInputWatermarkGauge.getValue().longValue()); + assertEquals(1L, headInput1WatermarkGauge.getValue().longValue()); + assertEquals(2L, headInput2WatermarkGauge.getValue().longValue()); + assertEquals(Long.MIN_VALUE, headInput3WatermarkGauge.getValue().longValue()); + assertEquals(Long.MIN_VALUE, headOutputWatermarkGauge.getValue().longValue()); + assertEquals(Long.MIN_VALUE, chainedInputWatermarkGauge.getValue().longValue()); + assertEquals(Long.MIN_VALUE, chainedOutputWatermarkGauge.getValue().longValue()); + + testHarness.processElement(new Watermark(2L), 2); + assertEquals(1L, taskInputWatermarkGauge.getValue().longValue()); + assertEquals(1L, headInputWatermarkGauge.getValue().longValue()); + assertEquals(1L, headInput1WatermarkGauge.getValue().longValue()); + assertEquals(2L, headInput2WatermarkGauge.getValue().longValue()); + assertEquals(2L, headInput3WatermarkGauge.getValue().longValue()); + assertEquals(1L, headOutputWatermarkGauge.getValue().longValue()); + assertEquals(1L, chainedInputWatermarkGauge.getValue().longValue()); + assertEquals(2L, chainedOutputWatermarkGauge.getValue().longValue()); + + testHarness.processElement(new Watermark(4L), 0); + testHarness.processElement(new Watermark(3L), 1); + assertEquals(2L, taskInputWatermarkGauge.getValue().longValue()); + assertEquals(2L, headInputWatermarkGauge.getValue().longValue()); + assertEquals(4L, headInput1WatermarkGauge.getValue().longValue()); + assertEquals(3L, headInput2WatermarkGauge.getValue().longValue()); + assertEquals(2L, headInput3WatermarkGauge.getValue().longValue()); + assertEquals(2L, headOutputWatermarkGauge.getValue().longValue()); + assertEquals(2L, chainedInputWatermarkGauge.getValue().longValue()); + assertEquals(4L, chainedOutputWatermarkGauge.getValue().longValue()); + + testHarness.endInput(); + testHarness.waitForTaskCompletion(); + } + } + + /** + * Tests the checkpoint related metrics are registered into {@link TaskIOMetricGroup} + * correctly while generating the {@link TwoInputStreamTask}. + */ + @Test + public void testCheckpointBarrierMetrics() throws Exception { + final Map metrics = new ConcurrentHashMap<>(); + final TaskMetricGroup taskMetricGroup = new StreamTaskTestHarness.TestTaskMetricGroup(metrics); + + try (StreamTaskMailboxTestHarness testHarness = + new MultipleInputStreamTaskTestHarnessBuilder<>(MultipleInputStreamTask::new, BasicTypeInfo.STRING_TYPE_INFO) + .addInput(BasicTypeInfo.STRING_TYPE_INFO, 2) + .addInput(BasicTypeInfo.INT_TYPE_INFO, 2) + .addInput(BasicTypeInfo.DOUBLE_TYPE_INFO, 2) + .setupOutputForSingletonOperatorChain(new MapToStringMultipleInputOperatorFactory()) + .setTaskMetricGroup(taskMetricGroup) + .build()) { + + assertThat(metrics, IsMapContaining.hasKey(MetricNames.CHECKPOINT_ALIGNMENT_TIME)); + assertThat(metrics, IsMapContaining.hasKey(MetricNames.CHECKPOINT_START_DELAY_TIME)); + + testHarness.endInput(); + testHarness.waitForTaskCompletion(); + } + } + private static class MapToStringMultipleInputOperator extends AbstractStreamOperatorV2 implements MultipleInputStreamOperator { private static final long serialVersionUID = 1L; @@ -389,16 +646,20 @@ public class MultipleInputStreamTaskTest { @Override public List getInputs() { return Arrays.asList( - new MapToStringInput(), - new MapToStringInput(), - new MapToStringInput()); + new MapToStringInput(this, 1), + new MapToStringInput(this, 2), + new MapToStringInput(this, 3)); } public boolean wasCloseCalled() { return closeCalled; } - public class MapToStringInput implements Input { + public class MapToStringInput extends AbstractInput { + public MapToStringInput(AbstractStreamOperatorV2 owner, int inputId) { + super(owner, inputId); + } + @Override public void processElement(StreamRecord element) throws Exception { if (!openCalled) { diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTaskTestHarnessBuilder.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTaskTestHarnessBuilder.java index 3dee5edae2..559b934175 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTaskTestHarnessBuilder.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTaskTestHarnessBuilder.java @@ -20,6 +20,7 @@ package org.apache.flink.streaming.runtime.tasks; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.io.network.partition.consumer.StreamTestSingleInputGate; import org.apache.flink.streaming.api.graph.StreamEdge; @@ -29,6 +30,8 @@ import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner; import org.apache.flink.util.function.FunctionWithException; +import javax.annotation.Nullable; + import java.util.ArrayList; import java.util.LinkedList; import java.util.List; @@ -52,6 +55,14 @@ public class MultipleInputStreamTaskTestHarnessBuilder extends StreamTaskMa } public MultipleInputStreamTaskTestHarnessBuilder addInput(TypeInformation inputType, int inputChannels) { + return addInput(inputType, inputChannels, null); + } + + public MultipleInputStreamTaskTestHarnessBuilder addInput( + TypeInformation inputType, + int inputChannels, + @Nullable KeySelector keySelector) { + streamConfig.setStatePartitioner(inputSerializers.size(), keySelector); inputSerializers.add(inputType.createSerializer(executionConfig)); inputChannelsPerGate.add(inputChannels); return this; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskMailboxTestHarnessBuilder.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskMailboxTestHarnessBuilder.java index b8dbd78938..9b1668688f 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskMailboxTestHarnessBuilder.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskMailboxTestHarnessBuilder.java @@ -185,5 +185,10 @@ public abstract class StreamTaskMailboxTestHarnessBuilder { this.taskMetricGroup = taskMetricGroup; return this; } + + public StreamTaskMailboxTestHarnessBuilder setKeyType(TypeInformation keyType) { + streamConfig.setStateKeySerializer(keyType.createSerializer(executionConfig)); + return this; + } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java index 9580156aac..2a868038cc 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java @@ -210,7 +210,7 @@ public class TwoInputStreamTaskTest { testHarness.processElement(new Watermark(initialTime + 6), 0, 0); testHarness.processElement(new Watermark(initialTime + 5), 1, 1); // this watermark should be advanced first testHarness.processElement(StreamStatus.IDLE, 1, 1); // once this is acknowledged, - // watermark (initial + 6) should be forwarded + testHarness.waitForInputProcessing(); expectedOutput.add(new Watermark(initialTime + 5)); // We don't expect to see Watermark(6) here because the idle status of one diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TestBoundedMultipleInputOperator.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TestBoundedMultipleInputOperator.java index e6ebe84a8b..6f18b58299 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TestBoundedMultipleInputOperator.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TestBoundedMultipleInputOperator.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.util; +import org.apache.flink.streaming.api.operators.AbstractInput; import org.apache.flink.streaming.api.operators.AbstractStreamOperatorV2; import org.apache.flink.streaming.api.operators.BoundedMultiInput; import org.apache.flink.streaming.api.operators.Input; @@ -46,9 +47,9 @@ public class TestBoundedMultipleInputOperator extends AbstractStreamOperatorV2 getInputs() { return Arrays.asList( - new TestInput(1), - new TestInput(2), - new TestInput(3) + new TestInput(this, 1), + new TestInput(this, 2), + new TestInput(this, 3) ); } @@ -63,16 +64,14 @@ public class TestBoundedMultipleInputOperator extends AbstractStreamOperatorV2 { - private final int inputIndex; - - public TestInput(int inputIndex) { - this.inputIndex = inputIndex; + class TestInput extends AbstractInput { + public TestInput(AbstractStreamOperatorV2 owner, int inputId) { + super(owner, inputId); } @Override public void processElement(StreamRecord element) throws Exception { - output.collect(element.replace("[" + name + "-" + inputIndex + "]: " + element.getValue())); + output.collect(element.replace("[" + name + "-" + inputId + "]: " + element.getValue())); } } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/operators/KeyedProcessOperatorWithWatermarkDelayTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/operators/KeyedProcessOperatorWithWatermarkDelayTest.scala index d419453726..8ebdeb87a9 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/operators/KeyedProcessOperatorWithWatermarkDelayTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/operators/KeyedProcessOperatorWithWatermarkDelayTest.scala @@ -38,7 +38,7 @@ class KeyedProcessOperatorWithWatermarkDelayTest extends TestLogger { val operator = new KeyedProcessOperatorWithWatermarkDelay[Integer, Integer, String]( new EmptyProcessFunction, 100) val testHarness = new KeyedOneInputStreamOperatorTestHarness[Integer, Integer, String]( - operator, new IdentityKeySelector, BasicTypeInfo.INT_TYPE_INFO) + operator, new IdentityKeySelector[Integer], BasicTypeInfo.INT_TYPE_INFO) testHarness.setup() testHarness.open() diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/MultipleInputITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/MultipleInputITCase.java index 32b64249ac..02181124e3 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/MultipleInputITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/MultipleInputITCase.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.MultipleConnectedStreams; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.operators.AbstractInput; import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; import org.apache.flink.streaming.api.operators.AbstractStreamOperatorV2; import org.apache.flink.streaming.api.operators.Input; @@ -94,15 +95,19 @@ public class MultipleInputITCase extends AbstractTestBase { @Override public List getInputs() { return Arrays.asList( - new SumInput(), - new SumInput(), - new SumInput()); + new SumInput(this, 1), + new SumInput(this, 2), + new SumInput(this, 3)); } /** * Summing input for {@link SumAllInputOperator}. */ - public class SumInput implements Input { + public class SumInput extends AbstractInput { + public SumInput(AbstractStreamOperatorV2 owner, int inputId) { + super(owner, inputId); + } + @Override public void processElement(StreamRecord element) throws Exception { sum += Long.valueOf(element.getValue().toString()); -- Gitee From a443b04d20764257bb45efd53b4dcec5f8be59f1 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Thu, 19 Mar 2020 17:35:43 +0100 Subject: [PATCH 303/885] [FLINK-16317][operators] Refactor AbstractStreamOperatorTest class This deduplicates code a little bit, fixes not closing issue and makes it ready for future extension that comes in next commit. --- .../operators/AbstractStreamOperatorTest.java | 451 ++++++++---------- 1 file changed, 203 insertions(+), 248 deletions(-) diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java index ebd2ed10d9..dc30675340 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java @@ -49,24 +49,39 @@ import static org.hamcrest.Matchers.contains; * with key-group reshuffling. */ public class AbstractStreamOperatorTest { - @Test - public void testStateDoesNotInterfere() throws Exception { - TestOperator testOperator = new TestOperator(); + protected KeyedOneInputStreamOperatorTestHarness, String> createTestHarness() throws Exception { + return createTestHarness(1, 1, 0); + } - KeyedOneInputStreamOperatorTestHarness, String> testHarness = - new KeyedOneInputStreamOperatorTestHarness<>(testOperator, new TestKeySelector(), BasicTypeInfo.INT_TYPE_INFO); + protected KeyedOneInputStreamOperatorTestHarness, String> createTestHarness( + int maxParalelism, + int numSubtasks, + int subtaskIndex) throws Exception { + TestOperator testOperator = new TestOperator(); + return new KeyedOneInputStreamOperatorTestHarness<>( + testOperator, + new TestKeySelector(), + BasicTypeInfo.INT_TYPE_INFO, + maxParalelism, + numSubtasks, + subtaskIndex); + } - testHarness.open(); + @Test + public void testStateDoesNotInterfere() throws Exception { + try (KeyedOneInputStreamOperatorTestHarness, String> testHarness = createTestHarness()) { + testHarness.open(); - testHarness.processElement(new Tuple2<>(0, "SET_STATE:HELLO"), 0); - testHarness.processElement(new Tuple2<>(1, "SET_STATE:CIAO"), 0); + testHarness.processElement(new Tuple2<>(0, "SET_STATE:HELLO"), 0); + testHarness.processElement(new Tuple2<>(1, "SET_STATE:CIAO"), 0); - testHarness.processElement(new Tuple2<>(1, "EMIT_STATE"), 0); - testHarness.processElement(new Tuple2<>(0, "EMIT_STATE"), 0); + testHarness.processElement(new Tuple2<>(1, "EMIT_STATE"), 0); + testHarness.processElement(new Tuple2<>(0, "EMIT_STATE"), 0); - assertThat( - extractResult(testHarness), - contains("ON_ELEMENT:1:CIAO", "ON_ELEMENT:0:HELLO")); + assertThat( + extractResult(testHarness), + contains("ON_ELEMENT:1:CIAO", "ON_ELEMENT:0:HELLO")); + } } /** @@ -75,33 +90,30 @@ public class AbstractStreamOperatorTest { */ @Test public void testEventTimeTimersDontInterfere() throws Exception { - TestOperator testOperator = new TestOperator(); + try (KeyedOneInputStreamOperatorTestHarness, String> testHarness = createTestHarness()) { + testHarness.open(); - KeyedOneInputStreamOperatorTestHarness, String> testHarness = - new KeyedOneInputStreamOperatorTestHarness<>(testOperator, new TestKeySelector(), BasicTypeInfo.INT_TYPE_INFO); + testHarness.processWatermark(0L); - testHarness.open(); + testHarness.processElement(new Tuple2<>(1, "SET_EVENT_TIME_TIMER:20"), 0); - testHarness.processWatermark(0L); + testHarness.processElement(new Tuple2<>(0, "SET_STATE:HELLO"), 0); + testHarness.processElement(new Tuple2<>(1, "SET_STATE:CIAO"), 0); - testHarness.processElement(new Tuple2<>(1, "SET_EVENT_TIME_TIMER:20"), 0); + testHarness.processElement(new Tuple2<>(0, "SET_EVENT_TIME_TIMER:10"), 0); - testHarness.processElement(new Tuple2<>(0, "SET_STATE:HELLO"), 0); - testHarness.processElement(new Tuple2<>(1, "SET_STATE:CIAO"), 0); + testHarness.processWatermark(10L); - testHarness.processElement(new Tuple2<>(0, "SET_EVENT_TIME_TIMER:10"), 0); + assertThat( + extractResult(testHarness), + contains("ON_EVENT_TIME:HELLO")); - testHarness.processWatermark(10L); + testHarness.processWatermark(20L); - assertThat( - extractResult(testHarness), - contains("ON_EVENT_TIME:HELLO")); - - testHarness.processWatermark(20L); - - assertThat( - extractResult(testHarness), - contains("ON_EVENT_TIME:CIAO")); + assertThat( + extractResult(testHarness), + contains("ON_EVENT_TIME:CIAO")); + } } /** @@ -110,33 +122,30 @@ public class AbstractStreamOperatorTest { */ @Test public void testProcessingTimeTimersDontInterfere() throws Exception { - TestOperator testOperator = new TestOperator(); - - KeyedOneInputStreamOperatorTestHarness, String> testHarness = - new KeyedOneInputStreamOperatorTestHarness<>(testOperator, new TestKeySelector(), BasicTypeInfo.INT_TYPE_INFO); + try (KeyedOneInputStreamOperatorTestHarness, String> testHarness = createTestHarness()) { + testHarness.open(); - testHarness.open(); + testHarness.setProcessingTime(0L); - testHarness.setProcessingTime(0L); + testHarness.processElement(new Tuple2<>(1, "SET_PROC_TIME_TIMER:20"), 0); - testHarness.processElement(new Tuple2<>(1, "SET_PROC_TIME_TIMER:20"), 0); + testHarness.processElement(new Tuple2<>(0, "SET_STATE:HELLO"), 0); + testHarness.processElement(new Tuple2<>(1, "SET_STATE:CIAO"), 0); - testHarness.processElement(new Tuple2<>(0, "SET_STATE:HELLO"), 0); - testHarness.processElement(new Tuple2<>(1, "SET_STATE:CIAO"), 0); + testHarness.processElement(new Tuple2<>(0, "SET_PROC_TIME_TIMER:10"), 0); - testHarness.processElement(new Tuple2<>(0, "SET_PROC_TIME_TIMER:10"), 0); + testHarness.setProcessingTime(10L); - testHarness.setProcessingTime(10L); + assertThat( + extractResult(testHarness), + contains("ON_PROC_TIME:HELLO")); - assertThat( - extractResult(testHarness), - contains("ON_PROC_TIME:HELLO")); + testHarness.setProcessingTime(20L); - testHarness.setProcessingTime(20L); - - assertThat( - extractResult(testHarness), - contains("ON_PROC_TIME:CIAO")); + assertThat( + extractResult(testHarness), + contains("ON_PROC_TIME:CIAO")); + } } /** @@ -144,49 +153,41 @@ public class AbstractStreamOperatorTest { */ @Test public void testEnsureProcessingTimeTimerRegisteredOnRestore() throws Exception { - TestOperator testOperator = new TestOperator(); - - KeyedOneInputStreamOperatorTestHarness, String> testHarness = - new KeyedOneInputStreamOperatorTestHarness<>(testOperator, new TestKeySelector(), BasicTypeInfo.INT_TYPE_INFO); - - testHarness.open(); + OperatorSubtaskState snapshot; + try (KeyedOneInputStreamOperatorTestHarness, String> testHarness = createTestHarness()) { + testHarness.open(); - testHarness.setProcessingTime(0L); + testHarness.setProcessingTime(0L); - testHarness.processElement(new Tuple2<>(1, "SET_PROC_TIME_TIMER:20"), 0); + testHarness.processElement(new Tuple2<>(1, "SET_PROC_TIME_TIMER:20"), 0); - testHarness.processElement(new Tuple2<>(0, "SET_STATE:HELLO"), 0); - testHarness.processElement(new Tuple2<>(1, "SET_STATE:CIAO"), 0); + testHarness.processElement(new Tuple2<>(0, "SET_STATE:HELLO"), 0); + testHarness.processElement(new Tuple2<>(1, "SET_STATE:CIAO"), 0); - testHarness.processElement(new Tuple2<>(0, "SET_PROC_TIME_TIMER:10"), 0); + testHarness.processElement(new Tuple2<>(0, "SET_PROC_TIME_TIMER:10"), 0); - OperatorSubtaskState snapshot = testHarness.snapshot(0, 0); - - TestOperator testOperator1 = new TestOperator(); - - KeyedOneInputStreamOperatorTestHarness, String> testHarness1 = - new KeyedOneInputStreamOperatorTestHarness<>( - testOperator1, - new TestKeySelector(), - BasicTypeInfo.INT_TYPE_INFO); + snapshot = testHarness.snapshot(0, 0); + } - testHarness1.setProcessingTime(0L); + try (KeyedOneInputStreamOperatorTestHarness, String> testHarness1 = createTestHarness()) { + testHarness1.setProcessingTime(0L); - testHarness1.setup(); - testHarness1.initializeState(snapshot); - testHarness1.open(); + testHarness1.setup(); + testHarness1.initializeState(snapshot); + testHarness1.open(); - testHarness1.setProcessingTime(10L); + testHarness1.setProcessingTime(10L); - assertThat( - extractResult(testHarness1), - contains("ON_PROC_TIME:HELLO")); + assertThat( + extractResult(testHarness1), + contains("ON_PROC_TIME:HELLO")); - testHarness1.setProcessingTime(20L); + testHarness1.setProcessingTime(20L); - assertThat( - extractResult(testHarness1), - contains("ON_PROC_TIME:CIAO")); + assertThat( + extractResult(testHarness1), + contains("ON_PROC_TIME:CIAO")); + } } @@ -195,32 +196,29 @@ public class AbstractStreamOperatorTest { */ @Test public void testProcessingTimeAndEventTimeDontInterfere() throws Exception { - TestOperator testOperator = new TestOperator(); - - KeyedOneInputStreamOperatorTestHarness, String> testHarness = - new KeyedOneInputStreamOperatorTestHarness<>(testOperator, new TestKeySelector(), BasicTypeInfo.INT_TYPE_INFO); + try (KeyedOneInputStreamOperatorTestHarness, String> testHarness = createTestHarness()) { + testHarness.open(); - testHarness.open(); + testHarness.setProcessingTime(0L); + testHarness.processWatermark(0L); - testHarness.setProcessingTime(0L); - testHarness.processWatermark(0L); + testHarness.processElement(new Tuple2<>(0, "SET_PROC_TIME_TIMER:10"), 0); + testHarness.processElement(new Tuple2<>(0, "SET_EVENT_TIME_TIMER:20"), 0); - testHarness.processElement(new Tuple2<>(0, "SET_PROC_TIME_TIMER:10"), 0); - testHarness.processElement(new Tuple2<>(0, "SET_EVENT_TIME_TIMER:20"), 0); + testHarness.processElement(new Tuple2<>(0, "SET_STATE:HELLO"), 0); - testHarness.processElement(new Tuple2<>(0, "SET_STATE:HELLO"), 0); + testHarness.processWatermark(20L); - testHarness.processWatermark(20L); + assertThat( + extractResult(testHarness), + contains("ON_EVENT_TIME:HELLO")); - assertThat( - extractResult(testHarness), - contains("ON_EVENT_TIME:HELLO")); + testHarness.setProcessingTime(10L); - testHarness.setProcessingTime(10L); - - assertThat( - extractResult(testHarness), - contains("ON_PROC_TIME:HELLO")); + assertThat( + extractResult(testHarness), + contains("ON_PROC_TIME:HELLO")); + } } /** @@ -242,116 +240,94 @@ public class AbstractStreamOperatorTest { int key1 = getKeyInKeyGroupRange(subKeyGroupRange1, maxParallelism); int key2 = getKeyInKeyGroupRange(subKeyGroupRange2, maxParallelism); - TestOperator testOperator = new TestOperator(); - - KeyedOneInputStreamOperatorTestHarness, String> testHarness = - new KeyedOneInputStreamOperatorTestHarness<>( - testOperator, - new TestKeySelector(), - BasicTypeInfo.INT_TYPE_INFO, - maxParallelism, - 1, /* num subtasks */ - 0 /* subtask index */); + OperatorSubtaskState snapshot; - testHarness.open(); + try (KeyedOneInputStreamOperatorTestHarness, String> testHarness = + createTestHarness(maxParallelism, 1, 0)) { + testHarness.open(); - testHarness.processWatermark(0L); - testHarness.setProcessingTime(0L); + testHarness.processWatermark(0L); + testHarness.setProcessingTime(0L); - testHarness.processElement(new Tuple2<>(key1, "SET_EVENT_TIME_TIMER:10"), 0); - testHarness.processElement(new Tuple2<>(key2, "SET_EVENT_TIME_TIMER:20"), 0); + testHarness.processElement(new Tuple2<>(key1, "SET_EVENT_TIME_TIMER:10"), 0); + testHarness.processElement(new Tuple2<>(key2, "SET_EVENT_TIME_TIMER:20"), 0); - testHarness.processElement(new Tuple2<>(key1, "SET_PROC_TIME_TIMER:10"), 0); - testHarness.processElement(new Tuple2<>(key2, "SET_PROC_TIME_TIMER:20"), 0); + testHarness.processElement(new Tuple2<>(key1, "SET_PROC_TIME_TIMER:10"), 0); + testHarness.processElement(new Tuple2<>(key2, "SET_PROC_TIME_TIMER:20"), 0); - testHarness.processElement(new Tuple2<>(key1, "SET_STATE:HELLO"), 0); - testHarness.processElement(new Tuple2<>(key2, "SET_STATE:CIAO"), 0); + testHarness.processElement(new Tuple2<>(key1, "SET_STATE:HELLO"), 0); + testHarness.processElement(new Tuple2<>(key2, "SET_STATE:CIAO"), 0); - assertTrue(extractResult(testHarness).isEmpty()); + assertTrue(extractResult(testHarness).isEmpty()); - OperatorSubtaskState snapshot = testHarness.snapshot(0, 0); + snapshot = testHarness.snapshot(0, 0); + } // now, restore in two operators, first operator 1 OperatorSubtaskState initState1 = AbstractStreamOperatorTestHarness.repartitionOperatorState( snapshot, maxParallelism, 1, 2, 0); - TestOperator testOperator1 = new TestOperator(); - - KeyedOneInputStreamOperatorTestHarness, String> testHarness1 = - new KeyedOneInputStreamOperatorTestHarness<>( - testOperator1, - new TestKeySelector(), - BasicTypeInfo.INT_TYPE_INFO, - maxParallelism, - 2, /* num subtasks */ - 0 /* subtask index */); - - testHarness1.setup(); - testHarness1.initializeState(initState1); - testHarness1.open(); + try (KeyedOneInputStreamOperatorTestHarness, String> testHarness1 = + createTestHarness(maxParallelism, 2, 0)) { + testHarness1.setup(); + testHarness1.initializeState(initState1); + testHarness1.open(); - testHarness1.processWatermark(10L); + testHarness1.processWatermark(10L); - assertThat(extractResult(testHarness1), contains("ON_EVENT_TIME:HELLO")); + assertThat(extractResult(testHarness1), contains("ON_EVENT_TIME:HELLO")); - assertTrue(extractResult(testHarness1).isEmpty()); + assertTrue(extractResult(testHarness1).isEmpty()); - // this should not trigger anything, the trigger for WM=20 should sit in the - // other operator subtask - testHarness1.processWatermark(20L); + // this should not trigger anything, the trigger for WM=20 should sit in the + // other operator subtask + testHarness1.processWatermark(20L); - assertTrue(extractResult(testHarness1).isEmpty()); + assertTrue(extractResult(testHarness1).isEmpty()); - testHarness1.setProcessingTime(10L); + testHarness1.setProcessingTime(10L); - assertThat(extractResult(testHarness1), contains("ON_PROC_TIME:HELLO")); + assertThat(extractResult(testHarness1), contains("ON_PROC_TIME:HELLO")); - assertTrue(extractResult(testHarness1).isEmpty()); + assertTrue(extractResult(testHarness1).isEmpty()); - // this should not trigger anything, the trigger for TIME=20 should sit in the - // other operator subtask - testHarness1.setProcessingTime(20L); + // this should not trigger anything, the trigger for TIME=20 should sit in the + // other operator subtask + testHarness1.setProcessingTime(20L); - assertTrue(extractResult(testHarness1).isEmpty()); + assertTrue(extractResult(testHarness1).isEmpty()); + } // now, for the second operator OperatorSubtaskState initState2 = AbstractStreamOperatorTestHarness.repartitionOperatorState( snapshot, maxParallelism, 1, 2, 1); - TestOperator testOperator2 = new TestOperator(); - - KeyedOneInputStreamOperatorTestHarness, String> testHarness2 = - new KeyedOneInputStreamOperatorTestHarness<>( - testOperator2, - new TestKeySelector(), - BasicTypeInfo.INT_TYPE_INFO, - maxParallelism, - 2, /* num subtasks */ - 1 /* subtask index */); - - testHarness2.setup(); - testHarness2.initializeState(initState2); - testHarness2.open(); + try (KeyedOneInputStreamOperatorTestHarness, String> testHarness2 = + createTestHarness(maxParallelism, 2, 1)) { + testHarness2.setup(); + testHarness2.initializeState(initState2); + testHarness2.open(); - testHarness2.processWatermark(10L); + testHarness2.processWatermark(10L); - // nothing should happen because this timer is in the other subtask - assertTrue(extractResult(testHarness2).isEmpty()); + // nothing should happen because this timer is in the other subtask + assertTrue(extractResult(testHarness2).isEmpty()); - testHarness2.processWatermark(20L); + testHarness2.processWatermark(20L); - assertThat(extractResult(testHarness2), contains("ON_EVENT_TIME:CIAO")); + assertThat(extractResult(testHarness2), contains("ON_EVENT_TIME:CIAO")); - testHarness2.setProcessingTime(10L); + testHarness2.setProcessingTime(10L); - // nothing should happen because this timer is in the other subtask - assertTrue(extractResult(testHarness2).isEmpty()); + // nothing should happen because this timer is in the other subtask + assertTrue(extractResult(testHarness2).isEmpty()); - testHarness2.setProcessingTime(20L); + testHarness2.setProcessingTime(20L); - assertThat(extractResult(testHarness2), contains("ON_PROC_TIME:CIAO")); + assertThat(extractResult(testHarness2), contains("ON_PROC_TIME:CIAO")); - assertTrue(extractResult(testHarness2).isEmpty()); + assertTrue(extractResult(testHarness2).isEmpty()); + } } @Test @@ -369,93 +345,69 @@ public class AbstractStreamOperatorTest { int key1 = getKeyInKeyGroupRange(subKeyGroupRange1, maxParallelism); int key2 = getKeyInKeyGroupRange(subKeyGroupRange2, maxParallelism); - TestOperator testOperator1 = new TestOperator(); - - KeyedOneInputStreamOperatorTestHarness, String> testHarness1 = - new KeyedOneInputStreamOperatorTestHarness<>( - testOperator1, - new TestKeySelector(), - BasicTypeInfo.INT_TYPE_INFO, - maxParallelism, - 2, /* num subtasks */ - 0 /* subtask index */); - - testHarness1.setup(); - testHarness1.open(); - - testHarness1.processWatermark(0L); - testHarness1.setProcessingTime(0L); + OperatorSubtaskState snapshot1, snapshot2; + // register some state with both instances and scale down to parallelism 1 + try (KeyedOneInputStreamOperatorTestHarness, String> testHarness1 = + createTestHarness(maxParallelism, 2, 0)) { - TestOperator testOperator2 = new TestOperator(); + testHarness1.setup(); + testHarness1.open(); - KeyedOneInputStreamOperatorTestHarness, String> testHarness2 = - new KeyedOneInputStreamOperatorTestHarness<>( - testOperator2, - new TestKeySelector(), - BasicTypeInfo.INT_TYPE_INFO, - maxParallelism, - 2, /* num subtasks */ - 1 /* subtask index */); + testHarness1.processWatermark(0L); + testHarness1.setProcessingTime(0L); - testHarness2.setup(); - testHarness2.open(); + testHarness1.processElement(new Tuple2<>(key1, "SET_EVENT_TIME_TIMER:30"), 0); + testHarness1.processElement(new Tuple2<>(key1, "SET_PROC_TIME_TIMER:30"), 0); + testHarness1.processElement(new Tuple2<>(key1, "SET_STATE:HELLO"), 0); - testHarness2.processWatermark(0L); - testHarness2.setProcessingTime(0L); + snapshot1 = testHarness1.snapshot(0, 0); + } - // register some state with both instances and scale down to parallelism 1 + try (KeyedOneInputStreamOperatorTestHarness, String> testHarness2 = + createTestHarness(maxParallelism, 2, 1)) { + testHarness2.setup(); + testHarness2.open(); - testHarness1.processElement(new Tuple2<>(key1, "SET_EVENT_TIME_TIMER:30"), 0); - testHarness1.processElement(new Tuple2<>(key1, "SET_PROC_TIME_TIMER:30"), 0); - testHarness1.processElement(new Tuple2<>(key1, "SET_STATE:HELLO"), 0); + testHarness2.processWatermark(0L); + testHarness2.setProcessingTime(0L); - testHarness2.processElement(new Tuple2<>(key2, "SET_EVENT_TIME_TIMER:40"), 0); - testHarness2.processElement(new Tuple2<>(key2, "SET_PROC_TIME_TIMER:40"), 0); - testHarness2.processElement(new Tuple2<>(key2, "SET_STATE:CIAO"), 0); + testHarness2.processElement(new Tuple2<>(key2, "SET_EVENT_TIME_TIMER:40"), 0); + testHarness2.processElement(new Tuple2<>(key2, "SET_PROC_TIME_TIMER:40"), 0); + testHarness2.processElement(new Tuple2<>(key2, "SET_STATE:CIAO"), 0); + snapshot2 = testHarness2.snapshot(0, 0); + } // take a snapshot from each one of the "parallel" instances of the operator // and combine them into one so that we can scale down OperatorSubtaskState repackagedState = - AbstractStreamOperatorTestHarness.repackageState( - testHarness1.snapshot(0, 0), - testHarness2.snapshot(0, 0) - ); + AbstractStreamOperatorTestHarness.repackageState(snapshot1, snapshot2); OperatorSubtaskState initSubTaskState = AbstractStreamOperatorTestHarness.repartitionOperatorState(repackagedState, maxParallelism, 2, 1, 0); - // now, for the third operator that scales down from parallelism of 2 to 1 - TestOperator testOperator3 = new TestOperator(); - - KeyedOneInputStreamOperatorTestHarness, String> testHarness3 = - new KeyedOneInputStreamOperatorTestHarness<>( - testOperator3, - new TestKeySelector(), - BasicTypeInfo.INT_TYPE_INFO, - maxParallelism, - 1, /* num subtasks */ - 0 /* subtask index */); - - testHarness3.setup(); - testHarness3.initializeState(initSubTaskState); - testHarness3.open(); - - testHarness3.processWatermark(30L); - assertThat(extractResult(testHarness3), contains("ON_EVENT_TIME:HELLO")); - assertTrue(extractResult(testHarness3).isEmpty()); - - testHarness3.processWatermark(40L); - assertThat(extractResult(testHarness3), contains("ON_EVENT_TIME:CIAO")); - assertTrue(extractResult(testHarness3).isEmpty()); - - testHarness3.setProcessingTime(30L); - assertThat(extractResult(testHarness3), contains("ON_PROC_TIME:HELLO")); - assertTrue(extractResult(testHarness3).isEmpty()); - - testHarness3.setProcessingTime(40L); - assertThat(extractResult(testHarness3), contains("ON_PROC_TIME:CIAO")); - assertTrue(extractResult(testHarness3).isEmpty()); + try (KeyedOneInputStreamOperatorTestHarness, String> testHarness3 = + createTestHarness(maxParallelism, 1, 0)) { + testHarness3.setup(); + testHarness3.initializeState(initSubTaskState); + testHarness3.open(); + + testHarness3.processWatermark(30L); + assertThat(extractResult(testHarness3), contains("ON_EVENT_TIME:HELLO")); + assertTrue(extractResult(testHarness3).isEmpty()); + + testHarness3.processWatermark(40L); + assertThat(extractResult(testHarness3), contains("ON_EVENT_TIME:CIAO")); + assertTrue(extractResult(testHarness3).isEmpty()); + + testHarness3.setProcessingTime(30L); + assertThat(extractResult(testHarness3), contains("ON_PROC_TIME:HELLO")); + assertTrue(extractResult(testHarness3).isEmpty()); + + testHarness3.setProcessingTime(40L); + assertThat(extractResult(testHarness3), contains("ON_PROC_TIME:CIAO")); + assertTrue(extractResult(testHarness3).isEmpty()); + } } /** @@ -474,7 +426,10 @@ public class AbstractStreamOperatorTest { return result; } - private static class TestKeySelector implements KeySelector, Integer> { + /** + * {@link KeySelector} for tests. + */ + protected static class TestKeySelector implements KeySelector, Integer> { private static final long serialVersionUID = 1L; @Override -- Gitee From 381a01a39b2026f1972fa9f0afd2ca3233896c64 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Thu, 19 Mar 2020 17:35:35 +0100 Subject: [PATCH 304/885] [FLINK-16317][operators] Implement AbstractStreamOperatorV2Test --- .../AbstractStreamOperatorV2Test.java | 136 ++++++++++++++++++ ...eyedOneInputStreamOperatorTestHarness.java | 24 +++- .../OneInputStreamOperatorTestHarness.java | 72 +++++++--- 3 files changed, 213 insertions(+), 19 deletions(-) create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorV2Test.java diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorV2Test.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorV2Test.java new file mode 100644 index 0000000000..504d969297 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorV2Test.java @@ -0,0 +1,136 @@ +/* + * 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.operators; + +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.state.VoidNamespace; +import org.apache.flink.runtime.state.VoidNamespaceSerializer; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness; + +import java.util.Collections; +import java.util.List; + +/** + * Tests for the facilities provided by {@link AbstractStreamOperatorV2}. + */ +public class AbstractStreamOperatorV2Test extends AbstractStreamOperatorTest { + @Override + protected KeyedOneInputStreamOperatorTestHarness, String> createTestHarness( + int maxParalelism, + int numSubtasks, + int subtaskIndex) throws Exception { + return new KeyedOneInputStreamOperatorTestHarness<>( + new TestOperatorFactory(), + new TestKeySelector(), + BasicTypeInfo.INT_TYPE_INFO, + maxParalelism, + numSubtasks, + subtaskIndex); + } + + private static class TestOperatorFactory extends AbstractStreamOperatorFactory { + @Override + public > T createStreamOperator(StreamOperatorParameters parameters) { + return (T) new SingleInputTestOperator(parameters); + } + + @Override + public Class getStreamOperatorClass(ClassLoader classLoader) { + return SingleInputTestOperator.class; + } + } + + /** + * Testing operator that can respond to commands by either setting/deleting state, emitting + * state or setting timers. + */ + private static class SingleInputTestOperator + extends AbstractStreamOperatorV2 + implements MultipleInputStreamOperator, + Triggerable { + + + private static final long serialVersionUID = 1L; + + private transient InternalTimerService timerService; + + private final ValueStateDescriptor stateDescriptor = + new ValueStateDescriptor<>("state", StringSerializer.INSTANCE); + + public SingleInputTestOperator(StreamOperatorParameters parameters) { + super(parameters, 1); + } + + @Override + public void open() throws Exception { + super.open(); + + this.timerService = getInternalTimerService( + "test-timers", + VoidNamespaceSerializer.INSTANCE, + this); + } + + @Override + public List getInputs() { + return Collections.singletonList(new AbstractInput, String>(this, 1) { + @Override + public void processElement(StreamRecord> element) throws Exception { + String[] command = element.getValue().f1.split(":"); + switch (command[0]) { + case "SET_STATE": + getPartitionedState(stateDescriptor).update(command[1]); + break; + case "DELETE_STATE": + getPartitionedState(stateDescriptor).clear(); + break; + case "SET_EVENT_TIME_TIMER": + timerService.registerEventTimeTimer(VoidNamespace.INSTANCE, Long.parseLong(command[1])); + break; + case "SET_PROC_TIME_TIMER": + timerService.registerProcessingTimeTimer(VoidNamespace.INSTANCE, Long.parseLong(command[1])); + break; + case "EMIT_STATE": + String stateValue = getPartitionedState(stateDescriptor).value(); + output.collect(new StreamRecord<>("ON_ELEMENT:" + element.getValue().f0 + ":" + stateValue)); + break; + default: + throw new IllegalArgumentException(); + } + } + }); + } + + @Override + public void onEventTime(InternalTimer timer) throws Exception { + String stateValue = getPartitionedState(stateDescriptor).value(); + output.collect(new StreamRecord<>("ON_EVENT_TIME:" + stateValue)); + } + + @Override + public void onProcessingTime(InternalTimer timer) throws Exception { + String stateValue = getPartitionedState(stateDescriptor).value(); + output.collect(new StreamRecord<>("ON_PROC_TIME:" + stateValue)); + } + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java index 31b10f58ed..44be4e120c 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java @@ -27,6 +27,8 @@ import org.apache.flink.runtime.state.KeyedStateBackend; import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; /** * Extension of {@link OneInputStreamOperatorTestHarness} that allows the operator to get @@ -42,13 +44,30 @@ public class KeyedOneInputStreamOperatorTestHarness int maxParallelism, int numSubtasks, int subtaskIndex) throws Exception { - super(operator, maxParallelism, numSubtasks, subtaskIndex); + this(SimpleOperatorFactory.of(operator), keySelector, keyType, maxParallelism, numSubtasks, subtaskIndex); + } + + public KeyedOneInputStreamOperatorTestHarness( + StreamOperatorFactory operatorFactory, + final KeySelector keySelector, + TypeInformation keyType, + int maxParallelism, + int numSubtasks, + int subtaskIndex) throws Exception { + super(operatorFactory, maxParallelism, numSubtasks, subtaskIndex); ClosureCleaner.clean(keySelector, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, false); config.setStatePartitioner(0, keySelector); config.setStateKeySerializer(keyType.createSerializer(executionConfig)); } + public KeyedOneInputStreamOperatorTestHarness( + StreamOperatorFactory operatorFactory, + final KeySelector keySelector, + TypeInformation keyType) throws Exception { + this(operatorFactory, keySelector, keyType, 1, 1, 0); + } + public KeyedOneInputStreamOperatorTestHarness( OneInputStreamOperator operator, final KeySelector keySelector, @@ -58,10 +77,9 @@ public class KeyedOneInputStreamOperatorTestHarness public KeyedOneInputStreamOperatorTestHarness( final OneInputStreamOperator operator, - final KeySelector keySelector, + final KeySelector keySelector, final TypeInformation keyType, final MockEnvironment environment) throws Exception { - super(operator, environment); ClosureCleaner.clean(keySelector, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, false); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java index 2fe5094126..f124c5546e 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java @@ -22,13 +22,21 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.operators.testutils.MockEnvironment; import org.apache.flink.streaming.api.operators.BoundedOneInput; +import org.apache.flink.streaming.api.operators.Input; +import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.Preconditions; +import java.util.ArrayList; import java.util.Collection; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkState; /** * A test harness for testing a {@link OneInputStreamOperator}. @@ -40,6 +48,9 @@ import java.util.Collection; public class OneInputStreamOperatorTestHarness extends AbstractStreamOperatorTestHarness { + /** Empty if the {@link #operator} is not {@link MultipleInputStreamOperator}. */ + private final List inputs = new ArrayList<>(); + private long currentWatermark; public OneInputStreamOperatorTestHarness( @@ -51,14 +62,13 @@ public class OneInputStreamOperatorTestHarness } public OneInputStreamOperatorTestHarness( - OneInputStreamOperator operator, - int maxParallelism, - int parallelism, - int subtaskIndex, - TypeSerializer typeSerializerIn, - OperatorID operatorID) throws Exception { - this(operator, maxParallelism, parallelism, subtaskIndex, operatorID); - + OneInputStreamOperator operator, + int maxParallelism, + int parallelism, + int subtaskIndex, + TypeSerializer typeSerializerIn, + OperatorID operatorID) throws Exception { + this(SimpleOperatorFactory.of(operator), maxParallelism, parallelism, subtaskIndex, operatorID); config.setTypeSerializersIn(Preconditions.checkNotNull(typeSerializerIn)); } @@ -84,16 +94,24 @@ public class OneInputStreamOperatorTestHarness int maxParallelism, int parallelism, int subtaskIndex) throws Exception { - this(operator, maxParallelism, parallelism, subtaskIndex, new OperatorID()); + this(SimpleOperatorFactory.of(operator), maxParallelism, parallelism, subtaskIndex); } public OneInputStreamOperatorTestHarness( - OneInputStreamOperator operator, + StreamOperatorFactory operatorFactory, + int maxParallelism, + int parallelism, + int subtaskIndex) throws Exception { + this(operatorFactory, maxParallelism, parallelism, subtaskIndex, new OperatorID()); + } + + public OneInputStreamOperatorTestHarness( + StreamOperatorFactory operatorFactory, int maxParallelism, int parallelism, int subtaskIndex, OperatorID operatorID) throws Exception { - super(operator, maxParallelism, parallelism, subtaskIndex, operatorID); + super(operatorFactory, maxParallelism, parallelism, subtaskIndex, operatorID); } public OneInputStreamOperatorTestHarness( @@ -142,6 +160,15 @@ public class OneInputStreamOperatorTestHarness super(factory, maxParallelism, parallelism, subtaskIndex, operatorID); } + @Override + public void setup(TypeSerializer outputSerializer) { + super.setup(outputSerializer); + if (operator instanceof MultipleInputStreamOperator) { + checkState(inputs.isEmpty()); + inputs.addAll(((MultipleInputStreamOperator) operator).getInputs()); + } + } + public OneInputStreamOperator getOneInputOperator() { return (OneInputStreamOperator) this.operator; } @@ -151,14 +178,20 @@ public class OneInputStreamOperatorTestHarness } public void processElement(StreamRecord element) throws Exception { - operator.setKeyContextElement1(element); - getOneInputOperator().processElement(element); + if (inputs.isEmpty()) { + operator.setKeyContextElement1(element); + getOneInputOperator().processElement(element); + } + else { + checkState(inputs.size() == 1); + Input input = inputs.get(0); + input.processElement(element); + } } public void processElements(Collection> elements) throws Exception { for (StreamRecord element: elements) { - operator.setKeyContextElement1(element); - getOneInputOperator().processElement(element); + processElement(element); } } @@ -168,7 +201,14 @@ public class OneInputStreamOperatorTestHarness public void processWatermark(Watermark mark) throws Exception { currentWatermark = mark.getTimestamp(); - getOneInputOperator().processWatermark(mark); + if (inputs.isEmpty()) { + getOneInputOperator().processWatermark(mark); + } + else { + checkState(inputs.size() == 1); + Input input = inputs.get(0); + input.processWatermark(mark); + } } public void endInput() throws Exception { -- Gitee From 05883b7fc38786e735e6476ae8e78e2f1dc022b7 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Fri, 20 Mar 2020 14:18:40 +0100 Subject: [PATCH 305/885] [FLINK-16317][operators] Provide support for key selector and latency marker in MultipleInputStreamOperator --- .../datastream/MultipleConnectedStreams.java | 4 +- .../streaming/api/graph/StreamGraph.java | 11 ++- .../api/graph/StreamGraphGenerator.java | 15 ++- .../flink/streaming/api/graph/StreamNode.java | 20 ++-- .../api/graph/StreamingJobGraphGenerator.java | 5 +- .../api/operators/AbstractInput.java | 23 +++++ .../flink/streaming/api/operators/Input.java | 11 +++ .../AbstractMultipleInputTransformation.java | 85 ++++++++++++++++ .../KeyedMultipleInputTransformation.java | 60 ++++++++++++ .../MultipleInputTransformation.java | 54 +--------- .../io/StreamMultipleInputProcessor.java | 5 +- .../flink/streaming/api/DataStreamTest.java | 10 +- .../tasks/MultipleInputStreamTaskTest.java | 28 ++++++ .../OneInputStreamOperatorTestHarness.java | 1 + .../runtime/MultipleInputITCase.java | 98 +++++++++++++++++++ 15 files changed, 346 insertions(+), 84 deletions(-) create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/AbstractMultipleInputTransformation.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/KeyedMultipleInputTransformation.java diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/MultipleConnectedStreams.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/MultipleConnectedStreams.java index f8341df692..987d7dbb59 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/MultipleConnectedStreams.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/MultipleConnectedStreams.java @@ -20,7 +20,7 @@ package org.apache.flink.streaming.api.datastream; import org.apache.flink.annotation.Experimental; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator; -import org.apache.flink.streaming.api.transformations.MultipleInputTransformation; +import org.apache.flink.streaming.api.transformations.AbstractMultipleInputTransformation; import static java.util.Objects.requireNonNull; @@ -41,7 +41,7 @@ public class MultipleConnectedStreams { return environment; } - public SingleOutputStreamOperator transform(MultipleInputTransformation transform) { + public SingleOutputStreamOperator transform(AbstractMultipleInputTransformation transform) { return new SingleOutputStreamOperator<>(environment, transform); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java index 0ecf90ddad..39393fc51e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java @@ -592,14 +592,19 @@ public class StreamGraph implements Pipeline { public void setOneInputStateKey(Integer vertexID, KeySelector keySelector, TypeSerializer keySerializer) { StreamNode node = getStreamNode(vertexID); - node.setStatePartitioner1(keySelector); + node.setStatePartitioners(keySelector); node.setStateKeySerializer(keySerializer); } public void setTwoInputStateKey(Integer vertexID, KeySelector keySelector1, KeySelector keySelector2, TypeSerializer keySerializer) { StreamNode node = getStreamNode(vertexID); - node.setStatePartitioner1(keySelector1); - node.setStatePartitioner2(keySelector2); + node.setStatePartitioners(keySelector1, keySelector2); + node.setStateKeySerializer(keySerializer); + } + + public void setMultipleInputStateKey(Integer vertexID, List> keySelectors, TypeSerializer keySerializer) { + StreamNode node = getStreamNode(vertexID); + node.setStatePartitioners(keySelectors.stream().toArray(KeySelector[]::new)); node.setStateKeySerializer(keySerializer); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java index e1bfe6ec41..dab9bb7be4 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java @@ -33,9 +33,10 @@ import org.apache.flink.streaming.api.environment.CheckpointConfig; import org.apache.flink.streaming.api.operators.InputFormatOperatorFactory; import org.apache.flink.streaming.api.operators.OutputFormatOperatorFactory; import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.transformations.AbstractMultipleInputTransformation; import org.apache.flink.streaming.api.transformations.CoFeedbackTransformation; import org.apache.flink.streaming.api.transformations.FeedbackTransformation; -import org.apache.flink.streaming.api.transformations.MultipleInputTransformation; +import org.apache.flink.streaming.api.transformations.KeyedMultipleInputTransformation; import org.apache.flink.streaming.api.transformations.OneInputTransformation; import org.apache.flink.streaming.api.transformations.PartitionTransformation; import org.apache.flink.streaming.api.transformations.PhysicalTransformation; @@ -255,8 +256,8 @@ public class StreamGraphGenerator { transformedIds = transformOneInputTransform((OneInputTransformation) transform); } else if (transform instanceof TwoInputTransformation) { transformedIds = transformTwoInputTransform((TwoInputTransformation) transform); - } else if (transform instanceof MultipleInputTransformation) { - transformedIds = transformMultipleInputTransform((MultipleInputTransformation) transform); + } else if (transform instanceof AbstractMultipleInputTransformation) { + transformedIds = transformMultipleInputTransform((AbstractMultipleInputTransformation) transform); } else if (transform instanceof SourceTransformation) { transformedIds = transformSource((SourceTransformation) transform); } else if (transform instanceof SinkTransformation) { @@ -741,7 +742,7 @@ public class StreamGraphGenerator { return Collections.singleton(transform.getId()); } - private Collection transformMultipleInputTransform(MultipleInputTransformation transform) { + private Collection transformMultipleInputTransform(AbstractMultipleInputTransformation transform) { checkArgument(!transform.getInputs().isEmpty(), "Empty inputs for MultipleInputTransformation. Did you forget to add inputs?"); MultipleInputSelectionHandler.checkSupportedInputCount(transform.getInputs().size()); @@ -776,6 +777,12 @@ public class StreamGraphGenerator { streamGraph.setParallelism(transform.getId(), parallelism); streamGraph.setMaxParallelism(transform.getId(), transform.getMaxParallelism()); + if (transform instanceof KeyedMultipleInputTransformation) { + KeyedMultipleInputTransformation keyedTransform = (KeyedMultipleInputTransformation) transform; + TypeSerializer keySerializer = keyedTransform.getStateKeyType().createSerializer(executionConfig); + streamGraph.setMultipleInputStateKey(transform.getId(), keyedTransform.getStateKeySelectors(), keySerializer); + } + for (int i = 0; i < allInputIds.size(); i++) { Collection inputIds = allInputIds.get(i); for (Integer inputId: inputIds) { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java index 2f655ef521..8df1206d75 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java @@ -61,8 +61,7 @@ public class StreamNode implements Serializable { private final String operatorName; private @Nullable String slotSharingGroup; private @Nullable String coLocationGroup; - private KeySelector statePartitioner1; - private KeySelector statePartitioner2; + private KeySelector[] statePartitioners = new KeySelector[0]; private TypeSerializer stateKeySerializer; private transient StreamOperatorFactory operatorFactory; @@ -304,20 +303,13 @@ public class StreamNode implements Serializable { return operatorName + "-" + id; } - public KeySelector getStatePartitioner1() { - return statePartitioner1; + public KeySelector[] getStatePartitioners() { + return statePartitioners; } - public KeySelector getStatePartitioner2() { - return statePartitioner2; - } - - public void setStatePartitioner1(KeySelector statePartitioner) { - this.statePartitioner1 = statePartitioner; - } - - public void setStatePartitioner2(KeySelector statePartitioner) { - this.statePartitioner2 = statePartitioner; + public void setStatePartitioners(KeySelector ...statePartitioners) { + checkArgument(statePartitioners.length > 0); + this.statePartitioners = statePartitioners; } public TypeSerializer getStateKeySerializer() { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java index 277e450ca1..5a3486322d 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java @@ -524,8 +524,9 @@ public class StreamingJobGraphGenerator { // so we use that one if checkpointing is not enabled config.setCheckpointMode(CheckpointingMode.AT_LEAST_ONCE); } - config.setStatePartitioner(0, vertex.getStatePartitioner1()); - config.setStatePartitioner(1, vertex.getStatePartitioner2()); + for (int i = 0; i < vertex.getStatePartitioners().length; i++) { + config.setStatePartitioner(i, vertex.getStatePartitioners()[i]); + } config.setStateKeySerializer(vertex.getStateKeySerializer()); Class vertexClass = vertex.getJobVertexClass(); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractInput.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractInput.java index e9d227c672..af8858ba7d 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractInput.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractInput.java @@ -19,9 +19,13 @@ package org.apache.flink.streaming.api.operators; import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import javax.annotation.Nullable; + import static org.apache.flink.util.Preconditions.checkArgument; /** @@ -30,6 +34,14 @@ import static org.apache.flink.util.Preconditions.checkArgument; */ @Experimental public abstract class AbstractInput implements Input { + /** + * {@code KeySelector} for extracting a key from an element being processed. This is used to + * scope keyed state to a key. This is null if the operator is not a keyed operator. + * + *

    This is for elements from the first input. + */ + @Nullable + protected final KeySelector stateKeySelector; protected final AbstractStreamOperatorV2 owner; protected final int inputId; protected final Output> output; @@ -38,6 +50,7 @@ public abstract class AbstractInput implements Input { checkArgument(inputId > 0, "Inputs are index from 1"); this.owner = owner; this.inputId = inputId; + this.stateKeySelector = owner.config.getStatePartitioner(inputId - 1, owner.getUserCodeClassloader()); this.output = owner.output; } @@ -45,4 +58,14 @@ public abstract class AbstractInput implements Input { public void processWatermark(Watermark mark) throws Exception { owner.reportWatermark(mark, inputId); } + + @Override + public void processLatencyMarker(LatencyMarker latencyMarker) throws Exception { + owner.reportOrForwardLatencyMarker(latencyMarker); + } + + @Override + public void setKeyContextElement(StreamRecord record) throws Exception { + owner.internalSetKeyContextElement(record, stateKeySelector); + } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/Input.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/Input.java index 9342db2e01..d4ca03caad 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/Input.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/Input.java @@ -20,6 +20,7 @@ package org.apache.flink.streaming.api.operators; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; /** @@ -40,4 +41,14 @@ public interface Input { * @see org.apache.flink.streaming.api.watermark.Watermark */ void processWatermark(Watermark mark) throws Exception; + + /** + * Processes a {@link LatencyMarker} that arrived on the first input of this two-input operator. + * This method is guaranteed to not be called concurrently with other methods of the operator. + * + * @see org.apache.flink.streaming.runtime.streamrecord.LatencyMarker + */ + void processLatencyMarker(LatencyMarker latencyMarker) throws Exception; + + void setKeyContextElement(StreamRecord record) throws Exception; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/AbstractMultipleInputTransformation.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/AbstractMultipleInputTransformation.java new file mode 100644 index 0000000000..bfcd20136e --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/AbstractMultipleInputTransformation.java @@ -0,0 +1,85 @@ +/* + * 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.transformations; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.streaming.api.operators.ChainingStrategy; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.stream.Collectors; + +/** + * Base class for transformations representing the application of a + * {@link org.apache.flink.streaming.api.operators.MultipleInputStreamOperator} + * to input {@code Transformations}. The result is again only one stream. + * + * @param The type of the elements that result from this {@code MultipleInputTransformation} + */ +@Internal +public abstract class AbstractMultipleInputTransformation extends PhysicalTransformation { + + protected final List> inputs = new ArrayList<>(); + protected final StreamOperatorFactory operatorFactory; + + public AbstractMultipleInputTransformation( + String name, + StreamOperatorFactory operatorFactory, + TypeInformation outputType, + int parallelism) { + super(name, outputType, parallelism); + this.operatorFactory = operatorFactory; + } + + public List> getInputs() { + return inputs; + } + + /** + * Returns the {@code TypeInformation} for the elements from the inputs. + */ + public List> getInputTypes() { + return inputs.stream() + .map(Transformation::getOutputType) + .collect(Collectors.toList()); + } + + /** + * Returns the {@code StreamOperatorFactory} of this Transformation. + */ + public StreamOperatorFactory getOperatorFactory() { + return operatorFactory; + } + + @Override + public Collection> getTransitivePredecessors() { + return inputs.stream() + .flatMap(input -> input.getTransitivePredecessors().stream()) + .collect(Collectors.toList()); + } + + @Override + public final void setChainingStrategy(ChainingStrategy strategy) { + operatorFactory.setChainingStrategy(strategy); + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/KeyedMultipleInputTransformation.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/KeyedMultipleInputTransformation.java new file mode 100644 index 0000000000..9d15c97282 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/KeyedMultipleInputTransformation.java @@ -0,0 +1,60 @@ +/* + * 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.transformations; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; + +import java.util.ArrayList; +import java.util.List; + +/** + * {@link AbstractMultipleInputTransformation} implementation for keyed streams. + */ +@Internal +public class KeyedMultipleInputTransformation extends AbstractMultipleInputTransformation { + private final List> stateKeySelectors = new ArrayList<>(); + protected final TypeInformation stateKeyType; + + public KeyedMultipleInputTransformation( + String name, + StreamOperatorFactory operatorFactory, + TypeInformation outputType, + int parallelism, + TypeInformation stateKeyType) { + super(name, operatorFactory, outputType, parallelism); + this.stateKeyType = stateKeyType; + } + + public void addInput(Transformation input, KeySelector keySelector) { + inputs.add(input); + getStateKeySelectors().add(keySelector); + } + + public TypeInformation getStateKeyType() { + return stateKeyType; + } + + public List> getStateKeySelectors() { + return stateKeySelectors; + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/MultipleInputTransformation.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/MultipleInputTransformation.java index f8e52cc825..9acc8938a4 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/MultipleInputTransformation.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/MultipleInputTransformation.java @@ -21,70 +21,22 @@ package org.apache.flink.streaming.api.transformations; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.dag.Transformation; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.StreamOperatorFactory; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.stream.Collectors; - /** - * This Transformation represents the application of a - * {@link org.apache.flink.streaming.api.operators.MultipleInputStreamOperator} - * to input {@code Transformations}. The result is again only one stream. - * - * @param The type of the elements that result from this {@code MultipleInputTransformation} + * {@link AbstractMultipleInputTransformation} implementation for non-keyed streams. */ @Internal -public class MultipleInputTransformation extends PhysicalTransformation { - - private final List> inputs = new ArrayList<>(); - - private final StreamOperatorFactory operatorFactory; - +public class MultipleInputTransformation extends AbstractMultipleInputTransformation { public MultipleInputTransformation( String name, StreamOperatorFactory operatorFactory, TypeInformation outputType, int parallelism) { - super(name, outputType, parallelism); - this.operatorFactory = operatorFactory; - } - - public List> getInputs() { - return inputs; - } - - /** - * Returns the {@code TypeInformation} for the elements from the inputs. - */ - public List> getInputTypes() { - return inputs.stream() - .map(Transformation::getOutputType) - .collect(Collectors.toList()); - } - - /** - * Returns the {@code StreamOperatorFactory} of this Transformation. - */ - public StreamOperatorFactory getOperatorFactory() { - return operatorFactory; + super(name, operatorFactory, outputType, parallelism); } public void addInput(Transformation input) { inputs.add(input); } - - @Override - public Collection> getTransitivePredecessors() { - return inputs.stream() - .flatMap(input -> input.getTransitivePredecessors().stream()) - .collect(Collectors.toList()); - } - - @Override - public final void setChainingStrategy(ChainingStrategy strategy) { - operatorFactory.setChainingStrategy(strategy); - } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java index b93e517935..e02fcb2aad 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java @@ -274,8 +274,7 @@ public final class StreamMultipleInputProcessor implements StreamInputProcessor @Override public void emitRecord(StreamRecord record) throws Exception { - //TODO: support keyed operators - //input.setKeyContextElement(record); + input.setKeyContextElement(record); input.processElement(record); numRecordsIn.inc(); inputSelectionHandler.nextSelection(); @@ -306,7 +305,7 @@ public final class StreamMultipleInputProcessor implements StreamInputProcessor @Override public void emitLatencyMarker(LatencyMarker latencyMarker) throws Exception { - throw new UnsupportedOperationException(); + input.processLatencyMarker(latencyMarker); } } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java index fa95f7fe0c..41db863310 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java @@ -1007,7 +1007,7 @@ public class DataStreamTest extends TestLogger { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataStreamSink sink = env.generateSequence(1, 100).print(); - assertTrue(getStreamGraph(env).getStreamNode(sink.getTransformation().getId()).getStatePartitioner1() == null); + assertEquals(0, getStreamGraph(env).getStreamNode(sink.getTransformation().getId()).getStatePartitioners().length); assertTrue(getStreamGraph(env).getStreamNode(sink.getTransformation().getId()).getInEdges().get(0).getPartitioner() instanceof ForwardPartitioner); KeySelector key1 = new KeySelector() { @@ -1022,10 +1022,10 @@ public class DataStreamTest extends TestLogger { DataStreamSink sink2 = env.generateSequence(1, 100).keyBy(key1).print(); - assertNotNull(getStreamGraph(env).getStreamNode(sink2.getTransformation().getId()).getStatePartitioner1()); + assertEquals(1, getStreamGraph(env).getStreamNode(sink2.getTransformation().getId()).getStatePartitioners().length); assertNotNull(getStreamGraph(env).getStreamNode(sink2.getTransformation().getId()).getStateKeySerializer()); assertNotNull(getStreamGraph(env).getStreamNode(sink2.getTransformation().getId()).getStateKeySerializer()); - assertEquals(key1, getStreamGraph(env).getStreamNode(sink2.getTransformation().getId()).getStatePartitioner1()); + assertEquals(key1, getStreamGraph(env).getStreamNode(sink2.getTransformation().getId()).getStatePartitioners()[0]); assertTrue(getStreamGraph(env).getStreamNode(sink2.getTransformation().getId()).getInEdges().get(0).getPartitioner() instanceof KeyGroupStreamPartitioner); KeySelector key2 = new KeySelector() { @@ -1040,8 +1040,8 @@ public class DataStreamTest extends TestLogger { DataStreamSink sink3 = env.generateSequence(1, 100).keyBy(key2).print(); - assertTrue(getStreamGraph(env).getStreamNode(sink3.getTransformation().getId()).getStatePartitioner1() != null); - assertEquals(key2, getStreamGraph(env).getStreamNode(sink3.getTransformation().getId()).getStatePartitioner1()); + assertEquals(1, getStreamGraph(env).getStreamNode(sink3.getTransformation().getId()).getStatePartitioners().length); + assertEquals(key2, getStreamGraph(env).getStreamNode(sink3.getTransformation().getId()).getStatePartitioners()[0]); assertTrue(getStreamGraph(env).getStreamNode(sink3.getTransformation().getId()).getInEdges().get(0).getPartitioner() instanceof KeyGroupStreamPartitioner); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTaskTest.java index 75b7e5bcd3..807da668e7 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTaskTest.java @@ -46,6 +46,7 @@ import org.apache.flink.streaming.api.operators.co.CoStreamMap; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.io.InputStatus; import org.apache.flink.streaming.runtime.io.StreamMultipleInputProcessor; +import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.streamstatus.StreamStatus; import org.apache.flink.streaming.runtime.tasks.OneInputStreamTaskTest.WatermarkMetricOperator; @@ -614,6 +615,33 @@ public class MultipleInputStreamTaskTest { } } + @Test + public void testLatencyMarker() throws Exception { + final Map metrics = new ConcurrentHashMap<>(); + final TaskMetricGroup taskMetricGroup = new StreamTaskTestHarness.TestTaskMetricGroup(metrics); + + try (StreamTaskMailboxTestHarness testHarness = + new MultipleInputStreamTaskTestHarnessBuilder<>(MultipleInputStreamTask::new, BasicTypeInfo.STRING_TYPE_INFO) + .addInput(BasicTypeInfo.STRING_TYPE_INFO) + .addInput(BasicTypeInfo.INT_TYPE_INFO) + .addInput(BasicTypeInfo.DOUBLE_TYPE_INFO) + .setupOutputForSingletonOperatorChain(new MapToStringMultipleInputOperatorFactory()) + .setTaskMetricGroup(taskMetricGroup) + .build()) { + ArrayDeque expectedOutput = new ArrayDeque<>(); + + OperatorID sourceId = new OperatorID(); + LatencyMarker latencyMarker = new LatencyMarker(42L, sourceId, 0); + testHarness.processElement(latencyMarker); + expectedOutput.add(latencyMarker); + + assertThat(testHarness.getOutput(), contains(expectedOutput.toArray())); + + testHarness.endInput(); + testHarness.waitForTaskCompletion(); + } + } + private static class MapToStringMultipleInputOperator extends AbstractStreamOperatorV2 implements MultipleInputStreamOperator { private static final long serialVersionUID = 1L; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java index f124c5546e..501f3b4492 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java @@ -185,6 +185,7 @@ public class OneInputStreamOperatorTestHarness else { checkState(inputs.size() == 1); Input input = inputs.get(0); + input.setKeyContextElement(element); input.processElement(element); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/MultipleInputITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/MultipleInputITCase.java index 02181124e3..d90ea4649a 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/MultipleInputITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/MultipleInputITCase.java @@ -17,7 +17,12 @@ package org.apache.flink.test.streaming.runtime; +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.MultipleConnectedStreams; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -28,6 +33,7 @@ import org.apache.flink.streaming.api.operators.Input; import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator; import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.streaming.api.transformations.KeyedMultipleInputTransformation; import org.apache.flink.streaming.api.transformations.MultipleInputTransformation; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.test.streaming.runtime.util.TestListResultSink; @@ -39,6 +45,8 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; import static org.junit.Assert.assertEquals; /** @@ -82,6 +90,96 @@ public class MultipleInputITCase extends AbstractTestBase { assertEquals(1 + 10 + 2 + 11 + 42 + 44, actualSum); } + @Test + public void testKeyedState() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + TestListResultSink resultSink = new TestListResultSink<>(); + + DataStream source1 = env.fromElements(0L, 3L); + DataStream source2 = env.fromElements(13L, 16L); + DataStream source3 = env.fromElements(101L, 104L); + + KeyedMultipleInputTransformation transform = new KeyedMultipleInputTransformation<>( + "My Operator", + new KeyedSumMultipleInputOperatorFactory(), + BasicTypeInfo.LONG_TYPE_INFO, + 1, + BasicTypeInfo.LONG_TYPE_INFO); + KeySelector keySelector = (KeySelector) value -> value % 3; + + transform.addInput(source1.getTransformation(), keySelector); + transform.addInput(source2.getTransformation(), keySelector); + transform.addInput(source3.getTransformation(), keySelector); + env.addOperator(transform); + + new MultipleConnectedStreams(env) + .transform(transform) + .addSink(resultSink); + + env.execute(); + + List result = resultSink.getResult(); + Collections.sort(result); + assertThat(result, contains(0L, 3L, 13L, 13L + 16L, 101L, 101L + 104L)); + } + + private static class KeyedSumMultipleInputOperator + extends AbstractStreamOperatorV2 implements MultipleInputStreamOperator { + + private ValueState sumState; + + public KeyedSumMultipleInputOperator(StreamOperatorParameters parameters) { + super(parameters, 3); + } + + @Override + public void initializeState(StateInitializationContext context) throws Exception { + super.initializeState(context); + + sumState = context + .getKeyedStateStore() + .getState(new ValueStateDescriptor<>("sum-state", LongSerializer.INSTANCE)); + } + + @Override + public List getInputs() { + return Arrays.asList( + new KeyedSumInput(this, 1), + new KeyedSumInput(this, 2), + new KeyedSumInput(this, 3) + ); + } + + private class KeyedSumInput extends AbstractInput { + public KeyedSumInput(AbstractStreamOperatorV2 owner, int inputId) { + super(owner, inputId); + } + + @Override + public void processElement(StreamRecord element) throws Exception { + if (sumState.value() == null) { + sumState.update(0L); + } + sumState.update(sumState.value() + element.getValue()); + output.collect(element.replace(sumState.value())); + } + } + } + + private static class KeyedSumMultipleInputOperatorFactory extends AbstractStreamOperatorFactory { + @Override + public > T createStreamOperator(StreamOperatorParameters parameters) { + return (T) new KeyedSumMultipleInputOperator(parameters); + } + + @Override + public Class> getStreamOperatorClass(ClassLoader classLoader) { + return KeyedSumMultipleInputOperator.class; + } + } + /** * 3 input operator that sums all of it inputs. */ -- Gitee From 1d5f44710270d1c615537f0d05ab49e699d3a6e5 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Wed, 25 Mar 2020 08:53:39 +0100 Subject: [PATCH 306/885] [FLINK-16317][operators] return this from (Keyed)MultipleInputTransformation#addInput --- .../KeyedMultipleInputTransformation.java | 3 ++- .../MultipleInputTransformation.java | 3 ++- .../api/graph/StreamGraphGeneratorTest.java | 8 ++++---- .../streaming/runtime/MultipleInputITCase.java | 17 ++++++++--------- 4 files changed, 16 insertions(+), 15 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/KeyedMultipleInputTransformation.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/KeyedMultipleInputTransformation.java index 9d15c97282..f9cd900ae2 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/KeyedMultipleInputTransformation.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/KeyedMultipleInputTransformation.java @@ -45,9 +45,10 @@ public class KeyedMultipleInputTransformation extends AbstractMultipleInput this.stateKeyType = stateKeyType; } - public void addInput(Transformation input, KeySelector keySelector) { + public KeyedMultipleInputTransformation addInput(Transformation input, KeySelector keySelector) { inputs.add(input); getStateKeySelectors().add(keySelector); + return this; } public TypeInformation getStateKeyType() { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/MultipleInputTransformation.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/MultipleInputTransformation.java index 9acc8938a4..cd650adcb7 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/MultipleInputTransformation.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/MultipleInputTransformation.java @@ -36,7 +36,8 @@ public class MultipleInputTransformation extends AbstractMultipleInputTrans super(name, operatorFactory, outputType, parallelism); } - public void addInput(Transformation input) { + public MultipleInputTransformation addInput(Transformation input) { inputs.add(input); + return this; } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java index 91ea7dd625..3d0f502b0d 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java @@ -331,11 +331,11 @@ public class StreamGraphGeneratorTest extends TestLogger { BasicTypeInfo.STRING_TYPE_INFO, 3); - transform.addInput(source1.getTransformation()); - transform.addInput(source2.getTransformation()); - transform.addInput(source3.getTransformation()); + env.addOperator(transform + .addInput(source1.getTransformation()) + .addInput(source2.getTransformation()) + .addInput(source3.getTransformation())); - env.addOperator(transform); StreamGraph streamGraph = env.getStreamGraph(); assertEquals(4, streamGraph.getStreamNodes().size()); diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/MultipleInputITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/MultipleInputITCase.java index d90ea4649a..28c23f65ce 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/MultipleInputITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/MultipleInputITCase.java @@ -72,11 +72,10 @@ public class MultipleInputITCase extends AbstractTestBase { BasicTypeInfo.LONG_TYPE_INFO, 1); - transform.addInput(source1.getTransformation()); - transform.addInput(source2.getTransformation()); - transform.addInput(source3.getTransformation()); - - env.addOperator(transform); + env.addOperator(transform + .addInput(source1.getTransformation()) + .addInput(source2.getTransformation()) + .addInput(source3.getTransformation())); new MultipleConnectedStreams(env) .transform(transform) @@ -109,10 +108,10 @@ public class MultipleInputITCase extends AbstractTestBase { BasicTypeInfo.LONG_TYPE_INFO); KeySelector keySelector = (KeySelector) value -> value % 3; - transform.addInput(source1.getTransformation(), keySelector); - transform.addInput(source2.getTransformation(), keySelector); - transform.addInput(source3.getTransformation(), keySelector); - env.addOperator(transform); + env.addOperator(transform + .addInput(source1.getTransformation(), keySelector) + .addInput(source2.getTransformation(), keySelector) + .addInput(source3.getTransformation(), keySelector)); new MultipleConnectedStreams(env) .transform(transform) -- Gitee From dee9156360122209c6d7b703bff5be882d1a6389 Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Wed, 25 Mar 2020 14:19:04 +0100 Subject: [PATCH 307/885] [FLINK-16778][e2e] Make sure all java e2e tests are running --- azure-pipelines.yml | 2 +- tools/azure-pipelines/build-apache-repo.yml | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/azure-pipelines.yml b/azure-pipelines.yml index 470ef35abd..0aa35500e0 100644 --- a/azure-pipelines.yml +++ b/azure-pipelines.yml @@ -63,7 +63,7 @@ jobs: vmImage: 'ubuntu-latest' e2e_pool_definition: vmImage: 'ubuntu-16.04' - environment: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11" + environment: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11 -Pe2e-hadoop -Pe2e-pre-commit" run_end_to_end: false container: flink-build-container jdk: jdk8 diff --git a/tools/azure-pipelines/build-apache-repo.yml b/tools/azure-pipelines/build-apache-repo.yml index e881a7356a..e227816916 100644 --- a/tools/azure-pipelines/build-apache-repo.yml +++ b/tools/azure-pipelines/build-apache-repo.yml @@ -63,7 +63,7 @@ stages: name: Default e2e_pool_definition: vmImage: 'ubuntu-16.04' - environment: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11" + environment: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11 -Pe2e-hadoop -Pe2e-pre-commit" run_end_to_end: false container: flink-build-container jdk: jdk8 @@ -80,7 +80,7 @@ stages: name: Default e2e_pool_definition: vmImage: 'ubuntu-16.04' - environment: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.4.1 -Pskip-hive-tests" + environment: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.4.1 -Pskip-hive-tests -Pe2e-hadoop -Pe2e-pre-commit" run_end_to_end: true container: flink-build-container jdk: jdk8 @@ -91,7 +91,7 @@ stages: name: Default e2e_pool_definition: vmImage: 'ubuntu-16.04' - environment: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.12 -Phive-1.2.1" + environment: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.12 -Phive-1.2.1 -Pe2e-hadoop -Pe2e-pre-commit" run_end_to_end: true container: flink-build-container jdk: jdk8 @@ -102,7 +102,7 @@ stages: name: Default e2e_pool_definition: vmImage: 'ubuntu-16.04' - environment: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11 -Djdk11" + environment: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11 -Djdk11 -Pe2e-hadoop -Pe2e-pre-commit" run_end_to_end: true container: flink-build-container jdk: jdk11 -- Gitee From 8fcadbff7c80832626b317dc8cf75bc49b2a47d7 Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Sat, 21 Mar 2020 17:06:13 +0300 Subject: [PATCH 308/885] [hotfix] Remove unused method in TaskExecutorProcessUtils --- .../clusterframework/TaskExecutorProcessUtils.java | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorProcessUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorProcessUtils.java index 9b7ae12512..e3ddd218a3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorProcessUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorProcessUtils.java @@ -115,16 +115,6 @@ public class TaskExecutorProcessUtils { .build(); } - public static ResourceProfile generateTotalAvailableResourceProfile(TaskExecutorProcessSpec taskExecutorProcessSpec) { - return ResourceProfile.newBuilder() - .setCpuCores(taskExecutorProcessSpec.getCpuCores()) - .setTaskHeapMemory(taskExecutorProcessSpec.getTaskHeapSize()) - .setTaskOffHeapMemory(taskExecutorProcessSpec.getTaskOffHeapSize()) - .setManagedMemory(taskExecutorProcessSpec.getManagedMemorySize()) - .setNetworkMemory(taskExecutorProcessSpec.getNetworkMemSize()) - .build(); - } - // ------------------------------------------------------------------------ // Memory Configuration Calculations // ------------------------------------------------------------------------ -- Gitee From d05d752f0f05fdc128a32b898da0f9d5e9cc37f7 Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Sat, 21 Mar 2020 17:09:56 +0300 Subject: [PATCH 309/885] [hotfix] add int type to JOB_MANAGER_HEAP_MEMORY_MB option --- .../java/org/apache/flink/configuration/JobManagerOptions.java | 1 + 1 file changed, 1 insertion(+) diff --git a/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java index 1de85c6f51..bf6b6235fa 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java @@ -114,6 +114,7 @@ public class JobManagerOptions { @Deprecated public static final ConfigOption JOB_MANAGER_HEAP_MEMORY_MB = key("jobmanager.heap.mb") + .intType() .defaultValue(1024) .withDescription("JVM heap size (in megabytes) for the JobManager."); -- Gitee From a39630d8fcc2a038240947b17b7d3d27e0347956 Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Sat, 21 Mar 2020 17:18:12 +0300 Subject: [PATCH 310/885] [hotfix] change type of JOB_MANAGER_HEAP_MEMORY option from String to MemorySize --- docs/_includes/generated/all_jobmanager_section.html | 4 ++-- docs/_includes/generated/job_manager_configuration.html | 4 ++-- .../org/apache/flink/configuration/ConfigurationUtils.java | 4 ++-- .../org/apache/flink/configuration/JobManagerOptions.java | 5 +++-- .../flink/kubernetes/cli/KubernetesSessionCliTest.java | 4 ++-- .../java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java | 2 +- .../java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java | 4 ++-- 7 files changed, 14 insertions(+), 13 deletions(-) diff --git a/docs/_includes/generated/all_jobmanager_section.html b/docs/_includes/generated/all_jobmanager_section.html index ab01a856e0..3369667cb2 100644 --- a/docs/_includes/generated/all_jobmanager_section.html +++ b/docs/_includes/generated/all_jobmanager_section.html @@ -28,8 +28,8 @@
    jobmanager.heap.size
    - "1024m" - String + 1 gb + MemorySize JVM heap size for the JobManager. diff --git a/docs/_includes/generated/job_manager_configuration.html b/docs/_includes/generated/job_manager_configuration.html index 3839890635..a5c6c83fb6 100644 --- a/docs/_includes/generated/job_manager_configuration.html +++ b/docs/_includes/generated/job_manager_configuration.html @@ -34,8 +34,8 @@
    jobmanager.heap.size
    - "1024m" - String + 1 gb + MemorySize JVM heap size for the JobManager. diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigurationUtils.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigurationUtils.java index 9eda0e467c..fa1555a96b 100755 --- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigurationUtils.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigurationUtils.java @@ -51,12 +51,12 @@ public class ConfigurationUtils { */ public static MemorySize getJobManagerHeapMemory(Configuration configuration) { if (configuration.containsKey(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY.key())) { - return MemorySize.parse(configuration.getString(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY)); + return configuration.get(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY); } else if (configuration.containsKey(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY_MB.key())) { return MemorySize.ofMebiBytes(configuration.getInteger(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY_MB)); } else { //use default value - return MemorySize.parse(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY.defaultValue()); + return JobManagerOptions.JOB_MANAGER_HEAP_MEMORY.defaultValue(); } } diff --git a/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java index bf6b6235fa..5c3e858725 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java @@ -102,9 +102,10 @@ public class JobManagerOptions { * JVM heap size for the JobManager with memory size. */ @Documentation.Section(Documentation.Sections.ALL_JOB_MANAGER) - public static final ConfigOption JOB_MANAGER_HEAP_MEMORY = + public static final ConfigOption JOB_MANAGER_HEAP_MEMORY = key("jobmanager.heap.size") - .defaultValue("1024m") + .memoryType() + .defaultValue(MemorySize.ofMebiBytes(1024)) .withDescription("JVM heap size for the JobManager."); /** diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/cli/KubernetesSessionCliTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/cli/KubernetesSessionCliTest.java index 8fa097b362..a7bf4782c8 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/cli/KubernetesSessionCliTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/cli/KubernetesSessionCliTest.java @@ -120,7 +120,7 @@ public class KubernetesSessionCliTest { final int taskManagerMemory = 7331; final int slotsPerTaskManager = 30; - configuration.setString(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY, jobManagerMemory + "m"); + configuration.set(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY, MemorySize.ofMebiBytes(jobManagerMemory)); configuration.set(TaskManagerOptions.TOTAL_PROCESS_MEMORY, MemorySize.ofMebiBytes(taskManagerMemory)); configuration.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, slotsPerTaskManager); @@ -150,7 +150,7 @@ public class KubernetesSessionCliTest { public void testConfigurationClusterSpecification() throws Exception { final Configuration configuration = new Configuration(); final int jobManagerMemory = 1337; - configuration.setString(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY, jobManagerMemory + "m"); + configuration.set(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY, MemorySize.ofMebiBytes(jobManagerMemory)); final int taskManagerMemory = 7331; configuration.set(TaskManagerOptions.TOTAL_PROCESS_MEMORY, MemorySize.ofMebiBytes(taskManagerMemory)); final int slotsPerTaskManager = 42; diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java index 9955e37cdb..352ca6e387 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java @@ -355,7 +355,7 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine { if (!MemorySize.MemoryUnit.hasUnit(jmMemoryVal)) { jmMemoryVal += "m"; } - effectiveConfiguration.setString(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY, jmMemoryVal); + effectiveConfiguration.set(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY, MemorySize.parse(jmMemoryVal)); } if (commandLine.hasOption(tmMemory.getOpt())) { diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java index a3831d6122..bf097fb797 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java @@ -314,7 +314,7 @@ public class FlinkYarnSessionCliTest extends TestLogger { final int taskManagerMemory = 7331; final int slotsPerTaskManager = 30; - configuration.setString(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY, jobManagerMemory + "m"); + configuration.set(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY, MemorySize.ofMebiBytes(jobManagerMemory)); configuration.set(TaskManagerOptions.TOTAL_PROCESS_MEMORY, MemorySize.ofMebiBytes(taskManagerMemory)); configuration.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, slotsPerTaskManager); @@ -340,7 +340,7 @@ public class FlinkYarnSessionCliTest extends TestLogger { public void testConfigurationClusterSpecification() throws Exception { final Configuration configuration = new Configuration(); final int jobManagerMemory = 1337; - configuration.setString(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY, jobManagerMemory + "m"); + configuration.set(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY, MemorySize.ofMebiBytes(jobManagerMemory)); final int taskManagerMemory = 7331; configuration.set(TaskManagerOptions.TOTAL_PROCESS_MEMORY, MemorySize.ofMebiBytes(taskManagerMemory)); final int slotsPerTaskManager = 42; -- Gitee From 3e754ed91fcd8be05e416d9f221ae04d118f1f0c Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Sun, 15 Mar 2020 20:53:05 +0300 Subject: [PATCH 311/885] [FLINK-16615] Introduce data structures and utilities to calculate Job Manager memory components This closes #11445. --- .../configuration/ConfigurationUtils.java | 2 +- .../configuration/JobManagerOptions.java | 107 +++ .../flink/dist/BashJavaUtilsITCase.java | 2 +- .../JavaCmdTaskManagerDecorator.java | 3 +- .../LaunchableMesosWorker.java | 3 +- .../clusterframework/BootstrapTools.java | 3 +- .../TaskExecutorProcessSpec.java | 116 ++- .../TaskExecutorProcessUtils.java | 686 ++---------------- .../jobmanager/JobManagerProcessSpec.java | 73 ++ .../jobmanager/JobManagerProcessUtils.java | 81 +++ .../flink/runtime/util/BashJavaUtils.java | 3 +- .../memory/CommonProcessMemorySpec.java | 97 +++ .../util/config/memory/FlinkMemory.java | 34 + .../util/config/memory/FlinkMemoryUtils.java | 32 + .../memory/JvmMetaspaceAndOverhead.java | 52 ++ .../JvmMetaspaceAndOverheadOptions.java | 59 ++ .../config/memory/LegacyMemoryOptions.java | 52 ++ .../MemoryBackwardsCompatibilityUtils.java | 94 +++ .../config/memory/ProcessMemoryOptions.java | 65 ++ .../util/config/memory/ProcessMemorySpec.java | 40 + .../config/memory/ProcessMemoryUtils.java | 281 +++++++ .../util/config/memory/RangeFraction.java | 52 ++ .../jobmanager/JobManagerFlinkMemory.java | 72 ++ .../JobManagerFlinkMemoryUtils.java | 104 +++ .../taskmanager/TaskExecutorFlinkMemory.java | 131 ++++ .../TaskExecutorFlinkMemoryUtils.java | 299 ++++++++ .../TaskExecutorProcessUtilsTest.java | 263 +------ .../JobManagerProcessUtilsTest.java | 226 ++++++ .../memory/ProcessMemoryUtilsTestBase.java | 354 +++++++++ 29 files changed, 2426 insertions(+), 960 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManagerProcessSpec.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManagerProcessUtils.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/CommonProcessMemorySpec.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/FlinkMemory.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/FlinkMemoryUtils.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/JvmMetaspaceAndOverhead.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/JvmMetaspaceAndOverheadOptions.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/LegacyMemoryOptions.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/MemoryBackwardsCompatibilityUtils.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/ProcessMemoryOptions.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/ProcessMemorySpec.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/ProcessMemoryUtils.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/RangeFraction.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/jobmanager/JobManagerFlinkMemory.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/jobmanager/JobManagerFlinkMemoryUtils.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/taskmanager/TaskExecutorFlinkMemory.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/taskmanager/TaskExecutorFlinkMemoryUtils.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerProcessUtilsTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/util/config/memory/ProcessMemoryUtilsTestBase.java diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigurationUtils.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigurationUtils.java index fa1555a96b..0408a37248 100755 --- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigurationUtils.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigurationUtils.java @@ -192,7 +192,7 @@ public class ConfigurationUtils { } @VisibleForTesting - public static Map parseTmResourceJvmParams(String jvmParamsStr) { + public static Map parseJvmArgString(String jvmParamsStr) { final String xmx = "-Xmx"; final String xms = "-Xms"; final String maxDirect = "-XX:MaxDirectMemorySize="; diff --git a/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java index 5c3e858725..8a54f1cb7c 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java @@ -32,6 +32,8 @@ import static org.apache.flink.configuration.description.TextElement.text; @PublicEvolving public class JobManagerOptions { + public static final MemorySize MIN_JVM_HEAP_SIZE = MemorySize.ofMebiBytes(128); + /** * The config parameter defining the network address to connect to * for communication with the job manager. @@ -119,6 +121,111 @@ public class JobManagerOptions { .defaultValue(1024) .withDescription("JVM heap size (in megabytes) for the JobManager."); + /** + * Total Process Memory size for the JobManager. + */ + @Documentation.Section(Documentation.Sections.COMMON_MEMORY) + @Documentation.ExcludeFromDocumentation("New JM memory model is still in development.") + public static final ConfigOption TOTAL_PROCESS_MEMORY = + key("jobmanager.memory.process.size") + .memoryType() + .noDefaultValue() + .withDescription("Total Process Memory size for the JobManager. This includes all the memory that a " + + "JobManager JVM process consumes, consisting of Total Flink Memory, JVM Metaspace, and JVM Overhead. " + + "In containerized setups, this should be set to the container memory. See also " + + "'jobmanager.memory.flink.size' for Total Flink Memory size configuration."); + + /** + * Total Flink Memory size for the JobManager. + */ + @Documentation.Section(Documentation.Sections.COMMON_MEMORY) + @Documentation.ExcludeFromDocumentation("New JM memory model is still in development.") + public static final ConfigOption TOTAL_FLINK_MEMORY = + key("jobmanager.memory.flink.size") + .memoryType() + .noDefaultValue() + .withDescription(String.format( + "Total Flink Memory size for the JobManager. This includes all the " + + "memory that a JobManager consumes, except for JVM Metaspace and JVM Overhead. It consists of " + + "JVM Heap Memory and Off-heap Memory. See also '%s' for total process memory size configuration.", + TOTAL_PROCESS_MEMORY.key())); + + /** + * JVM Heap Memory size for the JobManager. + */ + @Documentation.Section(Documentation.Sections.COMMON_MEMORY) + @Documentation.ExcludeFromDocumentation("New JM memory model is still in development.") + public static final ConfigOption JVM_HEAP_MEMORY = + key("jobmanager.memory.heap.size") + .memoryType() + .noDefaultValue() + .withDescription("JVM Heap Memory size for JobManager. The minimum recommended JVM Heap size is " + + MIN_JVM_HEAP_SIZE.toHumanReadableString() + '.'); + + /** + * Off-heap Memory size for the JobManager. + */ + @Documentation.Section(Documentation.Sections.COMMON_MEMORY) + @Documentation.ExcludeFromDocumentation("New JM memory model is still in development.") + public static final ConfigOption OFF_HEAP_MEMORY = + key("jobmanager.memory.off-heap.size") + .memoryType() + .defaultValue(MemorySize.ofMebiBytes(128)) + .withDescription("Off-heap Memory size for JobManager. The JVM direct memory limit of the Job Manager " + + "process (-XX:MaxDirectMemorySize) will be set to this value. This option covers all off-heap memory " + + "usage including direct and native memory allocation."); + + /** + * JVM Metaspace Size for the JobManager. + */ + @Documentation.Section(Documentation.Sections.COMMON_MEMORY) + @Documentation.ExcludeFromDocumentation("New JM memory model is still in development.") + public static final ConfigOption JVM_METASPACE = + key("jobmanager.memory.jvm-metaspace.size") + .memoryType() + .defaultValue(MemorySize.ofMebiBytes(256)) + .withDescription("JVM Metaspace Size for the JobManager."); + + private static final String JVM_OVERHEAD_DESCRIPTION = "This is off-heap memory reserved for JVM " + + "overhead, such as thread stack space, compile cache, etc. This includes native memory but not direct " + + "memory, and will not be counted when Flink calculates JVM max direct memory size parameter. The size " + + "of JVM Overhead is derived to make up the configured fraction of the Total Process Memory. If the " + + "derived size is less or greater than the configured min or max size, the min or max size will be used. The " + + "exact size of JVM Overhead can be explicitly specified by setting the min and max size to the same value."; + + /** + * Min JVM Overhead size for the JobManager. + */ + @Documentation.Section(Documentation.Sections.COMMON_MEMORY) + @Documentation.ExcludeFromDocumentation("New JM memory model is still in development.") + public static final ConfigOption JVM_OVERHEAD_MIN = + key("jobmanager.memory.jvm-overhead.min") + .memoryType() + .defaultValue(MemorySize.ofMebiBytes(192)) + .withDescription("Min JVM Overhead size for the JobManager. " + JVM_OVERHEAD_DESCRIPTION); + + /** + * Max JVM Overhead size for the TaskExecutors. + */ + @Documentation.Section(Documentation.Sections.COMMON_MEMORY) + @Documentation.ExcludeFromDocumentation("New JM memory model is still in development.") + public static final ConfigOption JVM_OVERHEAD_MAX = + key("jobmanager.memory.jvm-overhead.max") + .memoryType() + .defaultValue(MemorySize.parse("1g")) + .withDescription("Max JVM Overhead size for the JobManager. " + JVM_OVERHEAD_DESCRIPTION); + + /** + * Fraction of Total Process Memory to be reserved for JVM Overhead. + */ + @Documentation.Section(Documentation.Sections.COMMON_MEMORY) + @Documentation.ExcludeFromDocumentation("New JM memory model is still in development.") + public static final ConfigOption JVM_OVERHEAD_FRACTION = + key("jobmanager.memory.jvm-overhead.fraction") + .floatType() + .defaultValue(0.1f) + .withDescription("Fraction of Total Process Memory to be reserved for JVM Overhead. " + JVM_OVERHEAD_DESCRIPTION); + /** * The maximum number of prior execution attempts kept in history. */ diff --git a/flink-dist/src/test/java/org/apache/flink/dist/BashJavaUtilsITCase.java b/flink-dist/src/test/java/org/apache/flink/dist/BashJavaUtilsITCase.java index daf6f1b7a7..1281ed2cf1 100644 --- a/flink-dist/src/test/java/org/apache/flink/dist/BashJavaUtilsITCase.java +++ b/flink-dist/src/test/java/org/apache/flink/dist/BashJavaUtilsITCase.java @@ -43,7 +43,7 @@ public class BashJavaUtilsITCase extends JavaBashTestBase { List lines = Arrays.asList(executeScript(commands).split(System.lineSeparator())); assertEquals(2, lines.size()); - ConfigurationUtils.parseTmResourceJvmParams(lines.get(lines.size() - 2)); + ConfigurationUtils.parseJvmArgString(lines.get(lines.size() - 2)); ConfigurationUtils.parseTmResourceDynamicConfigs(lines.get(lines.size() - 1)); } } diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/JavaCmdTaskManagerDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/JavaCmdTaskManagerDecorator.java index 34332d62ef..64d4e46807 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/JavaCmdTaskManagerDecorator.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/JavaCmdTaskManagerDecorator.java @@ -27,6 +27,7 @@ import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameter import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec; import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils; import org.apache.flink.runtime.entrypoint.parser.CommandLineOptions; +import org.apache.flink.runtime.util.config.memory.ProcessMemoryUtils; import io.fabric8.kubernetes.api.model.Container; import io.fabric8.kubernetes.api.model.ContainerBuilder; @@ -87,7 +88,7 @@ public class JavaCmdTaskManagerDecorator extends AbstractKubernetesStepDecorator String mainClass, String mainArgs) { final TaskExecutorProcessSpec taskExecutorProcessSpec = tmParams.getTaskExecutorProcessSpec(); - final String jvmMemOpts = TaskExecutorProcessUtils.generateJvmParametersStr(taskExecutorProcessSpec); + final String jvmMemOpts = ProcessMemoryUtils.generateJvmParametersStr(taskExecutorProcessSpec); String args = TaskExecutorProcessUtils.generateDynamicConfigsStr(taskExecutorProcessSpec); if (mainArgs != null) { args += " " + mainArgs; diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java index f19e6a990e..ea1357ae27 100644 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java @@ -29,6 +29,7 @@ import org.apache.flink.mesos.util.MesosResourceAllocation; import org.apache.flink.runtime.clusterframework.ContainerSpecification; import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils; +import org.apache.flink.runtime.util.config.memory.ProcessMemoryUtils; import org.apache.flink.util.Preconditions; import com.netflix.fenzo.ConstraintEvaluator; @@ -276,7 +277,7 @@ public class LaunchableMesosWorker implements LaunchableTask { env.addVariables(variable(MesosConfigKeys.ENV_FLINK_CONTAINER_ID, taskInfo.getTaskId().getValue())); // finalize the memory parameters - jvmArgs.append(" ").append(TaskExecutorProcessUtils.generateJvmParametersStr(tmParams.getTaskExecutorProcessSpec())); + jvmArgs.append(" ").append(ProcessMemoryUtils.generateJvmParametersStr(tmParams.getTaskExecutorProcessSpec())); // pass dynamic system properties jvmArgs.append(' ').append( diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java index cc294b213f..741edd4fc2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java @@ -28,6 +28,7 @@ import org.apache.flink.configuration.CoreOptions; import org.apache.flink.configuration.ResourceManagerOptions; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.entrypoint.parser.CommandLineOptions; +import org.apache.flink.runtime.util.config.memory.ProcessMemoryUtils; import org.apache.flink.util.NetUtils; import org.apache.flink.util.OperatingSystem; @@ -401,7 +402,7 @@ public class BootstrapTools { startCommandValues.put("java", "$JAVA_HOME/bin/java"); final TaskExecutorProcessSpec taskExecutorProcessSpec = tmParams.getTaskExecutorProcessSpec(); - startCommandValues.put("jvmmem", TaskExecutorProcessUtils.generateJvmParametersStr(taskExecutorProcessSpec)); + startCommandValues.put("jvmmem", ProcessMemoryUtils.generateJvmParametersStr(taskExecutorProcessSpec)); String javaOpts = flinkConfig.getString(CoreOptions.FLINK_JVM_OPTIONS); if (flinkConfig.getString(CoreOptions.FLINK_TM_JVM_OPTIONS).length() > 0) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorProcessSpec.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorProcessSpec.java index e530b8ee27..c214199974 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorProcessSpec.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorProcessSpec.java @@ -18,10 +18,12 @@ package org.apache.flink.runtime.clusterframework; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.resources.CPUResource; import org.apache.flink.configuration.MemorySize; - -import java.io.Serializable; +import org.apache.flink.runtime.util.config.memory.CommonProcessMemorySpec; +import org.apache.flink.runtime.util.config.memory.JvmMetaspaceAndOverhead; +import org.apache.flink.runtime.util.config.memory.taskmanager.TaskExecutorFlinkMemory; /** * Describe the specifics of different resource dimensions of the TaskExecutor process. @@ -73,26 +75,12 @@ import java.io.Serializable; * └ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┘ * */ -public class TaskExecutorProcessSpec implements Serializable { +public class TaskExecutorProcessSpec extends CommonProcessMemorySpec { + private static final long serialVersionUID = 1L; private final CPUResource cpuCores; - private final MemorySize frameworkHeapSize; - - private final MemorySize frameworkOffHeapMemorySize; - - private final MemorySize taskHeapSize; - - private final MemorySize taskOffHeapSize; - - private final MemorySize networkMemSize; - - private final MemorySize managedMemorySize; - - private final MemorySize jvmMetaspaceSize; - - private final MemorySize jvmOverheadSize; - + @VisibleForTesting public TaskExecutorProcessSpec( CPUResource cpuCores, MemorySize frameworkHeapSize, @@ -104,81 +92,67 @@ public class TaskExecutorProcessSpec implements Serializable { MemorySize jvmMetaspaceSize, MemorySize jvmOverheadSize) { + this( + cpuCores, + new TaskExecutorFlinkMemory( + frameworkHeapSize, + frameworkOffHeapSize, + taskHeapSize, + taskOffHeapSize, + networkMemSize, + managedMemorySize), + new JvmMetaspaceAndOverhead(jvmMetaspaceSize, jvmOverheadSize)); + } + + protected TaskExecutorProcessSpec( + CPUResource cpuCores, + TaskExecutorFlinkMemory flinkMemory, + JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead) { + + super(flinkMemory, jvmMetaspaceAndOverhead); this.cpuCores = cpuCores; - this.frameworkHeapSize = frameworkHeapSize; - this.frameworkOffHeapMemorySize = frameworkOffHeapSize; - this.taskHeapSize = taskHeapSize; - this.taskOffHeapSize = taskOffHeapSize; - this.networkMemSize = networkMemSize; - this.managedMemorySize = managedMemorySize; - this.jvmMetaspaceSize = jvmMetaspaceSize; - this.jvmOverheadSize = jvmOverheadSize; } public CPUResource getCpuCores() { return cpuCores; } - public MemorySize getFrameworkHeapSize() { - return frameworkHeapSize; + MemorySize getFrameworkHeapSize() { + return getFlinkMemory().getFrameworkHeap(); } - public MemorySize getFrameworkOffHeapMemorySize() { - return frameworkOffHeapMemorySize; + MemorySize getFrameworkOffHeapMemorySize() { + return getFlinkMemory().getFrameworkOffHeap(); } - public MemorySize getTaskHeapSize() { - return taskHeapSize; + MemorySize getTaskHeapSize() { + return getFlinkMemory().getTaskHeap(); } - public MemorySize getTaskOffHeapSize() { - return taskOffHeapSize; + MemorySize getTaskOffHeapSize() { + return getFlinkMemory().getTaskOffHeap(); } - public MemorySize getNetworkMemSize() { - return networkMemSize; + MemorySize getNetworkMemSize() { + return getFlinkMemory().getNetwork(); } public MemorySize getManagedMemorySize() { - return managedMemorySize; - } - - public MemorySize getJvmMetaspaceSize() { - return jvmMetaspaceSize; - } - - public MemorySize getJvmOverheadSize() { - return jvmOverheadSize; - } - - public MemorySize getTotalFlinkMemorySize() { - return frameworkHeapSize.add(frameworkOffHeapMemorySize).add(taskHeapSize).add(taskOffHeapSize).add(networkMemSize).add(getManagedMemorySize()); - } - - public MemorySize getTotalProcessMemorySize() { - return getTotalFlinkMemorySize().add(jvmMetaspaceSize).add(jvmOverheadSize); - } - - public MemorySize getJvmHeapMemorySize() { - return frameworkHeapSize.add(taskHeapSize); - } - - public MemorySize getJvmDirectMemorySize() { - return frameworkOffHeapMemorySize.add(taskOffHeapSize).add(networkMemSize); + return getFlinkMemory().getManaged(); } @Override public String toString() { return "TaskExecutorProcessSpec {" + "cpuCores=" + cpuCores.getValue().doubleValue() - + ", frameworkHeapSize=" + frameworkHeapSize.toHumanReadableString() - + ", frameworkOffHeapSize=" + frameworkOffHeapMemorySize.toHumanReadableString() - + ", taskHeapSize=" + taskHeapSize.toHumanReadableString() - + ", taskOffHeapSize=" + taskOffHeapSize.toHumanReadableString() - + ", networkMemSize=" + networkMemSize.toHumanReadableString() - + ", managedMemorySize=" + managedMemorySize.toHumanReadableString() - + ", jvmMetaspaceSize=" + jvmMetaspaceSize.toHumanReadableString() - + ", jvmOverheadSize=" + jvmOverheadSize.toHumanReadableString() - + "}"; + + ", frameworkHeapSize=" + getFrameworkHeapSize().toHumanReadableString() + + ", frameworkOffHeapSize=" + getFrameworkOffHeapMemorySize().toHumanReadableString() + + ", taskHeapSize=" + getTaskHeapSize().toHumanReadableString() + + ", taskOffHeapSize=" + getTaskOffHeapSize().toHumanReadableString() + + ", networkMemSize=" + getNetworkMemSize().toHumanReadableString() + + ", managedMemorySize=" + getManagedMemorySize().toHumanReadableString() + + ", jvmMetaspaceSize=" + getJvmMetaspaceSize().toHumanReadableString() + + ", jvmOverheadSize=" + getJvmOverheadSize().toHumanReadableString() + + '}'; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorProcessUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorProcessUtils.java index e3ddd218a3..4c725c339d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorProcessUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/TaskExecutorProcessUtils.java @@ -23,22 +23,23 @@ import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.IllegalConfigurationException; import org.apache.flink.configuration.MemorySize; -import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; -import org.apache.flink.runtime.util.ConfigurationParserUtils; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - +import org.apache.flink.runtime.util.config.memory.CommonProcessMemorySpec; +import org.apache.flink.runtime.util.config.memory.JvmMetaspaceAndOverhead; +import org.apache.flink.runtime.util.config.memory.JvmMetaspaceAndOverheadOptions; +import org.apache.flink.runtime.util.config.memory.MemoryBackwardsCompatibilityUtils; +import org.apache.flink.runtime.util.config.memory.LegacyMemoryOptions; +import org.apache.flink.runtime.util.config.memory.ProcessMemoryOptions; +import org.apache.flink.runtime.util.config.memory.ProcessMemoryUtils; +import org.apache.flink.runtime.util.config.memory.taskmanager.TaskExecutorFlinkMemory; +import org.apache.flink.runtime.util.config.memory.taskmanager.TaskExecutorFlinkMemoryUtils; + +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; - -import static org.apache.flink.util.Preconditions.checkArgument; -import static org.apache.flink.util.Preconditions.checkNotNull; /** * Utility class for TaskExecutor memory configurations. @@ -46,24 +47,31 @@ import static org.apache.flink.util.Preconditions.checkNotNull; *

    See {@link TaskExecutorProcessSpec} for details about memory components of TaskExecutor and their relationships. */ public class TaskExecutorProcessUtils { - private static final Logger LOG = LoggerFactory.getLogger(TaskExecutorProcessUtils.class); - private TaskExecutorProcessUtils() {} + static final ProcessMemoryOptions TM_PROCESS_MEMORY_OPTIONS = new ProcessMemoryOptions( + Arrays.asList(TaskManagerOptions.TASK_HEAP_MEMORY, TaskManagerOptions.MANAGED_MEMORY_SIZE), + TaskManagerOptions.TOTAL_FLINK_MEMORY, + TaskManagerOptions.TOTAL_PROCESS_MEMORY, + new JvmMetaspaceAndOverheadOptions( + TaskManagerOptions.JVM_METASPACE, + TaskManagerOptions.JVM_OVERHEAD_MIN, + TaskManagerOptions.JVM_OVERHEAD_MAX, + TaskManagerOptions.JVM_OVERHEAD_FRACTION)); - // ------------------------------------------------------------------------ - // Generating JVM Parameters - // ------------------------------------------------------------------------ + @SuppressWarnings("deprecation") + static final LegacyMemoryOptions TM_LEGACY_HEAP_OPTIONS = + new LegacyMemoryOptions( + "FLINK_TM_HEAP", + TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY, + TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY_MB); - public static String generateJvmParametersStr(final TaskExecutorProcessSpec taskExecutorProcessSpec) { - final MemorySize jvmHeapSize = taskExecutorProcessSpec.getJvmHeapMemorySize(); - final MemorySize jvmDirectSize = taskExecutorProcessSpec.getJvmDirectMemorySize(); - final MemorySize jvmMetaspaceSize = taskExecutorProcessSpec.getJvmMetaspaceSize(); + private static final ProcessMemoryUtils PROCESS_MEMORY_UTILS = new ProcessMemoryUtils<>( + TM_PROCESS_MEMORY_OPTIONS, + new TaskExecutorFlinkMemoryUtils()); - return "-Xmx" + jvmHeapSize.getBytes() - + " -Xms" + jvmHeapSize.getBytes() - + " -XX:MaxDirectMemorySize=" + jvmDirectSize.getBytes() - + " -XX:MaxMetaspaceSize=" + jvmMetaspaceSize.getBytes(); - } + private static final MemoryBackwardsCompatibilityUtils LEGACY_MEMORY_UTILS = new MemoryBackwardsCompatibilityUtils(TM_LEGACY_HEAP_OPTIONS); + + private TaskExecutorProcessUtils() {} // ------------------------------------------------------------------------ // Generating Dynamic Config Options @@ -124,510 +132,15 @@ public class TaskExecutorProcessUtils { } public static TaskExecutorProcessSpec processSpecFromConfig(final Configuration config) { - if (isTaskHeapMemorySizeExplicitlyConfigured(config) && isManagedMemorySizeExplicitlyConfigured(config)) { - // both task heap memory and managed memory are configured, use these to derive total flink memory - return deriveProcessSpecWithExplicitTaskAndManagedMemory(config); - } else if (isTotalFlinkMemorySizeExplicitlyConfigured(config)) { - // either of task heap memory and managed memory is not configured, total flink memory is configured, - // derive from total flink memory - return deriveProcessSpecWithTotalFlinkMemory(config); - } else if (isTotalProcessMemorySizeExplicitlyConfigured(config)) { - // total flink memory is not configured, total process memory is configured, - // derive from total process memory - return deriveProcessSpecWithTotalProcessMemory(config); - } else { - throw new IllegalConfigurationException(String.format("Either Task Heap Memory size (%s) and Managed Memory size (%s), or Total Flink" - + " Memory size (%s), or Total Process Memory size (%s) need to be configured explicitly.", - TaskManagerOptions.TASK_HEAP_MEMORY.key(), - TaskManagerOptions.MANAGED_MEMORY_SIZE.key(), - TaskManagerOptions.TOTAL_FLINK_MEMORY.key(), - TaskManagerOptions.TOTAL_PROCESS_MEMORY.key())); - } - } - - public static boolean isTaskExecutorProcessResourceExplicitlyConfigured(final Configuration config) { - return (isTaskHeapMemorySizeExplicitlyConfigured(config) && isManagedMemorySizeExplicitlyConfigured(config)) - || isTotalFlinkMemorySizeExplicitlyConfigured(config) - || isTotalProcessMemorySizeExplicitlyConfigured(config); - } - - private static TaskExecutorProcessSpec deriveProcessSpecWithExplicitTaskAndManagedMemory(final Configuration config) { - // derive flink internal memory from explicitly configure task heap memory size and managed memory size - - final MemorySize taskHeapMemorySize = getTaskHeapMemorySize(config); - final MemorySize managedMemorySize = getManagedMemorySize(config); - - final MemorySize frameworkHeapMemorySize = getFrameworkHeapMemorySize(config); - final MemorySize frameworkOffHeapMemorySize = getFrameworkOffHeapMemorySize(config); - final MemorySize taskOffHeapMemorySize = getTaskOffHeapMemorySize(config); - - final MemorySize networkMemorySize; - final MemorySize totalFlinkExcludeNetworkMemorySize = - frameworkHeapMemorySize.add(frameworkOffHeapMemorySize).add(taskHeapMemorySize).add(taskOffHeapMemorySize).add(managedMemorySize); - - if (isTotalFlinkMemorySizeExplicitlyConfigured(config)) { - // derive network memory from total flink memory, and check against network min/max - final MemorySize totalFlinkMemorySize = getTotalFlinkMemorySize(config); - if (totalFlinkExcludeNetworkMemorySize.getBytes() > totalFlinkMemorySize.getBytes()) { - throw new IllegalConfigurationException( - "Sum of configured Framework Heap Memory (" + frameworkHeapMemorySize.toHumanReadableString() - + "), Framework Off-Heap Memory (" + frameworkOffHeapMemorySize.toHumanReadableString() - + "), Task Heap Memory (" + taskHeapMemorySize.toHumanReadableString() - + "), Task Off-Heap Memory (" + taskOffHeapMemorySize.toHumanReadableString() - + ") and Managed Memory (" + managedMemorySize.toHumanReadableString() - + ") exceed configured Total Flink Memory (" + totalFlinkMemorySize.toHumanReadableString() + ")."); - } - networkMemorySize = totalFlinkMemorySize.subtract(totalFlinkExcludeNetworkMemorySize); - sanityCheckNetworkMemoryWithExplicitlySetTotalFlinkAndHeapMemory(config, networkMemorySize, totalFlinkMemorySize); - } else { - // derive network memory from network configs - if (isUsingLegacyNetworkConfigs(config)) { - networkMemorySize = getNetworkMemorySizeWithLegacyConfig(config); - } else { - networkMemorySize = deriveNetworkMemoryWithInverseFraction(config, totalFlinkExcludeNetworkMemorySize); - } - } - - final FlinkInternalMemory flinkInternalMemory = new FlinkInternalMemory( - frameworkHeapMemorySize, - frameworkOffHeapMemorySize, - taskHeapMemorySize, - taskOffHeapMemorySize, - networkMemorySize, - managedMemorySize); - sanityCheckTotalFlinkMemory(config, flinkInternalMemory); - - // derive jvm metaspace and overhead - - final JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead = deriveJvmMetaspaceAndOverheadFromTotalFlinkMemory(config, flinkInternalMemory.getTotalFlinkMemorySize()); - - return createTaskExecutorProcessSpec(config, flinkInternalMemory, jvmMetaspaceAndOverhead); - } - - private static TaskExecutorProcessSpec deriveProcessSpecWithTotalFlinkMemory(final Configuration config) { - // derive flink internal memory from explicitly configured total flink memory - - final MemorySize totalFlinkMemorySize = getTotalFlinkMemorySize(config); - final FlinkInternalMemory flinkInternalMemory = deriveInternalMemoryFromTotalFlinkMemory(config, totalFlinkMemorySize); - - // derive jvm metaspace and overhead - - final JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead = deriveJvmMetaspaceAndOverheadFromTotalFlinkMemory(config, totalFlinkMemorySize); - - return createTaskExecutorProcessSpec(config, flinkInternalMemory, jvmMetaspaceAndOverhead); - } - - private static TaskExecutorProcessSpec deriveProcessSpecWithTotalProcessMemory(final Configuration config) { - // derive total flink memory from explicitly configured total process memory size - - final MemorySize totalProcessMemorySize = getTotalProcessMemorySize(config); - final MemorySize jvmMetaspaceSize = getJvmMetaspaceSize(config); - final MemorySize jvmOverheadSize = deriveJvmOverheadWithFraction(config, totalProcessMemorySize); - final JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead = new JvmMetaspaceAndOverhead(jvmMetaspaceSize, jvmOverheadSize); - - if (jvmMetaspaceAndOverhead.getTotalJvmMetaspaceAndOverheadSize().getBytes() > totalProcessMemorySize.getBytes()) { - throw new IllegalConfigurationException( - "Sum of configured JVM Metaspace (" + jvmMetaspaceAndOverhead.metaspace.toHumanReadableString() - + ") and JVM Overhead (" + jvmMetaspaceAndOverhead.overhead.toHumanReadableString() - + ") exceed configured Total Process Memory (" + totalProcessMemorySize.toHumanReadableString() + ")."); - } - final MemorySize totalFlinkMemorySize = totalProcessMemorySize.subtract(jvmMetaspaceAndOverhead.getTotalJvmMetaspaceAndOverheadSize()); - - // derive flink internal memory - - final FlinkInternalMemory flinkInternalMemory = deriveInternalMemoryFromTotalFlinkMemory(config, totalFlinkMemorySize); - - return createTaskExecutorProcessSpec(config, flinkInternalMemory, jvmMetaspaceAndOverhead); - } - - private static JvmMetaspaceAndOverhead deriveJvmMetaspaceAndOverheadFromTotalFlinkMemory( - final Configuration config, - final MemorySize totalFlinkMemorySize) { - final MemorySize jvmMetaspaceSize = getJvmMetaspaceSize(config); - final MemorySize totalFlinkAndJvmMetaspaceSize = totalFlinkMemorySize.add(jvmMetaspaceSize); - final JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead; - if (isTotalProcessMemorySizeExplicitlyConfigured(config)) { - final MemorySize totalProcessMemorySize = getTotalProcessMemorySize(config); - final MemorySize jvmOverheadSize = totalProcessMemorySize.subtract(totalFlinkAndJvmMetaspaceSize); - sanityCheckJvmOverhead(config, jvmOverheadSize, totalProcessMemorySize); - jvmMetaspaceAndOverhead = new JvmMetaspaceAndOverhead(jvmMetaspaceSize, jvmOverheadSize); - } else { - final MemorySize jvmOverheadSize = deriveJvmOverheadWithInverseFraction(config, totalFlinkAndJvmMetaspaceSize); - jvmMetaspaceAndOverhead = new JvmMetaspaceAndOverhead(jvmMetaspaceSize, jvmOverheadSize); - sanityCheckTotalProcessMemory(config, totalFlinkMemorySize, jvmMetaspaceAndOverhead); - } - return jvmMetaspaceAndOverhead; - } - - private static FlinkInternalMemory deriveInternalMemoryFromTotalFlinkMemory( - final Configuration config, - final MemorySize totalFlinkMemorySize) { - final MemorySize frameworkHeapMemorySize = getFrameworkHeapMemorySize(config); - final MemorySize frameworkOffHeapMemorySize = getFrameworkOffHeapMemorySize(config); - final MemorySize taskOffHeapMemorySize = getTaskOffHeapMemorySize(config); - - final MemorySize taskHeapMemorySize; - final MemorySize networkMemorySize; - final MemorySize managedMemorySize; - - if (isTaskHeapMemorySizeExplicitlyConfigured(config)) { - // task heap memory is configured, - // derive managed memory first, leave the remaining to network memory and check against network min/max - taskHeapMemorySize = getTaskHeapMemorySize(config); - managedMemorySize = deriveManagedMemoryAbsoluteOrWithFraction(config, totalFlinkMemorySize); - final MemorySize totalFlinkExcludeNetworkMemorySize = - frameworkHeapMemorySize.add(frameworkOffHeapMemorySize).add(taskHeapMemorySize).add(taskOffHeapMemorySize).add(managedMemorySize); - if (totalFlinkExcludeNetworkMemorySize.getBytes() > totalFlinkMemorySize.getBytes()) { - throw new IllegalConfigurationException( - "Sum of configured Framework Heap Memory (" + frameworkHeapMemorySize.toHumanReadableString() - + "), Framework Off-Heap Memory (" + frameworkOffHeapMemorySize.toHumanReadableString() - + "), Task Heap Memory (" + taskHeapMemorySize.toHumanReadableString() - + "), Task Off-Heap Memory (" + taskOffHeapMemorySize.toHumanReadableString() - + ") and Managed Memory (" + managedMemorySize.toHumanReadableString() - + ") exceed configured Total Flink Memory (" + totalFlinkMemorySize.toHumanReadableString() + ")."); - } - networkMemorySize = totalFlinkMemorySize.subtract(totalFlinkExcludeNetworkMemorySize); - sanityCheckNetworkMemoryWithExplicitlySetTotalFlinkAndHeapMemory(config, networkMemorySize, totalFlinkMemorySize); - } else { - // task heap memory is not configured - // derive managed memory and network memory, leave the remaining to task heap memory - managedMemorySize = deriveManagedMemoryAbsoluteOrWithFraction(config, totalFlinkMemorySize); - - if (isUsingLegacyNetworkConfigs(config)) { - networkMemorySize = getNetworkMemorySizeWithLegacyConfig(config); - } else { - networkMemorySize = deriveNetworkMemoryWithFraction(config, totalFlinkMemorySize); - } - final MemorySize totalFlinkExcludeTaskHeapMemorySize = - frameworkHeapMemorySize.add(frameworkOffHeapMemorySize).add(taskOffHeapMemorySize).add(managedMemorySize).add(networkMemorySize); - if (totalFlinkExcludeTaskHeapMemorySize.getBytes() > totalFlinkMemorySize.getBytes()) { - throw new IllegalConfigurationException( - "Sum of configured Framework Heap Memory (" + frameworkHeapMemorySize.toHumanReadableString() - + "), Framework Off-Heap Memory (" + frameworkOffHeapMemorySize.toHumanReadableString() - + "), Task Off-Heap Memory (" + taskOffHeapMemorySize.toHumanReadableString() - + "), Managed Memory (" + managedMemorySize.toHumanReadableString() - + ") and Network Memory (" + networkMemorySize.toHumanReadableString() - + ") exceed configured Total Flink Memory (" + totalFlinkMemorySize.toHumanReadableString() + ")."); - } - taskHeapMemorySize = totalFlinkMemorySize.subtract(totalFlinkExcludeTaskHeapMemorySize); - } - - final FlinkInternalMemory flinkInternalMemory = new FlinkInternalMemory( - frameworkHeapMemorySize, - frameworkOffHeapMemorySize, - taskHeapMemorySize, - taskOffHeapMemorySize, - networkMemorySize, - managedMemorySize); - sanityCheckTotalFlinkMemory(config, flinkInternalMemory); - - return flinkInternalMemory; - } - - private static MemorySize deriveManagedMemoryAbsoluteOrWithFraction(final Configuration config, final MemorySize base) { - if (isManagedMemorySizeExplicitlyConfigured(config)) { - return getManagedMemorySize(config); - } else { - return deriveWithFraction("managed memory", base, getManagedMemoryRangeFraction(config)); - } - } - - private static MemorySize deriveNetworkMemoryWithFraction(final Configuration config, final MemorySize base) { - return deriveWithFraction("network memory", base, getNetworkMemoryRangeFraction(config)); - } - - private static MemorySize deriveNetworkMemoryWithInverseFraction(final Configuration config, final MemorySize base) { - return deriveWithInverseFraction("network memory", base, getNetworkMemoryRangeFraction(config)); - } - - private static MemorySize deriveJvmOverheadWithFraction(final Configuration config, final MemorySize base) { - return deriveWithFraction("jvm overhead memory", base, getJvmOverheadRangeFraction(config)); - } - - private static MemorySize deriveJvmOverheadWithInverseFraction(final Configuration config, final MemorySize base) { - return deriveWithInverseFraction("jvm overhead memory", base, getJvmOverheadRangeFraction(config)); - } - - private static MemorySize deriveWithFraction( - final String memoryDescription, - final MemorySize base, - final RangeFraction rangeFraction) { - final MemorySize relative = base.multiply(rangeFraction.fraction); - return capToMinMax(memoryDescription, relative, rangeFraction); - } - - private static MemorySize deriveWithInverseFraction( - final String memoryDescription, - final MemorySize base, - final RangeFraction rangeFraction) { - checkArgument(rangeFraction.fraction < 1); - final MemorySize relative = base.multiply(rangeFraction.fraction / (1 - rangeFraction.fraction)); - return capToMinMax(memoryDescription, relative, rangeFraction); - } - - private static MemorySize capToMinMax( - final String memoryDescription, - final MemorySize relative, - final RangeFraction rangeFraction) { - long size = relative.getBytes(); - if (size > rangeFraction.maxSize.getBytes()) { - LOG.info( - "The derived from fraction {} ({}) is greater than its max value {}, max value will be used instead", - memoryDescription, - relative.toHumanReadableString(), - rangeFraction.maxSize.toHumanReadableString()); - size = rangeFraction.maxSize.getBytes(); - } else if (size < rangeFraction.minSize.getBytes()) { - LOG.info( - "The derived from fraction {} ({}) is less than its min value {}, min value will be used instead", - memoryDescription, - relative.toHumanReadableString(), - rangeFraction.minSize.toHumanReadableString()); - size = rangeFraction.minSize.getBytes(); - } - return new MemorySize(size); - } - - private static MemorySize getFrameworkHeapMemorySize(final Configuration config) { - return getMemorySizeFromConfig(config, TaskManagerOptions.FRAMEWORK_HEAP_MEMORY); - } - - private static MemorySize getFrameworkOffHeapMemorySize(final Configuration config) { - return getMemorySizeFromConfig(config, TaskManagerOptions.FRAMEWORK_OFF_HEAP_MEMORY); - } - - private static MemorySize getTaskHeapMemorySize(final Configuration config) { - checkArgument(isTaskHeapMemorySizeExplicitlyConfigured(config)); - return getMemorySizeFromConfig(config, TaskManagerOptions.TASK_HEAP_MEMORY); - } - - private static MemorySize getTaskOffHeapMemorySize(final Configuration config) { - return getMemorySizeFromConfig(config, TaskManagerOptions.TASK_OFF_HEAP_MEMORY); - } - - private static MemorySize getManagedMemorySize(final Configuration config) { - checkArgument(isManagedMemorySizeExplicitlyConfigured(config)); - return getMemorySizeFromConfig(config, TaskManagerOptions.MANAGED_MEMORY_SIZE); - } - - private static RangeFraction getManagedMemoryRangeFraction(final Configuration config) { - return getRangeFraction(MemorySize.ZERO, MemorySize.MAX_VALUE, TaskManagerOptions.MANAGED_MEMORY_FRACTION, config); - } - - private static MemorySize getNetworkMemorySizeWithLegacyConfig(final Configuration config) { - checkArgument(isUsingLegacyNetworkConfigs(config)); - @SuppressWarnings("deprecation") - final long numOfBuffers = config.getInteger(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS); - final long pageSize = ConfigurationParserUtils.getPageSize(config); - return new MemorySize(numOfBuffers * pageSize); - } - - private static RangeFraction getNetworkMemoryRangeFraction(final Configuration config) { - final MemorySize minSize = getMemorySizeFromConfig(config, TaskManagerOptions.NETWORK_MEMORY_MIN); - final MemorySize maxSize = getMemorySizeFromConfig(config, TaskManagerOptions.NETWORK_MEMORY_MAX); - return getRangeFraction(minSize, maxSize, TaskManagerOptions.NETWORK_MEMORY_FRACTION, config); - } - - private static MemorySize getJvmMetaspaceSize(final Configuration config) { - return getMemorySizeFromConfig(config, TaskManagerOptions.JVM_METASPACE); - } - - private static RangeFraction getJvmOverheadRangeFraction(final Configuration config) { - final MemorySize minSize = getMemorySizeFromConfig(config, TaskManagerOptions.JVM_OVERHEAD_MIN); - final MemorySize maxSize = getMemorySizeFromConfig(config, TaskManagerOptions.JVM_OVERHEAD_MAX); - return getRangeFraction(minSize, maxSize, TaskManagerOptions.JVM_OVERHEAD_FRACTION, config); - } - - private static RangeFraction getRangeFraction( - final MemorySize minSize, - final MemorySize maxSize, - ConfigOption fractionOption, - final Configuration config) { - final double fraction = config.getFloat(fractionOption); - try { - return new RangeFraction(minSize, maxSize, fraction); - } catch (IllegalArgumentException e) { - throw new IllegalConfigurationException( - String.format( - "Inconsistently configured %s (%s) and its min (%s), max (%s) value", - fractionOption, - fraction, - minSize.toHumanReadableString(), - maxSize.toHumanReadableString()), - e); - } - } - - private static MemorySize getTotalFlinkMemorySize(final Configuration config) { - checkArgument(isTotalFlinkMemorySizeExplicitlyConfigured(config)); - return getMemorySizeFromConfig(config, TaskManagerOptions.TOTAL_FLINK_MEMORY); - } - - private static MemorySize getTotalProcessMemorySize(final Configuration config) { - checkArgument(isTotalProcessMemorySizeExplicitlyConfigured(config)); - return getMemorySizeFromConfig(config, TaskManagerOptions.TOTAL_PROCESS_MEMORY); - } - - private static MemorySize getMemorySizeFromConfig(final Configuration config, final ConfigOption option) { - try { - return config.get(option); - } catch (Throwable t) { - throw new IllegalConfigurationException("Cannot read memory size from config option '" + option.key() + "'.", t); - } - } - - private static boolean isTaskHeapMemorySizeExplicitlyConfigured(final Configuration config) { - return config.contains(TaskManagerOptions.TASK_HEAP_MEMORY); - } - - public static boolean isManagedMemorySizeExplicitlyConfigured(final Configuration config) { - return config.contains(TaskManagerOptions.MANAGED_MEMORY_SIZE); - } - - private static boolean isUsingLegacyNetworkConfigs(final Configuration config) { - // use the legacy number-of-buffer config option only when it is explicitly configured and - // none of new config options is explicitly configured - @SuppressWarnings("deprecation") - final boolean legacyConfigured = config.contains(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS); - return !config.contains(TaskManagerOptions.NETWORK_MEMORY_MIN) && - !config.contains(TaskManagerOptions.NETWORK_MEMORY_MAX) && - !config.contains(TaskManagerOptions.NETWORK_MEMORY_FRACTION) && - legacyConfigured; - } - - private static boolean isNetworkMemoryFractionExplicitlyConfigured(final Configuration config) { - return config.contains(TaskManagerOptions.NETWORK_MEMORY_FRACTION); - } - - public static boolean isNetworkMemoryExplicitlyConfigured(final Configuration config) { - @SuppressWarnings("deprecation") - final boolean legacyConfigured = config.contains(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS); - return config.contains(TaskManagerOptions.NETWORK_MEMORY_MAX) || - config.contains(TaskManagerOptions.NETWORK_MEMORY_MIN) || - config.contains(TaskManagerOptions.NETWORK_MEMORY_FRACTION) || - legacyConfigured; - } - - private static boolean isJvmOverheadFractionExplicitlyConfigured(final Configuration config) { - return config.contains(TaskManagerOptions.JVM_OVERHEAD_FRACTION); - } - - private static boolean isTotalFlinkMemorySizeExplicitlyConfigured(final Configuration config) { - return config.contains(TaskManagerOptions.TOTAL_FLINK_MEMORY); - } - - private static boolean isTotalProcessMemorySizeExplicitlyConfigured(final Configuration config) { - return config.contains(TaskManagerOptions.TOTAL_PROCESS_MEMORY); - } - - private static void sanityCheckTotalFlinkMemory(final Configuration config, final FlinkInternalMemory flinkInternalMemory) { - if (isTotalFlinkMemorySizeExplicitlyConfigured(config)) { - final MemorySize configuredTotalFlinkMemorySize = getTotalFlinkMemorySize(config); - if (!configuredTotalFlinkMemorySize.equals(flinkInternalMemory.getTotalFlinkMemorySize())) { - throw new IllegalConfigurationException( - "Configured/Derived Flink internal memory sizes (total " + flinkInternalMemory.getTotalFlinkMemorySize().toHumanReadableString() - + ") do not add up to the configured Total Flink Memory size (" + configuredTotalFlinkMemorySize.toHumanReadableString() - + "). Configured/Derived Flink internal memory sizes are: " - + "Framework Heap Memory (" + flinkInternalMemory.frameworkHeap.toHumanReadableString() - + "), Framework Off-Heap Memory (" + flinkInternalMemory.frameworkOffHeap.toHumanReadableString() - + "), Task Heap Memory (" + flinkInternalMemory.taskHeap.toHumanReadableString() - + "), Task Off-Heap Memory (" + flinkInternalMemory.taskOffHeap.toHumanReadableString() - + "), Network Memory (" + flinkInternalMemory.network.toHumanReadableString() - + "), Managed Memory (" + flinkInternalMemory.managed.toHumanReadableString() + ")."); - } - } - } - - private static void sanityCheckTotalProcessMemory( - final Configuration config, - final MemorySize totalFlinkMemory, - final JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead) { - final MemorySize derivedTotalProcessMemorySize = - totalFlinkMemory.add(jvmMetaspaceAndOverhead.metaspace).add(jvmMetaspaceAndOverhead.overhead); - if (isTotalProcessMemorySizeExplicitlyConfigured(config)) { - final MemorySize configuredTotalProcessMemorySize = getTotalProcessMemorySize(config); - if (!configuredTotalProcessMemorySize.equals(derivedTotalProcessMemorySize)) { - throw new IllegalConfigurationException( - "Configured/Derived memory sizes (total " + derivedTotalProcessMemorySize.toHumanReadableString() - + ") do not add up to the configured Total Process Memory size (" + configuredTotalProcessMemorySize.toHumanReadableString() - + "). Configured/Derived memory sizes are: " - + "Total Flink Memory (" + totalFlinkMemory.toHumanReadableString() - + "), JVM Metaspace (" + jvmMetaspaceAndOverhead.metaspace.toHumanReadableString() - + "), JVM Overhead (" + jvmMetaspaceAndOverhead.overhead.toHumanReadableString() + ")."); - } - } - } - - private static void sanityCheckNetworkMemoryWithExplicitlySetTotalFlinkAndHeapMemory( - final Configuration config, - final MemorySize derivedNetworkMemorySize, - final MemorySize totalFlinkMemorySize) { - try { - sanityCheckNetworkMemory(config, derivedNetworkMemorySize, totalFlinkMemorySize); - } catch (IllegalConfigurationException e) { - throw new IllegalConfigurationException( - "If Total Flink, Task Heap and (or) Managed Memory sizes are explicitly configured then " + - "the Network Memory size is the rest of the Total Flink memory after subtracting all other " + - "configured types of memory, but the derived Network Memory is inconsistent with its configuration.", - e); - } + return createMemoryProcessSpec(config, PROCESS_MEMORY_UTILS.memoryProcessSpecFromConfig(config)); } - private static void sanityCheckNetworkMemory( + private static TaskExecutorProcessSpec createMemoryProcessSpec( final Configuration config, - final MemorySize derivedNetworkMemorySize, - final MemorySize totalFlinkMemorySize) { - if (isUsingLegacyNetworkConfigs(config)) { - final MemorySize configuredNetworkMemorySize = getNetworkMemorySizeWithLegacyConfig(config); - if (!configuredNetworkMemorySize.equals(derivedNetworkMemorySize)) { - throw new IllegalConfigurationException( - "Derived Network Memory size (" + derivedNetworkMemorySize.toHumanReadableString() - + ") does not match configured Network Memory size (" + configuredNetworkMemorySize.toHumanReadableString() + ")."); - } - } else { - final RangeFraction networkRangeFraction = getNetworkMemoryRangeFraction(config); - if (derivedNetworkMemorySize.getBytes() > networkRangeFraction.maxSize.getBytes() || - derivedNetworkMemorySize.getBytes() < networkRangeFraction.minSize.getBytes()) { - throw new IllegalConfigurationException("Derived Network Memory size (" - + derivedNetworkMemorySize.toHumanReadableString() + ") is not in configured Network Memory range [" - + networkRangeFraction.minSize.toHumanReadableString() + ", " - + networkRangeFraction.maxSize.toHumanReadableString() + "]."); - } - if (isNetworkMemoryFractionExplicitlyConfigured(config) && - !derivedNetworkMemorySize.equals(totalFlinkMemorySize.multiply(networkRangeFraction.fraction))) { - LOG.info( - "The derived Network Memory size ({}) does not match " + - "the configured Network Memory fraction ({}) from the configured Total Flink Memory size ({}). " + - "The derived Network Memory size will be used.", - derivedNetworkMemorySize.toHumanReadableString(), - networkRangeFraction.fraction, - totalFlinkMemorySize.toHumanReadableString()); - } - } - } - - private static void sanityCheckJvmOverhead( - final Configuration config, - final MemorySize derivedJvmOverheadSize, - final MemorySize totalProcessMemorySize) { - final RangeFraction jvmOverheadRangeFraction = getJvmOverheadRangeFraction(config); - if (derivedJvmOverheadSize.getBytes() > jvmOverheadRangeFraction.maxSize.getBytes() || - derivedJvmOverheadSize.getBytes() < jvmOverheadRangeFraction.minSize.getBytes()) { - throw new IllegalConfigurationException("Derived JVM Overhead size (" - + derivedJvmOverheadSize.toHumanReadableString() + ") is not in configured JVM Overhead range [" - + jvmOverheadRangeFraction.minSize.toHumanReadableString() + ", " - + jvmOverheadRangeFraction.maxSize.toHumanReadableString() + "]."); - } - if (isJvmOverheadFractionExplicitlyConfigured(config) && - !derivedJvmOverheadSize.equals(totalProcessMemorySize.multiply(jvmOverheadRangeFraction.fraction))) { - LOG.info( - "The derived JVM Overhead size ({}) does not match " + - "the configured JVM Overhead fraction ({}) from the configured Total Process Memory size ({}). " + - "The derived JVM OVerhead size will be used.", - derivedJvmOverheadSize.toHumanReadableString(), - jvmOverheadRangeFraction.fraction, - totalProcessMemorySize.toHumanReadableString()); - } + final CommonProcessMemorySpec processMemory) { + TaskExecutorFlinkMemory flinkMemory = processMemory.getFlinkMemory(); + JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead = processMemory.getJvmMetaspaceAndOverhead(); + return new TaskExecutorProcessSpec(getCpuCores(config), flinkMemory, jvmMetaspaceAndOverhead); } private static CPUResource getCpuCores(final Configuration config) { @@ -636,7 +149,7 @@ public class TaskExecutorProcessUtils { public static double getCpuCoresWithFallbackConfigOption(final Configuration config, ConfigOption fallbackOption) { double fallbackValue = config.getDouble(fallbackOption); - return TaskExecutorProcessUtils.getCpuCoresWithFallback(config, fallbackValue).getValue().doubleValue(); + return getCpuCoresWithFallback(config, fallbackValue).getValue().doubleValue(); } public static CPUResource getCpuCoresWithFallback(final Configuration config, double fallback) { @@ -659,124 +172,9 @@ public class TaskExecutorProcessUtils { return new CPUResource(cpuCores); } - private static TaskExecutorProcessSpec createTaskExecutorProcessSpec( - final Configuration config, - final FlinkInternalMemory flinkInternalMemory, - final JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead) { - return new TaskExecutorProcessSpec( - getCpuCores(config), - flinkInternalMemory.frameworkHeap, - flinkInternalMemory.frameworkOffHeap, - flinkInternalMemory.taskHeap, - flinkInternalMemory.taskOffHeap, - flinkInternalMemory.network, - flinkInternalMemory.managed, - jvmMetaspaceAndOverhead.metaspace, - jvmMetaspaceAndOverhead.overhead); - } - - private static class RangeFraction { - final MemorySize minSize; - final MemorySize maxSize; - final double fraction; - - RangeFraction(final MemorySize minSize, final MemorySize maxSize, final double fraction) { - this.minSize = minSize; - this.maxSize = maxSize; - this.fraction = fraction; - checkArgument(minSize.getBytes() <= maxSize.getBytes(), "min value must be less or equal to max value"); - checkArgument(fraction >= 0 && fraction < 1, "fraction must be in range [0, 1)"); - } - } - - private static class FlinkInternalMemory { - final MemorySize frameworkHeap; - final MemorySize frameworkOffHeap; - final MemorySize taskHeap; - final MemorySize taskOffHeap; - final MemorySize network; - final MemorySize managed; - - FlinkInternalMemory( - final MemorySize frameworkHeap, - final MemorySize frameworkOffHeap, - final MemorySize taskHeap, - final MemorySize taskOffHeap, - final MemorySize network, - final MemorySize managed) { - - this.frameworkHeap = checkNotNull(frameworkHeap); - this.frameworkOffHeap = checkNotNull(frameworkOffHeap); - this.taskHeap = checkNotNull(taskHeap); - this.taskOffHeap = checkNotNull(taskOffHeap); - this.network = checkNotNull(network); - this.managed = checkNotNull(managed); - } - - MemorySize getTotalFlinkMemorySize() { - return frameworkHeap.add(frameworkOffHeap).add(taskHeap).add(taskOffHeap).add(network).add(managed); - } - } - - private static class JvmMetaspaceAndOverhead { - final MemorySize metaspace; - final MemorySize overhead; - - JvmMetaspaceAndOverhead(final MemorySize jvmMetaspace, final MemorySize jvmOverhead) { - this.metaspace = checkNotNull(jvmMetaspace); - this.overhead = checkNotNull(jvmOverhead); - } - - MemorySize getTotalJvmMetaspaceAndOverheadSize() { - return metaspace.add(overhead); - } - } - public static Configuration getConfigurationMapLegacyTaskManagerHeapSizeToConfigOption( - final Configuration configuration, ConfigOption configOption) { - - if (configuration.contains(configOption)) { - return configuration; - } - - return getLegacyTaskManagerHeapMemoryIfExplicitlyConfigured(configuration).map(legacyHeapSize -> { - final Configuration copiedConfig = new Configuration(configuration); - copiedConfig.set(configOption, legacyHeapSize); - - LOG.info( - "'{}' is not specified, use the configured deprecated task manager heap value ({}) for it.", - configOption.key(), - legacyHeapSize.toHumanReadableString()); - - return copiedConfig; - }).orElse(configuration); - } - - @SuppressWarnings("deprecation") - private static Optional getLegacyTaskManagerHeapMemoryIfExplicitlyConfigured(final Configuration configuration) { - String totalProcessEnv = System.getenv("FLINK_TM_HEAP"); - if (totalProcessEnv != null) { - try { - return Optional.of(MemorySize.parse(totalProcessEnv)); - } catch (Throwable t) { - throw new IllegalConfigurationException("Cannot read total process memory size from environment variable value " - + totalProcessEnv + ".", t); - } - } - - if (configuration.contains(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY)) { - return Optional.of(getMemorySizeFromConfig(configuration, TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY)); - } - - if (configuration.contains(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY_MB)) { - final long legacyHeapMemoryMB = configuration.getInteger(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY_MB); - if (legacyHeapMemoryMB < 0) { - throw new IllegalConfigurationException("Configured total process memory size (" - + legacyHeapMemoryMB + "MB) must not be less than 0."); - } - return Optional.of(new MemorySize(legacyHeapMemoryMB << 20)); // megabytes to bytes; - } - - return Optional.empty(); + final Configuration configuration, + final ConfigOption configOption) { + return LEGACY_MEMORY_UTILS.getConfWithLegacyHeapSizeMappedToNewConfigOption(configuration, configOption); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManagerProcessSpec.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManagerProcessSpec.java new file mode 100644 index 0000000000..25d90cd2b9 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManagerProcessSpec.java @@ -0,0 +1,73 @@ +/* + * 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.runtime.jobmanager; + +import org.apache.flink.runtime.util.config.memory.CommonProcessMemorySpec; +import org.apache.flink.runtime.util.config.memory.JvmMetaspaceAndOverhead; +import org.apache.flink.runtime.util.config.memory.jobmanager.JobManagerFlinkMemory; + +/** + * Describe the specifics of different resource dimensions of the JobManager process. + * + *

    A JobManager's memory consists of the following components: + *

      + *
    • JVM Heap Memory
    • + *
    • Off-heap Memory
    • + *
    • JVM Metaspace
    • + *
    • JVM Overhead
    • + *
    + * We use Total Process Memory to refer to all the memory components, while Total Flink Memory refering to all + * the components except JVM Metaspace and JVM Overhead. + * + *

    The relationships of JobManager memory components are shown below. + *

    + *               ┌ ─ ─ Total Process Memory  ─ ─ ┐
    + *                ┌ ─ ─ Total Flink Memory  ─ ─ ┐
    + *               │ ┌───────────────────────────┐ │
    + *  On-Heap ----- ││      JVM Heap Memory      ││
    + *               │ └───────────────────────────┘ │
    + *               │ ┌───────────────────────────┐ │
    + *            ┌─  ││       Off-heap Memory     ││
    + *            │  │ └───────────────────────────┘ │
    + *            │   └ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┘
    + *            │  │┌─────────────────────────────┐│
    + *  Off-Heap ─|   │        JVM Metaspace        │
    + *            │  │└─────────────────────────────┘│
    + *            │   ┌─────────────────────────────┐
    + *            └─ ││        JVM Overhead         ││
    + *                └─────────────────────────────┘
    + *               └ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┘
    + * 
    + */ +public class JobManagerProcessSpec extends CommonProcessMemorySpec { + private static final long serialVersionUID = 1L; + + JobManagerProcessSpec(JobManagerFlinkMemory flinkMemory, JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead) { + super(flinkMemory, jvmMetaspaceAndOverhead); + } + + @Override + public String toString() { + return "JobManagerProcessSpec {" + + "jvmHeapSize=" + getJvmHeapMemorySize().toHumanReadableString() + ", " + + "offHeapSize=" + getJvmDirectMemorySize().toHumanReadableString() + ", " + + "jvmMetaspaceSize=" + getJvmMetaspaceSize().toHumanReadableString() + ", " + + "jvmOverheadSize=" + getJvmOverheadSize().toHumanReadableString() + '}'; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManagerProcessUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManagerProcessUtils.java new file mode 100644 index 0000000000..62f61bf7e4 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManagerProcessUtils.java @@ -0,0 +1,81 @@ +/* + * 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.runtime.jobmanager; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.runtime.util.config.memory.CommonProcessMemorySpec; +import org.apache.flink.runtime.util.config.memory.JvmMetaspaceAndOverheadOptions; +import org.apache.flink.runtime.util.config.memory.MemoryBackwardsCompatibilityUtils; +import org.apache.flink.runtime.util.config.memory.LegacyMemoryOptions; +import org.apache.flink.runtime.util.config.memory.ProcessMemoryOptions; +import org.apache.flink.runtime.util.config.memory.ProcessMemoryUtils; +import org.apache.flink.runtime.util.config.memory.jobmanager.JobManagerFlinkMemory; +import org.apache.flink.runtime.util.config.memory.jobmanager.JobManagerFlinkMemoryUtils; + +import java.util.Collections; + +/** + * JobManager utils to calculate {@link JobManagerProcessSpec} and JVM args. + */ +public class JobManagerProcessUtils { + + static final ProcessMemoryOptions JM_PROCESS_MEMORY_OPTIONS = new ProcessMemoryOptions( + Collections.singletonList(JobManagerOptions.JVM_HEAP_MEMORY), + JobManagerOptions.TOTAL_FLINK_MEMORY, + JobManagerOptions.TOTAL_PROCESS_MEMORY, + new JvmMetaspaceAndOverheadOptions( + JobManagerOptions.JVM_METASPACE, + JobManagerOptions.JVM_OVERHEAD_MIN, + JobManagerOptions.JVM_OVERHEAD_MAX, + JobManagerOptions.JVM_OVERHEAD_FRACTION)); + + @SuppressWarnings("deprecation") + static final LegacyMemoryOptions JM_LEGACY_HEAP_OPTIONS = + new LegacyMemoryOptions( + "FLINK_JM_HEAP", + JobManagerOptions.JOB_MANAGER_HEAP_MEMORY, + JobManagerOptions.JOB_MANAGER_HEAP_MEMORY_MB); + + private static final ProcessMemoryUtils PROCESS_MEMORY_UTILS = new ProcessMemoryUtils<>( + JM_PROCESS_MEMORY_OPTIONS, + new JobManagerFlinkMemoryUtils()); + + private static final MemoryBackwardsCompatibilityUtils LEGACY_MEMORY_UTILS = new MemoryBackwardsCompatibilityUtils(JM_LEGACY_HEAP_OPTIONS); + + private JobManagerProcessUtils() { + } + + public static JobManagerProcessSpec processSpecFromConfig(Configuration config) { + return createMemoryProcessSpec(PROCESS_MEMORY_UTILS.memoryProcessSpecFromConfig(config)); + } + + private static JobManagerProcessSpec createMemoryProcessSpec( + CommonProcessMemorySpec processMemory) { + return new JobManagerProcessSpec(processMemory.getFlinkMemory(), processMemory.getJvmMetaspaceAndOverhead()); + } + + public static Configuration getConfigurationWithLegacyHeapSizeMappedToNewConfigOption( + Configuration configuration, + ConfigOption configOption) { + return LEGACY_MEMORY_UTILS.getConfWithLegacyHeapSizeMappedToNewConfigOption(configuration, configOption); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/BashJavaUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/BashJavaUtils.java index a0db33d7d1..27b3c50aa6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/BashJavaUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/BashJavaUtils.java @@ -23,6 +23,7 @@ import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec; import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils; import org.apache.flink.runtime.taskexecutor.TaskManagerRunner; +import org.apache.flink.runtime.util.config.memory.ProcessMemoryUtils; import java.util.Arrays; @@ -55,7 +56,7 @@ public class BashJavaUtils { private static void getTmResourceParams(String[] args) throws Exception { Configuration configuration = getConfigurationForStandaloneTaskManagers(args); TaskExecutorProcessSpec taskExecutorProcessSpec = TaskExecutorProcessUtils.processSpecFromConfig(configuration); - System.out.println(EXECUTION_PREFIX + TaskExecutorProcessUtils.generateJvmParametersStr(taskExecutorProcessSpec)); + System.out.println(EXECUTION_PREFIX + ProcessMemoryUtils.generateJvmParametersStr(taskExecutorProcessSpec)); System.out.println(EXECUTION_PREFIX + TaskExecutorProcessUtils.generateDynamicConfigsStr(taskExecutorProcessSpec)); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/CommonProcessMemorySpec.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/CommonProcessMemorySpec.java new file mode 100644 index 0000000000..2d7de0ad6e --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/CommonProcessMemorySpec.java @@ -0,0 +1,97 @@ +/* + * 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.runtime.util.config.memory; + +import org.apache.flink.configuration.MemorySize; + +/** + * Common memory components of Flink processes (e.g. JM or TM). + * + *

    The process memory consists of the following components. + *

      + *
    • Total Flink Memory
    • + *
    • JVM Metaspace
    • + *
    • JVM Overhead
    • + *
    + * Among all the components, We use the Total Process Memory to refer to all the memory components, + * while the Total Flink Memory refers to all the internal components except JVM Metaspace and JVM Overhead. + * The internal components of Total Flink Memory, represented by {@link FlinkMemory}, are specific to concrete + * Flink process (e.g. JM or TM). + * + *

    The relationships of process memory components are shown below. + *

    + *               ┌ ─ ─ Total Process Memory  ─ ─ ┐
    + *               │┌─────────────────────────────┐│
    + *                │      Total Flink Memory     │
    + *               │└─────────────────────────────┘│
    + *               │┌─────────────────────────────┐│
    + *                │        JVM Metaspace        │
    + *               │└─────────────────────────────┘│
    + *                ┌─────────────────────────────┐
    + *               ││        JVM Overhead         ││
    + *                └─────────────────────────────┘
    + *               └ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┘
    + * 
    + */ +public class CommonProcessMemorySpec implements ProcessMemorySpec { + private static final long serialVersionUID = 1L; + + private final FM flinkMemory; + private final JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead; + + protected CommonProcessMemorySpec(FM flinkMemory, JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead) { + this.flinkMemory = flinkMemory; + this.jvmMetaspaceAndOverhead = jvmMetaspaceAndOverhead; + } + + public FM getFlinkMemory() { + return flinkMemory; + } + + public JvmMetaspaceAndOverhead getJvmMetaspaceAndOverhead() { + return jvmMetaspaceAndOverhead; + } + + @Override + public MemorySize getJvmHeapMemorySize() { + return flinkMemory.getJvmHeapMemorySize(); + } + + @Override + public MemorySize getJvmDirectMemorySize() { + return flinkMemory.getJvmDirectMemorySize(); + } + + public MemorySize getJvmMetaspaceSize() { + return getJvmMetaspaceAndOverhead().getMetaspace(); + } + + public MemorySize getJvmOverheadSize() { + return getJvmMetaspaceAndOverhead().getOverhead(); + } + + @Override + public MemorySize getTotalFlinkMemorySize() { + return flinkMemory.getTotalFlinkMemorySize(); + } + + public MemorySize getTotalProcessMemorySize() { + return flinkMemory.getTotalFlinkMemorySize().add(getJvmMetaspaceSize()).add(getJvmOverheadSize()); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/FlinkMemory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/FlinkMemory.java new file mode 100644 index 0000000000..976665dad7 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/FlinkMemory.java @@ -0,0 +1,34 @@ +/* + * 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.runtime.util.config.memory; + +import org.apache.flink.configuration.MemorySize; + +import java.io.Serializable; + +/** + * Memory components which constitute the Total Flink Memory. + */ +public interface FlinkMemory extends Serializable { + MemorySize getJvmHeapMemorySize(); + + MemorySize getJvmDirectMemorySize(); + + MemorySize getTotalFlinkMemorySize(); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/FlinkMemoryUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/FlinkMemoryUtils.java new file mode 100644 index 0000000000..28f45a1ae7 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/FlinkMemoryUtils.java @@ -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.flink.runtime.util.config.memory; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.MemorySize; + +/** + * Utility to derive Total Flink Memory from its required memory components and back. + * @param the FLink memory components + */ +public interface FlinkMemoryUtils { + FM deriveFromRequiredFineGrainedOptions(Configuration config); + + FM deriveFromTotalFlinkMemory(Configuration config, MemorySize totalFlinkMemorySize); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/JvmMetaspaceAndOverhead.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/JvmMetaspaceAndOverhead.java new file mode 100644 index 0000000000..cdfffe209e --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/JvmMetaspaceAndOverhead.java @@ -0,0 +1,52 @@ +/* + * 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.runtime.util.config.memory; + +import org.apache.flink.configuration.MemorySize; + +import java.io.Serializable; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * JVM metaspace and overhead memory sizes. + */ +public class JvmMetaspaceAndOverhead implements Serializable { + private static final long serialVersionUID = 1L; + + private final MemorySize metaspace; + private final MemorySize overhead; + + public JvmMetaspaceAndOverhead(MemorySize jvmMetaspace, MemorySize jvmOverhead) { + this.metaspace = checkNotNull(jvmMetaspace); + this.overhead = checkNotNull(jvmOverhead); + } + + MemorySize getTotalJvmMetaspaceAndOverheadSize() { + return getMetaspace().add(getOverhead()); + } + + public MemorySize getMetaspace() { + return metaspace; + } + + public MemorySize getOverhead() { + return overhead; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/JvmMetaspaceAndOverheadOptions.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/JvmMetaspaceAndOverheadOptions.java new file mode 100644 index 0000000000..c6b6bee0e3 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/JvmMetaspaceAndOverheadOptions.java @@ -0,0 +1,59 @@ +/* + * 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.runtime.util.config.memory; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.MemorySize; + +/** + * Options to calculate JVM Metaspace and Overhead. + */ +public class JvmMetaspaceAndOverheadOptions { + private final ConfigOption jvmMetaspaceOption; + private final ConfigOption jvmOverheadMin; + private final ConfigOption jvmOverheadMax; + private final ConfigOption jvmOverheadFraction; + + public JvmMetaspaceAndOverheadOptions( + ConfigOption jvmMetaspaceOption, + ConfigOption jvmOverheadMin, + ConfigOption jvmOverheadMax, + ConfigOption jvmOverheadFraction) { + this.jvmMetaspaceOption = jvmMetaspaceOption; + this.jvmOverheadMin = jvmOverheadMin; + this.jvmOverheadMax = jvmOverheadMax; + this.jvmOverheadFraction = jvmOverheadFraction; + } + + ConfigOption getJvmMetaspaceOption() { + return jvmMetaspaceOption; + } + + ConfigOption getJvmOverheadMin() { + return jvmOverheadMin; + } + + ConfigOption getJvmOverheadMax() { + return jvmOverheadMax; + } + + ConfigOption getJvmOverheadFraction() { + return jvmOverheadFraction; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/LegacyMemoryOptions.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/LegacyMemoryOptions.java new file mode 100644 index 0000000000..86164eb83d --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/LegacyMemoryOptions.java @@ -0,0 +1,52 @@ +/* + * 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.runtime.util.config.memory; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.MemorySize; + +/** + * Legacy JVM heap/process memory options. + * + *

    If these memory options are set, they are interpreted as other new memory options for the backwards compatibility + * in {@link MemoryBackwardsCompatibilityUtils}. + */ +public class LegacyMemoryOptions { + private final String envVar; + private final ConfigOption heap; + private final ConfigOption heapMb; + + public LegacyMemoryOptions(String envVar, ConfigOption heap, ConfigOption heapMb) { + this.envVar = envVar; + this.heap = heap; + this.heapMb = heapMb; + } + + String getEnvVar() { + return envVar; + } + + ConfigOption getHeap() { + return heap; + } + + ConfigOption getHeapMb() { + return heapMb; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/MemoryBackwardsCompatibilityUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/MemoryBackwardsCompatibilityUtils.java new file mode 100644 index 0000000000..f334922b0d --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/MemoryBackwardsCompatibilityUtils.java @@ -0,0 +1,94 @@ +/* + * 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.runtime.util.config.memory; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.configuration.MemorySize; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Optional; + +/** + * Utilities to fallback to new options from the legacy ones for the backwards compatibility. + * + *

    If {@link LegacyMemoryOptions} are set, they are interpreted as other new memory options for the backwards + * compatibility. + */ +public class MemoryBackwardsCompatibilityUtils { + private static final Logger LOG = LoggerFactory.getLogger(MemoryBackwardsCompatibilityUtils.class); + + private final LegacyMemoryOptions legacyMemoryOptions; + + public MemoryBackwardsCompatibilityUtils(LegacyMemoryOptions legacyMemoryOptions) { + this.legacyMemoryOptions = legacyMemoryOptions; + } + + public Configuration getConfWithLegacyHeapSizeMappedToNewConfigOption( + Configuration configuration, + ConfigOption configOption) { + if (configuration.contains(configOption)) { + return configuration; + } + return getLegacyTaskManagerHeapMemoryIfExplicitlyConfigured(configuration) + .map(legacyHeapSize -> { + Configuration copiedConfig = new Configuration(configuration); + copiedConfig.set(configOption, legacyHeapSize); + LOG.info( + "'{}' is not specified, use the configured deprecated task manager heap value ({}) for it.", + configOption.key(), + legacyHeapSize.toHumanReadableString()); + return copiedConfig; + }).orElse(configuration); + } + + private Optional getLegacyTaskManagerHeapMemoryIfExplicitlyConfigured(Configuration configuration) { + @SuppressWarnings("CallToSystemGetenv") + String totalProcessEnv = System.getenv(legacyMemoryOptions.getEnvVar()); + if (totalProcessEnv != null) { + //noinspection OverlyBroadCatchBlock + try { + return Optional.of(MemorySize.parse(totalProcessEnv)); + } catch (Throwable t) { + throw new IllegalConfigurationException( + "Cannot read total process memory size from environment variable value " + totalProcessEnv + '.', + t); + } + } + + if (configuration.contains(legacyMemoryOptions.getHeap())) { + return Optional.of(ProcessMemoryUtils.getMemorySizeFromConfig(configuration, legacyMemoryOptions.getHeap())); + } + + if (configuration.contains(legacyMemoryOptions.getHeapMb())) { + final long legacyHeapMemoryMB = configuration.getInteger(legacyMemoryOptions.getHeapMb()); + if (legacyHeapMemoryMB < 0) { + throw new IllegalConfigurationException( + "Configured total process memory size (" + legacyHeapMemoryMB + "MB) must not be less than 0."); + } + //noinspection MagicNumber + return Optional.of(new MemorySize(legacyHeapMemoryMB << 20)); // megabytes to bytes; + } + + return Optional.empty(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/ProcessMemoryOptions.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/ProcessMemoryOptions.java new file mode 100644 index 0000000000..6ce1dfede5 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/ProcessMemoryOptions.java @@ -0,0 +1,65 @@ +/* + * 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.runtime.util.config.memory; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.MemorySize; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Common Flink's options to describe its JVM process memory configuration for JM or TM. + */ +public class ProcessMemoryOptions { + private final List> requiredFineGrainedOptions; + private final ConfigOption totalFlinkMemoryOption; + private final ConfigOption totalProcessMemoryOption; + private final JvmMetaspaceAndOverheadOptions jvmOptions; + + public ProcessMemoryOptions( + List> requiredFineGrainedOptions, + ConfigOption totalFlinkMemoryOption, + ConfigOption totalProcessMemoryOption, + JvmMetaspaceAndOverheadOptions jvmOptions) { + this.requiredFineGrainedOptions = new ArrayList<>(checkNotNull(requiredFineGrainedOptions)); + this.totalFlinkMemoryOption = checkNotNull(totalFlinkMemoryOption); + this.totalProcessMemoryOption = checkNotNull(totalProcessMemoryOption); + this.jvmOptions = checkNotNull(jvmOptions); + } + + List> getRequiredFineGrainedOptions() { + return Collections.unmodifiableList(requiredFineGrainedOptions); + } + + ConfigOption getTotalFlinkMemoryOption() { + return totalFlinkMemoryOption; + } + + ConfigOption getTotalProcessMemoryOption() { + return totalProcessMemoryOption; + } + + JvmMetaspaceAndOverheadOptions getJvmOptions() { + return jvmOptions; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/ProcessMemorySpec.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/ProcessMemorySpec.java new file mode 100644 index 0000000000..592b6daf20 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/ProcessMemorySpec.java @@ -0,0 +1,40 @@ +/* + * 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.runtime.util.config.memory; + +import org.apache.flink.configuration.MemorySize; + +import java.io.Serializable; + +/** + * Common interface for Flink JVM process memory components. + */ +public interface ProcessMemorySpec extends Serializable { + MemorySize getJvmHeapMemorySize(); + + MemorySize getJvmDirectMemorySize(); + + MemorySize getJvmMetaspaceSize(); + + MemorySize getJvmOverheadSize(); + + MemorySize getTotalFlinkMemorySize(); + + MemorySize getTotalProcessMemorySize(); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/ProcessMemoryUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/ProcessMemoryUtils.java new file mode 100644 index 0000000000..e84429700c --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/ProcessMemoryUtils.java @@ -0,0 +1,281 @@ +/* + * 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.runtime.util.config.memory; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Common utils to parse JVM process memory configuration for JM or TM. + * + *

    The utility calculates all common process memory components from {@link CommonProcessMemorySpec}. + * + *

    It is required to configure at least one subset of the following options and recommended to configure only one: + *

      + *
    • {@link ProcessMemoryOptions#getRequiredFineGrainedOptions()}
    • + *
    • {@link ProcessMemoryOptions#getTotalFlinkMemoryOption()}
    • + *
    • {@link ProcessMemoryOptions#getTotalProcessMemoryOption()}
    • + *
    + * Otherwise the calculation fails. + * + *

    The utility derives the Total Process Memory from the Total Flink Memory and JVM components and back. + * To perform the calculations, it uses the provided {@link ProcessMemoryOptions} which are different for different + * Flink processes: JM/TM. + * + *

    The utility also calls the provided FlinkMemoryUtils to derive {@link FlinkMemory} components from + * {@link ProcessMemoryOptions#getRequiredFineGrainedOptions()} or from the Total Flink memory. The concrete + * {@link FlinkMemoryUtils} is implemented for the respective processes: JM/TM, according to the specific + * structure of their {@link FlinkMemory}. + * + * @param the FLink memory component structure + */ +public class ProcessMemoryUtils { + private static final Logger LOG = LoggerFactory.getLogger(ProcessMemoryUtils.class); + + private final ProcessMemoryOptions options; + private final FlinkMemoryUtils flinkMemoryUtils; + + public ProcessMemoryUtils(ProcessMemoryOptions options, FlinkMemoryUtils flinkMemoryUtils) { + this.options = checkNotNull(options); + this.flinkMemoryUtils = checkNotNull(flinkMemoryUtils); + } + + public CommonProcessMemorySpec memoryProcessSpecFromConfig(Configuration config) { + if (options.getRequiredFineGrainedOptions().stream().allMatch(config::contains)) { + // all internal memory options are configured, use these to derive total Flink and process memory + return deriveProcessSpecWithExplicitInternalMemory(config); + } else if (config.contains(options.getTotalFlinkMemoryOption())) { + // internal memory options are not configured, total Flink memory is configured, + // derive from total flink memory + return deriveProcessSpecWithTotalFlinkMemory(config); + } else if (config.contains(options.getTotalProcessMemoryOption())) { + // total Flink memory is not configured, total process memory is configured, + // derive from total process memory + return deriveProcessSpecWithTotalProcessMemory(config); + } + return failBecauseRequiredOptionsNotConfigured(); + } + + private CommonProcessMemorySpec deriveProcessSpecWithExplicitInternalMemory(Configuration config) { + FM flinkInternalMemory = flinkMemoryUtils.deriveFromRequiredFineGrainedOptions(config); + JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead = deriveJvmMetaspaceAndOverheadFromTotalFlinkMemory( + config, + flinkInternalMemory.getTotalFlinkMemorySize()); + return new CommonProcessMemorySpec<>(flinkInternalMemory, jvmMetaspaceAndOverhead); + } + + private CommonProcessMemorySpec deriveProcessSpecWithTotalFlinkMemory(Configuration config) { + MemorySize totalFlinkMemorySize = getMemorySizeFromConfig(config, options.getTotalFlinkMemoryOption()); + FM flinkInternalMemory = flinkMemoryUtils.deriveFromTotalFlinkMemory(config, totalFlinkMemorySize); + JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead = deriveJvmMetaspaceAndOverheadFromTotalFlinkMemory(config, totalFlinkMemorySize); + return new CommonProcessMemorySpec<>(flinkInternalMemory, jvmMetaspaceAndOverhead); + } + + private CommonProcessMemorySpec deriveProcessSpecWithTotalProcessMemory(Configuration config) { + MemorySize totalProcessMemorySize = getMemorySizeFromConfig(config, options.getTotalProcessMemoryOption()); + JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead = + deriveJvmMetaspaceAndOverheadWithTotalProcessMemory(config, totalProcessMemorySize); + MemorySize totalFlinkMemorySize = totalProcessMemorySize.subtract( + jvmMetaspaceAndOverhead.getTotalJvmMetaspaceAndOverheadSize()); + FM flinkInternalMemory = flinkMemoryUtils.deriveFromTotalFlinkMemory(config, totalFlinkMemorySize); + return new CommonProcessMemorySpec<>(flinkInternalMemory, jvmMetaspaceAndOverhead); + } + + private CommonProcessMemorySpec failBecauseRequiredOptionsNotConfigured() { + String[] internalMemoryOptionKeys = options.getRequiredFineGrainedOptions().stream().map(ConfigOption::key).toArray(String[]::new); + throw new IllegalConfigurationException(String.format( + "Either required fine-grained memory (%s), or Total Flink Memory size (%s), or Total Process Memory size " + + "(%s) need to be configured explicitly.", + String.join(" and ", internalMemoryOptionKeys), + options.getTotalFlinkMemoryOption(), + options.getTotalProcessMemoryOption())); + } + + private JvmMetaspaceAndOverhead deriveJvmMetaspaceAndOverheadWithTotalProcessMemory( + Configuration config, + MemorySize totalProcessMemorySize) { + MemorySize jvmMetaspaceSize = getMemorySizeFromConfig(config, options.getJvmOptions().getJvmMetaspaceOption()); + MemorySize jvmOverheadSize = deriveWithFraction( + "jvm overhead memory", + totalProcessMemorySize, + getJvmOverheadRangeFraction(config)); + JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead = new JvmMetaspaceAndOverhead(jvmMetaspaceSize, jvmOverheadSize); + + if (jvmMetaspaceAndOverhead.getTotalJvmMetaspaceAndOverheadSize().getBytes() > totalProcessMemorySize.getBytes()) { + throw new IllegalConfigurationException( + "Sum of configured JVM Metaspace (" + jvmMetaspaceAndOverhead.getMetaspace().toHumanReadableString() + + ") and JVM Overhead (" + jvmMetaspaceAndOverhead.getOverhead().toHumanReadableString() + + ") exceed configured Total Process Memory (" + totalProcessMemorySize.toHumanReadableString() + ")."); + } + + return jvmMetaspaceAndOverhead; + } + + private JvmMetaspaceAndOverhead deriveJvmMetaspaceAndOverheadFromTotalFlinkMemory( + Configuration config, + MemorySize totalFlinkMemorySize) { + MemorySize jvmMetaspaceSize = getMemorySizeFromConfig(config, options.getJvmOptions().getJvmMetaspaceOption()); + MemorySize totalFlinkAndJvmMetaspaceSize = totalFlinkMemorySize.add(jvmMetaspaceSize); + JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead; + if (config.contains(options.getTotalProcessMemoryOption())) { + MemorySize totalProcessMemorySize = getMemorySizeFromConfig(config, options.getTotalProcessMemoryOption()); + MemorySize jvmOverheadSize = totalProcessMemorySize.subtract(totalFlinkAndJvmMetaspaceSize); + sanityCheckJvmOverhead(config, jvmOverheadSize, totalProcessMemorySize); + jvmMetaspaceAndOverhead = new JvmMetaspaceAndOverhead(jvmMetaspaceSize, jvmOverheadSize); + } else { + MemorySize jvmOverheadSize = deriveWithInverseFraction( + "jvm overhead memory", + totalFlinkAndJvmMetaspaceSize, + getJvmOverheadRangeFraction(config)); + jvmMetaspaceAndOverhead = new JvmMetaspaceAndOverhead(jvmMetaspaceSize, jvmOverheadSize); + sanityCheckTotalProcessMemory(config, totalFlinkMemorySize, jvmMetaspaceAndOverhead); + } + return jvmMetaspaceAndOverhead; + } + + private void sanityCheckJvmOverhead( + Configuration config, + MemorySize derivedJvmOverheadSize, + MemorySize totalProcessMemorySize) { + RangeFraction jvmOverheadRangeFraction = getJvmOverheadRangeFraction(config); + if (derivedJvmOverheadSize.getBytes() > jvmOverheadRangeFraction.getMaxSize().getBytes() || + derivedJvmOverheadSize.getBytes() < jvmOverheadRangeFraction.getMinSize().getBytes()) { + throw new IllegalConfigurationException("Derived JVM Overhead size (" + + derivedJvmOverheadSize.toHumanReadableString() + ") is not in configured JVM Overhead range [" + + jvmOverheadRangeFraction.getMinSize().toHumanReadableString() + ", " + + jvmOverheadRangeFraction.getMaxSize().toHumanReadableString() + "]."); + } + if (config.contains(options.getJvmOptions().getJvmOverheadFraction()) && + !derivedJvmOverheadSize.equals(totalProcessMemorySize.multiply(jvmOverheadRangeFraction.getFraction()))) { + LOG.info( + "The derived JVM Overhead size ({}) does not match " + + "the configured JVM Overhead fraction ({}) from the configured Total Process Memory size ({}). " + + "The derived JVM OVerhead size will be used.", + derivedJvmOverheadSize.toHumanReadableString(), + jvmOverheadRangeFraction.getFraction(), + totalProcessMemorySize.toHumanReadableString()); + } + } + + private void sanityCheckTotalProcessMemory( + Configuration config, + MemorySize totalFlinkMemory, + JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead) { + MemorySize derivedTotalProcessMemorySize = + totalFlinkMemory.add(jvmMetaspaceAndOverhead.getMetaspace()).add(jvmMetaspaceAndOverhead.getOverhead()); + if (config.contains(options.getTotalProcessMemoryOption())) { + MemorySize configuredTotalProcessMemorySize = getMemorySizeFromConfig(config, options.getTotalProcessMemoryOption()); + if (!configuredTotalProcessMemorySize.equals(derivedTotalProcessMemorySize)) { + throw new IllegalConfigurationException(String.format( + "Configured and Derived memory sizes (total %s) do not add up to the configured Total Process " + + "Memory size (%s). Configured and Derived memory sizes are: Total Flink Memory (%s), " + + "JVM Metaspace (%s), JVM Overhead (%s).", + derivedTotalProcessMemorySize.toHumanReadableString(), + configuredTotalProcessMemorySize.toHumanReadableString(), + totalFlinkMemory.toHumanReadableString(), + jvmMetaspaceAndOverhead.getMetaspace().toHumanReadableString(), + jvmMetaspaceAndOverhead.getOverhead().toHumanReadableString())); + } + } + } + + private RangeFraction getJvmOverheadRangeFraction(Configuration config) { + MemorySize minSize = getMemorySizeFromConfig(config, options.getJvmOptions().getJvmOverheadMin()); + MemorySize maxSize = getMemorySizeFromConfig(config, options.getJvmOptions().getJvmOverheadMax()); + return getRangeFraction(minSize, maxSize, options.getJvmOptions().getJvmOverheadFraction(), config); + } + + public static MemorySize getMemorySizeFromConfig(Configuration config, ConfigOption option) { + try { + return Preconditions.checkNotNull(config.get(option), "The memory option is not set and has no default value."); + } catch (Throwable t) { + throw new IllegalConfigurationException("Cannot read memory size from config option '" + option.key() + "'.", t); + } + } + + public static RangeFraction getRangeFraction( + MemorySize minSize, + MemorySize maxSize, + ConfigOption fractionOption, + Configuration config) { + double fraction = config.getFloat(fractionOption); + try { + return new RangeFraction(minSize, maxSize, fraction); + } catch (IllegalArgumentException e) { + throw new IllegalConfigurationException( + String.format( + "Inconsistently configured %s (%s) and its min (%s), max (%s) value", + fractionOption, + fraction, + minSize.toHumanReadableString(), + maxSize.toHumanReadableString()), + e); + } + } + + public static MemorySize deriveWithFraction(String memoryDescription, MemorySize base, RangeFraction rangeFraction) { + MemorySize relative = base.multiply(rangeFraction.getFraction()); + return capToMinMax(memoryDescription, relative, rangeFraction); + } + + public static MemorySize deriveWithInverseFraction(String memoryDescription, MemorySize base, RangeFraction rangeFraction) { + checkArgument(rangeFraction.getFraction() < 1); + MemorySize relative = base.multiply(rangeFraction.getFraction() / (1 - rangeFraction.getFraction())); + return capToMinMax(memoryDescription, relative, rangeFraction); + } + + private static MemorySize capToMinMax( + String memoryDescription, + MemorySize relative, + RangeFraction rangeFraction) { + long size = relative.getBytes(); + if (size > rangeFraction.getMaxSize().getBytes()) { + LOG.info( + "The derived from fraction {} ({}) is greater than its max value {}, max value will be used instead", + memoryDescription, + relative.toHumanReadableString(), + rangeFraction.getMaxSize().toHumanReadableString()); + size = rangeFraction.getMaxSize().getBytes(); + } else if (size < rangeFraction.getMinSize().getBytes()) { + LOG.info( + "The derived from fraction {} ({}) is less than its min value {}, min value will be used instead", + memoryDescription, + relative.toHumanReadableString(), + rangeFraction.getMinSize().toHumanReadableString()); + size = rangeFraction.getMinSize().getBytes(); + } + return new MemorySize(size); + } + + public static String generateJvmParametersStr(ProcessMemorySpec processSpec) { + return "-Xmx" + processSpec.getJvmHeapMemorySize().getBytes() + + " -Xms" + processSpec.getJvmHeapMemorySize().getBytes() + + " -XX:MaxDirectMemorySize=" + processSpec.getJvmDirectMemorySize().getBytes() + + " -XX:MaxMetaspaceSize=" + processSpec.getJvmMetaspaceSize().getBytes(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/RangeFraction.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/RangeFraction.java new file mode 100644 index 0000000000..8a5d9d150a --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/RangeFraction.java @@ -0,0 +1,52 @@ +/* + * 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.runtime.util.config.memory; + +import org.apache.flink.configuration.MemorySize; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * Range and fraction of a memory component, which is a capped fraction of another component. + */ +public class RangeFraction { + private final MemorySize minSize; + private final MemorySize maxSize; + private final double fraction; + + RangeFraction(MemorySize minSize, MemorySize maxSize, double fraction) { + this.minSize = minSize; + this.maxSize = maxSize; + this.fraction = fraction; + checkArgument(minSize.getBytes() <= maxSize.getBytes(), "min value must be less or equal to max value"); + checkArgument(fraction >= 0 && fraction < 1, "fraction must be in range [0, 1)"); + } + + public MemorySize getMinSize() { + return minSize; + } + + public MemorySize getMaxSize() { + return maxSize; + } + + public double getFraction() { + return fraction; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/jobmanager/JobManagerFlinkMemory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/jobmanager/JobManagerFlinkMemory.java new file mode 100644 index 0000000000..152cf0e9fb --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/jobmanager/JobManagerFlinkMemory.java @@ -0,0 +1,72 @@ +/* + * 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.runtime.util.config.memory.jobmanager; + +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.runtime.util.config.memory.FlinkMemory; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Flink internal memory components of Job Manager. + * + *

    A Job Manager's internal Flink memory consists of the following components. + *

      + *
    • JVM Heap Memory
    • + *
    • Off-Heap Memory (also JVM Direct Memory)
    • + *
    + * + *

    The relationships of Job Manager Flink memory components are shown below. + *

    + *               ┌ ─ ─  Total Flink Memory - ─ ─ ┐
    + *                 ┌───────────────────────────┐
    + *               | │       JVM Heap Memory     │ |
    + *                 └───────────────────────────┘
    + *               │ ┌───────────────────────────┐ │
    + *                 |    Off-heap Heap Memory   │   -─ JVM Direct Memory
    + *               │ └───────────────────────────┘ │
    + *               └ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┘
    + * 
    + */ +public class JobManagerFlinkMemory implements FlinkMemory { + private static final long serialVersionUID = 1L; + + private final MemorySize jvmHeap; + private final MemorySize offHeapMemory; + + JobManagerFlinkMemory(MemorySize jvmHeap, MemorySize offHeapMemory) { + this.jvmHeap = checkNotNull(jvmHeap); + this.offHeapMemory = checkNotNull(offHeapMemory); + } + + @Override + public MemorySize getJvmHeapMemorySize() { + return jvmHeap; + } + + @Override + public MemorySize getJvmDirectMemorySize() { + return offHeapMemory; + } + + @Override + public MemorySize getTotalFlinkMemorySize() { + return jvmHeap.add(offHeapMemory); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/jobmanager/JobManagerFlinkMemoryUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/jobmanager/JobManagerFlinkMemoryUtils.java new file mode 100644 index 0000000000..6e52cee2ec --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/jobmanager/JobManagerFlinkMemoryUtils.java @@ -0,0 +1,104 @@ +/* + * 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.runtime.util.config.memory.jobmanager; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.runtime.util.config.memory.FlinkMemoryUtils; +import org.apache.flink.runtime.util.config.memory.ProcessMemoryUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * {@link FlinkMemoryUtils} for Job Manager. + */ +public class JobManagerFlinkMemoryUtils implements FlinkMemoryUtils { + private static final Logger LOG = LoggerFactory.getLogger(JobManagerFlinkMemoryUtils.class); + + @Override + public JobManagerFlinkMemory deriveFromRequiredFineGrainedOptions(Configuration config) { + MemorySize jvmHeapMemorySize = ProcessMemoryUtils.getMemorySizeFromConfig(config, JobManagerOptions.JVM_HEAP_MEMORY); + MemorySize offHeapMemorySize = ProcessMemoryUtils.getMemorySizeFromConfig(config, JobManagerOptions.OFF_HEAP_MEMORY); + MemorySize derivedTotalFlinkMemorySize = jvmHeapMemorySize.add(offHeapMemorySize); + + if (config.contains(JobManagerOptions.TOTAL_FLINK_MEMORY)) { + // derive network memory from total flink memory, and check against network min/max + MemorySize totalFlinkMemorySize = ProcessMemoryUtils.getMemorySizeFromConfig(config, JobManagerOptions.TOTAL_FLINK_MEMORY); + if (derivedTotalFlinkMemorySize.getBytes() != totalFlinkMemorySize.getBytes()) { + throw new IllegalConfigurationException(String.format( + "Sum of the configured JVM Heap Memory (%s) and the configured or default Off-heap Memory (%s) " + + "exceeds the configured Total Flink Memory (%s). Please, make the configuration consistent " + + "or configure only one option: either JVM Heap or Total Flink Memory.", + jvmHeapMemorySize.toHumanReadableString(), + offHeapMemorySize.toHumanReadableString(), + totalFlinkMemorySize.toHumanReadableString())); + } + } + + return createJobManagerFlinkMemory(config, jvmHeapMemorySize, offHeapMemorySize); + } + + @Override + public JobManagerFlinkMemory deriveFromTotalFlinkMemory(Configuration config, MemorySize totalFlinkMemorySize) { + MemorySize offHeapMemorySize = ProcessMemoryUtils.getMemorySizeFromConfig(config, JobManagerOptions.OFF_HEAP_MEMORY); + if (totalFlinkMemorySize.compareTo(offHeapMemorySize) < 1) { + throw new IllegalConfigurationException( + "The configured Total Flink Memory (%s) is less than the configured Off-heap Memory (%s).", + totalFlinkMemorySize.toHumanReadableString(), + offHeapMemorySize.toHumanReadableString()); + } + MemorySize derivedJvmHeapMemorySize = totalFlinkMemorySize.subtract(offHeapMemorySize); + return createJobManagerFlinkMemory(config, derivedJvmHeapMemorySize, offHeapMemorySize); + } + + private static JobManagerFlinkMemory createJobManagerFlinkMemory( + Configuration config, + MemorySize jvmHeap, + MemorySize offHeapMemory) { + verifyJvmHeapSize(jvmHeap); + verifyJobStoreCacheSize(config, offHeapMemory); + return new JobManagerFlinkMemory(jvmHeap, offHeapMemory); + } + + private static void verifyJvmHeapSize(MemorySize jvmHeapSize) { + if (jvmHeapSize.compareTo(JobManagerOptions.MIN_JVM_HEAP_SIZE) < 1) { + LOG.warn( + "The configured or derived JVM heap memory size ({}) is less than its recommended minimum value ({})", + jvmHeapSize.toHumanReadableString(), + JobManagerOptions.MIN_JVM_HEAP_SIZE); + } + } + + private static void verifyJobStoreCacheSize(Configuration config, MemorySize jvmHeapSize) { + MemorySize jobStoreCacheHeapSize = + MemorySize.parse(config.getLong(JobManagerOptions.JOB_STORE_CACHE_SIZE) + "b"); + if (jvmHeapSize.compareTo(jobStoreCacheHeapSize) < 1) { + LOG.warn( + "The configured or derived JVM heap memory size ({}: {}) is less than the configured or default size " + + "of the job store cache ({}: {})", + JobManagerOptions.JOB_STORE_CACHE_SIZE.key(), + jvmHeapSize.toHumanReadableString(), + JobManagerOptions.JVM_HEAP_MEMORY.key(), + jobStoreCacheHeapSize.toHumanReadableString()); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/taskmanager/TaskExecutorFlinkMemory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/taskmanager/TaskExecutorFlinkMemory.java new file mode 100644 index 0000000000..f9b8928db0 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/taskmanager/TaskExecutorFlinkMemory.java @@ -0,0 +1,131 @@ +/* + * 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.runtime.util.config.memory.taskmanager; + +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.runtime.util.config.memory.FlinkMemory; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Flink internal memory components of Task Executor. + * + *

    A TaskExecutor's internal Flink memory consists of the following components. + *

      + *
    • Framework Heap Memory
    • + *
    • Framework Off-Heap Memory
    • + *
    • Task Heap Memory
    • + *
    • Task Off-Heap Memory
    • + *
    • Network Memory
    • + *
    • Managed Memory
    • + *
    + * + *

    The relationships of TaskExecutor Flink memory components are shown below. + *

    + *               ┌ ─ ─  Total Flink Memory - ─ ─ ┐
    + *               |┌ ─ ─ - - - On-Heap - - - ─ ─ ┐|
    + *                 ┌───────────────────────────┐
    + *               |││   Framework Heap Memory   ││|
    + *                 └───────────────────────────┘
    + *               │ ┌───────────────────────────┐ │
    + *                ||      Task Heap Memory     ││
    + *               │ └───────────────────────────┘ │
    + *                └ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┘
    + *               |┌ ─ ─ - - - Off-Heap  - - ─ ─ ┐|
    + *                │┌───────────────────────────┐│
    + *               │ │ Framework Off-Heap Memory │ │ ─┐
    + *                │└───────────────────────────┘│   │
    + *               │ ┌───────────────────────────┐ │  │
    + *                ││   Task Off-Heap Memory    ││   ┼─ JVM Direct Memory
    + *               │ └───────────────────────────┘ │  │
    + *                │┌───────────────────────────┐│   │
    + *               │ │      Network Memory       │ │ ─┘
    + *                │└───────────────────────────┘│
    + *               │ ┌───────────────────────────┐ │
    + *                |│      Managed Memory       │|
    + *               │ └───────────────────────────┘ │
    + *                └ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┘
    + *               └ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┘
    + * 
    + */ +public class TaskExecutorFlinkMemory implements FlinkMemory { + private static final long serialVersionUID = 1L; + + private final MemorySize frameworkHeap; + private final MemorySize frameworkOffHeap; + private final MemorySize taskHeap; + private final MemorySize taskOffHeap; + private final MemorySize network; + private final MemorySize managed; + + public TaskExecutorFlinkMemory( + final MemorySize frameworkHeap, + final MemorySize frameworkOffHeap, + final MemorySize taskHeap, + final MemorySize taskOffHeap, + final MemorySize network, + final MemorySize managed) { + + this.frameworkHeap = checkNotNull(frameworkHeap); + this.frameworkOffHeap = checkNotNull(frameworkOffHeap); + this.taskHeap = checkNotNull(taskHeap); + this.taskOffHeap = checkNotNull(taskOffHeap); + this.network = checkNotNull(network); + this.managed = checkNotNull(managed); + } + + public MemorySize getFrameworkHeap() { + return frameworkHeap; + } + + public MemorySize getFrameworkOffHeap() { + return frameworkOffHeap; + } + + public MemorySize getTaskHeap() { + return taskHeap; + } + + public MemorySize getTaskOffHeap() { + return taskOffHeap; + } + + public MemorySize getNetwork() { + return network; + } + + public MemorySize getManaged() { + return managed; + } + + @Override + public MemorySize getJvmHeapMemorySize() { + return frameworkHeap.add(taskHeap); + } + + @Override + public MemorySize getJvmDirectMemorySize() { + return frameworkOffHeap.add(taskOffHeap).add(network); + } + + @Override + public MemorySize getTotalFlinkMemorySize() { + return frameworkHeap.add(frameworkOffHeap).add(taskHeap).add(taskOffHeap).add(network).add(managed); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/taskmanager/TaskExecutorFlinkMemoryUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/taskmanager/TaskExecutorFlinkMemoryUtils.java new file mode 100644 index 0000000000..c421d77e6a --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/config/memory/taskmanager/TaskExecutorFlinkMemoryUtils.java @@ -0,0 +1,299 @@ +/* + * 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.runtime.util.config.memory.taskmanager; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.runtime.util.ConfigurationParserUtils; +import org.apache.flink.runtime.util.config.memory.FlinkMemoryUtils; +import org.apache.flink.runtime.util.config.memory.ProcessMemoryUtils; +import org.apache.flink.runtime.util.config.memory.RangeFraction; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * {@link FlinkMemoryUtils} for Task Executor. + */ +public class TaskExecutorFlinkMemoryUtils implements FlinkMemoryUtils { + private static final Logger LOG = LoggerFactory.getLogger(TaskExecutorFlinkMemoryUtils.class); + + @Override + public TaskExecutorFlinkMemory deriveFromRequiredFineGrainedOptions(Configuration config) { + final MemorySize taskHeapMemorySize = getTaskHeapMemorySize(config); + final MemorySize managedMemorySize = getManagedMemorySize(config); + + final MemorySize frameworkHeapMemorySize = getFrameworkHeapMemorySize(config); + final MemorySize frameworkOffHeapMemorySize = getFrameworkOffHeapMemorySize(config); + final MemorySize taskOffHeapMemorySize = getTaskOffHeapMemorySize(config); + + final MemorySize networkMemorySize; + final MemorySize totalFlinkExcludeNetworkMemorySize = + frameworkHeapMemorySize.add(frameworkOffHeapMemorySize).add(taskHeapMemorySize).add(taskOffHeapMemorySize).add(managedMemorySize); + + if (isTotalFlinkMemorySizeExplicitlyConfigured(config)) { + // derive network memory from total flink memory, and check against network min/max + final MemorySize totalFlinkMemorySize = getTotalFlinkMemorySize(config); + if (totalFlinkExcludeNetworkMemorySize.getBytes() > totalFlinkMemorySize.getBytes()) { + throw new IllegalConfigurationException( + "Sum of configured Framework Heap Memory (" + frameworkHeapMemorySize.toHumanReadableString() + + "), Framework Off-Heap Memory (" + frameworkOffHeapMemorySize.toHumanReadableString() + + "), Task Heap Memory (" + taskHeapMemorySize.toHumanReadableString() + + "), Task Off-Heap Memory (" + taskOffHeapMemorySize.toHumanReadableString() + + ") and Managed Memory (" + managedMemorySize.toHumanReadableString() + + ") exceed configured Total Flink Memory (" + totalFlinkMemorySize.toHumanReadableString() + ")."); + } + networkMemorySize = totalFlinkMemorySize.subtract(totalFlinkExcludeNetworkMemorySize); + sanityCheckNetworkMemoryWithExplicitlySetTotalFlinkAndHeapMemory(config, networkMemorySize, totalFlinkMemorySize); + } else { + // derive network memory from network configs + networkMemorySize = isUsingLegacyNetworkConfigs(config) ? getNetworkMemorySizeWithLegacyConfig(config) : + deriveNetworkMemoryWithInverseFraction(config, totalFlinkExcludeNetworkMemorySize); + } + + final TaskExecutorFlinkMemory flinkInternalMemory = new TaskExecutorFlinkMemory( + frameworkHeapMemorySize, + frameworkOffHeapMemorySize, + taskHeapMemorySize, + taskOffHeapMemorySize, + networkMemorySize, + managedMemorySize); + sanityCheckTotalFlinkMemory(config, flinkInternalMemory); + + return flinkInternalMemory; + } + + @Override + public TaskExecutorFlinkMemory deriveFromTotalFlinkMemory( + final Configuration config, + final MemorySize totalFlinkMemorySize) { + final MemorySize frameworkHeapMemorySize = getFrameworkHeapMemorySize(config); + final MemorySize frameworkOffHeapMemorySize = getFrameworkOffHeapMemorySize(config); + final MemorySize taskOffHeapMemorySize = getTaskOffHeapMemorySize(config); + + final MemorySize taskHeapMemorySize; + final MemorySize networkMemorySize; + final MemorySize managedMemorySize; + + if (isTaskHeapMemorySizeExplicitlyConfigured(config)) { + // task heap memory is configured, + // derive managed memory first, leave the remaining to network memory and check against network min/max + taskHeapMemorySize = getTaskHeapMemorySize(config); + managedMemorySize = deriveManagedMemoryAbsoluteOrWithFraction(config, totalFlinkMemorySize); + final MemorySize totalFlinkExcludeNetworkMemorySize = + frameworkHeapMemorySize.add(frameworkOffHeapMemorySize).add(taskHeapMemorySize).add(taskOffHeapMemorySize).add(managedMemorySize); + if (totalFlinkExcludeNetworkMemorySize.getBytes() > totalFlinkMemorySize.getBytes()) { + throw new IllegalConfigurationException( + "Sum of configured Framework Heap Memory (" + frameworkHeapMemorySize.toHumanReadableString() + + "), Framework Off-Heap Memory (" + frameworkOffHeapMemorySize.toHumanReadableString() + + "), Task Heap Memory (" + taskHeapMemorySize.toHumanReadableString() + + "), Task Off-Heap Memory (" + taskOffHeapMemorySize.toHumanReadableString() + + ") and Managed Memory (" + managedMemorySize.toHumanReadableString() + + ") exceed configured Total Flink Memory (" + totalFlinkMemorySize.toHumanReadableString() + ")."); + } + networkMemorySize = totalFlinkMemorySize.subtract(totalFlinkExcludeNetworkMemorySize); + sanityCheckNetworkMemoryWithExplicitlySetTotalFlinkAndHeapMemory(config, networkMemorySize, totalFlinkMemorySize); + } else { + // task heap memory is not configured + // derive managed memory and network memory, leave the remaining to task heap memory + managedMemorySize = deriveManagedMemoryAbsoluteOrWithFraction(config, totalFlinkMemorySize); + + networkMemorySize = isUsingLegacyNetworkConfigs(config) ? getNetworkMemorySizeWithLegacyConfig(config) : + deriveNetworkMemoryWithFraction(config, totalFlinkMemorySize); + final MemorySize totalFlinkExcludeTaskHeapMemorySize = + frameworkHeapMemorySize.add(frameworkOffHeapMemorySize).add(taskOffHeapMemorySize).add(managedMemorySize).add(networkMemorySize); + if (totalFlinkExcludeTaskHeapMemorySize.getBytes() > totalFlinkMemorySize.getBytes()) { + throw new IllegalConfigurationException( + "Sum of configured Framework Heap Memory (" + frameworkHeapMemorySize.toHumanReadableString() + + "), Framework Off-Heap Memory (" + frameworkOffHeapMemorySize.toHumanReadableString() + + "), Task Off-Heap Memory (" + taskOffHeapMemorySize.toHumanReadableString() + + "), Managed Memory (" + managedMemorySize.toHumanReadableString() + + ") and Network Memory (" + networkMemorySize.toHumanReadableString() + + ") exceed configured Total Flink Memory (" + totalFlinkMemorySize.toHumanReadableString() + ")."); + } + taskHeapMemorySize = totalFlinkMemorySize.subtract(totalFlinkExcludeTaskHeapMemorySize); + } + + final TaskExecutorFlinkMemory flinkInternalMemory = new TaskExecutorFlinkMemory( + frameworkHeapMemorySize, + frameworkOffHeapMemorySize, + taskHeapMemorySize, + taskOffHeapMemorySize, + networkMemorySize, + managedMemorySize); + sanityCheckTotalFlinkMemory(config, flinkInternalMemory); + + return flinkInternalMemory; + } + + private static MemorySize deriveManagedMemoryAbsoluteOrWithFraction(final Configuration config, final MemorySize base) { + return isManagedMemorySizeExplicitlyConfigured(config) ? getManagedMemorySize(config) : + ProcessMemoryUtils.deriveWithFraction("managed memory", base, getManagedMemoryRangeFraction(config)); + } + + private static MemorySize deriveNetworkMemoryWithFraction(final Configuration config, final MemorySize base) { + return ProcessMemoryUtils.deriveWithFraction("network memory", base, getNetworkMemoryRangeFraction(config)); + } + + private static MemorySize deriveNetworkMemoryWithInverseFraction(final Configuration config, final MemorySize base) { + return ProcessMemoryUtils.deriveWithInverseFraction("network memory", base, getNetworkMemoryRangeFraction(config)); + } + + private static MemorySize getFrameworkHeapMemorySize(final Configuration config) { + return ProcessMemoryUtils.getMemorySizeFromConfig(config, TaskManagerOptions.FRAMEWORK_HEAP_MEMORY); + } + + private static MemorySize getFrameworkOffHeapMemorySize(final Configuration config) { + return ProcessMemoryUtils.getMemorySizeFromConfig(config, TaskManagerOptions.FRAMEWORK_OFF_HEAP_MEMORY); + } + + private static MemorySize getTaskHeapMemorySize(final Configuration config) { + checkArgument(isTaskHeapMemorySizeExplicitlyConfigured(config)); + return ProcessMemoryUtils.getMemorySizeFromConfig(config, TaskManagerOptions.TASK_HEAP_MEMORY); + } + + private static MemorySize getTaskOffHeapMemorySize(final Configuration config) { + return ProcessMemoryUtils.getMemorySizeFromConfig(config, TaskManagerOptions.TASK_OFF_HEAP_MEMORY); + } + + private static MemorySize getManagedMemorySize(final Configuration config) { + checkArgument(isManagedMemorySizeExplicitlyConfigured(config)); + return ProcessMemoryUtils.getMemorySizeFromConfig(config, TaskManagerOptions.MANAGED_MEMORY_SIZE); + } + + private static RangeFraction getManagedMemoryRangeFraction(final Configuration config) { + return ProcessMemoryUtils.getRangeFraction(MemorySize.ZERO, MemorySize.MAX_VALUE, TaskManagerOptions.MANAGED_MEMORY_FRACTION, config); + } + + private static MemorySize getNetworkMemorySizeWithLegacyConfig(final Configuration config) { + checkArgument(isUsingLegacyNetworkConfigs(config)); + @SuppressWarnings("deprecation") + final long numOfBuffers = config.getInteger(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS); + final long pageSize = ConfigurationParserUtils.getPageSize(config); + return new MemorySize(numOfBuffers * pageSize); + } + + private static RangeFraction getNetworkMemoryRangeFraction(final Configuration config) { + final MemorySize minSize = ProcessMemoryUtils.getMemorySizeFromConfig(config, TaskManagerOptions.NETWORK_MEMORY_MIN); + final MemorySize maxSize = ProcessMemoryUtils.getMemorySizeFromConfig(config, TaskManagerOptions.NETWORK_MEMORY_MAX); + return ProcessMemoryUtils.getRangeFraction(minSize, maxSize, TaskManagerOptions.NETWORK_MEMORY_FRACTION, config); + } + + private static MemorySize getTotalFlinkMemorySize(final Configuration config) { + checkArgument(isTotalFlinkMemorySizeExplicitlyConfigured(config)); + return ProcessMemoryUtils.getMemorySizeFromConfig(config, TaskManagerOptions.TOTAL_FLINK_MEMORY); + } + + private static boolean isTaskHeapMemorySizeExplicitlyConfigured(final Configuration config) { + return config.contains(TaskManagerOptions.TASK_HEAP_MEMORY); + } + + private static boolean isManagedMemorySizeExplicitlyConfigured(final Configuration config) { + return config.contains(TaskManagerOptions.MANAGED_MEMORY_SIZE); + } + + private static boolean isUsingLegacyNetworkConfigs(final Configuration config) { + // use the legacy number-of-buffer config option only when it is explicitly configured and + // none of new config options is explicitly configured + final boolean anyNetworkConfigured = config.contains(TaskManagerOptions.NETWORK_MEMORY_MIN) || + config.contains(TaskManagerOptions.NETWORK_MEMORY_MAX) || + config.contains(TaskManagerOptions.NETWORK_MEMORY_FRACTION); + final boolean legacyConfigured = config.contains(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS); + return !anyNetworkConfigured && legacyConfigured; + } + + private static boolean isNetworkMemoryFractionExplicitlyConfigured(final Configuration config) { + return config.contains(TaskManagerOptions.NETWORK_MEMORY_FRACTION); + } + + private static boolean isTotalFlinkMemorySizeExplicitlyConfigured(final Configuration config) { + return config.contains(TaskManagerOptions.TOTAL_FLINK_MEMORY); + } + + private static void sanityCheckTotalFlinkMemory(final Configuration config, final TaskExecutorFlinkMemory flinkInternalMemory) { + if (isTotalFlinkMemorySizeExplicitlyConfigured(config)) { + final MemorySize configuredTotalFlinkMemorySize = getTotalFlinkMemorySize(config); + if (!configuredTotalFlinkMemorySize.equals(flinkInternalMemory.getTotalFlinkMemorySize())) { + throw new IllegalConfigurationException( + "Configured and Derived Flink internal memory sizes (total " + flinkInternalMemory.getTotalFlinkMemorySize().toHumanReadableString() + + ") do not add up to the configured Total Flink Memory size (" + configuredTotalFlinkMemorySize.toHumanReadableString() + + "). Configured and Derived Flink internal memory sizes are: " + + "Framework Heap Memory (" + flinkInternalMemory.getFrameworkHeap().toHumanReadableString() + + "), Framework Off-Heap Memory (" + flinkInternalMemory.getFrameworkOffHeap().toHumanReadableString() + + "), Task Heap Memory (" + flinkInternalMemory.getTaskHeap().toHumanReadableString() + + "), Task Off-Heap Memory (" + flinkInternalMemory.getTaskOffHeap().toHumanReadableString() + + "), Network Memory (" + flinkInternalMemory.getNetwork().toHumanReadableString() + + "), Managed Memory (" + flinkInternalMemory.getManaged().toHumanReadableString() + ")."); + } + } + } + + private static void sanityCheckNetworkMemoryWithExplicitlySetTotalFlinkAndHeapMemory( + final Configuration config, + final MemorySize derivedNetworkMemorySize, + final MemorySize totalFlinkMemorySize) { + try { + sanityCheckNetworkMemory(config, derivedNetworkMemorySize, totalFlinkMemorySize); + } catch (IllegalConfigurationException e) { + throw new IllegalConfigurationException( + "If Total Flink, Task Heap and (or) Managed Memory sizes are explicitly configured then " + + "the Network Memory size is the rest of the Total Flink memory after subtracting all other " + + "configured types of memory, but the derived Network Memory is inconsistent with its configuration.", + e); + } + } + + private static void sanityCheckNetworkMemory( + final Configuration config, + final MemorySize derivedNetworkMemorySize, + final MemorySize totalFlinkMemorySize) { + if (isUsingLegacyNetworkConfigs(config)) { + final MemorySize configuredNetworkMemorySize = getNetworkMemorySizeWithLegacyConfig(config); + if (!configuredNetworkMemorySize.equals(derivedNetworkMemorySize)) { + throw new IllegalConfigurationException( + "Derived Network Memory size (" + derivedNetworkMemorySize.toHumanReadableString() + + ") does not match configured Network Memory size (" + configuredNetworkMemorySize.toHumanReadableString() + ")."); + } + } else { + final RangeFraction networkRangeFraction = getNetworkMemoryRangeFraction(config); + if (derivedNetworkMemorySize.getBytes() > networkRangeFraction.getMaxSize().getBytes() || + derivedNetworkMemorySize.getBytes() < networkRangeFraction.getMinSize().getBytes()) { + throw new IllegalConfigurationException("Derived Network Memory size (" + + derivedNetworkMemorySize.toHumanReadableString() + ") is not in configured Network Memory range [" + + networkRangeFraction.getMinSize().toHumanReadableString() + ", " + + networkRangeFraction.getMaxSize().toHumanReadableString() + "]."); + } + if (isNetworkMemoryFractionExplicitlyConfigured(config) && + !derivedNetworkMemorySize.equals(totalFlinkMemorySize.multiply(networkRangeFraction.getFraction()))) { + LOG.info( + "The derived Network Memory size ({}) does not match " + + "the configured Network Memory fraction ({}) from the configured Total Flink Memory size ({}). " + + "The derived Network Memory size will be used.", + derivedNetworkMemorySize.toHumanReadableString(), + networkRangeFraction.getFraction(), + totalFlinkMemorySize.toHumanReadableString()); + } + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/TaskExecutorProcessUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/TaskExecutorProcessUtilsTest.java index 5bbcd0afea..6fd2db5411 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/TaskExecutorProcessUtilsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/TaskExecutorProcessUtilsTest.java @@ -26,19 +26,17 @@ import org.apache.flink.configuration.IllegalConfigurationException; import org.apache.flink.configuration.MemorySize; import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; import org.apache.flink.configuration.TaskManagerOptions; -import org.apache.flink.core.testutils.CommonTestUtils; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; -import org.apache.flink.util.TestLogger; +import org.apache.flink.runtime.util.config.memory.ProcessMemoryUtilsTestBase; -import org.junit.After; -import org.junit.Before; import org.junit.Test; import java.util.Arrays; -import java.util.HashMap; import java.util.Map; import java.util.function.Consumer; +import static org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils.TM_LEGACY_HEAP_OPTIONS; +import static org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils.TM_PROCESS_MEMORY_OPTIONS; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.is; @@ -49,7 +47,7 @@ import static org.junit.Assert.fail; /** * Tests for {@link TaskExecutorProcessUtils}. */ -public class TaskExecutorProcessUtilsTest extends TestLogger { +public class TaskExecutorProcessUtilsTest extends ProcessMemoryUtilsTestBase { private static final MemorySize TASK_HEAP_SIZE = MemorySize.parse("100m"); private static final MemorySize MANAGED_MEM_SIZE = MemorySize.parse("200m"); @@ -77,18 +75,8 @@ public class TaskExecutorProcessUtilsTest extends TestLogger { .setManagedMemory(MemorySize.parse("3m")) .build(); - private static Map oldEnvVariables; - - @Before - public void setup() { - oldEnvVariables = System.getenv(); - } - - @After - public void teardown() { - if (oldEnvVariables != null) { - CommonTestUtils.setEnv(oldEnvVariables, true); - } + public TaskExecutorProcessUtilsTest() { + super(TM_PROCESS_MEMORY_OPTIONS, TM_LEGACY_HEAP_OPTIONS, TaskManagerOptions.TOTAL_PROCESS_MEMORY); } @Test @@ -106,17 +94,6 @@ public class TaskExecutorProcessUtilsTest extends TestLogger { assertThat(MemorySize.parse(configs.get(TaskManagerOptions.MANAGED_MEMORY_SIZE.key())), is(TM_RESOURCE_SPEC.getManagedMemorySize())); } - @Test - public void testGenerateJvmParameters() { - String jvmParamsStr = TaskExecutorProcessUtils.generateJvmParametersStr(TM_RESOURCE_SPEC); - Map configs = ConfigurationUtils.parseTmResourceJvmParams(jvmParamsStr); - - assertThat(MemorySize.parse(configs.get("-Xmx")), is(TM_RESOURCE_SPEC.getFrameworkHeapSize().add(TM_RESOURCE_SPEC.getTaskHeapSize()))); - assertThat(MemorySize.parse(configs.get("-Xms")), is(TM_RESOURCE_SPEC.getFrameworkHeapSize().add(TM_RESOURCE_SPEC.getTaskHeapSize()))); - assertThat(MemorySize.parse(configs.get("-XX:MaxDirectMemorySize=")), is(TM_RESOURCE_SPEC.getFrameworkOffHeapMemorySize().add(TM_RESOURCE_SPEC.getTaskOffHeapSize()).add(TM_RESOURCE_SPEC.getNetworkMemSize()))); - assertThat(MemorySize.parse(configs.get("-XX:MaxMetaspaceSize=")), is(TM_RESOURCE_SPEC.getJvmMetaspaceSize())); - } - @Test public void testConfigCpuCores() { final double cpuCores = 1.0; @@ -405,111 +382,6 @@ public class TaskExecutorProcessUtilsTest extends TestLogger { validateFailInConfigurationsWithoutExplicitManagedMem(conf); } - @Test - public void testConfigJvmMetaspaceSize() { - final MemorySize jvmMetaspaceSize = MemorySize.parse("50m"); - - Configuration conf = new Configuration(); - conf.set(TaskManagerOptions.JVM_METASPACE, jvmMetaspaceSize); - - validateInAllConfigurations(conf, taskExecutorProcessSpec -> assertThat(taskExecutorProcessSpec.getJvmMetaspaceSize(), is(jvmMetaspaceSize))); - } - - @Test - public void testConfigJvmOverheadRange() { - final MemorySize minSize = MemorySize.parse("50m"); - final MemorySize maxSize = MemorySize.parse("200m"); - - Configuration conf = new Configuration(); - conf.set(TaskManagerOptions.JVM_OVERHEAD_MAX, maxSize); - conf.set(TaskManagerOptions.JVM_OVERHEAD_MIN, minSize); - - validateInAllConfigurations(conf, taskExecutorProcessSpec -> { - assertThat(taskExecutorProcessSpec.getJvmOverheadSize().getBytes(), - greaterThanOrEqualTo(minSize.getBytes())); - assertThat(taskExecutorProcessSpec.getJvmOverheadSize().getBytes(), lessThanOrEqualTo(maxSize.getBytes())); - }); - } - - @Test - public void testConfigJvmOverheadRangeFailure() { - final MemorySize minSize = MemorySize.parse("200m"); - final MemorySize maxSize = MemorySize.parse("50m"); - - Configuration conf = new Configuration(); - conf.set(TaskManagerOptions.JVM_OVERHEAD_MAX, maxSize); - conf.set(TaskManagerOptions.JVM_OVERHEAD_MIN, minSize); - - validateFailInAllConfigurations(conf); - } - - @Test - public void testConfigJvmOverheadFraction() { - final MemorySize minSize = MemorySize.ZERO; - final MemorySize maxSize = MemorySize.parse("1t"); - final float fraction = 0.2f; - - Configuration conf = new Configuration(); - conf.set(TaskManagerOptions.JVM_OVERHEAD_MAX, maxSize); - conf.set(TaskManagerOptions.JVM_OVERHEAD_MIN, minSize); - conf.setFloat(TaskManagerOptions.JVM_OVERHEAD_FRACTION, fraction); - - validateInAllConfigurations(conf, taskExecutorProcessSpec -> - assertThat(taskExecutorProcessSpec.getJvmOverheadSize(), is(taskExecutorProcessSpec.getTotalProcessMemorySize().multiply(fraction)))); - } - - @Test - public void testConfigJvmOverheadFractionFailureNegative() { - final Configuration conf = new Configuration(); - conf.setFloat(TaskManagerOptions.JVM_OVERHEAD_FRACTION, -0.1f); - validateFailInConfigurationsWithoutExplicitManagedMem(conf); - } - - @Test - public void testConfigJvmOverheadFractionFailureNoLessThanOne() { - final Configuration conf = new Configuration(); - conf.setFloat(TaskManagerOptions.JVM_OVERHEAD_FRACTION, 1.0f); - validateFailInConfigurationsWithoutExplicitManagedMem(conf); - } - - @Test - public void testConfigJvmOverheadDeriveFromProcessAndFlinkMemorySize() { - final Configuration conf = new Configuration(); - conf.set(TaskManagerOptions.TOTAL_PROCESS_MEMORY, MemorySize.parse("1000m")); - conf.set(TaskManagerOptions.TOTAL_FLINK_MEMORY, MemorySize.parse("800m")); - conf.set(TaskManagerOptions.JVM_METASPACE, MemorySize.parse("100m")); - conf.set(TaskManagerOptions.JVM_OVERHEAD_MIN, MemorySize.parse("50m")); - conf.set(TaskManagerOptions.JVM_OVERHEAD_MAX, MemorySize.parse("200m")); - conf.set(TaskManagerOptions.JVM_OVERHEAD_FRACTION, 0.5f); - - TaskExecutorProcessSpec taskExecutorProcessSpec = TaskExecutorProcessUtils.processSpecFromConfig(conf); - assertThat(taskExecutorProcessSpec.getJvmOverheadSize(), is(MemorySize.parse("100m"))); - } - - @Test - public void testConfigJvmOverheadDeriveFromProcessAndFlinkMemorySizeFailure() { - final Configuration conf = new Configuration(); - conf.set(TaskManagerOptions.TOTAL_PROCESS_MEMORY, MemorySize.parse("1000m")); - conf.set(TaskManagerOptions.TOTAL_FLINK_MEMORY, MemorySize.parse("800m")); - conf.set(TaskManagerOptions.JVM_METASPACE, MemorySize.parse("100m")); - conf.set(TaskManagerOptions.JVM_OVERHEAD_MIN, MemorySize.parse("150m")); - conf.set(TaskManagerOptions.JVM_OVERHEAD_MAX, MemorySize.parse("200m")); - conf.set(TaskManagerOptions.JVM_OVERHEAD_FRACTION, 0.5f); - - validateFail(conf); - } - - @Test - public void testConfigTotalFlinkMemory() { - final MemorySize totalFlinkMemorySize = MemorySize.parse("1g"); - - Configuration conf = new Configuration(); - conf.set(TaskManagerOptions.TOTAL_FLINK_MEMORY, totalFlinkMemorySize); - - TaskExecutorProcessSpec taskExecutorProcessSpec = TaskExecutorProcessUtils.processSpecFromConfig(conf); - assertThat(taskExecutorProcessSpec.getTotalFlinkMemorySize(), is(totalFlinkMemorySize)); - } - @Test public void testFlinkInternalMemorySizeAddUpFailure() { final MemorySize totalFlinkMemory = MemorySize.parse("499m"); @@ -531,91 +403,6 @@ public class TaskExecutorProcessUtilsTest extends TestLogger { validateFail(conf); } - @Test - public void testConfigTotalProcessMemorySize() { - final MemorySize totalProcessMemorySize = MemorySize.parse("2g"); - - Configuration conf = new Configuration(); - conf.set(TaskManagerOptions.TOTAL_PROCESS_MEMORY, totalProcessMemorySize); - - TaskExecutorProcessSpec taskExecutorProcessSpec = TaskExecutorProcessUtils.processSpecFromConfig(conf); - assertThat(taskExecutorProcessSpec.getTotalProcessMemorySize(), is(totalProcessMemorySize)); - } - - @Test - public void testConfigLegacyTaskManagerHeapSize() { - final MemorySize taskManagerHeapSize = MemorySize.parse("1g"); - - Configuration conf = new Configuration(); - conf.set(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY, taskManagerHeapSize); - - testConfigLegacyTaskManagerHeapMemory(conf, taskManagerHeapSize); - } - - @Test - public void testConfigLegacyTaskManagerHeapMB() { - final MemorySize taskManagerHeapSize = MemorySize.parse("1g"); - - Configuration conf = new Configuration(); - conf.set(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY_MB, taskManagerHeapSize.getMebiBytes()); - - testConfigLegacyTaskManagerHeapMemory(conf, taskManagerHeapSize); - } - - @Test - public void testConfigLegacyTaskManagerHeapEnv() { - final MemorySize taskManagerHeapSize = MemorySize.parse("1g"); - - final Map env = new HashMap<>(); - env.put("FLINK_TM_HEAP", "1g"); - CommonTestUtils.setEnv(env); - - testConfigLegacyTaskManagerHeapMemory(new Configuration(), taskManagerHeapSize); - } - - @Test - public void testConfigBothTotalFlinkSizeAndLegacyTaskManagerHeapSize() { - final MemorySize totalFlinkSize = MemorySize.parse("1g"); - final MemorySize taskManagerHeapSize = MemorySize.parse("2g"); - - Configuration conf = new Configuration(); - conf.set(TaskManagerOptions.TOTAL_FLINK_MEMORY, totalFlinkSize); - conf.set(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY, taskManagerHeapSize); - - testConfigLegacyTaskManagerHeapMemoryStandalone(conf, totalFlinkSize); - } - - @Test - public void testConfigBothTotalProcessSizeAndLegacyTaskManagerHeapSize() { - final MemorySize totalProcess = MemorySize.parse("1g"); - final MemorySize taskManagerHeapSize = MemorySize.parse("2g"); - - Configuration conf = new Configuration(); - conf.set(TaskManagerOptions.TOTAL_PROCESS_MEMORY, totalProcess); - conf.set(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY, taskManagerHeapSize); - - testConfigLegacyTaskManagerHeapMemoryContainerized(conf, totalProcess); - } - - private void testConfigLegacyTaskManagerHeapMemory(final Configuration configuration, final MemorySize expected) { - testConfigLegacyTaskManagerHeapMemoryStandalone(configuration, expected); - testConfigLegacyTaskManagerHeapMemoryContainerized(configuration, expected); - } - - private void testConfigLegacyTaskManagerHeapMemoryStandalone(final Configuration configuration, final MemorySize expected) { - final Configuration adjustedConfig = TaskExecutorProcessUtils - .getConfigurationMapLegacyTaskManagerHeapSizeToConfigOption(configuration, TaskManagerOptions.TOTAL_FLINK_MEMORY); - TaskExecutorProcessSpec taskExecutorProcessSpec = TaskExecutorProcessUtils.processSpecFromConfig(adjustedConfig); - assertThat(taskExecutorProcessSpec.getTotalFlinkMemorySize(), is(expected)); - } - - private void testConfigLegacyTaskManagerHeapMemoryContainerized(final Configuration configuration, final MemorySize expected) { - final Configuration adjustedConfig = TaskExecutorProcessUtils - .getConfigurationMapLegacyTaskManagerHeapSizeToConfigOption(configuration, TaskManagerOptions.TOTAL_PROCESS_MEMORY); - TaskExecutorProcessSpec taskExecutorProcessSpec = TaskExecutorProcessUtils.processSpecFromConfig(adjustedConfig); - assertThat(taskExecutorProcessSpec.getTotalProcessMemorySize(), is(expected)); - } - @Test public void testFlinkInternalMemoryFractionAddUpFailure() { final float networkFraction = 0.6f; @@ -644,23 +431,6 @@ public class TaskExecutorProcessUtilsTest extends TestLogger { assertThat(taskExecutorProcessSpec.getTotalProcessMemorySize(), is(totalProcessMemorySize)); } - @Test - public void testConfigTotalProcessMemoryAddUpFailure() { - final MemorySize totalProcessMemory = MemorySize.parse("699m"); - final MemorySize totalFlinkMemory = MemorySize.parse("500m"); - final MemorySize jvmMetaspace = MemorySize.parse("100m"); - final MemorySize jvmOverhead = MemorySize.parse("100m"); - - Configuration conf = new Configuration(); - conf.set(TaskManagerOptions.TOTAL_PROCESS_MEMORY, totalProcessMemory); - conf.set(TaskManagerOptions.TOTAL_FLINK_MEMORY, totalFlinkMemory); - conf.set(TaskManagerOptions.JVM_METASPACE, jvmMetaspace); - conf.set(TaskManagerOptions.JVM_OVERHEAD_MIN, jvmOverhead); - conf.set(TaskManagerOptions.JVM_OVERHEAD_MAX, jvmOverhead); - - validateFail(conf); - } - @Test public void testCreateDefaultWorkerSlotProfiles() { assertThat( @@ -687,7 +457,8 @@ public class TaskExecutorProcessUtilsTest extends TestLogger { } } - private void validateInAllConfigurations(final Configuration customConfig, Consumer validateFunc) { + @Override + protected void validateInAllConfigurations(final Configuration customConfig, Consumer validateFunc) { validateInConfigWithExplicitTaskHeapAndManagedMem(customConfig, validateFunc); validateInConfigWithExplicitTotalFlinkMem(customConfig, validateFunc); validateInConfigWithExplicitTotalFlinkAndTaskHeapMem(customConfig, validateFunc); @@ -695,7 +466,8 @@ public class TaskExecutorProcessUtilsTest extends TestLogger { validateInConfigWithExplicitTotalProcessMem(customConfig, validateFunc); } - private void validateFailInAllConfigurations(final Configuration customConfig) { + @Override + protected void validateFailInAllConfigurations(final Configuration customConfig) { validateFailInConfigWithExplicitTaskHeapAndManagedMem(customConfig); validateFailInConfigWithExplicitTotalFlinkMem(customConfig); validateFailInConfigWithExplicitTotalFlinkAndTaskHeapMem(customConfig); @@ -809,7 +581,8 @@ public class TaskExecutorProcessUtilsTest extends TestLogger { validateFail(config); } - private void validateFail(final Configuration config) { + @Override + protected void validateFail(final Configuration config) { try { TaskExecutorProcessUtils.processSpecFromConfig(config); fail("Configuration did not fail as expected."); @@ -850,4 +623,16 @@ public class TaskExecutorProcessUtilsTest extends TestLogger { conf.set(TaskManagerOptions.TOTAL_PROCESS_MEMORY, TOTAL_PROCESS_MEM_SIZE); return conf; } + + @Override + protected TaskExecutorProcessSpec processSpecFromConfig(Configuration config) { + return TaskExecutorProcessUtils.processSpecFromConfig(config); + } + + @Override + protected Configuration getConfigurationWithLegacyHeapSizeMappedToNewConfigOption(Configuration config) { + return TaskExecutorProcessUtils.getConfigurationMapLegacyTaskManagerHeapSizeToConfigOption( + config, + getNewOptionForLegacyHeapOption()); + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerProcessUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerProcessUtilsTest.java new file mode 100644 index 0000000000..37c982387c --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerProcessUtilsTest.java @@ -0,0 +1,226 @@ +/* + * 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.runtime.jobmanager; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.runtime.util.config.memory.ProcessMemoryUtilsTestBase; + +import org.junit.Test; + +import java.util.function.Consumer; + +import static org.apache.flink.runtime.jobmanager.JobManagerProcessUtils.JM_LEGACY_HEAP_OPTIONS; +import static org.apache.flink.runtime.jobmanager.JobManagerProcessUtils.JM_PROCESS_MEMORY_OPTIONS; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; + +/** + * Tests for {@link JobManagerProcessUtils}. + */ +public class JobManagerProcessUtilsTest extends ProcessMemoryUtilsTestBase { + private static final MemorySize JVM_HEAP_SIZE = MemorySize.parse("1152m"); + private static final MemorySize TOTAL_FLINK_MEM_SIZE = MemorySize.parse("1280m"); + private static final MemorySize TOTAL_PROCESS_MEM_SIZE = MemorySize.parse("1536m"); + + public JobManagerProcessUtilsTest() { + super(JM_PROCESS_MEMORY_OPTIONS, JM_LEGACY_HEAP_OPTIONS, JobManagerOptions.TOTAL_PROCESS_MEMORY); + } + + @Test + public void testConfigJvmHeapMemory() { + MemorySize jvmHeapSize = MemorySize.parse("50m"); + + Configuration conf = new Configuration(); + conf.set(JobManagerOptions.JVM_HEAP_MEMORY, jvmHeapSize); + + JobManagerProcessSpec JobManagerProcessSpec = JobManagerProcessUtils.processSpecFromConfig(conf); + assertThat(JobManagerProcessSpec.getJvmHeapMemorySize(), is(jvmHeapSize)); + } + + @Test + public void testConfigOffHeapMemory() { + MemorySize offHeapMemory = MemorySize.parse("100m"); + + Configuration conf = new Configuration(); + conf.set(JobManagerOptions.OFF_HEAP_MEMORY, offHeapMemory); + + validateInAllConfigurationsWithoutExplicitTotalFlinkAndJvmHeapMem( + conf, + jobManagerProcessSpec -> assertThat(jobManagerProcessSpec.getJvmDirectMemorySize(), is(offHeapMemory))); + } + + @Test + public void testFlinkInternalMemorySizeAddUpFailure() { + MemorySize totalFlinkMemory = MemorySize.parse("199m"); + MemorySize jvmHeap = MemorySize.parse("100m"); + MemorySize offHeapMemory = MemorySize.parse("100m"); + + Configuration conf = new Configuration(); + conf.set(JobManagerOptions.TOTAL_FLINK_MEMORY, totalFlinkMemory); + conf.set(JobManagerOptions.JVM_HEAP_MEMORY, jvmHeap); + conf.set(JobManagerOptions.OFF_HEAP_MEMORY, offHeapMemory); + + validateFail(conf); + } + + @Override + protected JobManagerProcessSpec processSpecFromConfig(Configuration config) { + return JobManagerProcessUtils.processSpecFromConfig(config); + } + + @Override + protected Configuration getConfigurationWithLegacyHeapSizeMappedToNewConfigOption(Configuration config) { + return JobManagerProcessUtils.getConfigurationWithLegacyHeapSizeMappedToNewConfigOption( + config, + getNewOptionForLegacyHeapOption()); + } + + @Override + protected void validateInAllConfigurations(Configuration customConfig, Consumer validateFunc) { + validateInConfigWithExplicitJvmHeap(customConfig, validateFunc); + validateInConfigWithExplicitTotalFlinkMem(customConfig, validateFunc); + validateInConfigWithExplicitTotalFlinkAndJvmHeapMem(customConfig, validateFunc); + validateInConfigWithExplicitTotalProcessMem(customConfig, validateFunc); + } + + @Override + protected void validateFailInAllConfigurations(Configuration customConfig) { + validateFailInConfigWithExplicitJvmHeap(customConfig); + validateFailInConfigWithExplicitTotalFlinkMem(customConfig); + validateFailInConfigWithExplicitTotalFlinkAndJvmHeapMem(customConfig); + validateFailInConfigWithExplicitTotalProcessMem(customConfig); + } + + private void validateInAllConfigurationsWithoutExplicitTotalFlinkAndJvmHeapMem( + Configuration customConfig, + Consumer validateFunc) { + validateInConfigWithExplicitJvmHeap(customConfig, validateFunc); + validateInConfigWithExplicitTotalFlinkMem(customConfig, validateFunc); + validateInConfigWithExplicitTotalProcessMem(customConfig, validateFunc); + } + + private void validateInConfigWithExplicitJvmHeap( + Configuration customConfig, Consumer validateFunc) { + log.info("Validating in configuration with explicit jvm heap."); + Configuration config = configWithExplicitJvmHeap(); + config.addAll(customConfig); + JobManagerProcessSpec JobManagerProcessSpec = JobManagerProcessUtils.processSpecFromConfig(config); + assertThat(JobManagerProcessSpec.getJvmHeapMemorySize(), is(JVM_HEAP_SIZE)); + validateFunc.accept(JobManagerProcessSpec); + } + + private void validateFailInConfigWithExplicitJvmHeap(Configuration customConfig) { + log.info("Validating failing in configuration with explicit jvm heap."); + Configuration config = configWithExplicitJvmHeap(); + config.addAll(customConfig); + validateFail(config); + } + + private void validateInConfigWithExplicitTotalFlinkMem( + Configuration customConfig, Consumer validateFunc) { + log.info("Validating in configuration with explicit total flink memory size."); + Configuration config = configWithExplicitTotalFlinkMem(); + config.addAll(customConfig); + JobManagerProcessSpec JobManagerProcessSpec = JobManagerProcessUtils.processSpecFromConfig(config); + assertThat(JobManagerProcessSpec.getTotalFlinkMemorySize(), is(TOTAL_FLINK_MEM_SIZE)); + validateFunc.accept(JobManagerProcessSpec); + } + + private void validateFailInConfigWithExplicitTotalFlinkMem(Configuration customConfig) { + log.info("Validating failing in configuration with explicit total flink memory size."); + Configuration config = configWithExplicitTotalFlinkMem(); + config.addAll(customConfig); + validateFail(config); + } + + private void validateInConfigWithExplicitTotalFlinkAndJvmHeapMem( + Configuration customConfig, + Consumer validateFunc) { + log.info("Validating in configuration with explicit total flink and jvm heap memory size."); + Configuration config = configWithExplicitTotalFlinkAndJvmHeapMem(); + config.addAll(customConfig); + JobManagerProcessSpec JobManagerProcessSpec = JobManagerProcessUtils.processSpecFromConfig(config); + assertThat(JobManagerProcessSpec.getTotalFlinkMemorySize(), is(TOTAL_FLINK_MEM_SIZE)); + assertThat(JobManagerProcessSpec.getJvmHeapMemorySize(), is(JVM_HEAP_SIZE)); + validateFunc.accept(JobManagerProcessSpec); + } + + private void validateFailInConfigWithExplicitTotalFlinkAndJvmHeapMem(Configuration customConfig) { + log.info("Validating failing in configuration with explicit total flink and jvm heap memory size."); + Configuration config = configWithExplicitTotalFlinkAndJvmHeapMem(); + config.addAll(customConfig); + validateFail(config); + } + + private void validateInConfigWithExplicitTotalProcessMem( + Configuration customConfig, Consumer validateFunc) { + log.info("Validating in configuration with explicit total process memory size."); + Configuration config = configWithExplicitTotalProcessMem(); + config.addAll(customConfig); + JobManagerProcessSpec JobManagerProcessSpec = JobManagerProcessUtils.processSpecFromConfig(config); + assertThat(JobManagerProcessSpec.getTotalProcessMemorySize(), is(TOTAL_PROCESS_MEM_SIZE)); + validateFunc.accept(JobManagerProcessSpec); + } + + private void validateFailInConfigWithExplicitTotalProcessMem(Configuration customConfig) { + log.info("Validating failing in configuration with explicit total process memory size."); + Configuration config = configWithExplicitTotalProcessMem(); + config.addAll(customConfig); + validateFail(config); + } + + @Override + protected void validateFail(Configuration config) { + try { + JobManagerProcessUtils.processSpecFromConfig(config); + fail("Configuration did not fail as expected."); + } catch (IllegalConfigurationException e) { + // expected + } + } + + private static Configuration configWithExplicitJvmHeap() { + Configuration conf = new Configuration(); + conf.set(JobManagerOptions.JVM_HEAP_MEMORY, JVM_HEAP_SIZE); + return conf; + } + + private static Configuration configWithExplicitTotalFlinkMem() { + Configuration conf = new Configuration(); + conf.set(JobManagerOptions.TOTAL_FLINK_MEMORY, TOTAL_FLINK_MEM_SIZE); + return conf; + } + + private static Configuration configWithExplicitTotalFlinkAndJvmHeapMem() { + Configuration conf = new Configuration(); + conf.set(JobManagerOptions.TOTAL_FLINK_MEMORY, TOTAL_FLINK_MEM_SIZE); + conf.set(JobManagerOptions.JVM_HEAP_MEMORY, JVM_HEAP_SIZE); + return conf; + } + + private static Configuration configWithExplicitTotalProcessMem() { + Configuration conf = new Configuration(); + conf.set(JobManagerOptions.TOTAL_PROCESS_MEMORY, TOTAL_PROCESS_MEM_SIZE); + return conf; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/config/memory/ProcessMemoryUtilsTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/config/memory/ProcessMemoryUtilsTestBase.java new file mode 100644 index 0000000000..ddd29e56ef --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/config/memory/ProcessMemoryUtilsTestBase.java @@ -0,0 +1,354 @@ +/* + * 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.runtime.util.config.memory; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ConfigurationUtils; +import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.core.testutils.CommonTestUtils; +import org.apache.flink.runtime.jobmanager.JobManagerProcessUtils; +import org.apache.flink.util.TestLogger; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; +import java.util.function.Consumer; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.junit.Assert.assertThat; + +/** + * Base test suite for JM/TM memory calculations to test common methods in {@link ProcessMemoryUtils}. + */ +@SuppressWarnings("AbstractClassExtendsConcreteClass") +public abstract class ProcessMemoryUtilsTestBase extends TestLogger { + private static Map oldEnvVariables; + + private final ProcessMemoryOptions options; + private final LegacyMemoryOptions legacyMemoryOptions; + private final ConfigOption newOptionForLegacyHeapOption; + + @SuppressWarnings("JUnitTestCaseWithNonTrivialConstructors") + protected ProcessMemoryUtilsTestBase( + ProcessMemoryOptions options, + LegacyMemoryOptions legacyMemoryOptions, + ConfigOption newOptionForLegacyHeapOption) { + this.options = checkNotNull(options); + this.legacyMemoryOptions = checkNotNull(legacyMemoryOptions); + this.newOptionForLegacyHeapOption = checkNotNull(newOptionForLegacyHeapOption); + } + + @Before + public void setup() { + oldEnvVariables = System.getenv(); + } + + @After + public void teardown() { + if (oldEnvVariables != null) { + CommonTestUtils.setEnv(oldEnvVariables, true); + } + } + + @Test + public void testGenerateJvmParameters() { + MemorySize heap = MemorySize.ofMebiBytes(1); + MemorySize directMemory = MemorySize.ofMebiBytes(2); + MemorySize metaspace = MemorySize.ofMebiBytes(3); + String jvmParamsStr = ProcessMemoryUtils.generateJvmParametersStr(new JvmArgTestingProcessMemorySpec( + heap, + directMemory, + metaspace + )); + Map configs = ConfigurationUtils.parseJvmArgString(jvmParamsStr); + + assertThat(MemorySize.parse(configs.get("-Xmx")), is(heap)); + assertThat(MemorySize.parse(configs.get("-Xms")), is(heap)); + assertThat(MemorySize.parse(configs.get("-XX:MaxDirectMemorySize=")), is(directMemory)); + assertThat(MemorySize.parse(configs.get("-XX:MaxMetaspaceSize=")), is(metaspace)); + } + + @Test + public void testConfigTotalFlinkMemory() { + MemorySize totalFlinkMemorySize = MemorySize.parse("1g"); + + Configuration conf = new Configuration(); + conf.set(options.getTotalFlinkMemoryOption(), totalFlinkMemorySize); + + T processSpec = processSpecFromConfig(conf); + assertThat(processSpec.getTotalFlinkMemorySize(), is(totalFlinkMemorySize)); + } + + @Test + public void testConfigTotalProcessMemorySize() { + MemorySize totalProcessMemorySize = MemorySize.parse("2g"); + + Configuration conf = new Configuration(); + conf.set(options.getTotalProcessMemoryOption(), totalProcessMemorySize); + + T processSpec = processSpecFromConfig(conf); + assertThat(processSpec.getTotalProcessMemorySize(), is(totalProcessMemorySize)); + } + + @Test + public void testExceptionShouldContainRequiredConfigOptions() { + try { + JobManagerProcessUtils.processSpecFromConfig(new Configuration()); + } catch (IllegalConfigurationException e) { + options.getRequiredFineGrainedOptions().forEach(option -> assertThat(e.getMessage(), containsString(option.key()))); + assertThat(e.getMessage(), containsString(options.getTotalFlinkMemoryOption().key())); + assertThat(e.getMessage(), containsString(options.getTotalProcessMemoryOption().key())); + } + } + + @Test + public void testConfigJvmMetaspaceSize() { + MemorySize jvmMetaspaceSize = MemorySize.parse("50m"); + + Configuration conf = new Configuration(); + conf.set(options.getJvmOptions().getJvmMetaspaceOption(), jvmMetaspaceSize); + + validateInAllConfigurations( + conf, + processSpec -> assertThat(processSpec.getJvmMetaspaceSize(), is(jvmMetaspaceSize))); + } + + @Test + public void testConfigJvmOverheadRange() { + MemorySize minSize = MemorySize.parse("50m"); + MemorySize maxSize = MemorySize.parse("200m"); + + Configuration conf = new Configuration(); + conf.set(options.getJvmOptions().getJvmOverheadMax(), maxSize); + conf.set(options.getJvmOptions().getJvmOverheadMin(), minSize); + + validateInAllConfigurations(conf, JobManagerProcessSpec -> { + assertThat(JobManagerProcessSpec.getJvmOverheadSize().getBytes(), + greaterThanOrEqualTo(minSize.getBytes())); + assertThat(JobManagerProcessSpec.getJvmOverheadSize().getBytes(), lessThanOrEqualTo(maxSize.getBytes())); + }); + } + + @Test + public void testConfigJvmOverheadRangeFailure() { + MemorySize minSize = MemorySize.parse("200m"); + MemorySize maxSize = MemorySize.parse("50m"); + + Configuration conf = new Configuration(); + conf.set(options.getJvmOptions().getJvmOverheadMax(), maxSize); + conf.set(options.getJvmOptions().getJvmOverheadMin(), minSize); + + validateFailInAllConfigurations(conf); + } + + @Test + public void testConfigJvmOverheadFraction() { + MemorySize minSize = MemorySize.ZERO; + MemorySize maxSize = MemorySize.parse("1t"); + @SuppressWarnings("MagicNumber") float fraction = 0.2f; + + Configuration conf = new Configuration(); + conf.set(options.getJvmOptions().getJvmOverheadMax(), maxSize); + conf.set(options.getJvmOptions().getJvmOverheadMin(), minSize); + conf.setFloat(options.getJvmOptions().getJvmOverheadFraction(), fraction); + + validateInAllConfigurations( + conf, + jobManagerProcessSpec -> assertThat( + jobManagerProcessSpec.getJvmOverheadSize(), + is(jobManagerProcessSpec.getTotalProcessMemorySize().multiply(fraction)))); + } + + @Test + public void testConfigJvmOverheadFractionFailureNegative() { + Configuration conf = new Configuration(); + //noinspection MagicNumber + conf.setFloat(options.getJvmOptions().getJvmOverheadFraction(), -0.1f); + validateFailInAllConfigurations(conf); + } + + @Test + public void testConfigJvmOverheadFractionFailureNoLessThanOne() { + Configuration conf = new Configuration(); + conf.setFloat(options.getJvmOptions().getJvmOverheadFraction(), 1.0f); + validateFailInAllConfigurations(conf); + } + + @Test + public void testConfigJvmOverheadDeriveFromProcessAndFlinkMemorySize() { + Configuration conf = new Configuration(); + conf.set(options.getTotalProcessMemoryOption(), MemorySize.parse("1000m")); + conf.set(options.getTotalFlinkMemoryOption(), MemorySize.parse("800m")); + conf.set(options.getJvmOptions().getJvmMetaspaceOption(), MemorySize.parse("100m")); + conf.set(options.getJvmOptions().getJvmOverheadMin(), MemorySize.parse("50m")); + conf.set(options.getJvmOptions().getJvmOverheadMax(), MemorySize.parse("200m")); + //noinspection MagicNumber + conf.set(options.getJvmOptions().getJvmOverheadFraction(), 0.5f); + + T jobManagerProcessSpec = processSpecFromConfig(conf); + assertThat(jobManagerProcessSpec.getJvmOverheadSize(), is(MemorySize.parse("100m"))); + } + + @Test + public void testConfigJvmOverheadDeriveFromProcessAndFlinkMemorySizeFailure() { + Configuration conf = new Configuration(); + conf.set(options.getTotalProcessMemoryOption(), MemorySize.parse("1000m")); + conf.set(options.getTotalFlinkMemoryOption(), MemorySize.parse("800m")); + conf.set(options.getJvmOptions().getJvmMetaspaceOption(), MemorySize.parse("100m")); + conf.set(options.getJvmOptions().getJvmOverheadMin(), MemorySize.parse("150m")); + conf.set(options.getJvmOptions().getJvmOverheadMax(), MemorySize.parse("200m")); + //noinspection MagicNumber + conf.set(options.getJvmOptions().getJvmOverheadFraction(), 0.5f); + + validateFail(conf); + } + + @Test + public void testConfigLegacyHeapSize() { + MemorySize legacyHeapSize = MemorySize.parse("1g"); + + Configuration conf = new Configuration(); + conf.set(legacyMemoryOptions.getHeap(), legacyHeapSize); + + testConfigLegacyHeapMemory(conf, legacyHeapSize); + } + + @Test + public void testConfigLegacyHeapMB() { + MemorySize jvmHeapSize = MemorySize.parse("1g"); + + Configuration conf = new Configuration(); + conf.set(legacyMemoryOptions.getHeapMb(), jvmHeapSize.getMebiBytes()); + + testConfigLegacyHeapMemory(conf, jvmHeapSize); + } + + @Test + public void testConfigLegacyHeapEnv() { + MemorySize jvmHeapSize = MemorySize.parse("1g"); + + Map env = new HashMap<>(); + env.put(legacyMemoryOptions.getEnvVar(), "1g"); + CommonTestUtils.setEnv(env); + + testConfigLegacyHeapMemory(new Configuration(), jvmHeapSize); + } + + @Test + public void testConfigBothNewOptionAndLegacyHeapSize() { + MemorySize newOptionValue = MemorySize.parse("1g"); + MemorySize legacyHeapSize = MemorySize.parse("2g"); + + Configuration conf = new Configuration(); + conf.set(getNewOptionForLegacyHeapOption(), newOptionValue); + conf.set(legacyMemoryOptions.getHeap(), legacyHeapSize); + + testConfigLegacyHeapMemory(conf, newOptionValue); + } + + private void testConfigLegacyHeapMemory(Configuration configuration, MemorySize expected) { + MemorySize newOptionValue = getConfigurationWithLegacyHeapSizeMappedToNewConfigOption(configuration) + .get(getNewOptionForLegacyHeapOption()); + assertThat(newOptionValue, is(expected)); + } + + @Test + public void testConfigTotalProcessMemoryAddUpFailure() { + MemorySize totalProcessMemory = MemorySize.parse("699m"); + MemorySize totalFlinkMemory = MemorySize.parse("500m"); + MemorySize jvmMetaspace = MemorySize.parse("100m"); + MemorySize jvmOverhead = MemorySize.parse("100m"); + + Configuration conf = new Configuration(); + conf.set(options.getTotalProcessMemoryOption(), totalProcessMemory); + conf.set(options.getTotalFlinkMemoryOption(), totalFlinkMemory); + conf.set(options.getJvmOptions().getJvmMetaspaceOption(), jvmMetaspace); + conf.set(options.getJvmOptions().getJvmOverheadMin(), jvmOverhead); + conf.set(options.getJvmOptions().getJvmOverheadMax(), jvmOverhead); + + validateFail(conf); + } + + protected abstract void validateInAllConfigurations(Configuration customConfig, Consumer validateFunc); + + protected abstract void validateFailInAllConfigurations(Configuration customConfig); + + protected abstract void validateFail(Configuration config); + + protected abstract T processSpecFromConfig(Configuration config); + + protected abstract Configuration getConfigurationWithLegacyHeapSizeMappedToNewConfigOption(Configuration config); + + protected ConfigOption getNewOptionForLegacyHeapOption() { + return newOptionForLegacyHeapOption; + } + + private static class JvmArgTestingProcessMemorySpec implements ProcessMemorySpec { + private static final long serialVersionUID = 2863985135320165745L; + + private final MemorySize heap; + private final MemorySize directMemory; + private final MemorySize metaspace; + + private JvmArgTestingProcessMemorySpec(MemorySize heap, MemorySize directMemory, MemorySize metaspace) { + this.heap = heap; + this.directMemory = directMemory; + this.metaspace = metaspace; + } + + @Override + public MemorySize getJvmHeapMemorySize() { + return heap; + } + + @Override + public MemorySize getJvmDirectMemorySize() { + return directMemory; + } + + @Override + public MemorySize getJvmMetaspaceSize() { + return metaspace; + } + + @Override + public MemorySize getJvmOverheadSize() { + throw new UnsupportedOperationException(); + } + + @Override + public MemorySize getTotalFlinkMemorySize() { + throw new UnsupportedOperationException(); + } + + @Override + public MemorySize getTotalProcessMemorySize() { + throw new UnsupportedOperationException(); + } + } +} -- Gitee From 69f20e929b9f6c58b3220dd41b41b21ad5f4bcdd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=B6rn=20Kottmann?= Date: Sun, 22 Mar 2020 19:01:05 +0100 Subject: [PATCH 312/885] [FLINK-15438][metrics][datadog] Report counter deltas and not totals The Flink semantics of a counter are not matching with the counters in DataDog. In Flink a counter counts the total of increment and decrement calls. In DataDog a counter is the number of increment and decrement calls during the reporting interval. --- .../apache/flink/metrics/datadog/DCounter.java | 16 +++++++++++++++- .../metrics/datadog/DatadogHttpReporter.java | 1 + 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/flink-metrics/flink-metrics-datadog/src/main/java/org/apache/flink/metrics/datadog/DCounter.java b/flink-metrics/flink-metrics-datadog/src/main/java/org/apache/flink/metrics/datadog/DCounter.java index 16ee0cf63d..549787cb9c 100644 --- a/flink-metrics/flink-metrics-datadog/src/main/java/org/apache/flink/metrics/datadog/DCounter.java +++ b/flink-metrics/flink-metrics-datadog/src/main/java/org/apache/flink/metrics/datadog/DCounter.java @@ -28,6 +28,9 @@ import java.util.List; public class DCounter extends DMetric { private final Counter counter; + private long lastReportCount = 0; + private long currentReportCount = 0; + public DCounter(Counter c, String metricName, String host, List tags, Clock clock) { super(MetricType.count, metricName, host, tags, clock); counter = c; @@ -36,9 +39,20 @@ public class DCounter extends DMetric { /** * Visibility of this method must not be changed * since we deliberately not map it to json object in a Datadog-defined format. + * + *

    Note: DataDog counters count the number of events during the reporting interval. + * + * @return the number of events since the last retrieval */ @Override public Number getMetricValue() { - return counter.getCount(); + long currentCount = counter.getCount(); + long difference = currentCount - lastReportCount; + currentReportCount = currentCount; + return difference; + } + + public void ackReport() { + lastReportCount = currentReportCount; } } diff --git a/flink-metrics/flink-metrics-datadog/src/main/java/org/apache/flink/metrics/datadog/DatadogHttpReporter.java b/flink-metrics/flink-metrics-datadog/src/main/java/org/apache/flink/metrics/datadog/DatadogHttpReporter.java index b0a2fb49eb..78a3d7234f 100644 --- a/flink-metrics/flink-metrics-datadog/src/main/java/org/apache/flink/metrics/datadog/DatadogHttpReporter.java +++ b/flink-metrics/flink-metrics-datadog/src/main/java/org/apache/flink/metrics/datadog/DatadogHttpReporter.java @@ -134,6 +134,7 @@ public class DatadogHttpReporter implements MetricReporter, Scheduled { try { client.send(request); + counters.values().forEach(DCounter::ackReport); LOGGER.debug("Reported series with size {}.", request.getSeries().size()); } catch (SocketTimeoutException e) { LOGGER.warn("Failed reporting metrics to Datadog because of socket timeout: {}", e.getMessage()); -- Gitee From 2db31ff808da1217ba04438af9530b08b3d33cb6 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 26 Mar 2020 10:08:28 +0100 Subject: [PATCH 313/885] [FLINK-15438][metrics][datadog] Add DCounterTest --- flink-metrics/flink-metrics-datadog/pom.xml | 6 ++ .../flink/metrics/datadog/DCounterTest.java | 65 +++++++++++++++++++ 2 files changed, 71 insertions(+) create mode 100644 flink-metrics/flink-metrics-datadog/src/test/java/org/apache/flink/metrics/datadog/DCounterTest.java diff --git a/flink-metrics/flink-metrics-datadog/pom.xml b/flink-metrics/flink-metrics-datadog/pom.xml index c2cd85dc6a..269b066da9 100644 --- a/flink-metrics/flink-metrics-datadog/pom.xml +++ b/flink-metrics/flink-metrics-datadog/pom.xml @@ -58,6 +58,12 @@ under the License. okhttp 3.7.0 + + + org.apache.flink + flink-test-utils-junit + ${project.version} + diff --git a/flink-metrics/flink-metrics-datadog/src/test/java/org/apache/flink/metrics/datadog/DCounterTest.java b/flink-metrics/flink-metrics-datadog/src/test/java/org/apache/flink/metrics/datadog/DCounterTest.java new file mode 100644 index 0000000000..9449279f43 --- /dev/null +++ b/flink-metrics/flink-metrics-datadog/src/test/java/org/apache/flink/metrics/datadog/DCounterTest.java @@ -0,0 +1,65 @@ +/* + * 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.metrics.datadog; + +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.SimpleCounter; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import java.util.Collections; + +import static org.junit.Assert.assertEquals; + +/** + * Tests for the {@link DCounter}. + */ +public class DCounterTest extends TestLogger { + + @Test + public void testGetMetricValue() { + final Counter backingCounter = new SimpleCounter(); + final DCounter counter = new DCounter(backingCounter, "counter", "localhost", Collections.emptyList(), () -> 0); + + // sane initial state + assertEquals(0L, counter.getMetricValue()); + counter.ackReport(); + assertEquals(0L, counter.getMetricValue()); + + // value is compared against initial state 0 + backingCounter.inc(10); + assertEquals(10L, counter.getMetricValue()); + + // last value was not acked, should still be compared against initial state 0 + backingCounter.inc(10); + assertEquals(20L, counter.getMetricValue()); + + // last value (20) acked, now target of comparison + counter.ackReport(); + assertEquals(0L, counter.getMetricValue()); + + // we now compare against the acked value + backingCounter.inc(10); + assertEquals(10L, counter.getMetricValue()); + + // properly handle decrements + backingCounter.dec(10); + assertEquals(0L, counter.getMetricValue()); + } +} -- Gitee From e0517e01ebfb247b1918fb88fe6eb588a2a4a4a7 Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Thu, 26 Mar 2020 09:27:14 +0100 Subject: [PATCH 314/885] [FLINK-16795][AZP] Increase e2e test timeout by 40min This closes #11522 --- tools/azure-pipelines/jobs-template.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tools/azure-pipelines/jobs-template.yml b/tools/azure-pipelines/jobs-template.yml index ffcdd95ecb..821bcabe70 100644 --- a/tools/azure-pipelines/jobs-template.yml +++ b/tools/azure-pipelines/jobs-template.yml @@ -143,7 +143,7 @@ jobs: #condition: or(eq(variables['MODE'], 'e2e'), eq(${{parameters.run_end_to_end}}, 'true')) # We are running this in a separate pool pool: ${{parameters.e2e_pool_definition}} - timeoutInMinutes: 200 + timeoutInMinutes: 240 cancelTimeoutInMinutes: 1 workspace: clean: all -- Gitee From 6b78fe00f6ded8e4f267138c4bbc32c04fc37b0b Mon Sep 17 00:00:00 2001 From: jingwen-ywb <53991270+jingwen-ywb@users.noreply.github.com> Date: Thu, 26 Mar 2020 18:48:45 +0800 Subject: [PATCH 315/885] [FLINK-16790][python][doc] Enables the interpretation of backslash escapes (#11526) --- docs/getting-started/walkthroughs/python_table_api.md | 2 +- docs/getting-started/walkthroughs/python_table_api.zh.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/getting-started/walkthroughs/python_table_api.md b/docs/getting-started/walkthroughs/python_table_api.md index 5630fc7668..60731a654e 100644 --- a/docs/getting-started/walkthroughs/python_table_api.md +++ b/docs/getting-started/walkthroughs/python_table_api.md @@ -131,7 +131,7 @@ t_env.execute("tutorial_job") Firstly, you need to prepare input data in the "/tmp/input" file. You can choose the following command line to prepare the input data: {% highlight bash %} -$ echo "flink\npyflink\nflink" > /tmp/input +$ echo -e "flink\npyflink\nflink" > /tmp/input {% endhighlight %} Next, you can run this example on the command line (Note: if the result file "/tmp/output" has already existed, you need to remove the file before running the example): diff --git a/docs/getting-started/walkthroughs/python_table_api.zh.md b/docs/getting-started/walkthroughs/python_table_api.zh.md index 794cbcf37d..a82ceb3b09 100644 --- a/docs/getting-started/walkthroughs/python_table_api.zh.md +++ b/docs/getting-started/walkthroughs/python_table_api.zh.md @@ -135,7 +135,7 @@ t_env.execute("python_job") 首先,你需要在文件 “/tmp/input” 中准备好输入数据。你可以选择通过如下命令准备输入数据: {% highlight bash %} -$ echo "flink\npyflink\nflink" > /tmp/input +$ echo -e "flink\npyflink\nflink" > /tmp/input {% endhighlight %} 接下来,可以在命令行中运行作业(假设作业名为WordCount.py)(注意:如果输出结果文件“/tmp/output”已经存在,你需要先删除文件,否则程序将无法正确运行起来): -- Gitee From cb67adcacd6e7270e4c5c5841d282f234d34ba9e Mon Sep 17 00:00:00 2001 From: Hequn Cheng Date: Thu, 26 Mar 2020 20:50:28 +0800 Subject: [PATCH 316/885] [FLINK-16671][python] Support for defining scopes on Python metric group (#11470) --- .../generated/python_configuration.html | 6 ++ .../pyflink/fn_execution/operations.py | 5 +- flink-python/pyflink/metrics/__init__.py | 21 +++++ flink-python/pyflink/metrics/metricbase.py | 78 +++++++++++++++++++ .../pyflink/metrics/tests/__init__.py | 17 ++++ .../pyflink/metrics/tests/test_metric.py | 40 ++++++++++ flink-python/pyflink/table/udf.py | 8 +- flink-python/setup.py | 2 + .../python/AbstractPythonFunctionRunner.java | 12 ++- .../org/apache/flink/python/PythonConfig.java | 10 +++ .../apache/flink/python/PythonOptions.java | 9 +++ .../python/metric/FlinkMetricContainer.java | 35 +++++++++ .../AbstractPythonFunctionOperator.java | 6 ++ .../python/PythonScalarFunctionFlatMap.java | 9 ++- .../BaseRowPythonScalarFunctionOperator.java | 3 +- .../scalar/PythonScalarFunctionOperator.java | 3 +- .../ArrowPythonScalarFunctionOperator.java | 3 +- ...eRowArrowPythonScalarFunctionOperator.java | 3 +- .../BaseRowPythonTableFunctionOperator.java | 3 +- .../table/PythonTableFunctionOperator.java | 3 +- ...AbstractPythonStatelessFunctionRunner.java | 6 +- ...ractGeneralPythonScalarFunctionRunner.java | 6 +- .../AbstractPythonScalarFunctionRunner.java | 7 +- .../BaseRowPythonScalarFunctionRunner.java | 6 +- .../scalar/PythonScalarFunctionRunner.java | 6 +- ...stractArrowPythonScalarFunctionRunner.java | 6 +- .../ArrowPythonScalarFunctionRunner.java | 6 +- ...aseRowArrowPythonScalarFunctionRunner.java | 6 +- .../AbstractPythonTableFunctionRunner.java | 6 +- .../BaseRowPythonTableFunctionRunner.java | 6 +- .../table/PythonTableFunctionRunner.java | 6 +- .../flink/python/PythonOptionsTest.java | 13 ++++ ...seRowPythonScalarFunctionOperatorTest.java | 4 +- .../PythonScalarFunctionOperatorTest.java | 4 +- ...ArrowPythonScalarFunctionOperatorTest.java | 4 +- ...ArrowPythonScalarFunctionOperatorTest.java | 4 +- ...BaseRowPythonScalarFunctionRunnerTest.java | 4 +- .../PythonScalarFunctionRunnerTest.java | 7 +- .../ArrowPythonScalarFunctionRunnerTest.java | 4 +- .../BaseRowPythonTableFunctionRunnerTest.java | 4 +- .../table/PythonTableFunctionRunnerTest.java | 13 +++- ...hroughArrowPythonScalarFunctionRunner.java | 11 ++- ...PassThroughPythonScalarFunctionRunner.java | 11 ++- .../table/runtime/utils/PythonTestUtils.java | 13 ++++ 44 files changed, 386 insertions(+), 53 deletions(-) create mode 100644 flink-python/pyflink/metrics/__init__.py create mode 100644 flink-python/pyflink/metrics/metricbase.py create mode 100644 flink-python/pyflink/metrics/tests/__init__.py create mode 100644 flink-python/pyflink/metrics/tests/test_metric.py create mode 100644 flink-python/src/main/java/org/apache/flink/python/metric/FlinkMetricContainer.java diff --git a/docs/_includes/generated/python_configuration.html b/docs/_includes/generated/python_configuration.html index 212cec2880..0219696577 100644 --- a/docs/_includes/generated/python_configuration.html +++ b/docs/_includes/generated/python_configuration.html @@ -38,5 +38,11 @@ String The amount of memory to be allocated by the Python framework. The sum of the value of this configuration and "python.fn-execution.buffer.memory.size" represents the total memory of a Python worker. The memory will be accounted as managed memory if the actual memory allocated to an operator is no less than the total memory of a Python worker. Otherwise, this configuration takes no effect. + +

    python.metric.enabled
    + true + Boolean + When it is false, metric for Python will be disabled. You can disable the metric to achieve better performance at some circumstance. + diff --git a/flink-python/pyflink/fn_execution/operations.py b/flink-python/pyflink/fn_execution/operations.py index ec27e72273..24ffabbc15 100644 --- a/flink-python/pyflink/fn_execution/operations.py +++ b/flink-python/pyflink/fn_execution/operations.py @@ -26,6 +26,8 @@ from apache_beam.utils.windowed_value import WindowedValue from pyflink.fn_execution import flink_fn_execution_pb2 from pyflink.serializers import PickleSerializer +from pyflink.table import FunctionContext +from pyflink.metrics.metricbase import GenericMetricGroup SCALAR_FUNCTION_URN = "flink:transform:scalar_function:v1" TABLE_FUNCTION_URN = "flink:transform:table_function:v1" @@ -45,8 +47,9 @@ class StatelessFunctionOperation(Operation): self.variable_dict = {} self.user_defined_funcs = [] self.func = self.generate_func(self.spec.serialized_fn) + base_metric_group = GenericMetricGroup(None, None) for user_defined_func in self.user_defined_funcs: - user_defined_func.open(None) + user_defined_func.open(FunctionContext(base_metric_group)) def setup(self): super(StatelessFunctionOperation, self).setup() diff --git a/flink-python/pyflink/metrics/__init__.py b/flink-python/pyflink/metrics/__init__.py new file mode 100644 index 0000000000..aa5368368e --- /dev/null +++ b/flink-python/pyflink/metrics/__init__.py @@ -0,0 +1,21 @@ +################################################################################ +# 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. +################################################################################ + +from pyflink.metrics.metricbase import MetricGroup + +__all__ = ["MetricGroup"] diff --git a/flink-python/pyflink/metrics/metricbase.py b/flink-python/pyflink/metrics/metricbase.py new file mode 100644 index 0000000000..f832587ddb --- /dev/null +++ b/flink-python/pyflink/metrics/metricbase.py @@ -0,0 +1,78 @@ +################################################################################ +# 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. +################################################################################ +import abc +from enum import Enum + + +class MetricGroup(abc.ABC): + + def add_group(self, name: str, extra: str = None) -> 'MetricGroup': + """ + Creates a new MetricGroup and adds it to this groups sub-groups. + + If extra is not None, creates a new key-value MetricGroup pair. The key group + is added to this groups sub-groups, while the value group is added to the key + group's sub-groups. This method returns the value group. + + The only difference between calling this method and + `group.add_group(key).add_group(value)` is that get_all_variables() + of the value group return an additional `""="value"` pair. + """ + pass + + +class MetricGroupType(Enum): + """ + Indicate the type of MetricGroup. + """ + generic = 0 + key = 1 + value = 2 + + +class GenericMetricGroup(MetricGroup): + + def __init__( + self, + parent, + name, + metric_group_type=MetricGroupType.generic): + self._parent = parent + self._sub_groups = [] + self._name = name + self._metric_group_type = metric_group_type + + def _add_group(self, name: str, metric_group_type) -> 'MetricGroup': + for group in self._sub_groups: + if name == group._name and metric_group_type == group._metric_group_type: + # we don't create same metric group repeatedly + return group + + sub_group = GenericMetricGroup( + self, + name, + metric_group_type) + self._sub_groups.append(sub_group) + return sub_group + + def add_group(self, name: str, extra: str = None) -> 'MetricGroup': + if extra is None: + return self._add_group(name, MetricGroupType.generic) + else: + return self._add_group(name, MetricGroupType.key)\ + ._add_group(extra, MetricGroupType.value) diff --git a/flink-python/pyflink/metrics/tests/__init__.py b/flink-python/pyflink/metrics/tests/__init__.py new file mode 100644 index 0000000000..65b48d4d79 --- /dev/null +++ b/flink-python/pyflink/metrics/tests/__init__.py @@ -0,0 +1,17 @@ +################################################################################ +# 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. +################################################################################ diff --git a/flink-python/pyflink/metrics/tests/test_metric.py b/flink-python/pyflink/metrics/tests/test_metric.py new file mode 100644 index 0000000000..bba239c330 --- /dev/null +++ b/flink-python/pyflink/metrics/tests/test_metric.py @@ -0,0 +1,40 @@ +################################################################################ +# 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. +################################################################################ + +import unittest +from pyflink.metrics.metricbase import GenericMetricGroup, MetricGroup + + +class MetricTests(unittest.TestCase): + + base_metric_group = GenericMetricGroup(None, None) + + @staticmethod + def print_metric_group_path(mg: MetricGroup) -> str: + if mg._parent is None: + return 'root' + else: + return MetricTests.print_metric_group_path(mg._parent) + '.' + mg._name + + def test_add_group(self): + new_group = MetricTests.base_metric_group.add_group('my_group') + self.assertEqual(MetricTests.print_metric_group_path(new_group), 'root.my_group') + + def test_add_group_with_variable(self): + new_group = MetricTests.base_metric_group.add_group('key', 'value') + self.assertEqual(MetricTests.print_metric_group_path(new_group), 'root.key.value') diff --git a/flink-python/pyflink/table/udf.py b/flink-python/pyflink/table/udf.py index d150fa1ae4..3a059fa38d 100644 --- a/flink-python/pyflink/table/udf.py +++ b/flink-python/pyflink/table/udf.py @@ -21,6 +21,7 @@ import functools import inspect from pyflink.java_gateway import get_gateway +from pyflink.metrics import MetricGroup from pyflink.table.types import DataType, _to_java_type from pyflink.util import utils @@ -33,7 +34,12 @@ class FunctionContext(object): user-defined function is executed. The information includes the metric group, and global job parameters, etc. """ - pass + + def __init__(self, base_metric_group): + self._base_metric_group = base_metric_group + + def get_metric_group(self) -> MetricGroup: + return self._base_metric_group class UserDefinedFunction(abc.ABC): diff --git a/flink-python/setup.py b/flink-python/setup.py index 5f5037639f..0d183428e9 100644 --- a/flink-python/setup.py +++ b/flink-python/setup.py @@ -175,6 +175,8 @@ run sdist. 'pyflink.dataset', 'pyflink.common', 'pyflink.fn_execution', + 'pyflink.metrics', + 'pyflink.ml', 'pyflink.lib', 'pyflink.opt', 'pyflink.conf', diff --git a/flink-python/src/main/java/org/apache/flink/python/AbstractPythonFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/python/AbstractPythonFunctionRunner.java index 84e783826c..a5afe8183c 100644 --- a/flink-python/src/main/java/org/apache/flink/python/AbstractPythonFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/python/AbstractPythonFunctionRunner.java @@ -23,6 +23,7 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.core.memory.ByteArrayOutputStreamWithPos; import org.apache.flink.core.memory.DataOutputViewStreamWrapper; import org.apache.flink.python.env.PythonEnvironmentManager; +import org.apache.flink.python.metric.FlinkMetricContainer; import org.apache.flink.util.Preconditions; import org.apache.beam.model.pipeline.v1.RunnerApi; @@ -43,6 +44,8 @@ import org.apache.beam.sdk.options.PortablePipelineOptions; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Struct; +import javax.annotation.Nullable; + import java.util.Map; /** @@ -116,17 +119,24 @@ public abstract class AbstractPythonFunctionRunner implements PythonFunction */ protected transient DataOutputViewStreamWrapper baosWrapper; + /** + * The flinkMetricContainer will be set to null if metric is configured to be turned off. + */ + @Nullable private FlinkMetricContainer flinkMetricContainer; + public AbstractPythonFunctionRunner( String taskName, FnDataReceiver resultReceiver, PythonEnvironmentManager environmentManager, StateRequestHandler stateRequestHandler, - Map jobOptions) { + Map jobOptions, + @Nullable FlinkMetricContainer flinkMetricContainer) { this.taskName = Preconditions.checkNotNull(taskName); this.resultReceiver = Preconditions.checkNotNull(resultReceiver); this.environmentManager = Preconditions.checkNotNull(environmentManager); this.stateRequestHandler = Preconditions.checkNotNull(stateRequestHandler); this.jobOptions = Preconditions.checkNotNull(jobOptions); + this.flinkMetricContainer = flinkMetricContainer; } @Override diff --git a/flink-python/src/main/java/org/apache/flink/python/PythonConfig.java b/flink-python/src/main/java/org/apache/flink/python/PythonConfig.java index 2f0292a30f..3c7461c00f 100644 --- a/flink-python/src/main/java/org/apache/flink/python/PythonConfig.java +++ b/flink-python/src/main/java/org/apache/flink/python/PythonConfig.java @@ -103,6 +103,11 @@ public class PythonConfig implements Serializable { @Nullable private final String pythonExec; + /** + * Whether metric is enabled. + */ + private final boolean metricEnabled; + public PythonConfig(Configuration config) { maxBundleSize = config.get(PythonOptions.MAX_BUNDLE_SIZE); maxBundleTimeMills = config.get(PythonOptions.MAX_BUNDLE_TIME_MILLS); @@ -114,6 +119,7 @@ public class PythonConfig implements Serializable { pythonRequirementsCacheDirInfo = config.getString(PYTHON_REQUIREMENTS_CACHE, null); pythonArchivesInfo = config.getString(PYTHON_ARCHIVES, null); pythonExec = config.getString(PYTHON_EXEC, null); + metricEnabled = config.getBoolean(PythonOptions.PYTHON_METRIC_ENABLED); } public int getMaxBundleSize() { @@ -155,4 +161,8 @@ public class PythonConfig implements Serializable { public Optional getPythonExec() { return Optional.ofNullable(pythonExec); } + + public boolean isMetricEnabled() { + return metricEnabled; + } } diff --git a/flink-python/src/main/java/org/apache/flink/python/PythonOptions.java b/flink-python/src/main/java/org/apache/flink/python/PythonOptions.java index 84ce525b7f..020a6390e8 100644 --- a/flink-python/src/main/java/org/apache/flink/python/PythonOptions.java +++ b/flink-python/src/main/java/org/apache/flink/python/PythonOptions.java @@ -81,4 +81,13 @@ public class PythonOptions { "buffer of a Python worker. The memory will be accounted as managed memory if the " + "actual memory allocated to an operator is no less than the total memory of a Python " + "worker. Otherwise, this configuration takes no effect."); + + /** + * The configuration to enable or disable metric for Python execution. + */ + public static final ConfigOption PYTHON_METRIC_ENABLED = ConfigOptions + .key("python.metric.enabled") + .defaultValue(true) + .withDescription("When it is false, metric for Python will be disabled. You can " + + "disable the metric to achieve better performance at some circumstance."); } diff --git a/flink-python/src/main/java/org/apache/flink/python/metric/FlinkMetricContainer.java b/flink-python/src/main/java/org/apache/flink/python/metric/FlinkMetricContainer.java new file mode 100644 index 0000000000..30a83f3821 --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/python/metric/FlinkMetricContainer.java @@ -0,0 +1,35 @@ +/* + * 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.python.metric; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.MetricGroup; + +/** + * Helper class for forwarding Python metrics to Java accumulators and metrics. + */ +@Internal +public class FlinkMetricContainer { + + private final MetricGroup baseMetricGroup; + + public FlinkMetricContainer(MetricGroup metricGroup) { + this.baseMetricGroup = metricGroup; + } +} diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractPythonFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractPythonFunctionOperator.java index 1f725262fc..c0655fe31e 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractPythonFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractPythonFunctionOperator.java @@ -28,6 +28,7 @@ import org.apache.flink.python.PythonOptions; import org.apache.flink.python.env.ProcessPythonEnvironmentManager; import org.apache.flink.python.env.PythonDependencyInfo; import org.apache.flink.python.env.PythonEnvironmentManager; +import org.apache.flink.python.metric.FlinkMetricContainer; import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.memory.MemoryReservationException; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; @@ -353,4 +354,9 @@ public abstract class AbstractPythonFunctionOperator "Execution type '%s' is not supported.", pythonEnv.getExecType())); } } + + protected FlinkMetricContainer getFlinkMetricContainer() { + return this.config.isMetricEnabled() ? + new FlinkMetricContainer(getRuntimeContext().getMetricGroup()) : null; + } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/functions/python/PythonScalarFunctionFlatMap.java b/flink-python/src/main/java/org/apache/flink/table/runtime/functions/python/PythonScalarFunctionFlatMap.java index 10cee8dd4a..e445b55cf7 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/functions/python/PythonScalarFunctionFlatMap.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/functions/python/PythonScalarFunctionFlatMap.java @@ -34,6 +34,7 @@ import org.apache.flink.python.PythonOptions; import org.apache.flink.python.env.ProcessPythonEnvironmentManager; import org.apache.flink.python.env.PythonDependencyInfo; import org.apache.flink.python.env.PythonEnvironmentManager; +import org.apache.flink.python.metric.FlinkMetricContainer; import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.table.functions.python.PythonEnv; import org.apache.flink.table.functions.python.PythonFunctionInfo; @@ -294,7 +295,8 @@ public final class PythonScalarFunctionFlatMap createPythonEnvironmentManager(), udfInputType, udfOutputType, - jobOptions); + jobOptions, + getFlinkMetricContainer()); } private PythonEnvironmentManager createPythonEnvironmentManager() throws IOException { @@ -357,4 +359,9 @@ public final class PythonScalarFunctionFlatMap super.close(); } } + + private FlinkMetricContainer getFlinkMetricContainer() { + return this.config.isMetricEnabled() ? + new FlinkMetricContainer(getRuntimeContext().getMetricGroup()) : null; + } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/BaseRowPythonScalarFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/BaseRowPythonScalarFunctionOperator.java index c845090dd2..f618f014ab 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/BaseRowPythonScalarFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/BaseRowPythonScalarFunctionOperator.java @@ -90,6 +90,7 @@ public class BaseRowPythonScalarFunctionOperator extends AbstractBaseRowPythonSc pythonEnvironmentManager, userDefinedFunctionInputType, userDefinedFunctionOutputType, - jobOptions); + jobOptions, + getFlinkMetricContainer()); } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperator.java index 8fa27bfa71..335f844c1e 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperator.java @@ -91,6 +91,7 @@ public class PythonScalarFunctionOperator extends AbstractRowPythonScalarFunctio pythonEnvironmentManager, userDefinedFunctionInputType, userDefinedFunctionOutputType, - jobOptions); + jobOptions, + getFlinkMetricContainer()); } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperator.java index aac9294e94..1b69db872c 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperator.java @@ -104,7 +104,8 @@ public class ArrowPythonScalarFunctionOperator extends AbstractRowPythonScalarFu userDefinedFunctionInputType, userDefinedFunctionOutputType, getPythonConfig().getMaxArrowBatchSize(), - jobOptions); + jobOptions, + getFlinkMetricContainer()); } @Override diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/BaseRowArrowPythonScalarFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/BaseRowArrowPythonScalarFunctionOperator.java index bab87e6e1f..3ef8acc650 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/BaseRowArrowPythonScalarFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/BaseRowArrowPythonScalarFunctionOperator.java @@ -103,7 +103,8 @@ public class BaseRowArrowPythonScalarFunctionOperator extends AbstractBaseRowPyt userDefinedFunctionInputType, userDefinedFunctionOutputType, getPythonConfig().getMaxArrowBatchSize(), - jobOptions); + jobOptions, + getFlinkMetricContainer()); } @Override diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/table/BaseRowPythonTableFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/table/BaseRowPythonTableFunctionOperator.java index f7dd371a73..359849eb75 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/table/BaseRowPythonTableFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/table/BaseRowPythonTableFunctionOperator.java @@ -127,7 +127,8 @@ public class BaseRowPythonTableFunctionOperator pythonEnvironmentManager, userDefinedFunctionInputType, userDefinedFunctionOutputType, - jobOptions); + jobOptions, + getFlinkMetricContainer()); } private Projection createUdtfInputProjection() { diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/table/PythonTableFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/table/PythonTableFunctionOperator.java index 418d271a79..757e6ce43f 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/table/PythonTableFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/table/PythonTableFunctionOperator.java @@ -141,6 +141,7 @@ public class PythonTableFunctionOperator extends AbstractPythonTableFunctionOper pythonEnvironmentManager, userDefinedFunctionInputType, userDefinedFunctionOutputType, - jobOptions); + jobOptions, + getFlinkMetricContainer()); } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/AbstractPythonStatelessFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/AbstractPythonStatelessFunctionRunner.java index 5155bf6e95..89b2068e8a 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/AbstractPythonStatelessFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/AbstractPythonStatelessFunctionRunner.java @@ -24,6 +24,7 @@ import org.apache.flink.fnexecution.v1.FlinkFnApi; import org.apache.flink.python.AbstractPythonFunctionRunner; import org.apache.flink.python.PythonFunctionRunner; import org.apache.flink.python.env.PythonEnvironmentManager; +import org.apache.flink.python.metric.FlinkMetricContainer; import org.apache.flink.table.functions.python.PythonFunctionInfo; import org.apache.flink.table.runtime.typeutils.PythonTypeUtils; import org.apache.flink.table.types.logical.LogicalType; @@ -88,8 +89,9 @@ public abstract class AbstractPythonStatelessFunctionRunner extends Abstract RowType inputType, RowType outputType, String functionUrn, - Map jobOptions) { - super(taskName, resultReceiver, environmentManager, StateRequestHandler.unsupported(), jobOptions); + Map jobOptions, + FlinkMetricContainer flinkMetricContainer) { + super(taskName, resultReceiver, environmentManager, StateRequestHandler.unsupported(), jobOptions, flinkMetricContainer); this.functionUrn = functionUrn; this.inputType = Preconditions.checkNotNull(inputType); this.outputType = Preconditions.checkNotNull(outputType); diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/AbstractGeneralPythonScalarFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/AbstractGeneralPythonScalarFunctionRunner.java index eededf3e27..ea60ffdb4a 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/AbstractGeneralPythonScalarFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/AbstractGeneralPythonScalarFunctionRunner.java @@ -22,6 +22,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.python.PythonFunctionRunner; import org.apache.flink.python.env.PythonEnvironmentManager; +import org.apache.flink.python.metric.FlinkMetricContainer; import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.table.functions.python.PythonFunctionInfo; import org.apache.flink.table.types.logical.RowType; @@ -54,8 +55,9 @@ public abstract class AbstractGeneralPythonScalarFunctionRunner extends Abst PythonEnvironmentManager environmentManager, RowType inputType, RowType outputType, - Map jobOptions) { - super(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, jobOptions); + Map jobOptions, + FlinkMetricContainer flinkMetricContainer) { + super(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, jobOptions, flinkMetricContainer); } @Override diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/AbstractPythonScalarFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/AbstractPythonScalarFunctionRunner.java index 9e0cdf8368..91e1fc823a 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/AbstractPythonScalarFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/AbstractPythonScalarFunctionRunner.java @@ -23,6 +23,7 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.fnexecution.v1.FlinkFnApi; import org.apache.flink.python.PythonFunctionRunner; import org.apache.flink.python.env.PythonEnvironmentManager; +import org.apache.flink.python.metric.FlinkMetricContainer; import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.table.functions.python.PythonFunctionInfo; import org.apache.flink.table.runtime.runners.python.AbstractPythonStatelessFunctionRunner; @@ -52,8 +53,9 @@ public abstract class AbstractPythonScalarFunctionRunner extends AbstractPyt PythonEnvironmentManager environmentManager, RowType inputType, RowType outputType, - Map jobOptions) { - super(taskName, resultReceiver, environmentManager, inputType, outputType, SCALAR_FUNCTION_URN, jobOptions); + Map jobOptions, + FlinkMetricContainer flinkMetricContainer) { + super(taskName, resultReceiver, environmentManager, inputType, outputType, SCALAR_FUNCTION_URN, jobOptions, flinkMetricContainer); this.scalarFunctions = Preconditions.checkNotNull(scalarFunctions); } @@ -63,6 +65,7 @@ public abstract class AbstractPythonScalarFunctionRunner extends AbstractPyt @VisibleForTesting public FlinkFnApi.UserDefinedFunctions getUserDefinedFunctionsProto() { FlinkFnApi.UserDefinedFunctions.Builder builder = FlinkFnApi.UserDefinedFunctions.newBuilder(); + // add udf proto for (PythonFunctionInfo pythonFunctionInfo : scalarFunctions) { builder.addUdfs(getUserDefinedFunctionProto(pythonFunctionInfo)); } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/BaseRowPythonScalarFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/BaseRowPythonScalarFunctionRunner.java index 4a2ee5544c..2dc565ecb9 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/BaseRowPythonScalarFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/BaseRowPythonScalarFunctionRunner.java @@ -21,6 +21,7 @@ package org.apache.flink.table.runtime.runners.python.scalar; import org.apache.flink.annotation.Internal; import org.apache.flink.python.PythonFunctionRunner; import org.apache.flink.python.env.PythonEnvironmentManager; +import org.apache.flink.python.metric.FlinkMetricContainer; import org.apache.flink.table.dataformat.BaseRow; import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.table.functions.python.PythonFunctionInfo; @@ -46,8 +47,9 @@ public class BaseRowPythonScalarFunctionRunner extends AbstractGeneralPythonScal PythonEnvironmentManager environmentManager, RowType inputType, RowType outputType, - Map jobOptions) { - super(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, jobOptions); + Map jobOptions, + FlinkMetricContainer flinkMetricContainer) { + super(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, jobOptions, flinkMetricContainer); } @Override diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/PythonScalarFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/PythonScalarFunctionRunner.java index 8e48b46ec9..171e0a37bc 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/PythonScalarFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/PythonScalarFunctionRunner.java @@ -22,6 +22,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.java.typeutils.runtime.RowSerializer; import org.apache.flink.python.PythonFunctionRunner; import org.apache.flink.python.env.PythonEnvironmentManager; +import org.apache.flink.python.metric.FlinkMetricContainer; import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.table.functions.python.PythonFunctionInfo; import org.apache.flink.table.runtime.typeutils.PythonTypeUtils; @@ -46,8 +47,9 @@ public class PythonScalarFunctionRunner extends AbstractGeneralPythonScalarFunct PythonEnvironmentManager environmentManager, RowType inputType, RowType outputType, - Map jobOptions) { - super(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, jobOptions); + Map jobOptions, + FlinkMetricContainer flinkMetricContainer) { + super(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, jobOptions, flinkMetricContainer); } @Override diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/AbstractArrowPythonScalarFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/AbstractArrowPythonScalarFunctionRunner.java index dda5395db1..965cacf341 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/AbstractArrowPythonScalarFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/AbstractArrowPythonScalarFunctionRunner.java @@ -22,6 +22,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.python.PythonFunctionRunner; import org.apache.flink.python.env.PythonEnvironmentManager; +import org.apache.flink.python.metric.FlinkMetricContainer; import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.table.functions.python.PythonFunctionInfo; import org.apache.flink.table.runtime.arrow.ArrowUtils; @@ -100,8 +101,9 @@ public abstract class AbstractArrowPythonScalarFunctionRunner extends Abstra RowType inputType, RowType outputType, int maxArrowBatchSize, - Map jobOptions) { - super(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, jobOptions); + Map jobOptions, + FlinkMetricContainer flinkMetricContainer) { + super(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, jobOptions, flinkMetricContainer); this.maxArrowBatchSize = maxArrowBatchSize; } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/ArrowPythonScalarFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/ArrowPythonScalarFunctionRunner.java index 09f8a12da4..28014fbe69 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/ArrowPythonScalarFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/ArrowPythonScalarFunctionRunner.java @@ -21,6 +21,7 @@ package org.apache.flink.table.runtime.runners.python.scalar.arrow; import org.apache.flink.annotation.Internal; import org.apache.flink.python.PythonFunctionRunner; import org.apache.flink.python.env.PythonEnvironmentManager; +import org.apache.flink.python.metric.FlinkMetricContainer; import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.table.functions.python.PythonFunctionInfo; import org.apache.flink.table.runtime.arrow.ArrowUtils; @@ -47,8 +48,9 @@ public class ArrowPythonScalarFunctionRunner extends AbstractArrowPythonScalarFu RowType inputType, RowType outputType, int maxArrowBatchSize, - Map jobOptions) { - super(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, maxArrowBatchSize, jobOptions); + Map jobOptions, + FlinkMetricContainer flinkMetricContainer) { + super(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, maxArrowBatchSize, jobOptions, flinkMetricContainer); } @Override diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/BaseRowArrowPythonScalarFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/BaseRowArrowPythonScalarFunctionRunner.java index 848660b9d2..01de76514d 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/BaseRowArrowPythonScalarFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/BaseRowArrowPythonScalarFunctionRunner.java @@ -21,6 +21,7 @@ package org.apache.flink.table.runtime.runners.python.scalar.arrow; import org.apache.flink.annotation.Internal; import org.apache.flink.python.PythonFunctionRunner; import org.apache.flink.python.env.PythonEnvironmentManager; +import org.apache.flink.python.metric.FlinkMetricContainer; import org.apache.flink.table.dataformat.BaseRow; import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.table.functions.python.PythonFunctionInfo; @@ -47,8 +48,9 @@ public class BaseRowArrowPythonScalarFunctionRunner extends AbstractArrowPythonS RowType inputType, RowType outputType, int maxBatchSize, - Map jobOptions) { - super(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, maxBatchSize, jobOptions); + Map jobOptions, + FlinkMetricContainer flinkMetricContainer) { + super(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, maxBatchSize, jobOptions, flinkMetricContainer); } @Override diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/table/AbstractPythonTableFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/table/AbstractPythonTableFunctionRunner.java index 962f556ba2..50136c7d35 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/table/AbstractPythonTableFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/table/AbstractPythonTableFunctionRunner.java @@ -24,6 +24,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.fnexecution.v1.FlinkFnApi; import org.apache.flink.python.PythonFunctionRunner; import org.apache.flink.python.env.PythonEnvironmentManager; +import org.apache.flink.python.metric.FlinkMetricContainer; import org.apache.flink.table.functions.TableFunction; import org.apache.flink.table.functions.python.PythonFunctionInfo; import org.apache.flink.table.runtime.runners.python.AbstractPythonStatelessFunctionRunner; @@ -61,8 +62,9 @@ public abstract class AbstractPythonTableFunctionRunner extends AbstractPyth PythonEnvironmentManager environmentManager, RowType inputType, RowType outputType, - Map jobOptions) { - super(taskName, resultReceiver, environmentManager, inputType, outputType, TABLE_FUNCTION_URN, jobOptions); + Map jobOptions, + FlinkMetricContainer flinkMetricContainer) { + super(taskName, resultReceiver, environmentManager, inputType, outputType, TABLE_FUNCTION_URN, jobOptions, flinkMetricContainer); this.tableFunction = Preconditions.checkNotNull(tableFunction); } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/table/BaseRowPythonTableFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/table/BaseRowPythonTableFunctionRunner.java index 3ff6ca4598..0b77732ee7 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/table/BaseRowPythonTableFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/table/BaseRowPythonTableFunctionRunner.java @@ -21,6 +21,7 @@ package org.apache.flink.table.runtime.runners.python.table; import org.apache.flink.annotation.Internal; import org.apache.flink.python.PythonFunctionRunner; import org.apache.flink.python.env.PythonEnvironmentManager; +import org.apache.flink.python.metric.FlinkMetricContainer; import org.apache.flink.table.dataformat.BaseRow; import org.apache.flink.table.functions.TableFunction; import org.apache.flink.table.functions.python.PythonFunctionInfo; @@ -45,8 +46,9 @@ public class BaseRowPythonTableFunctionRunner extends AbstractPythonTableFunctio PythonEnvironmentManager environmentManager, RowType inputType, RowType outputType, - Map jobOptions) { - super(taskName, resultReceiver, tableFunction, environmentManager, inputType, outputType, jobOptions); + Map jobOptions, + FlinkMetricContainer flinkMetricContainer) { + super(taskName, resultReceiver, tableFunction, environmentManager, inputType, outputType, jobOptions, flinkMetricContainer); } @Override diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/table/PythonTableFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/table/PythonTableFunctionRunner.java index 48fa400b92..c83ab79eac 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/table/PythonTableFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/table/PythonTableFunctionRunner.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.typeutils.runtime.RowSerializer; import org.apache.flink.python.PythonFunctionRunner; import org.apache.flink.python.env.PythonEnvironmentManager; +import org.apache.flink.python.metric.FlinkMetricContainer; import org.apache.flink.table.functions.TableFunction; import org.apache.flink.table.functions.python.PythonFunctionInfo; import org.apache.flink.table.runtime.typeutils.PythonTypeUtils; @@ -47,8 +48,9 @@ public class PythonTableFunctionRunner extends AbstractPythonTableFunctionRunner PythonEnvironmentManager environmentManager, RowType inputType, RowType outputType, - Map jobOptions) { - super(taskName, resultReceiver, tableFunction, environmentManager, inputType, outputType, jobOptions); + Map jobOptions, + FlinkMetricContainer flinkMetricContainer) { + super(taskName, resultReceiver, tableFunction, environmentManager, inputType, outputType, jobOptions, flinkMetricContainer); } @Override diff --git a/flink-python/src/test/java/org/apache/flink/python/PythonOptionsTest.java b/flink-python/src/test/java/org/apache/flink/python/PythonOptionsTest.java index ddea89e84c..1e31b1cbe1 100644 --- a/flink-python/src/test/java/org/apache/flink/python/PythonOptionsTest.java +++ b/flink-python/src/test/java/org/apache/flink/python/PythonOptionsTest.java @@ -95,4 +95,17 @@ public class PythonOptionsTest { final int actualArrowBatchSize = configuration.getInteger(PythonOptions.MAX_ARROW_BATCH_SIZE); assertThat(actualArrowBatchSize, is(equalTo(expectedArrowBatchSize))); } + + @Test + public void testPythonMetricEnabled() { + final Configuration configuration = new Configuration(); + final boolean isMetricEnabled = configuration.getBoolean(PythonOptions.PYTHON_METRIC_ENABLED); + assertThat(isMetricEnabled, is(equalTo(PythonOptions.PYTHON_METRIC_ENABLED.defaultValue()))); + + final boolean expectedIsMetricEnabled = false; + configuration.setBoolean(PythonOptions.PYTHON_METRIC_ENABLED, false); + + final boolean actualIsMetricEnabled = configuration.getBoolean(PythonOptions.PYTHON_METRIC_ENABLED); + assertThat(actualIsMetricEnabled, is(equalTo(expectedIsMetricEnabled))); + } } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/BaseRowPythonScalarFunctionOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/BaseRowPythonScalarFunctionOperatorTest.java index ef4e618280..d46ecf1e22 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/BaseRowPythonScalarFunctionOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/BaseRowPythonScalarFunctionOperatorTest.java @@ -35,6 +35,7 @@ import org.apache.flink.table.runtime.typeutils.BaseRowSerializer; import org.apache.flink.table.runtime.typeutils.PythonTypeUtils; import org.apache.flink.table.runtime.util.BaseRowHarnessAssertor; import org.apache.flink.table.runtime.utils.PassThroughPythonScalarFunctionRunner; +import org.apache.flink.table.runtime.utils.PythonTestUtils; import org.apache.flink.table.types.logical.RowType; import org.apache.beam.sdk.fn.data.FnDataReceiver; @@ -125,7 +126,8 @@ public class BaseRowPythonScalarFunctionOperatorTest pythonEnvironmentManager, userDefinedFunctionInputType, userDefinedFunctionOutputType, - jobOptions) { + jobOptions, + PythonTestUtils.createMockFlinkMetricContainer()) { @Override public TypeSerializer getInputTypeSerializer() { return (BaseRowSerializer) PythonTypeUtils.toBlinkTypeSerializer(getInputType()); diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperatorTest.java index 459feb2a00..cc6c8be924 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperatorTest.java @@ -30,6 +30,7 @@ import org.apache.flink.table.functions.python.PythonFunctionInfo; import org.apache.flink.table.runtime.types.CRow; import org.apache.flink.table.runtime.typeutils.PythonTypeUtils; import org.apache.flink.table.runtime.utils.PassThroughPythonScalarFunctionRunner; +import org.apache.flink.table.runtime.utils.PythonTestUtils; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.types.Row; @@ -101,7 +102,8 @@ public class PythonScalarFunctionOperatorTest extends PythonScalarFunctionOperat pythonEnvironmentManager, userDefinedFunctionInputType, userDefinedFunctionOutputType, - jobOptions) { + jobOptions, + PythonTestUtils.createMockFlinkMetricContainer()) { @Override public TypeSerializer getInputTypeSerializer() { return (RowSerializer) PythonTypeUtils.toFlinkTypeSerializer(getInputType()); diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperatorTest.java index f4e6b2e624..4316bc9853 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperatorTest.java @@ -32,6 +32,7 @@ import org.apache.flink.table.runtime.operators.python.scalar.AbstractPythonScal import org.apache.flink.table.runtime.operators.python.scalar.PythonScalarFunctionOperatorTestBase; import org.apache.flink.table.runtime.types.CRow; import org.apache.flink.table.runtime.utils.PassThroughArrowPythonScalarFunctionRunner; +import org.apache.flink.table.runtime.utils.PythonTestUtils; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.types.Row; @@ -100,7 +101,8 @@ public class ArrowPythonScalarFunctionOperatorTest extends PythonScalarFunctionO userDefinedFunctionInputType, userDefinedFunctionOutputType, getPythonConfig().getMaxArrowBatchSize(), - jobOptions) { + jobOptions, + PythonTestUtils.createMockFlinkMetricContainer()) { @Override public ArrowWriter createArrowWriter() { return ArrowUtils.createRowArrowWriter(root, getInputType()); diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/BaseRowArrowPythonScalarFunctionOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/BaseRowArrowPythonScalarFunctionOperatorTest.java index 6fae4c953e..331333e749 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/BaseRowArrowPythonScalarFunctionOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/BaseRowArrowPythonScalarFunctionOperatorTest.java @@ -38,6 +38,7 @@ import org.apache.flink.table.runtime.operators.python.scalar.PythonScalarFuncti import org.apache.flink.table.runtime.typeutils.BaseRowSerializer; import org.apache.flink.table.runtime.util.BaseRowHarnessAssertor; import org.apache.flink.table.runtime.utils.PassThroughArrowPythonScalarFunctionRunner; +import org.apache.flink.table.runtime.utils.PythonTestUtils; import org.apache.flink.table.types.logical.RowType; import org.apache.beam.sdk.fn.data.FnDataReceiver; @@ -122,7 +123,8 @@ public class BaseRowArrowPythonScalarFunctionOperatorTest userDefinedFunctionInputType, userDefinedFunctionOutputType, getPythonConfig().getMaxArrowBatchSize(), - jobOptions) { + jobOptions, + PythonTestUtils.createMockFlinkMetricContainer()) { @Override public ArrowWriter createArrowWriter() { return ArrowUtils.createBaseRowArrowWriter(root, getInputType()); diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/scalar/BaseRowPythonScalarFunctionRunnerTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/scalar/BaseRowPythonScalarFunctionRunnerTest.java index 7f42b525ca..4aab15da3c 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/scalar/BaseRowPythonScalarFunctionRunnerTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/scalar/BaseRowPythonScalarFunctionRunnerTest.java @@ -25,6 +25,7 @@ import org.apache.flink.python.env.PythonEnvironmentManager; import org.apache.flink.table.dataformat.BaseRow; import org.apache.flink.table.functions.python.PythonFunctionInfo; import org.apache.flink.table.runtime.typeutils.BaseRowSerializer; +import org.apache.flink.table.runtime.utils.PythonTestUtils; import org.apache.flink.table.types.logical.RowType; import org.apache.beam.sdk.fn.data.FnDataReceiver; @@ -90,6 +91,7 @@ public class BaseRowPythonScalarFunctionRunnerTest extends AbstractPythonScalarF environmentManager, inputType, outputType, - Collections.emptyMap()); + Collections.emptyMap(), + PythonTestUtils.createMockFlinkMetricContainer()); } } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/scalar/PythonScalarFunctionRunnerTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/scalar/PythonScalarFunctionRunnerTest.java index 66f1eed35a..6cedf82e31 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/scalar/PythonScalarFunctionRunnerTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/scalar/PythonScalarFunctionRunnerTest.java @@ -28,6 +28,7 @@ import org.apache.flink.python.env.PythonEnvironmentManager; import org.apache.flink.table.functions.python.PythonFunctionInfo; import org.apache.flink.table.runtime.typeutils.PythonTypeUtils; import org.apache.flink.table.runtime.utils.PassThroughPythonScalarFunctionRunner; +import org.apache.flink.table.runtime.utils.PythonTestUtils; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.types.Row; @@ -211,7 +212,8 @@ public class PythonScalarFunctionRunnerTest extends AbstractPythonScalarFunction environmentManager, inputType, outputType, - Collections.emptyMap()); + Collections.emptyMap(), + PythonTestUtils.createMockFlinkMetricContainer()); } private AbstractGeneralPythonScalarFunctionRunner createUDFRunner( @@ -238,7 +240,8 @@ public class PythonScalarFunctionRunnerTest extends AbstractPythonScalarFunction rowType, rowType, Collections.emptyMap(), - jobBundleFactory) { + jobBundleFactory, + PythonTestUtils.createMockFlinkMetricContainer()) { @Override public TypeSerializer getInputTypeSerializer() { return (RowSerializer) PythonTypeUtils.toFlinkTypeSerializer(getInputType()); diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/ArrowPythonScalarFunctionRunnerTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/ArrowPythonScalarFunctionRunnerTest.java index 9d802e1497..06b28d6efa 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/ArrowPythonScalarFunctionRunnerTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/ArrowPythonScalarFunctionRunnerTest.java @@ -28,6 +28,7 @@ import org.apache.flink.table.runtime.arrow.writers.ArrowFieldWriter; import org.apache.flink.table.runtime.arrow.writers.BigIntWriter; import org.apache.flink.table.runtime.runners.python.scalar.AbstractPythonScalarFunctionRunnerTest; import org.apache.flink.table.runtime.utils.PassThroughArrowPythonScalarFunctionRunner; +import org.apache.flink.table.runtime.utils.PythonTestUtils; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.types.Row; @@ -174,7 +175,8 @@ public class ArrowPythonScalarFunctionRunnerTest extends AbstractPythonScalarFun outputType, maxArrowBatchSize, Collections.emptyMap(), - jobBundleFactory) { + jobBundleFactory, + PythonTestUtils.createMockFlinkMetricContainer()) { @Override public ArrowWriter createArrowWriter() { return ArrowUtils.createRowArrowWriter(root, getInputType()); diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/table/BaseRowPythonTableFunctionRunnerTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/table/BaseRowPythonTableFunctionRunnerTest.java index 23c6533b47..fcf0806b55 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/table/BaseRowPythonTableFunctionRunnerTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/table/BaseRowPythonTableFunctionRunnerTest.java @@ -25,6 +25,7 @@ import org.apache.flink.python.env.PythonEnvironmentManager; import org.apache.flink.table.dataformat.BaseRow; import org.apache.flink.table.functions.python.PythonFunctionInfo; import org.apache.flink.table.runtime.typeutils.serializers.python.BaseRowSerializer; +import org.apache.flink.table.runtime.utils.PythonTestUtils; import org.apache.flink.table.types.logical.RowType; import org.apache.beam.sdk.fn.data.FnDataReceiver; @@ -75,6 +76,7 @@ public class BaseRowPythonTableFunctionRunnerTest extends AbstractPythonTableFun environmentManager, inputType, outputType, - Collections.emptyMap()); + Collections.emptyMap(), + PythonTestUtils.createMockFlinkMetricContainer()); } } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/table/PythonTableFunctionRunnerTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/table/PythonTableFunctionRunnerTest.java index 5c396afa8d..0fce6df0b4 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/table/PythonTableFunctionRunnerTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/table/PythonTableFunctionRunnerTest.java @@ -24,8 +24,10 @@ import org.apache.flink.fnexecution.v1.FlinkFnApi; import org.apache.flink.python.env.ProcessPythonEnvironmentManager; import org.apache.flink.python.env.PythonDependencyInfo; import org.apache.flink.python.env.PythonEnvironmentManager; +import org.apache.flink.python.metric.FlinkMetricContainer; import org.apache.flink.table.functions.python.PythonFunctionInfo; import org.apache.flink.table.runtime.runners.python.scalar.AbstractPythonScalarFunctionRunnerTest; +import org.apache.flink.table.runtime.utils.PythonTestUtils; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.types.Row; @@ -97,7 +99,8 @@ public class PythonTableFunctionRunnerTest extends AbstractPythonTableFunctionRu environmentManager, inputType, outputType, - Collections.emptyMap()); + Collections.emptyMap(), + PythonTestUtils.createMockFlinkMetricContainer()); } private AbstractPythonTableFunctionRunner createUDTFRunner( @@ -121,7 +124,8 @@ public class PythonTableFunctionRunnerTest extends AbstractPythonTableFunctionRu environmentManager, rowType, rowType, - jobBundleFactory); + jobBundleFactory, + PythonTestUtils.createMockFlinkMetricContainer()); } private static class PythonTableFunctionRunnerTestHarness extends PythonTableFunctionRunner { @@ -135,8 +139,9 @@ public class PythonTableFunctionRunnerTest extends AbstractPythonTableFunctionRu PythonEnvironmentManager environmentManager, RowType inputType, RowType outputType, - JobBundleFactory jobBundleFactory) { - super(taskName, resultReceiver, tableFunction, environmentManager, inputType, outputType, Collections.emptyMap()); + JobBundleFactory jobBundleFactory, + FlinkMetricContainer flinkMetricContainer) { + super(taskName, resultReceiver, tableFunction, environmentManager, inputType, outputType, Collections.emptyMap(), flinkMetricContainer); this.jobBundleFactory = jobBundleFactory; } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughArrowPythonScalarFunctionRunner.java b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughArrowPythonScalarFunctionRunner.java index ed66123502..34c30ef1bb 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughArrowPythonScalarFunctionRunner.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughArrowPythonScalarFunctionRunner.java @@ -19,6 +19,7 @@ package org.apache.flink.table.runtime.utils; import org.apache.flink.python.env.PythonEnvironmentManager; +import org.apache.flink.python.metric.FlinkMetricContainer; import org.apache.flink.table.functions.python.PythonFunctionInfo; import org.apache.flink.table.runtime.runners.python.scalar.arrow.AbstractArrowPythonScalarFunctionRunner; import org.apache.flink.table.types.logical.RowType; @@ -51,8 +52,9 @@ public abstract class PassThroughArrowPythonScalarFunctionRunner extends Abs RowType inputType, RowType outputType, int maxArrowBatchSize, - Map jobOptions) { - this(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, maxArrowBatchSize, jobOptions, createMockJobBundleFactory()); + Map jobOptions, + FlinkMetricContainer flinkMetricContainer) { + this(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, maxArrowBatchSize, jobOptions, createMockJobBundleFactory(), flinkMetricContainer); } public PassThroughArrowPythonScalarFunctionRunner( @@ -64,8 +66,9 @@ public abstract class PassThroughArrowPythonScalarFunctionRunner extends Abs RowType outputType, int maxArrowBatchSize, Map jobOptions, - JobBundleFactory jobBundleFactory) { - super(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, maxArrowBatchSize, jobOptions); + JobBundleFactory jobBundleFactory, + FlinkMetricContainer flinkMetricContainer) { + super(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, maxArrowBatchSize, jobOptions, flinkMetricContainer); this.jobBundleFactory = jobBundleFactory; this.bufferedInputs = new ArrayList<>(); } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonScalarFunctionRunner.java b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonScalarFunctionRunner.java index 492eef6c4c..e112902567 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonScalarFunctionRunner.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonScalarFunctionRunner.java @@ -19,6 +19,7 @@ package org.apache.flink.table.runtime.utils; import org.apache.flink.python.env.PythonEnvironmentManager; +import org.apache.flink.python.metric.FlinkMetricContainer; import org.apache.flink.table.functions.python.PythonFunctionInfo; import org.apache.flink.table.runtime.runners.python.scalar.AbstractGeneralPythonScalarFunctionRunner; import org.apache.flink.table.types.logical.RowType; @@ -50,8 +51,9 @@ public abstract class PassThroughPythonScalarFunctionRunner extends Abstract PythonEnvironmentManager environmentManager, RowType inputType, RowType outputType, - Map jobOptions) { - this(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, jobOptions, createMockJobBundleFactory()); + Map jobOptions, + FlinkMetricContainer flinkMetricContainer) { + this(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, jobOptions, createMockJobBundleFactory(), flinkMetricContainer); } public PassThroughPythonScalarFunctionRunner( @@ -62,8 +64,9 @@ public abstract class PassThroughPythonScalarFunctionRunner extends Abstract RowType inputType, RowType outputType, Map jobOptions, - JobBundleFactory jobBundleFactory) { - super(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, jobOptions); + JobBundleFactory jobBundleFactory, + FlinkMetricContainer flinkMetricContainer) { + super(taskName, resultReceiver, scalarFunctions, environmentManager, inputType, outputType, jobOptions, flinkMetricContainer); this.jobBundleFactory = jobBundleFactory; this.bufferedInputs = new ArrayList<>(); } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PythonTestUtils.java b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PythonTestUtils.java index b7f4d2b5bf..df8edccecc 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PythonTestUtils.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PythonTestUtils.java @@ -18,6 +18,11 @@ package org.apache.flink.table.runtime.utils; +import org.apache.flink.python.metric.FlinkMetricContainer; +import org.apache.flink.runtime.metrics.NoOpMetricRegistry; +import org.apache.flink.runtime.metrics.groups.GenericMetricGroup; +import org.apache.flink.runtime.metrics.groups.MetricGroupTest; + import org.apache.beam.runners.fnexecution.control.JobBundleFactory; import org.apache.beam.runners.fnexecution.control.RemoteBundle; import org.apache.beam.runners.fnexecution.control.StageBundleFactory; @@ -52,4 +57,12 @@ public final class PythonTestUtils { when(remoteBundleSpy.getInputReceivers()).thenReturn(inputReceivers); return jobBundleFactorySpy; } + + public static FlinkMetricContainer createMockFlinkMetricContainer() { + return new FlinkMetricContainer( + new GenericMetricGroup( + NoOpMetricRegistry.INSTANCE, + new MetricGroupTest.DummyAbstractMetricGroup(NoOpMetricRegistry.INSTANCE), + "root")); + } } -- Gitee From cd5e3f47c4877f2d271aa69989e664a45f2fab5a Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Thu, 26 Mar 2020 13:48:47 +0100 Subject: [PATCH 317/885] [FLINK-16805][e2e] Remove pre-commit profile activation --- azure-pipelines.yml | 2 +- tools/azure-pipelines/build-apache-repo.yml | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/azure-pipelines.yml b/azure-pipelines.yml index 0aa35500e0..89a5089bd3 100644 --- a/azure-pipelines.yml +++ b/azure-pipelines.yml @@ -63,7 +63,7 @@ jobs: vmImage: 'ubuntu-latest' e2e_pool_definition: vmImage: 'ubuntu-16.04' - environment: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11 -Pe2e-hadoop -Pe2e-pre-commit" + environment: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11 -Pe2e-hadoop" run_end_to_end: false container: flink-build-container jdk: jdk8 diff --git a/tools/azure-pipelines/build-apache-repo.yml b/tools/azure-pipelines/build-apache-repo.yml index e227816916..1906fcaf94 100644 --- a/tools/azure-pipelines/build-apache-repo.yml +++ b/tools/azure-pipelines/build-apache-repo.yml @@ -63,7 +63,7 @@ stages: name: Default e2e_pool_definition: vmImage: 'ubuntu-16.04' - environment: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11 -Pe2e-hadoop -Pe2e-pre-commit" + environment: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11 -Pe2e-hadoop" run_end_to_end: false container: flink-build-container jdk: jdk8 @@ -80,7 +80,7 @@ stages: name: Default e2e_pool_definition: vmImage: 'ubuntu-16.04' - environment: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.4.1 -Pskip-hive-tests -Pe2e-hadoop -Pe2e-pre-commit" + environment: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.4.1 -Pskip-hive-tests -Pe2e-hadoop" run_end_to_end: true container: flink-build-container jdk: jdk8 @@ -91,7 +91,7 @@ stages: name: Default e2e_pool_definition: vmImage: 'ubuntu-16.04' - environment: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.12 -Phive-1.2.1 -Pe2e-hadoop -Pe2e-pre-commit" + environment: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.12 -Phive-1.2.1 -Pe2e-hadoop" run_end_to_end: true container: flink-build-container jdk: jdk8 @@ -102,7 +102,7 @@ stages: name: Default e2e_pool_definition: vmImage: 'ubuntu-16.04' - environment: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11 -Djdk11 -Pe2e-hadoop -Pe2e-pre-commit" + environment: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11 -Djdk11 -Pe2e-hadoop" run_end_to_end: true container: flink-build-container jdk: jdk11 -- Gitee From 7ce9d7c96fbdc6a871e720ca8b2f9f461b3971c7 Mon Sep 17 00:00:00 2001 From: Rong Rong Date: Thu, 26 Mar 2020 08:51:03 -0700 Subject: [PATCH 318/885] [FLINK-16786][hotfix] Fix pyarrow version in setup.py (#11517) --- flink-python/setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-python/setup.py b/flink-python/setup.py index 0d183428e9..aeb3d25966 100644 --- a/flink-python/setup.py +++ b/flink-python/setup.py @@ -227,7 +227,7 @@ run sdist. python_requires='>=3.5', install_requires=['py4j==0.10.8.1', 'python-dateutil==2.8.0', 'apache-beam==2.19.0', 'cloudpickle==1.2.2', 'avro-python3>=1.8.1,<=1.9.1', 'jsonpickle==1.2', - 'pandas>=0.23.4,<=0.25.3', 'pyarrow>=0.15.1,<=0.16.0', 'pytz>=2018.3'], + 'pandas>=0.23.4,<=0.25.3', 'pyarrow>=0.15.1,<0.16.0', 'pytz>=2018.3'], tests_require=['pytest==4.4.1'], description='Apache Flink Python API', long_description=long_description, -- Gitee From ce52f3bcddc032cc4b3cb54c33eb1376df42c887 Mon Sep 17 00:00:00 2001 From: "bowen.li" Date: Fri, 20 Mar 2020 17:39:09 -0700 Subject: [PATCH 319/885] [FLINK-16702] develop JDBCCatalogFactory, descriptor, and validator for service discovery closes #16702 --- flink-connectors/flink-jdbc/pom.xml | 8 ++ .../io/jdbc/catalog/AbstractJDBCCatalog.java | 15 +++ .../catalog/factory/JDBCCatalogFactory.java | 93 +++++++++++++++++++ .../descriptors/JDBCCatalogDescriptor.java | 60 ++++++++++++ .../descriptors/JDBCCatalogValidator.java | 43 +++++++++ ....apache.flink.table.factories.TableFactory | 1 + .../factory/JDBCCatalogFactoryTest.java | 85 +++++++++++++++++ .../JDBCCatalogDescriptorTest.java | 63 +++++++++++++ 8 files changed, 368 insertions(+) create mode 100644 flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/factory/JDBCCatalogFactory.java create mode 100644 flink-connectors/flink-jdbc/src/main/java/org/apache/flink/table/descriptors/JDBCCatalogDescriptor.java create mode 100644 flink-connectors/flink-jdbc/src/main/java/org/apache/flink/table/descriptors/JDBCCatalogValidator.java create mode 100644 flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/factory/JDBCCatalogFactoryTest.java create mode 100644 flink-connectors/flink-jdbc/src/test/java/org/apache/flink/table/descriptors/JDBCCatalogDescriptorTest.java diff --git a/flink-connectors/flink-jdbc/pom.xml b/flink-connectors/flink-jdbc/pom.xml index cb7afab76b..0af5588982 100644 --- a/flink-connectors/flink-jdbc/pom.xml +++ b/flink-connectors/flink-jdbc/pom.xml @@ -76,6 +76,14 @@ under the License. test + + org.apache.flink + flink-table-common + ${project.version} + test-jar + test + + org.apache.flink diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/AbstractJDBCCatalog.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/AbstractJDBCCatalog.java index 523de83d22..6e7dd02870 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/AbstractJDBCCatalog.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/AbstractJDBCCatalog.java @@ -102,6 +102,21 @@ public abstract class AbstractJDBCCatalog extends AbstractCatalog { LOG.info("Catalog {} closing", getName()); } + // ----- getters ------ + + public String getUsername() { + return username; + } + + public String getPassword() { + return pwd; + } + + public String getBaseUrl() { + return baseUrl; + } + + // ------ table factory ------ public Optional getTableFactory() { diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/factory/JDBCCatalogFactory.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/factory/JDBCCatalogFactory.java new file mode 100644 index 0000000000..05f08c7547 --- /dev/null +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/factory/JDBCCatalogFactory.java @@ -0,0 +1,93 @@ +/* + * 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.api.java.io.jdbc.catalog.factory; + +import org.apache.flink.api.java.io.jdbc.catalog.JDBCCatalog; +import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.descriptors.DescriptorProperties; +import org.apache.flink.table.descriptors.JDBCCatalogValidator; +import org.apache.flink.table.factories.CatalogFactory; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.table.descriptors.CatalogDescriptorValidator.CATALOG_DEFAULT_DATABASE; +import static org.apache.flink.table.descriptors.CatalogDescriptorValidator.CATALOG_PROPERTY_VERSION; +import static org.apache.flink.table.descriptors.CatalogDescriptorValidator.CATALOG_TYPE; +import static org.apache.flink.table.descriptors.JDBCCatalogValidator.CATALOG_JDBC_BASE_URL; +import static org.apache.flink.table.descriptors.JDBCCatalogValidator.CATALOG_JDBC_PASSWORD; +import static org.apache.flink.table.descriptors.JDBCCatalogValidator.CATALOG_JDBC_USERNAME; +import static org.apache.flink.table.descriptors.JDBCCatalogValidator.CATALOG_TYPE_VALUE_JDBC; + +/** + * Factory for {@link org.apache.flink.api.java.io.jdbc.catalog.JDBCCatalog}. + */ +public class JDBCCatalogFactory implements CatalogFactory { + + private static final Logger LOG = LoggerFactory.getLogger(JDBCCatalogFactory.class); + + @Override + public Map requiredContext() { + Map context = new HashMap<>(); + context.put(CATALOG_TYPE, CATALOG_TYPE_VALUE_JDBC); // jdbc + context.put(CATALOG_PROPERTY_VERSION, "1"); // backwards compatibility + return context; + } + + @Override + public List supportedProperties() { + List properties = new ArrayList<>(); + + // default database + properties.add(CATALOG_DEFAULT_DATABASE); + + properties.add(CATALOG_JDBC_BASE_URL); + properties.add(CATALOG_JDBC_USERNAME); + properties.add(CATALOG_JDBC_PASSWORD); + + return properties; + } + + @Override + public Catalog createCatalog(String name, Map properties) { + final DescriptorProperties prop = getValidatedProperties(properties); + + return new JDBCCatalog( + name, + prop.getString(CATALOG_DEFAULT_DATABASE), + prop.getString(CATALOG_JDBC_USERNAME), + prop.getString(CATALOG_JDBC_PASSWORD), + prop.getString(CATALOG_JDBC_BASE_URL)); + } + + private static DescriptorProperties getValidatedProperties(Map properties) { + final DescriptorProperties descriptorProperties = new DescriptorProperties(true); + descriptorProperties.putProperties(properties); + + new JDBCCatalogValidator().validate(descriptorProperties); + + return descriptorProperties; + } + +} diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/table/descriptors/JDBCCatalogDescriptor.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/table/descriptors/JDBCCatalogDescriptor.java new file mode 100644 index 0000000000..dc03ae205c --- /dev/null +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/table/descriptors/JDBCCatalogDescriptor.java @@ -0,0 +1,60 @@ +/* + * 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.table.descriptors; + +import java.util.Map; + +import static org.apache.flink.table.descriptors.CatalogDescriptorValidator.CATALOG_DEFAULT_DATABASE; +import static org.apache.flink.table.descriptors.JDBCCatalogValidator.CATALOG_JDBC_BASE_URL; +import static org.apache.flink.table.descriptors.JDBCCatalogValidator.CATALOG_JDBC_PASSWORD; +import static org.apache.flink.table.descriptors.JDBCCatalogValidator.CATALOG_JDBC_USERNAME; +import static org.apache.flink.table.descriptors.JDBCCatalogValidator.CATALOG_TYPE_VALUE_JDBC; + +/** + * Descriptor for {@link org.apache.flink.api.java.io.jdbc.catalog.JDBCCatalog}. + */ +public class JDBCCatalogDescriptor extends CatalogDescriptor { + + private final String defaultDatabase; + private final String username; + private final String pwd; + private final String baseUrl; + + public JDBCCatalogDescriptor(String defaultDatabase, String username, String pwd, String baseUrl) { + + super(CATALOG_TYPE_VALUE_JDBC, 1); + + this.defaultDatabase = defaultDatabase; + this.username = username; + this.pwd = pwd; + this.baseUrl = baseUrl; + } + + @Override + protected Map toCatalogProperties() { + final DescriptorProperties properties = new DescriptorProperties(); + + properties.putString(CATALOG_DEFAULT_DATABASE, defaultDatabase); + properties.putString(CATALOG_JDBC_USERNAME, username); + properties.putString(CATALOG_JDBC_PASSWORD, pwd); + properties.putString(CATALOG_JDBC_BASE_URL, baseUrl); + + return properties.asMap(); + } +} diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/table/descriptors/JDBCCatalogValidator.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/table/descriptors/JDBCCatalogValidator.java new file mode 100644 index 0000000000..14730b4420 --- /dev/null +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/table/descriptors/JDBCCatalogValidator.java @@ -0,0 +1,43 @@ +/* + * 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.table.descriptors; + +import org.apache.flink.api.java.io.jdbc.catalog.JDBCCatalog; + +/** + * Validator for {@link JDBCCatalog}. + */ +public class JDBCCatalogValidator extends CatalogDescriptorValidator { + + public static final String CATALOG_TYPE_VALUE_JDBC = "jdbc"; + + public static final String CATALOG_JDBC_USERNAME = "username"; + public static final String CATALOG_JDBC_PASSWORD = "password"; + public static final String CATALOG_JDBC_BASE_URL = "base-url"; + + @Override + public void validate(DescriptorProperties properties) { + super.validate(properties); + properties.validateValue(CATALOG_TYPE, CATALOG_TYPE_VALUE_JDBC, false); + properties.validateString(CATALOG_JDBC_BASE_URL, false, 1); + properties.validateString(CATALOG_JDBC_USERNAME, false, 1); + properties.validateString(CATALOG_JDBC_PASSWORD, false, 1); + properties.validateString(CATALOG_DEFAULT_DATABASE, false, 1); + } +} diff --git a/flink-connectors/flink-jdbc/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink-connectors/flink-jdbc/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory index dbd648d597..fa0b4e68a2 100644 --- a/flink-connectors/flink-jdbc/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory +++ b/flink-connectors/flink-jdbc/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory @@ -14,3 +14,4 @@ # limitations under the License. org.apache.flink.api.java.io.jdbc.JDBCTableSourceSinkFactory +org.apache.flink.api.java.io.jdbc.catalog.factory.JDBCCatalogFactory diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/factory/JDBCCatalogFactoryTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/factory/JDBCCatalogFactoryTest.java new file mode 100644 index 0000000000..271617d6c8 --- /dev/null +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/factory/JDBCCatalogFactoryTest.java @@ -0,0 +1,85 @@ +/* + * 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.api.java.io.jdbc.catalog.factory; + +import org.apache.flink.api.java.io.jdbc.catalog.JDBCCatalog; +import org.apache.flink.api.java.io.jdbc.catalog.PostgresCatalog; +import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.descriptors.CatalogDescriptor; +import org.apache.flink.table.descriptors.JDBCCatalogDescriptor; +import org.apache.flink.table.factories.CatalogFactory; +import org.apache.flink.table.factories.TableFactoryService; + +import com.opentable.db.postgres.junit.EmbeddedPostgresRules; +import com.opentable.db.postgres.junit.SingleInstancePostgresRule; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; + +import java.sql.SQLException; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +/** + * Test for {@link JDBCCatalogFactory}. + */ +public class JDBCCatalogFactoryTest { + @ClassRule + public static SingleInstancePostgresRule pg = EmbeddedPostgresRules.singleInstance(); + + protected static String baseUrl; + protected static JDBCCatalog catalog; + + protected static final String TEST_CATALOG_NAME = "mypg"; + protected static final String TEST_USERNAME = "postgres"; + protected static final String TEST_PWD = "postgres"; + + @BeforeClass + public static void setup() throws SQLException { + // jdbc:postgresql://localhost:50807/postgres?user=postgres + String embeddedJdbcUrl = pg.getEmbeddedPostgres().getJdbcUrl(TEST_USERNAME, TEST_PWD); + // jdbc:postgresql://localhost:50807/ + baseUrl = embeddedJdbcUrl.substring(0, embeddedJdbcUrl.lastIndexOf("/") + 1); + + catalog = new JDBCCatalog( + TEST_CATALOG_NAME, PostgresCatalog.DEFAULT_DATABASE, TEST_USERNAME, TEST_PWD, baseUrl); + } + + @Test + public void test() { + final CatalogDescriptor catalogDescriptor = + new JDBCCatalogDescriptor(PostgresCatalog.DEFAULT_DATABASE, TEST_USERNAME, TEST_PWD, baseUrl); + + final Map properties = catalogDescriptor.toProperties(); + + final Catalog actualCatalog = TableFactoryService.find(CatalogFactory.class, properties) + .createCatalog(TEST_CATALOG_NAME, properties); + + checkEquals(catalog, (JDBCCatalog) actualCatalog); + } + + private static void checkEquals(JDBCCatalog c1, JDBCCatalog c2) { + assertEquals(c1.getName(), c2.getName()); + assertEquals(c1.getDefaultDatabase(), c2.getDefaultDatabase()); + assertEquals(c1.getUsername(), c2.getUsername()); + assertEquals(c1.getPassword(), c2.getPassword()); + assertEquals(c1.getBaseUrl(), c2.getBaseUrl()); + } +} diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/table/descriptors/JDBCCatalogDescriptorTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/table/descriptors/JDBCCatalogDescriptorTest.java new file mode 100644 index 0000000000..d8151a6180 --- /dev/null +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/table/descriptors/JDBCCatalogDescriptorTest.java @@ -0,0 +1,63 @@ +/* + * 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.table.descriptors; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.table.descriptors.JDBCCatalogValidator.CATALOG_TYPE_VALUE_JDBC; + +/** + * Test for {@link JDBCCatalogDescriptor}. + */ +public class JDBCCatalogDescriptorTest extends DescriptorTestBase { + + private static final String TEST_DB = "db"; + private static final String TEST_USERNAME = "user"; + private static final String TEST_PWD = "pwd"; + private static final String TEST_BASE_URL = "xxx"; + + @Override + protected List descriptors() { + final Descriptor descriptor = new JDBCCatalogDescriptor( + TEST_DB, TEST_USERNAME, TEST_PWD, TEST_BASE_URL); + + return Arrays.asList(descriptor); + } + + @Override + protected List> properties() { + final Map props = new HashMap<>(); + props.put("type", CATALOG_TYPE_VALUE_JDBC); + props.put("property-version", "1"); + props.put("default-database", TEST_DB); + props.put("username", TEST_USERNAME); + props.put("password", TEST_PWD); + props.put("base-url", TEST_BASE_URL); + + return Arrays.asList(props); + } + + @Override + protected DescriptorValidator validator() { + return new JDBCCatalogValidator(); + } +} -- Gitee From 358e4e76645bd8f418631a9e81f578b65be47a25 Mon Sep 17 00:00:00 2001 From: "bowen.li" Date: Tue, 24 Mar 2020 10:26:54 -0700 Subject: [PATCH 320/885] [hotfix][jdbc] rename PostgresCatalogITCase to PostgresCatalogTest --- .../{PostgresCatalogITCase.java => PostgresCatalogTest.java} | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) rename flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/{PostgresCatalogITCase.java => PostgresCatalogTest.java} (99%) diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogITCase.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogTest.java similarity index 99% rename from flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogITCase.java rename to flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogTest.java index b197bf0409..2a78059842 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogITCase.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogTest.java @@ -48,7 +48,7 @@ import static org.junit.Assert.assertTrue; /** * Test for {@link PostgresCatalog}. */ -public class PostgresCatalogITCase { +public class PostgresCatalogTest { @Rule public ExpectedException exception = ExpectedException.none(); -- Gitee From 25f2d626043145c3a87373754d79b512394017a8 Mon Sep 17 00:00:00 2001 From: "bowen.li" Date: Sun, 22 Mar 2020 23:16:32 -0700 Subject: [PATCH 321/885] [FLINK-16498] connect PostgresCatalog to table planner --- .../api/java/io/jdbc/catalog/JDBCCatalog.java | 11 +- .../java/io/jdbc/catalog/PostgresCatalog.java | 23 +- .../io/jdbc/catalog/PostgresTablePath.java | 6 +- .../java/io/jdbc/dialect/JDBCDialects.java | 5 + .../descriptors/JDBCCatalogDescriptor.java | 8 + .../io/jdbc/catalog/PostgresCatalogTest.java | 181 +-------------- .../jdbc/catalog/PostgresCatalogTestBase.java | 215 ++++++++++++++++++ .../factory/JDBCCatalogFactoryTest.java | 3 + 8 files changed, 264 insertions(+), 188 deletions(-) create mode 100644 flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogTestBase.java diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/JDBCCatalog.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/JDBCCatalog.java index 629412c744..d040a1b8fe 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/JDBCCatalog.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/JDBCCatalog.java @@ -19,7 +19,7 @@ package org.apache.flink.api.java.io.jdbc.catalog; import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.table.catalog.CatalogBaseTable; import org.apache.flink.table.catalog.CatalogDatabase; import org.apache.flink.table.catalog.ObjectPath; @@ -40,7 +40,7 @@ public class JDBCCatalog extends AbstractJDBCCatalog { private static final Logger LOG = LoggerFactory.getLogger(JDBCCatalog.class); - private final Catalog internal; + private final AbstractJDBCCatalog internal; public JDBCCatalog(String catalogName, String defaultDatabase, String username, String pwd, String baseUrl) { super(catalogName, defaultDatabase, username, pwd, baseUrl); @@ -81,4 +81,11 @@ public class JDBCCatalog extends AbstractJDBCCatalog { return false; } } + + // ------ getters ------ + + @VisibleForTesting + public AbstractJDBCCatalog getInternal() { + return internal; + } } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalog.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalog.java index c5980737b3..e5503f6fec 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalog.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalog.java @@ -45,8 +45,17 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; +import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_PROPERTY_VERSION; +import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE; +import static org.apache.flink.table.descriptors.JDBCValidator.CONNECTOR_PASSWORD; +import static org.apache.flink.table.descriptors.JDBCValidator.CONNECTOR_TABLE; +import static org.apache.flink.table.descriptors.JDBCValidator.CONNECTOR_TYPE_VALUE_JDBC; +import static org.apache.flink.table.descriptors.JDBCValidator.CONNECTOR_URL; +import static org.apache.flink.table.descriptors.JDBCValidator.CONNECTOR_USERNAME; + /** * Catalog for PostgreSQL. */ @@ -169,7 +178,8 @@ public class PostgresCatalog extends AbstractJDBCCatalog { PostgresTablePath pgPath = PostgresTablePath.fromFlinkTableName(tablePath.getObjectName()); - try (Connection conn = DriverManager.getConnection(baseUrl + tablePath.getDatabaseName(), username, pwd)) { + String dbUrl = baseUrl + tablePath.getDatabaseName(); + try (Connection conn = DriverManager.getConnection(dbUrl, username, pwd)) { PreparedStatement ps = conn.prepareStatement( String.format("SELECT * FROM %s;", pgPath.getFullPath())); @@ -186,9 +196,18 @@ public class PostgresCatalog extends AbstractJDBCCatalog { TableSchema tableSchema = new TableSchema.Builder().fields(names, types).build(); + Map props = new HashMap<>(); + props.put(CONNECTOR_TYPE, CONNECTOR_TYPE_VALUE_JDBC); + props.put(CONNECTOR_PROPERTY_VERSION, "1"); + + props.put(CONNECTOR_URL, dbUrl); + props.put(CONNECTOR_TABLE, pgPath.getFullPath()); + props.put(CONNECTOR_USERNAME, username); + props.put(CONNECTOR_PASSWORD, pwd); + return new CatalogTableImpl( tableSchema, - new HashMap<>(), + props, "" ); } catch (Exception e) { diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresTablePath.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresTablePath.java index 99cc2b44e7..5c36cb73a0 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresTablePath.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresTablePath.java @@ -64,13 +64,9 @@ public class PostgresTablePath { return String.format("%s.%s", pgSchemaName, pgTableName); } - public String getFullPathWithQuotes() { - return String.format("`%s.%s`", pgSchemaName, pgTableName); - } - @Override public String toString() { - return getFullPathWithQuotes(); + return getFullPath(); } @Override diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialects.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialects.java index 743d16a015..b09e36f73c 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialects.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialects.java @@ -348,6 +348,11 @@ public final class JDBCDialects { ); } + @Override + public String quoteIdentifier(String identifier) { + return identifier; + } + @Override public String dialectName() { return "postgresql"; diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/table/descriptors/JDBCCatalogDescriptor.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/table/descriptors/JDBCCatalogDescriptor.java index dc03ae205c..abf5fdde5f 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/table/descriptors/JDBCCatalogDescriptor.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/table/descriptors/JDBCCatalogDescriptor.java @@ -18,6 +18,8 @@ package org.apache.flink.table.descriptors; +import org.apache.flink.util.StringUtils; + import java.util.Map; import static org.apache.flink.table.descriptors.CatalogDescriptorValidator.CATALOG_DEFAULT_DATABASE; @@ -25,6 +27,7 @@ import static org.apache.flink.table.descriptors.JDBCCatalogValidator.CATALOG_JD import static org.apache.flink.table.descriptors.JDBCCatalogValidator.CATALOG_JDBC_PASSWORD; import static org.apache.flink.table.descriptors.JDBCCatalogValidator.CATALOG_JDBC_USERNAME; import static org.apache.flink.table.descriptors.JDBCCatalogValidator.CATALOG_TYPE_VALUE_JDBC; +import static org.apache.flink.util.Preconditions.checkArgument; /** * Descriptor for {@link org.apache.flink.api.java.io.jdbc.catalog.JDBCCatalog}. @@ -40,6 +43,11 @@ public class JDBCCatalogDescriptor extends CatalogDescriptor { super(CATALOG_TYPE_VALUE_JDBC, 1); + checkArgument(!StringUtils.isNullOrWhitespaceOnly(defaultDatabase)); + checkArgument(!StringUtils.isNullOrWhitespaceOnly(username)); + checkArgument(!StringUtils.isNullOrWhitespaceOnly(pwd)); + checkArgument(!StringUtils.isNullOrWhitespaceOnly(baseUrl)); + this.defaultDatabase = defaultDatabase; this.username = username; this.pwd = pwd; diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogTest.java index 2a78059842..808f9d806f 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogTest.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogTest.java @@ -18,26 +18,14 @@ package org.apache.flink.api.java.io.jdbc.catalog; -import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.catalog.Catalog; import org.apache.flink.table.catalog.CatalogBaseTable; import org.apache.flink.table.catalog.ObjectPath; import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; import org.apache.flink.table.catalog.exceptions.TableNotExistException; -import com.opentable.db.postgres.junit.EmbeddedPostgresRules; -import com.opentable.db.postgres.junit.SingleInstancePostgresRule; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.ExpectedException; -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.SQLException; -import java.sql.Statement; import java.util.Arrays; import java.util.List; @@ -48,52 +36,7 @@ import static org.junit.Assert.assertTrue; /** * Test for {@link PostgresCatalog}. */ -public class PostgresCatalogTest { - @Rule - public ExpectedException exception = ExpectedException.none(); - - @ClassRule - public static SingleInstancePostgresRule pg = EmbeddedPostgresRules.singleInstance(); - - protected static final String TEST_USERNAME = "postgres"; - protected static final String TEST_PWD = "postgres"; - protected static final String TEST_DB = "test"; - protected static final String TEST_SCHEMA = "test_schema"; - protected static final String TABLE1 = "t1"; - protected static final String TABLE2 = "t2"; - protected static final String TABLE3 = "t3"; - - protected static String baseUrl; - protected static Catalog catalog; - - public static Catalog createCatalog(String name, String defaultDb, String username, String pwd, String jdbcUrl) { - return new PostgresCatalog("mypg", PostgresCatalog.DEFAULT_DATABASE, username, pwd, jdbcUrl); - } - - @BeforeClass - public static void setup() throws SQLException { - // jdbc:postgresql://localhost:50807/postgres?user=postgres - String embeddedJdbcUrl = pg.getEmbeddedPostgres().getJdbcUrl(TEST_USERNAME, TEST_PWD); - // jdbc:postgresql://localhost:50807/ - baseUrl = embeddedJdbcUrl.substring(0, embeddedJdbcUrl.lastIndexOf("/") + 1); - - catalog = createCatalog("mypg", PostgresCatalog.DEFAULT_DATABASE, TEST_USERNAME, TEST_PWD, baseUrl); - - // create test database and schema - createDatabase(TEST_DB); - createSchema(TEST_DB, TEST_SCHEMA); - - // create test tables - // table: postgres.public.user1 - createTable(PostgresTablePath.fromFlinkTableName(TABLE1), getSimpleTable().pgSchemaSql); - - // table: testdb.public.user2 - // table: testdb.testschema.user3 - // table: testdb.public.datatypes - createTable(TEST_DB, PostgresTablePath.fromFlinkTableName(TABLE2), getSimpleTable().pgSchemaSql); - createTable(TEST_DB, new PostgresTablePath(TEST_SCHEMA, TABLE3), getSimpleTable().pgSchemaSql); - createTable(TEST_DB, PostgresTablePath.fromFlinkTableName("datatypes"), getDataTypesTable().pgSchemaSql); - } +public class PostgresCatalogTest extends PostgresCatalogTestBase { // ------ databases ------ @@ -127,7 +70,7 @@ public class PostgresCatalogTest { public void testListTables() throws DatabaseNotExistException { List actual = catalog.listTables(PostgresCatalog.DEFAULT_DATABASE); - assertEquals(Arrays.asList("public.t1"), actual); + assertEquals(Arrays.asList("public.t1", "public.t4", "public.t5"), actual); actual = catalog.listTables(TEST_DB); @@ -202,124 +145,4 @@ public class PostgresCatalogTest { assertEquals(getDataTypesTable().schema, table.getSchema()); } - - private static class TestTable { - TableSchema schema; - String pgSchemaSql; - - public TestTable(TableSchema schema, String pgSchemaSql) { - this.schema = schema; - this.pgSchemaSql = pgSchemaSql; - } - } - - private static TestTable getSimpleTable() { - return new TestTable( - TableSchema.builder() - .field("name", DataTypes.INT()) - .build(), - "name integer" - ); - } - - private static TestTable getDataTypesTable() { - return new TestTable( - TableSchema.builder() - .field("int", DataTypes.INT()) - .field("int_arr", DataTypes.ARRAY(DataTypes.INT())) - .field("bytea", DataTypes.BYTES()) - .field("bytea_arr", DataTypes.ARRAY(DataTypes.BYTES())) - .field("short", DataTypes.SMALLINT()) - .field("short_arr", DataTypes.ARRAY(DataTypes.SMALLINT())) - .field("long", DataTypes.BIGINT()) - .field("long_arr", DataTypes.ARRAY(DataTypes.BIGINT())) - .field("real", DataTypes.FLOAT()) - .field("real_arr", DataTypes.ARRAY(DataTypes.FLOAT())) - .field("double_precision", DataTypes.DOUBLE()) - .field("double_precision_arr", DataTypes.ARRAY(DataTypes.DOUBLE())) - .field("numeric", DataTypes.DECIMAL(10, 5)) - .field("numeric_arr", DataTypes.ARRAY(DataTypes.DECIMAL(10, 5))) - .field("boolean", DataTypes.BOOLEAN()) - .field("boolean_arr", DataTypes.ARRAY(DataTypes.BOOLEAN())) - .field("text", DataTypes.STRING()) - .field("text_arr", DataTypes.ARRAY(DataTypes.STRING())) - .field("char", DataTypes.CHAR(1)) - .field("char_arr", DataTypes.ARRAY(DataTypes.CHAR(1))) - .field("character", DataTypes.CHAR(3)) - .field("character_arr", DataTypes.ARRAY(DataTypes.CHAR(3))) - .field("character_varying", DataTypes.VARCHAR(20)) - .field("character_varying_arr", DataTypes.ARRAY(DataTypes.VARCHAR(20))) - .field("timestamp", DataTypes.TIMESTAMP(5)) - .field("timestamp_arr", DataTypes.ARRAY(DataTypes.TIMESTAMP(5))) - .field("timestamptz", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(4)) - .field("timestamptz_arr", DataTypes.ARRAY(DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(4))) - .field("date", DataTypes.DATE()) - .field("date_arr", DataTypes.ARRAY(DataTypes.DATE())) - .field("time", DataTypes.TIME(3)) - .field("time_arr", DataTypes.ARRAY(DataTypes.TIME(3))) - .build(), - "int integer, " + - "int_arr integer[], " + - "bytea bytea, " + - "bytea_arr bytea[], " + - "short smallint, " + - "short_arr smallint[], " + - "long bigint, " + - "long_arr bigint[], " + - "real real, " + - "real_arr real[], " + - "double_precision double precision, " + - "double_precision_arr double precision[], " + - "numeric numeric(10, 5), " + - "numeric_arr numeric(10, 5)[], " + - "boolean boolean, " + - "boolean_arr boolean[], " + - "text text, " + - "text_arr text[], " + - "char char, " + - "char_arr char[], " + - "character character(3), " + - "character_arr character(3)[], " + - "character_varying character varying(20), " + - "character_varying_arr character varying(20)[], " + - "timestamp timestamp(5), " + - "timestamp_arr timestamp(5)[], " + - "timestamptz timestamptz(4), " + - "timestamptz_arr timestamptz(4)[], " + - "date date, " + - "date_arr date[], " + - "time time(3), " + - "time_arr time(3)[]" - ); - } - - private static void createTable(PostgresTablePath tablePath, String tableSchemaSql) throws SQLException { - executeSQL(PostgresCatalog.DEFAULT_DATABASE, String.format("CREATE TABLE %s(%s);", tablePath.getFullPath(), tableSchemaSql)); - } - - private static void createTable(String db, PostgresTablePath tablePath, String tableSchemaSql) throws SQLException { - executeSQL(db, String.format("CREATE TABLE %s(%s);", tablePath.getFullPath(), tableSchemaSql)); - } - - private static void createSchema(String db, String schema) throws SQLException { - executeSQL(db, String.format("CREATE SCHEMA %s", schema)); - } - - private static void createDatabase(String database) throws SQLException { - executeSQL(String.format("CREATE DATABASE %s;", database)); - } - - private static void executeSQL(String sql) throws SQLException { - executeSQL("", sql); - } - - private static void executeSQL(String db, String sql) throws SQLException { - try (Connection conn = DriverManager.getConnection(baseUrl + db, TEST_USERNAME, TEST_PWD); - Statement statement = conn.createStatement()) { - statement.executeUpdate(sql); - } catch (SQLException e) { - throw e; - } - } - } diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogTestBase.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogTestBase.java new file mode 100644 index 0000000000..1cab3b02a2 --- /dev/null +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogTestBase.java @@ -0,0 +1,215 @@ +/* + * 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.api.java.io.jdbc.catalog; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; + +import com.opentable.db.postgres.junit.EmbeddedPostgresRules; +import com.opentable.db.postgres.junit.SingleInstancePostgresRule; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.rules.ExpectedException; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; + +import static org.apache.flink.api.java.io.jdbc.catalog.PostgresCatalog.DEFAULT_DATABASE; + +/** + * Test base for {@link PostgresCatalog}. + */ +public class PostgresCatalogTestBase { + + @Rule + public ExpectedException exception = ExpectedException.none(); + + @ClassRule + public static SingleInstancePostgresRule pg = EmbeddedPostgresRules.singleInstance(); + + protected static final String TEST_CATALOG_NAME = "mypg"; + protected static final String TEST_USERNAME = "postgres"; + protected static final String TEST_PWD = "postgres"; + protected static final String TEST_DB = "test"; + protected static final String TEST_SCHEMA = "test_schema"; + protected static final String TABLE1 = "t1"; + protected static final String TABLE2 = "t2"; + protected static final String TABLE3 = "t3"; + protected static final String TABLE4 = "t4"; + protected static final String TABLE5 = "t5"; + + protected static String baseUrl; + protected static PostgresCatalog catalog; + + @BeforeClass + public static void init() throws SQLException { + // jdbc:postgresql://localhost:50807/postgres?user=postgres + String embeddedJdbcUrl = pg.getEmbeddedPostgres().getJdbcUrl(TEST_USERNAME, TEST_PWD); + // jdbc:postgresql://localhost:50807/ + baseUrl = embeddedJdbcUrl.substring(0, embeddedJdbcUrl.lastIndexOf("/") + 1); + + catalog = new PostgresCatalog(TEST_CATALOG_NAME, PostgresCatalog.DEFAULT_DATABASE, TEST_USERNAME, TEST_PWD, baseUrl); + + // create test database and schema + createDatabase(TEST_DB); + createSchema(TEST_DB, TEST_SCHEMA); + + // create test tables + // table: postgres.public.t1 + // table: postgres.public.t4 + // table: postgres.public.t5 + createTable(PostgresTablePath.fromFlinkTableName(TABLE1), getSimpleTable().pgSchemaSql); + createTable(PostgresTablePath.fromFlinkTableName(TABLE4), getSimpleTable().pgSchemaSql); + createTable(PostgresTablePath.fromFlinkTableName(TABLE5), getSimpleTable().pgSchemaSql); + + executeSQL(DEFAULT_DATABASE, String.format("insert into public.%s values (1);", TABLE1)); + + // table: testdb.public.t2 + // table: testdb.test_schema.t3 + // table: testdb.public.datatypes + createTable(TEST_DB, PostgresTablePath.fromFlinkTableName(TABLE2), getSimpleTable().pgSchemaSql); + createTable(TEST_DB, new PostgresTablePath(TEST_SCHEMA, TABLE3), getSimpleTable().pgSchemaSql); + createTable(TEST_DB, PostgresTablePath.fromFlinkTableName("datatypes"), getDataTypesTable().pgSchemaSql); + } + + public static void createTable(PostgresTablePath tablePath, String tableSchemaSql) throws SQLException { + executeSQL(PostgresCatalog.DEFAULT_DATABASE, String.format("CREATE TABLE %s(%s);", tablePath.getFullPath(), tableSchemaSql)); + } + + public static void createTable(String db, PostgresTablePath tablePath, String tableSchemaSql) throws SQLException { + executeSQL(db, String.format("CREATE TABLE %s(%s);", tablePath.getFullPath(), tableSchemaSql)); + } + + public static void createSchema(String db, String schema) throws SQLException { + executeSQL(db, String.format("CREATE SCHEMA %s", schema)); + } + + public static void createDatabase(String database) throws SQLException { + executeSQL(String.format("CREATE DATABASE %s;", database)); + } + + public static void executeSQL(String sql) throws SQLException { + executeSQL("", sql); + } + + public static void executeSQL(String db, String sql) throws SQLException { + try (Connection conn = DriverManager.getConnection(baseUrl + db, TEST_USERNAME, TEST_PWD); + Statement statement = conn.createStatement()) { + statement.executeUpdate(sql); + } catch (SQLException e) { + throw e; + } + } + + /** + * Object holding schema and corresponding sql. + */ + public static class TestTable { + TableSchema schema; + String pgSchemaSql; + + public TestTable(TableSchema schema, String pgSchemaSql) { + this.schema = schema; + this.pgSchemaSql = pgSchemaSql; + } + } + + public static TestTable getSimpleTable() { + return new TestTable( + TableSchema.builder() + .field("id", DataTypes.INT()) + .build(), + "id integer" + ); + } + + public static TestTable getDataTypesTable() { + return new TestTable( + TableSchema.builder() + .field("int", DataTypes.INT()) + .field("int_arr", DataTypes.ARRAY(DataTypes.INT())) + .field("bytea", DataTypes.BYTES()) + .field("bytea_arr", DataTypes.ARRAY(DataTypes.BYTES())) + .field("short", DataTypes.SMALLINT()) + .field("short_arr", DataTypes.ARRAY(DataTypes.SMALLINT())) + .field("long", DataTypes.BIGINT()) + .field("long_arr", DataTypes.ARRAY(DataTypes.BIGINT())) + .field("real", DataTypes.FLOAT()) + .field("real_arr", DataTypes.ARRAY(DataTypes.FLOAT())) + .field("double_precision", DataTypes.DOUBLE()) + .field("double_precision_arr", DataTypes.ARRAY(DataTypes.DOUBLE())) + .field("numeric", DataTypes.DECIMAL(10, 5)) + .field("numeric_arr", DataTypes.ARRAY(DataTypes.DECIMAL(10, 5))) + .field("boolean", DataTypes.BOOLEAN()) + .field("boolean_arr", DataTypes.ARRAY(DataTypes.BOOLEAN())) + .field("text", DataTypes.STRING()) + .field("text_arr", DataTypes.ARRAY(DataTypes.STRING())) + .field("char", DataTypes.CHAR(1)) + .field("char_arr", DataTypes.ARRAY(DataTypes.CHAR(1))) + .field("character", DataTypes.CHAR(3)) + .field("character_arr", DataTypes.ARRAY(DataTypes.CHAR(3))) + .field("character_varying", DataTypes.VARCHAR(20)) + .field("character_varying_arr", DataTypes.ARRAY(DataTypes.VARCHAR(20))) + .field("timestamp", DataTypes.TIMESTAMP(5)) + .field("timestamp_arr", DataTypes.ARRAY(DataTypes.TIMESTAMP(5))) + .field("timestamptz", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(4)) + .field("timestamptz_arr", DataTypes.ARRAY(DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(4))) + .field("date", DataTypes.DATE()) + .field("date_arr", DataTypes.ARRAY(DataTypes.DATE())) + .field("time", DataTypes.TIME(3)) + .field("time_arr", DataTypes.ARRAY(DataTypes.TIME(3))) + .build(), + "int integer, " + + "int_arr integer[], " + + "bytea bytea, " + + "bytea_arr bytea[], " + + "short smallint, " + + "short_arr smallint[], " + + "long bigint, " + + "long_arr bigint[], " + + "real real, " + + "real_arr real[], " + + "double_precision double precision, " + + "double_precision_arr double precision[], " + + "numeric numeric(10, 5), " + + "numeric_arr numeric(10, 5)[], " + + "boolean boolean, " + + "boolean_arr boolean[], " + + "text text, " + + "text_arr text[], " + + "char char, " + + "char_arr char[], " + + "character character(3), " + + "character_arr character(3)[], " + + "character_varying character varying(20), " + + "character_varying_arr character varying(20)[], " + + "timestamp timestamp(5), " + + "timestamp_arr timestamp(5)[], " + + "timestamptz timestamptz(4), " + + "timestamptz_arr timestamptz(4)[], " + + "date date, " + + "date_arr date[], " + + "time time(3), " + + "time_arr time(3)[]" + ); + } +} diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/factory/JDBCCatalogFactoryTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/factory/JDBCCatalogFactoryTest.java index 271617d6c8..a1e2c90e30 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/factory/JDBCCatalogFactoryTest.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/factory/JDBCCatalogFactoryTest.java @@ -36,6 +36,7 @@ import java.sql.SQLException; import java.util.Map; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; /** * Test for {@link JDBCCatalogFactory}. @@ -73,6 +74,8 @@ public class JDBCCatalogFactoryTest { .createCatalog(TEST_CATALOG_NAME, properties); checkEquals(catalog, (JDBCCatalog) actualCatalog); + + assertTrue(((JDBCCatalog) actualCatalog).getInternal() instanceof PostgresCatalog); } private static void checkEquals(JDBCCatalog c1, JDBCCatalog c2) { -- Gitee From 6a0805141088ba5dd2773aa138af7bc43bbfa432 Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Wed, 4 Mar 2020 14:25:11 +0100 Subject: [PATCH 322/885] [FLINK-16411][AZP] Use google mvn mirror globally This closes #11314 --- .travis.yml | 3 +- flink-end-to-end-tests/run-nightly-tests.sh | 9 +-- flink-end-to-end-tests/run-single-test.sh | 1 + .../test-scripts/common_ssl.sh | 2 +- .../test-scripts/test-runner-common.sh | 2 +- .../test_datastream_walkthroughs.sh | 4 +- .../test-scripts/test_quickstarts.sh | 4 +- .../test-scripts/test_streaming_gcp_pubsub.sh | 2 +- .../test-scripts/test_table_walkthroughs.sh | 4 +- tools/azure-pipelines/jobs-template.yml | 6 +- tools/azure-pipelines/prepare_precommit.sh | 5 +- tools/azure_controller.sh | 20 +++--- .../google-mirror-settings.xml | 0 tools/ci/maven-utils.sh | 69 +++++++++++++++++++ tools/travis/setup_maven.sh | 39 ----------- tools/travis_controller.sh | 2 +- tools/travis_watchdog.sh | 15 ++-- tools/verify_scala_suffixes.sh | 2 +- 18 files changed, 109 insertions(+), 80 deletions(-) rename tools/{azure-pipelines => ci}/google-mirror-settings.xml (100%) create mode 100755 tools/ci/maven-utils.sh delete mode 100755 tools/travis/setup_maven.sh diff --git a/.travis.yml b/.travis.yml index 8d99bda37d..3b4d212746 100644 --- a/.travis.yml +++ b/.travis.yml @@ -59,7 +59,8 @@ before_script: # Install maven 3.2.5 since xenial uses 3.3.9 for which shading is broken before_install: - - source ./tools/travis/setup_maven.sh + - source ./tools/ci/maven-utils.sh + - setup_maven notifications: slack: diff --git a/flink-end-to-end-tests/run-nightly-tests.sh b/flink-end-to-end-tests/run-nightly-tests.sh index 1fb904435a..a16fc8946a 100755 --- a/flink-end-to-end-tests/run-nightly-tests.sh +++ b/flink-end-to-end-tests/run-nightly-tests.sh @@ -32,6 +32,7 @@ if [ -z "$FLINK_DIR" ] ; then exit 1 fi +source "${END_TO_END_DIR}/../tools/ci/maven-utils.sh" source "${END_TO_END_DIR}/test-scripts/test-runner-common.sh" # On Azure CI, set artifacts dir @@ -57,7 +58,7 @@ echo "Flink distribution directory: $FLINK_DIR" echo "Java and Maven version" java -version -mvn -version +run_mvn -version echo "Free disk space" df -h @@ -230,13 +231,13 @@ printf "======================================================================== LOG4J_PROPERTIES=${END_TO_END_DIR}/../tools/log4j-travis.properties -MVN_LOGGING_OPTIONS="-Dlog.dir=${ARTIFACTS_DIR} -DlogBackupDir=${ARTIFACTS_DIR} -Dlog4j.configurationFile=file://$LOG4J_PROPERTIES -Dorg.slf4j.simpleLogger.log.org.apache.maven.cli.transfer.Slf4jMavenTransferListener=warn" -MVN_COMMON_OPTIONS="-nsu -B -Dflink.forkCount=2 -Dflink.forkCountTestPackage=2 -Dmaven.wagon.http.pool=false -Dfast -Pskip-webui-build" +MVN_LOGGING_OPTIONS="-Dlog.dir=${ARTIFACTS_DIR} -DlogBackupDir=${ARTIFACTS_DIR} -Dlog4j.configurationFile=file://$LOG4J_PROPERTIES" +MVN_COMMON_OPTIONS="-Dflink.forkCount=2 -Dflink.forkCountTestPackage=2 -Dfast -Pskip-webui-build" e2e_modules=$(find flink-end-to-end-tests -mindepth 2 -maxdepth 5 -name 'pom.xml' -printf '%h\n' | sort -u | tr '\n' ',') e2e_modules="${e2e_modules},$(find flink-walkthroughs -mindepth 2 -maxdepth 2 -name 'pom.xml' -printf '%h\n' | sort -u | tr '\n' ',')" PROFILE="$PROFILE -Pe2e-travis1 -Pe2e-travis2 -Pe2e-travis3 -Pe2e-travis4 -Pe2e-travis5 -Pe2e-travis6" -mvn ${MVN_COMMON_OPTIONS} ${MVN_LOGGING_OPTIONS} ${PROFILE} verify -pl ${e2e_modules} -DdistDir=$(readlink -e build-target) +run_mvn ${MVN_COMMON_OPTIONS} ${MVN_LOGGING_OPTIONS} ${PROFILE} verify -pl ${e2e_modules} -DdistDir=$(readlink -e build-target) EXIT_CODE=$? diff --git a/flink-end-to-end-tests/run-single-test.sh b/flink-end-to-end-tests/run-single-test.sh index 06628eb8cc..9ec56ed574 100755 --- a/flink-end-to-end-tests/run-single-test.sh +++ b/flink-end-to-end-tests/run-single-test.sh @@ -47,6 +47,7 @@ if [ -z "$FLINK_DIR" ] ; then fi source "${END_TO_END_DIR}/test-scripts/test-runner-common.sh" +source "${END_TO_END_DIR}/../tools/ci/maven-utils.sh" cleanup_tmp_files diff --git a/flink-end-to-end-tests/test-scripts/common_ssl.sh b/flink-end-to-end-tests/test-scripts/common_ssl.sh index b3a892fcc1..f9e618e69a 100644 --- a/flink-end-to-end-tests/test-scripts/common_ssl.sh +++ b/flink-end-to-end-tests/test-scripts/common_ssl.sh @@ -80,7 +80,7 @@ function _set_conf_ssl_helper { git clone https://github.com/apache/flink-shaded.git cd flink-shaded git checkout "release-${FLINK_SHADED_VERSION}" - mvn clean package -Pinclude-netty-tcnative-static -pl flink-shaded-netty-tcnative-static + run_mvn clean package -Pinclude-netty-tcnative-static -pl flink-shaded-netty-tcnative-static cp flink-shaded-netty-tcnative-static/target/flink-shaded-netty-tcnative-static-*.jar $FLINK_DIR/lib/ cd .. rm -rf flink-shaded diff --git a/flink-end-to-end-tests/test-scripts/test-runner-common.sh b/flink-end-to-end-tests/test-scripts/test-runner-common.sh index 326043ec41..8c1cf022f2 100644 --- a/flink-end-to-end-tests/test-scripts/test-runner-common.sh +++ b/flink-end-to-end-tests/test-scripts/test-runner-common.sh @@ -19,7 +19,7 @@ source "${END_TO_END_DIR}"/test-scripts/common.sh -export FLINK_VERSION=$(mvn --file ${END_TO_END_DIR}/pom.xml org.apache.maven.plugins:maven-help-plugin:3.1.0:evaluate -Dexpression=project.version -q -DforceStdout) +export FLINK_VERSION=$(MVN_RUN_VERBOSE=false run_mvn --file ${END_TO_END_DIR}/pom.xml org.apache.maven.plugins:maven-help-plugin:3.1.0:evaluate -Dexpression=project.version -q -DforceStdout) ####################################### # Prints the given description, runs the given test and prints how long the execution took. diff --git a/flink-end-to-end-tests/test-scripts/test_datastream_walkthroughs.sh b/flink-end-to-end-tests/test-scripts/test_datastream_walkthroughs.sh index b3c4e53cb6..83b84c990e 100755 --- a/flink-end-to-end-tests/test-scripts/test_datastream_walkthroughs.sh +++ b/flink-end-to-end-tests/test-scripts/test_datastream_walkthroughs.sh @@ -31,7 +31,7 @@ cd "${TEST_DATA_DIR}" ARTIFACT_ID=flink-walkthrough-datastream-${TEST_TYPE} ARTIFACT_VERSION=0.1 -mvn archetype:generate \ +run_mvn archetype:generate \ -DarchetypeGroupId=org.apache.flink \ -DarchetypeArtifactId=flink-walkthrough-datastream-${TEST_TYPE} \ -DarchetypeVersion=${FLINK_VERSION} \ @@ -44,7 +44,7 @@ mvn archetype:generate \ cd "${ARTIFACT_ID}" -mvn clean package -nsu > compile-output.txt +run_mvn clean package > compile-output.txt if [[ `grep -c "BUILD FAILURE" compile-output.txt` -eq '1' ]]; then echo "Failure: The walkthrough did not successfully compile" diff --git a/flink-end-to-end-tests/test-scripts/test_quickstarts.sh b/flink-end-to-end-tests/test-scripts/test_quickstarts.sh index e1d71c66ad..3b06e19fba 100755 --- a/flink-end-to-end-tests/test-scripts/test_quickstarts.sh +++ b/flink-end-to-end-tests/test-scripts/test_quickstarts.sh @@ -43,7 +43,7 @@ cd "${TEST_DATA_DIR}" ARTIFACT_ID=flink-quickstart-${TEST_TYPE} ARTIFACT_VERSION=0.1 -mvn archetype:generate \ +run_mvn archetype:generate \ -DarchetypeGroupId=org.apache.flink \ -DarchetypeArtifactId=flink-quickstart-${TEST_TYPE} \ -DarchetypeVersion=${FLINK_VERSION} \ @@ -68,7 +68,7 @@ sed -i -e ''$(($position + 1))'i\ sed -i -e "s/org.apache.flink.quickstart.StreamingJob/org.apache.flink.quickstart.$TEST_CLASS_NAME/" pom.xml -mvn clean package -nsu +run_mvn clean package cd target jar tvf flink-quickstart-${TEST_TYPE}-0.1.jar > contentsInJar.txt diff --git a/flink-end-to-end-tests/test-scripts/test_streaming_gcp_pubsub.sh b/flink-end-to-end-tests/test-scripts/test_streaming_gcp_pubsub.sh index 976e76428f..8fa9e5a8da 100755 --- a/flink-end-to-end-tests/test-scripts/test_streaming_gcp_pubsub.sh +++ b/flink-end-to-end-tests/test-scripts/test_streaming_gcp_pubsub.sh @@ -19,4 +19,4 @@ cd "${END_TO_END_DIR}/flink-connector-gcp-pubsub-emulator-tests" -mvn test -DskipTests=false +run_mvn test -DskipTests=false diff --git a/flink-end-to-end-tests/test-scripts/test_table_walkthroughs.sh b/flink-end-to-end-tests/test-scripts/test_table_walkthroughs.sh index 83a3cc48a0..b8f4db50fd 100755 --- a/flink-end-to-end-tests/test-scripts/test_table_walkthroughs.sh +++ b/flink-end-to-end-tests/test-scripts/test_table_walkthroughs.sh @@ -31,7 +31,7 @@ cd "${TEST_DATA_DIR}" ARTIFACT_ID=flink-walkthrough-table-${TEST_TYPE} ARTIFACT_VERSION=0.1 -mvn archetype:generate \ +run_mvn archetype:generate \ -DarchetypeGroupId=org.apache.flink \ -DarchetypeArtifactId=flink-walkthrough-table-${TEST_TYPE} \ -DarchetypeVersion=${FLINK_VERSION} \ @@ -44,7 +44,7 @@ mvn archetype:generate \ cd "${ARTIFACT_ID}" -mvn clean package -nsu > compile-output.txt +run_mvn clean package > compile-output.txt if [[ `grep -c "BUILD FAILURE" compile-output.txt` -eq '1' ]]; then echo "Failure: The walk-through did not successfully compile" diff --git a/tools/azure-pipelines/jobs-template.yml b/tools/azure-pipelines/jobs-template.yml index 821bcabe70..911fa7f040 100644 --- a/tools/azure-pipelines/jobs-template.yml +++ b/tools/azure-pipelines/jobs-template.yml @@ -160,14 +160,16 @@ jobs: echo "##vso[task.setvariable variable=PATH]$JAVA_HOME_11_X64/bin:$PATH" displayName: "Set to jdk11" condition: eq('${{parameters.jdk}}', 'jdk11') - - script: ./tools/travis/setup_maven.sh + - script: | + source ./tools/ci/maven-utils.sh + setup_maven displayName: Setup Maven 3.2.5 - script: ./tools/azure-pipelines/setup_docker.sh displayName: Setup Docker - script: ./tools/azure-pipelines/free_disk_space.sh displayName: Free up disk space - script: sudo apt-get install -y bc - - script: M2_HOME=/home/vsts/maven_cache/apache-maven-3.2.5/ PATH=/home/vsts/maven_cache/apache-maven-3.2.5/bin:$PATH ${{parameters.environment}} STAGE=compile ./tools/azure_controller.sh compile + - script: ${{parameters.environment}} STAGE=compile ./tools/azure_controller.sh compile displayName: Build Flink # TODO remove pre-commit tests script by adding the tests to the nightly script # - script: FLINK_DIR=build-target ./flink-end-to-end-tests/run-pre-commit-tests.sh diff --git a/tools/azure-pipelines/prepare_precommit.sh b/tools/azure-pipelines/prepare_precommit.sh index 23d8e70382..cac545bde4 100755 --- a/tools/azure-pipelines/prepare_precommit.sh +++ b/tools/azure-pipelines/prepare_precommit.sh @@ -31,9 +31,8 @@ find . -type f -name '*.timestamp' | xargs touch export M2_HOME=/home/vsts/maven_cache/apache-maven-3.2.5/ export PATH=/home/vsts/maven_cache/apache-maven-3.2.5/bin:$PATH -mvn -version -MVN_CALL="mvn install --settings ./tools/azure-pipelines/google-mirror-settings.xml -DskipTests -Drat.skip $PROFILE" -echo "Invoking Maven: '$MVN_CALL'" +run_mvn -version +MVN_CALL="run_mvn install -DskipTests -Drat.skip" $MVN_CALL EXIT_CODE=$? diff --git a/tools/azure_controller.sh b/tools/azure_controller.sh index 6e3f37cf22..a77ea31d59 100755 --- a/tools/azure_controller.sh +++ b/tools/azure_controller.sh @@ -17,15 +17,6 @@ # limitations under the License. ################################################################################ -echo $M2_HOME -echo $PATH -echo $MAVEN_OPTS - -mvn -version -echo "Commit: $(git rev-parse HEAD)" - - - HERE="`dirname \"$0\"`" # relative HERE="`( cd \"$HERE\" && pwd )`" # absolutized and normalized if [ -z "$HERE" ] ; then @@ -36,6 +27,14 @@ fi source "${HERE}/travis/stage.sh" source "${HERE}/travis/shade.sh" +source "${HERE}/ci/maven-utils.sh" + +echo $M2_HOME +echo $PATH +echo $MAVEN_OPTS + +run_mvn -version +echo "Commit: $(git rev-parse HEAD)" print_system_info() { echo "CPU information" @@ -63,8 +62,7 @@ EXIT_CODE=0 # mirror. We use a different mirror because the official maven central mirrors # often lead to connection timeouts (probably due to rate-limiting) -# adding -Dmaven.wagon.http.pool=false (see https://developercommunity.visualstudio.com/content/problem/851041/microsoft-hosted-agents-run-into-maven-central-tim.html) -MVN="mvn clean install --settings ./tools/azure-pipelines/google-mirror-settings.xml $MAVEN_OPTS -nsu -Dflink.convergence.phase=install -Pcheck-convergence -Dflink.forkCount=2 -Dflink.forkCountTestPackage=2 -Dmaven.wagon.http.pool=false -Dmaven.javadoc.skip=true -B -U -DskipTests $PROFILE" +MVN="run_mvn clean install $MAVEN_OPTS -Dflink.convergence.phase=install -Pcheck-convergence -Dflink.forkCount=2 -Dflink.forkCountTestPackage=2 -Dmaven.javadoc.skip=true -U -DskipTests" # Run actual compile&test steps if [ $STAGE == "$STAGE_COMPILE" ]; then diff --git a/tools/azure-pipelines/google-mirror-settings.xml b/tools/ci/google-mirror-settings.xml similarity index 100% rename from tools/azure-pipelines/google-mirror-settings.xml rename to tools/ci/google-mirror-settings.xml diff --git a/tools/ci/maven-utils.sh b/tools/ci/maven-utils.sh new file mode 100755 index 0000000000..f36e374998 --- /dev/null +++ b/tools/ci/maven-utils.sh @@ -0,0 +1,69 @@ +#!/usr/bin/env bash +# 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. + +CI_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" + +MAVEN_VERSION="3.2.5" +MAVEN_CACHE_DIR=${HOME}/maven_cache +MAVEN_VERSIONED_DIR=${MAVEN_CACHE_DIR}/apache-maven-${MAVEN_VERSION} + +export MVN_GLOBAL_OPTIONS="" +# see https://developercommunity.visualstudio.com/content/problem/851041/microsoft-hosted-agents-run-into-maven-central-tim.html +MVN_GLOBAL_OPTIONS+="-Dmaven.wagon.http.pool=false " +# use google mirror everywhere +MVN_GLOBAL_OPTIONS+="--settings $CI_DIR/google-mirror-settings.xml " +# logging +MVN_GLOBAL_OPTIONS+="-Dorg.slf4j.simpleLogger.showDateTime=true -Dorg.slf4j.simpleLogger.dateTimeFormat=HH:mm:ss.SSS -Dorg.slf4j.simpleLogger.log.org.apache.maven.cli.transfer.Slf4jMavenTransferListener=warn " +# suppress snapshot updates +MVN_GLOBAL_OPTIONS+="--no-snapshot-updates " +# enable non-interactive batch mode +MVN_GLOBAL_OPTIONS+="-B " +# globally control the build profile details +MVN_GLOBAL_OPTIONS+="$PROFILE " + +# Utility for invoking Maven in CI +function run_mvn { + MVN_CMD="mvn" + if [[ "$M2_HOME" != "" ]]; then + MVN_CMD="${M2_HOME}/bin/mvn" + fi + + ARGS=$@ + INVOCATION="$MVN_CMD $MVN_GLOBAL_OPTIONS $ARGS" + if [[ "$MVN_RUN_VERBOSE" != "false" ]]; then + echo "Invoking mvn with '$INVOCATION'" + fi + ${INVOCATION} +} +export -f run_mvn + +function setup_maven { + if [ ! -d "${MAVEN_VERSIONED_DIR}" ]; then + wget https://archive.apache.org/dist/maven/maven-3/${MAVEN_VERSION}/binaries/apache-maven-${MAVEN_VERSION}-bin.zip + unzip -d "${MAVEN_CACHE_DIR}" -qq "apache-maven-${MAVEN_VERSION}-bin.zip" + rm "apache-maven-${MAVEN_VERSION}-bin.zip" + fi + + export M2_HOME="${MAVEN_VERSIONED_DIR}" + echo "##vso[task.setvariable variable=M2_HOME]$M2_HOME" + + # just in case: clean up the .m2 home and remove invalid jar files + if [ -d "${HOME}/.m2/repository/" ]; then + find ${HOME}/.m2/repository/ -name "*.jar" -exec sh -c 'if ! zip -T {} >/dev/null ; then echo "deleting invalid file: {}"; rm -f {} ; fi' \; + fi + + echo "Installed Maven ${MAVEN_VERSION} to ${M2_HOME}" +} diff --git a/tools/travis/setup_maven.sh b/tools/travis/setup_maven.sh deleted file mode 100755 index b5c44747d9..0000000000 --- a/tools/travis/setup_maven.sh +++ /dev/null @@ -1,39 +0,0 @@ -#!/usr/bin/env bash -################################################################################ -# 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. -################################################################################ - -MAVEN_VERSION="3.2.5" -MAVEN_CACHE_DIR=${HOME}/maven_cache -MAVEN_VERSIONED_DIR=${MAVEN_CACHE_DIR}/apache-maven-${MAVEN_VERSION} - -if [ ! -d "${MAVEN_VERSIONED_DIR}" ]; then - wget https://archive.apache.org/dist/maven/maven-3/${MAVEN_VERSION}/binaries/apache-maven-${MAVEN_VERSION}-bin.zip - unzip -d "${MAVEN_CACHE_DIR}" -qq "apache-maven-${MAVEN_VERSION}-bin.zip" - rm "apache-maven-${MAVEN_VERSION}-bin.zip" -fi - -export M2_HOME="${MAVEN_VERSIONED_DIR}" -export PATH=${M2_HOME}/bin:${PATH} -export MAVEN_OPTS="${MAVEN_OPTS} -Dorg.slf4j.simpleLogger.showDateTime=true -Dorg.slf4j.simpleLogger.dateTimeFormat=HH:mm:ss.SSS" - -# just in case: clean up the .m2 home and remove invalid jar files -if [ -d "${HOME}/.m2/repository/" ]; then - find ${HOME}/.m2/repository/ -name "*.jar" -exec sh -c 'if ! zip -T {} >/dev/null ; then echo "deleting invalid file: {}"; rm -f {} ; fi' \; -fi - -echo "Installed Maven ${MAVEN_VERSION} to ${M2_HOME}" diff --git a/tools/travis_controller.sh b/tools/travis_controller.sh index c2e28b71de..7f8fd0b34e 100755 --- a/tools/travis_controller.sh +++ b/tools/travis_controller.sh @@ -90,7 +90,7 @@ EXIT_CODE=0 # Run actual compile&test steps if [ $STAGE == "$STAGE_COMPILE" ]; then - MVN="mvn clean install -nsu -Dflink.convergence.phase=install -Pcheck-convergence -Dflink.forkCount=2 -Dflink.forkCountTestPackage=2 -Dmaven.javadoc.skip=true -B -DskipTests $PROFILE -Dorg.slf4j.simpleLogger.log.org.apache.maven.cli.transfer.Slf4jMavenTransferListener=warn -Dmaven.wagon.http.pool=false" + MVN="run_mvn clean install -Dflink.convergence.phase=install -Pcheck-convergence -Dflink.forkCount=2 -Dflink.forkCountTestPackage=2 -Dmaven.javadoc.skip=true -DskipTests" $MVN EXIT_CODE=$? diff --git a/tools/travis_watchdog.sh b/tools/travis_watchdog.sh index 055ed75c15..bb05113b4f 100755 --- a/tools/travis_watchdog.sh +++ b/tools/travis_watchdog.sh @@ -26,6 +26,7 @@ if [ -z "$HERE" ] ; then fi source "${HERE}/travis/stage.sh" +source "${HERE}/ci/maven-utils.sh" ARTIFACTS_DIR="${HERE}/artifacts" @@ -63,21 +64,17 @@ MVN_TEST_MODULES=$(get_test_modules_for_stage ${TEST}) # Maven command to run. We set the forkCount manually, because otherwise Maven sees too many cores # on the Travis VMs. Set forkCountTestPackage to 1 for container-based environment (4 GiB memory) # and 2 for sudo-enabled environment (7.5 GiB memory). -# -# -nsu option forbids downloading snapshot artifacts. The only snapshot artifacts we depend are from -# Flink, which however should all be built locally. see FLINK-7230 -# -MVN_LOGGING_OPTIONS="-Dlog.dir=${ARTIFACTS_DIR} -Dlog4j.configurationFile=file://$LOG4J_PROPERTIES -Dorg.slf4j.simpleLogger.log.org.apache.maven.cli.transfer.Slf4jMavenTransferListener=warn" -MVN_COMMON_OPTIONS="-nsu -Dflink.forkCount=2 -Dflink.forkCountTestPackage=2 -Dfast -Dmaven.wagon.http.pool=false -B -Pskip-webui-build $MVN_LOGGING_OPTIONS" +MVN_LOGGING_OPTIONS="-Dlog.dir=${ARTIFACTS_DIR} -Dlog4j.configurationFile=file://$LOG4J_PROPERTIES" +MVN_COMMON_OPTIONS="-Dflink.forkCount=2 -Dflink.forkCountTestPackage=2 -Dfast -Pskip-webui-build $MVN_LOGGING_OPTIONS" MVN_COMPILE_OPTIONS="-DskipTests" MVN_TEST_OPTIONS="-Dflink.tests.with-openssl" e2e_modules=$(find flink-end-to-end-tests -mindepth 2 -maxdepth 5 -name 'pom.xml' -printf '%h\n' | sort -u | tr '\n' ',') -MVN_COMPILE="mvn $MVN_COMMON_OPTIONS $MVN_COMPILE_OPTIONS $PROFILE $MVN_COMPILE_MODULES install" -MVN_TEST="mvn $MVN_COMMON_OPTIONS $MVN_TEST_OPTIONS $PROFILE $MVN_TEST_MODULES verify" +MVN_COMPILE="run_mvn $MVN_COMMON_OPTIONS $MVN_COMPILE_OPTIONS $PROFILE $MVN_COMPILE_MODULES install" +MVN_TEST="run_mvn $MVN_COMMON_OPTIONS $MVN_TEST_OPTIONS $PROFILE $MVN_TEST_MODULES verify" # don't move the e2e-pre-commit profile activation into the misc entry in .travis.yml, since it breaks caching -MVN_E2E="mvn $MVN_COMMON_OPTIONS $MVN_TEST_OPTIONS $PROFILE -Pe2e-pre-commit -pl ${e2e_modules},flink-dist verify" +MVN_E2E="run_mvn $MVN_COMMON_OPTIONS $MVN_TEST_OPTIONS -Pe2e-pre-commit -pl ${e2e_modules},flink-dist verify" MVN_PID="${ARTIFACTS_DIR}/watchdog.mvn.pid" MVN_EXIT="${ARTIFACTS_DIR}/watchdog.mvn.exit" diff --git a/tools/verify_scala_suffixes.sh b/tools/verify_scala_suffixes.sh index 0b021fd488..714f6dbc19 100755 --- a/tools/verify_scala_suffixes.sh +++ b/tools/verify_scala_suffixes.sh @@ -117,7 +117,7 @@ while read line; do block_infected=1 fi fi -done < <(mvn -nsu dependency:tree -Dincludes=org.scala-lang,:*_2.1*:: -pl ${excluded_modules} ${MAVEN_ARGUMENTS} | tee /dev/tty) +done < <(run_mvn dependency:tree -Dincludes=org.scala-lang,:*_2.1*:: -pl ${excluded_modules} ${MAVEN_ARGUMENTS} | tee /dev/tty) # deduplicate and sort -- Gitee From 8cf8d8d86121092714ef71109f91670506457710 Mon Sep 17 00:00:00 2001 From: "bowen.li" Date: Thu, 26 Mar 2020 11:22:53 -0700 Subject: [PATCH 323/885] [FLINK-16810][jdbc] add back PostgresCatalogITCase add back PostgresCatalogITCase which is supposed to be part of #11468 --- .../jdbc/catalog/PostgresCatalogITCase.java | 72 +++++++++++++++++++ 1 file changed, 72 insertions(+) create mode 100644 flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogITCase.java diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogITCase.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogITCase.java new file mode 100644 index 0000000000..143206c745 --- /dev/null +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogITCase.java @@ -0,0 +1,72 @@ +package org.apache.flink.api.java.io.jdbc.catalog; + +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableUtils; +import org.apache.flink.types.Row; + +import org.junit.Test; + +import java.util.List; + +import static org.apache.flink.api.java.io.jdbc.catalog.PostgresCatalog.DEFAULT_DATABASE; +import static org.apache.flink.table.api.config.ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM; +import static org.junit.Assert.assertEquals; + +/** + * E2E test for {@link PostgresCatalog}. + */ +public class PostgresCatalogITCase extends PostgresCatalogTestBase { + + @Test + public void test_withoutSchema() throws Exception { + TableEnvironment tEnv = getTableEnvWithPgCatalog(); + + List results = TableUtils.collectToList( + tEnv.sqlQuery(String.format("select * from %s", TABLE1))); + assertEquals("[1]", results.toString()); + } + + @Test + public void test_withSchema() throws Exception { + TableEnvironment tEnv = getTableEnvWithPgCatalog(); + + List results = TableUtils.collectToList( + tEnv.sqlQuery(String.format("select * from `%s`", PostgresTablePath.fromFlinkTableName(TABLE1)))); + assertEquals("[1]", results.toString()); + } + + @Test + public void test_fullPath() throws Exception { + TableEnvironment tEnv = getTableEnvWithPgCatalog(); + + List results = TableUtils.collectToList( + tEnv.sqlQuery(String.format("select * from %s.%s.`%s`", + TEST_CATALOG_NAME, + DEFAULT_DATABASE, + PostgresTablePath.fromFlinkTableName(TABLE1)))); + assertEquals("[1]", results.toString()); + } + + @Test + public void test_insert() throws Exception { + TableEnvironment tEnv = getTableEnvWithPgCatalog(); + + tEnv.sqlUpdate(String.format("insert into %s select * from `%s`", TABLE4, TABLE1)); + tEnv.execute("test"); + + List results = TableUtils.collectToList( + tEnv.sqlQuery(String.format("select * from %s", TABLE1))); + assertEquals("[1]", results.toString()); + } + + private TableEnvironment getTableEnvWithPgCatalog() { + EnvironmentSettings settings = EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build(); + TableEnvironment tableEnv = TableEnvironment.create(settings); + tableEnv.getConfig().getConfiguration().setInteger(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM.key(), 1); + + tableEnv.registerCatalog(TEST_CATALOG_NAME, catalog); + tableEnv.useCatalog(TEST_CATALOG_NAME); + return tableEnv; + } +} -- Gitee From ebb9f5991090a32f0dfd40760f508bdf494b9a8b Mon Sep 17 00:00:00 2001 From: tison Date: Fri, 27 Mar 2020 03:48:50 +0800 Subject: [PATCH 324/885] [hotfix][connector] Correct exception message (#11523) --- .../connectors/kafka/internal/KafkaPartitionDiscoverer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaPartitionDiscoverer.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaPartitionDiscoverer.java index 1c871bca42..25f84792a5 100644 --- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaPartitionDiscoverer.java +++ b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaPartitionDiscoverer.java @@ -77,7 +77,7 @@ public class KafkaPartitionDiscoverer extends AbstractPartitionDiscoverer { final List kafkaPartitions = kafkaConsumer.partitionsFor(topic); if (kafkaPartitions == null) { - throw new RuntimeException("Could not fetch partitions for %s. Make sure that the topic exists.".format(topic)); + throw new RuntimeException(String.format("Could not fetch partitions for %s. Make sure that the topic exists.", topic)); } for (PartitionInfo partitionInfo : kafkaPartitions) { -- Gitee From eeb91086420a956eba3439ff9530685e25f772cf Mon Sep 17 00:00:00 2001 From: "bowen.li" Date: Thu, 26 Mar 2020 15:57:23 -0700 Subject: [PATCH 325/885] [hotfix][jdbc] add license header to PostgresCatalogITCase --- .../io/jdbc/catalog/PostgresCatalogITCase.java | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogITCase.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogITCase.java index 143206c745..8421cf60ce 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogITCase.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogITCase.java @@ -1,3 +1,21 @@ +/* + * 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.api.java.io.jdbc.catalog; import org.apache.flink.table.api.EnvironmentSettings; -- Gitee From 7feba945907e6b5a31f8dcf27958594248f1f932 Mon Sep 17 00:00:00 2001 From: Xuannan Date: Fri, 27 Mar 2020 09:59:27 +0800 Subject: [PATCH 326/885] [FLINK-16127][docs-zh] Translate "Fault Tolerance Guarantees" page of connectors into Chinese This closes #11129 --- docs/dev/connectors/guarantees.zh.md | 55 +++++++++++++--------------- 1 file changed, 25 insertions(+), 30 deletions(-) diff --git a/docs/dev/connectors/guarantees.zh.md b/docs/dev/connectors/guarantees.zh.md index cd9c52d8f5..2c96466c62 100644 --- a/docs/dev/connectors/guarantees.zh.md +++ b/docs/dev/connectors/guarantees.zh.md @@ -23,14 +23,11 @@ specific language governing permissions and limitations under the License. --> -Flink's fault tolerance mechanism recovers programs in the presence of failures and -continues to execute them. Such failures include machine hardware failures, network failures, -transient program failures, etc. +当程序出现错误的时候,Flink 的容错机制能恢复并继续运行程序。这种错误包括机器硬件故障、网络故障、瞬态程序故障等等。 -Flink can guarantee exactly-once state updates to user-defined state only when the source participates in the -snapshotting mechanism. The following table lists the state update guarantees of Flink coupled with the bundled connectors. +只有当 source 参与了快照机制的时候,Flink 才能保证对自定义状态的精确一次更新。下表列举了 Flink 与其自带连接器的状态更新的保证。 -Please read the documentation of each connector to understand the details of the fault tolerance guarantees. +请阅读各个连接器的文档来了解容错保证的细节。 @@ -43,50 +40,48 @@ Please read the documentation of each connector to understand the details of the - - + + - + - + - + - + - + - + - +
    Apache Kafkaexactly onceUse the appropriate Kafka connector for your version精确一次根据你的版本用恰当的 Kafka 连接器
    AWS Kinesis Streamsexactly once精确一次
    RabbitMQat most once (v 0.10) / exactly once (v 1.0) 至多一次 (v 0.10) / 精确一次 (v 1.0)
    Twitter Streaming APIat most once至多一次
    Google PubSubat least once至少一次
    Collectionsexactly once精确一次
    Filesexactly once精确一次
    Socketsat most once至多一次
    -To guarantee end-to-end exactly-once record delivery (in addition to exactly-once state semantics), the data sink needs -to take part in the checkpointing mechanism. The following table lists the delivery guarantees (assuming exactly-once -state updates) of Flink coupled with bundled sinks: +为了保证端到端精确一次的数据交付(在精确一次的状态语义上更进一步),sink需要参与 checkpointing 机制。下表列举了 Flink 与其自带 sink 的交付保证(假设精确一次状态更新)。 @@ -99,47 +94,47 @@ state updates) of Flink coupled with bundled sinks: - - + + - + - - + + - - + + - + - + - + - + - + -- Gitee From 66c2913742c2d6e92042c7466a398fb0f468d6b3 Mon Sep 17 00:00:00 2001 From: Xuannan Date: Fri, 27 Mar 2020 10:01:51 +0800 Subject: [PATCH 327/885] [FLINK-16128][docs-zh] Translate "Google Cloud PubSub" page into Chinese This closes #11151 --- docs/dev/connectors/pubsub.md | 4 +-- docs/dev/connectors/pubsub.zh.md | 54 ++++++++++++++------------------ 2 files changed, 25 insertions(+), 33 deletions(-) diff --git a/docs/dev/connectors/pubsub.md b/docs/dev/connectors/pubsub.md index 59b3e0096b..6055d9f67a 100644 --- a/docs/dev/connectors/pubsub.md +++ b/docs/dev/connectors/pubsub.md @@ -143,13 +143,13 @@ env.addSource(pubsubSource) -### Atleast once guarantee +### At least once guarantee #### SourceFunction There are several reasons why a message might be send multiple times, such as failure scenarios on Google PubSub's side. -Another reason is when the acknowledgement deadline has passed. This is the time between receiving the message and between acknowledging the message. The PubSubSource will only acknowledge a message on successful checkpoints to guarantee Atleast-Once. This does mean if the time between successful checkpoints is larger than the acknowledgment deadline of your subscription messages will most likely be processed multiple times. +Another reason is when the acknowledgement deadline has passed. This is the time between receiving the message and acknowledging the message. The PubSubSource will only acknowledge a message on successful checkpoints to guarantee at-least-once. This does mean if the time between successful checkpoints is larger than the acknowledgment deadline of your subscription messages will most likely be processed multiple times. For this reason it's recommended to have a (much) lower checkpoint interval than acknowledgement deadline. diff --git a/docs/dev/connectors/pubsub.zh.md b/docs/dev/connectors/pubsub.zh.md index 59b3e0096b..eaf5f582c8 100644 --- a/docs/dev/connectors/pubsub.zh.md +++ b/docs/dev/connectors/pubsub.zh.md @@ -23,9 +23,7 @@ specific language governing permissions and limitations under the License. --> -This connector provides a Source and Sink that can read from and write to -[Google Cloud PubSub](https://cloud.google.com/pubsub). To use this connector, add the -following dependency to your project: +这个连接器可向 [Google Cloud PubSub](https://cloud.google.com/pubsub) 读取与写入数据。添加下面的依赖来使用此连接器: {% highlight xml %} @@ -36,27 +34,20 @@ following dependency to your project: {% endhighlight %}

    -Note: This connector has been added to Flink recently. It has not received widespread testing yet. +注意:此连接器最近才加到 Flink 里,还未接受广泛测试。

    -Note that the streaming connectors are currently not part of the binary -distribution. See -[here]({{ site.baseurl }}/dev/projectsetup/dependencies.html) -for information about how to package the program with the libraries for -cluster execution. - - +注意连接器目前还不是二进制发行版的一部分,添加依赖、打包配置以及集群运行信息请参考[这里]({{ site.baseurl }}/zh/dev/projectsetup/dependencies.html) ## Consuming or Producing PubSubMessages -The connector provides a connectors for receiving and sending messages from and to Google PubSub. -Google PubSub has an `at-least-once` guarantee and as such the connector delivers the same guarantees. +连接器可以接收和发送 Google PubSub 的信息。和 Google PubSub 一样,这个连接器能够保证`至少一次`的语义。 ### PubSub SourceFunction -The class `PubSubSource` has a builder to create PubSubsources: `PubSubSource.newBuilder(...)` +`PubSubSource` 类的对象由构建类来构建: `PubSubSource.newBuilder(...)` -There are several optional methods to alter how the PubSubSource is created, the bare minimum is to provide a Google project, Pubsub subscription and a way to deserialize the PubSubMessages. +有多种可选的方法来创建 PubSubSource,但最低要求是要提供 Google Project、Pubsub 订阅和反序列化 PubSubMessages 的方法。 Example: @@ -77,13 +68,13 @@ streamExecEnv.addSource(source); -Currently the source functions [pulls](https://cloud.google.com/pubsub/docs/pull) messages from PubSub, [push endpoints](https://cloud.google.com/pubsub/docs/push) are not supported. +当前还不支持 PubSub 的 source functions [pulls](https://cloud.google.com/pubsub/docs/pull) messages 和 [push endpoints](https://cloud.google.com/pubsub/docs/push)。 ### PubSub Sink -The class `PubSubSink` has a builder to create PubSubSinks. `PubSubSink.newBuilder(...)` +`PubSubSink` 类的对象由构建类来构建: `PubSubSink.newBuilder(...)` -This builder works in a similar way to the PubSubSource. +构建类的使用方式与 PubSubSource 类似。 Example: @@ -106,17 +97,18 @@ dataStream.addSink(pubsubSink); ### Google Credentials -Google uses [Credentials](https://cloud.google.com/docs/authentication/production) to authenticate and authorize applications so that they can use Google Cloud Platform resources (such as PubSub). +应用程序需要使用 [Credentials](https://cloud.google.com/docs/authentication/production) 来通过认证和授权才能使用 Google Cloud Platform 的资源,例如 PubSub。 + +上述的两个构建类都允许你提供 Credentials, 但是连接器默认会通过环境变量: [GOOGLE_APPLICATION_CREDENTIALS](https://cloud.google.com/docs/authentication/production#obtaining_and_providing_service_account_credentials_manually) 来获取 Credentials 的路径。 -Both builders allow you to provide these credentials but by default the connectors will look for an environment variable: [GOOGLE_APPLICATION_CREDENTIALS](https://cloud.google.com/docs/authentication/production#obtaining_and_providing_service_account_credentials_manually) which should point to a file containing the credentials. +如果你想手动提供 Credentials,例如你想从外部系统读取 Credentials,你可以使用 `PubSubSource.newBuilder(...).withCredentials(...)`。 -If you want to provide Credentials manually, for instance if you read the Credentials yourself from an external system, you can use `PubSubSource.newBuilder(...).withCredentials(...)`. +### 集成测试 -### Integration testing +在集成测试的时候,如果你不想直接连 PubSub 而是想读取和写入一个 docker container,可以参照 [PubSub testing locally](https://cloud.google.com/pubsub/docs/emulator)。 -When running integration tests you might not want to connect to PubSub directly but use a docker container to read and write to. (See: [PubSub testing locally](https://cloud.google.com/pubsub/docs/emulator)) +下面的例子展示了如何使用 source 来从仿真器读取信息并发送回去: -The following example shows how you would create a source to read messages from the emulator and send them back:
    {% highlight java %} @@ -143,22 +135,22 @@ env.addSource(pubsubSource)
    -### Atleast once guarantee +### 至少一次语义保证 #### SourceFunction -There are several reasons why a message might be send multiple times, such as failure scenarios on Google PubSub's side. +有很多原因导致会一个信息会被多次发出,例如 Google PubSub 的故障。 -Another reason is when the acknowledgement deadline has passed. This is the time between receiving the message and between acknowledging the message. The PubSubSource will only acknowledge a message on successful checkpoints to guarantee Atleast-Once. This does mean if the time between successful checkpoints is larger than the acknowledgment deadline of your subscription messages will most likely be processed multiple times. +另一个可能的原因是超过了确认的截止时间,即收到与确认信息之间的时间间隔。PubSubSource 只有在信息被成功快照之后才会确认以保证至少一次的语义。这意味着,如果你的快照间隔大于信息确认的截止时间,那么你订阅的信息很有可能会被多次处理。 -For this reason it's recommended to have a (much) lower checkpoint interval than acknowledgement deadline. +因此,我们建议把快照的间隔设置得比信息确认截止时间更短。 -See [PubSub](https://cloud.google.com/pubsub/docs/subscriber) for details on how to increase the acknowledgment deadline of your subscription. +参照 [PubSub](https://cloud.google.com/pubsub/docs/subscriber) 来增加信息确认截止时间。 -Note: The metric `PubSubMessagesProcessedNotAcked` shows how many messages are waiting for the next checkpoint before they will be acknowledged. +注意: `PubSubMessagesProcessedNotAcked` 显示了有多少信息正在等待下一个 checkpoint 还没被确认。 #### SinkFunction -The sink function buffers messages that are to be send to PubSub for a short amount of time for performance reasons. Before each checkpoint this buffer is flushed and the checkpoint will not succeed unless the messages have been delivered to PubSub. +Sink function 会把准备发到 PubSub 的信息短暂地缓存以提高性能。每次 checkpoint 前,它会刷新缓冲区,并且只有当所有信息成功发送到 PubSub 之后,checkpoint 才会成功完成。 {% top %} -- Gitee From e91bbcf0a3369465dfe3e5e0b8332000e8e0f1f1 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Fri, 27 Mar 2020 10:12:40 +0800 Subject: [PATCH 328/885] [hotfix][python] Improve the imports to use full path when importing a PyFlink module (#11525) --- flink-python/pyflink/fn_execution/coder_impl.py | 2 +- flink-python/pyflink/fn_execution/coders.py | 2 +- flink-python/pyflink/ml/api/ml_environment.py | 6 +++--- flink-python/pyflink/ml/api/ml_environment_factory.py | 6 +++--- flink-python/pyflink/table/table_config.py | 4 ++-- flink-python/pyflink/testing/source_sink_utils.py | 2 +- flink-python/pyflink/testing/test_case_utils.py | 7 ++++--- 7 files changed, 15 insertions(+), 14 deletions(-) diff --git a/flink-python/pyflink/fn_execution/coder_impl.py b/flink-python/pyflink/fn_execution/coder_impl.py index f453a88e47..ca7ea64db0 100644 --- a/flink-python/pyflink/fn_execution/coder_impl.py +++ b/flink-python/pyflink/fn_execution/coder_impl.py @@ -27,7 +27,7 @@ import pyarrow as pa from apache_beam.coders.coder_impl import StreamCoderImpl, create_InputStream, create_OutputStream from pyflink.fn_execution.ResettableIO import ResettableIO -from pyflink.table import Row +from pyflink.table.types import Row class FlattenRowCoderImpl(StreamCoderImpl): diff --git a/flink-python/pyflink/fn_execution/coders.py b/flink-python/pyflink/fn_execution/coders.py index 759fed839e..a9da9bfe89 100644 --- a/flink-python/pyflink/fn_execution/coders.py +++ b/flink-python/pyflink/fn_execution/coders.py @@ -32,7 +32,7 @@ from apache_beam.typehints import typehints from pyflink.fn_execution import coder_impl from pyflink.fn_execution import flink_fn_execution_pb2 from pyflink.fn_execution.sdk_worker_main import pipeline_options -from pyflink.table import Row +from pyflink.table.types import Row FLINK_SCALAR_FUNCTION_SCHEMA_CODER_URN = "flink:coder:schema:scalar_function:v1" FLINK_TABLE_FUNCTION_SCHEMA_CODER_URN = "flink:coder:schema:table_function:v1" diff --git a/flink-python/pyflink/ml/api/ml_environment.py b/flink-python/pyflink/ml/api/ml_environment.py index 0895eca30e..b74ba12029 100644 --- a/flink-python/pyflink/ml/api/ml_environment.py +++ b/flink-python/pyflink/ml/api/ml_environment.py @@ -16,9 +16,9 @@ # limitations under the License. ################################################################################ -from pyflink.dataset import ExecutionEnvironment -from pyflink.datastream import StreamExecutionEnvironment -from pyflink.table import BatchTableEnvironment, StreamTableEnvironment +from pyflink.dataset.execution_environment import ExecutionEnvironment +from pyflink.datastream.stream_execution_environment import StreamExecutionEnvironment +from pyflink.table.table_environment import BatchTableEnvironment, StreamTableEnvironment class MLEnvironment(object): diff --git a/flink-python/pyflink/ml/api/ml_environment_factory.py b/flink-python/pyflink/ml/api/ml_environment_factory.py index 0825fb07d0..1001798d8c 100644 --- a/flink-python/pyflink/ml/api/ml_environment_factory.py +++ b/flink-python/pyflink/ml/api/ml_environment_factory.py @@ -18,9 +18,9 @@ from typing import Optional from pyflink.ml.api.ml_environment import MLEnvironment -from pyflink.dataset import ExecutionEnvironment -from pyflink.datastream import StreamExecutionEnvironment -from pyflink.table import BatchTableEnvironment, StreamTableEnvironment +from pyflink.dataset.execution_environment import ExecutionEnvironment +from pyflink.datastream.stream_execution_environment import StreamExecutionEnvironment +from pyflink.table.table_environment import BatchTableEnvironment, StreamTableEnvironment from pyflink.java_gateway import get_gateway import threading diff --git a/flink-python/pyflink/table/table_config.py b/flink-python/pyflink/table/table_config.py index 72389b41e5..299e563751 100644 --- a/flink-python/pyflink/table/table_config.py +++ b/flink-python/pyflink/table/table_config.py @@ -18,10 +18,10 @@ from py4j.compat import long -from pyflink.common import Configuration +from pyflink.common.configuration import Configuration from pyflink.common.dependency_manager import DependencyManager from pyflink.java_gateway import get_gateway -from pyflink.table import SqlDialect +from pyflink.table.sql_dialect import SqlDialect __all__ = ['TableConfig'] diff --git a/flink-python/pyflink/testing/source_sink_utils.py b/flink-python/pyflink/testing/source_sink_utils.py index 2406d988d7..e8fd012df4 100644 --- a/flink-python/pyflink/testing/source_sink_utils.py +++ b/flink-python/pyflink/testing/source_sink_utils.py @@ -23,7 +23,7 @@ from py4j.java_gateway import java_import from pyflink.find_flink_home import _find_flink_source_root from pyflink.java_gateway import get_gateway -from pyflink.table import TableSink +from pyflink.table.sinks import TableSink from pyflink.table.types import _to_java_type from pyflink.util import utils diff --git a/flink-python/pyflink/testing/test_case_utils.py b/flink-python/pyflink/testing/test_case_utils.py index a48bd1922e..2ada340751 100644 --- a/flink-python/pyflink/testing/test_case_utils.py +++ b/flink-python/pyflink/testing/test_case_utils.py @@ -29,10 +29,11 @@ from py4j.java_gateway import JavaObject from py4j.protocol import Py4JJavaError from pyflink.table.sources import CsvTableSource -from pyflink.dataset import ExecutionEnvironment -from pyflink.datastream import StreamExecutionEnvironment +from pyflink.dataset.execution_environment import ExecutionEnvironment +from pyflink.datastream.stream_execution_environment import StreamExecutionEnvironment from pyflink.find_flink_home import _find_flink_home, _find_flink_source_root -from pyflink.table import BatchTableEnvironment, StreamTableEnvironment, EnvironmentSettings +from pyflink.table.table_environment import BatchTableEnvironment, StreamTableEnvironment +from pyflink.table.environment_settings import EnvironmentSettings from pyflink.java_gateway import get_gateway -- Gitee From 530943eb35ab5b3e7e4c887bb2790f54b8589094 Mon Sep 17 00:00:00 2001 From: Rui Li Date: Mon, 23 Mar 2020 11:54:45 +0800 Subject: [PATCH 329/885] [FLINK-16647][hadoop-compatibility] Refactor HadoopUtils to extract a possible Hadoop conf util method This closes #11440 --- .../java/hadoop/mapred/utils/HadoopUtils.java | 53 +++++++++++-------- 1 file changed, 31 insertions(+), 22 deletions(-) diff --git a/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/api/java/hadoop/mapred/utils/HadoopUtils.java b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/api/java/hadoop/mapred/utils/HadoopUtils.java index 464dcdf430..9e2e2c9f4a 100644 --- a/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/api/java/hadoop/mapred/utils/HadoopUtils.java +++ b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/api/java/hadoop/mapred/utils/HadoopUtils.java @@ -28,7 +28,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.File; +import java.util.Arrays; import java.util.Map; +import java.util.Objects; /** * Utility class to work with Apache Hadoop MapRed classes. @@ -86,31 +88,20 @@ public final class HadoopUtils { } // 2. Approach environment variables - String[] possibleHadoopConfPaths = new String[4]; - possibleHadoopConfPaths[0] = flinkConfiguration.getString(ConfigConstants.PATH_HADOOP_CONFIG, null); - possibleHadoopConfPaths[1] = System.getenv("HADOOP_CONF_DIR"); + for (String possibleHadoopConfPath : possibleHadoopConfPaths(flinkConfiguration)) { + if (new File(possibleHadoopConfPath).exists()) { + if (new File(possibleHadoopConfPath + "/core-site.xml").exists()) { + retConf.addResource(new org.apache.hadoop.fs.Path(possibleHadoopConfPath + "/core-site.xml")); - if (System.getenv("HADOOP_HOME") != null) { - possibleHadoopConfPaths[2] = System.getenv("HADOOP_HOME") + "/conf"; - possibleHadoopConfPaths[3] = System.getenv("HADOOP_HOME") + "/etc/hadoop"; // hadoop 2.2 - } - - for (String possibleHadoopConfPath : possibleHadoopConfPaths) { - if (possibleHadoopConfPath != null) { - if (new File(possibleHadoopConfPath).exists()) { - if (new File(possibleHadoopConfPath + "/core-site.xml").exists()) { - retConf.addResource(new org.apache.hadoop.fs.Path(possibleHadoopConfPath + "/core-site.xml")); - - if (LOG.isDebugEnabled()) { - LOG.debug("Adding " + possibleHadoopConfPath + "/core-site.xml to hadoop configuration"); - } + if (LOG.isDebugEnabled()) { + LOG.debug("Adding " + possibleHadoopConfPath + "/core-site.xml to hadoop configuration"); } - if (new File(possibleHadoopConfPath + "/hdfs-site.xml").exists()) { - retConf.addResource(new org.apache.hadoop.fs.Path(possibleHadoopConfPath + "/hdfs-site.xml")); + } + if (new File(possibleHadoopConfPath + "/hdfs-site.xml").exists()) { + retConf.addResource(new org.apache.hadoop.fs.Path(possibleHadoopConfPath + "/hdfs-site.xml")); - if (LOG.isDebugEnabled()) { - LOG.debug("Adding " + possibleHadoopConfPath + "/hdfs-site.xml to hadoop configuration"); - } + if (LOG.isDebugEnabled()) { + LOG.debug("Adding " + possibleHadoopConfPath + "/hdfs-site.xml to hadoop configuration"); } } } @@ -118,6 +109,24 @@ public final class HadoopUtils { return retConf; } + /** + * Get possible Hadoop conf dir paths, based on environment variables and flink configuration. + * + * @param flinkConfiguration The flink configuration that may contain the path to Hadoop conf dir. + * @return an array of possible paths + */ + public static String[] possibleHadoopConfPaths(org.apache.flink.configuration.Configuration flinkConfiguration) { + String[] possiblePaths = new String[4]; + possiblePaths[0] = flinkConfiguration.getString(ConfigConstants.PATH_HADOOP_CONFIG, null); + possiblePaths[1] = System.getenv("HADOOP_CONF_DIR"); + + if (System.getenv("HADOOP_HOME") != null) { + possiblePaths[2] = System.getenv("HADOOP_HOME") + "/conf"; + possiblePaths[3] = System.getenv("HADOOP_HOME") + "/etc/hadoop"; // hadoop 2.2 + } + return Arrays.stream(possiblePaths).filter(Objects::nonNull).toArray(String[]::new); + } + /** * Private constructor to prevent instantiation. */ -- Gitee From 7984d02dcbccf05aaed47c02e191a2a3727665cf Mon Sep 17 00:00:00 2001 From: Rui Li Date: Mon, 23 Mar 2020 11:55:28 +0800 Subject: [PATCH 330/885] [FLINK-16647][hive] Hive connector should read mapred-site.xml This closes #11440 --- .../flink/table/catalog/hive/HiveCatalog.java | 16 ++++++++++-- .../factories/HiveCatalogFactoryTest.java | 25 ++++++++++++++----- 2 files changed, 33 insertions(+), 8 deletions(-) diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalog.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalog.java index 8cf3abcf2e..e114f1faf0 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalog.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalog.java @@ -70,6 +70,8 @@ import org.apache.flink.table.factories.FunctionDefinitionFactory; import org.apache.flink.table.factories.TableFactory; import org.apache.flink.util.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.common.StatsSetupConst; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.TableType; @@ -97,6 +99,7 @@ import org.slf4j.LoggerFactory; import javax.annotation.Nullable; +import java.io.File; import java.net.MalformedURLException; import java.nio.file.Paths; import java.util.ArrayList; @@ -174,8 +177,17 @@ public class HiveCatalog extends AbstractCatalog { } // create HiveConf from hadoop configuration - return new HiveConf(HadoopUtils.getHadoopConfiguration(new org.apache.flink.configuration.Configuration()), - HiveConf.class); + Configuration hadoopConf = HadoopUtils.getHadoopConfiguration(new org.apache.flink.configuration.Configuration()); + + // Add mapred-site.xml. We need to read configurations like compression codec. + for (String possibleHadoopConfPath : HadoopUtils.possibleHadoopConfPaths(new org.apache.flink.configuration.Configuration())) { + File mapredSite = new File(new File(possibleHadoopConfPath), "mapred-site.xml"); + if (mapredSite.exists()) { + hadoopConf.addResource(new Path(mapredSite.getAbsolutePath())); + break; + } + } + return new HiveConf(hadoopConf, HiveConf.class); } @VisibleForTesting diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/factories/HiveCatalogFactoryTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/factories/HiveCatalogFactoryTest.java index 1339a94c26..0db52b499b 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/factories/HiveCatalogFactoryTest.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/factories/HiveCatalogFactoryTest.java @@ -66,18 +66,29 @@ public class HiveCatalogFactoryTest extends TestLogger { } @Test - public void testLoadHDFSConfigFromEnv() throws IOException { - final String k1 = "what is connector?"; - final String v1 = "Hive"; + public void testLoadHadoopConfigFromEnv() throws IOException { + Map customProps = new HashMap<>(); + String k1 = "what is connector?"; + String v1 = "Hive"; final String catalogName = "HiveCatalog"; // set HADOOP_CONF_DIR env final File hadoopConfDir = tempFolder.newFolder(); final File hdfsSiteFile = new File(hadoopConfDir, "hdfs-site.xml"); writeProperty(hdfsSiteFile, k1, v1); + customProps.put(k1, v1); + + // add mapred-site file + final File mapredSiteFile = new File(hadoopConfDir, "mapred-site.xml"); + k1 = "mapred.site.config.key"; + v1 = "mapred.site.config.val"; + writeProperty(mapredSiteFile, k1, v1); + customProps.put(k1, v1); + final Map originalEnv = System.getenv(); final Map newEnv = new HashMap<>(originalEnv); newEnv.put("HADOOP_CONF_DIR", hadoopConfDir.getAbsolutePath()); + newEnv.remove("HADOOP_HOME"); CommonTestUtils.setEnv(newEnv); // create HiveCatalog use the Hadoop Configuration @@ -86,14 +97,16 @@ public class HiveCatalogFactoryTest extends TestLogger { final HiveConf hiveConf; try { final HiveCatalog hiveCatalog = (HiveCatalog) TableFactoryService.find(CatalogFactory.class, properties) - .createCatalog(catalogName, properties); + .createCatalog(catalogName, properties); hiveConf = hiveCatalog.getHiveConf(); } finally { // set the Env back CommonTestUtils.setEnv(originalEnv); } - //validate the result - assertEquals(v1, hiveConf.get(k1, null)); + // validate the result + for (String key : customProps.keySet()) { + assertEquals(customProps.get(key), hiveConf.get(key, null)); + } } private static void checkEquals(HiveCatalog c1, HiveCatalog c2) { -- Gitee From 51b1a4f6e9baaf1bf5276aa1c8cf3fe4ebb57fe7 Mon Sep 17 00:00:00 2001 From: Rui Li Date: Mon, 23 Mar 2020 11:55:59 +0800 Subject: [PATCH 331/885] [FLINK-16647][table-runtime-blink][hive] Miss file extension when inserting to hive table with compression This closes #11440 --- .../hive/HiveOutputFormatFactory.java | 17 +++++++++----- .../flink/connectors/hive/HiveTableSink.java | 20 ++++++++++++++-- .../table/catalog/hive/client/HiveShim.java | 8 ++++++- .../catalog/hive/client/HiveShimV100.java | 21 ++++++++++++----- .../catalog/hive/client/HiveShimV110.java | 23 ++++++++++++++----- .../hive/HiveOutputFormatFactoryTest.java | 5 ++-- .../hive/TableEnvHiveConnectorTest.java | 22 ++++++++++++++++++ .../sink/filesystem/OutputFileConfig.java | 4 ++-- .../filesystem/FileSystemOutputFormat.java | 18 ++++++++++++--- .../filesystem/PartitionTempFileManager.java | 18 ++++++++++++--- 10 files changed, 125 insertions(+), 31 deletions(-) diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveOutputFormatFactory.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveOutputFormatFactory.java index 9629566418..378d9ef540 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveOutputFormatFactory.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveOutputFormatFactory.java @@ -61,7 +61,7 @@ public class HiveOutputFormatFactory implements OutputFormatFactory { private static final long serialVersionUID = 1L; - private final String outputFormat; + private final Class hiveOutputFormatClz; private final SerDeInfo serDeInfo; @@ -77,6 +77,8 @@ public class HiveOutputFormatFactory implements OutputFormatFactory { private final HiveShim hiveShim; + private final boolean isCompressed; + // number of non-partitioning columns private transient int numNonPartitionColumns; @@ -93,20 +95,24 @@ public class HiveOutputFormatFactory implements OutputFormatFactory { public HiveOutputFormatFactory( JobConf jobConf, - String outputFormat, + Class hiveOutputFormatClz, SerDeInfo serDeInfo, TableSchema schema, String[] partitionColumns, Properties tableProperties, - HiveShim hiveShim) { + HiveShim hiveShim, + boolean isCompressed) { + Preconditions.checkArgument(org.apache.hadoop.hive.ql.io.HiveOutputFormat.class.isAssignableFrom(hiveOutputFormatClz), + "The output format should be an instance of HiveOutputFormat"); this.confWrapper = new JobConfWrapper(jobConf); - this.outputFormat = outputFormat; + this.hiveOutputFormatClz = hiveOutputFormatClz; this.serDeInfo = serDeInfo; this.allColumns = schema.getFieldNames(); this.allTypes = schema.getFieldDataTypes(); this.partitionColumns = partitionColumns; this.tableProperties = tableProperties; this.hiveShim = hiveShim; + this.isCompressed = isCompressed; } private void init() throws Exception { @@ -145,7 +151,6 @@ public class HiveOutputFormatFactory implements OutputFormatFactory { JobConf conf = new JobConf(confWrapper.conf()); - final boolean isCompressed = conf.getBoolean(HiveConf.ConfVars.COMPRESSRESULT.varname, false); if (isCompressed) { String codecStr = conf.get(HiveConf.ConfVars.COMPRESSINTERMEDIATECODEC.varname); if (!StringUtils.isNullOrWhitespaceOnly(codecStr)) { @@ -164,7 +169,7 @@ public class HiveOutputFormatFactory implements OutputFormatFactory { RecordWriter recordWriter = hiveShim.getHiveRecordWriter( conf, - outputFormat, + hiveOutputFormatClz, recordSerDe.getSerializedClass(), isCompressed, tableProperties, diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSink.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSink.java index c4f4124c4c..283525a5be 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSink.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSink.java @@ -20,6 +20,7 @@ package org.apache.flink.connectors.hive; import org.apache.flink.api.common.io.OutputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.api.functions.sink.filesystem.OutputFileConfig; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.ObjectPath; @@ -47,6 +48,8 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.io.HiveOutputFormat; import org.apache.hadoop.mapred.JobConf; import org.apache.thrift.TException; @@ -104,26 +107,39 @@ public class HiveTableSink extends OutputFormatTableSink implements Partiti builder.setDynamicGrouped(dynamicGrouping); builder.setPartitionColumns(partitionColumns); builder.setFileSystemFactory(new HadoopFileSystemFactory(jobConf)); + + boolean isCompressed = jobConf.getBoolean(HiveConf.ConfVars.COMPRESSRESULT.varname, false); + Class hiveOutputFormatClz = hiveShim.getHiveOutputFormatClass(Class.forName(sd.getOutputFormat())); builder.setFormatFactory(new HiveOutputFormatFactory( jobConf, - sd.getOutputFormat(), + hiveOutputFormatClz, sd.getSerdeInfo(), tableSchema, partitionColumns, HiveReflectionUtils.getTableMetadata(hiveShim, table), - hiveShim)); + hiveShim, + isCompressed)); builder.setMetaStoreFactory( new HiveTableMetaStoreFactory(jobConf, hiveVersion, dbName, tableName)); builder.setOverwrite(overwrite); builder.setStaticPartitions(staticPartitionSpec); builder.setTempPath(new org.apache.flink.core.fs.Path( toStagingDir(sd.getLocation(), jobConf))); + String extension = Utilities.getFileExtension(jobConf, isCompressed, + (HiveOutputFormat) hiveOutputFormatClz.newInstance()); + extension = extension == null ? "" : extension; + OutputFileConfig outputFileConfig = new OutputFileConfig("", extension); + builder.setOutputFileConfig(outputFileConfig); return builder.build(); } catch (TException e) { throw new CatalogException("Failed to query Hive metaStore", e); } catch (IOException e) { throw new FlinkRuntimeException("Failed to create staging dir", e); + } catch (ClassNotFoundException e) { + throw new FlinkHiveException("Failed to get output format class", e); + } catch (IllegalAccessException | InstantiationException e) { + throw new FlinkHiveException("Failed to instantiate output format instance", e); } } diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShim.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShim.java index 346a8e16a9..258c52cbfa 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShim.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShim.java @@ -34,6 +34,7 @@ import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.api.UnknownDBException; import org.apache.hadoop.hive.ql.exec.FileSinkOperator; import org.apache.hadoop.hive.ql.exec.FunctionInfo; +import org.apache.hadoop.hive.ql.io.HiveOutputFormat; import org.apache.hadoop.hive.ql.udf.generic.SimpleGenericUDAFParameterInfo; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.io.Writable; @@ -140,9 +141,14 @@ public interface HiveShim extends Serializable { /** * Get Hive's FileSinkOperator.RecordWriter. */ - FileSinkOperator.RecordWriter getHiveRecordWriter(JobConf jobConf, String outputFormatClzName, + FileSinkOperator.RecordWriter getHiveRecordWriter(JobConf jobConf, Class outputFormatClz, Class outValClz, boolean isCompressed, Properties tableProps, Path outPath); + /** + * For a given OutputFormat class, get the corresponding {@link HiveOutputFormat} class. + */ + Class getHiveOutputFormatClass(Class outputFormatClz); + /** * Get Hive table schema from deserializer. */ diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV100.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV100.java index a6fcd78287..dae39d1b7b 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV100.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV100.java @@ -188,16 +188,12 @@ public class HiveShimV100 implements HiveShim { } @Override - public FileSinkOperator.RecordWriter getHiveRecordWriter(JobConf jobConf, String outputFormatClzName, + public FileSinkOperator.RecordWriter getHiveRecordWriter(JobConf jobConf, Class outputFormatClz, Class outValClz, boolean isCompressed, Properties tableProps, Path outPath) { try { - Class outputFormatClz = Class.forName(outputFormatClzName); Class utilClass = HiveFileFormatUtils.class; - Method utilMethod = utilClass.getDeclaredMethod("getOutputFormatSubstitute", Class.class, boolean.class); - outputFormatClz = (Class) utilMethod.invoke(null, outputFormatClz, false); - Preconditions.checkState(outputFormatClz != null, "No Hive substitute output format for " + outputFormatClzName); HiveOutputFormat outputFormat = (HiveOutputFormat) outputFormatClz.newInstance(); - utilMethod = utilClass.getDeclaredMethod("getRecordWriter", JobConf.class, HiveOutputFormat.class, + Method utilMethod = utilClass.getDeclaredMethod("getRecordWriter", JobConf.class, HiveOutputFormat.class, Class.class, boolean.class, Properties.class, Path.class, Reporter.class); return (FileSinkOperator.RecordWriter) utilMethod.invoke(null, jobConf, outputFormat, outValClz, isCompressed, tableProps, outPath, Reporter.NULL); @@ -206,6 +202,19 @@ public class HiveShimV100 implements HiveShim { } } + @Override + public Class getHiveOutputFormatClass(Class outputFormatClz) { + try { + Class utilClass = HiveFileFormatUtils.class; + Method utilMethod = utilClass.getDeclaredMethod("getOutputFormatSubstitute", Class.class, boolean.class); + Class res = (Class) utilMethod.invoke(null, outputFormatClz, false); + Preconditions.checkState(res != null, "No Hive substitute output format for " + outputFormatClz); + return res; + } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) { + throw new FlinkHiveException("Failed to get HiveOutputFormat for " + outputFormatClz, e); + } + } + @Override public List getFieldsFromDeserializer(Configuration conf, Table table, boolean skipConfError) { try { diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV110.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV110.java index 691c14a5d0..8d1885da18 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV110.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV110.java @@ -18,6 +18,7 @@ package org.apache.flink.table.catalog.hive.client; +import org.apache.flink.connectors.hive.FlinkHiveException; import org.apache.flink.table.catalog.exceptions.CatalogException; import org.apache.flink.util.Preconditions; @@ -33,6 +34,7 @@ import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.OutputFormat; import org.apache.hadoop.mapred.Reporter; +import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.util.List; import java.util.Properties; @@ -43,16 +45,12 @@ import java.util.Properties; public class HiveShimV110 extends HiveShimV101 { @Override - public FileSinkOperator.RecordWriter getHiveRecordWriter(JobConf jobConf, String outputFormatClzName, + public FileSinkOperator.RecordWriter getHiveRecordWriter(JobConf jobConf, Class outputFormatClz, Class outValClz, boolean isCompressed, Properties tableProps, Path outPath) { try { - Class outputFormatClz = Class.forName(outputFormatClzName); Class utilClass = HiveFileFormatUtils.class; - Method utilMethod = utilClass.getDeclaredMethod("getOutputFormatSubstitute", Class.class); - outputFormatClz = (Class) utilMethod.invoke(null, outputFormatClz); - Preconditions.checkState(outputFormatClz != null, "No Hive substitute output format for " + outputFormatClzName); OutputFormat outputFormat = (OutputFormat) outputFormatClz.newInstance(); - utilMethod = utilClass.getDeclaredMethod("getRecordWriter", JobConf.class, OutputFormat.class, + Method utilMethod = utilClass.getDeclaredMethod("getRecordWriter", JobConf.class, OutputFormat.class, Class.class, boolean.class, Properties.class, Path.class, Reporter.class); return (FileSinkOperator.RecordWriter) utilMethod.invoke(null, jobConf, outputFormat, outValClz, isCompressed, tableProps, outPath, Reporter.NULL); @@ -61,6 +59,19 @@ public class HiveShimV110 extends HiveShimV101 { } } + @Override + public Class getHiveOutputFormatClass(Class outputFormatClz) { + try { + Class utilClass = HiveFileFormatUtils.class; + Method utilMethod = utilClass.getDeclaredMethod("getOutputFormatSubstitute", Class.class); + Class res = (Class) utilMethod.invoke(null, outputFormatClz); + Preconditions.checkState(res != null, "No Hive substitute output format for " + outputFormatClz); + return res; + } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) { + throw new FlinkHiveException("Failed to get HiveOutputFormat for " + outputFormatClz, e); + } + } + @Override public List getFieldsFromDeserializer(Configuration conf, Table table, boolean skipConfError) { try { diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveOutputFormatFactoryTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveOutputFormatFactoryTest.java index 70152dcd5b..fe70b98b4a 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveOutputFormatFactoryTest.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveOutputFormatFactoryTest.java @@ -54,11 +54,12 @@ public class HiveOutputFormatFactoryTest { SerDeInfo serDeInfo = new SerDeInfo("name", LazySimpleSerDe.class.getName(), Collections.emptyMap()); HiveOutputFormatFactory factory = new HiveOutputFormatFactory( new JobConf(), - VerifyURIOutputFormat.class.getName(), + VerifyURIOutputFormat.class, serDeInfo, schema, new String[0], new Properties(), - HiveShimLoader.loadHiveShim(HiveShimLoader.getHiveVersion())); + HiveShimLoader.loadHiveShim(HiveShimLoader.getHiveVersion()), + false); org.apache.flink.core.fs.Path path = new org.apache.flink.core.fs.Path(TEST_URI_SCHEME, TEST_URI_AUTHORITY, "/foo/path"); factory.createOutputFormat(path); } diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java index 5a3726fc8b..47483577cf 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java @@ -543,6 +543,28 @@ public class TableEnvHiveConnectorTest { } } + @Test + public void testCompressTextTable() throws Exception { + hiveShell.execute("create database db1"); + try { + hiveShell.execute("create table db1.src (x string,y string)"); + hiveShell.execute("create table db1.dest like db1.src"); + HiveTestUtils.createTextTableInserter(hiveShell, "db1", "src") + .addRow(new Object[]{"a", "b"}) + .addRow(new Object[]{"c", "d"}) + .commit(); + hiveCatalog.getHiveConf().setBoolVar(HiveConf.ConfVars.COMPRESSRESULT, true); + TableEnvironment tableEnv = getTableEnvWithHiveCatalog(); + tableEnv.sqlUpdate("insert overwrite db1.dest select * from db1.src"); + tableEnv.execute("insert dest"); + List expected = Arrays.asList("a\tb", "c\td"); + verifyHiveQueryResult("select * from db1.dest", expected); + verifyFlinkQueryResult(tableEnv.sqlQuery("select * from db1.dest"), expected); + } finally { + hiveShell.execute("drop database db1 cascade"); + } + } + private TableEnvironment getTableEnvWithHiveCatalog() { TableEnvironment tableEnv = HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(); tableEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/OutputFileConfig.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/OutputFileConfig.java index 98b84d178a..e139b170e4 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/OutputFileConfig.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/OutputFileConfig.java @@ -47,14 +47,14 @@ public class OutputFileConfig implements Serializable { /** * The prefix for the part name. */ - String getPartPrefix() { + public String getPartPrefix() { return partPrefix; } /** * The suffix for the part name. */ - String getPartSuffix() { + public String getPartSuffix() { return partSuffix; } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/FileSystemOutputFormat.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/FileSystemOutputFormat.java index 18549677ea..56011ab701 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/FileSystemOutputFormat.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/FileSystemOutputFormat.java @@ -24,6 +24,7 @@ import org.apache.flink.api.common.io.OutputFormat; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.Path; +import org.apache.flink.streaming.api.functions.sink.filesystem.OutputFileConfig; import org.apache.flink.table.api.TableException; import java.io.IOException; @@ -53,6 +54,7 @@ public class FileSystemOutputFormat implements OutputFormat, FinalizeOnMas private final LinkedHashMap staticPartitions; private final PartitionComputer computer; private final OutputFormatFactory formatFactory; + private final OutputFileConfig outputFileConfig; private transient PartitionWriter writer; private transient Configuration parameters; @@ -66,7 +68,8 @@ public class FileSystemOutputFormat implements OutputFormat, FinalizeOnMas boolean dynamicGrouped, LinkedHashMap staticPartitions, OutputFormatFactory formatFactory, - PartitionComputer computer) { + PartitionComputer computer, + OutputFileConfig outputFileConfig) { this.fsFactory = fsFactory; this.msFactory = msFactory; this.overwrite = overwrite; @@ -76,6 +79,7 @@ public class FileSystemOutputFormat implements OutputFormat, FinalizeOnMas this.staticPartitions = staticPartitions; this.formatFactory = formatFactory; this.computer = computer; + this.outputFileConfig = outputFileConfig; } @Override @@ -102,7 +106,7 @@ public class FileSystemOutputFormat implements OutputFormat, FinalizeOnMas public void open(int taskNumber, int numTasks) throws IOException { try { PartitionTempFileManager fileManager = new PartitionTempFileManager( - fsFactory, tmpPath, taskNumber, CHECKPOINT_ID); + fsFactory, tmpPath, taskNumber, CHECKPOINT_ID, outputFileConfig); PartitionWriter.Context context = new PartitionWriter.Context<>( parameters, formatFactory); writer = PartitionWriterFactory.get( @@ -149,6 +153,8 @@ public class FileSystemOutputFormat implements OutputFormat, FinalizeOnMas private PartitionComputer computer; + private OutputFileConfig outputFileConfig = new OutputFileConfig("", ""); + public Builder setPartitionColumns(String[] partitionColumns) { this.partitionColumns = partitionColumns; return this; @@ -194,6 +200,11 @@ public class FileSystemOutputFormat implements OutputFormat, FinalizeOnMas return this; } + public Builder setOutputFileConfig(OutputFileConfig outputFileConfig) { + this.outputFileConfig = outputFileConfig; + return this; + } + public FileSystemOutputFormat build() { checkNotNull(partitionColumns, "partitionColumns should not be null"); checkNotNull(formatFactory, "formatFactory should not be null"); @@ -210,7 +221,8 @@ public class FileSystemOutputFormat implements OutputFormat, FinalizeOnMas dynamicGrouped, staticPartitions, formatFactory, - computer); + computer, + outputFileConfig); } } } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/PartitionTempFileManager.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/PartitionTempFileManager.java index 77534e9e8b..26e9c8127e 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/PartitionTempFileManager.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/PartitionTempFileManager.java @@ -22,6 +22,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.core.fs.FileStatus; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.Path; +import org.apache.flink.streaming.api.functions.sink.filesystem.OutputFileConfig; import java.io.IOException; import java.util.ArrayList; @@ -54,6 +55,7 @@ public class PartitionTempFileManager { private final int taskNumber; private final long checkpointId; private final Path taskTmpDir; + private final OutputFileConfig outputFileConfig; private transient int nameCounter = 0; @@ -62,9 +64,19 @@ public class PartitionTempFileManager { Path tmpPath, int taskNumber, long checkpointId) throws IOException { + this(factory, tmpPath, taskNumber, checkpointId, new OutputFileConfig("", "")); + } + + PartitionTempFileManager( + FileSystemFactory factory, + Path tmpPath, + int taskNumber, + long checkpointId, + OutputFileConfig outputFileConfig) throws IOException { checkArgument(checkpointId != -1, "checkpoint id start with 0."); this.taskNumber = taskNumber; this.checkpointId = checkpointId; + this.outputFileConfig = outputFileConfig; // generate and clean task temp dir. this.taskTmpDir = new Path( @@ -85,9 +97,9 @@ public class PartitionTempFileManager { } private String newFileName() { - return String.format( - checkpointName(checkpointId) + "-" + taskName(taskNumber) + "-file-%d", - nameCounter++); + return String.format("%s%s-%s-file-%d%s", + outputFileConfig.getPartPrefix(), checkpointName(checkpointId), + taskName(taskNumber), nameCounter++, outputFileConfig.getPartSuffix()); } private static boolean isTaskDir(String fileName) { -- Gitee From b752bfbe0073bb446de702f902b90564f41953cb Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Fri, 27 Mar 2020 08:32:51 +0100 Subject: [PATCH 332/885] [FLINK-16821][e2e] Use constant minikube version instead of latest --- flink-end-to-end-tests/test-scripts/common_kubernetes.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-end-to-end-tests/test-scripts/common_kubernetes.sh b/flink-end-to-end-tests/test-scripts/common_kubernetes.sh index 261c10cd2d..1e0c063bc4 100755 --- a/flink-end-to-end-tests/test-scripts/common_kubernetes.sh +++ b/flink-end-to-end-tests/test-scripts/common_kubernetes.sh @@ -39,7 +39,7 @@ function setup_kubernetes_for_linux { # Download minikube. if ! [ -x "$(command -v minikube)" ]; then echo "Installing minikube ..." - curl -Lo minikube https://storage.googleapis.com/minikube/releases/latest/minikube-linux-amd64 && \ + curl -Lo minikube https://storage.googleapis.com/minikube/releases/v1.8.2/minikube-linux-amd64 && \ chmod +x minikube && sudo mv minikube /usr/local/bin/ fi } -- Gitee From eb489cc2f7e01fa43cae1bf2106fc30f0cf829fb Mon Sep 17 00:00:00 2001 From: vthinkxie Date: Wed, 18 Mar 2020 16:15:39 +0800 Subject: [PATCH 333/885] [FLINK-11404][web] Add load more feature to exception page This closes #11436. --- .../exceptions/job-exceptions.component.html | 5 ++++ .../exceptions/job-exceptions.component.ts | 26 ++++++++++++++----- .../src/app/services/job.service.ts | 7 +++-- 3 files changed, 29 insertions(+), 9 deletions(-) diff --git a/flink-runtime-web/web-dashboard/src/app/pages/job/exceptions/job-exceptions.component.html b/flink-runtime-web/web-dashboard/src/app/pages/job/exceptions/job-exceptions.component.html index f3ee3bd560..ea58a7f3f0 100644 --- a/flink-runtime-web/web-dashboard/src/app/pages/job/exceptions/job-exceptions.component.html +++ b/flink-runtime-web/web-dashboard/src/app/pages/job/exceptions/job-exceptions.component.html @@ -53,6 +53,11 @@ +
    + + diff --git a/flink-runtime-web/web-dashboard/src/app/pages/job/exceptions/job-exceptions.component.ts b/flink-runtime-web/web-dashboard/src/app/pages/job/exceptions/job-exceptions.component.ts index fd955369de..cb72604406 100644 --- a/flink-runtime-web/web-dashboard/src/app/pages/job/exceptions/job-exceptions.component.ts +++ b/flink-runtime-web/web-dashboard/src/app/pages/job/exceptions/job-exceptions.component.ts @@ -19,7 +19,7 @@ import { formatDate } from '@angular/common'; import { Component, OnInit, ChangeDetectionStrategy, ChangeDetectorRef } from '@angular/core'; import { JobExceptionItemInterface } from 'interfaces'; -import { distinctUntilChanged, flatMap } from 'rxjs/operators'; +import { distinctUntilChanged, flatMap, tap } from 'rxjs/operators'; import { JobService } from 'services'; @Component({ @@ -31,18 +31,24 @@ import { JobService } from 'services'; export class JobExceptionsComponent implements OnInit { rootException = ''; listOfException: JobExceptionItemInterface[] = []; + truncated = false; + isLoading = false; + maxExceptions = 0; trackExceptionBy(_: number, node: JobExceptionItemInterface) { return node.timestamp; } - - constructor(private jobService: JobService, private cdr: ChangeDetectorRef) {} - - ngOnInit() { + loadMore() { + this.isLoading = true; + this.maxExceptions += 10; this.jobService.jobDetail$ .pipe( distinctUntilChanged((pre, next) => pre.jid === next.jid), - flatMap(job => this.jobService.loadExceptions(job.jid)) + flatMap(job => this.jobService.loadExceptions(job.jid, this.maxExceptions)), + tap(() => { + this.isLoading = false; + this.cdr.markForCheck(); + }) ) .subscribe(data => { // @ts-ignore @@ -51,8 +57,14 @@ export class JobExceptionsComponent implements OnInit { } else { this.rootException = 'No Root Exception'; } + this.truncated = data.truncated; this.listOfException = data['all-exceptions']; - this.cdr.markForCheck(); }); } + + constructor(private jobService: JobService, private cdr: ChangeDetectorRef) {} + + ngOnInit() { + this.loadMore(); + } } diff --git a/flink-runtime-web/web-dashboard/src/app/services/job.service.ts b/flink-runtime-web/web-dashboard/src/app/services/job.service.ts index 4abaa3f04d..e1d057e516 100644 --- a/flink-runtime-web/web-dashboard/src/app/services/job.service.ts +++ b/flink-runtime-web/web-dashboard/src/app/services/job.service.ts @@ -155,9 +155,12 @@ export class JobService { /** * Get job exception * @param jobId + * @param maxExceptions */ - loadExceptions(jobId: string) { - return this.httpClient.get(`${BASE_URL}/jobs/${jobId}/exceptions`); + loadExceptions(jobId: string, maxExceptions: number) { + return this.httpClient.get( + `${BASE_URL}/jobs/${jobId}/exceptions?maxExceptions=${maxExceptions}` + ); } /** -- Gitee From 42dd176e170d2a4343d6e40cc4f15d0fb379d43c Mon Sep 17 00:00:00 2001 From: Alexander Fedulov <1492164+afedulov@users.noreply.github.com> Date: Thu, 19 Mar 2020 23:41:09 +0100 Subject: [PATCH 334/885] [FLINK-16825][prometheus][tests] Use Path returned by DownloadCache If the test is ran multiple times, newly downloaded files will get numbered prefixes (e.g. file.tar.gz.1). This causes an error because the wrong bath is used. This commit fixes this issue. --- .../apache/flink/tests/util/cache/AbstractDownloadCache.java | 2 +- .../prometheus/tests/PrometheusReporterEndToEndITCase.java | 5 ++--- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/AbstractDownloadCache.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/AbstractDownloadCache.java index 48a1c5fdad..20b72ff22e 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/AbstractDownloadCache.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/AbstractDownloadCache.java @@ -113,8 +113,8 @@ abstract class AbstractDownloadCache implements DownloadCache { final Path cacheFile; if (cachedFile.isPresent()) { - log.info("Using cached version of {}.", url); cacheFile = cachedFile.get(); + log.info("Using cached version of {} from {}", url, cacheFile.toAbsolutePath()); } else { final Path scopedDownloadDir = downloadsDir.resolve(String.valueOf(url.hashCode())); Files.createDirectories(scopedDownloadDir); diff --git a/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java index 663744061e..1dba568251 100644 --- a/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java +++ b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java @@ -139,14 +139,13 @@ public class PrometheusReporterEndToEndITCase extends TestLogger { @Test public void testReporter() throws Exception { final Path tmpPrometheusDir = tmp.newFolder().toPath().resolve("prometheus"); - final Path prometheusArchive = tmpPrometheusDir.resolve(PROMETHEUS_FILE_NAME + ".tar.gz"); final Path prometheusBinDir = tmpPrometheusDir.resolve(PROMETHEUS_FILE_NAME); final Path prometheusConfig = prometheusBinDir.resolve("prometheus.yml"); final Path prometheusBinary = prometheusBinDir.resolve("prometheus"); Files.createDirectory(tmpPrometheusDir); - downloadCache.getOrDownload( - "https://github.com/prometheus/prometheus/releases/download/v" + PROMETHEUS_VERSION + '/' + prometheusArchive.getFileName(), + final Path prometheusArchive = downloadCache.getOrDownload( + "https://github.com/prometheus/prometheus/releases/download/v" + PROMETHEUS_VERSION + '/' + PROMETHEUS_FILE_NAME + ".tar.gz", tmpPrometheusDir ); -- Gitee From fbfb639e500e7cfa06485f1328671cfe645dfd76 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Fri, 27 Mar 2020 12:18:17 +0100 Subject: [PATCH 335/885] [FLINK-16798][scripts] Properly forward BashJavaUtils logging output --- flink-dist/src/main/flink-bin/bin/taskmanager.sh | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/flink-dist/src/main/flink-bin/bin/taskmanager.sh b/flink-dist/src/main/flink-bin/bin/taskmanager.sh index befeb99760..4ffceb0559 100755 --- a/flink-dist/src/main/flink-bin/bin/taskmanager.sh +++ b/flink-dist/src/main/flink-bin/bin/taskmanager.sh @@ -48,7 +48,9 @@ if [[ $STARTSTOP == "start" ]] || [[ $STARTSTOP == "start-foreground" ]]; then # Startup parameters - params_output=$(runBashJavaUtilsCmd GET_TM_RESOURCE_PARAMS ${FLINK_CONF_DIR} | tail -n 2) + java_utils_output=$(runBashJavaUtilsCmd GET_TM_RESOURCE_PARAMS ${FLINK_CONF_DIR}) + logging_output=$(echo "${java_utils_output}" | head -n -2) + params_output=$(echo "${java_utils_output}" | tail -n 2) jvm_params=$(extractExecutionParams "$(echo "$params_output" | head -n 1)") if [[ $? -ne 0 ]]; then @@ -72,6 +74,7 @@ $FLINK_INHERITED_LOGS TM_RESOURCE_PARAMS extraction logs: jvm_params: $jvm_params dynamic_configs: $dynamic_configs +logs: $logging_output " fi -- Gitee From 200d3466bdb937eecf7038c22558e535d6e22aa3 Mon Sep 17 00:00:00 2001 From: Alexander Fedulov <1492164+afedulov@users.noreply.github.com> Date: Thu, 26 Mar 2020 13:34:18 +0100 Subject: [PATCH 336/885] [FLINK-16826][e2e] Support copying of jars JARs copying into multiple locations is required for testing plugins-loading mechanism. Currently only "move" operations is supported, this commit extends it to also support "copy" operations. --- .../tests/util/flink/FlinkDistribution.java | 22 ++++++++++++++----- .../tests/util/flink/FlinkResourceSetup.java | 21 +++++++++++------- .../flink/{JarMove.java => JarOperation.java} | 17 +++++++++++--- .../flink/LocalStandaloneFlinkResource.java | 4 ++-- 4 files changed, 45 insertions(+), 19 deletions(-) rename flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/{JarMove.java => JarOperation.java} (76%) diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkDistribution.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkDistribution.java index 970dbef62f..bbb95d2a6b 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkDistribution.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkDistribution.java @@ -206,22 +206,32 @@ final class FlinkDistribution { AutoClosableProcess.runBlocking(commands.toArray(new String[0])); } - public void moveJar(JarMove move) throws IOException { - final Path source = mapJarLocationToPath(move.getSource()); - final Path target = mapJarLocationToPath(move.getTarget()); + public void performJarOperation(JarOperation operation) throws IOException { + final Path source = mapJarLocationToPath(operation.getSource()); + final Path target = mapJarLocationToPath(operation.getTarget()); final Optional jarOptional; try (Stream files = Files.walk(source)) { jarOptional = files - .filter(path -> path.getFileName().toString().startsWith(move.getJarNamePrefix())) + .filter(path -> path.getFileName().toString().startsWith(operation.getJarNamePrefix())) .findFirst(); } if (jarOptional.isPresent()) { final Path sourceJar = jarOptional.get(); final Path targetJar = target.resolve(sourceJar.getFileName()); - Files.move(sourceJar, targetJar); + switch (operation.getOperationType()){ + case COPY: + Files.copy(sourceJar, targetJar); + break; + case MOVE: + Files.move(sourceJar, targetJar); + break; + default: + throw new IllegalStateException(); + } + } else { - throw new FileNotFoundException("No jar could be found matching the pattern " + move.getJarNamePrefix() + "."); + throw new FileNotFoundException("No jar could be found matching the pattern " + operation.getJarNamePrefix() + "."); } } diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkResourceSetup.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkResourceSetup.java index 7eebe41b01..2e4c3ab642 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkResourceSetup.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkResourceSetup.java @@ -34,19 +34,19 @@ public class FlinkResourceSetup { @Nullable private final Configuration config; - private final Collection jarMoveOperations; + private final Collection jarOperations; - private FlinkResourceSetup(@Nullable Configuration config, Collection jarMoveOperations) { + private FlinkResourceSetup(@Nullable Configuration config, Collection jarOperations) { this.config = config; - this.jarMoveOperations = Preconditions.checkNotNull(jarMoveOperations); + this.jarOperations = Preconditions.checkNotNull(jarOperations); } public Optional getConfig() { return Optional.ofNullable(config); } - public Collection getJarMoveOperations() { - return jarMoveOperations; + public Collection getJarOperations() { + return jarOperations; } public static FlinkResourceSetupBuilder builder() { @@ -59,7 +59,7 @@ public class FlinkResourceSetup { public static class FlinkResourceSetupBuilder { private Configuration config; - private final Collection jarMoveOperations = new ArrayList<>(); + private final Collection jarOperations = new ArrayList<>(); private FlinkResourceSetupBuilder() { } @@ -70,12 +70,17 @@ public class FlinkResourceSetup { } public FlinkResourceSetupBuilder moveJar(String jarNamePrefix, JarLocation source, JarLocation target) { - this.jarMoveOperations.add(new JarMove(jarNamePrefix, source, target)); + this.jarOperations.add(new JarOperation(jarNamePrefix, source, target, JarOperation.OperationType.MOVE)); + return this; + } + + public FlinkResourceSetupBuilder copyJar(String jarNamePrefix, JarLocation source, JarLocation target) { + this.jarOperations.add(new JarOperation(jarNamePrefix, source, target, JarOperation.OperationType.COPY)); return this; } public FlinkResourceSetup build() { - return new FlinkResourceSetup(config, Collections.unmodifiableCollection(jarMoveOperations)); + return new FlinkResourceSetup(config, Collections.unmodifiableCollection(jarOperations)); } } diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/JarMove.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/JarOperation.java similarity index 76% rename from flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/JarMove.java rename to flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/JarOperation.java index d9c407692f..0695f9f28d 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/JarMove.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/JarOperation.java @@ -18,18 +18,20 @@ package org.apache.flink.tests.util.flink; /** - * Represents a move operation for a jar. + * Represents a move/copy operation for a jar. */ -class JarMove { +class JarOperation { private final String jarNamePrefix; private final JarLocation source; private final JarLocation target; + private final OperationType operationType; - JarMove(String jarNamePrefix, JarLocation source, JarLocation target) { + JarOperation(String jarNamePrefix, JarLocation source, JarLocation target, OperationType operationType) { this.jarNamePrefix = jarNamePrefix; this.source = source; this.target = target; + this.operationType = operationType; } public String getJarNamePrefix() { @@ -43,4 +45,13 @@ class JarMove { public JarLocation getTarget() { return target; } + + public OperationType getOperationType() { + return operationType; + } + + public enum OperationType { + COPY, + MOVE; + } } diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResource.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResource.java index 74ccf6e489..6ea327e68e 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResource.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResource.java @@ -79,8 +79,8 @@ public class LocalStandaloneFlinkResource implements FlinkResource { TestUtils.copyDirectory(distributionDirectory, tmp); distribution = new FlinkDistribution(tmp); - for (JarMove jarMove : setup.getJarMoveOperations()) { - distribution.moveJar(jarMove); + for (JarOperation jarOperation : setup.getJarOperations()) { + distribution.performJarOperation(jarOperation); } if (setup.getConfig().isPresent()) { distribution.appendConfiguration(setup.getConfig().get()); -- Gitee From 9dc77312a08e15d378fb0d2dc5ee16a7c0a9024f Mon Sep 17 00:00:00 2001 From: Alexander Fedulov <1492164+afedulov@users.noreply.github.com> Date: Fri, 21 Feb 2020 15:50:36 +0100 Subject: [PATCH 337/885] [FLINK-16828][prometheus][metrics] Support factories --- .../PrometheusPushGatewayReporter.java | 1 + .../PrometheusPushGatewayReporterFactory.java | 33 +++++++++++++++++++ .../prometheus/PrometheusReporter.java | 2 ++ .../prometheus/PrometheusReporterFactory.java | 33 +++++++++++++++++++ ...ink.metrics.reporter.MetricReporterFactory | 17 ++++++++++ 5 files changed, 86 insertions(+) create mode 100644 flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusPushGatewayReporterFactory.java create mode 100644 flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusReporterFactory.java create mode 100644 flink-metrics/flink-metrics-prometheus/src/main/resources/META-INF/services/org.apache.flink.metrics.reporter.MetricReporterFactory diff --git a/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusPushGatewayReporter.java b/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusPushGatewayReporter.java index 06f72032e3..38bd2083ec 100644 --- a/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusPushGatewayReporter.java +++ b/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusPushGatewayReporter.java @@ -45,6 +45,7 @@ import static org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporterO * {@link MetricReporter} that exports {@link Metric Metrics} via Prometheus {@link PushGateway}. */ @PublicEvolving +@InstantiateViaFactory(factoryClassName = "org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporterFactory") public class PrometheusPushGatewayReporter extends AbstractPrometheusReporter implements Scheduled { private PushGateway pushGateway; diff --git a/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusPushGatewayReporterFactory.java b/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusPushGatewayReporterFactory.java new file mode 100644 index 0000000000..7b7372e960 --- /dev/null +++ b/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusPushGatewayReporterFactory.java @@ -0,0 +1,33 @@ +/* + * 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.metrics.prometheus; + +import org.apache.flink.metrics.reporter.MetricReporterFactory; + +import java.util.Properties; + +/** + * {@link MetricReporterFactory} for {@link PrometheusPushGatewayReporter}. + */ +public class PrometheusPushGatewayReporterFactory implements MetricReporterFactory { + + @Override + public PrometheusPushGatewayReporter createMetricReporter(Properties properties) { + return new PrometheusPushGatewayReporter(); + } +} diff --git a/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusReporter.java b/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusReporter.java index 4697685b8c..bd9575bdc2 100644 --- a/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusReporter.java +++ b/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusReporter.java @@ -22,6 +22,7 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.metrics.Metric; import org.apache.flink.metrics.MetricConfig; +import org.apache.flink.metrics.reporter.InstantiateViaFactory; import org.apache.flink.metrics.reporter.MetricReporter; import org.apache.flink.util.NetUtils; import org.apache.flink.util.Preconditions; @@ -35,6 +36,7 @@ import java.util.Iterator; * {@link MetricReporter} that exports {@link Metric Metrics} via Prometheus. */ @PublicEvolving +@InstantiateViaFactory(factoryClassName = "org.apache.flink.metrics.prometheus.PrometheusReporterFactory") public class PrometheusReporter extends AbstractPrometheusReporter { static final String ARG_PORT = "port"; diff --git a/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusReporterFactory.java b/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusReporterFactory.java new file mode 100644 index 0000000000..a1ad3657b1 --- /dev/null +++ b/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusReporterFactory.java @@ -0,0 +1,33 @@ +/* + * 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.metrics.prometheus; + +import org.apache.flink.metrics.reporter.MetricReporterFactory; + +import java.util.Properties; + +/** + * {@link MetricReporterFactory} for {@link PrometheusReporter}. + */ +public class PrometheusReporterFactory implements MetricReporterFactory { + + @Override + public PrometheusReporter createMetricReporter(Properties properties) { + return new PrometheusReporter(); + } +} diff --git a/flink-metrics/flink-metrics-prometheus/src/main/resources/META-INF/services/org.apache.flink.metrics.reporter.MetricReporterFactory b/flink-metrics/flink-metrics-prometheus/src/main/resources/META-INF/services/org.apache.flink.metrics.reporter.MetricReporterFactory new file mode 100644 index 0000000000..9b8c734682 --- /dev/null +++ b/flink-metrics/flink-metrics-prometheus/src/main/resources/META-INF/services/org.apache.flink.metrics.reporter.MetricReporterFactory @@ -0,0 +1,17 @@ +# 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. + +org.apache.flink.metrics.prometheus.PrometheusReporterFactory +org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporterFactory -- Gitee From d8b467979d884567dcbc6f1277d0e58aea8046db Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Fri, 29 Nov 2019 14:42:25 +0100 Subject: [PATCH 338/885] [hotfix][coordination] Exit early for empty collection --- .../partition/TaskExecutorPartitionTrackerImpl.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImpl.java index 2b6fc6920d..e1833f6d24 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImpl.java @@ -52,6 +52,10 @@ public class TaskExecutorPartitionTrackerImpl extends AbstractPartitionTracker partitionsToRelease) { + if (partitionsToRelease.isEmpty()) { + return; + } + stopTrackingPartitions(partitionsToRelease); shuffleEnvironment.releasePartitionsLocally(partitionsToRelease); } @@ -66,6 +70,10 @@ public class TaskExecutorPartitionTrackerImpl extends AbstractPartitionTracker partitionsToPromote) { + if (partitionsToPromote.isEmpty()) { + return; + } + final Collection> partitionTrackerEntries = stopTrackingPartitions(partitionsToPromote); final Map> newClusterPartitions = partitionTrackerEntries.stream() -- Gitee From 2b1193228c484c62e3dc7ce69a83e6972aff6c6f Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 3 Mar 2020 14:07:52 +0100 Subject: [PATCH 339/885] [hotfix][tests] Rename variable --- .../TaskExecutorPartitionLifecycleTest.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java index 82660ef9bc..0313ed4704 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java @@ -251,14 +251,14 @@ public class TaskExecutorPartitionLifecycleTest extends TestLogger { public void testPartitionPromotion() throws Exception { testPartitionRelease( partitionTracker -> { - final CompletableFuture> releasePartitionsFuture = new CompletableFuture<>(); - partitionTracker.setPromotePartitionsConsumer(releasePartitionsFuture::complete); - return releasePartitionsFuture; + final CompletableFuture> promotePartitionsFuture = new CompletableFuture<>(); + partitionTracker.setPromotePartitionsConsumer(promotePartitionsFuture::complete); + return promotePartitionsFuture; }, - (jobId, partitionId, taskExecutor, taskExecutorGateway, releasePartitionsFuture) -> { + (jobId, partitionId, taskExecutor, taskExecutorGateway, promotePartitionsFuture) -> { taskExecutorGateway.releaseOrPromotePartitions(jobId, Collections.emptySet(), Collections.singleton(partitionId)); - assertThat(releasePartitionsFuture.get(), hasItems(partitionId)); + assertThat(promotePartitionsFuture.get(), hasItems(partitionId)); } ); } -- Gitee From 22a3e8a96b971eb76a708690aa2dd3086ae64e1a Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 14 Nov 2019 16:23:22 +0100 Subject: [PATCH 340/885] [FLINK-14792][coordination] Rework partition storage --- .../TaskExecutorPartitionTrackerImpl.java | 65 ++++++++++++------- 1 file changed, 40 insertions(+), 25 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImpl.java index e1833f6d24..e7f364b6e7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImpl.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.io.network.partition; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.shuffle.ShuffleEnvironment; import org.apache.flink.runtime.taskexecutor.partition.ClusterPartitionReport; import org.apache.flink.util.CollectionUtil; @@ -25,6 +26,7 @@ import org.apache.flink.util.Preconditions; import java.util.Collection; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -35,7 +37,7 @@ import java.util.stream.Collectors; */ public class TaskExecutorPartitionTrackerImpl extends AbstractPartitionTracker implements TaskExecutorPartitionTracker { - private final Map> clusterPartitions = new HashMap<>(); + private final Map clusterPartitions = new HashMap<>(); private final ShuffleEnvironment shuffleEnvironment; public TaskExecutorPartitionTrackerImpl(ShuffleEnvironment shuffleEnvironment) { @@ -76,39 +78,52 @@ public class TaskExecutorPartitionTrackerImpl extends AbstractPartitionTracker> partitionTrackerEntries = stopTrackingPartitions(partitionsToPromote); - final Map> newClusterPartitions = partitionTrackerEntries.stream() - .collect(Collectors.groupingBy( - PartitionTrackerEntry::getMetaInfo, - Collectors.mapping(PartitionTrackerEntry::getResultPartitionId, Collectors.toSet()))); - - newClusterPartitions.forEach( - (dataSetMetaInfo, newPartitionEntries) -> clusterPartitions.compute(dataSetMetaInfo, (ignored, existingPartitions) -> { - if (existingPartitions == null) { - return newPartitionEntries; - } else { - existingPartitions.addAll(newPartitionEntries); - return existingPartitions; - } - })); + for (PartitionTrackerEntry partitionTrackerEntry : partitionTrackerEntries) { + final TaskExecutorPartitionInfo dataSetMetaInfo = partitionTrackerEntry.getMetaInfo(); + final DataSetEntry dataSetEntry = clusterPartitions.computeIfAbsent( + dataSetMetaInfo.getIntermediateDataSetId(), + ignored -> new DataSetEntry(dataSetMetaInfo.getNumberOfPartitions())); + dataSetEntry.addPartition(partitionTrackerEntry.getResultPartitionId()); + } } @Override public void stopTrackingAndReleaseAllClusterPartitions() { - clusterPartitions.values().forEach(shuffleEnvironment::releasePartitionsLocally); + clusterPartitions.values().stream().map(DataSetEntry::getPartitionIds).forEach(shuffleEnvironment::releasePartitionsLocally); clusterPartitions.clear(); } @Override public ClusterPartitionReport createClusterPartitionReport() { - List collect = clusterPartitions.entrySet().stream().map(entry -> { - TaskExecutorPartitionInfo dataSetMetaInfo = entry.getKey(); - Set partitionsIds = entry.getValue(); - return new ClusterPartitionReport.ClusterPartitionReportEntry( - dataSetMetaInfo.getIntermediateDataSetId(), - partitionsIds, - dataSetMetaInfo.getNumberOfPartitions()); - }).collect(Collectors.toList()); + List reportEntries = clusterPartitions.entrySet().stream().map(entry -> + new ClusterPartitionReport.ClusterPartitionReportEntry( + entry.getKey(), + entry.getValue().getPartitionIds(), + entry.getValue().getTotalNumberOfPartitions())) + .collect(Collectors.toList()); + + return new ClusterPartitionReport(reportEntries); + } - return new ClusterPartitionReport(collect); + private static class DataSetEntry { + + private final Set partitionIds = new HashSet<>(); + private final int totalNumberOfPartitions; + + private DataSetEntry(int totalNumberOfPartitions) { + this.totalNumberOfPartitions = totalNumberOfPartitions; + } + + void addPartition(ResultPartitionID resultPartitionId) { + partitionIds.add(resultPartitionId); + } + + public Set getPartitionIds() { + return partitionIds; + } + + public int getTotalNumberOfPartitions() { + return totalNumberOfPartitions; + } } } -- Gitee From 1ce9fcbc7df783b5a064eac745bb4f42bf53deed Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Fri, 29 Nov 2019 14:43:17 +0100 Subject: [PATCH 341/885] [FLINK-14792][coordination] Implement TE cluster partition release --- .../TaskExecutorPartitionTracker.java | 8 ++++ .../TaskExecutorPartitionTrackerImpl.java | 9 ++++ .../runtime/taskexecutor/TaskExecutor.java | 6 +++ .../taskexecutor/TaskExecutorGateway.java | 9 ++++ .../TaskExecutorPartitionTrackerImplTest.java | 21 ++++++++++ .../TestingTaskExecutorPartitionTracker.java | 11 +++++ .../TaskExecutorPartitionLifecycleTest.java | 42 ++++++++++++++----- .../TestingTaskExecutorGateway.java | 5 +++ 8 files changed, 101 insertions(+), 10 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTracker.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTracker.java index a586d81f36..18c60cfd6a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTracker.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTracker.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.io.network.partition; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.taskexecutor.partition.ClusterPartitionReport; import java.util.Collection; @@ -50,6 +51,13 @@ public interface TaskExecutorPartitionTracker extends PartitionTracker partitionsToPromote); + /** + * Releases partitions associated with the given datasets and stops tracking of partitions that were released. + * + * @param dataSetsToRelease data sets to release + */ + void stopTrackingAndReleaseClusterPartitions(Collection dataSetsToRelease); + /** * Releases and stops tracking all partitions. */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImpl.java index e7f364b6e7..d8780cf172 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImpl.java @@ -87,6 +87,15 @@ public class TaskExecutorPartitionTrackerImpl extends AbstractPartitionTracker dataSetsToRelease) { + for (IntermediateDataSetID dataSetID : dataSetsToRelease) { + final DataSetEntry dataSetEntry = clusterPartitions.remove(dataSetID); + final Set partitionIds = dataSetEntry.getPartitionIds(); + shuffleEnvironment.releasePartitionsLocally(partitionIds); + } + } + @Override public void stopTrackingAndReleaseAllClusterPartitions() { clusterPartitions.values().stream().map(DataSetEntry::getPartitionIds).forEach(shuffleEnvironment::releasePartitionsLocally); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java index ec9faec420..90b51a29e3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java @@ -57,6 +57,7 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNo import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.TaskExecutorPartitionInfo; import org.apache.flink.runtime.io.network.partition.TaskExecutorPartitionTracker; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; import org.apache.flink.runtime.jobgraph.tasks.TaskOperatorEventGateway; @@ -773,6 +774,11 @@ public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway { // TODO: Maybe it's better to return an Acknowledge here to notify the JM about the success/failure with an Exception } + @Override + public void releaseClusterPartitions(Collection dataSetsToRelease, Time timeout) { + partitionTracker.stopTrackingAndReleaseClusterPartitions(dataSetsToRelease); + } + // ---------------------------------------------------------------------- // Heartbeat RPC // ---------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java index a6ecc60f25..69257be8b8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java @@ -31,6 +31,7 @@ import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.PartitionInfo; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobmaster.AllocatedSlotReport; import org.apache.flink.runtime.jobmaster.JobMasterId; @@ -114,6 +115,14 @@ public interface TaskExecutorGateway extends RpcGateway, TaskExecutorOperatorEve */ void releaseOrPromotePartitions(JobID jobId, Set partitionToRelease, Set partitionsToPromote); + /** + * Releases all cluster partitions belong to any of the given data sets. + * + * @param dataSetsToRelease data sets for which all cluster partitions should be released + * @param timeout for the partitions release operation + */ + void releaseClusterPartitions(Collection dataSetsToRelease, @RpcTimeout Time timeout); + /** * Trigger the checkpoint for the given task. The checkpoint is identified by the checkpoint ID * and the checkpoint timestamp. diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImplTest.java index ec8af9c31d..7e2b27200b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImplTest.java @@ -151,6 +151,27 @@ public class TaskExecutorPartitionTrackerImplTest extends TestLogger { assertThat(shuffleReleaseFuture.get(), hasItem(resultPartitionId1)); } + @Test + public void stopTrackingAndReleaseClusterPartitions() throws Exception { + final TestingShuffleEnvironment testingShuffleEnvironment = new TestingShuffleEnvironment(); + final CompletableFuture> shuffleReleaseFuture = new CompletableFuture<>(); + testingShuffleEnvironment.releasePartitionsLocallyFuture = shuffleReleaseFuture; + + final ResultPartitionID resultPartitionId1 = new ResultPartitionID(); + final ResultPartitionID resultPartitionId2 = new ResultPartitionID(); + + final IntermediateDataSetID dataSetId1 = new IntermediateDataSetID(); + final IntermediateDataSetID dataSetId2 = new IntermediateDataSetID(); + + final TaskExecutorPartitionTracker partitionTracker = new TaskExecutorPartitionTrackerImpl(testingShuffleEnvironment); + partitionTracker.startTrackingPartition(new JobID(), new TaskExecutorPartitionInfo(resultPartitionId1, dataSetId1, 1)); + partitionTracker.startTrackingPartition(new JobID(), new TaskExecutorPartitionInfo(resultPartitionId2, dataSetId2, 1)); + partitionTracker.promoteJobPartitions(Collections.singleton(resultPartitionId1)); + + partitionTracker.stopTrackingAndReleaseClusterPartitions(Collections.singleton(dataSetId1)); + assertThat(shuffleReleaseFuture.get(), hasItem(resultPartitionId1)); + } + private static class TestingShuffleEnvironment implements ShuffleEnvironment { private final ShuffleEnvironment backingShuffleEnvironment = diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TestingTaskExecutorPartitionTracker.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TestingTaskExecutorPartitionTracker.java index d57df2cea2..42d1e56638 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TestingTaskExecutorPartitionTracker.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TestingTaskExecutorPartitionTracker.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.io.network.partition; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.taskexecutor.partition.ClusterPartitionReport; import java.util.Collection; @@ -40,6 +41,7 @@ public class TestingTaskExecutorPartitionTracker implements TaskExecutorPartitio private Function, Collection>> stopTrackingPartitionsFunction = ignored -> Collections.emptySet(); private Runnable stopTrackingAllClusterPartitionsRunnable = () -> {}; private Consumer> promotePartitionsConsumer = ignored -> {}; + private Consumer> releaseClusterPartitionsConsumer = ignored -> {}; public void setStartTrackingPartitionsConsumer(BiConsumer startTrackingPartitionsConsumer) { this.startTrackingPartitionsConsumer = startTrackingPartitionsConsumer; @@ -73,6 +75,10 @@ public class TestingTaskExecutorPartitionTracker implements TaskExecutorPartitio this.stopTrackingPartitionsFunction = stopTrackingPartitionsFunction; } + public void setReleaseClusterPartitionsConsumer(Consumer> releaseClusterPartitionsConsumer) { + this.releaseClusterPartitionsConsumer = releaseClusterPartitionsConsumer; + } + @Override public void startTrackingPartition(JobID producingJobId, TaskExecutorPartitionInfo partitionInfo) { startTrackingPartitionsConsumer.accept(producingJobId, partitionInfo); @@ -93,6 +99,11 @@ public class TestingTaskExecutorPartitionTracker implements TaskExecutorPartitio promotePartitionsConsumer.accept(partitionsToPromote); } + @Override + public void stopTrackingAndReleaseClusterPartitions(Collection dataSetsToRelease) { + releaseClusterPartitionsConsumer.accept(dataSetsToRelease); + } + @Override public void stopTrackingAndReleaseAllClusterPartitions() { stopTrackingAllClusterPartitionsRunnable.run(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java index 0313ed4704..bc23968800 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java @@ -43,6 +43,7 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.io.network.partition.TaskExecutorPartitionTracker; import org.apache.flink.runtime.io.network.partition.TestingTaskExecutorPartitionTracker; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobmaster.JMTMRegistrationSuccess; import org.apache.flink.runtime.jobmaster.JobMasterGateway; @@ -222,8 +223,7 @@ public class TaskExecutorPartitionLifecycleTest extends TestLogger { partitionTracker.setStopTrackingAndReleaseAllPartitionsConsumer(releasePartitionsForJobFuture::complete); return releasePartitionsForJobFuture; }, - (jobId, partitionId, taskExecutor, taskExecutorGateway, releasePartitionsForJobFuture) -> { - + (jobId, resultPartitionDeploymentDescriptor, taskExecutor, taskExecutorGateway, releasePartitionsForJobFuture) -> { taskExecutorGateway.disconnectJobManager(jobId, new Exception("test")); assertThat(releasePartitionsForJobFuture.get(), equalTo(jobId)); @@ -239,10 +239,12 @@ public class TaskExecutorPartitionLifecycleTest extends TestLogger { partitionTracker.setStopTrackingAndReleasePartitionsConsumer(releasePartitionsFuture::complete); return releasePartitionsFuture; }, - (jobId, partitionId, taskExecutor, taskExecutorGateway, releasePartitionsFuture) -> { - taskExecutorGateway.releaseOrPromotePartitions(jobId, Collections.singleton(partitionId), Collections.emptySet()); + (jobId, resultPartitionDeploymentDescriptor, taskExecutor, taskExecutorGateway, releasePartitionsFuture) -> { + final ResultPartitionID resultPartitionId = resultPartitionDeploymentDescriptor.getShuffleDescriptor().getResultPartitionID(); + + taskExecutorGateway.releaseOrPromotePartitions(jobId, Collections.singleton(resultPartitionId), Collections.emptySet()); - assertThat(releasePartitionsFuture.get(), hasItems(partitionId)); + assertThat(releasePartitionsFuture.get(), hasItems(resultPartitionId)); } ); } @@ -255,10 +257,30 @@ public class TaskExecutorPartitionLifecycleTest extends TestLogger { partitionTracker.setPromotePartitionsConsumer(promotePartitionsFuture::complete); return promotePartitionsFuture; }, - (jobId, partitionId, taskExecutor, taskExecutorGateway, promotePartitionsFuture) -> { - taskExecutorGateway.releaseOrPromotePartitions(jobId, Collections.emptySet(), Collections.singleton(partitionId)); + (jobId, resultPartitionDeploymentDescriptor, taskExecutor, taskExecutorGateway, promotePartitionsFuture) -> { + final ResultPartitionID resultPartitionId = resultPartitionDeploymentDescriptor.getShuffleDescriptor().getResultPartitionID(); + + taskExecutorGateway.releaseOrPromotePartitions(jobId, Collections.emptySet(), Collections.singleton(resultPartitionId)); + + assertThat(promotePartitionsFuture.get(), hasItems(resultPartitionId)); + } + ); + } + + @Test + public void testClusterPartitionRelease() throws Exception { + testPartitionRelease( + partitionTracker -> { + final CompletableFuture> releasePartitionsFuture = new CompletableFuture<>(); + partitionTracker.setReleaseClusterPartitionsConsumer(releasePartitionsFuture::complete); + return releasePartitionsFuture; + }, + (jobId, resultPartitionDeploymentDescriptor, taskExecutor, taskExecutorGateway, releasePartitionsFuture) -> { + final IntermediateDataSetID dataSetId = resultPartitionDeploymentDescriptor.getResultId(); + + taskExecutorGateway.releaseClusterPartitions(Collections.singleton(dataSetId), timeout); - assertThat(promotePartitionsFuture.get(), hasItems(partitionId)); + assertThat(releasePartitionsFuture.get(), hasItems(dataSetId)); } ); } @@ -402,7 +424,7 @@ public class TaskExecutorPartitionLifecycleTest extends TestLogger { testAction.accept( jobId, - taskResultPartitionDescriptor.getShuffleDescriptor().getResultPartitionID(), + taskResultPartitionDescriptor, taskExecutor, taskExecutorGateway, partitionTrackerSetupResult); @@ -464,6 +486,6 @@ public class TaskExecutorPartitionLifecycleTest extends TestLogger { @FunctionalInterface private interface TestAction { - void accept(JobID jobId, ResultPartitionID resultPartitionId, TaskExecutor taskExecutor, TaskExecutorGateway taskExecutorGateway, C partitionTrackerSetupResult) throws Exception; + void accept(JobID jobId, ResultPartitionDeploymentDescriptor resultPartitionDeploymentDescriptor, TaskExecutor taskExecutor, TaskExecutorGateway taskExecutorGateway, C partitionTrackerSetupResult) throws Exception; } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGateway.java index 8a025aa3a8..a15994754e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGateway.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGateway.java @@ -32,6 +32,7 @@ import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.PartitionInfo; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobmaster.AllocatedSlotReport; import org.apache.flink.runtime.jobmaster.JobMasterId; @@ -142,6 +143,10 @@ public class TestingTaskExecutorGateway implements TaskExecutorGateway { releaseOrPromotePartitionsConsumer.accept(jobId, partitionToRelease, partitionsToPromote); } + @Override + public void releaseClusterPartitions(Collection dataSetsToRelease, Time timeout) { + } + @Override public CompletableFuture triggerCheckpoint(ExecutionAttemptID executionAttemptID, long checkpointID, long checkpointTimestamp, CheckpointOptions checkpointOptions, boolean advanceToEndOfEventTime) { return CompletableFuture.completedFuture(Acknowledge.get()); -- Gitee From 37d10431b162f1a1159d31198ac03599280d28df Mon Sep 17 00:00:00 2001 From: Alexander Fedulov <1492164+afedulov@users.noreply.github.com> Date: Tue, 17 Mar 2020 21:29:22 +0100 Subject: [PATCH 342/885] [hotfix][metrics] Do not log reporter config to prevent credentials leak --- .../java/org/apache/flink/runtime/metrics/ReporterSetup.java | 1 - 1 file changed, 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java index 98824cc567..40fb9ddbf6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java @@ -120,7 +120,6 @@ public final class ReporterSetup { } private static ReporterSetup createReporterSetup(String reporterName, MetricConfig metricConfig, MetricReporter reporter) { - LOG.info("Configuring {} with {}.", reporterName, metricConfig); reporter.open(metricConfig); return new ReporterSetup(reporterName, metricConfig, reporter); -- Gitee From eec8fea54ba29ebdacccb3dd916dcebf3598af95 Mon Sep 17 00:00:00 2001 From: czhang2 Date: Fri, 27 Mar 2020 20:17:09 +0800 Subject: [PATCH 343/885] [FLINK-16594][runtime] Fix typo in GlobalAggregateManager Javadoc This closes #11548. --- .../flink/runtime/taskexecutor/GlobalAggregateManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/GlobalAggregateManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/GlobalAggregateManager.java index 9a3e95451d..9b748a7160 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/GlobalAggregateManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/GlobalAggregateManager.java @@ -23,7 +23,7 @@ import org.apache.flink.api.common.functions.AggregateFunction; /** - * This interface gives access to transient, named, global aggregates. This can be sued to share + * This interface gives access to transient, named, global aggregates. This can be used to share * state amongst parallel tasks in a job. It is not designed for high throughput updates * and the aggregates do NOT survive a job failure. Each call to the updateGlobalAggregate() * method results in serialized RPC communication with the JobMaster so use with care. -- Gitee From d91cd98d7ef9506c2ebff87e6034972008e208fb Mon Sep 17 00:00:00 2001 From: Alexander Fedulov <1492164+afedulov@users.noreply.github.com> Date: Thu, 26 Mar 2020 17:57:36 +0100 Subject: [PATCH 344/885] [FLINK-16831][e2e] Support plugin directory as JarLocation Adds the plugins directory to the set of supported jar locations. Every plugin must be contained in a sub-directory within the plugins directory, so for simplicity we always create a parent directory regardless of what the target location is. --- .../apache/flink/tests/util/flink/FlinkDistribution.java | 7 ++++++- .../org/apache/flink/tests/util/flink/JarLocation.java | 1 + 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkDistribution.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkDistribution.java index bbb95d2a6b..731277082f 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkDistribution.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkDistribution.java @@ -71,6 +71,7 @@ final class FlinkDistribution { private final Path conf; private final Path log; private final Path bin; + private final Path plugins; private final Configuration defaultConfig; @@ -80,6 +81,7 @@ final class FlinkDistribution { lib = distributionDir.resolve("lib"); conf = distributionDir.resolve("conf"); log = distributionDir.resolve("log"); + plugins = distributionDir.resolve("plugins"); defaultConfig = new UnmodifiableConfiguration(GlobalConfiguration.loadConfiguration(conf.toAbsolutePath().toString())); } @@ -218,7 +220,8 @@ final class FlinkDistribution { } if (jarOptional.isPresent()) { final Path sourceJar = jarOptional.get(); - final Path targetJar = target.resolve(sourceJar.getFileName()); + final Path targetJar = target.resolve(operation.getJarNamePrefix()).resolve(sourceJar.getFileName()); + Files.createDirectories(targetJar.getParent()); switch (operation.getOperationType()){ case COPY: Files.copy(sourceJar, targetJar); @@ -241,6 +244,8 @@ final class FlinkDistribution { return lib; case OPT: return opt; + case PLUGINS: + return plugins; default: throw new IllegalStateException(); } diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/JarLocation.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/JarLocation.java index 5af964a8eb..f15ba12792 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/JarLocation.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/JarLocation.java @@ -23,4 +23,5 @@ package org.apache.flink.tests.util.flink; public enum JarLocation { LIB, OPT, + PLUGINS } -- Gitee From 35f4ae7f33ac357cd42eca26a5a3a6e5e9b25be6 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Fri, 27 Mar 2020 16:06:18 +0100 Subject: [PATCH 345/885] [hotfix][metrics][prometheus] Add missing import --- .../flink/metrics/prometheus/PrometheusPushGatewayReporter.java | 1 + 1 file changed, 1 insertion(+) diff --git a/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusPushGatewayReporter.java b/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusPushGatewayReporter.java index 38bd2083ec..d32f393a46 100644 --- a/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusPushGatewayReporter.java +++ b/flink-metrics/flink-metrics-prometheus/src/main/java/org/apache/flink/metrics/prometheus/PrometheusPushGatewayReporter.java @@ -21,6 +21,7 @@ package org.apache.flink.metrics.prometheus; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.metrics.Metric; import org.apache.flink.metrics.MetricConfig; +import org.apache.flink.metrics.reporter.InstantiateViaFactory; import org.apache.flink.metrics.reporter.MetricReporter; import org.apache.flink.metrics.reporter.Scheduled; import org.apache.flink.util.AbstractID; -- Gitee From a0ea5e9980fee01d961a2f7c249e400325220a98 Mon Sep 17 00:00:00 2001 From: Alexander Fedulov <1492164+afedulov@users.noreply.github.com> Date: Tue, 17 Mar 2020 15:57:39 +0100 Subject: [PATCH 346/885] [FLINK-16832][metrics] Refactor ReporterSetup Introduce methods for better readibility. --- .../flink/runtime/metrics/ReporterSetup.java | 76 +++++++++++-------- 1 file changed, 46 insertions(+), 30 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java index 40fb9ddbf6..62cb5435eb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java @@ -127,23 +127,39 @@ public final class ReporterSetup { public static List fromConfiguration(final Configuration configuration) { String includedReportersString = configuration.getString(MetricOptions.REPORTERS_LIST, ""); + + Set namedReporters = findEnabledReportersInConfiguration(configuration, includedReportersString); + + if (namedReporters.isEmpty()) { + return Collections.emptyList(); + } + + final List> reporterConfigurations = loadReporterConfigurations(configuration, namedReporters); + + final Map reporterFactories = loadReporterFactories(); + + return setupReporters(reporterFactories, reporterConfigurations); + } + + private static Set findEnabledReportersInConfiguration(Configuration configuration, String includedReportersString) { Set includedReporters = reporterListPattern.splitAsStream(includedReportersString) .filter(r -> !r.isEmpty()) // splitting an empty string results in an empty string on jdk9+ .collect(Collectors.toSet()); // use a TreeSet to make the reporter order deterministic, which is useful for testing - Set namedReporters = new TreeSet<>(String::compareTo); - // scan entire configuration for "metric.reporter" keys and parse individual reporter configurations + Set namedOrderedReporters = new TreeSet<>(String::compareTo); + + // scan entire configuration for keys starting with METRICS_REPORTER_PREFIX and determine the set of enabled reporters for (String key : configuration.keySet()) { if (key.startsWith(ConfigConstants.METRICS_REPORTER_PREFIX)) { Matcher matcher = reporterClassPattern.matcher(key); if (matcher.matches()) { String reporterName = matcher.group(1); if (includedReporters.isEmpty() || includedReporters.contains(reporterName)) { - if (namedReporters.contains(reporterName)) { + if (namedOrderedReporters.contains(reporterName)) { LOG.warn("Duplicate class configuration detected for reporter {}.", reporterName); } else { - namedReporters.add(reporterName); + namedOrderedReporters.add(reporterName); } } else { LOG.info("Excluding reporter {}, not configured in reporter list ({}).", reporterName, includedReportersString); @@ -151,12 +167,11 @@ public final class ReporterSetup { } } } + return namedOrderedReporters; + } - if (namedReporters.isEmpty()) { - return Collections.emptyList(); - } - - List> reporterConfigurations = new ArrayList<>(namedReporters.size()); + private static List> loadReporterConfigurations(Configuration configuration, Set namedReporters) { + final List> reporterConfigurations = new ArrayList<>(namedReporters.size()); for (String namedReporter: namedReporters) { DelegatingConfiguration delegatingConfiguration = new DelegatingConfiguration( @@ -165,27 +180,7 @@ public final class ReporterSetup { reporterConfigurations.add(Tuple2.of(namedReporter, delegatingConfiguration)); } - - final Map reporterFactories = loadReporterFactories(); - List reporterArguments = new ArrayList<>(reporterConfigurations.size()); - for (Tuple2 reporterConfiguration: reporterConfigurations) { - String reporterName = reporterConfiguration.f0; - Configuration reporterConfig = reporterConfiguration.f1; - - try { - Optional metricReporterOptional = loadReporter(reporterName, reporterConfig, reporterFactories); - metricReporterOptional.ifPresent(reporter -> { - MetricConfig metricConfig = new MetricConfig(); - reporterConfig.addAllToProperties(metricConfig); - - reporterArguments.add(createReporterSetup(reporterName, metricConfig, reporter)); - }); - } - catch (Throwable t) { - LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t); - } - } - return reporterArguments; + return reporterConfigurations; } private static Map loadReporterFactories() { @@ -207,6 +202,27 @@ public final class ReporterSetup { return Collections.unmodifiableMap(reporterFactories); } + private static List setupReporters(Map reporterFactories, List> reporterConfigurations) { + List reporterSetups = new ArrayList<>(reporterConfigurations.size()); + for (Tuple2 reporterConfiguration: reporterConfigurations) { + String reporterName = reporterConfiguration.f0; + Configuration reporterConfig = reporterConfiguration.f1; + + try { + Optional metricReporterOptional = loadReporter(reporterName, reporterConfig, reporterFactories); + metricReporterOptional.ifPresent(reporter -> { + MetricConfig metricConfig = new MetricConfig(); + reporterConfig.addAllToProperties(metricConfig); + reporterSetups.add(createReporterSetup(reporterName, metricConfig, reporter)); + }); + } + catch (Throwable t) { + LOG.error("Could not instantiate metrics reporter {}. Metrics might not be exposed/reported.", reporterName, t); + } + } + return reporterSetups; + } + private static Optional loadReporter( final String reporterName, final Configuration reporterConfig, -- Gitee From 4308e0c5b4e77e0f5fe0ae09ec8f6c6efc361a2a Mon Sep 17 00:00:00 2001 From: Alexander Fedulov <1492164+afedulov@users.noreply.github.com> Date: Fri, 21 Feb 2020 15:46:48 +0100 Subject: [PATCH 347/885] [FLINK-16222][core] Relax Plugin bounded type parameter constraint --- .../java/org/apache/flink/core/plugin/PluginLoader.java | 7 +++---- .../java/org/apache/flink/core/plugin/PluginManager.java | 2 +- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/core/plugin/PluginLoader.java b/flink-core/src/main/java/org/apache/flink/core/plugin/PluginLoader.java index 6dbe6753ab..7287399cf3 100644 --- a/flink-core/src/main/java/org/apache/flink/core/plugin/PluginLoader.java +++ b/flink-core/src/main/java/org/apache/flink/core/plugin/PluginLoader.java @@ -20,7 +20,6 @@ package org.apache.flink.core.plugin; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.util.ArrayUtils; -import org.apache.flink.util.ChildFirstClassLoader; import org.apache.flink.util.TemporaryClassLoaderContext; import javax.annotation.concurrent.ThreadSafe; @@ -35,7 +34,7 @@ import java.util.ServiceLoader; /** * A {@link PluginLoader} is used by the {@link PluginManager} to load a single plugin. It is essentially a combination - * of a {@link ChildFirstClassLoader} and {@link ServiceLoader}. This class can locate and load service implementations + * of a {@link PluginClassLoader} and {@link ServiceLoader}. This class can locate and load service implementations * from the plugin for a given SPI. The {@link PluginDescriptor}, which among other information contains the resource * URLs, is provided at construction. */ @@ -69,7 +68,7 @@ public class PluginLoader { * @param

    Type of the requested plugin service. * @return An iterator of all implementations of the given service interface that could be loaded from the plugin. */ - public

    Iterator

    load(Class

    service) { + public

    Iterator

    load(Class

    service) { try (TemporaryClassLoaderContext ignored = TemporaryClassLoaderContext.of(pluginClassLoader)) { return new ContextClassLoaderSettingIterator<>( ServiceLoader.load(service, pluginClassLoader).iterator(), @@ -83,7 +82,7 @@ public class PluginLoader { * * @param

    type of the iterated plugin element. */ - static class ContextClassLoaderSettingIterator

    implements Iterator

    { + static class ContextClassLoaderSettingIterator

    implements Iterator

    { private final Iterator

    delegate; private final ClassLoader pluginClassLoader; diff --git a/flink-core/src/main/java/org/apache/flink/core/plugin/PluginManager.java b/flink-core/src/main/java/org/apache/flink/core/plugin/PluginManager.java index 23857326d5..8d50585411 100644 --- a/flink-core/src/main/java/org/apache/flink/core/plugin/PluginManager.java +++ b/flink-core/src/main/java/org/apache/flink/core/plugin/PluginManager.java @@ -63,7 +63,7 @@ public class PluginManager { * @param

    Type of the requested plugin service. * @return Iterator over all implementations of the given service that could be loaded from all known plugins. */ - public

    Iterator

    load(Class

    service) { + public

    Iterator

    load(Class

    service) { ArrayList> combinedIterators = new ArrayList<>(pluginDescriptors.size()); for (PluginDescriptor pluginDescriptor : pluginDescriptors) { PluginLoader pluginLoader = PluginLoader.create(pluginDescriptor, parentClassLoader, alwaysParentFirstPatterns); -- Gitee From 850c7098fd4021d50146ecd40c26816bfab66906 Mon Sep 17 00:00:00 2001 From: Alexander Fedulov <1492164+afedulov@users.noreply.github.com> Date: Sat, 15 Jun 2019 16:00:56 +0200 Subject: [PATCH 348/885] [FLINK-16222][runtime] Introduce PluginManager to ReporterSetup --- .../entrypoint/MesosJobClusterEntrypoint.java | 5 ++-- .../MesosSessionClusterEntrypoint.java | 5 ++-- .../entrypoint/MesosTaskExecutorRunner.java | 7 ++++-- .../runtime/entrypoint/ClusterEntrypoint.java | 24 ++++++++++--------- .../flink/runtime/metrics/ReporterSetup.java | 5 +++- .../runtime/minicluster/MiniCluster.java | 2 +- .../taskexecutor/TaskManagerRunner.java | 14 ++++++----- .../metrics/MetricRegistryImplTest.java | 2 +- .../runtime/metrics/ReporterSetupTest.java | 24 +++++++++---------- .../taskexecutor/TaskManagerRunnerTest.java | 5 +++- ...TaskManagerProcessFailureRecoveryTest.java | 5 +++- ...ManagerHAProcessFailureRecoveryITCase.java | 5 +++- .../flink/yarn/YarnTaskExecutorRunner.java | 7 ++++-- 13 files changed, 67 insertions(+), 43 deletions(-) diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java index 052c3aa3b8..f51d903edc 100755 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java @@ -20,6 +20,7 @@ package org.apache.flink.mesos.entrypoint; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.core.plugin.PluginManager; import org.apache.flink.mesos.runtime.clusterframework.MesosResourceManagerFactory; import org.apache.flink.mesos.runtime.clusterframework.services.MesosServices; import org.apache.flink.mesos.runtime.clusterframework.services.MesosServicesUtils; @@ -70,8 +71,8 @@ public class MesosJobClusterEntrypoint extends JobClusterEntrypoint { } @Override - protected void initializeServices(Configuration config) throws Exception { - super.initializeServices(config); + protected void initializeServices(Configuration config, PluginManager pluginManager) throws Exception { + super.initializeServices(config, pluginManager); final String hostname = config.getString(JobManagerOptions.ADDRESS); diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java index d7223564b4..6c6c64af64 100755 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java @@ -20,6 +20,7 @@ package org.apache.flink.mesos.entrypoint; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.core.plugin.PluginManager; import org.apache.flink.mesos.runtime.clusterframework.MesosResourceManagerFactory; import org.apache.flink.mesos.runtime.clusterframework.services.MesosServices; import org.apache.flink.mesos.runtime.clusterframework.services.MesosServicesUtils; @@ -67,8 +68,8 @@ public class MesosSessionClusterEntrypoint extends SessionClusterEntrypoint { } @Override - protected void initializeServices(Configuration config) throws Exception { - super.initializeServices(config); + protected void initializeServices(Configuration config, PluginManager pluginManager) throws Exception { + super.initializeServices(config, pluginManager); final String hostname = config.getString(JobManagerOptions.ADDRESS); diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosTaskExecutorRunner.java b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosTaskExecutorRunner.java index d87ab7bb85..92f4c13757 100644 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosTaskExecutorRunner.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosTaskExecutorRunner.java @@ -21,6 +21,7 @@ package org.apache.flink.mesos.entrypoint; import org.apache.flink.configuration.AkkaOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.plugin.PluginManager; import org.apache.flink.core.plugin.PluginUtils; import org.apache.flink.mesos.runtime.clusterframework.MesosConfigKeys; import org.apache.flink.mesos.util.MesosUtils; @@ -86,8 +87,10 @@ public class MesosTaskExecutorRunner { final Map envs = System.getenv(); + final PluginManager pluginManager = PluginUtils.createPluginManagerFromRootFolder(configuration); + // configure the filesystems - FileSystem.initialize(configuration, PluginUtils.createPluginManagerFromRootFolder(configuration)); + FileSystem.initialize(configuration, pluginManager); // tell akka to die in case of an error configuration.setBoolean(AkkaOptions.JVM_EXIT_ON_FATAL_ERROR, true); @@ -103,7 +106,7 @@ public class MesosTaskExecutorRunner { try { SecurityUtils.getInstalledContext().runSecured(() -> { - TaskManagerRunner.runTaskManager(configuration, resourceId); + TaskManagerRunner.runTaskManager(configuration, resourceId, pluginManager); return 0; }); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java index 59d1eb6141..8566ccd23c 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java @@ -30,6 +30,7 @@ import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.RestOptions; import org.apache.flink.configuration.WebOptions; import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.plugin.PluginManager; import org.apache.flink.core.plugin.PluginUtils; import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.clusterframework.ApplicationStatus; @@ -159,13 +160,13 @@ public abstract class ClusterEntrypoint implements AutoCloseableAsync, FatalErro LOG.info("Starting {}.", getClass().getSimpleName()); try { - - configureFileSystems(configuration); + PluginManager pluginManager = PluginUtils.createPluginManagerFromRootFolder(configuration); + configureFileSystems(configuration, pluginManager); SecurityContext securityContext = installSecurityContext(configuration); securityContext.runSecured((Callable) () -> { - runCluster(configuration); + runCluster(configuration, pluginManager); return null; }); @@ -188,9 +189,9 @@ public abstract class ClusterEntrypoint implements AutoCloseableAsync, FatalErro } } - private void configureFileSystems(Configuration configuration) { + private void configureFileSystems(Configuration configuration, PluginManager pluginManager) { LOG.info("Install default filesystem."); - FileSystem.initialize(configuration, PluginUtils.createPluginManagerFromRootFolder(configuration)); + FileSystem.initialize(configuration, pluginManager); } private SecurityContext installSecurityContext(Configuration configuration) throws Exception { @@ -201,9 +202,10 @@ public abstract class ClusterEntrypoint implements AutoCloseableAsync, FatalErro return SecurityUtils.getInstalledContext(); } - private void runCluster(Configuration configuration) throws Exception { + private void runCluster(Configuration configuration, PluginManager pluginManager) throws Exception { synchronized (lock) { - initializeServices(configuration); + + initializeServices(configuration, pluginManager); // write host information into configuration configuration.setString(JobManagerOptions.ADDRESS, commonRpcService.getAddress()); @@ -242,7 +244,7 @@ public abstract class ClusterEntrypoint implements AutoCloseableAsync, FatalErro } } - protected void initializeServices(Configuration configuration) throws Exception { + protected void initializeServices(Configuration configuration, PluginManager pluginManager) throws Exception { LOG.info("Initializing cluster services."); @@ -265,7 +267,7 @@ public abstract class ClusterEntrypoint implements AutoCloseableAsync, FatalErro blobServer = new BlobServer(configuration, haServices.createBlobStore()); blobServer.start(); heartbeatServices = createHeartbeatServices(configuration); - metricRegistry = createMetricRegistry(configuration); + metricRegistry = createMetricRegistry(configuration, pluginManager); final RpcService metricQueryServiceRpcService = MetricUtils.startMetricsRpcService(configuration, commonRpcService.getAddress()); metricRegistry.startQueryService(metricQueryServiceRpcService, null); @@ -308,10 +310,10 @@ public abstract class ClusterEntrypoint implements AutoCloseableAsync, FatalErro return HeartbeatServices.fromConfiguration(configuration); } - protected MetricRegistryImpl createMetricRegistry(Configuration configuration) { + protected MetricRegistryImpl createMetricRegistry(Configuration configuration, PluginManager pluginManager) { return new MetricRegistryImpl( MetricRegistryConfiguration.fromConfiguration(configuration), - ReporterSetup.fromConfiguration(configuration)); + ReporterSetup.fromConfiguration(configuration, pluginManager)); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java index 62cb5435eb..4fdc68a5b6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java @@ -23,6 +23,7 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.DelegatingConfiguration; import org.apache.flink.configuration.MetricOptions; +import org.apache.flink.core.plugin.PluginManager; import org.apache.flink.metrics.MetricConfig; import org.apache.flink.metrics.reporter.InstantiateViaFactory; import org.apache.flink.metrics.reporter.MetricReporter; @@ -32,6 +33,8 @@ import org.apache.flink.runtime.metrics.scope.ScopeFormat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -125,7 +128,7 @@ public final class ReporterSetup { return new ReporterSetup(reporterName, metricConfig, reporter); } - public static List fromConfiguration(final Configuration configuration) { + public static List fromConfiguration(final Configuration configuration, @Nullable final PluginManager pluginManager) { String includedReportersString = configuration.getString(MetricOptions.REPORTERS_LIST, ""); Set namedReporters = findEnabledReportersInConfiguration(configuration, includedReportersString); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java index 41e3bdf461..4f4316f5cb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java @@ -721,7 +721,7 @@ public class MiniCluster implements JobExecutorService, AutoCloseableAsync { protected MetricRegistryImpl createMetricRegistry(Configuration config) { return new MetricRegistryImpl( MetricRegistryConfiguration.fromConfiguration(config), - ReporterSetup.fromConfiguration(config)); + ReporterSetup.fromConfiguration(config, null)); } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java index 0478953e8e..6fbd874a12 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java @@ -26,6 +26,7 @@ import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.configuration.WebOptions; import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.plugin.PluginManager; import org.apache.flink.core.plugin.PluginUtils; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.akka.AkkaUtils; @@ -118,7 +119,7 @@ public class TaskManagerRunner implements FatalErrorHandler, AutoCloseableAsync private boolean shutdown; - public TaskManagerRunner(Configuration configuration, ResourceID resourceId) throws Exception { + public TaskManagerRunner(Configuration configuration, ResourceID resourceId, PluginManager pluginManager) throws Exception { this.configuration = checkNotNull(configuration); this.resourceId = checkNotNull(resourceId); @@ -139,7 +140,7 @@ public class TaskManagerRunner implements FatalErrorHandler, AutoCloseableAsync metricRegistry = new MetricRegistryImpl( MetricRegistryConfiguration.fromConfiguration(configuration), - ReporterSetup.fromConfiguration(configuration)); + ReporterSetup.fromConfiguration(configuration, pluginManager)); final RpcService metricQueryServiceRpcService = MetricUtils.startMetricsRpcService(configuration, rpcService.getAddress()); metricRegistry.startQueryService(metricQueryServiceRpcService, resourceId); @@ -307,8 +308,8 @@ public class TaskManagerRunner implements FatalErrorHandler, AutoCloseableAsync return GlobalConfiguration.loadConfiguration(clusterConfiguration.getConfigDir(), dynamicProperties); } - public static void runTaskManager(Configuration configuration, ResourceID resourceId) throws Exception { - final TaskManagerRunner taskManagerRunner = new TaskManagerRunner(configuration, resourceId); + public static void runTaskManager(Configuration configuration, ResourceID resourceId, PluginManager pluginManager) throws Exception { + final TaskManagerRunner taskManagerRunner = new TaskManagerRunner(configuration, resourceId, pluginManager); taskManagerRunner.start(); } @@ -317,12 +318,13 @@ public class TaskManagerRunner implements FatalErrorHandler, AutoCloseableAsync try { final Configuration configuration = loadConfiguration(args); - FileSystem.initialize(configuration, PluginUtils.createPluginManagerFromRootFolder(configuration)); + final PluginManager pluginManager = PluginUtils.createPluginManagerFromRootFolder(configuration); + FileSystem.initialize(configuration, pluginManager); SecurityUtils.install(new SecurityConfiguration(configuration)); SecurityUtils.getInstalledContext().runSecured(() -> { - runTaskManager(configuration, resourceID); + runTaskManager(configuration, resourceID, pluginManager); return null; }); } catch (Throwable t) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryImplTest.java index 0f69aca2d2..ed9d913eec 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryImplTest.java @@ -280,7 +280,7 @@ public class MetricRegistryImplTest extends TestLogger { config.setString(MetricOptions.SCOPE_DELIMITER, "_"); config.setString(MetricOptions.SCOPE_NAMING_TM, "A.B.C.D.E"); - MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(config), ReporterSetup.fromConfiguration(config)); + MetricRegistryImpl registry = new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(config), ReporterSetup.fromConfiguration(config, null)); TaskManagerMetricGroup tmGroup = new TaskManagerMetricGroup(registry, "host", "id"); assertEquals("A_B_C_D_E_name", tmGroup.getMetricIdentifier("name")); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/ReporterSetupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/ReporterSetupTest.java index bd1715b8d4..f687c66034 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/ReporterSetupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/ReporterSetupTest.java @@ -68,7 +68,7 @@ public class ReporterSetupTest extends TestLogger { configureReporter1(config); - final List reporterSetups = ReporterSetup.fromConfiguration(config); + final List reporterSetups = ReporterSetup.fromConfiguration(config, null); Assert.assertEquals(1, reporterSetups.size()); @@ -86,7 +86,7 @@ public class ReporterSetupTest extends TestLogger { configureReporter1(config); configureReporter2(config); - final List reporterSetups = ReporterSetup.fromConfiguration(config); + final List reporterSetups = ReporterSetup.fromConfiguration(config, null); Assert.assertEquals(2, reporterSetups.size()); @@ -117,7 +117,7 @@ public class ReporterSetupTest extends TestLogger { config.setString(MetricOptions.REPORTERS_LIST, "reporter2"); - final List reporterSetups = ReporterSetup.fromConfiguration(config); + final List reporterSetups = ReporterSetup.fromConfiguration(config, null); Assert.assertEquals(1, reporterSetups.size()); @@ -131,7 +131,7 @@ public class ReporterSetupTest extends TestLogger { config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "reporter1." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter1.class.getName()); - final List reporterSetups = ReporterSetup.fromConfiguration(config); + final List reporterSetups = ReporterSetup.fromConfiguration(config, null); Assert.assertEquals(1, reporterSetups.size()); @@ -151,7 +151,7 @@ public class ReporterSetupTest extends TestLogger { config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test2." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter12.class.getName()); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test3." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter13.class.getName()); - List reporterSetups = ReporterSetup.fromConfiguration(config); + List reporterSetups = ReporterSetup.fromConfiguration(config, null); assertEquals(3, reporterSetups.size()); @@ -230,7 +230,7 @@ public class ReporterSetupTest extends TestLogger { config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test." + ConfigConstants.METRICS_REPORTER_FACTORY_CLASS_SUFFIX, TestReporterFactory.class.getName()); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test." + ConfigConstants.METRICS_REPORTER_EXCLUDED_VARIABLES, excludedVariable1 + ";" + excludedVariable2); - final List reporterSetups = ReporterSetup.fromConfiguration(config); + final List reporterSetups = ReporterSetup.fromConfiguration(config, null); assertEquals(1, reporterSetups.size()); @@ -247,7 +247,7 @@ public class ReporterSetupTest extends TestLogger { final Configuration config = new Configuration(); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test." + ConfigConstants.METRICS_REPORTER_FACTORY_CLASS_SUFFIX, TestReporterFactory.class.getName()); - final List reporterSetups = ReporterSetup.fromConfiguration(config); + final List reporterSetups = ReporterSetup.fromConfiguration(config, null); assertEquals(1, reporterSetups.size()); @@ -265,7 +265,7 @@ public class ReporterSetupTest extends TestLogger { config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test." + ConfigConstants.METRICS_REPORTER_FACTORY_CLASS_SUFFIX, InstantiationTypeTrackingTestReporterFactory.class.getName()); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, InstantiationTypeTrackingTestReporter.class.getName()); - final List reporterSetups = ReporterSetup.fromConfiguration(config); + final List reporterSetups = ReporterSetup.fromConfiguration(config, null); assertEquals(1, reporterSetups.size()); @@ -284,7 +284,7 @@ public class ReporterSetupTest extends TestLogger { config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test." + ConfigConstants.METRICS_REPORTER_FACTORY_CLASS_SUFFIX, TestReporterFactory.class.getName()); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "fail." + ConfigConstants.METRICS_REPORTER_FACTORY_CLASS_SUFFIX, FailingFactory.class.getName()); - final List reporterSetups = ReporterSetup.fromConfiguration(config); + final List reporterSetups = ReporterSetup.fromConfiguration(config, null); assertEquals(1, reporterSetups.size()); } @@ -298,7 +298,7 @@ public class ReporterSetupTest extends TestLogger { config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test1." + ConfigConstants.METRICS_REPORTER_FACTORY_CLASS_SUFFIX, InstantiationTypeTrackingTestReporterFactory.class.getName()); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test2." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, InstantiationTypeTrackingTestReporter.class.getName()); - final List reporterSetups = ReporterSetup.fromConfiguration(config); + final List reporterSetups = ReporterSetup.fromConfiguration(config, null); assertEquals(2, reporterSetups.size()); @@ -317,7 +317,7 @@ public class ReporterSetupTest extends TestLogger { config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test." + ConfigConstants.METRICS_REPORTER_FACTORY_CLASS_SUFFIX, ConfigExposingReporterFactory.class.getName()); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test.arg", "hello"); - ReporterSetup.fromConfiguration(config); + ReporterSetup.fromConfiguration(config, null); Properties passedConfig = ConfigExposingReporterFactory.lastConfig; assertEquals("hello", passedConfig.getProperty("arg")); @@ -331,7 +331,7 @@ public class ReporterSetupTest extends TestLogger { final Configuration config = new Configuration(); config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, InstantiationTypeTrackingTestReporter2.class.getName()); - final List reporterSetups = ReporterSetup.fromConfiguration(config); + final List reporterSetups = ReporterSetup.fromConfiguration(config, null); assertEquals(1, reporterSetups.size()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunnerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunnerTest.java index 3ce10f67c3..2af131912f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunnerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunnerTest.java @@ -21,6 +21,8 @@ package org.apache.flink.runtime.taskexecutor; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.core.plugin.PluginManager; +import org.apache.flink.core.plugin.PluginUtils; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.testutils.SystemExitTrackingSecurityManager; import org.apache.flink.util.TestLogger; @@ -92,7 +94,8 @@ public class TaskManagerRunnerTest extends TestLogger { } private static TaskManagerRunner createTaskManagerRunner(final Configuration configuration) throws Exception { - TaskManagerRunner taskManagerRunner = new TaskManagerRunner(configuration, ResourceID.generate()); + final PluginManager pluginManager = PluginUtils.createPluginManagerFromRootFolder(configuration); + TaskManagerRunner taskManagerRunner = new TaskManagerRunner(configuration, ResourceID.generate(), pluginManager); taskManagerRunner.start(); return taskManagerRunner; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java index d9c82303c0..f71d193105 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java @@ -27,6 +27,8 @@ import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.MemorySize; import org.apache.flink.configuration.RestOptions; import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.core.plugin.PluginManager; +import org.apache.flink.core.plugin.PluginUtils; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.entrypoint.StandaloneSessionClusterEntrypoint; import org.apache.flink.runtime.taskexecutor.TaskManagerRunner; @@ -321,8 +323,9 @@ public abstract class AbstractTaskManagerProcessFailureRecoveryTest extends Test try { final ParameterTool parameterTool = ParameterTool.fromArgs(args); Configuration cfg = parameterTool.getConfiguration(); + final PluginManager pluginManager = PluginUtils.createPluginManagerFromRootFolder(cfg); - TaskManagerRunner.runTaskManager(cfg, ResourceID.generate()); + TaskManagerRunner.runTaskManager(cfg, ResourceID.generate(), pluginManager); } catch (Throwable t) { LOG.error("Failed to start TaskManager process", t); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureRecoveryITCase.java index 8400bb4c6a..325ed196ea 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureRecoveryITCase.java @@ -32,6 +32,8 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.configuration.MemorySize; import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.core.plugin.PluginManager; +import org.apache.flink.core.plugin.PluginUtils; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter; @@ -269,9 +271,10 @@ public class JobManagerHAProcessFailureRecoveryITCase extends TestLogger { config, TestingUtils.defaultExecutor()); + final PluginManager pluginManager = PluginUtils.createPluginManagerFromRootFolder(config); // Start the task manager process for (int i = 0; i < numberOfTaskManagers; i++) { - taskManagerRunners[i] = new TaskManagerRunner(config, ResourceID.generate()); + taskManagerRunners[i] = new TaskManagerRunner(config, ResourceID.generate(), pluginManager); taskManagerRunners[i].start(); } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskExecutorRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskExecutorRunner.java index bf28974420..78b4187a22 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskExecutorRunner.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskExecutorRunner.java @@ -24,6 +24,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.SecurityOptions; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.plugin.PluginManager; import org.apache.flink.core.plugin.PluginUtils; import org.apache.flink.runtime.clusterframework.BootstrapTools; import org.apache.flink.runtime.clusterframework.types.ResourceID; @@ -92,7 +93,9 @@ public class YarnTaskExecutorRunner { final Configuration configuration = TaskManagerRunner.loadConfiguration(args); - FileSystem.initialize(configuration, PluginUtils.createPluginManagerFromRootFolder(configuration)); + final PluginManager pluginManager = PluginUtils.createPluginManagerFromRootFolder(configuration); + + FileSystem.initialize(configuration, pluginManager); setupConfigurationAndInstallSecurityContext(configuration, currDir, ENV); @@ -101,7 +104,7 @@ public class YarnTaskExecutorRunner { "ContainerId variable %s not set", YarnResourceManager.ENV_FLINK_CONTAINER_ID); SecurityUtils.getInstalledContext().runSecured((Callable) () -> { - TaskManagerRunner.runTaskManager(configuration, new ResourceID(containerId)); + TaskManagerRunner.runTaskManager(configuration, new ResourceID(containerId), pluginManager); return null; }); } -- Gitee From 1a7acefdfac5169e173e2b221fec7458049c3021 Mon Sep 17 00:00:00 2001 From: Alexander Fedulov <1492164+afedulov@users.noreply.github.com> Date: Tue, 17 Mar 2020 17:56:48 +0100 Subject: [PATCH 349/885] [FLINK-16222][metrics] Support loading reporters as plugins --- .../flink/runtime/metrics/ReporterSetup.java | 31 +++++++++++++++---- 1 file changed, 25 insertions(+), 6 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java index 4fdc68a5b6..da15079d14 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/ReporterSetup.java @@ -30,11 +30,14 @@ import org.apache.flink.metrics.reporter.MetricReporter; import org.apache.flink.metrics.reporter.MetricReporterFactory; import org.apache.flink.runtime.metrics.scope.ScopeFormat; +import org.apache.flink.shaded.guava18.com.google.common.collect.Iterators; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.annotation.Nullable; +import java.io.File; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -139,7 +142,7 @@ public final class ReporterSetup { final List> reporterConfigurations = loadReporterConfigurations(configuration, namedReporters); - final Map reporterFactories = loadReporterFactories(); + final Map reporterFactories = loadAvailableReporterFactories(pluginManager); return setupReporters(reporterFactories, reporterConfigurations); } @@ -186,17 +189,24 @@ public final class ReporterSetup { return reporterConfigurations; } - private static Map loadReporterFactories() { - final ServiceLoader serviceLoader = ServiceLoader.load(MetricReporterFactory.class); - + private static Map loadAvailableReporterFactories(@Nullable PluginManager pluginManager) { final Map reporterFactories = new HashMap<>(2); - final Iterator factoryIterator = serviceLoader.iterator(); + final Iterator factoryIterator = getAllReporterFactories(pluginManager); // do not use streams or for-each loops here because they do not allow catching individual ServiceConfigurationErrors // such an error might be caused if the META-INF/services contains an entry to a non-existing factory class while (factoryIterator.hasNext()) { try { MetricReporterFactory factory = factoryIterator.next(); - reporterFactories.put(factory.getClass().getName(), factory); + String factoryClassName = factory.getClass().getName(); + MetricReporterFactory existingFactory = reporterFactories.get(factoryClassName); + if (existingFactory == null) { + reporterFactories.put(factoryClassName, factory); + LOG.debug("Found reporter factory {} at {} ", + factoryClassName, + new File(factory.getClass().getProtectionDomain().getCodeSource().getLocation().toURI()).getCanonicalPath()); + } else { + LOG.warn("Multiple implementations of the same reporter were found in 'lib' and/or 'plugins' directories for {}. It is recommended to remove redundant reporter JARs to resolve used versions' ambiguity.", factoryClassName); + } } catch (Exception | ServiceConfigurationError e) { LOG.warn("Error while loading reporter factory.", e); } @@ -205,6 +215,15 @@ public final class ReporterSetup { return Collections.unmodifiableMap(reporterFactories); } + private static Iterator getAllReporterFactories(@Nullable PluginManager pluginManager) { + final Iterator factoryIteratorSPI = ServiceLoader.load(MetricReporterFactory.class).iterator(); + final Iterator factoryIteratorPlugins = pluginManager != null + ? pluginManager.load(MetricReporterFactory.class) + : Collections.emptyIterator(); + + return Iterators.concat(factoryIteratorPlugins, factoryIteratorSPI); + } + private static List setupReporters(Map reporterFactories, List> reporterConfigurations) { List reporterSetups = new ArrayList<>(reporterConfigurations.size()); for (Tuple2 reporterConfiguration: reporterConfigurations) { -- Gitee From e25e5213054442dd3bb913437e56bfaf4f00a37b Mon Sep 17 00:00:00 2001 From: Alexander Fedulov <1492164+afedulov@users.noreply.github.com> Date: Fri, 21 Feb 2020 15:50:36 +0100 Subject: [PATCH 350/885] [FLINK-16222][metrics][prometheus] Add plugin e2e test --- .../PrometheusReporterEndToEndITCase.java | 92 +++++++++++++++++-- 1 file changed, 83 insertions(+), 9 deletions(-) diff --git a/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java index 1dba568251..5bde088b06 100644 --- a/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java +++ b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java @@ -21,6 +21,7 @@ package org.apache.flink.metrics.prometheus.tests; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.prometheus.PrometheusReporter; +import org.apache.flink.metrics.prometheus.PrometheusReporterFactory; import org.apache.flink.tests.util.AutoClosableProcess; import org.apache.flink.tests.util.CommandLineWrapper; import org.apache.flink.tests.util.cache.DownloadCache; @@ -46,15 +47,22 @@ import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.nio.file.Files; import java.nio.file.Path; +import java.util.Arrays; +import java.util.Collection; import java.util.List; +import java.util.function.Consumer; import java.util.regex.Pattern; import java.util.stream.Collectors; +import static org.apache.flink.metrics.prometheus.tests.PrometheusReporterEndToEndITCase.TestParams.InstantiationType.FACTORY; +import static org.apache.flink.metrics.prometheus.tests.PrometheusReporterEndToEndITCase.TestParams.InstantiationType.REFLECTION; import static org.apache.flink.tests.util.AutoClosableProcess.runBlocking; import static org.apache.flink.tests.util.AutoClosableProcess.runNonBlocking; @@ -62,6 +70,7 @@ import static org.apache.flink.tests.util.AutoClosableProcess.runNonBlocking; * End-to-end test for the PrometheusReporter. */ @Category(TravisGroup1.class) +@RunWith(Parameterized.class) public class PrometheusReporterEndToEndITCase extends TestLogger { private static final Logger LOG = LoggerFactory.getLogger(PrometheusReporterEndToEndITCase.class); @@ -70,6 +79,7 @@ public class PrometheusReporterEndToEndITCase extends TestLogger { private static final String PROMETHEUS_VERSION = "2.4.3"; private static final String PROMETHEUS_FILE_NAME; + private static final String PROMETHEUS_JAR_PREFIX = "flink-metrics-prometheus"; static { final String base = "prometheus-" + PROMETHEUS_VERSION + '.'; @@ -114,13 +124,36 @@ public class PrometheusReporterEndToEndITCase extends TestLogger { Assume.assumeFalse("This test does not run on Windows.", OperatingSystem.isWindows()); } + @Parameterized.Parameters(name = "{index}: {0}") + public static Collection testParameters() { + return Arrays.asList( + TestParams.from("Jar in 'lib'", + builder -> builder.copyJar(PROMETHEUS_JAR_PREFIX, JarLocation.OPT, JarLocation.LIB), + REFLECTION), + TestParams.from("Jar in 'lib'", + builder -> builder.copyJar(PROMETHEUS_JAR_PREFIX, JarLocation.OPT, JarLocation.LIB), + FACTORY), + TestParams.from("Jar in 'plugins'", + builder -> builder.copyJar(PROMETHEUS_JAR_PREFIX, JarLocation.OPT, JarLocation.PLUGINS), + FACTORY), + TestParams.from("Jar in 'lib' and 'plugins'", + builder -> { + builder.copyJar(PROMETHEUS_JAR_PREFIX, JarLocation.OPT, JarLocation.LIB); + builder.copyJar(PROMETHEUS_JAR_PREFIX, JarLocation.OPT, JarLocation.PLUGINS); + }, + FACTORY) + ); + } + @Rule - public final FlinkResource dist = new LocalStandaloneFlinkResourceFactory() - .create(FlinkResourceSetup.builder() - .moveJar("flink-metrics-prometheus", JarLocation.OPT, JarLocation.LIB) - .addConfiguration(getFlinkConfig()) - .build()) - .get(); + public final FlinkResource dist; + + public PrometheusReporterEndToEndITCase(TestParams params) { + final FlinkResourceSetup.FlinkResourceSetupBuilder builder = FlinkResourceSetup.builder(); + params.getBuilderSetup().accept(builder); + builder.addConfiguration(getFlinkConfig(params.getInstantiationType())); + dist = new LocalStandaloneFlinkResourceFactory().create(builder.build()).get(); + } @Rule public final TemporaryFolder tmp = new TemporaryFolder(); @@ -128,11 +161,18 @@ public class PrometheusReporterEndToEndITCase extends TestLogger { @Rule public final DownloadCache downloadCache = DownloadCache.get(); - private static Configuration getFlinkConfig() { + private static Configuration getFlinkConfig(TestParams.InstantiationType instantiationType) { final Configuration config = new Configuration(); - config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, PrometheusReporter.class.getCanonicalName()); - config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom.port", "9000-9100"); + switch (instantiationType) { + case FACTORY: + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom." + ConfigConstants.METRICS_REPORTER_FACTORY_CLASS_SUFFIX, PrometheusReporterFactory.class.getName()); + break; + case REFLECTION: + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, PrometheusReporter.class.getCanonicalName()); + } + + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "prom.port", "9000-9100"); return config; } @@ -244,4 +284,38 @@ public class PrometheusReporterEndToEndITCase extends TestLogger { } throw new AssertionError("Could not retrieve metric " + metric + " from Prometheus.", reportedException); } + + static class TestParams { + private final String jarLocationDescription; + private final Consumer builderSetup; + private final InstantiationType instantiationType; + + private TestParams(String jarLocationDescription, Consumer builderSetup, InstantiationType instantiationType) { + this.jarLocationDescription = jarLocationDescription; + this.builderSetup = builderSetup; + this.instantiationType = instantiationType; + } + + public static TestParams from(String jarLocationDesription, Consumer builderSetup, InstantiationType instantiationType) { + return new TestParams(jarLocationDesription, builderSetup, instantiationType); + } + + public Consumer getBuilderSetup() { + return builderSetup; + } + + public InstantiationType getInstantiationType() { + return instantiationType; + } + + @Override + public String toString() { + return jarLocationDescription + ", instantiated via " + instantiationType.name().toLowerCase(); + } + + public enum InstantiationType { + REFLECTION, + FACTORY + } + } } -- Gitee From 4f6efedd31d0e7705a094725824ddec8940efb3b Mon Sep 17 00:00:00 2001 From: "Jiangjie (Becket) Qin" Date: Sun, 8 Dec 2019 21:33:07 +0800 Subject: [PATCH 351/885] [FLINK-15100][connector/common] Add abstract implementation for SourceReader (FLIP-27). --- flink-connectors/flink-connector-base/pom.xml | 54 ++++ .../base/source/reader/RecordEmitter.java | 48 ++++ .../base/source/reader/RecordsBySplits.java | 116 ++++++++ .../source/reader/RecordsWithSplitIds.java | 50 ++++ ...SingleThreadMultiplexSourceReaderBase.java | 57 ++++ .../base/source/reader/SourceReaderBase.java | 205 ++++++++++++++ .../source/reader/SourceReaderOptions.java | 62 ++++ .../source/reader/SplitsRecordIterator.java | 96 +++++++ .../source/reader/fetcher/AddSplitsTask.java | 72 +++++ .../base/source/reader/fetcher/FetchTask.java | 86 ++++++ .../fetcher/SingleThreadFetcherManager.java | 55 ++++ .../source/reader/fetcher/SplitFetcher.java | 266 ++++++++++++++++++ .../reader/fetcher/SplitFetcherManager.java | 145 ++++++++++ .../reader/fetcher/SplitFetcherTask.java | 41 +++ .../reader/splitreader/SplitReader.java | 60 ++++ .../reader/splitreader/SplitsAddition.java | 33 +++ .../reader/splitreader/SplitsChange.java | 41 +++ .../FutureCompletingBlockingQueue.java | 90 ++++++ .../synchronization/FutureNotifier.java | 66 +++++ .../source/reader/SourceReaderBaseTest.java | 140 +++++++++ .../source/reader/SourceReaderTestBase.java | 200 +++++++++++++ .../reader/mocks/MockRecordEmitter.java | 37 +++ .../source/reader/mocks/MockSourceReader.java | 62 ++++ .../source/reader/mocks/MockSplitReader.java | 107 +++++++ .../synchronization/FutureNotifierTest.java | 131 +++++++++ flink-connectors/pom.xml | 1 + .../connector/source/SourceReaderContext.java | 6 +- .../flink/util/ThrowableCatchingRunnable.java | 47 ++++ .../source/mocks/MockSourceSplit.java | 108 +++++++ .../mocks/MockSourceSplitSerializer.java | 49 ++++ .../ExecutionJobVertexCoordinatorContext.java | 5 + .../coordination/OperatorCoordinator.java | 2 + 32 files changed, 2533 insertions(+), 5 deletions(-) create mode 100644 flink-connectors/flink-connector-base/pom.xml create mode 100644 flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/RecordEmitter.java create mode 100644 flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/RecordsBySplits.java create mode 100644 flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/RecordsWithSplitIds.java create mode 100644 flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/SingleThreadMultiplexSourceReaderBase.java create mode 100644 flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/SourceReaderBase.java create mode 100644 flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/SourceReaderOptions.java create mode 100644 flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/SplitsRecordIterator.java create mode 100644 flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/fetcher/AddSplitsTask.java create mode 100644 flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/fetcher/FetchTask.java create mode 100644 flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/fetcher/SingleThreadFetcherManager.java create mode 100644 flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/fetcher/SplitFetcher.java create mode 100644 flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/fetcher/SplitFetcherManager.java create mode 100644 flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/fetcher/SplitFetcherTask.java create mode 100644 flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/splitreader/SplitReader.java create mode 100644 flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/splitreader/SplitsAddition.java create mode 100644 flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/splitreader/SplitsChange.java create mode 100644 flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/synchronization/FutureCompletingBlockingQueue.java create mode 100644 flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/synchronization/FutureNotifier.java create mode 100644 flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/SourceReaderBaseTest.java create mode 100644 flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/SourceReaderTestBase.java create mode 100644 flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/mocks/MockRecordEmitter.java create mode 100644 flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/mocks/MockSourceReader.java create mode 100644 flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/mocks/MockSplitReader.java create mode 100644 flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/synchronization/FutureNotifierTest.java create mode 100644 flink-core/src/main/java/org/apache/flink/util/ThrowableCatchingRunnable.java create mode 100644 flink-core/src/test/java/org/apache/flink/api/connector/source/mocks/MockSourceSplit.java create mode 100644 flink-core/src/test/java/org/apache/flink/api/connector/source/mocks/MockSourceSplitSerializer.java diff --git a/flink-connectors/flink-connector-base/pom.xml b/flink-connectors/flink-connector-base/pom.xml new file mode 100644 index 0000000000..b91a86d74f --- /dev/null +++ b/flink-connectors/flink-connector-base/pom.xml @@ -0,0 +1,54 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-connectors + 1.11-SNAPSHOT + .. + + + flink-connector-base + flink-connector-base + + jar + + + + + org.apache.flink + flink-core + ${project.version} + + + + + org.apache.flink + flink-core + ${project.version} + test + test-jar + + + diff --git a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/RecordEmitter.java b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/RecordEmitter.java new file mode 100644 index 0000000000..6a26aa570b --- /dev/null +++ b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/RecordEmitter.java @@ -0,0 +1,48 @@ +/* + 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.connector.base.source.reader; + +import org.apache.flink.api.connector.source.SourceOutput; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; + +/** + * Emit a record to the downstream. + * + * @param the type of the record emitted by the {@link SplitReader} + * @param the type of records that are eventually emitted to the {@link SourceOutput}. + * @param the mutable type of split state. + */ +public interface RecordEmitter { + + /** + * Process and emit the records to the {@link SourceOutput}. A few recommendations to the implementation + * are following: + * + *

      + *
    • The method maybe interrupted in the middle. In that case, the same set of records will be passed + * to the record emitter again later. The implementation needs to make sure it reades + *
    • + *
    + * + * @param element The intermediate element read by the SplitReader. + * @param output The output to which the final records are emit to. + * @param splitState The state of the split. + */ + void emitRecord(E element, SourceOutput output, SplitStateT splitState) throws Exception; +} diff --git a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/RecordsBySplits.java b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/RecordsBySplits.java new file mode 100644 index 0000000000..bdd16fff60 --- /dev/null +++ b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/RecordsBySplits.java @@ -0,0 +1,116 @@ +/* + 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.connector.base.source.reader; + +import org.apache.flink.api.connector.source.SourceSplit; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Set; + +/** + * An implementation of RecordsWithSplitIds to host all the records by splits. + */ +public class RecordsBySplits implements RecordsWithSplitIds { + private Map> recordsBySplits = new LinkedHashMap<>(); + private Set finishedSplits = new HashSet<>(); + + /** + * Add the record from the given split ID. + * + * @param splitId the split ID the record was from. + * @param record the record to add. + */ + public void add(String splitId, E record) { + recordsBySplits.computeIfAbsent(splitId, sid -> new ArrayList<>()).add(record); + } + + /** + * Add the record from the given source split. + * + * @param split the source split the record was from. + * @param record the record to add. + */ + public void add(SourceSplit split, E record) { + add(split.splitId(), record); + } + + /** + * Add multiple records from the given split ID. + * + * @param splitId the split ID given the records were from. + * @param records the records to add. + */ + public void addAll(String splitId, Collection records) { + this.recordsBySplits.compute(splitId, (id, r) -> { + if (r == null) { + r = records; + } else { + r.addAll(records); + } + return r; + }); + } + + /** + * Add multiple records from the given source split. + * + * @param split the source split the records were from. + * @param records the records to add. + */ + public void addAll(SourceSplit split, Collection records) { + addAll(split.splitId(), records); + } + + /** + * Mark the split with the given ID as finished. + * + * @param splitId the ID of the finished split. + */ + public void addFinishedSplit(String splitId) { + finishedSplits.add(splitId); + } + + /** + * Mark multiple splits with the given IDs as finished. + * + * @param splitIds the IDs of the finished splits. + */ + public void addFinishedSplits(Collection splitIds) { + finishedSplits.addAll(splitIds); + } + + @Override + public Set finishedSplits() { + return finishedSplits; + } + + @Override + public Collection splitIds() { + return recordsBySplits.keySet(); + } + + @Override + public Map> recordsBySplits() { + return recordsBySplits; + } +} diff --git a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/RecordsWithSplitIds.java b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/RecordsWithSplitIds.java new file mode 100644 index 0000000000..813c6a5089 --- /dev/null +++ b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/RecordsWithSplitIds.java @@ -0,0 +1,50 @@ +/* + 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.connector.base.source.reader; + +import java.util.Collection; +import java.util.Map; +import java.util.Set; + +/** + * An interface for the elements passed from the fetchers to the source reader. + */ +public interface RecordsWithSplitIds { + + /** + * Get all the split ids. + * + * @return a collection of split ids. + */ + Collection splitIds(); + + /** + * Get all the records by Splits. + * + * @return a mapping from split ids to the records. + */ + Map> recordsBySplits(); + + /** + * Get the finished splits. + * + * @return the finished splits after this RecordsWithSplitIds is returned. + */ + Set finishedSplits(); +} diff --git a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/SingleThreadMultiplexSourceReaderBase.java b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/SingleThreadMultiplexSourceReaderBase.java new file mode 100644 index 0000000000..0e22ef0c08 --- /dev/null +++ b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/SingleThreadMultiplexSourceReaderBase.java @@ -0,0 +1,57 @@ +/* + 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.connector.base.source.reader; + +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.base.source.reader.fetcher.SingleThreadFetcherManager; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.apache.flink.connector.base.source.reader.synchronization.FutureNotifier; + +import java.util.function.Supplier; + +/** + * A abstract {@link SourceReader} implementation that assign all the splits to a single thread to consume. + * @param + * @param + * @param + * @param + */ +public abstract class SingleThreadMultiplexSourceReaderBase + extends SourceReaderBase { + + public SingleThreadMultiplexSourceReaderBase( + FutureNotifier futureNotifier, + FutureCompletingBlockingQueue> elementsQueue, + Supplier> splitFetcherSupplier, + RecordEmitter recordEmitter, + Configuration config, + SourceReaderContext context) { + super( + futureNotifier, + elementsQueue, + new SingleThreadFetcherManager<>(futureNotifier, elementsQueue, splitFetcherSupplier), + recordEmitter, + config, + context); + } +} diff --git a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/SourceReaderBase.java b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/SourceReaderBase.java new file mode 100644 index 0000000000..043b735f20 --- /dev/null +++ b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/SourceReaderBase.java @@ -0,0 +1,205 @@ +/* + 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.connector.base.source.reader; + +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SourceOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.base.source.reader.fetcher.SplitFetcherManager; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.apache.flink.connector.base.source.reader.synchronization.FutureNotifier; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; + +/** + * An abstract implementation of {@link SourceReader} which provides some sychronization between + * the mail box main thread and the SourceReader internal threads. This class allows user to + * just provide a {@link SplitReader} and snapshot the split state. + * + * @param The rich element type that contains information for split state update or timestamp extraction. + * @param The final element type to emit. + * @param the immutable split type. + * @param the mutable type of split state. + */ +public abstract class SourceReaderBase + implements SourceReader { + private static final Logger LOG = LoggerFactory.getLogger(SourceReaderBase.class); + + /** A future notifier to notify when this reader requires attention. */ + private final FutureNotifier futureNotifier; + + /** A queue to buffer the elements fetched by the fetcher thread. */ + private final BlockingQueue> elementsQueue; + + /** The state of the splits. */ + private final Map splitStates; + + /** The record emitter to handle the records read by the SplitReaders. */ + protected final RecordEmitter recordEmitter; + + /** The split fetcher manager to run split fetchers. */ + protected final SplitFetcherManager splitFetcherManager; + + /** The configuration for the reader. */ + protected final SourceReaderOptions options; + + /** The raw configurations that may be used by subclasses. */ + protected final Configuration config; + + /** The context of this source reader. */ + protected SourceReaderContext context; + + /** The last element to ensure it is fully handled. */ + private SplitsRecordIterator splitIter; + + public SourceReaderBase( + FutureNotifier futureNotifier, + FutureCompletingBlockingQueue> elementsQueue, + SplitFetcherManager splitFetcherManager, + RecordEmitter recordEmitter, + Configuration config, + SourceReaderContext context) { + this.futureNotifier = futureNotifier; + this.elementsQueue = elementsQueue; + this.splitFetcherManager = splitFetcherManager; + this.recordEmitter = recordEmitter; + this.splitStates = new HashMap<>(); + this.splitIter = null; + this.options = new SourceReaderOptions(config); + this.config = config; + this.context = context; + } + + @Override + public void start() { + + } + + @Override + public Status pollNext(SourceOutput sourceOutput) throws Exception { + splitFetcherManager.checkErrors(); + // poll from the queue if the last element was successfully handled. Otherwise + // just pass the last element again. + RecordsWithSplitIds recordsWithSplitId = null; + boolean newFetch = splitIter == null || !splitIter.hasNext(); + if (newFetch) { + recordsWithSplitId = elementsQueue.poll(); + } + + Status status; + if (newFetch && recordsWithSplitId == null) { + // No element available, set to available later if needed. + status = Status.AVAILABLE_LATER; + } else { + // Update the record iterator if it is a new fetch. + if (newFetch) { + splitIter = new SplitsRecordIterator<>(recordsWithSplitId); + } + + if (splitIter.hasNext()) { + // emit the record. + recordEmitter.emitRecord(splitIter.next(), sourceOutput, splitStates.get(splitIter.currentSplitId())); + } else { + // First remove the state of the split. + splitIter.finishedSplitIds().forEach(splitStates::remove); + // Handle the finished splits. + onSplitFinished(splitIter.finishedSplitIds()); + } + // Prepare the return status based on the availability of the next element. + status = elementsQueue.isEmpty() ? Status.AVAILABLE_LATER : Status.AVAILABLE_NOW; + } + return status; + } + + @Override + public CompletableFuture isAvailable() { + // The order matters here. We first get the future. After this point, if the queue + // is empty or there is no error in the split fetcher manager, we can ensure that + // the future will be completed by the fetcher once it put an element into the element queue, + // or it will be completed when an error occurs. + CompletableFuture future = futureNotifier.future(); + splitFetcherManager.checkErrors(); + if (!elementsQueue.isEmpty()) { + // The fetcher got the new elements after the last poll, or their is a finished split. + // Simply complete the future and return; + futureNotifier.notifyComplete(); + } + return future; + } + + @Override + public List snapshotState() { + List splits = new ArrayList<>(); + splitStates.forEach((id, state) -> splits.add(toSplitType(id, state))); + return splits; + } + + @Override + public void addSplits(List splits) { + LOG.trace("Adding splits {}", splits); + // Initialize the state for each split. + splits.forEach(s -> splitStates.put(s.splitId(), initializedState(s))); + // Hand over the splits to the split fetcher to start fetch. + splitFetcherManager.addSplits(splits); + } + + @Override + public void handleSourceEvents(SourceEvent sourceEvent) { + // Default action is do nothing. + } + + @Override + public void close() throws Exception { + splitFetcherManager.close(options.sourceReaderCloseTimeout); + } + + // -------------------- Abstract method to allow different implementations ------------------ + /** + * Handles the finished splits to clean the state if needed. + */ + protected abstract void onSplitFinished(Collection finishedSplitIds); + + /** + * When new splits are added to the reader. The initialize the state of the new splits. + * + * @param split a newly added split. + */ + protected abstract SplitStateT initializedState(SplitT split); + + /** + * Convert a mutable SplitStateT to immutable SplitT. + * + * @param splitState splitState. + * @return an immutable Split state. + */ + protected abstract SplitT toSplitType(String splitId, SplitStateT splitState); +} diff --git a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/SourceReaderOptions.java b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/SourceReaderOptions.java new file mode 100644 index 0000000000..294aa5740a --- /dev/null +++ b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/SourceReaderOptions.java @@ -0,0 +1,62 @@ +/* + 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.connector.base.source.reader; + +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.Configuration; + +/** + * The options tht can be set for the {@link SourceReaderBase}. + */ +public class SourceReaderOptions { + + public static final ConfigOption SOURCE_READER_CLOSE_TIMEOUT = + ConfigOptions + .key("source.reader.close.timeout") + .longType() + .defaultValue(30000L) + .withDescription("The timeout when closing the source reader"); + + public static final ConfigOption SOURCE_READER_BOUNDEDNESS = + ConfigOptions + .key("boundedness") + .stringType() + .noDefaultValue() + .withDescription("The boundedness of the source."); + + public static final ConfigOption ELEMENT_QUEUE_CAPACITY = + ConfigOptions + .key("source.reader.element.queue.capacity") + .intType() + .defaultValue(1) + .withDescription("The capacity of the element queue in the source reader."); + + // --------------- final fields ---------------------- + public final long sourceReaderCloseTimeout; + public final Boundedness boundedness; + public final int elementQueueCapacity; + + public SourceReaderOptions(Configuration config) { + this.sourceReaderCloseTimeout = config.getLong(SOURCE_READER_CLOSE_TIMEOUT); + this.boundedness = config.getEnum(Boundedness.class, SOURCE_READER_BOUNDEDNESS); + this.elementQueueCapacity = config.getInteger(ELEMENT_QUEUE_CAPACITY); + } +} diff --git a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/SplitsRecordIterator.java b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/SplitsRecordIterator.java new file mode 100644 index 0000000000..47dfd877fd --- /dev/null +++ b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/SplitsRecordIterator.java @@ -0,0 +1,96 @@ +/* + 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.connector.base.source.reader; + +import java.util.Collection; +import java.util.Iterator; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Set; + +/** + * A class that wraps around a {@link RecordsWithSplitIds} and provide a consistent iterator. + */ +public class SplitsRecordIterator { + private final Map> recordsBySplits; + private final Set finishedSplitIds; + private final Iterator>> splitIter; + private String currentSplitId; + private Iterator recordsIter; + + /** + * Construct a cross-splits iterator for the records. + * + * @param recordsWithSplitIds the records by splits. + */ + public SplitsRecordIterator(RecordsWithSplitIds recordsWithSplitIds) { + this.recordsBySplits = recordsWithSplitIds.recordsBySplits(); + // Remove empty splits; + recordsBySplits.entrySet().removeIf(e -> e.getValue().isEmpty()); + this.splitIter = recordsBySplits.entrySet().iterator(); + this.finishedSplitIds = recordsWithSplitIds.finishedSplits(); + } + + /** + * Whether their are more records available. + * + * @return true if there are more records, false otherwise. + */ + public boolean hasNext() { + if (recordsIter == null || !recordsIter.hasNext()) { + if (splitIter.hasNext()) { + Map.Entry> entry = splitIter.next(); + currentSplitId = entry.getKey(); + recordsIter = entry.getValue().iterator(); + } else { + return false; + } + } + return recordsIter.hasNext() || splitIter.hasNext(); + } + + /** + * Get the next record. + * @return the next record. + */ + public E next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + return recordsIter.next(); + } + + /** + * Get the split id of the last returned record. + * + * @return the split id of the last returned record. + */ + public String currentSplitId() { + return currentSplitId; + } + + /** + * The split Ids that are finished after all the records in this iterator are emitted. + * + * @return a set of finished split Ids. + */ + public Set finishedSplitIds() { + return finishedSplitIds; + } +} diff --git a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/fetcher/AddSplitsTask.java b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/fetcher/AddSplitsTask.java new file mode 100644 index 0000000000..8f9fdc57c8 --- /dev/null +++ b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/fetcher/AddSplitsTask.java @@ -0,0 +1,72 @@ +/* + 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.connector.base.source.reader.fetcher; + +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; + +import java.util.List; +import java.util.Map; +import java.util.Queue; + +/** + * The task to add splits. + */ +class AddSplitsTask implements SplitFetcherTask { + private final SplitReader splitReader; + private final List splitsToAdd; + private final Queue> splitsChanges; + private final Map assignedSplits; + private boolean splitsChangesAdded; + + AddSplitsTask( + SplitReader splitReader, + List splitsToAdd, + Queue> splitsChanges, + Map assignedSplits) { + this.splitReader = splitReader; + this.splitsToAdd = splitsToAdd; + this.splitsChanges = splitsChanges; + this.assignedSplits = assignedSplits; + this.splitsChangesAdded = false; + } + + @Override + public boolean run() throws InterruptedException { + if (!splitsChangesAdded) { + splitsChanges.add(new SplitsAddition<>(splitsToAdd)); + splitsToAdd.forEach(s -> assignedSplits.put(s.splitId(), s)); + splitsChangesAdded = true; + } + splitReader.handleSplitsChanges(splitsChanges); + return splitsChanges.isEmpty(); + } + + @Override + public void wakeUp() { + // Do nothing. + } + + @Override + public String toString() { + return String.format("AddSplitsTask: [%s]", splitsToAdd); + } +} diff --git a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/fetcher/FetchTask.java b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/fetcher/FetchTask.java new file mode 100644 index 0000000000..84ab5d1665 --- /dev/null +++ b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/fetcher/FetchTask.java @@ -0,0 +1,86 @@ +/* + 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.connector.base.source.reader.fetcher; + +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; + +import java.util.Collection; +import java.util.concurrent.BlockingQueue; +import java.util.function.Consumer; + +/** + * The default fetch task that fetches the records into the element queue. + */ +class FetchTask implements SplitFetcherTask { + private final SplitReader splitReader; + private final BlockingQueue> elementsQueue; + private final Consumer> splitFinishedCallback; + private RecordsWithSplitIds lastRecords; + private Thread runningThread; + private volatile boolean wakeup; + + FetchTask( + SplitReader splitReader, + BlockingQueue> elementsQueue, + Consumer> splitFinishedCallback, + Thread runningThread) { + this.splitReader = splitReader; + this.elementsQueue = elementsQueue; + this.splitFinishedCallback = splitFinishedCallback; + this.lastRecords = null; + this.runningThread = runningThread; + this.wakeup = false; + } + + @Override + public boolean run() throws InterruptedException { + if (lastRecords == null) { + lastRecords = splitReader.fetch(); + } + if (!wakeup) { + elementsQueue.put(lastRecords); + splitFinishedCallback.accept(lastRecords.finishedSplits()); + } + synchronized (this) { + wakeup = false; + lastRecords = null; + } + // The return value of fetch task does not matter. + return true; + } + + @Override + public void wakeUp() { + synchronized (this) { + wakeup = true; + if (lastRecords == null) { + splitReader.wakeUp(); + } else { + runningThread.interrupt(); + } + } + } + + @Override + public String toString() { + return "FetchTask"; + } +} diff --git a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/fetcher/SingleThreadFetcherManager.java b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/fetcher/SingleThreadFetcherManager.java new file mode 100644 index 0000000000..2745a36969 --- /dev/null +++ b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/fetcher/SingleThreadFetcherManager.java @@ -0,0 +1,55 @@ +/* + 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.connector.base.source.reader.fetcher; + +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.apache.flink.connector.base.source.reader.synchronization.FutureNotifier; + +import java.util.List; +import java.util.function.Supplier; + +/** + * A Fetcher manager with a single fetcher and assign all the splits to it. + */ +public class SingleThreadFetcherManager + extends SplitFetcherManager { + + public SingleThreadFetcherManager( + FutureNotifier futureNotifier, + FutureCompletingBlockingQueue> elementsQueue, + Supplier> splitReaderSupplier) { + super(futureNotifier, elementsQueue, splitReaderSupplier); + } + + @Override + public void addSplits(List splitsToAdd) { + SplitFetcher fetcher = fetchers.get(0); + if (fetcher == null) { + fetcher = createSplitFetcher(); + // Add the splits to the fetchers. + fetcher.addSplits(splitsToAdd); + startFetcher(fetcher); + } else { + fetcher.addSplits(splitsToAdd); + } + } +} diff --git a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/fetcher/SplitFetcher.java b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/fetcher/SplitFetcher.java new file mode 100644 index 0000000000..792b888ff7 --- /dev/null +++ b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/fetcher/SplitFetcher.java @@ -0,0 +1,266 @@ +/* + 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.connector.base.source.reader.fetcher; + +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Queue; +import java.util.concurrent.BlockingDeque; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * The internal fetcher runnable responsible for polling message from the external system. + */ +public class SplitFetcher implements Runnable { + private static final Logger LOG = LoggerFactory.getLogger(SplitFetcher.class); + private final int id; + private final BlockingDeque taskQueue; + // track the assigned splits so we can suspend the reader when there is no splits assigned. + private final Map assignedSplits; + /** The current split assignments for this fetcher. */ + private final Queue> splitChanges; + private final BlockingQueue> elementsQueue; + private final SplitReader splitReader; + private final Runnable shutdownHook; + private final AtomicBoolean wakeUp; + private final AtomicBoolean closed; + private FetchTask fetchTask; + private volatile Thread runningThread; + private volatile SplitFetcherTask runningTask = null; + + SplitFetcher( + int id, + BlockingQueue> elementsQueue, + SplitReader splitReader, + Runnable shutdownHook) { + + this.id = id; + this.taskQueue = new LinkedBlockingDeque<>(); + this.splitChanges = new LinkedList<>(); + this.elementsQueue = elementsQueue; + this.assignedSplits = new HashMap<>(); + this.splitReader = splitReader; + this.shutdownHook = shutdownHook; + this.wakeUp = new AtomicBoolean(false); + this.closed = new AtomicBoolean(false); + } + + @Override + public void run() { + LOG.info("Starting split fetcher {}", id); + try { + // Remove the split from the assignments if it is already done. + runningThread = Thread.currentThread(); + this.fetchTask = new FetchTask<>( + splitReader, elementsQueue, ids -> ids.forEach(assignedSplits::remove), runningThread); + while (!closed.get()) { + runOnce(); + } + } finally { + // Reset the interrupted flag so the shutdown hook do not got interrupted. + Thread.interrupted(); + shutdownHook.run(); + LOG.info("Split fetcher {} exited.", id); + } + } + + /** + * Package private method to help unit test. + */ + void runOnce() { + try { + // The fetch task should run if the split assignment is not empty or there is a split change. + if (shouldRunFetchTask()) { + runningTask = fetchTask; + } else { + runningTask = taskQueue.take(); + } + // Now the running task is not null. If wakeUp() is called after this point, the fetcher + // thread will not be interrupted. Instead task.wakeUp() will be called. On the other hand, + // If the wakeUp() call was make before this point, the wakeUp flag must have already been + // have been set, and the fetcher thread may or may not be interrupted, depending on + // whether the wakeUp() call was before or after the runningTask assignment. So the + // code does the following: + // 1. check and clear the interrupt flag on the fetcher thread to avoid interruption in + // later code. + // 2. check the wakeUp flag to avoid unnecessary task run. + // Note that the runningTask may still encounter the case that the task is waken up before + // the it starts running. + LOG.debug("Prepare to run {}", runningTask); + if (!Thread.interrupted() && !wakeUp.get() && runningTask.run()) { + LOG.debug("Finished running task {}", runningTask); + // the task has finished running. Set it to null so it won't be enqueued. + runningTask = null; + } + } catch (InterruptedException ie) { + if (closed.get()) { + // The fetcher is closed, just return; + return; + } else if (wakeUp.get()) { + // The fetcher thread has just been waken up. So ignore the interrupted exception + // and continue; + LOG.debug("Split fetcher has been waken up."); + } else { + throw new RuntimeException(String.format( + "SplitFetcher thread %d interrupted while polling the records", id), ie); + } + } + // If the task is not null that means this task needs to be re-executed. This only + // happens when the task is the fetching task or the task was interrupted. + maybeEnqueueTask(runningTask); + synchronized (wakeUp) { + // Set the running task to null. It is necessary for the shutdown method to avoid + // unnecessarily interrupt the running task. + runningTask = null; + // Set the wakeUp flag to false. + wakeUp.set(false); + } + } + + /** + * Add splits to the split fetcher. This operation is asynchronous. + * + * @param splitsToAdd the splits to add. + */ + public void addSplits(List splitsToAdd) { + maybeEnqueueTask(new AddSplitsTask<>(splitReader, splitsToAdd, splitChanges, assignedSplits)); + wakeUp(true); + } + + /** + * Shutdown the split fetcher. + */ + public void shutdown() { + if (closed.compareAndSet(false, true)) { + LOG.info("Shutting down split fetcher {}", id); + wakeUp(false); + } + } + + /** + * Package private for unit test. + * @return the assigned splits. + */ + Map assignedSplits() { + return assignedSplits; + } + + /** + * Package private for unit test. + * @return true if task queue is not empty, false otherwise. + */ + boolean isIdle() { + return taskQueue.isEmpty() && assignedSplits.isEmpty(); + } + + /** + * Check whether the fetch task should run. The fetch task should only run when all + * the following conditions are met. + * 1. there is no task in the task queue to run. + * 2. there are assigned splits + * Package private for testing purpose. + * + * @return whether the fetch task should be run. + */ + boolean shouldRunFetchTask() { + return taskQueue.isEmpty() && !assignedSplits.isEmpty(); + } + + /** + * Wake up the fetcher thread. There are only two blocking points in a running fetcher. + * 1. Taking the next task out of the task queue. + * 2. Running a task. + * + *

    They need to be waken up differently. If the fetcher is blocking waiting on the + * next task in the task queue, we should just interrupt the fetcher thread. + * If the fetcher is running the user split reader, we should call SplitReader.wakeUp() + * instead of naively interrupt the thread. + * + *

    The correctness can be think of in the following way. The purpose of wake up + * is to let the fetcher thread go to the very beginning of the running loop. + * There are three major events in each run of the loop. + *

      + *
    1. pick a task (blocking) + *
    2. assign the task to runningTask variable. + *
    3. run the runningTask. (blocking) + *
    + * We don't need to worry about things after step 3 because there is no blocking point + * anymore. + * + *

    We always first set the wakeup flag when waking up the fetcher, then use the + * value of running task to determine where the fetcher thread is. + *

      + *
    • + * If runningThread is null, it is before step 2, so we should interrupt fetcher. + * This interruption will not be propagated to the split reader, because the + * wakeUp flag will prevent the fetchTask from running. + *
    • + *
    • + * If runningThread is not null, it is after step 2. so we should wakeUp the + * split reader instead of interrupt the fetcher. + *
    • + *
    + * + *

    The above logic only works in the same {@link #runOnce()} invocation. So we need to + * synchronize to ensure the wake up logic do not touch a different invocation. + */ + void wakeUp(boolean taskOnly) { + // Synchronize to make sure the wake up only work for the current invocation of runOnce(). + synchronized (wakeUp) { + // Do not wake up repeatedly. + if (wakeUp.compareAndSet(false, true)) { + // Now the wakeUp flag is set. + SplitFetcherTask currentTask = runningTask; + if (currentTask != null) { + // The running task may have missed our wakeUp flag and running, wake it up. + LOG.debug("Waking up running task {}", currentTask); + currentTask.wakeUp(); + } else if (!taskOnly && runningThread != null) { + // The task has not started running yet, and it will not run for this + // runOnce() invocation due to the wakeUp flag. But we might have to + // interrupt the fetcher thread in case it is blocking on the task queue. + LOG.debug("Interrupting fetcher thread."); + // Only interrupt when the thread has started and there is no running task. + runningThread.interrupt(); + } + } + } + } + + private void maybeEnqueueTask(SplitFetcherTask task) { + // Only enqueue unfinished non-fetch task. + if (!closed.get() && task != null && task != fetchTask && !taskQueue.offerFirst(task)) { + throw new RuntimeException( + "The task queue is full. This is only theoretically possible when really bad thing happens."); + } + LOG.debug("Enqueued task {}", task); + } +} diff --git a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/fetcher/SplitFetcherManager.java b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/fetcher/SplitFetcherManager.java new file mode 100644 index 0000000000..468b8997d9 --- /dev/null +++ b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/fetcher/SplitFetcherManager.java @@ -0,0 +1,145 @@ +/* + 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.connector.base.source.reader.fetcher; + +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.SourceReaderBase; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.apache.flink.connector.base.source.reader.synchronization.FutureNotifier; +import org.apache.flink.util.ThrowableCatchingRunnable; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; +import java.util.function.Supplier; + +/** + * A class responsible for starting the {@link SplitFetcher} and manage the life cycles of them. + * This class works with the {@link SourceReaderBase}. + * + *

    The split fetcher manager could be used to support different threading models by implementing + * the {@link #addSplits(List)} method differently. For example, a single thread split fetcher + * manager would only start a single fetcher and assign all the splits to it. A one-thread-per-split + * fetcher may spawn a new thread every time a new split is assigned. + */ +public abstract class SplitFetcherManager { + private static final Logger LOG = LoggerFactory.getLogger(SplitFetcherManager.class); + + private final Consumer errorHandler; + + /** An atomic integer to generate monotonically increasing fetcher ids. */ + private final AtomicInteger fetcherIdGenerator; + + /** A supplier to provide split readers. */ + private final Supplier> splitReaderFactory; + + /** Uncaught exception in the split fetchers.*/ + private final AtomicReference uncaughtFetcherException; + + /** The element queue that the split fetchers will put elements into. */ + private final BlockingQueue> elementsQueue; + + /** A map keeping track of all the split fetchers. */ + protected final Map> fetchers; + + /** An executor service with two threads. One for the fetcher and one for the future completing thread. */ + private final ExecutorService executors; + + /** + * Create a split fetcher manager. + * + * @param futureNotifier a notifier to notify the complete of a future. + * @param elementsQueue the queue that split readers will put elements into. + * @param splitReaderFactory a supplier that could be used to create split readers. + */ + public SplitFetcherManager( + FutureNotifier futureNotifier, + FutureCompletingBlockingQueue> elementsQueue, + Supplier> splitReaderFactory) { + this.elementsQueue = elementsQueue; + this.errorHandler = new Consumer() { + @Override + public void accept(Throwable t) { + LOG.error("Received uncaught exception.", t); + if (!uncaughtFetcherException.compareAndSet(null, t)) { + // Add the exception to the exception list. + uncaughtFetcherException.get().addSuppressed(t); + // Wake up the main thread to let it know the exception. + futureNotifier.notifyComplete(); + } + } + }; + this.splitReaderFactory = splitReaderFactory; + this.uncaughtFetcherException = new AtomicReference<>(null); + this.fetcherIdGenerator = new AtomicInteger(0); + this.fetchers = new HashMap<>(); + + // Create the executor with a thread factory that fails the source reader if one of + // the fetcher thread exits abnormally. + this.executors = Executors.newCachedThreadPool(r -> new Thread(r, "SourceFetcher")); + } + + public abstract void addSplits(List splitsToAdd); + + protected void startFetcher(SplitFetcher fetcher) { + executors.submit(new ThrowableCatchingRunnable(errorHandler, fetcher)); + } + + protected SplitFetcher createSplitFetcher() { + // Create SplitReader. + SplitReader splitReader = splitReaderFactory.get(); + + int fetcherId = fetcherIdGenerator.getAndIncrement(); + SplitFetcher splitFetcher = new SplitFetcher<>( + fetcherId, + elementsQueue, + splitReader, + () -> fetchers.remove(fetcherId)); + fetchers.put(fetcherId, splitFetcher); + return splitFetcher; + } + + public void close(long timeoutMs) throws Exception { + fetchers.values().forEach(SplitFetcher::shutdown); + executors.shutdown(); + if (!executors.awaitTermination(timeoutMs, TimeUnit.MILLISECONDS)) { + LOG.warn("Failed to close the source reader in {} ms. There are still {} split fetchers running", + timeoutMs, fetchers.size()); + } + } + + public void checkErrors() { + if (uncaughtFetcherException.get() != null) { + throw new RuntimeException("One or more fetchers have encountered exception", + uncaughtFetcherException.get()); + } + } +} diff --git a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/fetcher/SplitFetcherTask.java b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/fetcher/SplitFetcherTask.java new file mode 100644 index 0000000000..997cb65fb8 --- /dev/null +++ b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/fetcher/SplitFetcherTask.java @@ -0,0 +1,41 @@ +/* + 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.connector.base.source.reader.fetcher; + +/** + * An interface similar to {@link Runnable} but allows throwing exceptions and wakeup. + */ +public interface SplitFetcherTask { + + /** + * Run the logic. This method allows throwing an interrupted exception on wakeup, but the + * implementation does not have to. It is preferred to finish the work elegantly + * and return a boolean to indicate whether all the jobs have been done or more + * invocation is needed. + * + * @return whether the runnable has successfully finished running. + * @throws InterruptedException when interrupted. + */ + boolean run() throws InterruptedException; + + /** + * Wake up the running thread. + */ + void wakeUp(); +} diff --git a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/splitreader/SplitReader.java b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/splitreader/SplitReader.java new file mode 100644 index 0000000000..e8c1987f09 --- /dev/null +++ b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/splitreader/SplitReader.java @@ -0,0 +1,60 @@ +/* + 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.connector.base.source.reader.splitreader; + +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; + +import java.util.Queue; + +/** + * An interface used to read from splits. The implementation could either read from a single split or from + * multiple splits. + * + * @param the element type. + * @param the split type. + */ +public interface SplitReader { + + /** + * Fetch elements into the blocking queue for the given splits. The fetch call could be blocking + * but it should get unblocked when {@link #wakeUp()} is invoked. In that case, the implementation + * may either decide to return without throwing an exception, or it can just throw an interrupted + * exception. In either case, this method should be reentrant, meaning that the next fetch call + * should just resume from where the last fetch call was waken up or interrupted. + * + * @return the Ids of the finished splits. + * + * @throws InterruptedException when interrupted + */ + RecordsWithSplitIds fetch() throws InterruptedException; + + /** + * Handle the split changes. This call should be non-blocking. + * + * @param splitsChanges a queue with split changes that has not been handled by this SplitReader. + */ + void handleSplitsChanges(Queue> splitsChanges); + + /** + * Wake up the split reader in case the fetcher thread is blocking in + * {@link #fetch()}. + */ + void wakeUp(); +} diff --git a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/splitreader/SplitsAddition.java b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/splitreader/SplitsAddition.java new file mode 100644 index 0000000000..a6b6b4ec50 --- /dev/null +++ b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/splitreader/SplitsAddition.java @@ -0,0 +1,33 @@ +/* + 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.connector.base.source.reader.splitreader; + +import java.util.List; + +/** + * A change to add splits. + * + * @param the split type. + */ +public class SplitsAddition extends SplitsChange { + + public SplitsAddition(List splits) { + super(splits); + } +} diff --git a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/splitreader/SplitsChange.java b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/splitreader/SplitsChange.java new file mode 100644 index 0000000000..1af330bd38 --- /dev/null +++ b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/splitreader/SplitsChange.java @@ -0,0 +1,41 @@ +/* + 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.connector.base.source.reader.splitreader; + +import java.util.Collections; +import java.util.List; + +/** + * An abstract class to host splits change. + */ +public abstract class SplitsChange { + private final List splits; + + SplitsChange(List splits) { + this.splits = splits; + } + + /** + * @return the list of splits. + */ + public List splits() { + return Collections.unmodifiableList(splits); + } + +} diff --git a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/synchronization/FutureCompletingBlockingQueue.java b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/synchronization/FutureCompletingBlockingQueue.java new file mode 100644 index 0000000000..407f3050f8 --- /dev/null +++ b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/synchronization/FutureCompletingBlockingQueue.java @@ -0,0 +1,90 @@ +/* + 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.connector.base.source.reader.synchronization; + +import java.util.Collection; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; + +/** + * A BlockingQueue that allows a consuming thread to be notified asynchronously on element + * availability when the queue is empty. + * + *

    Implementation wise, it is a subclass of {@link LinkedBlockingQueue} that ensures all + * the methods adding elements into the queue will complete the elements availability future. + * + *

    The overriding methods must first put the elements into the queue then check and complete + * the future if needed. This is required to ensure the thread waiting for more messages will + * not lose a notification. + * + * @param the type of the elements in the queue. + */ +public class FutureCompletingBlockingQueue extends LinkedBlockingQueue { + private final FutureNotifier futureNotifier; + + public FutureCompletingBlockingQueue(FutureNotifier futureNotifier) { + this.futureNotifier = futureNotifier; + } + + @Override + public void put(T t) throws InterruptedException { + super.put(t); + futureNotifier.notifyComplete(); + } + + @Override + public boolean offer(T t, long timeout, TimeUnit unit) throws InterruptedException { + if (super.offer(t, timeout, unit)) { + futureNotifier.notifyComplete(); + return true; + } else { + return false; + } + } + + @Override + public boolean offer(T t) { + if (super.offer(t)) { + futureNotifier.notifyComplete(); + return true; + } else { + return false; + } + } + + @Override + public boolean add(T t) { + if (super.add(t)) { + futureNotifier.notifyComplete(); + return true; + } else { + return false; + } + } + + @Override + public boolean addAll(Collection c) { + if (super.addAll(c)) { + futureNotifier.notifyComplete(); + return true; + } else { + return false; + } + } +} diff --git a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/synchronization/FutureNotifier.java b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/synchronization/FutureNotifier.java new file mode 100644 index 0000000000..53e60f3129 --- /dev/null +++ b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/synchronization/FutureNotifier.java @@ -0,0 +1,66 @@ +/* + 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.connector.base.source.reader.synchronization; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicReference; + +/** + * A class facilitating the asynchronous communication among threads. + */ +public class FutureNotifier { + /** A future reference. */ + private final AtomicReference> futureRef; + + public FutureNotifier() { + this.futureRef = new AtomicReference<>(null); + } + + /** + * Get the future out of this notifier. The future will be completed when someone invokes + * {@link #notifyComplete()}. If there is already an uncompleted future, that existing + * future will be returned instead of a new one. + * + * @return a future that will be completed when {@link #notifyComplete()} is invoked. + */ + public CompletableFuture future() { + CompletableFuture prevFuture = futureRef.get(); + if (prevFuture != null) { + // Someone has created a future for us, don't create a new one. + return prevFuture; + } else { + CompletableFuture newFuture = new CompletableFuture<>(); + boolean newFutureSet = futureRef.compareAndSet(null, newFuture); + // If someone created a future after our previous check, use that future. + // Otherwise, use the new future. + return newFutureSet ? newFuture : future(); + } + } + + /** + * Complete the future if there is one. This will release the thread that is waiting for data. + */ + public void notifyComplete() { + CompletableFuture future = futureRef.get(); + // If there are multiple threads trying to complete the future, only the first one succeeds. + if (future != null && future.complete(null)) { + futureRef.compareAndSet(future, null); + } + } +} diff --git a/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/SourceReaderBaseTest.java b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/SourceReaderBaseTest.java new file mode 100644 index 0000000000..57f1b7cb7f --- /dev/null +++ b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/SourceReaderBaseTest.java @@ -0,0 +1,140 @@ +/* + 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.connector.base.source.reader; + +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.mocks.MockSourceSplit; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.base.source.reader.mocks.MockSourceReader; +import org.apache.flink.connector.base.source.reader.mocks.MockSplitReader; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.apache.flink.connector.base.source.reader.synchronization.FutureNotifier; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Queue; + +/** + * A unit test class for {@link SourceReaderBase}. + */ +public class SourceReaderBaseTest extends SourceReaderTestBase { + + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + @Test + public void testExceptionInSplitReader() throws Exception { + expectedException.expect(RuntimeException.class); + expectedException.expectMessage("One or more fetchers have encountered exception"); + final String errMsg = "Testing Exception"; + + FutureNotifier futureNotifier = new FutureNotifier(); + FutureCompletingBlockingQueue> elementsQueue = + new FutureCompletingBlockingQueue<>(futureNotifier); + MockSourceReader reader = new MockSourceReader( + futureNotifier, + elementsQueue, + () -> new SplitReader() { + @Override + public RecordsWithSplitIds fetch() { + throw new RuntimeException(errMsg); + } + + @Override + public void handleSplitsChanges(Queue> splitsChanges) { + // We have to handle split changes first, otherwise fetch will not be called. + splitsChanges.clear(); + } + + @Override + public void wakeUp() {} + }, + getConfig(Boundedness.BOUNDED), + null); + + ValidatingSourceOutput output = new ValidatingSourceOutput(); + reader.addSplits(Collections.singletonList(getSplit(0, NUM_RECORDS_PER_SPLIT, Boundedness.CONTINUOUS_UNBOUNDED))); + // This is not a real infinite loop, it is supposed to throw exception after two polls. + while (true) { + reader.pollNext(output); + // Add a sleep to avoid tight loop. + Thread.sleep(1); + } + } + + // ---------------- helper methods ----------------- + + @Override + protected MockSourceReader createReader(Boundedness boundedness) { + FutureNotifier futureNotifier = new FutureNotifier(); + FutureCompletingBlockingQueue> elementsQueue = + new FutureCompletingBlockingQueue<>(futureNotifier); + MockSplitReader mockSplitReader = + new MockSplitReader(2, true, true); + return new MockSourceReader( + futureNotifier, + elementsQueue, + () -> mockSplitReader, + getConfig(boundedness), + null); + } + + @Override + protected List getSplits(int numSplits, int numRecordsPerSplit, Boundedness boundedness) { + List mockSplits = new ArrayList<>(); + for (int i = 0; i < numSplits; i++) { + mockSplits.add(getSplit(i, numRecordsPerSplit, boundedness)); + } + return mockSplits; + } + + @Override + protected MockSourceSplit getSplit(int splitId, int numRecords, Boundedness boundedness) { + MockSourceSplit mockSplit; + if (boundedness == Boundedness.BOUNDED) { + mockSplit = new MockSourceSplit(splitId, 0, numRecords); + } else { + mockSplit = new MockSourceSplit(splitId); + } + for (int j = 0; j < numRecords; j++) { + mockSplit.addRecord(splitId * 10 + j); + } + return mockSplit; + } + + @Override + protected long getIndex(MockSourceSplit split) { + return split.index(); + } + + private Configuration getConfig(Boundedness boundedness) { + Configuration config = new Configuration(); + config.setInteger(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY, 1); + config.setLong(SourceReaderOptions.SOURCE_READER_CLOSE_TIMEOUT, 30000L); + config.setString(SourceReaderOptions.SOURCE_READER_BOUNDEDNESS, boundedness.name()); + return config; + } +} diff --git a/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/SourceReaderTestBase.java b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/SourceReaderTestBase.java new file mode 100644 index 0000000000..157bf76128 --- /dev/null +++ b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/SourceReaderTestBase.java @@ -0,0 +1,200 @@ +/* + 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.connector.base.source.reader; + +import org.apache.flink.api.common.eventtime.Watermark; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.SourceOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.util.TestLogger; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.CompletableFuture; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; + +/** + * An abstract test class for all the unit tests of {@link SourceReader} to inherit. + * + * @param the type of the splits. + */ +public abstract class SourceReaderTestBase extends TestLogger { + + protected static final int NUM_SPLITS = 10; + protected static final int NUM_RECORDS_PER_SPLIT = 10; + + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + /** + * Simply test the reader reads all the splits fine. + */ + @Test + public void testRead() throws Exception { + SourceReader reader = createReader(Boundedness.BOUNDED); + reader.addSplits(getSplits(NUM_SPLITS, NUM_RECORDS_PER_SPLIT, Boundedness.BOUNDED)); + ValidatingSourceOutput output = new ValidatingSourceOutput(); + while (output.count < 100) { + reader.pollNext(output); + } + output.validate(); + } + + @Test + public void testAddSplitToExistingFetcher() throws Exception { + ValidatingSourceOutput output = new ValidatingSourceOutput(); + // Add a split to start the fetcher. + List splits = Collections.singletonList(getSplit(0, NUM_RECORDS_PER_SPLIT, Boundedness.BOUNDED)); + // Poll 5 records and let it block on the element queue which only have capacity of 1; + SourceReader reader = consumeRecords(splits, output, 5, Boundedness.BOUNDED); + List newSplits = new ArrayList<>(); + for (int i = 1; i < NUM_SPLITS; i++) { + newSplits.add(getSplit(i, NUM_RECORDS_PER_SPLIT, Boundedness.BOUNDED)); + } + reader.addSplits(newSplits); + + while (output.count() < 100) { + reader.pollNext(output); + } + output.validate(); + } + + @Test (timeout = 30000L) + public void testPollingFromEmptyQueue() throws Exception { + ValidatingSourceOutput output = new ValidatingSourceOutput(); + List splits = Collections.singletonList(getSplit(0, NUM_RECORDS_PER_SPLIT, Boundedness.BOUNDED)); + // Consumer all the records in the s;oit. + SourceReader reader = consumeRecords(splits, output, NUM_RECORDS_PER_SPLIT, Boundedness.BOUNDED); + // Now let the main thread poll again. + assertEquals("The status should be ", SourceReader.Status.AVAILABLE_LATER, reader.pollNext(output)); + } + + @Test (timeout = 30000L) + public void testAvailableOnEmptyQueue() throws Exception { + ValidatingSourceOutput output = new ValidatingSourceOutput(); + List splits = Collections.singletonList(getSplit(0, NUM_RECORDS_PER_SPLIT, Boundedness.BOUNDED)); + // Consumer all the records in the split. + SourceReader reader = consumeRecords(splits, output, NUM_RECORDS_PER_SPLIT, Boundedness.BOUNDED); + + CompletableFuture future = reader.isAvailable(); + assertFalse("There should be no records ready for poll.", future.isDone()); + // Add a split to the reader so there are more records to be read. + reader.addSplits(Collections.singletonList(getSplit(1, NUM_RECORDS_PER_SPLIT, Boundedness.BOUNDED))); + // THe future should be completed fairly soon. Otherwise the test will hit timeout and fail. + future.get(); + } + + @Test (timeout = 30000L) + public void testSnapshot() throws Exception { + ValidatingSourceOutput output = new ValidatingSourceOutput(); + // Add a split to start the fetcher. + List splits = getSplits(NUM_SPLITS, NUM_RECORDS_PER_SPLIT, Boundedness.BOUNDED); + // Poll 5 records. That means split 0 and 1 will at index 2, split 1 will at index 1. + SourceReader reader = + consumeRecords(splits, output, NUM_SPLITS * NUM_RECORDS_PER_SPLIT, Boundedness.BOUNDED); + + List state = reader.snapshotState(); + assertEquals("The snapshot should only have 10 splits. ", NUM_SPLITS, state.size()); + for (int i = 0; i < NUM_SPLITS; i++) { + assertEquals("The first four splits should have been fully consumed.", NUM_RECORDS_PER_SPLIT, getIndex(state.get(i))); + } + } + + // ---------------- helper methods ----------------- + + protected abstract SourceReader createReader(Boundedness boundedness); + + protected abstract List getSplits(int numSplits, int numRecordsPerSplit, Boundedness boundedness); + + protected abstract SplitT getSplit(int splitId, int numRecords, Boundedness boundedness); + + protected abstract long getIndex(SplitT split); + + private SourceReader consumeRecords( + List splits, + ValidatingSourceOutput output, + int n, + Boundedness boundedness) throws Exception { + SourceReader reader = createReader(boundedness); + // Add splits to start the fetcher. + reader.addSplits(splits); + // Poll all the n records of the single split. + while (output.count() < n) { + reader.pollNext(output); + } + return reader; + } + + // ---------------- helper classes ----------------- + + /** + * A source output that validates the output. + */ + protected static class ValidatingSourceOutput implements SourceOutput { + private Set consumedValues = new HashSet<>(); + private int max = Integer.MIN_VALUE; + private int min = Integer.MAX_VALUE; + + private int count = 0; + + @Override + public void collect(Integer element) { + max = Math.max(element, max); + min = Math.min(element, min); + count++; + consumedValues.add(element); + } + + @Override + public void collect(Integer element, long timestamp) { + collect(element); + } + + public void validate() { + assertEquals("Should be 100 distinct elements in total", 100, consumedValues.size()); + assertEquals("Should be 100 elements in total", 100, count); + assertEquals("The min value should be 0", 0, min); + assertEquals("The max value should be 99", 99, max); + } + + public int count() { + return count; + } + + @Override + public void emitWatermark(Watermark watermark) { + + } + + @Override + public void markIdle() { + + } + } +} diff --git a/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/mocks/MockRecordEmitter.java b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/mocks/MockRecordEmitter.java new file mode 100644 index 0000000000..66fa3bccd8 --- /dev/null +++ b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/mocks/MockRecordEmitter.java @@ -0,0 +1,37 @@ +/* + 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.connector.base.source.reader.mocks; + +import org.apache.flink.api.connector.source.SourceOutput; +import org.apache.flink.connector.base.source.reader.RecordEmitter; + +import java.util.concurrent.atomic.AtomicInteger; + +/** + * A mock {@link RecordEmitter} that works with the {@link MockSplitReader} and {@link MockSourceReader}. + */ +public class MockRecordEmitter implements RecordEmitter { + @Override + public void emitRecord(int[] record, SourceOutput output, AtomicInteger splitState) throws Exception { + // The value is the first element. + output.collect(record[0]); + // The state will be next index. + splitState.set(record[1] + 1); + } +} diff --git a/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/mocks/MockSourceReader.java b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/mocks/MockSourceReader.java new file mode 100644 index 0000000000..aada92c05d --- /dev/null +++ b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/mocks/MockSourceReader.java @@ -0,0 +1,62 @@ +/* + 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.connector.base.source.reader.mocks; + +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.mocks.MockSourceSplit; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.apache.flink.connector.base.source.reader.synchronization.FutureNotifier; + +import java.util.Collection; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; + +/** + * A mock SourceReader class. + */ +public class MockSourceReader + extends SingleThreadMultiplexSourceReaderBase { + + public MockSourceReader(FutureNotifier futureNotifier, + FutureCompletingBlockingQueue> elementsQueue, + Supplier> splitFetcherSupplier, + Configuration config, + SourceReaderContext context) { + super(futureNotifier, elementsQueue, splitFetcherSupplier, new MockRecordEmitter(), config, context); + } + + @Override + protected void onSplitFinished(Collection finishedSplitIds) { + + } + + @Override + protected AtomicInteger initializedState(MockSourceSplit split) { + return new AtomicInteger(split.index()); + } + + @Override + protected MockSourceSplit toSplitType(String splitId, AtomicInteger splitState) { + return new MockSourceSplit(Integer.parseInt(splitId), splitState.get()); + } +} diff --git a/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/mocks/MockSplitReader.java b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/mocks/MockSplitReader.java new file mode 100644 index 0000000000..99f915f644 --- /dev/null +++ b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/mocks/MockSplitReader.java @@ -0,0 +1,107 @@ +/* + 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.connector.base.source.reader.mocks; + +import org.apache.flink.api.connector.source.mocks.MockSourceSplit; +import org.apache.flink.connector.base.source.reader.RecordsBySplits; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; + +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Queue; + +/** + * A mock split reader for unit tests. The mock split reader provides configurable behaviours. + * 1. Blocking fetch or non blocking fetch. + * - A blocking fetch can only be waken up by an interruption. + * - A non-blocking fetch do not expect to be interrupted. + * 2. handle splits changes in one handleSplitsChanges call or handle one change in each call + * of handleSplitsChanges. + */ +public class MockSplitReader implements SplitReader { + // Use LinkedHashMap for determinism. + private final Map splits = new LinkedHashMap<>(); + private final int numRecordsPerSplitPerFetch; + private final boolean blockingFetch; + private final boolean handleSplitsInOneShot; + private volatile Thread runningThread; + + public MockSplitReader( + int numRecordsPerSplitPerFetch, + boolean blockingFetch, + boolean handleSplitsInOneShot) { + this.numRecordsPerSplitPerFetch = numRecordsPerSplitPerFetch; + this.blockingFetch = blockingFetch; + this.handleSplitsInOneShot = handleSplitsInOneShot; + this.runningThread = null; + } + + @Override + public RecordsWithSplitIds fetch() throws InterruptedException { + try { + if (runningThread == null) { + runningThread = Thread.currentThread(); + } + return getRecords(); + } catch (InterruptedException ie) { + if (!blockingFetch) { + throw new RuntimeException("Caught unexpected interrupted exception."); + } else { + throw ie; + } + } + } + + @Override + public void handleSplitsChanges(Queue> splitsChanges) { + do { + SplitsChange splitsChange = splitsChanges.poll(); + if (splitsChange instanceof SplitsAddition) { + splitsChange.splits().forEach(s -> splits.put(s.splitId(), s)); + } + } while (handleSplitsInOneShot && !splitsChanges.isEmpty()); + } + + @Override + public void wakeUp() { + if (blockingFetch) { + runningThread.interrupt(); + } + } + + private RecordsBySplits getRecords() throws InterruptedException { + RecordsBySplits records = new RecordsBySplits<>(); + for (Map.Entry entry : splits.entrySet()) { + MockSourceSplit split = entry.getValue(); + for (int i = 0; i < numRecordsPerSplitPerFetch && !split.isFinished(); i++) { + int[] record = split.getNext(blockingFetch); + if (record != null) { + records.add(entry.getKey(), record); + if (split.isFinished()) { + records.addFinishedSplit(entry.getKey()); + } + } + } + } + return records; + } +} diff --git a/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/synchronization/FutureNotifierTest.java b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/synchronization/FutureNotifierTest.java new file mode 100644 index 0000000000..b7752546e8 --- /dev/null +++ b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/synchronization/FutureNotifierTest.java @@ -0,0 +1,131 @@ +/* + 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.connector.base.source.reader.synchronization; + +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** + * The unit tests for {@link FutureNotifier}. + */ +public class FutureNotifierTest { + + @Test + public void testGetFuture() { + FutureNotifier notifier = new FutureNotifier(); + CompletableFuture future = notifier.future(); + // The future should not be null. + assertNotNull(future); + // Calling the future again should return the same future. + assertEquals(future, notifier.future()); + } + + @Test + public void testCompleteFuture() { + FutureNotifier notifier = new FutureNotifier(); + CompletableFuture future = notifier.future(); + assertFalse(future.isDone()); + notifier.notifyComplete(); + assertTrue(future.isDone()); + } + + @Test + public void testConcurrency() throws InterruptedException, ExecutionException { + final int times = 1_000_000; + final int nThreads = 5; + FutureNotifier notifier = new FutureNotifier(); + // A thread pool that simply gets futures out of the notifier. + ExecutorService listenerExecutor = Executors.newFixedThreadPool(nThreads); + // A thread pool that completes the futures. + ExecutorService notifierExecutor = Executors.newFixedThreadPool(nThreads); + + CountDownLatch runningListeners = new CountDownLatch(nThreads); + CountDownLatch startCommand = new CountDownLatch(1); + CountDownLatch finishLine = new CountDownLatch(1); + + List> executionFutures = new ArrayList<>(); + // Start nThreads thread getting futures out of the notifier. + for (int i = 0; i < nThreads; i++) { + executionFutures.add(listenerExecutor.submit(() -> { + try { + List> futures = new ArrayList<>(times); + startCommand.await(); + for (int j = 0; j < times; j++) { + futures.add(notifier.future()); + } + runningListeners.countDown(); + // Wait for the notifying thread to finish. + finishLine.await(); + // All the futures should have been completed. + futures.forEach(f -> { + assertNotNull(f); + assertTrue(f.isDone()); + }); + } catch (Exception e) { + fail(); + } + })); + } + + // Start nThreads thread notifying the completion. + for (int i = 0; i < nThreads; i++) { + notifierExecutor.submit(() -> { + try { + startCommand.await(); + while (runningListeners.getCount() > 0) { + notifier.notifyComplete(); + } + notifier.notifyComplete(); + finishLine.countDown(); + } catch (Exception e) { + fail(); + } + }); + } + + // Kick off the threads. + startCommand.countDown(); + + try { + for (Future executionFuture : executionFutures) { + executionFuture.get(); + } + } finally { + listenerExecutor.shutdown(); + notifierExecutor.shutdown(); + listenerExecutor.awaitTermination(30L, TimeUnit.SECONDS); + notifierExecutor.awaitTermination(30L, TimeUnit.SECONDS); + } + } +} diff --git a/flink-connectors/pom.xml b/flink-connectors/pom.xml index 296bf899e4..ff3e1a1d68 100644 --- a/flink-connectors/pom.xml +++ b/flink-connectors/pom.xml @@ -57,6 +57,7 @@ under the License. flink-connector-gcp-pubsub flink-connector-kinesis flink-sql-connector-elasticsearch7 + flink-connector-base @@ -196,6 +201,12 @@ under the License. ${beam.version} + + org.apache.beam + beam-runners-core-java + ${beam.version} + + com.google.protobuf protobuf-java diff --git a/flink-python/pyflink/fn_execution/flink_fn_execution_pb2.py b/flink-python/pyflink/fn_execution/flink_fn_execution_pb2.py index 18d4475001..84be4f83a2 100644 --- a/flink-python/pyflink/fn_execution/flink_fn_execution_pb2.py +++ b/flink-python/pyflink/fn_execution/flink_fn_execution_pb2.py @@ -36,7 +36,7 @@ DESCRIPTOR = _descriptor.FileDescriptor( name='flink-fn-execution.proto', package='org.apache.flink.fn_execution.v1', syntax='proto3', - serialized_pb=_b('\n\x18\x66link-fn-execution.proto\x12 org.apache.flink.fn_execution.v1\"\xfc\x01\n\x13UserDefinedFunction\x12\x0f\n\x07payload\x18\x01 \x01(\x0c\x12K\n\x06inputs\x18\x02 \x03(\x0b\x32;.org.apache.flink.fn_execution.v1.UserDefinedFunction.Input\x1a\x86\x01\n\x05Input\x12\x44\n\x03udf\x18\x01 \x01(\x0b\x32\x35.org.apache.flink.fn_execution.v1.UserDefinedFunctionH\x00\x12\x15\n\x0binputOffset\x18\x02 \x01(\x05H\x00\x12\x17\n\rinputConstant\x18\x03 \x01(\x0cH\x00\x42\x07\n\x05input\"[\n\x14UserDefinedFunctions\x12\x43\n\x04udfs\x18\x01 \x03(\x0b\x32\x35.org.apache.flink.fn_execution.v1.UserDefinedFunction\"\xe6\x0f\n\x06Schema\x12>\n\x06\x66ields\x18\x01 \x03(\x0b\x32..org.apache.flink.fn_execution.v1.Schema.Field\x1a\x97\x01\n\x07MapInfo\x12\x44\n\x08key_type\x18\x01 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x12\x46\n\nvalue_type\x18\x02 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x1a\x1d\n\x08TimeInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a\"\n\rTimestampInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a,\n\x17LocalZonedTimestampInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a\'\n\x12ZonedTimestampInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a/\n\x0b\x44\x65\x63imalInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x12\r\n\x05scale\x18\x02 \x01(\x05\x1a\x1c\n\nBinaryInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\x1f\n\rVarBinaryInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\x1a\n\x08\x43harInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\x1d\n\x0bVarCharInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\xb0\x08\n\tFieldType\x12\x44\n\ttype_name\x18\x01 \x01(\x0e\x32\x31.org.apache.flink.fn_execution.v1.Schema.TypeName\x12\x10\n\x08nullable\x18\x02 \x01(\x08\x12U\n\x17\x63ollection_element_type\x18\x03 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldTypeH\x00\x12\x44\n\x08map_info\x18\x04 \x01(\x0b\x32\x30.org.apache.flink.fn_execution.v1.Schema.MapInfoH\x00\x12>\n\nrow_schema\x18\x05 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.SchemaH\x00\x12L\n\x0c\x64\x65\x63imal_info\x18\x06 \x01(\x0b\x32\x34.org.apache.flink.fn_execution.v1.Schema.DecimalInfoH\x00\x12\x46\n\ttime_info\x18\x07 \x01(\x0b\x32\x31.org.apache.flink.fn_execution.v1.Schema.TimeInfoH\x00\x12P\n\x0etimestamp_info\x18\x08 \x01(\x0b\x32\x36.org.apache.flink.fn_execution.v1.Schema.TimestampInfoH\x00\x12\x66\n\x1alocal_zoned_timestamp_info\x18\t \x01(\x0b\x32@.org.apache.flink.fn_execution.v1.Schema.LocalZonedTimestampInfoH\x00\x12[\n\x14zoned_timestamp_info\x18\n \x01(\x0b\x32;.org.apache.flink.fn_execution.v1.Schema.ZonedTimestampInfoH\x00\x12J\n\x0b\x62inary_info\x18\x0b \x01(\x0b\x32\x33.org.apache.flink.fn_execution.v1.Schema.BinaryInfoH\x00\x12Q\n\x0fvar_binary_info\x18\x0c \x01(\x0b\x32\x36.org.apache.flink.fn_execution.v1.Schema.VarBinaryInfoH\x00\x12\x46\n\tchar_info\x18\r \x01(\x0b\x32\x31.org.apache.flink.fn_execution.v1.Schema.CharInfoH\x00\x12M\n\rvar_char_info\x18\x0e \x01(\x0b\x32\x34.org.apache.flink.fn_execution.v1.Schema.VarCharInfoH\x00\x42\x0b\n\ttype_info\x1al\n\x05\x46ield\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x13\n\x0b\x64\x65scription\x18\x02 \x01(\t\x12@\n\x04type\x18\x03 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\"\x9b\x02\n\x08TypeName\x12\x07\n\x03ROW\x10\x00\x12\x0b\n\x07TINYINT\x10\x01\x12\x0c\n\x08SMALLINT\x10\x02\x12\x07\n\x03INT\x10\x03\x12\n\n\x06\x42IGINT\x10\x04\x12\x0b\n\x07\x44\x45\x43IMAL\x10\x05\x12\t\n\x05\x46LOAT\x10\x06\x12\n\n\x06\x44OUBLE\x10\x07\x12\x08\n\x04\x44\x41TE\x10\x08\x12\x08\n\x04TIME\x10\t\x12\r\n\tTIMESTAMP\x10\n\x12\x0b\n\x07\x42OOLEAN\x10\x0b\x12\n\n\x06\x42INARY\x10\x0c\x12\r\n\tVARBINARY\x10\r\x12\x08\n\x04\x43HAR\x10\x0e\x12\x0b\n\x07VARCHAR\x10\x0f\x12\t\n\x05\x41RRAY\x10\x10\x12\x07\n\x03MAP\x10\x11\x12\x0c\n\x08MULTISET\x10\x12\x12\x19\n\x15LOCAL_ZONED_TIMESTAMP\x10\x13\x12\x13\n\x0fZONED_TIMESTAMP\x10\x14\x42-\n\x1forg.apache.flink.fnexecution.v1B\nFlinkFnApib\x06proto3') + serialized_pb=_b('\n\x18\x66link-fn-execution.proto\x12 org.apache.flink.fn_execution.v1\"\xfc\x01\n\x13UserDefinedFunction\x12\x0f\n\x07payload\x18\x01 \x01(\x0c\x12K\n\x06inputs\x18\x02 \x03(\x0b\x32;.org.apache.flink.fn_execution.v1.UserDefinedFunction.Input\x1a\x86\x01\n\x05Input\x12\x44\n\x03udf\x18\x01 \x01(\x0b\x32\x35.org.apache.flink.fn_execution.v1.UserDefinedFunctionH\x00\x12\x15\n\x0binputOffset\x18\x02 \x01(\x05H\x00\x12\x17\n\rinputConstant\x18\x03 \x01(\x0cH\x00\x42\x07\n\x05input\"s\n\x14UserDefinedFunctions\x12\x43\n\x04udfs\x18\x01 \x03(\x0b\x32\x35.org.apache.flink.fn_execution.v1.UserDefinedFunction\x12\x16\n\x0emetric_enabled\x18\x02 \x01(\x08\"\xe6\x0f\n\x06Schema\x12>\n\x06\x66ields\x18\x01 \x03(\x0b\x32..org.apache.flink.fn_execution.v1.Schema.Field\x1a\x97\x01\n\x07MapInfo\x12\x44\n\x08key_type\x18\x01 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x12\x46\n\nvalue_type\x18\x02 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x1a\x1d\n\x08TimeInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a\"\n\rTimestampInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a,\n\x17LocalZonedTimestampInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a\'\n\x12ZonedTimestampInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a/\n\x0b\x44\x65\x63imalInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x12\r\n\x05scale\x18\x02 \x01(\x05\x1a\x1c\n\nBinaryInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\x1f\n\rVarBinaryInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\x1a\n\x08\x43harInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\x1d\n\x0bVarCharInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\xb0\x08\n\tFieldType\x12\x44\n\ttype_name\x18\x01 \x01(\x0e\x32\x31.org.apache.flink.fn_execution.v1.Schema.TypeName\x12\x10\n\x08nullable\x18\x02 \x01(\x08\x12U\n\x17\x63ollection_element_type\x18\x03 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldTypeH\x00\x12\x44\n\x08map_info\x18\x04 \x01(\x0b\x32\x30.org.apache.flink.fn_execution.v1.Schema.MapInfoH\x00\x12>\n\nrow_schema\x18\x05 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.SchemaH\x00\x12L\n\x0c\x64\x65\x63imal_info\x18\x06 \x01(\x0b\x32\x34.org.apache.flink.fn_execution.v1.Schema.DecimalInfoH\x00\x12\x46\n\ttime_info\x18\x07 \x01(\x0b\x32\x31.org.apache.flink.fn_execution.v1.Schema.TimeInfoH\x00\x12P\n\x0etimestamp_info\x18\x08 \x01(\x0b\x32\x36.org.apache.flink.fn_execution.v1.Schema.TimestampInfoH\x00\x12\x66\n\x1alocal_zoned_timestamp_info\x18\t \x01(\x0b\x32@.org.apache.flink.fn_execution.v1.Schema.LocalZonedTimestampInfoH\x00\x12[\n\x14zoned_timestamp_info\x18\n \x01(\x0b\x32;.org.apache.flink.fn_execution.v1.Schema.ZonedTimestampInfoH\x00\x12J\n\x0b\x62inary_info\x18\x0b \x01(\x0b\x32\x33.org.apache.flink.fn_execution.v1.Schema.BinaryInfoH\x00\x12Q\n\x0fvar_binary_info\x18\x0c \x01(\x0b\x32\x36.org.apache.flink.fn_execution.v1.Schema.VarBinaryInfoH\x00\x12\x46\n\tchar_info\x18\r \x01(\x0b\x32\x31.org.apache.flink.fn_execution.v1.Schema.CharInfoH\x00\x12M\n\rvar_char_info\x18\x0e \x01(\x0b\x32\x34.org.apache.flink.fn_execution.v1.Schema.VarCharInfoH\x00\x42\x0b\n\ttype_info\x1al\n\x05\x46ield\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x13\n\x0b\x64\x65scription\x18\x02 \x01(\t\x12@\n\x04type\x18\x03 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\"\x9b\x02\n\x08TypeName\x12\x07\n\x03ROW\x10\x00\x12\x0b\n\x07TINYINT\x10\x01\x12\x0c\n\x08SMALLINT\x10\x02\x12\x07\n\x03INT\x10\x03\x12\n\n\x06\x42IGINT\x10\x04\x12\x0b\n\x07\x44\x45\x43IMAL\x10\x05\x12\t\n\x05\x46LOAT\x10\x06\x12\n\n\x06\x44OUBLE\x10\x07\x12\x08\n\x04\x44\x41TE\x10\x08\x12\x08\n\x04TIME\x10\t\x12\r\n\tTIMESTAMP\x10\n\x12\x0b\n\x07\x42OOLEAN\x10\x0b\x12\n\n\x06\x42INARY\x10\x0c\x12\r\n\tVARBINARY\x10\r\x12\x08\n\x04\x43HAR\x10\x0e\x12\x0b\n\x07VARCHAR\x10\x0f\x12\t\n\x05\x41RRAY\x10\x10\x12\x07\n\x03MAP\x10\x11\x12\x0c\n\x08MULTISET\x10\x12\x12\x19\n\x15LOCAL_ZONED_TIMESTAMP\x10\x13\x12\x13\n\x0fZONED_TIMESTAMP\x10\x14\x42-\n\x1forg.apache.flink.fnexecution.v1B\nFlinkFnApib\x06proto3') ) @@ -134,8 +134,8 @@ _SCHEMA_TYPENAME = _descriptor.EnumDescriptor( ], containing_type=None, options=None, - serialized_start=2150, - serialized_end=2433, + serialized_start=2174, + serialized_end=2457, ) _sym_db.RegisterEnumDescriptor(_SCHEMA_TYPENAME) @@ -239,6 +239,13 @@ _USERDEFINEDFUNCTIONS = _descriptor.Descriptor( message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='metric_enabled', full_name='org.apache.flink.fn_execution.v1.UserDefinedFunctions.metric_enabled', index=1, + number=2, type=8, cpp_type=7, label=1, + has_default_value=False, default_value=False, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), ], extensions=[ ], @@ -252,7 +259,7 @@ _USERDEFINEDFUNCTIONS = _descriptor.Descriptor( oneofs=[ ], serialized_start=317, - serialized_end=408, + serialized_end=432, ) @@ -289,8 +296,8 @@ _SCHEMA_MAPINFO = _descriptor.Descriptor( extension_ranges=[], oneofs=[ ], - serialized_start=486, - serialized_end=637, + serialized_start=510, + serialized_end=661, ) _SCHEMA_TIMEINFO = _descriptor.Descriptor( @@ -319,8 +326,8 @@ _SCHEMA_TIMEINFO = _descriptor.Descriptor( extension_ranges=[], oneofs=[ ], - serialized_start=639, - serialized_end=668, + serialized_start=663, + serialized_end=692, ) _SCHEMA_TIMESTAMPINFO = _descriptor.Descriptor( @@ -349,8 +356,8 @@ _SCHEMA_TIMESTAMPINFO = _descriptor.Descriptor( extension_ranges=[], oneofs=[ ], - serialized_start=670, - serialized_end=704, + serialized_start=694, + serialized_end=728, ) _SCHEMA_LOCALZONEDTIMESTAMPINFO = _descriptor.Descriptor( @@ -379,8 +386,8 @@ _SCHEMA_LOCALZONEDTIMESTAMPINFO = _descriptor.Descriptor( extension_ranges=[], oneofs=[ ], - serialized_start=706, - serialized_end=750, + serialized_start=730, + serialized_end=774, ) _SCHEMA_ZONEDTIMESTAMPINFO = _descriptor.Descriptor( @@ -409,8 +416,8 @@ _SCHEMA_ZONEDTIMESTAMPINFO = _descriptor.Descriptor( extension_ranges=[], oneofs=[ ], - serialized_start=752, - serialized_end=791, + serialized_start=776, + serialized_end=815, ) _SCHEMA_DECIMALINFO = _descriptor.Descriptor( @@ -446,8 +453,8 @@ _SCHEMA_DECIMALINFO = _descriptor.Descriptor( extension_ranges=[], oneofs=[ ], - serialized_start=793, - serialized_end=840, + serialized_start=817, + serialized_end=864, ) _SCHEMA_BINARYINFO = _descriptor.Descriptor( @@ -476,8 +483,8 @@ _SCHEMA_BINARYINFO = _descriptor.Descriptor( extension_ranges=[], oneofs=[ ], - serialized_start=842, - serialized_end=870, + serialized_start=866, + serialized_end=894, ) _SCHEMA_VARBINARYINFO = _descriptor.Descriptor( @@ -506,8 +513,8 @@ _SCHEMA_VARBINARYINFO = _descriptor.Descriptor( extension_ranges=[], oneofs=[ ], - serialized_start=872, - serialized_end=903, + serialized_start=896, + serialized_end=927, ) _SCHEMA_CHARINFO = _descriptor.Descriptor( @@ -536,8 +543,8 @@ _SCHEMA_CHARINFO = _descriptor.Descriptor( extension_ranges=[], oneofs=[ ], - serialized_start=905, - serialized_end=931, + serialized_start=929, + serialized_end=955, ) _SCHEMA_VARCHARINFO = _descriptor.Descriptor( @@ -566,8 +573,8 @@ _SCHEMA_VARCHARINFO = _descriptor.Descriptor( extension_ranges=[], oneofs=[ ], - serialized_start=933, - serialized_end=962, + serialized_start=957, + serialized_end=986, ) _SCHEMA_FIELDTYPE = _descriptor.Descriptor( @@ -690,8 +697,8 @@ _SCHEMA_FIELDTYPE = _descriptor.Descriptor( name='type_info', full_name='org.apache.flink.fn_execution.v1.Schema.FieldType.type_info', index=0, containing_type=None, fields=[]), ], - serialized_start=965, - serialized_end=2037, + serialized_start=989, + serialized_end=2061, ) _SCHEMA_FIELD = _descriptor.Descriptor( @@ -734,8 +741,8 @@ _SCHEMA_FIELD = _descriptor.Descriptor( extension_ranges=[], oneofs=[ ], - serialized_start=2039, - serialized_end=2147, + serialized_start=2063, + serialized_end=2171, ) _SCHEMA = _descriptor.Descriptor( @@ -765,8 +772,8 @@ _SCHEMA = _descriptor.Descriptor( extension_ranges=[], oneofs=[ ], - serialized_start=411, - serialized_end=2433, + serialized_start=435, + serialized_end=2457, ) _USERDEFINEDFUNCTION_INPUT.fields_by_name['udf'].message_type = _USERDEFINEDFUNCTION diff --git a/flink-python/pyflink/fn_execution/operations.py b/flink-python/pyflink/fn_execution/operations.py index 24ffabbc15..976d4e39b5 100644 --- a/flink-python/pyflink/fn_execution/operations.py +++ b/flink-python/pyflink/fn_execution/operations.py @@ -46,10 +46,13 @@ class StatelessFunctionOperation(Operation): self.variable_dict = {} self.user_defined_funcs = [] - self.func = self.generate_func(self.spec.serialized_fn) - base_metric_group = GenericMetricGroup(None, None) + self.func = self.generate_func(self.spec.serialized_fn.udfs) + self._metric_enabled = self.spec.serialized_fn.metric_enabled + self.base_metric_group = None + if self._metric_enabled: + self.base_metric_group = GenericMetricGroup(None, None) for user_defined_func in self.user_defined_funcs: - user_defined_func.open(FunctionContext(base_metric_group)) + user_defined_func.open(FunctionContext(self.base_metric_group)) def setup(self): super(StatelessFunctionOperation, self).setup() @@ -59,7 +62,9 @@ class StatelessFunctionOperation(Operation): super(StatelessFunctionOperation, self).start() def finish(self): - super(StatelessFunctionOperation, self).finish() + with self.scoped_finish_state: + super(StatelessFunctionOperation, self).finish() + self._update_gauge(self.base_metric_group) def needs_finalization(self): return False @@ -68,8 +73,9 @@ class StatelessFunctionOperation(Operation): super(StatelessFunctionOperation, self).reset() def teardown(self): - for user_defined_func in self.user_defined_funcs: - user_defined_func.close(None) + with self.scoped_finish_state: + for user_defined_func in self.user_defined_funcs: + user_defined_func.close() def progress_metrics(self): metrics = super(StatelessFunctionOperation, self).progress_metrics() @@ -81,9 +87,14 @@ class StatelessFunctionOperation(Operation): return metrics def process(self, o: WindowedValue): - output_stream = self.consumer.output_stream - self._value_coder_impl.encode_to_stream(self.func(o.value), output_stream, True) - output_stream.maybe_flush() + with self.scoped_process_state: + output_stream = self.consumer.output_stream + self._value_coder_impl.encode_to_stream(self.func(o.value), output_stream, True) + output_stream.maybe_flush() + + def monitoring_infos(self, transform_id): + # only pass user metric to Java + return super().user_monitoring_infos(transform_id) def generate_func(self, udfs): pass @@ -162,6 +173,16 @@ class StatelessFunctionOperation(Operation): self.variable_dict[constant_value_name] = parsed_constant_value return constant_value_name + @staticmethod + def _update_gauge(base_metric_group): + if base_metric_group is not None: + for name in base_metric_group._flink_gauge: + flink_gauge = base_metric_group._flink_gauge[name] + beam_gauge = base_metric_group._beam_gauge[name] + beam_gauge.set(flink_gauge()) + for sub_group in base_metric_group._sub_groups: + StatelessFunctionOperation._update_gauge(sub_group) + class ScalarFunctionOperation(StatelessFunctionOperation): def __init__(self, name, spec, counter_factory, sampler, consumers): @@ -198,14 +219,14 @@ class TableFunctionOperation(StatelessFunctionOperation): SCALAR_FUNCTION_URN, flink_fn_execution_pb2.UserDefinedFunctions) def create_scalar_function(factory, transform_id, transform_proto, parameter, consumers): return _create_user_defined_function_operation( - factory, transform_proto, consumers, parameter.udfs, ScalarFunctionOperation) + factory, transform_proto, consumers, parameter, ScalarFunctionOperation) @bundle_processor.BeamTransformFactory.register_urn( TABLE_FUNCTION_URN, flink_fn_execution_pb2.UserDefinedFunctions) def create_table_function(factory, transform_id, transform_proto, parameter, consumers): return _create_user_defined_function_operation( - factory, transform_proto, consumers, parameter.udfs, TableFunctionOperation) + factory, transform_proto, consumers, parameter, TableFunctionOperation) def _create_user_defined_function_operation(factory, transform_proto, consumers, udfs_proto, diff --git a/flink-python/pyflink/metrics/__init__.py b/flink-python/pyflink/metrics/__init__.py index aa5368368e..a8ae9feea6 100644 --- a/flink-python/pyflink/metrics/__init__.py +++ b/flink-python/pyflink/metrics/__init__.py @@ -16,6 +16,6 @@ # limitations under the License. ################################################################################ -from pyflink.metrics.metricbase import MetricGroup +from pyflink.metrics.metricbase import MetricGroup, Metric, Counter, Meter, Distribution -__all__ = ["MetricGroup"] +__all__ = ["MetricGroup", "Metric", "Counter", "Meter", "Distribution"] diff --git a/flink-python/pyflink/metrics/metricbase.py b/flink-python/pyflink/metrics/metricbase.py index f832587ddb..acbd6a993b 100644 --- a/flink-python/pyflink/metrics/metricbase.py +++ b/flink-python/pyflink/metrics/metricbase.py @@ -16,7 +16,9 @@ # limitations under the License. ################################################################################ import abc +import json from enum import Enum +from typing import Callable class MetricGroup(abc.ABC): @@ -35,6 +37,31 @@ class MetricGroup(abc.ABC): """ pass + def counter(self, name: str) -> 'Counter': + """ + Registers a new `Counter` with Flink. + """ + pass + + def gauge(self, name: str, obj: Callable[[], int]) -> None: + """ + Registers a new `Gauge` with Flink. + """ + pass + + def meter(self, name: str, time_span_in_seconds: int = 60) -> 'Meter': + """ + Registers a new `Meter` with Flink. + """ + # There is no meter type in Beam, use counter to implement meter + pass + + def distribution(self, name: str) -> 'Distribution': + """ + Registers a new `Distribution` with Flink. + """ + pass + class MetricGroupType(Enum): """ @@ -56,6 +83,8 @@ class GenericMetricGroup(MetricGroup): self._sub_groups = [] self._name = name self._metric_group_type = metric_group_type + self._flink_gauge = {} + self._beam_gauge = {} def _add_group(self, name: str, metric_group_type) -> 'MetricGroup': for group in self._sub_groups: @@ -76,3 +105,91 @@ class GenericMetricGroup(MetricGroup): else: return self._add_group(name, MetricGroupType.key)\ ._add_group(extra, MetricGroupType.value) + + def counter(self, name: str) -> 'Counter': + from apache_beam.metrics.metric import Metrics + return Counter(Metrics.counter(self._get_namespace(), name)) + + def gauge(self, name: str, obj: Callable[[], int]) -> None: + from apache_beam.metrics.metric import Metrics + self._flink_gauge[name] = obj + self._beam_gauge[name] = Metrics.gauge(self._get_namespace(), name) + + def meter(self, name: str, time_span_in_seconds: int = 60) -> 'Meter': + from apache_beam.metrics.metric import Metrics + # There is no meter type in Beam, use counter to implement meter + return Meter(Metrics.counter(self._get_namespace(time_span_in_seconds), name)) + + def distribution(self, name: str) -> 'Distribution': + from apache_beam.metrics.metric import Metrics + return Distribution(Metrics.distribution(self._get_namespace(), name)) + + def _get_metric_group_names_and_types(self) -> ([], []): + if self._name is None: + return [], [] + else: + names, types = self._parent._get_metric_group_names_and_types() + names.append(self._name) + types.append(str(self._metric_group_type)) + return names, types + + def _get_namespace(self, time=None) -> str: + names, metric_group_type = self._get_metric_group_names_and_types() + names.extend(metric_group_type) + if time is not None: + names.append(str(time)) + return json.dumps(names) + + +class Metric(object): + """Base interface of a metric object.""" + pass + + +class Counter(Metric): + """Counter metric interface. Allows a count to be incremented/decremented + during pipeline execution.""" + + def __init__(self, inner_counter): + self._inner_counter = inner_counter + + def inc(self, n=1): + self._inner_counter.inc(n) + + def dec(self, n=1): + self.inc(-n) + + def get_count(self): + from apache_beam.metrics.execution import MetricsEnvironment + container = MetricsEnvironment.current_container() + return container.get_counter(self._inner_counter.metric_name).get_cumulative() + + +class Distribution(Metric): + """Distribution Metric interface. + + Allows statistics about the distribution of a variable to be collected during + pipeline execution.""" + + def __init__(self, inner_distribution): + self._inner_distribution = inner_distribution + + def update(self, value): + self._inner_distribution.update(value) + + +class Meter(Metric): + """Meter Metric interface. + + Metric for measuring throughput.""" + + def __init__(self, inner_counter): + self._inner_counter = inner_counter + + def make_event(self, value=1): + self._inner_counter.inc(value) + + def get_count(self): + from apache_beam.metrics.execution import MetricsEnvironment + container = MetricsEnvironment.current_container() + return container.get_counter(self._inner_counter.metric_name).get_cumulative() diff --git a/flink-python/pyflink/metrics/tests/test_metric.py b/flink-python/pyflink/metrics/tests/test_metric.py index bba239c330..f43ddbd029 100644 --- a/flink-python/pyflink/metrics/tests/test_metric.py +++ b/flink-python/pyflink/metrics/tests/test_metric.py @@ -18,6 +18,14 @@ import unittest from pyflink.metrics.metricbase import GenericMetricGroup, MetricGroup +from pyflink.table import FunctionContext + +from apache_beam.runners.worker import statesampler +from apache_beam.utils import counters +from apache_beam.metrics.execution import MetricsContainer +from apache_beam.metrics.execution import MetricsEnvironment +from apache_beam.metrics.metricbase import MetricName +from apache_beam.metrics.cells import DistributionData class MetricTests(unittest.TestCase): @@ -38,3 +46,52 @@ class MetricTests(unittest.TestCase): def test_add_group_with_variable(self): new_group = MetricTests.base_metric_group.add_group('key', 'value') self.assertEqual(MetricTests.print_metric_group_path(new_group), 'root.key.value') + + def test_metric_not_enabled(self): + fc = FunctionContext(None) + with self.assertRaises(RuntimeError): + fc.get_metric_group() + + def test_get_metric_name(self): + new_group = MetricTests.base_metric_group.add_group('my_group') + self.assertEqual( + '["my_group", "MetricGroupType.generic"]', + new_group._get_namespace()) + self.assertEqual( + '["my_group", "MetricGroupType.generic", "60"]', + new_group._get_namespace('60')) + + def test_metrics(self): + sampler = statesampler.StateSampler('', counters.CounterFactory()) + statesampler.set_current_tracker(sampler) + state1 = sampler.scoped_state( + 'mystep', 'myState', metrics_container=MetricsContainer('mystep')) + + try: + sampler.start() + with state1: + counter = MetricTests.base_metric_group.counter("my_counter") + meter = MetricTests.base_metric_group.meter("my_meter") + distribution = MetricTests.base_metric_group.distribution("my_distribution") + container = MetricsEnvironment.current_container() + + self.assertEqual(0, counter.get_count()) + self.assertEqual(0, meter.get_count()) + self.assertEqual( + DistributionData( + 0, 0, 0, 0), container.get_distribution( + MetricName( + '[]', 'my_distribution')).get_cumulative()) + counter.inc(-2) + meter.make_event(3) + distribution.update(10) + distribution.update(2) + self.assertEqual(-2, counter.get_count()) + self.assertEqual(3, meter.get_count()) + self.assertEqual( + DistributionData( + 12, 2, 2, 10), container.get_distribution( + MetricName( + '[]', 'my_distribution')).get_cumulative()) + finally: + sampler.stop() diff --git a/flink-python/pyflink/proto/flink-fn-execution.proto b/flink-python/pyflink/proto/flink-fn-execution.proto index 02489b4083..5e98cfe16d 100644 --- a/flink-python/pyflink/proto/flink-fn-execution.proto +++ b/flink-python/pyflink/proto/flink-fn-execution.proto @@ -49,6 +49,7 @@ message UserDefinedFunction { // A list of user-defined functions to be executed in a batch. message UserDefinedFunctions { repeated UserDefinedFunction udfs = 1; + bool metric_enabled = 2; } // A representation of the data schema. diff --git a/flink-python/pyflink/shell.py b/flink-python/pyflink/shell.py index 604779f8a5..d7f353a66a 100644 --- a/flink-python/pyflink/shell.py +++ b/flink-python/pyflink/shell.py @@ -26,6 +26,9 @@ from pyflink.table import * from pyflink.table.catalog import * from pyflink.table.descriptors import * from pyflink.table.window import * +from pyflink.metrics import * +from pyflink.ml.api import * +from pyflink.ml.lib import * utf8_out = open(sys.stdout.fileno(), mode='w', encoding='utf8', buffering=1) diff --git a/flink-python/pyflink/table/tests/test_udf.py b/flink-python/pyflink/table/tests/test_udf.py index a599aacb6d..bc9860eb01 100644 --- a/flink-python/pyflink/table/tests/test_udf.py +++ b/flink-python/pyflink/table/tests/test_udf.py @@ -18,6 +18,7 @@ import datetime import pytz +import unittest from pyflink.table import DataTypes from pyflink.table.udf import ScalarFunction, udf @@ -30,6 +31,8 @@ from pyflink.testing.test_case_utils import PyFlinkStreamTableTestCase, \ class UserDefinedFunctionTests(object): def test_scalar_function(self): + # test metric disabled. + self.t_env.get_config().get_configuration().set_string('python.metric.enabled', 'false') # test lambda function self.t_env.register_function( "add_one", udf(lambda i: i + 1, DataTypes.BIGINT(), DataTypes.BIGINT())) @@ -238,6 +241,7 @@ class UserDefinedFunctionTests(object): self.assert_equals(actual, ["2", "6", "3"]) def test_open(self): + self.t_env.get_config().get_configuration().set_string('python.metric.enabled', 'true') self.t_env.register_function( "subtract", udf(Subtract(), DataTypes.BIGINT(), DataTypes.BIGINT())) table_sink = source_sink_utils.TestAppendSink( @@ -613,15 +617,19 @@ class SubtractOne(ScalarFunction): return i - 1 -class Subtract(ScalarFunction): - - def __init__(self): - self.subtracted_value = 0 +class Subtract(ScalarFunction, unittest.TestCase): def open(self, function_context): self.subtracted_value = 1 + mg = function_context.get_metric_group() + self.counter = mg.add_group("key", "value").counter("my_counter") + self.counter_sum = 0 def eval(self, i): + # counter + self.counter.inc(i) + self.counter_sum += i + self.assertEqual(self.counter_sum, self.counter.get_count()) return i - self.subtracted_value diff --git a/flink-python/pyflink/table/udf.py b/flink-python/pyflink/table/udf.py index 3a059fa38d..924827558e 100644 --- a/flink-python/pyflink/table/udf.py +++ b/flink-python/pyflink/table/udf.py @@ -39,6 +39,9 @@ class FunctionContext(object): self._base_metric_group = base_metric_group def get_metric_group(self) -> MetricGroup: + if self._base_metric_group is None: + raise RuntimeError("Metric has not been enabled. You can enable " + "metric with the 'python.metric.enabled' configuration.") return self._base_metric_group diff --git a/flink-python/setup.py b/flink-python/setup.py index aeb3d25966..c74abeb30a 100644 --- a/flink-python/setup.py +++ b/flink-python/setup.py @@ -177,6 +177,10 @@ run sdist. 'pyflink.fn_execution', 'pyflink.metrics', 'pyflink.ml', + 'pyflink.ml.api', + 'pyflink.ml.api.param', + 'pyflink.ml.lib', + 'pyflink.ml.lib.param', 'pyflink.lib', 'pyflink.opt', 'pyflink.conf', diff --git a/flink-python/src/main/java/org/apache/flink/python/AbstractPythonFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/python/AbstractPythonFunctionRunner.java index a5afe8183c..1a095613b1 100644 --- a/flink-python/src/main/java/org/apache/flink/python/AbstractPythonFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/python/AbstractPythonFunctionRunner.java @@ -26,6 +26,7 @@ import org.apache.flink.python.env.PythonEnvironmentManager; import org.apache.flink.python.metric.FlinkMetricContainer; import org.apache.flink.util.Preconditions; +import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.runners.core.construction.PipelineOptionsTranslation; import org.apache.beam.runners.core.construction.graph.ExecutableStage; @@ -122,7 +123,7 @@ public abstract class AbstractPythonFunctionRunner implements PythonFunction /** * The flinkMetricContainer will be set to null if metric is configured to be turned off. */ - @Nullable private FlinkMetricContainer flinkMetricContainer; + @Nullable protected FlinkMetricContainer flinkMetricContainer; public AbstractPythonFunctionRunner( String taskName, @@ -161,7 +162,29 @@ public abstract class AbstractPythonFunctionRunner implements PythonFunction jobBundleFactory = createJobBundleFactory(pipelineOptions); stageBundleFactory = createStageBundleFactory(); - progressHandler = BundleProgressHandler.ignored(); + progressHandler = getProgressHandler(flinkMetricContainer); + } + + /** + * Ignore bundle progress if flinkMetricContainer is null. The flinkMetricContainer will be set + * to null if metric is configured to be turned off. + */ + private BundleProgressHandler getProgressHandler(FlinkMetricContainer flinkMetricContainer) { + if (flinkMetricContainer == null) { + return BundleProgressHandler.ignored(); + } else { + return new BundleProgressHandler() { + @Override + public void onProgress(BeamFnApi.ProcessBundleProgressResponse progress) { + flinkMetricContainer.updateMetrics(taskName, progress.getMonitoringInfosList()); + } + + @Override + public void onCompleted(BeamFnApi.ProcessBundleResponse response) { + flinkMetricContainer.updateMetrics(taskName, response.getMonitoringInfosList()); + } + }; + } } /** diff --git a/flink-python/src/main/java/org/apache/flink/python/metric/FlinkMetricContainer.java b/flink-python/src/main/java/org/apache/flink/python/metric/FlinkMetricContainer.java index 30a83f3821..b5b3da534a 100644 --- a/flink-python/src/main/java/org/apache/flink/python/metric/FlinkMetricContainer.java +++ b/flink-python/src/main/java/org/apache/flink/python/metric/FlinkMetricContainer.java @@ -19,17 +19,263 @@ package org.apache.flink.python.metric; import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.configuration.MetricOptions; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MeterView; import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + +import org.apache.beam.model.pipeline.v1.MetricsApi.MonitoringInfo; +import org.apache.beam.runners.core.metrics.MetricsContainerImpl; +import org.apache.beam.runners.core.metrics.MetricsContainerStepMap; +import org.apache.beam.runners.core.metrics.MonitoringInfoConstants; +import org.apache.beam.runners.core.metrics.MonitoringInfoMetricName; +import org.apache.beam.sdk.metrics.DistributionResult; +import org.apache.beam.sdk.metrics.GaugeResult; +import org.apache.beam.sdk.metrics.MetricKey; +import org.apache.beam.sdk.metrics.MetricName; +import org.apache.beam.sdk.metrics.MetricQueryResults; +import org.apache.beam.sdk.metrics.MetricResult; +import org.apache.beam.sdk.metrics.MetricResults; +import org.apache.beam.sdk.metrics.MetricsFilter; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.beam.runners.core.metrics.MetricsContainerStepMap.asAttemptedOnlyMetricResults; +import static org.apache.flink.util.Preconditions.checkNotNull; + /** * Helper class for forwarding Python metrics to Java accumulators and metrics. */ @Internal -public class FlinkMetricContainer { +public final class FlinkMetricContainer { + + private static final String METRIC_KEY_SEPARATOR = + GlobalConfiguration.loadConfiguration().getString(MetricOptions.SCOPE_DELIMITER); + private final MetricsContainerStepMap metricsContainers; private final MetricGroup baseMetricGroup; + private final Map flinkCounterCache; + private final Map flinkMeterCache; + private final Map flinkDistributionGaugeCache; + private final Map flinkGaugeCache; public FlinkMetricContainer(MetricGroup metricGroup) { - this.baseMetricGroup = metricGroup; + this.baseMetricGroup = checkNotNull(metricGroup); + this.flinkCounterCache = new HashMap<>(); + this.flinkMeterCache = new HashMap<>(); + this.flinkDistributionGaugeCache = new HashMap<>(); + this.flinkGaugeCache = new HashMap<>(); + this.metricsContainers = new MetricsContainerStepMap(); + } + + private MetricsContainerImpl getMetricsContainer(String stepName) { + return metricsContainers.getContainer(stepName); + } + + /** + * Update this container with metrics from the passed {@link MonitoringInfo}s, and send updates + * along to Flink's internal metrics framework. + */ + public void updateMetrics(String stepName, List monitoringInfos) { + getMetricsContainer(stepName).update(monitoringInfos); + updateMetrics(stepName); + } + + /** + * Update Flink's internal metrics ({@link #flinkCounterCache}) with the latest metrics for + * a given step. + */ + private void updateMetrics(String stepName) { + MetricResults metricResults = asAttemptedOnlyMetricResults(metricsContainers); + MetricQueryResults metricQueryResults = + metricResults.queryMetrics(MetricsFilter.builder().addStep(stepName).build()); + updateCounterOrMeter(metricQueryResults.getCounters()); + updateDistributions(metricQueryResults.getDistributions()); + updateGauge(metricQueryResults.getGauges()); + } + + private boolean isUserMetric(MetricResult metricResult) { + MetricName metricName = metricResult.getKey().metricName(); + return (metricName instanceof MonitoringInfoMetricName) && + ((MonitoringInfoMetricName) metricName).getUrn() + .contains(MonitoringInfoConstants.Urns.USER_COUNTER); + } + + private void updateCounterOrMeter(Iterable> counters) { + for (MetricResult metricResult : counters) { + if (!isUserMetric(metricResult)) { + continue; + } + // get identifier + String flinkMetricIdentifier = getFlinkMetricIdentifierString(metricResult.getKey()); + + // get metric type + ArrayList scopeComponents = getNameSpaceArray(metricResult.getKey()); + if ((scopeComponents.size() % 2) != 0) { + Meter meter = flinkMeterCache.get(flinkMetricIdentifier); + if (null == meter) { + int timeSpanInSeconds = + Integer.parseInt(scopeComponents.get(scopeComponents.size() - 1)); + MetricGroup metricGroup = + registerMetricGroup(metricResult.getKey(), baseMetricGroup); + meter = metricGroup.meter( + metricResult.getKey().metricName().getName(), + new MeterView(timeSpanInSeconds)); + flinkMeterCache.put(flinkMetricIdentifier, meter); + } + + Long update = metricResult.getAttempted(); + meter.markEvent(update); + } else { + Counter counter = flinkCounterCache.get(flinkMetricIdentifier); + if (null == counter) { + MetricGroup metricGroup = + registerMetricGroup(metricResult.getKey(), baseMetricGroup); + counter = metricGroup.counter(metricResult.getKey().metricName().getName()); + flinkCounterCache.put(flinkMetricIdentifier, counter); + } + + Long update = metricResult.getAttempted(); + counter.inc(update - counter.getCount()); + } + } + } + + private void updateDistributions(Iterable> distributions) { + for (MetricResult metricResult : distributions) { + if (!isUserMetric(metricResult)) { + continue; + } + // get identifier + String flinkMetricIdentifier = getFlinkMetricIdentifierString(metricResult.getKey()); + DistributionResult update = metricResult.getAttempted(); + + // update flink metric + FlinkDistributionGauge gauge = flinkDistributionGaugeCache.get(flinkMetricIdentifier); + if (gauge == null) { + MetricGroup metricGroup = + registerMetricGroup(metricResult.getKey(), baseMetricGroup); + gauge = metricGroup.gauge( + metricResult.getKey().metricName().getName(), + new FlinkDistributionGauge(update)); + flinkDistributionGaugeCache.put(flinkMetricIdentifier, gauge); + } else { + gauge.update(update); + } + } + } + + private void updateGauge(Iterable> gauges) { + for (MetricResult metricResult : gauges) { + if (!isUserMetric(metricResult)) { + continue; + } + // get identifier + String flinkMetricIdentifier = getFlinkMetricIdentifierString(metricResult.getKey()); + + GaugeResult update = metricResult.getAttempted(); + + // update flink metric + FlinkGauge gauge = flinkGaugeCache.get(flinkMetricIdentifier); + if (gauge == null) { + MetricGroup metricGroup = registerMetricGroup(metricResult.getKey(), baseMetricGroup); + gauge = metricGroup.gauge( + metricResult.getKey().metricName().getName(), + new FlinkGauge(update)); + flinkGaugeCache.put(flinkMetricIdentifier, gauge); + } else { + gauge.update(update); + } + } + } + + @VisibleForTesting + static ArrayList getNameSpaceArray(MetricKey metricKey) { + MetricName metricName = metricKey.metricName(); + try { + return new ObjectMapper().readValue(metricName.getNamespace(), ArrayList.class); + } catch (JsonProcessingException e) { + throw new RuntimeException( + String.format("Parse namespace[%s] error. ", metricName.getNamespace()), e); + } + } + + @VisibleForTesting + static String getFlinkMetricIdentifierString(MetricKey metricKey) { + MetricName metricName = metricKey.metricName(); + ArrayList scopeComponents = getNameSpaceArray(metricKey); + List results = scopeComponents.subList(0, scopeComponents.size() / 2); + results.add(metricName.getName()); + return String.join(METRIC_KEY_SEPARATOR, results); + } + + @VisibleForTesting + static MetricGroup registerMetricGroup(MetricKey metricKey, MetricGroup metricGroup) { + ArrayList scopeComponents = getNameSpaceArray(metricKey); + int size = scopeComponents.size(); + List metricGroupNames = scopeComponents.subList(0, size / 2); + List metricGroupTypes = scopeComponents.subList(size / 2, size); + for (int i = 0; i < metricGroupNames.size(); ++i) { + if (metricGroupTypes.get(i).equals("MetricGroupType.generic")) { + metricGroup = metricGroup.addGroup(metricGroupNames.get(i)); + } else if (metricGroupTypes.get(i).equals("MetricGroupType.key")) { + metricGroup = metricGroup.addGroup( + metricGroupNames.get(i), + metricGroupNames.get(++i)); + } + } + return metricGroup; + } + + /** + * Flink {@link Gauge} for {@link DistributionResult}. + */ + public static class FlinkDistributionGauge implements Gauge { + + private DistributionResult data; + + FlinkDistributionGauge(DistributionResult data) { + this.data = data; + } + + void update(DistributionResult data) { + this.data = data; + } + + @Override + public DistributionResult getValue() { + return data; + } + } + + /** + * Flink {@link Gauge} for {@link GaugeResult}. + */ + public static class FlinkGauge implements Gauge { + + private GaugeResult data; + + FlinkGauge(GaugeResult data) { + this.data = data; + } + + void update(GaugeResult update) { + this.data = update; + } + + @Override + public Long getValue() { + return data.getValue(); + } } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/AbstractPythonScalarFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/AbstractPythonScalarFunctionRunner.java index 91e1fc823a..eaeaf98306 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/AbstractPythonScalarFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/scalar/AbstractPythonScalarFunctionRunner.java @@ -69,6 +69,7 @@ public abstract class AbstractPythonScalarFunctionRunner extends AbstractPyt for (PythonFunctionInfo pythonFunctionInfo : scalarFunctions) { builder.addUdfs(getUserDefinedFunctionProto(pythonFunctionInfo)); } + builder.setMetricEnabled(flinkMetricContainer != null); return builder.build(); } } diff --git a/flink-python/src/test/java/org/apache/flink/python/metric/FlinkMetricContainerTest.java b/flink-python/src/test/java/org/apache/flink/python/metric/FlinkMetricContainerTest.java new file mode 100644 index 0000000000..f05f1b60a8 --- /dev/null +++ b/flink-python/src/test/java/org/apache/flink/python/metric/FlinkMetricContainerTest.java @@ -0,0 +1,218 @@ +/* + * 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.python.metric; + +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MeterView; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.SimpleCounter; +import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.NoOpMetricRegistry; +import org.apache.flink.runtime.metrics.groups.GenericMetricGroup; +import org.apache.flink.runtime.metrics.groups.MetricGroupTest; + +import org.apache.beam.model.pipeline.v1.MetricsApi; +import org.apache.beam.model.pipeline.v1.MetricsApi.MonitoringInfo; +import org.apache.beam.runners.core.construction.BeamUrns; +import org.apache.beam.runners.core.metrics.MonitoringInfoConstants; +import org.apache.beam.runners.core.metrics.SimpleMonitoringInfoBuilder; +import org.apache.beam.sdk.metrics.DistributionResult; +import org.apache.beam.sdk.metrics.MetricKey; +import org.apache.beam.sdk.metrics.MetricName; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; +import org.junit.Before; +import org.junit.Test; +import org.mockito.ArgumentMatcher; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import java.util.Arrays; +import java.util.List; + +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.ArgumentMatchers.argThat; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** + * Tests for {@link FlinkMetricContainer}. + */ +public class FlinkMetricContainerTest { + + @Mock + private RuntimeContext runtimeContext; + @Mock + private MetricGroup metricGroup; + + private FlinkMetricContainer container; + + private static final String GAUGE_URN = + BeamUrns.getUrn(MetricsApi.MonitoringInfoTypeUrns.Enum.LATEST_INT64_TYPE); + + private static final List DEFAULT_SCOPE_COMPONENTS = Arrays.asList( + "key", + "value", + "MetricGroupType.key", + "MetricGroupType.value"); + + private static final String DEFAULT_NAMESPACE = + "[\"key\", \"value\", \"MetricGroupType.key\", \"MetricGroupType.value\"]"; + + @Before + public void beforeTest() { + MockitoAnnotations.initMocks(this); + when(runtimeContext.getMetricGroup()).thenReturn(metricGroup); + when(metricGroup.addGroup(any(), any())).thenReturn(metricGroup); + when(metricGroup.addGroup(any())).thenReturn(metricGroup); + container = new FlinkMetricContainer(runtimeContext.getMetricGroup()); + } + + @Test + public void testGetNameSpaceArray() { + String json = "[\"key\", \"value\", \"MetricGroupType.key\", \"MetricGroupType.value\"]"; + MetricKey key = MetricKey.create("step", MetricName.named(json, "name")); + assertThat(FlinkMetricContainer.getNameSpaceArray(key), is(DEFAULT_SCOPE_COMPONENTS)); + } + + @Test + public void testGetFlinkMetricIdentifierString() { + MetricKey key = MetricKey.create("step", MetricName.named(DEFAULT_NAMESPACE, "name")); + assertThat(FlinkMetricContainer.getFlinkMetricIdentifierString(key), is("key.value.name")); + } + + @Test + public void testRegisterMetricGroup() { + MetricKey key = MetricKey.create("step", MetricName.named(DEFAULT_NAMESPACE, "name")); + + MetricRegistry registry = NoOpMetricRegistry.INSTANCE; + GenericMetricGroup root = new GenericMetricGroup( + registry, + new MetricGroupTest.DummyAbstractMetricGroup(registry), + "root"); + MetricGroup metricGroup = FlinkMetricContainer.registerMetricGroup(key, root); + + assertThat(metricGroup.getScopeComponents(), is(Arrays.asList("root", "key", "value").toArray())); + } + + @Test + public void testCounterMonitoringInfoUpdate() { + SimpleCounter userCounter = new SimpleCounter(); + when(metricGroup.counter("myCounter")).thenReturn(userCounter); + + MonitoringInfo userMonitoringInfo = + new SimpleMonitoringInfoBuilder() + .setUrn(MonitoringInfoConstants.Urns.USER_COUNTER) + .setLabel(MonitoringInfoConstants.Labels.NAMESPACE, DEFAULT_NAMESPACE) + .setLabel(MonitoringInfoConstants.Labels.NAME, "myCounter") + .setLabel(MonitoringInfoConstants.Labels.PTRANSFORM, "anyPTransform") + .setInt64Value(111) + .build(); + + assertThat(userCounter.getCount(), is(0L)); + container.updateMetrics( + "step", ImmutableList.of(userMonitoringInfo)); + assertThat(userCounter.getCount(), is(111L)); + } + + @Test + public void testMeterMonitoringInfoUpdate() { + MeterView userMeter = new MeterView(new SimpleCounter()); + when(metricGroup.meter(eq("myMeter"), any(Meter.class))).thenReturn(userMeter); + String namespace = "[\"key\", \"value\", \"MetricGroupType.key\", \"MetricGroupType.value\", \"60\"]"; + + MonitoringInfo userMonitoringInfo = + new SimpleMonitoringInfoBuilder() + .setUrn(MonitoringInfoConstants.Urns.USER_COUNTER) + .setLabel(MonitoringInfoConstants.Labels.NAMESPACE, namespace) + .setLabel(MonitoringInfoConstants.Labels.NAME, "myMeter") + .setLabel(MonitoringInfoConstants.Labels.PTRANSFORM, "anyPTransform") + .setInt64Value(111) + .build(); + assertThat(userMeter.getCount(), is(0L)); + assertThat(userMeter.getRate(), is(0.0)); + container.updateMetrics( + "step", ImmutableList.of(userMonitoringInfo)); + userMeter.update(); + assertThat(userMeter.getCount(), is(111L)); + assertThat(userMeter.getRate(), is(1.85)); // 111 div 60 = 1.85 + } + + @Test + public void testGaugeMonitoringInfoUpdate() { + MonitoringInfo userMonitoringInfo = MonitoringInfo.newBuilder() + .setUrn(MonitoringInfoConstants.Urns.USER_COUNTER) + .putLabels(MonitoringInfoConstants.Labels.NAMESPACE, DEFAULT_NAMESPACE) + .putLabels(MonitoringInfoConstants.Labels.NAME, "myGauge") + .putLabels(MonitoringInfoConstants.Labels.PTRANSFORM, "anyPTransform") + .setMetric(MetricsApi.Metric + .newBuilder() + .setCounterData( + MetricsApi.CounterData.newBuilder() + .setInt64Value(111L))) + .setType(GAUGE_URN) + .build(); + + container.updateMetrics("step", ImmutableList.of(userMonitoringInfo)); + verify(metricGroup) + .gauge( + eq("myGauge"), + argThat( + (ArgumentMatcher) argument -> { + Long actual = argument.getValue(); + return actual.equals(111L); + })); + } + + @Test + public void testDistributionMonitoringInfoUpdate() { + MonitoringInfo userMonitoringInfo = MonitoringInfo.newBuilder() + .setUrn(MonitoringInfoConstants.Urns.USER_DISTRIBUTION_COUNTER) + .putLabels(MonitoringInfoConstants.Labels.NAMESPACE, DEFAULT_NAMESPACE) + .putLabels(MonitoringInfoConstants.Labels.NAME, "myDistribution") + .putLabels(MonitoringInfoConstants.Labels.PTRANSFORM, "anyPTransform") + .setMetric( + MetricsApi.Metric.newBuilder() + .setDistributionData( + MetricsApi.DistributionData.newBuilder() + .setIntDistributionData( + MetricsApi.IntDistributionData.newBuilder() + .setSum(30) + .setCount(10) + .setMin(1) + .setMax(5)))) + .build(); + + container.updateMetrics("step", ImmutableList.of(userMonitoringInfo)); + // The one Flink distribution that gets created is a FlinkDistributionGauge; here we verify + // its initial (and in this test, final) value + verify(metricGroup) + .gauge( + eq("myDistribution"), + argThat( + (ArgumentMatcher) argument -> { + DistributionResult actual = argument.getValue(); + DistributionResult expected = DistributionResult.create(30, 10, 1, 5); + return actual.equals(expected); + })); + } +} -- Gitee From f40cd6939a8a57ad430fcb5eb60d7f2c56629281 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Mon, 30 Mar 2020 10:29:13 +0200 Subject: [PATCH 357/885] [FLINK-16862][quickstarts] Remove example url --- .../src/main/resources/archetype-resources/pom.xml | 1 - .../src/main/resources/archetype-resources/pom.xml | 1 - 2 files changed, 2 deletions(-) diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml index 73bf1d44d5..206d5bd7d4 100644 --- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml +++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml @@ -26,7 +26,6 @@ under the License. jar Flink Quickstart Job - http://www.myorganization.org UTF-8 diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml index 6a34ad61dc..478a41898d 100644 --- a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml +++ b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml @@ -27,7 +27,6 @@ under the License. jar Flink Quickstart Job - http://www.myorganization.org -- Gitee From ff491bdbd9f4f6fe935bbe1485afacaf729ddd88 Mon Sep 17 00:00:00 2001 From: qqibrow Date: Wed, 25 Mar 2020 16:39:13 -0700 Subject: [PATCH 358/885] [FLINK-13483] Retry delete when checking path existence in AbstractHadoopFileSystemITTest This closes #11516. --- .../hdfs/AbstractHadoopFileSystemITTest.java | 20 ++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/flink-filesystems/flink-hadoop-fs/src/test/java/org/apache/flink/runtime/fs/hdfs/AbstractHadoopFileSystemITTest.java b/flink-filesystems/flink-hadoop-fs/src/test/java/org/apache/flink/runtime/fs/hdfs/AbstractHadoopFileSystemITTest.java index d2695d7a5f..083a8801da 100644 --- a/flink-filesystems/flink-hadoop-fs/src/test/java/org/apache/flink/runtime/fs/hdfs/AbstractHadoopFileSystemITTest.java +++ b/flink-filesystems/flink-hadoop-fs/src/test/java/org/apache/flink/runtime/fs/hdfs/AbstractHadoopFileSystemITTest.java @@ -27,6 +27,7 @@ import org.apache.flink.core.fs.Path; import org.apache.flink.util.TestLogger; import org.junit.AfterClass; +import org.junit.Assert; import org.junit.Test; import java.io.BufferedReader; @@ -137,26 +138,27 @@ public abstract class AbstractHadoopFileSystemITTest extends TestLogger { } finally { // clean up - fs.delete(directory, true); + cleanupDirectoryWithRetry(fs, directory, deadline); } - - // now directory must be gone - checkPathExistence(directory, false, deadline); } @AfterClass public static void teardown() throws IOException, InterruptedException { try { if (fs != null) { - // clean up - fs.delete(basePath, true); - - // now directory must be gone - checkPathExistence(basePath, false, deadline); + cleanupDirectoryWithRetry(fs, basePath, deadline); } } finally { FileSystem.initialize(new Configuration()); } } + + public static void cleanupDirectoryWithRetry(FileSystem fs, Path path, long deadline) throws IOException, InterruptedException { + while (fs.exists(path) && System.nanoTime() < deadline) { + fs.delete(path, true); + Thread.sleep(50L); + } + Assert.assertFalse(fs.exists(path)); + } } -- Gitee From 0dad640a2037b108df24031b645e2b60c1e9fc68 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Thu, 26 Mar 2020 13:08:48 +0100 Subject: [PATCH 359/885] [FLINK-16806][operators] Fix shouldSetAvailableForAnotherInput for MultipleInputStreamOperator --- .../io/MultipleInputSelectionHandler.java | 2 +- .../io/MultipleInputSelectionHandlerTest.java | 52 +++++++++++++++++++ 2 files changed, 53 insertions(+), 1 deletion(-) create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/MultipleInputSelectionHandlerTest.java diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/MultipleInputSelectionHandler.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/MultipleInputSelectionHandler.java index f143a6dfaa..9211c49011 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/MultipleInputSelectionHandler.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/MultipleInputSelectionHandler.java @@ -112,7 +112,7 @@ public class MultipleInputSelectionHandler { } boolean shouldSetAvailableForAnotherInput() { - return availableInputsMask != allSelectedMask && inputSelection.areAllInputsSelected(); + return (inputSelection.getInputMask() & allSelectedMask & ~availableInputsMask) != 0; } void setAvailableInput(int inputIndex) { diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/MultipleInputSelectionHandlerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/MultipleInputSelectionHandlerTest.java new file mode 100644 index 0000000000..57196b5d7f --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/MultipleInputSelectionHandlerTest.java @@ -0,0 +1,52 @@ +/* + * 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.runtime.io; + +import org.apache.flink.streaming.api.operators.InputSelection; + +import org.junit.Test; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +/** + * Tests for {@link MultipleInputSelectionHandler}. + */ +public class MultipleInputSelectionHandlerTest { + @Test + public void testShouldSetAvailableForAnotherInput() { + InputSelection secondAndThird = new InputSelection.Builder().select(2).select(3).build(); + + MultipleInputSelectionHandler selectionHandler = new MultipleInputSelectionHandler(() -> secondAndThird, 3); + selectionHandler.nextSelection(); + + assertFalse(selectionHandler.shouldSetAvailableForAnotherInput()); + + selectionHandler.setUnavailableInput(0); + assertFalse(selectionHandler.shouldSetAvailableForAnotherInput()); + + selectionHandler.setUnavailableInput(2); + assertTrue(selectionHandler.shouldSetAvailableForAnotherInput()); + + selectionHandler.setAvailableInput(0); + assertTrue(selectionHandler.shouldSetAvailableForAnotherInput()); + + selectionHandler.setAvailableInput(2); + assertFalse(selectionHandler.shouldSetAvailableForAnotherInput()); + } +} -- Gitee From f7c3d1bba21d6822d30488a73043c9266c5a0b39 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Thu, 26 Mar 2020 11:34:03 +0100 Subject: [PATCH 360/885] [FLINK-16806][operators] Add support for InputSelection to MultipleInputStreamOperator --- .../io/StreamMultipleInputProcessor.java | 26 +- .../tasks/MultipleInputStreamTask.java | 5 +- .../tasks/StreamTaskMailboxTestHarness.java | 9 +- ...TaskMultipleInputSelectiveReadingTest.java | 376 ++++++++++++++++++ ...estAnyModeMultipleInputStreamOperator.java | 86 ++++ ...SequentialMultipleInputStreamOperator.java | 66 +++ 6 files changed, 546 insertions(+), 22 deletions(-) create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskMultipleInputSelectiveReadingTest.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TestAnyModeMultipleInputStreamOperator.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TestSequentialMultipleInputStreamOperator.java diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java index e02fcb2aad..3505f75584 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamMultipleInputProcessor.java @@ -110,11 +110,16 @@ public final class StreamMultipleInputProcessor implements StreamInputProcessor @Override public CompletableFuture getAvailableFuture() { - if (inputSelectionHandler.areAllInputsSelected()) { - return isAnyInputAvailable(); - } else { - throw new UnsupportedOperationException(); + if (inputSelectionHandler.isAnyInputAvailable() || inputSelectionHandler.areAllInputsFinished()) { + return AVAILABLE; + } + final CompletableFuture anyInputAvailable = new CompletableFuture<>(); + for (int i = 0; i < inputProcessors.length; i++) { + if (!inputSelectionHandler.isInputFinished(i) && inputSelectionHandler.isInputSelected(i)) { + inputProcessors[i].networkInput.getAvailableFuture().thenRun(() -> anyInputAvailable.complete(null)); + } } + return anyInputAvailable; } @Override @@ -202,19 +207,6 @@ public final class StreamMultipleInputProcessor implements StreamInputProcessor } } - private CompletableFuture isAnyInputAvailable() { - if (inputSelectionHandler.isAnyInputAvailable() || inputSelectionHandler.areAllInputsFinished()) { - return AVAILABLE; - } - final CompletableFuture anyInputAvailable = new CompletableFuture<>(); - for (int i = 0; i < inputProcessors.length; i++) { - if (!inputSelectionHandler.isInputFinished(i)) { - inputProcessors[i].networkInput.getAvailableFuture().thenRun(() -> anyInputAvailable.complete(null)); - } - } - return anyInputAvailable; - } - private int getInputId(int inputIndex) { return inputIndex + 1; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTask.java index 7562be7e06..a69e495d64 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTask.java @@ -88,11 +88,8 @@ public class MultipleInputStreamTask extends StreamTask[] inputGates, TypeSerializer[] inputDeserializers, WatermarkGauge[] inputWatermarkGauges) { - if (headOperator instanceof InputSelectable) { - throw new UnsupportedOperationException(); - } MultipleInputSelectionHandler selectionHandler = new MultipleInputSelectionHandler( - null, + headOperator instanceof InputSelectable ? (InputSelectable) headOperator : null, inputGates.length); InputGate[] unionedInputGates = new InputGate[inputGates.length]; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskMailboxTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskMailboxTestHarness.java index d98cbfc172..72a9326ecb 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskMailboxTestHarness.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskMailboxTestHarness.java @@ -121,9 +121,16 @@ public class StreamTaskMailboxTestHarness implements AutoCloseable { } public void process() throws Exception { - while (streamTask.inputProcessor.isAvailable() && streamTask.mailboxProcessor.isMailboxLoopRunning()) { + while (processSingleStep()) { + } + } + + public boolean processSingleStep() throws Exception { + if (streamTask.inputProcessor.isAvailable() && streamTask.mailboxProcessor.isMailboxLoopRunning()) { streamTask.runMailboxStep(); + return true; } + return false; } public void endInput() { diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskMultipleInputSelectiveReadingTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskMultipleInputSelectiveReadingTest.java new file mode 100644 index 0000000000..ef01cf3b94 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskMultipleInputSelectiveReadingTest.java @@ -0,0 +1,376 @@ +/* + * 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.runtime.tasks; + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorV2; +import org.apache.flink.streaming.api.operators.BoundedMultiInput; +import org.apache.flink.streaming.api.operators.Input; +import org.apache.flink.streaming.api.operators.InputSelectable; +import org.apache.flink.streaming.api.operators.InputSelection; +import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.util.TestAnyModeMultipleInputStreamOperator; +import org.apache.flink.streaming.util.TestAnyModeMultipleInputStreamOperator.ToStringInput; +import org.apache.flink.streaming.util.TestSequentialMultipleInputStreamOperator; +import org.apache.flink.util.ExceptionUtils; + +import org.junit.Test; + +import java.util.ArrayDeque; +import java.util.Arrays; +import java.util.List; +import java.util.Queue; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** + * Test selective reading. + */ +public class StreamTaskMultipleInputSelectiveReadingTest { + + private static final StreamRecord[] INPUT1 = new StreamRecord[] { + new StreamRecord<>("Hello-1"), + new StreamRecord<>("Hello-2"), + new StreamRecord<>("Hello-3") + }; + + private static final StreamRecord[] INPUT2 = new StreamRecord[] { + new StreamRecord<>(1), + new StreamRecord<>(2), + new StreamRecord<>(3), + new StreamRecord<>(4) + }; + + @Test + public void testAnyOrderedReading() throws Exception { + ArrayDeque expectedOutput = new ArrayDeque<>(); + expectedOutput.add(new StreamRecord<>("[1]: Hello-1")); + expectedOutput.add(new StreamRecord<>("[2]: 1")); + expectedOutput.add(new StreamRecord<>("[1]: Hello-2")); + expectedOutput.add(new StreamRecord<>("[2]: 2")); + expectedOutput.add(new StreamRecord<>("[1]: Hello-3")); + expectedOutput.add(new StreamRecord<>("[2]: 3")); + expectedOutput.add(new StreamRecord<>("[2]: 4")); + + testInputSelection(new TestAnyModeMultipleInputStreamOperator.Factory(), false, expectedOutput, true); + } + + @Test + public void testAnyUnorderedReading() throws Exception { + ArrayDeque expectedOutput = new ArrayDeque<>(); + expectedOutput.add(new StreamRecord<>("[1]: Hello-1")); + expectedOutput.add(new StreamRecord<>("[2]: 1")); + expectedOutput.add(new StreamRecord<>("[1]: Hello-2")); + expectedOutput.add(new StreamRecord<>("[2]: 2")); + expectedOutput.add(new StreamRecord<>("[1]: Hello-3")); + expectedOutput.add(new StreamRecord<>("[2]: 3")); + expectedOutput.add(new StreamRecord<>("[2]: 4")); + + testInputSelection(new TestAnyModeMultipleInputStreamOperator.Factory(), true, expectedOutput, false); + } + + @Test + public void testSequentialReading() throws Exception { + ArrayDeque expectedOutput = new ArrayDeque<>(); + expectedOutput.add(new StreamRecord<>("[1]: Hello-1")); + expectedOutput.add(new StreamRecord<>("[1]: Hello-2")); + expectedOutput.add(new StreamRecord<>("[1]: Hello-3")); + expectedOutput.add(new StreamRecord<>("[2]: 1")); + expectedOutput.add(new StreamRecord<>("[2]: 2")); + expectedOutput.add(new StreamRecord<>("[2]: 3")); + expectedOutput.add(new StreamRecord<>("[2]: 4")); + + testInputSelection(new TestSequentialMultipleInputStreamOperator.Factory(), true, expectedOutput, true); + } + + @Test + public void testSpecialRuleReading() throws Exception { + ArrayDeque expectedOutput = new ArrayDeque<>(); + expectedOutput.add(new StreamRecord<>("[1]: Hello-1")); + expectedOutput.add(new StreamRecord<>("[1]: Hello-2")); + expectedOutput.add(new StreamRecord<>("[2]: 1")); + expectedOutput.add(new StreamRecord<>("[2]: 2")); + expectedOutput.add(new StreamRecord<>("[1]: Hello-3")); + expectedOutput.add(new StreamRecord<>("[2]: 3")); + expectedOutput.add(new StreamRecord<>("[2]: 4")); + + testInputSelection(new SpecialRuleReadingStreamOperatorFactory(3, 4, 2), true, expectedOutput, true); + } + + @Test + public void testReadFinishedInput() throws Exception { + try { + testInputSelection(new TestReadFinishedInputStreamOperatorFactory(), true, new ArrayDeque<>(), true); + fail("should throw an IOException"); + } catch (Exception t) { + if (!ExceptionUtils.findThrowableWithMessage(t, "Can not make a progress: all selected inputs are already finished").isPresent()) { + throw t; + } + } + } + + private void testInputSelection( + StreamOperatorFactory streamOperatorFactory, + boolean autoProcess, + ArrayDeque expectedOutput, + boolean orderedCheck) throws Exception { + try (StreamTaskMailboxTestHarness testHarness = + new MultipleInputStreamTaskTestHarnessBuilder<>(MultipleInputStreamTask::new, BasicTypeInfo.STRING_TYPE_INFO) + .addInput(BasicTypeInfo.STRING_TYPE_INFO) + .addInput(BasicTypeInfo.INT_TYPE_INFO) + .setupOutputForSingletonOperatorChain(streamOperatorFactory) + .build()) { + + testHarness.setAutoProcess(autoProcess); + for (StreamRecord record : INPUT1) { + testHarness.processElement(record, 0); + } + for (StreamRecord record : INPUT2) { + testHarness.processElement(record, 1); + } + + testHarness.endInput(); + + if (!autoProcess) { + testHarness.process(); + } + testHarness.waitForTaskCompletion(); + + if (orderedCheck) { + assertThat(testHarness.getOutput(), contains(expectedOutput.toArray())); + } else { + assertThat(testHarness.getOutput(), containsInAnyOrder(expectedOutput.toArray())); + } + } + } + + /** + * Setup three inputs only two selected and make sure that neither of the two inputs is starved, + * when one has some data all the time, but the other only rarely. + */ + @Test + public void testInputStarvation() throws Exception { + try (StreamTaskMailboxTestHarness testHarness = + new MultipleInputStreamTaskTestHarnessBuilder<>(MultipleInputStreamTask::new, BasicTypeInfo.STRING_TYPE_INFO) + .addInput(BasicTypeInfo.STRING_TYPE_INFO) + .addInput(BasicTypeInfo.STRING_TYPE_INFO) + .addInput(BasicTypeInfo.STRING_TYPE_INFO) + .setupOutputForSingletonOperatorChain(new TestInputStarvationMultipleInputOperatorFactory()) + .build()) { + + Queue expectedOutput = new ArrayDeque<>(); + + testHarness.setAutoProcess(false); + // StreamMultipleInputProcessor starts with all inputs available. Let it rotate and refresh properly. + testHarness.processSingleStep(); + assertTrue(testHarness.getOutput().isEmpty()); + + testHarness.processElement(new StreamRecord<>("NOT_SELECTED"), 0); + + testHarness.processElement(new StreamRecord<>("1"), 1); + testHarness.processElement(new StreamRecord<>("2"), 1); + testHarness.processElement(new StreamRecord<>("3"), 1); + testHarness.processElement(new StreamRecord<>("4"), 1); + + testHarness.processSingleStep(); + expectedOutput.add(new StreamRecord<>("[2]: 1")); + testHarness.processSingleStep(); + expectedOutput.add(new StreamRecord<>("[2]: 2")); + assertThat(testHarness.getOutput(), contains(expectedOutput.toArray())); + + // InputGate 2 was not available in previous steps, so let's check if we are not starving it + testHarness.processElement(new StreamRecord<>("1"), 2); + testHarness.processSingleStep(); + testHarness.processSingleStep(); + + // One of those processing single step should pick up InputGate 2, however it's not + // important which one. We just must avoid starvation. + expectedOutput.add(new StreamRecord<>("[3]: 1")); + expectedOutput.add(new StreamRecord<>("[2]: 3")); + + assertThat(testHarness.getOutput(), containsInAnyOrder(expectedOutput.toArray())); + } + } + + // ------------------------------------------------------------------------ + // Utilities + // ------------------------------------------------------------------------ + + private static class SpecialRuleReadingStreamOperator extends AbstractStreamOperatorV2 + implements MultipleInputStreamOperator, InputSelectable, BoundedMultiInput { + + private final int input1Records; + private final int input2Records; + private final int maxContinuousReadingRecords; + + private int input1ReadingRecords; + private int input2ReadingRecords; + + private int continuousReadingRecords; + private InputSelection inputSelection = InputSelection.FIRST; + + SpecialRuleReadingStreamOperator(StreamOperatorParameters parameters, int input1Records, int input2Records, int maxContinuousReadingRecords) { + super(parameters, 2); + + this.input1Records = input1Records; + this.input2Records = input2Records; + this.maxContinuousReadingRecords = maxContinuousReadingRecords; + } + + @Override + public InputSelection nextSelection() { + return inputSelection; + } + + @Override + public void endInput(int inputId) { + inputSelection = (inputId == 1) ? InputSelection.SECOND : InputSelection.FIRST; + } + + @Override + public List getInputs() { + return Arrays.asList( + new ToStringInput(this, 1) { + @Override + public void processElement(StreamRecord element) { + super.processElement(element); + input1ReadingRecords++; + continuousReadingRecords++; + if (continuousReadingRecords == maxContinuousReadingRecords) { + continuousReadingRecords = 0; + if (input2ReadingRecords < input2Records) { + inputSelection = InputSelection.SECOND; + return; + } + } + + inputSelection = InputSelection.FIRST; + } + }, + new ToStringInput(this, 2) { + @Override + public void processElement(StreamRecord element) { + super.processElement(element); + input2ReadingRecords++; + continuousReadingRecords++; + if (continuousReadingRecords == maxContinuousReadingRecords) { + continuousReadingRecords = 0; + if (input1ReadingRecords < input1Records) { + inputSelection = InputSelection.FIRST; + return; + } + } + + inputSelection = InputSelection.SECOND; + } + } + ); + } + } + + private static class SpecialRuleReadingStreamOperatorFactory extends AbstractStreamOperatorFactory { + private final int input1Records; + private final int input2Records; + private final int maxContinuousReadingRecords; + + public SpecialRuleReadingStreamOperatorFactory( + int input1Records, + int input2Records, + int maxContinuousReadingRecords) { + this.input1Records = input1Records; + this.input2Records = input2Records; + this.maxContinuousReadingRecords = maxContinuousReadingRecords; + } + + @Override + public > T createStreamOperator(StreamOperatorParameters parameters) { + return (T) new SpecialRuleReadingStreamOperator(parameters, input1Records, input2Records, maxContinuousReadingRecords); + } + + @Override + public Class getStreamOperatorClass(ClassLoader classLoader) { + return SpecialRuleReadingStreamOperator.class; + } + } + + private static class TestReadFinishedInputStreamOperator extends TestAnyModeMultipleInputStreamOperator { + TestReadFinishedInputStreamOperator(StreamOperatorParameters parameters) { + super(parameters); + } + + @Override + public InputSelection nextSelection() { + return InputSelection.FIRST; + } + } + + private static class TestReadFinishedInputStreamOperatorFactory extends AbstractStreamOperatorFactory { + @Override + public > T createStreamOperator(StreamOperatorParameters parameters) { + return (T) new TestReadFinishedInputStreamOperator(parameters); + } + + @Override + public Class getStreamOperatorClass(ClassLoader classLoader) { + return TestReadFinishedInputStreamOperator.class; + } + } + + private static class TestInputStarvationMultipleInputOperator extends AbstractStreamOperatorV2 + implements MultipleInputStreamOperator, InputSelectable { + + public TestInputStarvationMultipleInputOperator(StreamOperatorParameters parameters) { + super(parameters, 3); + } + + @Override + public InputSelection nextSelection() { + return new InputSelection.Builder().select(2).select(3).build(); + } + + @Override + public List getInputs() { + return Arrays.asList( + new ToStringInput(this, 1), + new ToStringInput(this, 2), + new ToStringInput(this, 3)); + } + } + + private static class TestInputStarvationMultipleInputOperatorFactory extends AbstractStreamOperatorFactory { + @Override + public > T createStreamOperator(StreamOperatorParameters parameters) { + return (T) new TestInputStarvationMultipleInputOperator(parameters); + } + + @Override + public Class getStreamOperatorClass(ClassLoader classLoader) { + return TestInputStarvationMultipleInputOperator.class; + } + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TestAnyModeMultipleInputStreamOperator.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TestAnyModeMultipleInputStreamOperator.java new file mode 100644 index 0000000000..c19c4c2c37 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TestAnyModeMultipleInputStreamOperator.java @@ -0,0 +1,86 @@ +/* + * 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.util; + +import org.apache.flink.streaming.api.operators.AbstractInput; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorV2; +import org.apache.flink.streaming.api.operators.Input; +import org.apache.flink.streaming.api.operators.InputSelectable; +import org.apache.flink.streaming.api.operators.InputSelection; +import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; + +import java.util.Arrays; +import java.util.List; + +/** + * A test operator class for any mode reading. + */ +public class TestAnyModeMultipleInputStreamOperator extends AbstractStreamOperatorV2 + implements MultipleInputStreamOperator, InputSelectable { + + public TestAnyModeMultipleInputStreamOperator(StreamOperatorParameters parameters) { + super(parameters, 2); + } + + @Override + public InputSelection nextSelection() { + return InputSelection.ALL; + } + + @Override + public List getInputs() { + return Arrays.asList( + new ToStringInput(this, 1), + new ToStringInput(this, 2)); + } + + /** + * Factory to construct {@link TestAnyModeMultipleInputStreamOperator}. + */ + public static class Factory extends AbstractStreamOperatorFactory { + + @Override + public > T createStreamOperator(StreamOperatorParameters parameters) { + return (T) new TestAnyModeMultipleInputStreamOperator(parameters); + } + + @Override + public Class getStreamOperatorClass(ClassLoader classLoader) { + return TestAnyModeMultipleInputStreamOperator.class; + } + } + + /** + * {@link AbstractInput} that converts argument to string pre-pending input id. + */ + public static class ToStringInput extends AbstractInput { + public ToStringInput(AbstractStreamOperatorV2 owner, int inputId) { + super(owner, inputId); + } + + @Override + public void processElement(StreamRecord element) { + output.collect(element.replace(String.format("[%d]: %s", inputId, element.getValue()))); + } + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TestSequentialMultipleInputStreamOperator.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TestSequentialMultipleInputStreamOperator.java new file mode 100644 index 0000000000..c479500008 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TestSequentialMultipleInputStreamOperator.java @@ -0,0 +1,66 @@ +/* + * 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.util; + +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.BoundedMultiInput; +import org.apache.flink.streaming.api.operators.InputSelection; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; + +/** + * A test operator class for sequential reading. + */ +public class TestSequentialMultipleInputStreamOperator extends TestAnyModeMultipleInputStreamOperator + implements BoundedMultiInput { + + private InputSelection inputSelection = InputSelection.FIRST; + + public TestSequentialMultipleInputStreamOperator(StreamOperatorParameters parameters) { + super(parameters); + } + + @Override + public InputSelection nextSelection() { + return inputSelection; + } + + @Override + public void endInput(int inputId) { + if (inputId == 1) { + inputSelection = InputSelection.SECOND; + } + } + + /** + * Factory to construct {@link TestSequentialMultipleInputStreamOperator}. + */ + public static class Factory extends AbstractStreamOperatorFactory { + + @Override + public > T createStreamOperator(StreamOperatorParameters parameters) { + return (T) new TestSequentialMultipleInputStreamOperator(parameters); + } + + @Override + public Class getStreamOperatorClass(ClassLoader classLoader) { + return TestSequentialMultipleInputStreamOperator.class; + } + } +} -- Gitee From 750ef6837aaec571814eab7ce5fc5555e1a99b2c Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Fri, 27 Mar 2020 16:18:16 +0100 Subject: [PATCH 361/885] [hotfix][operators] Rename InputSelectable inputSelector field to inputSelectable --- .../runtime/io/MultipleInputSelectionHandler.java | 8 ++++---- .../streaming/runtime/io/TwoInputSelectionHandler.java | 8 ++++---- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/MultipleInputSelectionHandler.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/MultipleInputSelectionHandler.java index 9211c49011..f9e3ca76a3 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/MultipleInputSelectionHandler.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/MultipleInputSelectionHandler.java @@ -37,7 +37,7 @@ public class MultipleInputSelectionHandler { public static final int MAX_SUPPORTED_INPUT_COUNT = Long.SIZE; @Nullable - private final InputSelectable inputSelector; + private final InputSelectable inputSelectable; private InputSelection inputSelection = InputSelection.ALL; @@ -49,7 +49,7 @@ public class MultipleInputSelectionHandler { public MultipleInputSelectionHandler(@Nullable InputSelectable inputSelectable, int inputCount) { checkSupportedInputCount(inputCount); - this.inputSelector = inputSelectable; + this.inputSelectable = inputSelectable; this.allSelectedMask = (1 << inputCount) - 1; this.availableInputsMask = allSelectedMask; this.notFinishedInputsMask = allSelectedMask; @@ -98,10 +98,10 @@ public class MultipleInputSelectionHandler { } void nextSelection() { - if (inputSelector == null) { + if (inputSelectable == null) { inputSelection = InputSelection.ALL; } else { - inputSelection = inputSelector.nextSelection(); + inputSelection = inputSelectable.nextSelection(); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/TwoInputSelectionHandler.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/TwoInputSelectionHandler.java index bb84d565a6..48bf0fa4c2 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/TwoInputSelectionHandler.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/TwoInputSelectionHandler.java @@ -31,22 +31,22 @@ import javax.annotation.Nullable; public class TwoInputSelectionHandler { @Nullable - private final InputSelectable inputSelector; + private final InputSelectable inputSelectable; private InputSelection inputSelection; private int availableInputsMask; public TwoInputSelectionHandler(@Nullable InputSelectable inputSelectable) { - this.inputSelector = inputSelectable; + this.inputSelectable = inputSelectable; this.availableInputsMask = (int) new InputSelection.Builder().select(1).select(2).build().getInputMask(); } void nextSelection() { - if (inputSelector == null) { + if (inputSelectable == null) { inputSelection = InputSelection.ALL; } else { - inputSelection = inputSelector.nextSelection(); + inputSelection = inputSelectable.nextSelection(); } } -- Gitee From da3082764117841d885f41c645961f8993a331a0 Mon Sep 17 00:00:00 2001 From: vthinkxie Date: Tue, 24 Mar 2020 17:26:55 +0800 Subject: [PATCH 362/885] [FLINK-16741][web] Add TM log list and TM log detail page This closes #11498. --- .../src/app/interfaces/task-manager.ts | 4 + .../logs/job-manager-logs.component.html | 2 +- .../stdout/job-manager-stdout.component.html | 2 +- .../task-manager-log-detail.component.html | 36 ++++++++ .../task-manager-log-detail.component.less | 54 +++++++++++ .../task-manager-log-detail.component.ts | 89 +++++++++++++++++++ .../task-manager-log-list.component.html | 39 ++++++++ .../task-manager-log-list.component.ts | 50 +++++++++++ .../logs/task-manager-logs.component.html | 21 ----- .../logs/task-manager-logs.component.less | 28 ------ .../logs/task-manager-logs.component.ts | 60 ------------- .../status/task-manager-status.component.ts | 6 +- .../stdout/task-manager-stdout.component.html | 20 ----- .../stdout/task-manager-stdout.component.less | 28 ------ .../stdout/task-manager-stdout.component.ts | 60 ------------- .../task-manager-routing.module.ts | 25 ++++-- .../pages/task-manager/task-manager.module.ts | 8 +- .../src/app/services/task-manager.service.ts | 39 +++++--- .../common/navigation/navigation.component.ts | 28 +++--- .../refresh-download.component.html | 21 ++++- .../refresh-download.component.less | 4 + .../refresh-download.component.ts | 9 ++ 22 files changed, 370 insertions(+), 263 deletions(-) create mode 100644 flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-detail/task-manager-log-detail.component.html create mode 100644 flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-detail/task-manager-log-detail.component.less create mode 100644 flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-detail/task-manager-log-detail.component.ts create mode 100644 flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-list/task-manager-log-list.component.html create mode 100644 flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-list/task-manager-log-list.component.ts delete mode 100644 flink-runtime-web/web-dashboard/src/app/pages/task-manager/logs/task-manager-logs.component.html delete mode 100644 flink-runtime-web/web-dashboard/src/app/pages/task-manager/logs/task-manager-logs.component.less delete mode 100644 flink-runtime-web/web-dashboard/src/app/pages/task-manager/logs/task-manager-logs.component.ts delete mode 100644 flink-runtime-web/web-dashboard/src/app/pages/task-manager/stdout/task-manager-stdout.component.html delete mode 100644 flink-runtime-web/web-dashboard/src/app/pages/task-manager/stdout/task-manager-stdout.component.less delete mode 100644 flink-runtime-web/web-dashboard/src/app/pages/task-manager/stdout/task-manager-stdout.component.ts diff --git a/flink-runtime-web/web-dashboard/src/app/interfaces/task-manager.ts b/flink-runtime-web/web-dashboard/src/app/interfaces/task-manager.ts index f001f9d4e7..2172cdd4d3 100644 --- a/flink-runtime-web/web-dashboard/src/app/interfaces/task-manager.ts +++ b/flink-runtime-web/web-dashboard/src/app/interfaces/task-manager.ts @@ -31,6 +31,10 @@ export interface TaskManagerDetailInterface { metrics: Metrics; } +export interface TaskManagerLogInterface { + logs: { name: string; size: number }[]; +} + export interface TaskmanagersItemInterface { id: string; path: string; diff --git a/flink-runtime-web/web-dashboard/src/app/pages/job-manager/logs/job-manager-logs.component.html b/flink-runtime-web/web-dashboard/src/app/pages/job-manager/logs/job-manager-logs.component.html index 6ba7783345..f2f9461fe2 100644 --- a/flink-runtime-web/web-dashboard/src/app/pages/job-manager/logs/job-manager-logs.component.html +++ b/flink-runtime-web/web-dashboard/src/app/pages/job-manager/logs/job-manager-logs.component.html @@ -17,4 +17,4 @@ --> - + diff --git a/flink-runtime-web/web-dashboard/src/app/pages/job-manager/stdout/job-manager-stdout.component.html b/flink-runtime-web/web-dashboard/src/app/pages/job-manager/stdout/job-manager-stdout.component.html index 4221237c15..8528b2b0df 100644 --- a/flink-runtime-web/web-dashboard/src/app/pages/job-manager/stdout/job-manager-stdout.component.html +++ b/flink-runtime-web/web-dashboard/src/app/pages/job-manager/stdout/job-manager-stdout.component.html @@ -17,4 +17,4 @@ --> - + diff --git a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-detail/task-manager-log-detail.component.html b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-detail/task-manager-log-detail.component.html new file mode 100644 index 0000000000..b5a46adb39 --- /dev/null +++ b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-detail/task-manager-log-detail.component.html @@ -0,0 +1,36 @@ + + + + diff --git a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-detail/task-manager-log-detail.component.less b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-detail/task-manager-log-detail.component.less new file mode 100644 index 0000000000..d9bd69a31f --- /dev/null +++ b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-detail/task-manager-log-detail.component.less @@ -0,0 +1,54 @@ +/* + * 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. + */ + +@import "theme"; + +:host { + display: block; + height: 100%; + &.full-screen { + position: fixed; + top: 0; + bottom: 0; + right: 0; + left: 0; + background: @component-background; + z-index: 99; + flink-monaco-editor { + height: calc(~"100vh - 65px"); + } + } +} + +flink-monaco-editor { + height: calc(~"100vh - 386px"); + border: 1px solid @border-color-split; +} + +.breadcrumb { + background: @component-background; + border: 1px solid @border-color-split; + margin-bottom: 16px; + padding: 12px 24px; + position: relative; +} + +flink-refresh-download { + position: absolute; + right: 12px; + top: 0; + line-height: 47px; +} diff --git a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-detail/task-manager-log-detail.component.ts b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-detail/task-manager-log-detail.component.ts new file mode 100644 index 0000000000..f589122c7d --- /dev/null +++ b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-detail/task-manager-log-detail.component.ts @@ -0,0 +1,89 @@ +/* + * 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. + */ + +import { ChangeDetectionStrategy, ChangeDetectorRef, Component, OnInit, ViewChild } from '@angular/core'; +import { ActivatedRoute } from '@angular/router'; +import { TaskManagerDetailInterface } from 'interfaces'; +import { TaskManagerService } from 'services'; +import { first } from 'rxjs/operators'; +import { MonacoEditorComponent } from 'share/common/monaco-editor/monaco-editor.component'; + +@Component({ + selector: 'flink-task-manager-log-detail', + templateUrl: './task-manager-log-detail.component.html', + changeDetection: ChangeDetectionStrategy.OnPush, + host: { + '[class.full-screen]': 'isFullScreen' + }, + styleUrls: ['./task-manager-log-detail.component.less'] +}) +export class TaskManagerLogDetailComponent implements OnInit { + logs = ''; + logName = ''; + downloadUrl = ''; + isLoading = false; + taskManagerDetail: TaskManagerDetailInterface; + isFullScreen = false; + hasLogName = false; + @ViewChild(MonacoEditorComponent) monacoEditorComponent: MonacoEditorComponent; + + constructor( + private taskManagerService: TaskManagerService, + private cdr: ChangeDetectorRef, + private activatedRoute: ActivatedRoute + ) {} + + reloadLog() { + this.isLoading = true; + this.cdr.markForCheck(); + this.taskManagerService.loadLog(this.taskManagerDetail.id, this.logName, this.hasLogName).subscribe( + data => { + this.logs = data.data; + this.downloadUrl = data.url; + this.isLoading = false; + this.layoutEditor(); + this.cdr.markForCheck(); + }, + () => { + this.isLoading = false; + this.layoutEditor(); + this.cdr.markForCheck(); + } + ); + } + + toggleFullScreen(fullScreen: boolean) { + this.isFullScreen = fullScreen; + this.layoutEditor(); + } + + layoutEditor(): void { + setTimeout(() => this.monacoEditorComponent.layout()); + } + + ngOnInit() { + this.taskManagerService.taskManagerDetail$.pipe(first()).subscribe(data => { + this.taskManagerDetail = data; + this.hasLogName = this.activatedRoute.snapshot.data.hasLogName; + if (this.hasLogName) { + this.logName = this.activatedRoute.snapshot.params.logName; + } else { + this.logName = `taskmanager_${data.id}_log`; + } + this.reloadLog(); + }); + } +} diff --git a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-list/task-manager-log-list.component.html b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-list/task-manager-log-list.component.html new file mode 100644 index 0000000000..c406156269 --- /dev/null +++ b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-list/task-manager-log-list.component.html @@ -0,0 +1,39 @@ + + + + +

    + + + + + + + + + + + + + diff --git a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-list/task-manager-log-list.component.ts b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-list/task-manager-log-list.component.ts new file mode 100644 index 0000000000..f565759500 --- /dev/null +++ b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/log-list/task-manager-log-list.component.ts @@ -0,0 +1,50 @@ +/* + * 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. + */ + +import { ChangeDetectionStrategy, ChangeDetectorRef, Component, OnInit } from '@angular/core'; +import { first, flatMap } from 'rxjs/operators'; +import { TaskManagerService } from 'services'; + +@Component({ + selector: 'flink-task-manager-log-list', + templateUrl: './task-manager-log-list.component.html', + changeDetection: ChangeDetectionStrategy.OnPush +}) +export class TaskManagerLogListComponent implements OnInit { + listOfLog: { name: string; size: number }[] = []; + isLoading = true; + + constructor(private taskManagerService: TaskManagerService, private cdr: ChangeDetectorRef) {} + + ngOnInit() { + this.taskManagerService.taskManagerDetail$ + .pipe( + first(), + flatMap(data => this.taskManagerService.loadLogList(data.id)) + ) + .subscribe( + data => { + this.listOfLog = data; + this.isLoading = false; + this.cdr.markForCheck(); + }, + () => { + this.isLoading = false; + this.cdr.markForCheck(); + } + ); + } +} diff --git a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/logs/task-manager-logs.component.html b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/logs/task-manager-logs.component.html deleted file mode 100644 index e48b7dab3d..0000000000 --- a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/logs/task-manager-logs.component.html +++ /dev/null @@ -1,21 +0,0 @@ - - - - - diff --git a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/logs/task-manager-logs.component.less b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/logs/task-manager-logs.component.less deleted file mode 100644 index df8052556a..0000000000 --- a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/logs/task-manager-logs.component.less +++ /dev/null @@ -1,28 +0,0 @@ -/* - * 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. - */ -@import "theme"; - -flink-monaco-editor { - height: calc(~"100vh - 310px"); -} - -:host { - position: relative; - display: block; - border: 1px solid @border-color-split; -} diff --git a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/logs/task-manager-logs.component.ts b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/logs/task-manager-logs.component.ts deleted file mode 100644 index 90f089366f..0000000000 --- a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/logs/task-manager-logs.component.ts +++ /dev/null @@ -1,60 +0,0 @@ -/* - * 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. - */ - -import { ChangeDetectorRef, Component, OnInit, ViewChild, ChangeDetectionStrategy } from '@angular/core'; -import { TaskManagerDetailInterface } from 'interfaces'; -import { first } from 'rxjs/operators'; -import { TaskManagerService } from 'services'; -import { MonacoEditorComponent } from 'share/common/monaco-editor/monaco-editor.component'; - -@Component({ - selector: 'flink-task-manager-logs', - templateUrl: './task-manager-logs.component.html', - styleUrls: ['./task-manager-logs.component.less'], - changeDetection: ChangeDetectionStrategy.OnPush -}) -export class TaskManagerLogsComponent implements OnInit { - @ViewChild(MonacoEditorComponent) monacoEditorComponent: MonacoEditorComponent; - logs = ''; - taskManagerDetail: TaskManagerDetailInterface; - - reload() { - if (this.taskManagerDetail) { - this.taskManagerService.loadLogs(this.taskManagerDetail.id).subscribe( - data => { - this.monacoEditorComponent.layout(); - this.logs = data; - this.cdr.markForCheck(); - }, - () => { - this.cdr.markForCheck(); - } - ); - } - } - - constructor(private taskManagerService: TaskManagerService, private cdr: ChangeDetectorRef) {} - - ngOnInit() { - this.taskManagerService.taskManagerDetail$.pipe(first()).subscribe(data => { - this.taskManagerDetail = data; - this.reload(); - this.cdr.markForCheck(); - }); - } -} diff --git a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/status/task-manager-status.component.ts b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/status/task-manager-status.component.ts index d54e4ec077..b244445a23 100644 --- a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/status/task-manager-status.component.ts +++ b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/status/task-manager-status.component.ts @@ -30,11 +30,7 @@ import { TaskManagerService } from 'services'; }) export class TaskManagerStatusComponent implements OnInit, OnDestroy { @Input() isLoading = true; - listOfNavigation = [ - { path: 'metrics', title: 'Metrics' }, - { path: 'logs', title: 'Logs' }, - { path: 'stdout', title: 'Stdout' } - ]; + listOfNavigation = [{ path: 'metrics', title: 'Metrics' }, { path: 'log-list', title: 'Log' }]; taskManagerDetail: TaskManagerDetailInterface; private destroy$ = new Subject(); diff --git a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/stdout/task-manager-stdout.component.html b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/stdout/task-manager-stdout.component.html deleted file mode 100644 index 31f9dbf942..0000000000 --- a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/stdout/task-manager-stdout.component.html +++ /dev/null @@ -1,20 +0,0 @@ - - - - diff --git a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/stdout/task-manager-stdout.component.less b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/stdout/task-manager-stdout.component.less deleted file mode 100644 index df8052556a..0000000000 --- a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/stdout/task-manager-stdout.component.less +++ /dev/null @@ -1,28 +0,0 @@ -/* - * 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. - */ -@import "theme"; - -flink-monaco-editor { - height: calc(~"100vh - 310px"); -} - -:host { - position: relative; - display: block; - border: 1px solid @border-color-split; -} diff --git a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/stdout/task-manager-stdout.component.ts b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/stdout/task-manager-stdout.component.ts deleted file mode 100644 index a674f09351..0000000000 --- a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/stdout/task-manager-stdout.component.ts +++ /dev/null @@ -1,60 +0,0 @@ -/* - * 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. - */ - -import { ChangeDetectionStrategy, ChangeDetectorRef, Component, OnInit, ViewChild } from '@angular/core'; -import { TaskManagerDetailInterface } from 'interfaces'; -import { first } from 'rxjs/operators'; -import { TaskManagerService } from 'services'; -import { MonacoEditorComponent } from 'share/common/monaco-editor/monaco-editor.component'; - -@Component({ - selector: 'flink-task-manager-stdout', - templateUrl: './task-manager-stdout.component.html', - styleUrls: ['./task-manager-stdout.component.less'], - changeDetection: ChangeDetectionStrategy.OnPush -}) -export class TaskManagerStdoutComponent implements OnInit { - @ViewChild(MonacoEditorComponent) monacoEditorComponent: MonacoEditorComponent; - stdout = ''; - taskManagerDetail: TaskManagerDetailInterface; - - reload() { - if (this.taskManagerDetail) { - this.taskManagerService.loadStdout(this.taskManagerDetail.id).subscribe( - data => { - this.monacoEditorComponent.layout(); - this.stdout = data; - this.cdr.markForCheck(); - }, - () => { - this.cdr.markForCheck(); - } - ); - } - } - - constructor(private taskManagerService: TaskManagerService, private cdr: ChangeDetectorRef) {} - - ngOnInit() { - this.taskManagerService.taskManagerDetail$.pipe(first()).subscribe(data => { - this.taskManagerDetail = data; - this.reload(); - this.cdr.markForCheck(); - }); - } -} diff --git a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/task-manager-routing.module.ts b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/task-manager-routing.module.ts index 0f5c7113ee..25e0ae107c 100644 --- a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/task-manager-routing.module.ts +++ b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/task-manager-routing.module.ts @@ -18,11 +18,11 @@ import { NgModule } from '@angular/core'; import { Routes, RouterModule } from '@angular/router'; +import { TaskManagerLogDetailComponent } from './log-detail/task-manager-log-detail.component'; +import { TaskManagerLogListComponent } from './log-list/task-manager-log-list.component'; import { TaskManagerComponent } from './task-manager.component'; import { TaskManagerListComponent } from './list/task-manager-list.component'; -import { TaskManagerLogsComponent } from './logs/task-manager-logs.component'; import { TaskManagerMetricsComponent } from './metrics/task-manager-metrics.component'; -import { TaskManagerStdoutComponent } from './stdout/task-manager-stdout.component'; const routes: Routes = [ { @@ -41,17 +41,26 @@ const routes: Routes = [ } }, { - path: 'logs', - component: TaskManagerLogsComponent, + path: 'log-list', + component: TaskManagerLogListComponent, + data: { + path: 'log-list' + } + }, + { + path: 'log-list/:logName', + component: TaskManagerLogDetailComponent, data: { - path: 'logs' + path: 'log-list', + hasLogName: true } }, { - path: 'stdout', - component: TaskManagerStdoutComponent, + path: 'logs', + component: TaskManagerLogDetailComponent, data: { - path: 'stdout' + path: 'log-list', + hasLogName: false } }, { diff --git a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/task-manager.module.ts b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/task-manager.module.ts index 0a521b7f46..d2ea2cbe2d 100644 --- a/flink-runtime-web/web-dashboard/src/app/pages/task-manager/task-manager.module.ts +++ b/flink-runtime-web/web-dashboard/src/app/pages/task-manager/task-manager.module.ts @@ -19,14 +19,14 @@ import { NgModule } from '@angular/core'; import { CommonModule } from '@angular/common'; import { ShareModule } from 'share/share.module'; +import { TaskManagerLogDetailComponent } from './log-detail/task-manager-log-detail.component'; +import { TaskManagerLogListComponent } from './log-list/task-manager-log-list.component'; import { TaskManagerRoutingModule } from './task-manager-routing.module'; import { TaskManagerListComponent } from './list/task-manager-list.component'; import { TaskManagerMetricsComponent } from './metrics/task-manager-metrics.component'; import { TaskManagerComponent } from './task-manager.component'; import { TaskManagerStatusComponent } from './status/task-manager-status.component'; -import { TaskManagerLogsComponent } from './logs/task-manager-logs.component'; -import { TaskManagerStdoutComponent } from './stdout/task-manager-stdout.component'; @NgModule({ imports: [CommonModule, ShareModule, TaskManagerRoutingModule], @@ -35,8 +35,8 @@ import { TaskManagerStdoutComponent } from './stdout/task-manager-stdout.compone TaskManagerMetricsComponent, TaskManagerComponent, TaskManagerStatusComponent, - TaskManagerLogsComponent, - TaskManagerStdoutComponent + TaskManagerLogListComponent, + TaskManagerLogDetailComponent ] }) export class TaskManagerModule {} diff --git a/flink-runtime-web/web-dashboard/src/app/services/task-manager.service.ts b/flink-runtime-web/web-dashboard/src/app/services/task-manager.service.ts index f2b3ff9686..bd790bf07a 100644 --- a/flink-runtime-web/web-dashboard/src/app/services/task-manager.service.ts +++ b/flink-runtime-web/web-dashboard/src/app/services/task-manager.service.ts @@ -21,7 +21,7 @@ import { Injectable } from '@angular/core'; import { EMPTY, of, ReplaySubject } from 'rxjs'; import { catchError, map } from 'rxjs/operators'; import { BASE_URL } from 'config'; -import { TaskManagerListInterface, TaskManagerDetailInterface } from 'interfaces'; +import { TaskManagerListInterface, TaskManagerDetailInterface, TaskManagerLogInterface } from 'interfaces'; @Injectable({ providedIn: 'root' @@ -50,25 +50,38 @@ export class TaskManagerService { } /** - * Load TM logs + * Load TM log list * @param taskManagerId */ - loadLogs(taskManagerId: string) { - return this.httpClient.get(`${BASE_URL}/taskmanagers/${taskManagerId}/log`, { - responseType: 'text', - headers: new HttpHeaders().append('Cache-Control', 'no-cache') - }); + loadLogList(taskManagerId: string) { + return this.httpClient + .get(`${BASE_URL}/taskmanagers/${taskManagerId}/logs`) + .pipe(map(data => data.logs)); } /** - * Load TM stdout + * Load TM log * @param taskManagerId + * @param logName + * @param hasLogName */ - loadStdout(taskManagerId: string) { - return this.httpClient.get(`${BASE_URL}/taskmanagers/${taskManagerId}/stdout`, { - responseType: 'text', - headers: new HttpHeaders().append('Cache-Control', 'no-cache') - }); + loadLog(taskManagerId: string, logName: string, hasLogName: boolean) { + let url = ''; + if (hasLogName) { + url = `${BASE_URL}/taskmanagers/${taskManagerId}/logs/${logName}`; + } else { + url = `${BASE_URL}/taskmanagers/${taskManagerId}/log`; + } + return this.httpClient + .get(url, { responseType: 'text', headers: new HttpHeaders().append('Cache-Control', 'no-cache') }) + .pipe( + map(data => { + return { + data, + url + }; + }) + ); } constructor(private httpClient: HttpClient) {} diff --git a/flink-runtime-web/web-dashboard/src/app/share/common/navigation/navigation.component.ts b/flink-runtime-web/web-dashboard/src/app/share/common/navigation/navigation.component.ts index a5b1d6068e..b48e4e01d4 100644 --- a/flink-runtime-web/web-dashboard/src/app/share/common/navigation/navigation.component.ts +++ b/flink-runtime-web/web-dashboard/src/app/share/common/navigation/navigation.component.ts @@ -17,9 +17,9 @@ */ import { ChangeDetectionStrategy, ChangeDetectorRef, Component, Input, OnDestroy, OnInit } from '@angular/core'; -import { ActivatedRoute, Router } from '@angular/router'; +import { ActivatedRoute, NavigationEnd, Router } from '@angular/router'; import { Subject } from 'rxjs'; -import { map, takeUntil } from 'rxjs/operators'; +import { filter, flatMap, map, startWith, takeUntil } from 'rxjs/operators'; @Component({ selector: 'flink-navigation', @@ -41,17 +41,19 @@ export class NavigationComponent implements OnInit, OnDestroy { constructor(private activatedRoute: ActivatedRoute, private router: Router, private cdr: ChangeDetectorRef) {} ngOnInit() { - if (this.activatedRoute && this.activatedRoute.firstChild) { - this.activatedRoute.firstChild.data - .pipe( - takeUntil(this.destroy$), - map(data => data.path) - ) - .subscribe(data => { - this.navIndex = this.listOfNavigation.map(nav => nav.path).indexOf(data); - this.cdr.markForCheck(); - }); - } + this.router.events + .pipe( + filter(e => e instanceof NavigationEnd), + startWith(true), + filter(() => !!(this.activatedRoute && this.activatedRoute.firstChild)), + flatMap(() => this.activatedRoute!.firstChild!.data), + takeUntil(this.destroy$), + map(data => data.path) + ) + .subscribe(data => { + this.navIndex = this.listOfNavigation.map(nav => nav.path).indexOf(data); + this.cdr.markForCheck(); + }); } ngOnDestroy() { diff --git a/flink-runtime-web/web-dashboard/src/app/share/customize/refresh-download/refresh-download.component.html b/flink-runtime-web/web-dashboard/src/app/share/customize/refresh-download/refresh-download.component.html index 41e94fdf92..8994733cb4 100644 --- a/flink-runtime-web/web-dashboard/src/app/share/customize/refresh-download/refresh-download.component.html +++ b/flink-runtime-web/web-dashboard/src/app/share/customize/refresh-download/refresh-download.component.html @@ -16,7 +16,26 @@ ~ limitations under the License. --> - + + + + + Loading... + + Reload + + + Download + + + + Exit FullScreen + + + FullScreen + + + diff --git a/flink-runtime-web/web-dashboard/src/app/share/customize/refresh-download/refresh-download.component.less b/flink-runtime-web/web-dashboard/src/app/share/customize/refresh-download/refresh-download.component.less index c80e517eb4..b9f7bf224b 100644 --- a/flink-runtime-web/web-dashboard/src/app/share/customize/refresh-download/refresh-download.component.less +++ b/flink-runtime-web/web-dashboard/src/app/share/customize/refresh-download/refresh-download.component.less @@ -21,3 +21,7 @@ top: 8px; right: 32px; } + +.operate-icon { + margin-right: 6px; +} diff --git a/flink-runtime-web/web-dashboard/src/app/share/customize/refresh-download/refresh-download.component.ts b/flink-runtime-web/web-dashboard/src/app/share/customize/refresh-download/refresh-download.component.ts index a134675106..506ae7e223 100644 --- a/flink-runtime-web/web-dashboard/src/app/share/customize/refresh-download/refresh-download.component.ts +++ b/flink-runtime-web/web-dashboard/src/app/share/customize/refresh-download/refresh-download.component.ts @@ -27,5 +27,14 @@ import { Component, ChangeDetectionStrategy, Input, Output, EventEmitter } from export class RefreshDownloadComponent { @Input() downloadName: string; @Input() downloadHref: string; + @Input() isLoading = false; + @Input() compactMode = false; @Output() reload = new EventEmitter(); + @Output() fullScreen = new EventEmitter(); + isFullScreen = false; + + toggleFullScreen() { + this.isFullScreen = !this.isFullScreen; + this.fullScreen.emit(this.isFullScreen); + } } -- Gitee From 919075a5f8aef717071bb9ea80947b7b8559e90d Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Sat, 28 Mar 2020 23:04:03 +0800 Subject: [PATCH 363/885] [FLINK-16847][python] Support LocalZonedTimestampType in vectorized Python UDF --- .../pyflink/fn_execution/coder_impl.py | 41 +++++++++-- flink-python/pyflink/fn_execution/coders.py | 65 +++++++++-------- .../pyflink/table/tests/test_pandas_udf.py | 46 +++++++++++++ flink-python/pyflink/table/types.py | 55 +++++++++++++++ .../flink/table/runtime/arrow/ArrowUtils.java | 29 +++++++- .../arrow/readers/TimestampFieldReader.java | 65 +++++++++++++++++ .../vectors/ArrowTimestampColumnVector.java | 69 +++++++++++++++++++ .../arrow/writers/BaseRowTimestampWriter.java | 67 ++++++++++++++++++ .../arrow/writers/TimestampWriter.java | 66 ++++++++++++++++++ .../runtime/typeutils/PythonTypeUtils.java | 20 ++++++ .../table/runtime/arrow/ArrowUtilsTest.java | 13 ++++ .../arrow/BaseRowArrowReaderWriterTest.java | 22 ++++-- .../arrow/RowArrowReaderWriterTest.java | 17 +++-- .../table/runtime/util/StreamRecordUtils.java | 5 ++ 14 files changed, 536 insertions(+), 44 deletions(-) create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/TimestampFieldReader.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowTimestampColumnVector.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowTimestampWriter.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/TimestampWriter.java diff --git a/flink-python/pyflink/fn_execution/coder_impl.py b/flink-python/pyflink/fn_execution/coder_impl.py index ca7ea64db0..743ce7b754 100644 --- a/flink-python/pyflink/fn_execution/coder_impl.py +++ b/flink-python/pyflink/fn_execution/coder_impl.py @@ -23,11 +23,12 @@ from typing import Any from typing import Generator from typing import List +import pandas as pd import pyarrow as pa from apache_beam.coders.coder_impl import StreamCoderImpl, create_InputStream, create_OutputStream from pyflink.fn_execution.ResettableIO import ResettableIO -from pyflink.table.types import Row +from pyflink.table.types import Row, DataType, LocalZonedTimestampType class FlattenRowCoderImpl(StreamCoderImpl): @@ -420,8 +421,10 @@ class LocalZonedTimestampCoderImpl(TimestampCoderImpl): class ArrowCoderImpl(StreamCoderImpl): - def __init__(self, schema): + def __init__(self, schema, row_type, timezone): self._schema = schema + self._field_types = row_type.field_types() + self._timezone = timezone self._resettable_io = ResettableIO() self._batch_reader = ArrowCoderImpl._load_from_stream(self._resettable_io) self._batch_writer = pa.RecordBatchStreamWriter(self._resettable_io, self._schema) @@ -455,14 +458,44 @@ class ArrowCoderImpl(StreamCoderImpl): "pyarrow.Array (%s)." raise RuntimeError(error_msg % (s.dtype, t), e) - arrays = [create_array(cols[i], self._schema.types[i]) for i in range(0, len(self._schema))] + arrays = [create_array( + ArrowCoderImpl.tz_convert_to_internal(cols[i], self._field_types[i], self._timezone), + self._schema.types[i]) for i in range(0, len(self._schema))] return pa.RecordBatch.from_arrays(arrays, self._schema) def _decode_one_batch_from_stream(self, in_stream: create_InputStream) -> List: self._resettable_io.set_input_bytes(in_stream.read_all(True)) # there is only one arrow batch in the underlying input stream table = pa.Table.from_batches([next(self._batch_reader)]) - return [c.to_pandas(date_as_object=True) for c in table.itercolumns()] + return [ArrowCoderImpl.tz_convert_from_internal( + c.to_pandas(date_as_object=True), t, self._timezone) + for c, t in zip(table.itercolumns(), self._field_types)] + + @staticmethod + def tz_convert_from_internal(s: pd.Series, t: DataType, local_tz) -> pd.Series: + """ + Converts the timestamp series from internal according to the specified local timezone. + + Returns the same series if the series is not a timestamp series. Otherwise, + returns a converted series. + """ + if type(t) == LocalZonedTimestampType: + return s.dt.tz_localize(local_tz) + else: + return s + + @staticmethod + def tz_convert_to_internal(s: pd.Series, t: DataType, local_tz) -> pd.Series: + """ + Converts the timestamp series to internal according to the specified local timezone. + """ + from pandas.api.types import is_datetime64_dtype, is_datetime64tz_dtype + if type(t) == LocalZonedTimestampType: + if is_datetime64_dtype(s.dtype): + return s.dt.tz_localize(None) + elif is_datetime64tz_dtype(s.dtype): + return s.dt.tz_convert(local_tz).dt.tz_localize(None) + return s def __repr__(self): return 'ArrowCoderImpl[%s]' % self._schema diff --git a/flink-python/pyflink/fn_execution/coders.py b/flink-python/pyflink/fn_execution/coders.py index a9da9bfe89..361af5a0b7 100644 --- a/flink-python/pyflink/fn_execution/coders.py +++ b/flink-python/pyflink/fn_execution/coders.py @@ -32,7 +32,9 @@ from apache_beam.typehints import typehints from pyflink.fn_execution import coder_impl from pyflink.fn_execution import flink_fn_execution_pb2 from pyflink.fn_execution.sdk_worker_main import pipeline_options -from pyflink.table.types import Row +from pyflink.table.types import Row, TinyIntType, SmallIntType, IntType, BigIntType, BooleanType, \ + FloatType, DoubleType, VarCharType, VarBinaryType, DecimalType, DateType, TimeType, \ + LocalZonedTimestampType, RowType, RowField, to_arrow_type FLINK_SCALAR_FUNCTION_SCHEMA_CODER_URN = "flink:coder:schema:scalar_function:v1" FLINK_TABLE_FUNCTION_SCHEMA_CODER_URN = "flink:coder:schema:table_function:v1" @@ -400,11 +402,13 @@ class ArrowCoder(DeterministicCoder): """ Coder for Arrow. """ - def __init__(self, schema): + def __init__(self, schema, row_type, timezone): self._schema = schema + self._row_type = row_type + self._timezone = timezone def _create_impl(self): - return coder_impl.ArrowCoderImpl(self._schema) + return coder_impl.ArrowCoderImpl(self._schema, self._row_type, self._timezone) def to_type_hint(self): import pandas as pd @@ -413,48 +417,51 @@ class ArrowCoder(DeterministicCoder): @Coder.register_urn(FLINK_SCALAR_FUNCTION_SCHEMA_ARROW_CODER_URN, flink_fn_execution_pb2.Schema) def _pickle_from_runner_api_parameter(schema_proto, unused_components, unused_context): - def _to_arrow_type(field): + def _to_arrow_schema(row_type): + return pa.schema([pa.field(n, to_arrow_type(t), t._nullable) + for n, t in zip(row_type.field_names(), row_type.field_types())]) + + def _to_data_type(field): if field.type.type_name == flink_fn_execution_pb2.Schema.TypeName.TINYINT: - return pa.field(field.name, pa.int8(), field.type.nullable) + return TinyIntType(field.type.nullable) elif field.type.type_name == flink_fn_execution_pb2.Schema.TypeName.SMALLINT: - return pa.field(field.name, pa.int16(), field.type.nullable) + return SmallIntType(field.type.nullable) elif field.type.type_name == flink_fn_execution_pb2.Schema.TypeName.INT: - return pa.field(field.name, pa.int32(), field.type.nullable) + return IntType(field.type.nullable) elif field.type.type_name == flink_fn_execution_pb2.Schema.TypeName.BIGINT: - return pa.field(field.name, pa.int64(), field.type.nullable) + return BigIntType(field.type.nullable) elif field.type.type_name == flink_fn_execution_pb2.Schema.TypeName.BOOLEAN: - return pa.field(field.name, pa.bool_(), field.type.nullable) + return BooleanType(field.type.nullable) elif field.type.type_name == flink_fn_execution_pb2.Schema.TypeName.FLOAT: - return pa.field(field.name, pa.float32(), field.type.nullable) + return FloatType(field.type.nullable) elif field.type.type_name == flink_fn_execution_pb2.Schema.TypeName.DOUBLE: - return pa.field(field.name, pa.float64(), field.type.nullable) + return DoubleType(field.type.nullable) elif field.type.type_name == flink_fn_execution_pb2.Schema.TypeName.VARCHAR: - return pa.field(field.name, pa.utf8(), field.type.nullable) + return VarCharType(0x7fffffff, field.type.nullable) elif field.type.type_name == flink_fn_execution_pb2.Schema.TypeName.VARBINARY: - return pa.field(field.name, pa.binary(), field.type.nullable) + return VarBinaryType(0x7fffffff, field.type.nullable) elif field.type.type_name == flink_fn_execution_pb2.Schema.TypeName.DECIMAL: - return pa.field(field.name, - pa.decimal128(field.type.decimal_info.precision, - field.type.decimal_info.scale), - field.type.nullable) + return DecimalType(field.type.decimal_info.precision, + field.type.decimal_info.scale, + field.type.nullable) elif field.type.type_name == flink_fn_execution_pb2.Schema.TypeName.DATE: - return pa.field(field.name, pa.date32(), field.type.nullable) + return DateType(field.type.nullable) elif field.type.type_name == flink_fn_execution_pb2.Schema.TypeName.TIME: - if field.type.time_info.precision == 0: - return pa.field(field.name, pa.time32('s'), field.type.nullable) - elif 1 <= field.type.time_type.precision <= 3: - return pa.field(field.name, pa.time32('ms'), field.type.nullable) - elif 4 <= field.type.time_type.precision <= 6: - return pa.field(field.name, pa.time64('us'), field.type.nullable) - else: - return pa.field(field.name, pa.time64('ns'), field.type.nullable) + return TimeType(field.type.time_info.precision, field.type.nullable) + elif field.type.type_name == \ + flink_fn_execution_pb2.Schema.TypeName.LOCAL_ZONED_TIMESTAMP: + return LocalZonedTimestampType(field.type.local_zoned_timestamp_info.precision, + field.type.nullable) else: raise ValueError("field_type %s is not supported." % field.type) - def _to_arrow_schema(row_schema): - return pa.schema([_to_arrow_type(f) for f in row_schema.fields]) + def _to_row_type(row_schema): + return RowType([RowField(f.name, _to_data_type(f)) for f in row_schema.fields]) - return ArrowCoder(_to_arrow_schema(schema_proto)) + timezone = pytz.timezone(pipeline_options.view_as(DebugOptions).lookup_experiment( + "table.exec.timezone")) + row_type = _to_row_type(schema_proto) + return ArrowCoder(_to_arrow_schema(row_type), row_type, timezone) def __repr__(self): return 'ArrowCoder[%s]' % self._schema diff --git a/flink-python/pyflink/table/tests/test_pandas_udf.py b/flink-python/pyflink/table/tests/test_pandas_udf.py index 95468e0c11..878dbb3f41 100644 --- a/flink-python/pyflink/table/tests/test_pandas_udf.py +++ b/flink-python/pyflink/table/tests/test_pandas_udf.py @@ -19,6 +19,8 @@ import datetime import decimal import unittest +import pytz + from pyflink.table import DataTypes from pyflink.table.tests.test_udf import SubtractOne from pyflink.table.udf import udf @@ -243,20 +245,64 @@ class PandasUDFITTests(object): "1000000000000000000.059999999999999999,2014-09-13,01:00:01"]) +class BlinkPandasUDFITTests(object): + + def test_data_types_only_supported_in_blink_planner(self): + import pandas as pd + + timezone = self.t_env.get_config().get_local_timezone() + local_datetime = pytz.timezone(timezone).localize( + datetime.datetime(1970, 1, 1, 0, 0, 0, 123000)) + + def local_zoned_timestamp_func(local_zoned_timestamp_param): + assert isinstance(local_zoned_timestamp_param, pd.Series) + assert isinstance(local_zoned_timestamp_param[0], datetime.datetime), \ + 'local_zoned_timestamp_param of wrong type %s !' % type( + local_zoned_timestamp_param[0]) + assert local_zoned_timestamp_param[0] == local_datetime, \ + 'local_zoned_timestamp_param is wrong value %s, %s!' % \ + (local_zoned_timestamp_param[0], local_datetime) + return local_zoned_timestamp_param + + self.t_env.register_function( + "local_zoned_timestamp_func", + udf(local_zoned_timestamp_func, + [DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3)], + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3), + udf_type="pandas")) + + table_sink = source_sink_utils.TestAppendSink( + ['a'], [DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3)]) + self.t_env.register_table_sink("Results", table_sink) + + t = self.t_env.from_elements( + [(local_datetime,)], + DataTypes.ROW([DataTypes.FIELD("a", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3))])) + + t.select("local_zoned_timestamp_func(local_zoned_timestamp_func(a))") \ + .insert_into("Results") + self.t_env.execute("test") + actual = source_sink_utils.results() + self.assert_equals(actual, ["1970-01-01T00:00:00.123Z"]) + + class StreamPandasUDFITTests(PandasUDFITTests, PyFlinkStreamTableTestCase): pass class BlinkBatchPandasUDFITTests(PandasUDFITTests, + BlinkPandasUDFITTests, PyFlinkBlinkBatchTableTestCase): pass class BlinkStreamPandasUDFITTests(PandasUDFITTests, + BlinkPandasUDFITTests, PyFlinkBlinkStreamTableTestCase): pass + @udf(input_types=[DataTypes.BIGINT(), DataTypes.BIGINT()], result_type=DataTypes.BIGINT(), udf_type='pandas') def add(i, j): diff --git a/flink-python/pyflink/table/types.py b/flink-python/pyflink/table/types.py index c193bc3aeb..d45a62f1f4 100644 --- a/flink-python/pyflink/table/types.py +++ b/flink-python/pyflink/table/types.py @@ -1179,6 +1179,12 @@ class RowType(DataType): """ return list(self.names) + def field_types(self): + """ + Returns all field types in a list. + """ + return list([f.data_type for f in self.fields]) + def need_conversion(self): # We need convert Row()/namedtuple into tuple() return True @@ -2282,6 +2288,55 @@ def _create_type_verifier(data_type, name=None): return verify +def to_arrow_type(data_type): + """ + Converts the specified Flink data type to pyarrow data type. + """ + import pyarrow as pa + if type(data_type) == TinyIntType: + return pa.int8() + elif type(data_type) == SmallIntType: + return pa.int16() + elif type(data_type) == IntType: + return pa.int32() + elif type(data_type) == BigIntType: + return pa.int64() + elif type(data_type) == BooleanType: + return pa.bool_() + elif type(data_type) == FloatType: + return pa.float32() + elif type(data_type) == DoubleType: + return pa.float64() + elif type(data_type) == VarCharType: + return pa.utf8() + elif type(data_type) == VarBinaryType: + return pa.binary() + elif type(data_type) == DecimalType: + return pa.decimal128(data_type.precision, data_type.scale) + elif type(data_type) == DateType: + return pa.date32() + elif type(data_type) == TimeType: + if data_type.precision == 0: + return pa.time32('s') + elif 1 <= data_type.precision <= 3: + return pa.time32('ms') + elif 4 <= data_type.precision <= 6: + return pa.time64('us') + else: + return pa.time64('ns') + elif type(data_type) == LocalZonedTimestampType: + if data_type.precision == 0: + return pa.timestamp('s') + elif 1 <= data_type.precision <= 3: + return pa.timestamp('ms') + elif 4 <= data_type.precision <= 6: + return pa.timestamp('us') + else: + return pa.timestamp('ns') + else: + raise ValueError("field_type %s is not supported." % data_type) + + class DataTypes(object): """ A :class:`DataType` can be used to declare input and/or output types of operations. diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java index 4ad3af7e7c..ac92cf2485 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java @@ -32,6 +32,7 @@ import org.apache.flink.table.runtime.arrow.readers.IntFieldReader; import org.apache.flink.table.runtime.arrow.readers.RowArrowReader; import org.apache.flink.table.runtime.arrow.readers.SmallIntFieldReader; import org.apache.flink.table.runtime.arrow.readers.TimeFieldReader; +import org.apache.flink.table.runtime.arrow.readers.TimestampFieldReader; import org.apache.flink.table.runtime.arrow.readers.TinyIntFieldReader; import org.apache.flink.table.runtime.arrow.readers.VarBinaryFieldReader; import org.apache.flink.table.runtime.arrow.readers.VarCharFieldReader; @@ -44,6 +45,7 @@ import org.apache.flink.table.runtime.arrow.vectors.ArrowFloatColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowSmallIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowTimeColumnVector; +import org.apache.flink.table.runtime.arrow.vectors.ArrowTimestampColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowTinyIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowVarBinaryColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowVarCharColumnVector; @@ -58,6 +60,7 @@ import org.apache.flink.table.runtime.arrow.writers.BaseRowFloatWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowSmallIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowTimeWriter; +import org.apache.flink.table.runtime.arrow.writers.BaseRowTimestampWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowTinyIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowVarBinaryWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowVarCharWriter; @@ -70,6 +73,7 @@ import org.apache.flink.table.runtime.arrow.writers.FloatWriter; import org.apache.flink.table.runtime.arrow.writers.IntWriter; import org.apache.flink.table.runtime.arrow.writers.SmallIntWriter; import org.apache.flink.table.runtime.arrow.writers.TimeWriter; +import org.apache.flink.table.runtime.arrow.writers.TimestampWriter; import org.apache.flink.table.runtime.arrow.writers.TinyIntWriter; import org.apache.flink.table.runtime.arrow.writers.VarBinaryWriter; import org.apache.flink.table.runtime.arrow.writers.VarCharWriter; @@ -81,6 +85,7 @@ import org.apache.flink.table.types.logical.DoubleType; import org.apache.flink.table.types.logical.FloatType; import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.LegacyTypeInformationType; +import org.apache.flink.table.types.logical.LocalZonedTimestampType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.SmallIntType; @@ -105,6 +110,7 @@ import org.apache.arrow.vector.TimeMicroVector; import org.apache.arrow.vector.TimeMilliVector; import org.apache.arrow.vector.TimeNanoVector; import org.apache.arrow.vector.TimeSecVector; +import org.apache.arrow.vector.TimeStampVector; import org.apache.arrow.vector.TinyIntVector; import org.apache.arrow.vector.VarBinaryVector; import org.apache.arrow.vector.VarCharVector; @@ -191,6 +197,8 @@ public final class ArrowUtils { } else if (vector instanceof TimeSecVector || vector instanceof TimeMilliVector || vector instanceof TimeMicroVector || vector instanceof TimeNanoVector) { return new TimeWriter(vector); + } else if (vector instanceof TimeStampVector && ((ArrowType.Timestamp) vector.getField().getType()).getTimezone() == null) { + return new TimestampWriter(vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -236,9 +244,11 @@ public final class ArrowUtils { return new BaseRowDecimalWriter(decimalVector, getPrecision(decimalVector), decimalVector.getScale()); } else if (vector instanceof DateDayVector) { return new BaseRowDateWriter((DateDayVector) vector); - } else if (vector instanceof TimeSecVector || vector instanceof TimeMilliVector || + } else if (vector instanceof TimeSecVector || vector instanceof TimeMilliVector || vector instanceof TimeMicroVector || vector instanceof TimeNanoVector) { return new BaseRowTimeWriter(vector); + } else if (vector instanceof TimeStampVector && ((ArrowType.Timestamp) vector.getField().getType()).getTimezone() == null) { + return new BaseRowTimestampWriter(vector, ((LocalZonedTimestampType) fieldType).getPrecision()); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -284,6 +294,8 @@ public final class ArrowUtils { } else if (vector instanceof TimeSecVector || vector instanceof TimeMilliVector || vector instanceof TimeMicroVector || vector instanceof TimeNanoVector) { return new TimeFieldReader(vector); + } else if (vector instanceof TimeStampVector && ((ArrowType.Timestamp) vector.getField().getType()).getTimezone() == null) { + return new TimestampFieldReader(vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -329,6 +341,8 @@ public final class ArrowUtils { } else if (vector instanceof TimeSecVector || vector instanceof TimeMilliVector || vector instanceof TimeMicroVector || vector instanceof TimeNanoVector) { return new ArrowTimeColumnVector(vector); + } else if (vector instanceof TimeStampVector && ((ArrowType.Timestamp) vector.getField().getType()).getTimezone() == null) { + return new ArrowTimestampColumnVector(vector); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -407,6 +421,19 @@ public final class ArrowUtils { } } + @Override + public ArrowType visit(LocalZonedTimestampType localZonedTimestampType) { + if (localZonedTimestampType.getPrecision() == 0) { + return new ArrowType.Timestamp(TimeUnit.SECOND, null); + } else if (localZonedTimestampType.getPrecision() >= 1 && localZonedTimestampType.getPrecision() <= 3) { + return new ArrowType.Timestamp(TimeUnit.MILLISECOND, null); + } else if (localZonedTimestampType.getPrecision() >= 4 && localZonedTimestampType.getPrecision() <= 6) { + return new ArrowType.Timestamp(TimeUnit.MICROSECOND, null); + } else { + return new ArrowType.Timestamp(TimeUnit.NANOSECOND, null); + } + } + @Override protected ArrowType defaultMethod(LogicalType logicalType) { if (logicalType instanceof LegacyTypeInformationType) { diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/TimestampFieldReader.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/TimestampFieldReader.java new file mode 100644 index 0000000000..4014f066e3 --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/TimestampFieldReader.java @@ -0,0 +1,65 @@ +/* + * 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.table.runtime.arrow.readers; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.runtime.typeutils.PythonTypeUtils; +import org.apache.flink.util.Preconditions; + +import org.apache.arrow.vector.TimeStampMicroVector; +import org.apache.arrow.vector.TimeStampMilliVector; +import org.apache.arrow.vector.TimeStampNanoVector; +import org.apache.arrow.vector.TimeStampSecVector; +import org.apache.arrow.vector.TimeStampVector; +import org.apache.arrow.vector.ValueVector; +import org.apache.arrow.vector.types.pojo.ArrowType; + +import java.sql.Timestamp; + +/** + * {@link ArrowFieldReader} for Timestamp. + */ +@Internal +public final class TimestampFieldReader extends ArrowFieldReader { + + public TimestampFieldReader(ValueVector valueVector) { + super(valueVector); + Preconditions.checkState(valueVector instanceof TimeStampVector && ((ArrowType.Timestamp) valueVector.getField().getType()).getTimezone() == null); + } + + @Override + public Timestamp read(int i) { + ValueVector valueVector = getValueVector(); + if (valueVector.isNull(i)) { + return null; + } else { + long millisecond; + if (valueVector instanceof TimeStampSecVector) { + millisecond = ((TimeStampSecVector) valueVector).get(i) * 1000; + } else if (valueVector instanceof TimeStampMilliVector) { + millisecond = ((TimeStampMilliVector) valueVector).get(i); + } else if (valueVector instanceof TimeStampMicroVector) { + millisecond = ((TimeStampMicroVector) valueVector).get(i) / 1000; + } else { + millisecond = ((TimeStampNanoVector) valueVector).get(i) / 1_000_000; + } + return PythonTypeUtils.internalToTimestamp(millisecond); + } + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowTimestampColumnVector.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowTimestampColumnVector.java new file mode 100644 index 0000000000..92c333193f --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowTimestampColumnVector.java @@ -0,0 +1,69 @@ +/* + * 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.table.runtime.arrow.vectors; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.dataformat.SqlTimestamp; +import org.apache.flink.table.dataformat.vector.TimestampColumnVector; +import org.apache.flink.util.Preconditions; + +import org.apache.arrow.vector.TimeStampMicroVector; +import org.apache.arrow.vector.TimeStampMilliVector; +import org.apache.arrow.vector.TimeStampNanoVector; +import org.apache.arrow.vector.TimeStampSecVector; +import org.apache.arrow.vector.TimeStampVector; +import org.apache.arrow.vector.ValueVector; +import org.apache.arrow.vector.types.pojo.ArrowType; + +/** + * Arrow column vector for Timestamp. + */ +@Internal +public final class ArrowTimestampColumnVector implements TimestampColumnVector { + + /** + * Container which is used to store the sequence of timestamp values of a column to read. + */ + private final ValueVector valueVector; + + public ArrowTimestampColumnVector(ValueVector valueVector) { + this.valueVector = Preconditions.checkNotNull(valueVector); + Preconditions.checkState(valueVector instanceof TimeStampVector && ((ArrowType.Timestamp) valueVector.getField().getType()).getTimezone() == null); + } + + @Override + public SqlTimestamp getTimestamp(int i, int precision) { + if (valueVector instanceof TimeStampSecVector) { + return SqlTimestamp.fromEpochMillis(((TimeStampSecVector) valueVector).get(i) * 1000); + } else if (valueVector instanceof TimeStampMilliVector) { + return SqlTimestamp.fromEpochMillis(((TimeStampMilliVector) valueVector).get(i)); + } else if (valueVector instanceof TimeStampMicroVector) { + long micros = ((TimeStampMicroVector) valueVector).get(i); + return SqlTimestamp.fromEpochMillis(micros / 1000, (int) (micros % 1000) * 1000); + } else { + long nanos = ((TimeStampNanoVector) valueVector).get(i); + return SqlTimestamp.fromEpochMillis(nanos / 1_000_000, (int) (nanos % 1_000_000)); + } + } + + @Override + public boolean isNullAt(int i) { + return valueVector.isNull(i); + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowTimestampWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowTimestampWriter.java new file mode 100644 index 0000000000..41d41bd5e1 --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowTimestampWriter.java @@ -0,0 +1,67 @@ +/* + * 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.table.runtime.arrow.writers; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.dataformat.BaseRow; +import org.apache.flink.table.dataformat.SqlTimestamp; +import org.apache.flink.util.Preconditions; + +import org.apache.arrow.vector.TimeStampMicroVector; +import org.apache.arrow.vector.TimeStampMilliVector; +import org.apache.arrow.vector.TimeStampNanoVector; +import org.apache.arrow.vector.TimeStampSecVector; +import org.apache.arrow.vector.TimeStampVector; +import org.apache.arrow.vector.ValueVector; +import org.apache.arrow.vector.types.pojo.ArrowType; + +/** + * {@link ArrowFieldWriter} for Timestamp. + */ +@Internal +public final class BaseRowTimestampWriter extends ArrowFieldWriter { + + private final int precision; + + public BaseRowTimestampWriter(ValueVector valueVector, int precision) { + super(valueVector); + Preconditions.checkState(valueVector instanceof TimeStampVector && ((ArrowType.Timestamp) valueVector.getField().getType()).getTimezone() == null); + this.precision = precision; + } + + @Override + public void doWrite(BaseRow row, int ordinal) { + ValueVector valueVector = getValueVector(); + if (row.isNullAt(ordinal)) { + ((TimeStampVector) valueVector).setNull(getCount()); + } else { + SqlTimestamp sqlTimestamp = row.getTimestamp(ordinal, precision); + + if (valueVector instanceof TimeStampSecVector) { + ((TimeStampSecVector) valueVector).setSafe(getCount(), sqlTimestamp.getMillisecond() / 1000); + } else if (valueVector instanceof TimeStampMilliVector) { + ((TimeStampMilliVector) valueVector).setSafe(getCount(), sqlTimestamp.getMillisecond()); + } else if (valueVector instanceof TimeStampMicroVector) { + ((TimeStampMicroVector) valueVector).setSafe(getCount(), sqlTimestamp.getMillisecond() * 1000 + sqlTimestamp.getNanoOfMillisecond() / 1000); + } else { + ((TimeStampNanoVector) valueVector).setSafe(getCount(), sqlTimestamp.getMillisecond() * 1_000_000 + sqlTimestamp.getNanoOfMillisecond()); + } + } + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/TimestampWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/TimestampWriter.java new file mode 100644 index 0000000000..1f2e895a9d --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/TimestampWriter.java @@ -0,0 +1,66 @@ +/* + * 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.table.runtime.arrow.writers; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.runtime.typeutils.PythonTypeUtils; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + +import org.apache.arrow.vector.BaseFixedWidthVector; +import org.apache.arrow.vector.TimeStampMicroVector; +import org.apache.arrow.vector.TimeStampMilliVector; +import org.apache.arrow.vector.TimeStampNanoVector; +import org.apache.arrow.vector.TimeStampSecVector; +import org.apache.arrow.vector.TimeStampVector; +import org.apache.arrow.vector.ValueVector; +import org.apache.arrow.vector.types.pojo.ArrowType; + +import java.sql.Timestamp; + +/** + * {@link ArrowFieldWriter} for Timestamp. + */ +@Internal +public final class TimestampWriter extends ArrowFieldWriter { + + public TimestampWriter(ValueVector valueVector) { + super(valueVector); + Preconditions.checkState(valueVector instanceof TimeStampVector && ((ArrowType.Timestamp) valueVector.getField().getType()).getTimezone() == null); + } + + @Override + public void doWrite(Row row, int ordinal) { + ValueVector valueVector = getValueVector(); + if (row.getField(ordinal) == null) { + ((BaseFixedWidthVector) getValueVector()).setNull(getCount()); + } else { + long millisecond = PythonTypeUtils.timestampToInternal((Timestamp) row.getField(ordinal)); + if (valueVector instanceof TimeStampSecVector) { + ((TimeStampSecVector) valueVector).setSafe(getCount(), millisecond / 1000); + } else if (valueVector instanceof TimeStampMilliVector) { + ((TimeStampMilliVector) valueVector).setSafe(getCount(), millisecond); + } else if (valueVector instanceof TimeStampMicroVector) { + ((TimeStampMicroVector) valueVector).setSafe(getCount(), millisecond * 1000); + } else { + ((TimeStampNanoVector) valueVector).setSafe(getCount(), millisecond * 1_000_000); + } + } + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java index 7569f1cfa3..9ef14d5219 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java @@ -142,6 +142,26 @@ public final class PythonTypeUtils { return (int) (ts / MILLIS_PER_DAY); } + /** + * Converts the internal representation of a SQL TIMESTAMP (long) to the Java + * type used for UDF parameters ({@link java.sql.Timestamp}). + * + *

    Note: The implementation refers to {@link SqlDateTimeUtils#internalToTimestamp}. + */ + public static java.sql.Timestamp internalToTimestamp(long v) { + return new java.sql.Timestamp(v - LOCAL_TZ.getOffset(v)); + } + + /** Converts the Java type used for UDF parameters of SQL TIMESTAMP type + * ({@link java.sql.Timestamp}) to internal representation (long). + * + *

    Note: The implementation refers to {@link SqlDateTimeUtils#timestampToInternal}. + */ + public static long timestampToInternal(java.sql.Timestamp ts) { + long time = ts.getTime(); + return time + LOCAL_TZ.getOffset(time); + } + /** * Convert LogicalType to conversion class for flink planner. */ diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java index 7fb5f2fc1d..b2b06968a2 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java @@ -32,6 +32,7 @@ import org.apache.flink.table.runtime.arrow.readers.IntFieldReader; import org.apache.flink.table.runtime.arrow.readers.RowArrowReader; import org.apache.flink.table.runtime.arrow.readers.SmallIntFieldReader; import org.apache.flink.table.runtime.arrow.readers.TimeFieldReader; +import org.apache.flink.table.runtime.arrow.readers.TimestampFieldReader; import org.apache.flink.table.runtime.arrow.readers.TinyIntFieldReader; import org.apache.flink.table.runtime.arrow.readers.VarBinaryFieldReader; import org.apache.flink.table.runtime.arrow.readers.VarCharFieldReader; @@ -44,6 +45,7 @@ import org.apache.flink.table.runtime.arrow.vectors.ArrowFloatColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowSmallIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowTimeColumnVector; +import org.apache.flink.table.runtime.arrow.vectors.ArrowTimestampColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowTinyIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowVarBinaryColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowVarCharColumnVector; @@ -58,6 +60,7 @@ import org.apache.flink.table.runtime.arrow.writers.BaseRowFloatWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowSmallIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowTimeWriter; +import org.apache.flink.table.runtime.arrow.writers.BaseRowTimestampWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowTinyIntWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowVarBinaryWriter; import org.apache.flink.table.runtime.arrow.writers.BaseRowVarCharWriter; @@ -70,6 +73,7 @@ import org.apache.flink.table.runtime.arrow.writers.FloatWriter; import org.apache.flink.table.runtime.arrow.writers.IntWriter; import org.apache.flink.table.runtime.arrow.writers.SmallIntWriter; import org.apache.flink.table.runtime.arrow.writers.TimeWriter; +import org.apache.flink.table.runtime.arrow.writers.TimestampWriter; import org.apache.flink.table.runtime.arrow.writers.TinyIntWriter; import org.apache.flink.table.runtime.arrow.writers.VarBinaryWriter; import org.apache.flink.table.runtime.arrow.writers.VarCharWriter; @@ -80,6 +84,7 @@ import org.apache.flink.table.types.logical.DecimalType; import org.apache.flink.table.types.logical.DoubleType; import org.apache.flink.table.types.logical.FloatType; import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LocalZonedTimestampType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.SmallIntType; @@ -148,6 +153,14 @@ public class ArrowUtilsTest { TimeWriter.class, BaseRowTimeWriter.class, TimeFieldReader.class, ArrowTimeColumnVector.class)); testFields.add(Tuple7.of("f16", new TimeType(8), new ArrowType.Time(TimeUnit.NANOSECOND, 64), TimeWriter.class, BaseRowTimeWriter.class, TimeFieldReader.class, ArrowTimeColumnVector.class)); + testFields.add(Tuple7.of("f17", new LocalZonedTimestampType(0), new ArrowType.Timestamp(TimeUnit.SECOND, null), + TimestampWriter.class, BaseRowTimestampWriter.class, TimestampFieldReader.class, ArrowTimestampColumnVector.class)); + testFields.add(Tuple7.of("f18", new LocalZonedTimestampType(2), new ArrowType.Timestamp(TimeUnit.MILLISECOND, null), + TimestampWriter.class, BaseRowTimestampWriter.class, TimestampFieldReader.class, ArrowTimestampColumnVector.class)); + testFields.add(Tuple7.of("f19", new LocalZonedTimestampType(4), new ArrowType.Timestamp(TimeUnit.MICROSECOND, null), + TimestampWriter.class, BaseRowTimestampWriter.class, TimestampFieldReader.class, ArrowTimestampColumnVector.class)); + testFields.add(Tuple7.of("f20", new LocalZonedTimestampType(8), new ArrowType.Timestamp(TimeUnit.NANOSECOND, null), + TimestampWriter.class, BaseRowTimestampWriter.class, TimestampFieldReader.class, ArrowTimestampColumnVector.class)); List rowFields = new ArrayList<>(); for (Tuple7, Class, Class, Class> field : testFields) { diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java index db4bbbea47..80b1ff1fc5 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java @@ -23,6 +23,7 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.table.dataformat.BaseRow; import org.apache.flink.table.dataformat.BinaryRow; import org.apache.flink.table.dataformat.Decimal; +import org.apache.flink.table.dataformat.SqlTimestamp; import org.apache.flink.table.runtime.typeutils.BaseRowSerializer; import org.apache.flink.table.runtime.util.StreamRecordUtils; import org.apache.flink.table.types.logical.BigIntType; @@ -32,6 +33,7 @@ import org.apache.flink.table.types.logical.DecimalType; import org.apache.flink.table.types.logical.DoubleType; import org.apache.flink.table.types.logical.FloatType; import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LocalZonedTimestampType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.SmallIntType; @@ -106,6 +108,10 @@ public class BaseRowArrowReaderWriterTest extends ArrowReaderWriterTestBase rowFields = new ArrayList<>(); for (int i = 0; i < fieldTypes.size(); i++) { @@ -133,12 +139,16 @@ public class BaseRowArrowReaderWriterTest extends ArrowReaderWriterTestBase { fieldTypes.add(new TimeType(2)); fieldTypes.add(new TimeType(4)); fieldTypes.add(new TimeType(8)); + fieldTypes.add(new LocalZonedTimestampType(0)); + fieldTypes.add(new LocalZonedTimestampType(2)); + fieldTypes.add(new LocalZonedTimestampType(4)); + fieldTypes.add(new LocalZonedTimestampType(8)); List rowFields = new ArrayList<>(); for (int i = 0; i < fieldTypes.size(); i++) { @@ -102,12 +108,15 @@ public class RowArrowReaderWriterTest extends ArrowReaderWriterTestBase { @Override public Row[] getTestData() { Row row1 = Row.of((byte) 1, (short) 2, 3, 4L, true, 1.0f, 1.0, "hello", "hello".getBytes(), new BigDecimal(1), SqlDateTimeUtils.internalToDate(100), - SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000)); + SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), + new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000)); Row row2 = Row.of(null, (short) 2, 3, 4L, false, 1.0f, 1.0, "中文", "中文".getBytes(), new BigDecimal(1), SqlDateTimeUtils.internalToDate(100), - SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000)); + SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), + new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000)); Row row3 = Row.of((byte) 1, null, 3, 4L, true, 1.0f, 1.0, "hello", "hello".getBytes(), new BigDecimal(1), SqlDateTimeUtils.internalToDate(100), - SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000)); - Row row4 = Row.of(null, null, null, null, null, null, null, null, null, null, null, null, null, null, null); + SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), + new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000)); + Row row4 = Row.of(null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null); return new Row[]{row1, row2, row3, row4}; } } diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/util/StreamRecordUtils.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/util/StreamRecordUtils.java index fa1ea1f3e9..5ef8f27fb1 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/util/StreamRecordUtils.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/util/StreamRecordUtils.java @@ -18,6 +18,7 @@ package org.apache.flink.table.runtime.util; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.table.dataformat.BaseRow; import org.apache.flink.table.dataformat.BinaryRow; @@ -25,6 +26,7 @@ import org.apache.flink.table.dataformat.BinaryRowWriter; import org.apache.flink.table.dataformat.BinaryString; import org.apache.flink.table.dataformat.Decimal; import org.apache.flink.table.dataformat.GenericRow; +import org.apache.flink.table.dataformat.SqlTimestamp; import org.apache.flink.table.dataformat.util.BaseRowUtil; import static org.apache.flink.table.dataformat.BinaryString.fromString; @@ -121,6 +123,9 @@ public class StreamRecordUtils { } else if (value instanceof Decimal) { Decimal decimal = (Decimal) value; writer.writeDecimal(j, decimal, decimal.getPrecision()); + } else if (value instanceof Tuple2 && ((Tuple2) value).f0 instanceof SqlTimestamp) { + SqlTimestamp timestamp = (SqlTimestamp) ((Tuple2) value).f0; + writer.writeTimestamp(j, timestamp, (int) ((Tuple2) value).f1); } else { throw new RuntimeException("Not support yet!"); } -- Gitee From 851a830536c7811df7f207bc47dca851be17f56b Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Sun, 29 Mar 2020 11:39:58 +0800 Subject: [PATCH 364/885] [FLINK-16847][python] Support TimestampType in vectorized Python UDF --- flink-python/pyflink/fn_execution/coders.py | 4 ++- .../pyflink/table/tests/test_pandas_udf.py | 32 +++++++++++++++---- flink-python/pyflink/table/types.py | 2 +- .../flink/table/runtime/arrow/ArrowUtils.java | 22 ++++++++++++- .../table/runtime/arrow/ArrowUtilsTest.java | 9 ++++++ .../arrow/BaseRowArrowReaderWriterTest.java | 13 ++++++-- .../arrow/RowArrowReaderWriterTest.java | 10 +++++- 7 files changed, 80 insertions(+), 12 deletions(-) diff --git a/flink-python/pyflink/fn_execution/coders.py b/flink-python/pyflink/fn_execution/coders.py index 361af5a0b7..ebff1b8ad3 100644 --- a/flink-python/pyflink/fn_execution/coders.py +++ b/flink-python/pyflink/fn_execution/coders.py @@ -34,7 +34,7 @@ from pyflink.fn_execution import flink_fn_execution_pb2 from pyflink.fn_execution.sdk_worker_main import pipeline_options from pyflink.table.types import Row, TinyIntType, SmallIntType, IntType, BigIntType, BooleanType, \ FloatType, DoubleType, VarCharType, VarBinaryType, DecimalType, DateType, TimeType, \ - LocalZonedTimestampType, RowType, RowField, to_arrow_type + LocalZonedTimestampType, RowType, RowField, to_arrow_type, TimestampType FLINK_SCALAR_FUNCTION_SCHEMA_CODER_URN = "flink:coder:schema:scalar_function:v1" FLINK_TABLE_FUNCTION_SCHEMA_CODER_URN = "flink:coder:schema:table_function:v1" @@ -452,6 +452,8 @@ class ArrowCoder(DeterministicCoder): flink_fn_execution_pb2.Schema.TypeName.LOCAL_ZONED_TIMESTAMP: return LocalZonedTimestampType(field.type.local_zoned_timestamp_info.precision, field.type.nullable) + elif field.type.type_name == flink_fn_execution_pb2.Schema.TypeName.TIMESTAMP: + return TimestampType(field.type.timestamp_info.precision, field.type.nullable) else: raise ValueError("field_type %s is not supported." % field.type) diff --git a/flink-python/pyflink/table/tests/test_pandas_udf.py b/flink-python/pyflink/table/tests/test_pandas_udf.py index 878dbb3f41..093be8991f 100644 --- a/flink-python/pyflink/table/tests/test_pandas_udf.py +++ b/flink-python/pyflink/table/tests/test_pandas_udf.py @@ -142,6 +142,17 @@ class PandasUDFITTests(object): 'time_param of wrong type %s !' % type(time_param[0]) return time_param + timestamp_value = datetime.datetime(1970, 1, 1, 0, 0, 0, 123000) + + def timestamp_func(timestamp_param): + assert isinstance(timestamp_param, pd.Series) + assert isinstance(timestamp_param[0], datetime.datetime), \ + 'timestamp_param of wrong type %s !' % type(timestamp_param[0]) + assert timestamp_param[0] == timestamp_value, \ + 'timestamp_param is wrong value %s, should be %s!' % (timestamp_param[0], + timestamp_value) + return timestamp_param + self.t_env.register_function( "tinyint_func", udf(tinyint_func, [DataTypes.TINYINT()], DataTypes.TINYINT(), udf_type="pandas")) @@ -191,19 +202,25 @@ class PandasUDFITTests(object): "time_func", udf(time_func, [DataTypes.TIME()], DataTypes.TIME(), udf_type="pandas")) + self.t_env.register_function( + "timestamp_func", + udf(timestamp_func, [DataTypes.TIMESTAMP(3)], DataTypes.TIMESTAMP(3), + udf_type="pandas")) + table_sink = source_sink_utils.TestAppendSink( - ['a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j', 'k', 'l', 'm', 'n', 'o'], + ['a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j', 'k', 'l', 'm', 'n', 'o', 'p'], [DataTypes.TINYINT(), DataTypes.SMALLINT(), DataTypes.INT(), DataTypes.BIGINT(), DataTypes.BOOLEAN(), DataTypes.BOOLEAN(), DataTypes.FLOAT(), DataTypes.DOUBLE(), DataTypes.STRING(), DataTypes.STRING(), DataTypes.BYTES(), DataTypes.DECIMAL(38, 18), - DataTypes.DECIMAL(38, 18), DataTypes.DATE(), DataTypes.TIME()]) + DataTypes.DECIMAL(38, 18), DataTypes.DATE(), DataTypes.TIME(), DataTypes.TIMESTAMP(3)]) self.t_env.register_table_sink("Results", table_sink) t = self.t_env.from_elements( [(1, 32767, -2147483648, 1, True, False, 1.0, 1.0, 'hello', '中文', bytearray(b'flink'), decimal.Decimal('1000000000000000000.05'), decimal.Decimal('1000000000000000000.05999999999999999899999999999'), - datetime.date(2014, 9, 13), datetime.time(hour=1, minute=0, second=1))], + datetime.date(2014, 9, 13), datetime.time(hour=1, minute=0, second=1), + timestamp_value)], DataTypes.ROW( [DataTypes.FIELD("a", DataTypes.TINYINT()), DataTypes.FIELD("b", DataTypes.SMALLINT()), @@ -219,7 +236,8 @@ class PandasUDFITTests(object): DataTypes.FIELD("l", DataTypes.DECIMAL(38, 18)), DataTypes.FIELD("m", DataTypes.DECIMAL(38, 18)), DataTypes.FIELD("n", DataTypes.DATE()), - DataTypes.FIELD("o", DataTypes.TIME())])) + DataTypes.FIELD("o", DataTypes.TIME()), + DataTypes.FIELD("p", DataTypes.TIMESTAMP(3))])) t.select("tinyint_func(a)," "smallint_func(b)," @@ -235,14 +253,16 @@ class PandasUDFITTests(object): "decimal_func(l)," "decimal_func(m)," "date_func(n)," - "time_func(o)") \ + "time_func(o)," + "timestamp_func(p)") \ .insert_into("Results") self.t_env.execute("test") actual = source_sink_utils.results() self.assert_equals(actual, ["1,32767,-2147483648,1,true,false,1.0,1.0,hello,中文," "[102, 108, 105, 110, 107],1000000000000000000.050000000000000000," - "1000000000000000000.059999999999999999,2014-09-13,01:00:01"]) + "1000000000000000000.059999999999999999,2014-09-13,01:00:01," + "1970-01-01 00:00:00.123"]) class BlinkPandasUDFITTests(object): diff --git a/flink-python/pyflink/table/types.py b/flink-python/pyflink/table/types.py index d45a62f1f4..6b26fa9ec8 100644 --- a/flink-python/pyflink/table/types.py +++ b/flink-python/pyflink/table/types.py @@ -2324,7 +2324,7 @@ def to_arrow_type(data_type): return pa.time64('us') else: return pa.time64('ns') - elif type(data_type) == LocalZonedTimestampType: + elif type(data_type) in [LocalZonedTimestampType, TimestampType]: if data_type.precision == 0: return pa.timestamp('s') elif 1 <= data_type.precision <= 3: diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java index ac92cf2485..d38a1e9dc6 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java @@ -90,6 +90,7 @@ import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.SmallIntType; import org.apache.flink.table.types.logical.TimeType; +import org.apache.flink.table.types.logical.TimestampType; import org.apache.flink.table.types.logical.TinyIntType; import org.apache.flink.table.types.logical.VarBinaryType; import org.apache.flink.table.types.logical.VarCharType; @@ -248,7 +249,13 @@ public final class ArrowUtils { vector instanceof TimeMicroVector || vector instanceof TimeNanoVector) { return new BaseRowTimeWriter(vector); } else if (vector instanceof TimeStampVector && ((ArrowType.Timestamp) vector.getField().getType()).getTimezone() == null) { - return new BaseRowTimestampWriter(vector, ((LocalZonedTimestampType) fieldType).getPrecision()); + int precision; + if (fieldType instanceof LocalZonedTimestampType) { + precision = ((LocalZonedTimestampType) fieldType).getPrecision(); + } else { + precision = ((TimestampType) fieldType).getPrecision(); + } + return new BaseRowTimestampWriter(vector, precision); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -434,6 +441,19 @@ public final class ArrowUtils { } } + @Override + public ArrowType visit(TimestampType timestampType) { + if (timestampType.getPrecision() == 0) { + return new ArrowType.Timestamp(TimeUnit.SECOND, null); + } else if (timestampType.getPrecision() >= 1 && timestampType.getPrecision() <= 3) { + return new ArrowType.Timestamp(TimeUnit.MILLISECOND, null); + } else if (timestampType.getPrecision() >= 4 && timestampType.getPrecision() <= 6) { + return new ArrowType.Timestamp(TimeUnit.MICROSECOND, null); + } else { + return new ArrowType.Timestamp(TimeUnit.NANOSECOND, null); + } + } + @Override protected ArrowType defaultMethod(LogicalType logicalType) { if (logicalType instanceof LegacyTypeInformationType) { diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java index b2b06968a2..8190d8dfd9 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java @@ -89,6 +89,7 @@ import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.SmallIntType; import org.apache.flink.table.types.logical.TimeType; +import org.apache.flink.table.types.logical.TimestampType; import org.apache.flink.table.types.logical.TinyIntType; import org.apache.flink.table.types.logical.VarBinaryType; import org.apache.flink.table.types.logical.VarCharType; @@ -161,6 +162,14 @@ public class ArrowUtilsTest { TimestampWriter.class, BaseRowTimestampWriter.class, TimestampFieldReader.class, ArrowTimestampColumnVector.class)); testFields.add(Tuple7.of("f20", new LocalZonedTimestampType(8), new ArrowType.Timestamp(TimeUnit.NANOSECOND, null), TimestampWriter.class, BaseRowTimestampWriter.class, TimestampFieldReader.class, ArrowTimestampColumnVector.class)); + testFields.add(Tuple7.of("f21", new TimestampType(0), new ArrowType.Timestamp(TimeUnit.SECOND, null), + TimestampWriter.class, BaseRowTimestampWriter.class, TimestampFieldReader.class, ArrowTimestampColumnVector.class)); + testFields.add(Tuple7.of("f22", new TimestampType(2), new ArrowType.Timestamp(TimeUnit.MILLISECOND, null), + TimestampWriter.class, BaseRowTimestampWriter.class, TimestampFieldReader.class, ArrowTimestampColumnVector.class)); + testFields.add(Tuple7.of("f23", new TimestampType(4), new ArrowType.Timestamp(TimeUnit.MICROSECOND, null), + TimestampWriter.class, BaseRowTimestampWriter.class, TimestampFieldReader.class, ArrowTimestampColumnVector.class)); + testFields.add(Tuple7.of("f24", new TimestampType(8), new ArrowType.Timestamp(TimeUnit.NANOSECOND, null), + TimestampWriter.class, BaseRowTimestampWriter.class, TimestampFieldReader.class, ArrowTimestampColumnVector.class)); List rowFields = new ArrayList<>(); for (Tuple7, Class, Class, Class> field : testFields) { diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java index 80b1ff1fc5..f0428d7344 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java @@ -38,6 +38,7 @@ import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.SmallIntType; import org.apache.flink.table.types.logical.TimeType; +import org.apache.flink.table.types.logical.TimestampType; import org.apache.flink.table.types.logical.TinyIntType; import org.apache.flink.table.types.logical.VarBinaryType; import org.apache.flink.table.types.logical.VarCharType; @@ -112,6 +113,10 @@ public class BaseRowArrowReaderWriterTest extends ArrowReaderWriterTestBase rowFields = new ArrayList<>(); for (int i = 0; i < fieldTypes.size(); i++) { @@ -140,15 +145,19 @@ public class BaseRowArrowReaderWriterTest extends ArrowReaderWriterTestBase { fieldTypes.add(new LocalZonedTimestampType(2)); fieldTypes.add(new LocalZonedTimestampType(4)); fieldTypes.add(new LocalZonedTimestampType(8)); + fieldTypes.add(new TimestampType(0)); + fieldTypes.add(new TimestampType(2)); + fieldTypes.add(new TimestampType(4)); + fieldTypes.add(new TimestampType(8)); List rowFields = new ArrayList<>(); for (int i = 0; i < fieldTypes.size(); i++) { @@ -109,14 +114,17 @@ public class RowArrowReaderWriterTest extends ArrowReaderWriterTestBase { public Row[] getTestData() { Row row1 = Row.of((byte) 1, (short) 2, 3, 4L, true, 1.0f, 1.0, "hello", "hello".getBytes(), new BigDecimal(1), SqlDateTimeUtils.internalToDate(100), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), + new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000)); Row row2 = Row.of(null, (short) 2, 3, 4L, false, 1.0f, 1.0, "中文", "中文".getBytes(), new BigDecimal(1), SqlDateTimeUtils.internalToDate(100), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), + new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000)); Row row3 = Row.of((byte) 1, null, 3, 4L, true, 1.0f, 1.0, "hello", "hello".getBytes(), new BigDecimal(1), SqlDateTimeUtils.internalToDate(100), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), + new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000)); - Row row4 = Row.of(null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null); + Row row4 = Row.of(null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null); return new Row[]{row1, row2, row3, row4}; } } -- Gitee From e9315155fb8ae916bc4f0bd3173c249698d6a588 Mon Sep 17 00:00:00 2001 From: "yuzhao.cyz" Date: Tue, 17 Mar 2020 18:59:45 +0800 Subject: [PATCH 365/885] [FLINK-14338][flink-table] Update flink table POM and NOTICE file Calcite dependency to 1.22.0 This closes #11340 --- flink-table/flink-sql-parser/pom.xml | 34 ++++++++- flink-table/flink-table-planner-blink/pom.xml | 70 ++++++++---------- .../src/main/resources/META-INF/NOTICE | 6 +- flink-table/flink-table-planner/pom.xml | 74 +++++++++---------- .../src/main/resources/META-INF/NOTICE | 6 +- flink-table/flink-table-runtime-blink/pom.xml | 6 +- .../src/main/resources/META-INF/NOTICE | 2 +- flink-table/pom.xml | 26 ++++++- 8 files changed, 133 insertions(+), 91 deletions(-) diff --git a/flink-table/flink-sql-parser/pom.xml b/flink-table/flink-sql-parser/pom.xml index 7658301869..87bbf219a5 100644 --- a/flink-table/flink-sql-parser/pom.xml +++ b/flink-table/flink-sql-parser/pom.xml @@ -38,6 +38,12 @@ under the License. + + com.google.guava + guava + ${guava.version} + provided + org.apache.calcite calcite-core @@ -45,11 +51,11 @@ under the License. ${calcite.version} @@ -57,6 +63,14 @@ under the License. org.apache.calcite.avatica avatica-metrics + + org.apache.calcite.avatica + avatica-server + + + commons-io + commons-io + com.google.protobuf protobuf-java @@ -140,6 +154,10 @@ under the License. calcite-core ${calcite.version} + + org.apache.calcite.avatica + avatica-server + com.fasterxml.jackson.core jackson-core @@ -156,6 +174,14 @@ under the License. test test-jar + + + org.junit.jupiter + junit-jupiter-engine + 5.5.2 + test + diff --git a/flink-table/flink-table-planner-blink/pom.xml b/flink-table/flink-table-planner-blink/pom.xml index 6dbf516993..b61f7d1fde 100644 --- a/flink-table/flink-table-planner-blink/pom.xml +++ b/flink-table/flink-table-planner-blink/pom.xml @@ -39,30 +39,22 @@ under the License. jar - - - - - com.google.guava - guava - 19.0 - - - - org.codehaus.janino - commons-compiler - ${janino.version} - - - - org.codehaus.janino - janino - ${janino.version} - - - - + + + com.google.guava + guava + + + org.codehaus.janino + commons-compiler + + + + org.codehaus.janino + janino + @@ -128,12 +120,6 @@ under the License. provided - - - org.codehaus.janino - janino - - org.apache.flink flink-cep_${scala.binary.version} @@ -149,20 +135,28 @@ under the License. ${calcite.version} + + org.apache.calcite.avatica + avatica-server + org.apache.calcite.avatica avatica-metrics diff --git a/flink-table/flink-table-planner-blink/src/main/resources/META-INF/NOTICE b/flink-table/flink-table-planner-blink/src/main/resources/META-INF/NOTICE index 9770943b43..381d6d5194 100644 --- a/flink-table/flink-table-planner-blink/src/main/resources/META-INF/NOTICE +++ b/flink-table/flink-table-planner-blink/src/main/resources/META-INF/NOTICE @@ -12,9 +12,9 @@ This project bundles the following dependencies under the Apache Software Licens - com.fasterxml.jackson.core:jackson-databind:2.10.1 - com.jayway.jsonpath:json-path:2.4.0 - joda-time:joda-time:2.5 -- org.apache.calcite:calcite-core:1.21.0 -- org.apache.calcite:calcite-linq4j:1.21.0 -- org.apache.calcite.avatica:avatica-core:1.15.0 +- org.apache.calcite:calcite-core:1.22.0 +- org.apache.calcite:calcite-linq4j:1.22.0 +- org.apache.calcite.avatica:avatica-core:1.16.0 - commons-codec:commons-codec:1.10 This project bundles the following dependencies under the BSD license. diff --git a/flink-table/flink-table-planner/pom.xml b/flink-table/flink-table-planner/pom.xml index 7dabdf32cf..bd54093bd7 100644 --- a/flink-table/flink-table-planner/pom.xml +++ b/flink-table/flink-table-planner/pom.xml @@ -37,30 +37,22 @@ under the License. jar - - - - - com.google.guava - guava - 19.0 - - - - org.codehaus.janino - commons-compiler - ${janino.version} - - - - org.codehaus.janino - janino - ${janino.version} - - - - + + + com.google.guava + guava + + + org.codehaus.janino + commons-compiler + + + + org.codehaus.janino + janino + @@ -91,6 +83,10 @@ under the License. org.apache.calcite calcite-core + + com.google.guava + guava + @@ -121,12 +117,6 @@ under the License. provided - - - org.codehaus.janino - janino - - org.apache.calcite @@ -135,21 +125,29 @@ under the License. ${calcite.version} + + org.apache.calcite.avatica + avatica-server + org.apache.calcite.avatica avatica-metrics diff --git a/flink-table/flink-table-planner/src/main/resources/META-INF/NOTICE b/flink-table/flink-table-planner/src/main/resources/META-INF/NOTICE index 1227b0bc0f..dbe6a3efeb 100644 --- a/flink-table/flink-table-planner/src/main/resources/META-INF/NOTICE +++ b/flink-table/flink-table-planner/src/main/resources/META-INF/NOTICE @@ -12,9 +12,9 @@ This project bundles the following dependencies under the Apache Software Licens - com.fasterxml.jackson.core:jackson-databind:2.10.1 - com.jayway.jsonpath:json-path:2.4.0 - joda-time:joda-time:2.5 -- org.apache.calcite:calcite-core:1.21.0 -- org.apache.calcite:calcite-linq4j:1.21.0 -- org.apache.calcite.avatica:avatica-core:1.15.0 +- org.apache.calcite:calcite-core:1.22.0 +- org.apache.calcite:calcite-linq4j:1.22.0 +- org.apache.calcite.avatica:avatica-core:1.16.0 - commons-codec:commons-codec:1.10 This project bundles the following dependencies under the BSD license. diff --git a/flink-table/flink-table-runtime-blink/pom.xml b/flink-table/flink-table-runtime-blink/pom.xml index 81e3324e42..e8ed8afdf6 100644 --- a/flink-table/flink-table-runtime-blink/pom.xml +++ b/flink-table/flink-table-runtime-blink/pom.xml @@ -78,7 +78,7 @@ under the License. org.apache.calcite.avatica avatica-core - 1.15.0 + 1.16.0 diff --git a/flink-table/flink-table-runtime-blink/src/main/resources/META-INF/NOTICE b/flink-table/flink-table-runtime-blink/src/main/resources/META-INF/NOTICE index 3aac7ecdee..7ca85493f0 100644 --- a/flink-table/flink-table-runtime-blink/src/main/resources/META-INF/NOTICE +++ b/flink-table/flink-table-runtime-blink/src/main/resources/META-INF/NOTICE @@ -6,4 +6,4 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) -- org.apache.calcite.avatica:avatica-core:1.13.0 +- org.apache.calcite.avatica:avatica-core:1.16.0 diff --git a/flink-table/pom.xml b/flink-table/pom.xml index 5bf84f99ff..717b59f4eb 100644 --- a/flink-table/pom.xml +++ b/flink-table/pom.xml @@ -47,9 +47,33 @@ under the License. flink-sql-parser + + + + + com.google.guava + guava + ${guava.version} + + + + org.codehaus.janino + commons-compiler + ${janino.version} + + + + org.codehaus.janino + janino + ${janino.version} + + + + 3.0.9 - 1.21.0 + 1.22.0 + 19.0 -- Gitee From 78499853bff8eccc2b174a883a7cfdc4125eb499 Mon Sep 17 00:00:00 2001 From: "yuzhao.cyz" Date: Tue, 17 Mar 2020 19:00:55 +0800 Subject: [PATCH 366/885] [FLINK-14338][sql-parser] Bump sql parser Calcite dependency to 1.22.0 * Add Junit5 supports for tests * Move classes under package calcite.sql to flink.sql.parser.type * Remove ExtendedSqlBasicTypeNameSpec, use SqlAlienSystemTypeNameSpec instead * In Parser.tdd, re-organize the imports order to alphabetical order, remove the useless tailing commas, extends nonReservedKeywordsToAdd * Fix the JavaCC compile warnings --- .../src/main/codegen/data/Parser.tdd | 201 ++++----- .../src/main/codegen/includes/parserImpls.ftl | 19 +- .../org/apache/calcite/sql/package-info.java | 27 -- .../flink/sql/parser/ddl/SqlCreateTable.java | 18 +- .../type/ExtendedSqlBasicTypeNameSpec.java | 57 --- .../type}/ExtendedSqlRowTypeNameSpec.java | 8 +- .../sql/parser/type}/SqlMapTypeNameSpec.java | 6 +- .../parser/validate/FlinkSqlConformance.java | 10 + .../sql/parser/FlinkDDLDataTypeTest.java | 4 +- .../sql/parser/FlinkSqlParserImplTest.java | 392 ++++++++++-------- 10 files changed, 365 insertions(+), 377 deletions(-) delete mode 100644 flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/package-info.java delete mode 100644 flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/ExtendedSqlBasicTypeNameSpec.java rename flink-table/flink-sql-parser/src/main/java/org/apache/{calcite/sql => flink/sql/parser/type}/ExtendedSqlRowTypeNameSpec.java (93%) rename flink-table/flink-sql-parser/src/main/java/org/apache/{calcite/sql => flink/sql/parser/type}/SqlMapTypeNameSpec.java (93%) diff --git a/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd b/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd index 1783a5e722..14158c73ce 100644 --- a/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd +++ b/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd @@ -21,69 +21,70 @@ # List of additional classes and packages to import. # Example. "org.apache.calcite.sql.*", "java.util.List". + # Please keep the import classes in alphabetical order if new class is added. imports: [ - "org.apache.calcite.sql.ExtendedSqlRowTypeNameSpec", - "org.apache.calcite.sql.SqlMapTypeNameSpec", - "org.apache.flink.sql.parser.ddl.SqlCreateTable", + "org.apache.flink.sql.parser.ddl.SqlAlterDatabase" + "org.apache.flink.sql.parser.ddl.SqlAlterFunction" + "org.apache.flink.sql.parser.ddl.SqlAlterTable" + "org.apache.flink.sql.parser.ddl.SqlAlterTableProperties" + "org.apache.flink.sql.parser.ddl.SqlAlterTableRename" + "org.apache.flink.sql.parser.ddl.SqlCreateCatalog" + "org.apache.flink.sql.parser.ddl.SqlCreateDatabase" + "org.apache.flink.sql.parser.ddl.SqlCreateFunction" + "org.apache.flink.sql.parser.ddl.SqlCreateTable" + "org.apache.flink.sql.parser.ddl.SqlCreateTable.TableCreationContext" + "org.apache.flink.sql.parser.ddl.SqlCreateView" + "org.apache.flink.sql.parser.ddl.SqlDropDatabase" + "org.apache.flink.sql.parser.ddl.SqlDropFunction" "org.apache.flink.sql.parser.ddl.SqlDropTable" - "org.apache.flink.sql.parser.ddl.SqlCreateTable.TableCreationContext", - "org.apache.flink.sql.parser.ddl.SqlCreateView", - "org.apache.flink.sql.parser.ddl.SqlDropView", - "org.apache.flink.sql.parser.ddl.SqlTableColumn", - "org.apache.flink.sql.parser.ddl.SqlTableOption", - "org.apache.flink.sql.parser.ddl.SqlWatermark", - "org.apache.flink.sql.parser.ddl.SqlUseCatalog", - "org.apache.flink.sql.parser.ddl.SqlUseDatabase", - "org.apache.flink.sql.parser.ddl.SqlCreateDatabase", - "org.apache.flink.sql.parser.ddl.SqlDropDatabase", - "org.apache.flink.sql.parser.ddl.SqlAlterDatabase", - "org.apache.flink.sql.parser.ddl.SqlAlterFunction", - "org.apache.flink.sql.parser.ddl.SqlCreateFunction", - "org.apache.flink.sql.parser.ddl.SqlDropFunction", - "org.apache.flink.sql.parser.ddl.SqlAlterTable", - "org.apache.flink.sql.parser.ddl.SqlAlterTableRename", - "org.apache.flink.sql.parser.ddl.SqlAlterTableProperties", - "org.apache.flink.sql.parser.ddl.SqlCreateCatalog", - "org.apache.flink.sql.parser.dml.RichSqlInsert", - "org.apache.flink.sql.parser.dml.RichSqlInsertKeyword", - "org.apache.flink.sql.parser.dql.SqlShowCatalogs", - "org.apache.flink.sql.parser.dql.SqlDescribeCatalog", - "org.apache.flink.sql.parser.dql.SqlShowDatabases", - "org.apache.flink.sql.parser.dql.SqlShowFunctions", - "org.apache.flink.sql.parser.dql.SqlDescribeDatabase", - "org.apache.flink.sql.parser.dql.SqlShowTables", - "org.apache.flink.sql.parser.dql.SqlRichDescribeTable", - "org.apache.flink.sql.parser.type.ExtendedSqlBasicTypeNameSpec", - "org.apache.flink.sql.parser.type.ExtendedSqlCollectionTypeNameSpec", - "org.apache.flink.sql.parser.utils.SqlTimeUnit", - "org.apache.flink.sql.parser.utils.ParserResource", - "org.apache.flink.sql.parser.validate.FlinkSqlConformance", - "org.apache.flink.sql.parser.SqlProperty", - "org.apache.calcite.sql.SqlDrop", - "org.apache.calcite.sql.SqlCreate", - "java.util.List", + "org.apache.flink.sql.parser.ddl.SqlDropView" + "org.apache.flink.sql.parser.ddl.SqlTableColumn" + "org.apache.flink.sql.parser.ddl.SqlTableOption" + "org.apache.flink.sql.parser.ddl.SqlUseCatalog" + "org.apache.flink.sql.parser.ddl.SqlUseDatabase" + "org.apache.flink.sql.parser.ddl.SqlWatermark" + "org.apache.flink.sql.parser.dml.RichSqlInsert" + "org.apache.flink.sql.parser.dml.RichSqlInsertKeyword" + "org.apache.flink.sql.parser.dql.SqlDescribeCatalog" + "org.apache.flink.sql.parser.dql.SqlDescribeDatabase" + "org.apache.flink.sql.parser.dql.SqlShowCatalogs" + "org.apache.flink.sql.parser.dql.SqlShowDatabases" + "org.apache.flink.sql.parser.dql.SqlShowFunctions" + "org.apache.flink.sql.parser.dql.SqlShowTables" + "org.apache.flink.sql.parser.dql.SqlRichDescribeTable" + "org.apache.flink.sql.parser.type.ExtendedSqlCollectionTypeNameSpec" + "org.apache.flink.sql.parser.type.ExtendedSqlRowTypeNameSpec" + "org.apache.flink.sql.parser.type.SqlMapTypeNameSpec" + "org.apache.flink.sql.parser.utils.SqlTimeUnit" + "org.apache.flink.sql.parser.utils.ParserResource" + "org.apache.flink.sql.parser.validate.FlinkSqlConformance" + "org.apache.flink.sql.parser.SqlProperty" + "org.apache.calcite.sql.SqlAlienSystemTypeNameSpec" + "org.apache.calcite.sql.SqlCreate" + "org.apache.calcite.sql.SqlDrop" + "java.util.List" "java.util.ArrayList" ] # List of new keywords. Example: "DATABASES", "TABLES". If the keyword is not a reserved # keyword, please also add it to 'nonReservedKeywords' section. keywords: [ - "COMMENT", - "PARTITIONED", - "IF", - "WATERMARK", - "OVERWRITE", - "STRING", - "BYTES", - "RAW", - "CATALOGS", - "USE", - "DATABASES", - "FUNCTIONS", - "EXTENDED", - "SCALA", - "TABLES", + "BYTES" + "CATALOGS" + "COMMENT" + "DATABASES" + "EXTENDED" + "FUNCTIONS" + "IF" + "OVERWRITE" + "PARTITIONED" + "RAW" "RENAME" + "SCALA" + "STRING" + "TABLES" + "USE" + "WATERMARK" ] # List of keywords from "keywords" section that are not reserved. @@ -109,15 +110,14 @@ "C" "CASCADE" "CATALOG" - "CATALOGS" "CATALOG_NAME" "CENTURY" "CHAIN" + "CHARACTERISTICS" + "CHARACTERS" "CHARACTER_SET_CATALOG" "CHARACTER_SET_NAME" "CHARACTER_SET_SCHEMA" - "CHARACTERISTICS" - "CHARACTERS" "CLASS_ORIGIN" "COBOL" "COLLATION" @@ -128,22 +128,22 @@ "COMMAND_FUNCTION" "COMMAND_FUNCTION_CODE" "COMMITTED" - "CONDITION_NUMBER" "CONDITIONAL" + "CONDITION_NUMBER" "CONNECTION" "CONNECTION_NAME" "CONSTRAINT_CATALOG" "CONSTRAINT_NAME" - "CONSTRAINT_SCHEMA" "CONSTRAINTS" + "CONSTRAINT_SCHEMA" "CONSTRUCTOR" "CONTINUE" "CURSOR_NAME" "DATA" "DATABASE" - "DATABASES" "DATETIME_INTERVAL_CODE" "DATETIME_INTERVAL_PRECISION" + "DAYS" "DECADE" "DEFAULTS" "DEFERRABLE" @@ -169,7 +169,6 @@ "EXCEPTION" "EXCLUDE" "EXCLUDING" - "EXTENDED" "FINAL" "FIRST" "FOLLOWING" @@ -177,7 +176,6 @@ "FORTRAN" "FOUND" "FRAC_SECOND" - "FUNCTIONS" "G" "GENERAL" "GENERATED" @@ -186,6 +184,7 @@ "GOTO" "GRANTED" "HIERARCHY" + "HOURS" "IGNORE" "IMMEDIATE" "IMMEDIATELY" @@ -198,8 +197,8 @@ "INSTANTIABLE" "INVOKER" "ISODOW" - "ISOYEAR" "ISOLATION" + "ISOYEAR" "JAVA" "JSON" "K" @@ -213,15 +212,18 @@ "LIBRARY" "LOCATOR" "M" + "MAP" "MATCHED" "MAXVALUE" - "MICROSECOND" "MESSAGE_LENGTH" "MESSAGE_OCTET_LENGTH" "MESSAGE_TEXT" - "MILLISECOND" + "MICROSECOND" "MILLENNIUM" + "MILLISECOND" + "MINUTES" "MINVALUE" + "MONTHS" "MORE_" "MUMPS" "NAME" @@ -265,7 +267,6 @@ "QUARTER" "READ" "RELATIVE" - "RENAME" "REPEATABLE" "REPLACE" "RESPECT" @@ -289,6 +290,7 @@ "SCOPE_CATALOGS" "SCOPE_NAME" "SCOPE_SCHEMA" + "SECONDS" "SECTION" "SECURITY" "SELF" @@ -329,8 +331,8 @@ "SQL_INTERVAL_YEAR" "SQL_INTERVAL_YEAR_TO_MONTH" "SQL_LONGVARBINARY" - "SQL_LONGVARNCHAR" "SQL_LONGVARCHAR" + "SQL_LONGVARNCHAR" "SQL_NCHAR" "SQL_NCLOB" "SQL_NUMERIC" @@ -358,7 +360,6 @@ "STYLE" "SUBCLASS_ORIGIN" "SUBSTITUTE" - "TABLES" "TABLE_NAME" "TEMPORARY" "TIES" @@ -374,6 +375,7 @@ "TRIGGER_CATALOG" "TRIGGER_NAME" "TRIGGER_SCHEMA" + "TUMBLE" "TYPE" "UNBOUNDED" "UNCOMMITTED" @@ -381,45 +383,54 @@ "UNDER" "UNNAMED" "USAGE" - "USE" "USER_DEFINED_TYPE_CATALOG" "USER_DEFINED_TYPE_CODE" "USER_DEFINED_TYPE_NAME" "USER_DEFINED_TYPE_SCHEMA" - "UTF8" "UTF16" "UTF32" + "UTF8" "VERSION" "VIEW" "WEEK" - "WRAPPER" "WORK" + "WRAPPER" "WRITE" "XML" - "ZONE", + "YEARS" + "ZONE" + ] + # List of non-reserved keywords to add; + # items in this list become non-reserved + nonReservedKeywordsToAdd: [ # not in core, added in Flink - "PARTITIONED", - "IF", + "IF" "OVERWRITE" + "PARTITIONED" + ] + + # List of non-reserved keywords to remove; + # items in this list become reserved + nonReservedKeywordsToRemove: [ ] # List of methods for parsing custom SQL statements. # Return type of method implementation should be 'SqlNode'. # Example: SqlShowDatabases(), SqlShowTables(). statementParserMethods: [ - "RichSqlInsert()", - "SqlShowCatalogs()", - "SqlDescribeCatalog()", - "SqlUseCatalog()", - "SqlShowDatabases()", - "SqlUseDatabase()", - "SqlAlterDatabase()", - "SqlDescribeDatabase()", - "SqlAlterFunction()", - "SqlShowFunctions()", - "SqlShowTables()", - "SqlRichDescribeTable()", + "RichSqlInsert()" + "SqlShowCatalogs()" + "SqlDescribeCatalog()" + "SqlUseCatalog()" + "SqlShowDatabases()" + "SqlUseDatabase()" + "SqlAlterDatabase()" + "SqlDescribeDatabase()" + "SqlAlterFunction()" + "SqlShowFunctions()" + "SqlShowTables()" + "SqlRichDescribeTable()" "SqlAlterTable()" ] @@ -433,9 +444,9 @@ # Return type of method implementation should be "SqlTypeNameSpec". # Example: SqlParseTimeStampZ(). dataTypeParserMethods: [ - "ExtendedSqlBasicTypeName()", - "CustomizedCollectionsTypeName()", - "SqlMapTypeName()", + "ExtendedSqlBasicTypeName()" + "CustomizedCollectionsTypeName()" + "SqlMapTypeName()" "ExtendedSqlRowTypeName()" ] @@ -454,18 +465,18 @@ # List of methods for parsing extensions to "CREATE [OR REPLACE]" calls. # Each must accept arguments "(SqlParserPos pos, boolean replace)". createStatementParserMethods: [ - "SqlCreateCatalog", - "SqlCreateTable", - "SqlCreateView", - "SqlCreateDatabase", + "SqlCreateCatalog" + "SqlCreateTable" + "SqlCreateView" + "SqlCreateDatabase" "SqlCreateFunction" ] # List of methods for parsing extensions to "DROP" calls. # Each must accept arguments "(Span s)". dropStatementParserMethods: [ - "SqlDropTable", - "SqlDropView", + "SqlDropTable" + "SqlDropView" "SqlDropDatabase" "SqlDropFunction" ] diff --git a/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl b/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl index a93ef6675f..96a25a42fe 100644 --- a/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl +++ b/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl @@ -119,7 +119,10 @@ SqlCreate SqlCreateDatabase(Span s, boolean replace) : } { { startPos = getPos(); } - [ { ifNotExists = true; } ] + [ + LOOKAHEAD(3) + { ifNotExists = true; } + ] databaseName = CompoundIdentifier() [ { @@ -217,7 +220,10 @@ SqlCreate SqlCreateFunction(Span s, boolean replace) : - [ { ifNotExists = true; } ] + [ + LOOKAHEAD(3) + { ifNotExists = true; } + ] functionIdentifier = CompoundIdentifier() @@ -248,12 +254,13 @@ SqlDrop SqlDropFunction(Span s, boolean replace) : boolean isSystemFunction = false; } { - [ {isTemporary = true;} + [ + {isTemporary = true;} [ { isSystemFunction = true; } ] ] - [ { ifExists = true; } ] + [ LOOKAHEAD(2) { ifExists = true; } ] functionIdentifier = CompoundIdentifier() @@ -281,7 +288,7 @@ SqlAlterFunction SqlAlterFunction() : { startPos = getPos(); } - [ { ifExists = true; } ] + [ LOOKAHEAD(2) { ifExists = true; } ] functionIdentifier = CompoundIdentifier() @@ -806,7 +813,7 @@ SqlTypeNameSpec ExtendedSqlBasicTypeName() : } ) { - return new ExtendedSqlBasicTypeNameSpec(typeAlias, typeName, precision, getPos()); + return new SqlAlienSystemTypeNameSpec(typeAlias, typeName, precision, getPos()); } } diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/package-info.java b/flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/package-info.java deleted file mode 100644 index 439912b52b..0000000000 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/package-info.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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. - */ - -/** - * This package is needed because the constructor of SqlTypeNameSpec is package scope, - * we should merge this package into org.apache.flink.sql.parser.type when CALCITE-3360 - * is resolved. - */ -@PackageMarker -package org.apache.calcite.sql; - -import org.apache.calcite.avatica.util.PackageMarker; diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateTable.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateTable.java index c4c7f1199b..19839a0205 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateTable.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateTable.java @@ -20,8 +20,8 @@ package org.apache.flink.sql.parser.ddl; import org.apache.flink.sql.parser.ExtendedSqlNode; import org.apache.flink.sql.parser.error.SqlValidateException; +import org.apache.flink.sql.parser.type.ExtendedSqlRowTypeNameSpec; -import org.apache.calcite.sql.ExtendedSqlRowTypeNameSpec; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlCall; import org.apache.calcite.sql.SqlCharStringLiteral; @@ -69,10 +69,8 @@ public class SqlCreateTable extends SqlCreate implements ExtendedSqlNode { private final SqlNodeList partitionKeyList; - @Nullable private final SqlWatermark watermark; - @Nullable private final SqlCharStringLiteral comment; public SqlCreateTable( @@ -83,8 +81,8 @@ public class SqlCreateTable extends SqlCreate implements ExtendedSqlNode { List uniqueKeysList, SqlNodeList propertyList, SqlNodeList partitionKeyList, - SqlWatermark watermark, - SqlCharStringLiteral comment) { + @Nullable SqlWatermark watermark, + @Nullable SqlCharStringLiteral comment) { super(OPERATOR, pos, false, false); this.tableName = requireNonNull(tableName, "tableName should not be null"); this.columnList = requireNonNull(columnList, "columnList should not be null"); @@ -219,10 +217,12 @@ public class SqlCreateTable extends SqlCreate implements ExtendedSqlNode { * have been reversed. */ public String getColumnSqlString() { - SqlPrettyWriter writer = new SqlPrettyWriter(AnsiSqlDialect.DEFAULT); - writer.setAlwaysUseParentheses(true); - writer.setSelectListItemsOnSeparateLines(false); - writer.setIndentation(0); + SqlPrettyWriter writer = new SqlPrettyWriter( + SqlPrettyWriter.config() + .withDialect(AnsiSqlDialect.DEFAULT) + .withAlwaysUseParentheses(true) + .withSelectListItemsOnSeparateLines(false) + .withIndentation(0)); writer.startList("", ""); for (SqlNode column : columnList) { writer.sep(","); diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/ExtendedSqlBasicTypeNameSpec.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/ExtendedSqlBasicTypeNameSpec.java deleted file mode 100644 index ec81a0dd6c..0000000000 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/ExtendedSqlBasicTypeNameSpec.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * 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.sql.parser.type; - -import org.apache.calcite.sql.SqlBasicTypeNameSpec; -import org.apache.calcite.sql.SqlWriter; -import org.apache.calcite.sql.parser.SqlParserPos; -import org.apache.calcite.sql.type.SqlTypeName; - -/** - * A sql type name specification of extended basic data type, it has a counterpart basic - * sql type name but always represents as a special alias in Flink. - * - *

    For example, STRING is synonym of VARCHAR(INT_MAX) - * and BYTES is synonym of VARBINARY(INT_MAX). - */ -public class ExtendedSqlBasicTypeNameSpec extends SqlBasicTypeNameSpec { - // Type alias used for unparsing. - private final String typeAlias; - - /** - * Creates a {@code ExtendedSqlBuiltinTypeNameSpec} instance. - * - * @param typeName type name - * @param precision type precision - * @param pos parser position - */ - public ExtendedSqlBasicTypeNameSpec( - String typeAlias, - SqlTypeName typeName, - int precision, - SqlParserPos pos) { - super(typeName, precision, pos); - this.typeAlias = typeAlias; - } - - @Override - public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { - writer.keyword(typeAlias); - } -} diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/ExtendedSqlRowTypeNameSpec.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/ExtendedSqlRowTypeNameSpec.java similarity index 93% rename from flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/ExtendedSqlRowTypeNameSpec.java rename to flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/ExtendedSqlRowTypeNameSpec.java index 42828d8c8c..37e841be6f 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/ExtendedSqlRowTypeNameSpec.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/ExtendedSqlRowTypeNameSpec.java @@ -16,10 +16,16 @@ * limitations under the License. */ -package org.apache.calcite.sql; +package org.apache.flink.sql.parser.type; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.sql.SqlCharStringLiteral; +import org.apache.calcite.sql.SqlDataTypeSpec; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlRowTypeNameSpec; +import org.apache.calcite.sql.SqlTypeNameSpec; +import org.apache.calcite.sql.SqlWriter; import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.sql.validate.SqlValidator; diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/SqlMapTypeNameSpec.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/SqlMapTypeNameSpec.java similarity index 93% rename from flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/SqlMapTypeNameSpec.java rename to flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/SqlMapTypeNameSpec.java index 0e22dbfd6d..5725456a6f 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/calcite/sql/SqlMapTypeNameSpec.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/SqlMapTypeNameSpec.java @@ -16,9 +16,13 @@ * limitations under the License. */ -package org.apache.calcite.sql; +package org.apache.flink.sql.parser.type; import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.sql.SqlDataTypeSpec; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlTypeNameSpec; +import org.apache.calcite.sql.SqlWriter; import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.sql.validate.SqlValidator; diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/validate/FlinkSqlConformance.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/validate/FlinkSqlConformance.java index d53b050f2d..cf32026680 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/validate/FlinkSqlConformance.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/validate/FlinkSqlConformance.java @@ -146,6 +146,16 @@ public enum FlinkSqlConformance implements SqlConformance { return false; } + @Override + public boolean allowPluralTimeUnits() { + return false; + } + + @Override + public boolean allowQualifyingCommonColumn() { + return true; + } + /** * Whether to allow "create table T(i int, j int) partitioned by (i)" grammar. */ diff --git a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkDDLDataTypeTest.java b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkDDLDataTypeTest.java index f5edc4f4fc..5e0e0e559c 100644 --- a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkDDLDataTypeTest.java +++ b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkDDLDataTypeTest.java @@ -44,6 +44,7 @@ import org.apache.calcite.sql.parser.SqlParserTest; import org.apache.calcite.sql.parser.SqlParserUtil; import org.apache.calcite.sql.pretty.SqlPrettyWriter; import org.apache.calcite.sql.test.SqlTestFactory; +import org.apache.calcite.sql.test.SqlTests; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.sql.validate.SqlConformance; import org.apache.calcite.sql.validate.SqlConformanceEnum; @@ -51,7 +52,6 @@ import org.apache.calcite.sql.validate.SqlValidator; import org.apache.calcite.sql.validate.SqlValidatorCatalogReader; import org.apache.calcite.sql.validate.SqlValidatorUtil; import org.apache.calcite.test.MockSqlOperatorTable; -import org.apache.calcite.test.SqlValidatorTestCase; import org.apache.calcite.test.catalog.MockCatalogReaderSimple; import org.apache.calcite.util.SourceStringReader; import org.apache.calcite.util.Util; @@ -439,7 +439,7 @@ public class FlinkDDLDataTypeTest { private void checkEx(String expectedMsgPattern, SqlParserUtil.StringAndPos sap, Throwable thrown) { - SqlValidatorTestCase.checkEx(thrown, expectedMsgPattern, sap); + SqlTests.checkEx(thrown, expectedMsgPattern, sap, SqlTests.Stage.VALIDATE); } } diff --git a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java index 3057324c99..0509e91533 100644 --- a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java +++ b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java @@ -37,6 +37,7 @@ import org.junit.Before; import org.junit.Test; import java.io.Reader; +import java.util.function.UnaryOperator; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; @@ -51,9 +52,10 @@ public class FlinkSqlParserImplTest extends SqlParserTest { return FlinkSqlParserImpl.FACTORY; } - protected SqlParser getSqlParser(Reader source) { + protected SqlParser getSqlParser(Reader source, + UnaryOperator transform) { if (conformance0 == null) { - return super.getSqlParser(source); + return super.getSqlParser(source, transform); } else { // overwrite the default sql conformance. return SqlParser.create(source, @@ -75,12 +77,12 @@ public class FlinkSqlParserImplTest extends SqlParserTest { @Test public void testShowCatalogs() { - check("show catalogs", "SHOW CATALOGS"); + sql("show catalogs").ok("SHOW CATALOGS"); } @Test public void testDescribeCatalog() { - check("describe catalog a", "DESCRIBE CATALOG `A`"); + sql("describe catalog a").ok("DESCRIBE CATALOG `A`"); } /** @@ -92,7 +94,7 @@ public class FlinkSqlParserImplTest extends SqlParserTest { @Test public void testUseCatalog() { - check("use catalog a", "USE CATALOG `A`"); + sql("use catalog a").ok("USE CATALOG `A`"); } @Test @@ -112,91 +114,94 @@ public class FlinkSqlParserImplTest extends SqlParserTest { @Test public void testShowDataBases() { - check("show databases", "SHOW DATABASES"); + sql("show databases").ok("SHOW DATABASES"); } @Test public void testUseDataBase() { - check("use default_db", "USE `DEFAULT_DB`"); - check("use defaultCatalog.default_db", "USE `DEFAULTCATALOG`.`DEFAULT_DB`"); + sql("use default_db").ok("USE `DEFAULT_DB`"); + sql("use defaultCatalog.default_db").ok("USE `DEFAULTCATALOG`.`DEFAULT_DB`"); } @Test public void testCreateDatabase() { - check("create database db1", "CREATE DATABASE `DB1`"); - check("create database if not exists db1", "CREATE DATABASE IF NOT EXISTS `DB1`"); - check("create database catalog1.db1", "CREATE DATABASE `CATALOG1`.`DB1`"); - check("create database db1 comment 'test create database'", - "CREATE DATABASE `DB1`\n" + - "COMMENT 'test create database'"); - check("create database db1 comment 'test create database'" + - "with ( 'key1' = 'value1', 'key2.a' = 'value2.a')", - "CREATE DATABASE `DB1`\n" + - "COMMENT 'test create database' WITH (\n" + - " 'key1' = 'value1',\n" + - " 'key2.a' = 'value2.a'\n" + - ")"); + sql("create database db1").ok("CREATE DATABASE `DB1`"); + sql("create database if not exists db1").ok("CREATE DATABASE IF NOT EXISTS `DB1`"); + sql("create database catalog1.db1").ok("CREATE DATABASE `CATALOG1`.`DB1`"); + final String sql = "create database db1 comment 'test create database'"; + final String expected = "CREATE DATABASE `DB1`\n" + + "COMMENT 'test create database'"; + sql(sql).ok(expected); + final String sql1 = "create database db1 comment 'test create database'" + + "with ( 'key1' = 'value1', 'key2.a' = 'value2.a')"; + final String expected1 = "CREATE DATABASE `DB1`\n" + + "COMMENT 'test create database' WITH (\n" + + " 'key1' = 'value1',\n" + + " 'key2.a' = 'value2.a'\n" + + ")"; + sql(sql1).ok(expected1); } @Test public void testDropDatabase() { - check("drop database db1", "DROP DATABASE `DB1` RESTRICT"); - check("drop database catalog1.db1", "DROP DATABASE `CATALOG1`.`DB1` RESTRICT"); - check("drop database db1 RESTRICT", "DROP DATABASE `DB1` RESTRICT"); - check("drop database db1 CASCADE", "DROP DATABASE `DB1` CASCADE"); + sql("drop database db1").ok("DROP DATABASE `DB1` RESTRICT"); + sql("drop database catalog1.db1").ok("DROP DATABASE `CATALOG1`.`DB1` RESTRICT"); + sql("drop database db1 RESTRICT").ok("DROP DATABASE `DB1` RESTRICT"); + sql("drop database db1 CASCADE").ok("DROP DATABASE `DB1` CASCADE"); } @Test public void testAlterDatabase() { - check("alter database db1 set ('key1' = 'value1','key2.a' = 'value2.a')", - "ALTER DATABASE `DB1` SET (\n" + - " 'key1' = 'value1',\n" + - " 'key2.a' = 'value2.a'\n" + - ")"); + final String sql = "alter database db1 set ('key1' = 'value1','key2.a' = 'value2.a')"; + final String expected = "ALTER DATABASE `DB1` SET (\n" + + " 'key1' = 'value1',\n" + + " 'key2.a' = 'value2.a'\n" + + ")"; + sql(sql).ok(expected); } @Test public void testDescribeDatabase() { - check("describe database db1", "DESCRIBE DATABASE `DB1`"); - check("describe database catlog1.db1", "DESCRIBE DATABASE `CATLOG1`.`DB1`"); - check("describe database extended db1", "DESCRIBE DATABASE EXTENDED `DB1`"); + sql("describe database db1").ok("DESCRIBE DATABASE `DB1`"); + sql("describe database catlog1.db1").ok("DESCRIBE DATABASE `CATLOG1`.`DB1`"); + sql("describe database extended db1").ok("DESCRIBE DATABASE EXTENDED `DB1`"); } @Test public void testAlterFunction() { - check("alter function function1 as 'org.apache.fink.function.function1'", - "ALTER FUNCTION `FUNCTION1` AS 'org.apache.fink.function.function1'"); + sql("alter function function1 as 'org.apache.fink.function.function1'") + .ok("ALTER FUNCTION `FUNCTION1` AS 'org.apache.fink.function.function1'"); - check("alter temporary function function1 as 'org.apache.fink.function.function1'", - "ALTER TEMPORARY FUNCTION `FUNCTION1` AS 'org.apache.fink.function.function1'"); + sql("alter temporary function function1 as 'org.apache.fink.function.function1'") + .ok("ALTER TEMPORARY FUNCTION `FUNCTION1` AS 'org.apache.fink.function.function1'"); - check("alter temporary function function1 as 'org.apache.fink.function.function1' language scala", - "ALTER TEMPORARY FUNCTION `FUNCTION1` AS 'org.apache.fink.function.function1' LANGUAGE SCALA"); + sql("alter temporary function function1 as 'org.apache.fink.function.function1' language scala") + .ok("ALTER TEMPORARY FUNCTION `FUNCTION1` AS 'org.apache.fink.function.function1' LANGUAGE SCALA"); - check ("alter temporary system function function1 as 'org.apache.fink.function.function1'", - "ALTER TEMPORARY SYSTEM FUNCTION `FUNCTION1` AS 'org.apache.fink.function.function1'"); + sql("alter temporary system function function1 as 'org.apache.fink.function.function1'") + .ok("ALTER TEMPORARY SYSTEM FUNCTION `FUNCTION1` AS 'org.apache.fink.function.function1'"); - check("alter temporary system function function1 as 'org.apache.fink.function.function1' language java", - "ALTER TEMPORARY SYSTEM FUNCTION `FUNCTION1` AS 'org.apache.fink.function.function1' LANGUAGE JAVA"); + sql("alter temporary system function function1 as 'org.apache.fink.function.function1' language java") + .ok("ALTER TEMPORARY SYSTEM FUNCTION `FUNCTION1` AS 'org.apache.fink.function.function1' LANGUAGE JAVA"); } @Test public void testShowFuntions() { - check("show functions", "SHOW FUNCTIONS"); - check("show functions db1", "SHOW FUNCTIONS `DB1`"); - check("show functions catalog1.db1", "SHOW FUNCTIONS `CATALOG1`.`DB1`"); + sql("show functions").ok("SHOW FUNCTIONS"); + sql("show functions db1").ok("SHOW FUNCTIONS `DB1`"); + sql("show functions catalog1.db1").ok("SHOW FUNCTIONS `CATALOG1`.`DB1`"); } @Test public void testShowTables() { - check("show tables", "SHOW TABLES"); + sql("show tables").ok("SHOW TABLES"); } @Test public void testDescribeTable() { - check("describe tbl", "DESCRIBE `TBL`"); - check("describe catlog1.db1.tbl", "DESCRIBE `CATLOG1`.`DB1`.`TBL`"); - check("describe extended db1", "DESCRIBE EXTENDED `DB1`"); + sql("describe tbl").ok("DESCRIBE `TBL`"); + sql("describe catlog1.db1.tbl").ok("DESCRIBE `CATLOG1`.`DB1`.`TBL`"); + sql("describe extended db1").ok("DESCRIBE EXTENDED `DB1`"); } /** @@ -208,18 +213,19 @@ public class FlinkSqlParserImplTest extends SqlParserTest { @Test public void testAlterTable() { - check("alter table t1 rename to t2", "ALTER TABLE `T1` RENAME TO `T2`"); - check("alter table c1.d1.t1 rename to t2", "ALTER TABLE `C1`.`D1`.`T1` RENAME TO `T2`"); - check("alter table t1 set ('key1'='value1')", - "ALTER TABLE `T1` SET (\n" + - " 'key1' = 'value1'\n" + - ")"); + sql("alter table t1 rename to t2").ok("ALTER TABLE `T1` RENAME TO `T2`"); + sql("alter table c1.d1.t1 rename to t2").ok("ALTER TABLE `C1`.`D1`.`T1` RENAME TO `T2`"); + final String sql = "alter table t1 set ('key1'='value1')"; + final String expected = "ALTER TABLE `T1` SET (\n" + + " 'key1' = 'value1'\n" + + ")"; + sql(sql).ok(expected); } @Test public void testCreateTable() { conformance0 = FlinkSqlConformance.HIVE; - check("CREATE TABLE tbl1 (\n" + + final String sql = "CREATE TABLE tbl1 (\n" + " a bigint,\n" + " h varchar, \n" + " g as 2 * (a + 1), \n" + @@ -232,8 +238,8 @@ public class FlinkSqlParserImplTest extends SqlParserTest { " with (\n" + " 'connector' = 'kafka', \n" + " 'kafka.topic' = 'log.test'\n" + - ")\n", - "CREATE TABLE `TBL1` (\n" + + ")\n"; + final String expected = "CREATE TABLE `TBL1` (\n" + " `A` BIGINT,\n" + " `H` VARCHAR,\n" + " `G` AS (2 * (`A` + 1)),\n" + @@ -246,13 +252,14 @@ public class FlinkSqlParserImplTest extends SqlParserTest { "WITH (\n" + " 'connector' = 'kafka',\n" + " 'kafka.topic' = 'log.test'\n" + - ")"); + ")"; + sql(sql).ok(expected); } @Test public void testCreateTableWithComment() { conformance0 = FlinkSqlConformance.HIVE; - check("CREATE TABLE tbl1 (\n" + + final String sql = "CREATE TABLE tbl1 (\n" + " a bigint comment 'test column comment AAA.',\n" + " h varchar, \n" + " g as 2 * (a + 1), \n" + @@ -266,8 +273,8 @@ public class FlinkSqlParserImplTest extends SqlParserTest { " with (\n" + " 'connector' = 'kafka', \n" + " 'kafka.topic' = 'log.test'\n" + - ")\n", - "CREATE TABLE `TBL1` (\n" + + ")\n"; + final String expected = "CREATE TABLE `TBL1` (\n" + " `A` BIGINT COMMENT 'test column comment AAA.',\n" + " `H` VARCHAR,\n" + " `G` AS (2 * (`A` + 1)),\n" + @@ -281,13 +288,14 @@ public class FlinkSqlParserImplTest extends SqlParserTest { "WITH (\n" + " 'connector' = 'kafka',\n" + " 'kafka.topic' = 'log.test'\n" + - ")"); + ")"; + sql(sql).ok(expected); } @Test public void testCreateTableWithPrimaryKeyAndUniqueKey() { conformance0 = FlinkSqlConformance.HIVE; - check("CREATE TABLE tbl1 (\n" + + final String sql = "CREATE TABLE tbl1 (\n" + " a bigint comment 'test column comment AAA.',\n" + " h varchar, \n" + " g as 2 * (a + 1), \n" + @@ -302,8 +310,8 @@ public class FlinkSqlParserImplTest extends SqlParserTest { " with (\n" + " 'connector' = 'kafka', \n" + " 'kafka.topic' = 'log.test'\n" + - ")\n", - "CREATE TABLE `TBL1` (\n" + + ")\n"; + final String expected = "CREATE TABLE `TBL1` (\n" + " `A` BIGINT COMMENT 'test column comment AAA.',\n" + " `H` VARCHAR,\n" + " `G` AS (2 * (`A` + 1)),\n" + @@ -318,12 +326,13 @@ public class FlinkSqlParserImplTest extends SqlParserTest { "WITH (\n" + " 'connector' = 'kafka',\n" + " 'kafka.topic' = 'log.test'\n" + - ")"); + ")"; + sql(sql).ok(expected); } @Test public void testCreateTableWithWatermark() { - String sql = "CREATE TABLE tbl1 (\n" + + final String sql = "CREATE TABLE tbl1 (\n" + " ts timestamp(3),\n" + " id varchar, \n" + " watermark FOR ts AS ts - interval '3' second\n" + @@ -332,20 +341,20 @@ public class FlinkSqlParserImplTest extends SqlParserTest { " 'connector' = 'kafka', \n" + " 'kafka.topic' = 'log.test'\n" + ")\n"; - check(sql, - "CREATE TABLE `TBL1` (\n" + + final String expected = "CREATE TABLE `TBL1` (\n" + " `TS` TIMESTAMP(3),\n" + " `ID` VARCHAR,\n" + " WATERMARK FOR `TS` AS (`TS` - INTERVAL '3' SECOND)\n" + ") WITH (\n" + " 'connector' = 'kafka',\n" + " 'kafka.topic' = 'log.test'\n" + - ")"); + ")"; + sql(sql).ok(expected); } @Test public void testCreateTableWithWatermarkOnComputedColumn() { - String sql = "CREATE TABLE tbl1 (\n" + + final String sql = "CREATE TABLE tbl1 (\n" + " log_ts varchar,\n" + " ts as to_timestamp(log_ts), \n" + " WATERMARK FOR ts AS ts + interval '1' second\n" + @@ -354,34 +363,35 @@ public class FlinkSqlParserImplTest extends SqlParserTest { " 'connector' = 'kafka', \n" + " 'kafka.topic' = 'log.test'\n" + ")\n"; - check(sql, - "CREATE TABLE `TBL1` (\n" + + final String expected = "CREATE TABLE `TBL1` (\n" + " `LOG_TS` VARCHAR,\n" + " `TS` AS `TO_TIMESTAMP`(`LOG_TS`),\n" + " WATERMARK FOR `TS` AS (`TS` + INTERVAL '1' SECOND)\n" + ") WITH (\n" + " 'connector' = 'kafka',\n" + " 'kafka.topic' = 'log.test'\n" + - ")"); + ")"; + sql(sql).ok(expected); } @Test public void testCreateTableWithWatermarkOnNestedField() { - check("CREATE TABLE tbl1 (\n" + + final String sql = "CREATE TABLE tbl1 (\n" + " f1 row, q3 boolean>,\n" + " WATERMARK FOR f1.q2.t1 AS NOW()\n" + ")\n" + " with (\n" + " 'connector' = 'kafka', \n" + " 'kafka.topic' = 'log.test'\n" + - ")\n", - "CREATE TABLE `TBL1` (\n" + + ")\n"; + final String expected = "CREATE TABLE `TBL1` (\n" + " `F1` ROW< `Q1` BIGINT, `Q2` ROW< `T1` TIMESTAMP, `T2` VARCHAR >, `Q3` BOOLEAN >,\n" + " WATERMARK FOR `F1`.`Q2`.`T1` AS `NOW`()\n" + ") WITH (\n" + " 'connector' = 'kafka',\n" + " 'kafka.topic' = 'log.test'\n" + - ")"); + ")"; + sql(sql).ok(expected); } @Test @@ -434,48 +444,52 @@ public class FlinkSqlParserImplTest extends SqlParserTest { @Test public void testCreateTableWithComplexType() { - check("CREATE TABLE tbl1 (\n" + - " a ARRAY, \n" + - " b MAP,\n" + - " c ROW,\n" + - " d MULTISET,\n" + - " PRIMARY KEY (a, b) \n" + - ") with (\n" + - " 'x' = 'y', \n" + - " 'asd' = 'data'\n" + - ")\n", "CREATE TABLE `TBL1` (\n" + - " `A` ARRAY< BIGINT >,\n" + - " `B` MAP< INTEGER, VARCHAR >,\n" + - " `C` ROW< `CC0` INTEGER, `CC1` FLOAT, `CC2` VARCHAR >,\n" + - " `D` MULTISET< VARCHAR >,\n" + - " PRIMARY KEY (`A`, `B`)\n" + - ") WITH (\n" + - " 'x' = 'y',\n" + - " 'asd' = 'data'\n" + - ")"); + final String sql = "CREATE TABLE tbl1 (\n" + + " a ARRAY, \n" + + " b MAP,\n" + + " c ROW,\n" + + " d MULTISET,\n" + + " PRIMARY KEY (a, b) \n" + + ") with (\n" + + " 'x' = 'y', \n" + + " 'asd' = 'data'\n" + + ")\n"; + final String expected = "CREATE TABLE `TBL1` (\n" + + " `A` ARRAY< BIGINT >,\n" + + " `B` MAP< INTEGER, VARCHAR >,\n" + + " `C` ROW< `CC0` INTEGER, `CC1` FLOAT, `CC2` VARCHAR >,\n" + + " `D` MULTISET< VARCHAR >,\n" + + " PRIMARY KEY (`A`, `B`)\n" + + ") WITH (\n" + + " 'x' = 'y',\n" + + " 'asd' = 'data'\n" + + ")"; + sql(sql).ok(expected); } @Test public void testCreateTableWithNestedComplexType() { - check("CREATE TABLE tbl1 (\n" + - " a ARRAY>, \n" + - " b MAP, ARRAY>,\n" + - " c ROW, cc1 float, cc2 varchar>,\n" + - " d MULTISET>,\n" + - " PRIMARY KEY (a, b) \n" + - ") with (\n" + - " 'x' = 'y', \n" + - " 'asd' = 'data'\n" + - ")\n", "CREATE TABLE `TBL1` (\n" + - " `A` ARRAY< ARRAY< BIGINT > >,\n" + - " `B` MAP< MAP< INTEGER, VARCHAR >, ARRAY< VARCHAR > >,\n" + - " `C` ROW< `CC0` ARRAY< INTEGER >, `CC1` FLOAT, `CC2` VARCHAR >,\n" + - " `D` MULTISET< ARRAY< INTEGER > >,\n" + - " PRIMARY KEY (`A`, `B`)\n" + - ") WITH (\n" + - " 'x' = 'y',\n" + - " 'asd' = 'data'\n" + - ")"); + final String sql = "CREATE TABLE tbl1 (\n" + + " a ARRAY>, \n" + + " b MAP, ARRAY>,\n" + + " c ROW, cc1 float, cc2 varchar>,\n" + + " d MULTISET>,\n" + + " PRIMARY KEY (a, b) \n" + + ") with (\n" + + " 'x' = 'y', \n" + + " 'asd' = 'data'\n" + + ")\n"; + final String expected = "CREATE TABLE `TBL1` (\n" + + " `A` ARRAY< ARRAY< BIGINT > >,\n" + + " `B` MAP< MAP< INTEGER, VARCHAR >, ARRAY< VARCHAR > >,\n" + + " `C` ROW< `CC0` ARRAY< INTEGER >, `CC1` FLOAT, `CC2` VARCHAR >,\n" + + " `D` MULTISET< ARRAY< INTEGER > >,\n" + + " PRIMARY KEY (`A`, `B`)\n" + + ") WITH (\n" + + " 'x' = 'y',\n" + + " 'asd' = 'data'\n" + + ")"; + sql(sql).ok(expected); } @Test @@ -494,7 +508,7 @@ public class FlinkSqlParserImplTest extends SqlParserTest { " 'k1' = 'v1',\n" + " 'k2' = 'v2'\n" + ")"; - check(sql, expected); + sql(sql).ok(expected); } @Test @@ -530,7 +544,7 @@ public class FlinkSqlParserImplTest extends SqlParserTest { @Test public void testColumnSqlString() { - String sql = "CREATE TABLE sls_stream (\n" + + final String sql = "CREATE TABLE sls_stream (\n" + " a bigint, \n" + " f as a + 1, \n" + " b varchar,\n" + @@ -542,7 +556,7 @@ public class FlinkSqlParserImplTest extends SqlParserTest { " 'x' = 'y', \n" + " 'asd' = 'data'\n" + ")\n"; - String expected = "`A`, (`A` + 1) AS `F`, `B`, " + final String expected = "`A`, (`A` + 1) AS `F`, `B`, " + "`TOTIMESTAMP`(`B`, 'yyyy-MM-dd HH:mm:ss') AS `TS`, " + "`PROCTIME`() AS `PROC`, `C`"; sql(sql).node(new ValidationMatcher() @@ -552,7 +566,7 @@ public class FlinkSqlParserImplTest extends SqlParserTest { @Test public void testCreateInvalidPartitionedTable() { conformance0 = FlinkSqlConformance.HIVE; - String sql = "create table sls_stream1(\n" + + final String sql = "create table sls_stream1(\n" + " a bigint,\n" + " b VARCHAR,\n" + " PRIMARY KEY(a, b)\n" + @@ -567,7 +581,7 @@ public class FlinkSqlParserImplTest extends SqlParserTest { @Test public void testNotAllowedCreatePartition() { conformance0 = FlinkSqlConformance.DEFAULT; - String sql = "create table sls_stream1(\n" + + final String sql = "create table sls_stream1(\n" + " a bigint,\n" + " b VARCHAR\n" + ") PARTITIONED BY (a^)^ with ( 'x' = 'y', 'asd' = 'dada')"; @@ -576,7 +590,7 @@ public class FlinkSqlParserImplTest extends SqlParserTest { @Test public void testCreateTableWithMinusInOptionKey() { - String sql = "create table source_table(\n" + + final String sql = "create table source_table(\n" + " a int,\n" + " b bigint,\n" + " c string\n" + @@ -586,7 +600,7 @@ public class FlinkSqlParserImplTest extends SqlParserTest { " 'a.b-c-connector.e-f.g' = 'ada',\n" + " 'a.b-c-d.e-1231.g' = 'ada',\n" + " 'a.b-c-d.*' = 'adad')\n"; - String expected = "CREATE TABLE `SOURCE_TABLE` (\n" + + final String expected = "CREATE TABLE `SOURCE_TABLE` (\n" + " `A` INTEGER,\n" + " `B` BIGINT,\n" + " `C` STRING\n" + @@ -597,12 +611,12 @@ public class FlinkSqlParserImplTest extends SqlParserTest { " 'a.b-c-d.e-1231.g' = 'ada',\n" + " 'a.b-c-d.*' = 'adad'\n" + ")"; - check(sql, expected); + sql(sql).ok(expected); } @Test public void testCreateTableWithOptionKeyAsIdentifier() { - String sql = "create table source_table(\n" + + final String sql = "create table source_table(\n" + " a int,\n" + " b bigint,\n" + " c string\n" + @@ -614,14 +628,16 @@ public class FlinkSqlParserImplTest extends SqlParserTest { @Test public void testDropTable() { - String sql = "DROP table catalog1.db1.tbl1"; - check(sql, "DROP TABLE `CATALOG1`.`DB1`.`TBL1`"); + final String sql = "DROP table catalog1.db1.tbl1"; + final String expected = "DROP TABLE `CATALOG1`.`DB1`.`TBL1`"; + sql(sql).ok(expected); } @Test public void testDropIfExists() { - String sql = "DROP table IF EXISTS catalog1.db1.tbl1"; - check(sql, "DROP TABLE IF EXISTS `CATALOG1`.`DB1`.`TBL1`"); + final String sql = "DROP table IF EXISTS catalog1.db1.tbl1"; + final String expected = "DROP TABLE IF EXISTS `CATALOG1`.`DB1`.`TBL1`"; + sql(sql).ok(expected); } @Test @@ -658,7 +674,7 @@ public class FlinkSqlParserImplTest extends SqlParserTest { @Test public void testInsertExtendedColumnAsStaticPartition1() { - String expected = "INSERT INTO `EMPS` EXTEND (`Z` BOOLEAN) (`X`, `Y`)\n" + final String expected = "INSERT INTO `EMPS` EXTEND (`Z` BOOLEAN) (`X`, `Y`)\n" + "PARTITION (`Z` = 'ab')\n" + "(SELECT *\n" + "FROM `EMPS`)"; @@ -676,17 +692,19 @@ public class FlinkSqlParserImplTest extends SqlParserTest { @Test public void testInsertOverwrite() { // non-partitioned - check("INSERT OVERWRITE myDB.myTbl SELECT * FROM src", - "INSERT OVERWRITE `MYDB`.`MYTBL`\n" + final String sql = "INSERT OVERWRITE myDB.myTbl SELECT * FROM src"; + final String expected = "INSERT OVERWRITE `MYDB`.`MYTBL`\n" + "(SELECT *\n" - + "FROM `SRC`)"); + + "FROM `SRC`)"; + sql(sql).ok(expected); // partitioned - check("INSERT OVERWRITE myTbl PARTITION (p1='v1',p2='v2') SELECT * FROM src", - "INSERT OVERWRITE `MYTBL`\n" + final String sql1 = "INSERT OVERWRITE myTbl PARTITION (p1='v1',p2='v2') SELECT * FROM src"; + final String expected1 = "INSERT OVERWRITE `MYTBL`\n" + "PARTITION (`P1` = 'v1', `P2` = 'v2')\n" + "(SELECT *\n" - + "FROM `SRC`)"); + + "FROM `SRC`)"; + sql(sql1).ok(expected1); } @Test @@ -702,7 +720,7 @@ public class FlinkSqlParserImplTest extends SqlParserTest { "AS\n" + "SELECT `COL1`\n" + "FROM `TBL`"; - check(sql, expected); + sql(sql).ok(expected); } @Test @@ -713,7 +731,7 @@ public class FlinkSqlParserImplTest extends SqlParserTest { "AS\n" + "SELECT `COL1`\n" + "FROM `TBL`"; - check(sql, expected); + sql(sql).ok(expected); } @Test @@ -723,7 +741,7 @@ public class FlinkSqlParserImplTest extends SqlParserTest { "AS\n" + "SELECT `COL3`, `COL4`\n" + "FROM `TBL`"; - check(sql, expected); + sql(sql).ok(expected); } @Test @@ -731,31 +749,41 @@ public class FlinkSqlParserImplTest extends SqlParserTest { final String sql = "create view v(^*^) COMMENT 'this is a view' as select col1 from tbl"; final String expected = "(?s).*Encountered \"\\*\" at line 1, column 15.*"; - checkFails(sql, expected); + sql(sql).fails(expected); } @Test public void testDropView() { final String sql = "DROP VIEW IF EXISTS view_name"; - check(sql, "DROP VIEW IF EXISTS `VIEW_NAME`"); + final String expected = "DROP VIEW IF EXISTS `VIEW_NAME`"; + sql(sql).ok(expected); } // Override the test because our ROW field type default is nullable, // which is different with Calcite. @Override public void testCastAsRowType() { - checkExp("cast(a as row(f0 int, f1 varchar))", - "CAST(`A` AS ROW(`F0` INTEGER, `F1` VARCHAR))"); - checkExp("cast(a as row(f0 int not null, f1 varchar null))", - "CAST(`A` AS ROW(`F0` INTEGER NOT NULL, `F1` VARCHAR))"); - checkExp("cast(a as row(f0 row(ff0 int not null, ff1 varchar null) null," + - " f1 timestamp not null))", - "CAST(`A` AS ROW(`F0` ROW(`FF0` INTEGER NOT NULL, `FF1` VARCHAR)," + - " `F1` TIMESTAMP NOT NULL))"); - checkExp("cast(a as row(f0 bigint not null, f1 decimal null) array)", - "CAST(`A` AS ROW(`F0` BIGINT NOT NULL, `F1` DECIMAL) ARRAY)"); - checkExp("cast(a as row(f0 varchar not null, f1 timestamp null) multiset)", - "CAST(`A` AS ROW(`F0` VARCHAR NOT NULL, `F1` TIMESTAMP) MULTISET)"); + final String expr = "cast(a as row(f0 int, f1 varchar))"; + final String expected = "CAST(`A` AS ROW(`F0` INTEGER, `F1` VARCHAR))"; + expr(expr).ok(expected); + + final String expr1 = "cast(a as row(f0 int not null, f1 varchar null))"; + final String expected1 = "CAST(`A` AS ROW(`F0` INTEGER NOT NULL, `F1` VARCHAR))"; + expr(expr1).ok(expected1); + + final String expr2 = "cast(a as row(f0 row(ff0 int not null, ff1 varchar null) null," + + " f1 timestamp not null))"; + final String expected2 = "CAST(`A` AS ROW(`F0` ROW(`FF0` INTEGER NOT NULL, `FF1` VARCHAR)," + + " `F1` TIMESTAMP NOT NULL))"; + expr(expr2).ok(expected2); + + final String expr3 = "cast(a as row(f0 bigint not null, f1 decimal null) array)"; + final String expected3 = "CAST(`A` AS ROW(`F0` BIGINT NOT NULL, `F1` DECIMAL) ARRAY)"; + expr(expr3).ok(expected3); + + final String expr4 = "cast(a as row(f0 varchar not null, f1 timestamp null) multiset)"; + final String expected4 = "CAST(`A` AS ROW(`F0` VARCHAR NOT NULL, `F1` TIMESTAMP) MULTISET)"; + expr(expr4).ok(expected4); } @Test @@ -772,44 +800,50 @@ public class FlinkSqlParserImplTest extends SqlParserTest { @Test public void testCreateFunction() { - check("create function catalog1.db1.function1 as 'org.apache.fink.function.function1'", - "CREATE FUNCTION `CATALOG1`.`DB1`.`FUNCTION1` AS 'org.apache.fink.function.function1'"); + sql("create function catalog1.db1.function1 as 'org.apache.fink.function.function1'") + .ok("CREATE FUNCTION `CATALOG1`.`DB1`.`FUNCTION1` AS 'org.apache.fink.function.function1'"); - check("create temporary function catalog1.db1.function1 as 'org.apache.fink.function.function1'", - "CREATE TEMPORARY FUNCTION `CATALOG1`.`DB1`.`FUNCTION1` AS 'org.apache.fink.function.function1'"); + sql("create temporary function catalog1.db1.function1 as 'org.apache.fink.function.function1'") + .ok("CREATE TEMPORARY FUNCTION `CATALOG1`.`DB1`.`FUNCTION1` AS 'org.apache.fink.function.function1'"); - check("create temporary system function catalog1.db1.function1 as 'org.apache.fink.function.function1'", - "CREATE TEMPORARY SYSTEM FUNCTION `CATALOG1`.`DB1`.`FUNCTION1` AS 'org.apache.fink.function.function1'"); + sql("create temporary system function catalog1.db1.function1 as 'org.apache.fink.function.function1'") + .ok("CREATE TEMPORARY SYSTEM FUNCTION `CATALOG1`.`DB1`.`FUNCTION1` AS 'org.apache.fink.function.function1'"); - check("create temporary function db1.function1 as 'org.apache.fink.function.function1'", - "CREATE TEMPORARY FUNCTION `DB1`.`FUNCTION1` AS 'org.apache.fink.function.function1'"); + sql("create temporary function db1.function1 as 'org.apache.fink.function.function1'") + .ok("CREATE TEMPORARY FUNCTION `DB1`.`FUNCTION1` AS 'org.apache.fink.function.function1'"); - check("create temporary function function1 as 'org.apache.fink.function.function1'", - "CREATE TEMPORARY FUNCTION `FUNCTION1` AS 'org.apache.fink.function.function1'"); + sql("create temporary function function1 as 'org.apache.fink.function.function1'") + .ok("CREATE TEMPORARY FUNCTION `FUNCTION1` AS 'org.apache.fink.function.function1'"); - check("create temporary function if not exists catalog1.db1.function1 as 'org.apache.fink.function.function1'", - "CREATE TEMPORARY FUNCTION IF NOT EXISTS `CATALOG1`.`DB1`.`FUNCTION1` AS 'org.apache.fink.function.function1'"); + sql("create temporary function if not exists catalog1.db1.function1 as 'org.apache.fink.function.function1'") + .ok("CREATE TEMPORARY FUNCTION IF NOT EXISTS `CATALOG1`.`DB1`.`FUNCTION1` AS 'org.apache.fink.function.function1'"); - check("create temporary function function1 as 'org.apache.fink.function.function1' language java", - "CREATE TEMPORARY FUNCTION `FUNCTION1` AS 'org.apache.fink.function.function1' LANGUAGE JAVA"); + sql("create temporary function function1 as 'org.apache.fink.function.function1' language java") + .ok("CREATE TEMPORARY FUNCTION `FUNCTION1` AS 'org.apache.fink.function.function1' LANGUAGE JAVA"); - check("create temporary system function function1 as 'org.apache.fink.function.function1' language scala", - "CREATE TEMPORARY SYSTEM FUNCTION `FUNCTION1` AS 'org.apache.fink.function.function1' LANGUAGE SCALA"); + sql("create temporary system function function1 as 'org.apache.fink.function.function1' language scala") + .ok("CREATE TEMPORARY SYSTEM FUNCTION `FUNCTION1` AS 'org.apache.fink.function.function1' LANGUAGE SCALA"); } @Test public void testDropTemporaryFunction() { - check("drop temporary function catalog1.db1.function1", - "DROP TEMPORARY FUNCTION `CATALOG1`.`DB1`.`FUNCTION1`"); + sql("drop temporary function catalog1.db1.function1") + .ok("DROP TEMPORARY FUNCTION `CATALOG1`.`DB1`.`FUNCTION1`"); - check("drop temporary system function catalog1.db1.function1", - "DROP TEMPORARY SYSTEM FUNCTION `CATALOG1`.`DB1`.`FUNCTION1`"); + sql("drop temporary system function catalog1.db1.function1") + .ok("DROP TEMPORARY SYSTEM FUNCTION `CATALOG1`.`DB1`.`FUNCTION1`"); - check("drop temporary function if exists catalog1.db1.function1", - "DROP TEMPORARY FUNCTION IF EXISTS `CATALOG1`.`DB1`.`FUNCTION1`"); + sql("drop temporary function if exists catalog1.db1.function1") + .ok("DROP TEMPORARY FUNCTION IF EXISTS `CATALOG1`.`DB1`.`FUNCTION1`"); - check("drop temporary system function if exists catalog1.db1.function1", - "DROP TEMPORARY SYSTEM FUNCTION IF EXISTS `CATALOG1`.`DB1`.`FUNCTION1`"); + sql("drop temporary system function if exists catalog1.db1.function1") + .ok("DROP TEMPORARY SYSTEM FUNCTION IF EXISTS `CATALOG1`.`DB1`.`FUNCTION1`"); + } + + @Override + public void testTableHintsInInsert() { + // Override the superclass tests because Flink insert parse block + // is totally customized, and the hints are not supported yet. } /** Matcher that invokes the #validate() of the {@link ExtendedSqlNode} instance. **/ -- Gitee From 07dec981ccb368f64a86181ffba7b48b6409621c Mon Sep 17 00:00:00 2001 From: "yuzhao.cyz" Date: Tue, 17 Mar 2020 19:09:48 +0800 Subject: [PATCH 367/885] [FLINK-14338][table-planner][table-planner-blink] Implements new RelMetadataQuery extension * This feature is introduced in CALCITE-3446 --- .../apache/calcite/plan/RelOptCluster.java | 203 ----------------- .../plan/metadata/FlinkRelMetadataQuery.java | 64 +++--- .../calcite/FlinkRelOptClusterFactory.scala | 15 +- .../AggCallSelectivityEstimatorTest.scala | 4 +- .../metadata/FlinkRelMdHandlerTestBase.scala | 4 +- .../metadata/SelectivityEstimatorTest.scala | 4 +- .../apache/calcite/plan/RelOptCluster.java | 204 ------------------ .../calcite/FlinkRelOptClusterFactory.scala | 6 +- 8 files changed, 43 insertions(+), 461 deletions(-) delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/plan/RelOptCluster.java delete mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/plan/RelOptCluster.java diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/plan/RelOptCluster.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/plan/RelOptCluster.java deleted file mode 100644 index e32a5973a3..0000000000 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/plan/RelOptCluster.java +++ /dev/null @@ -1,203 +0,0 @@ -/* - * 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.calcite.plan; - -import org.apache.calcite.rel.RelNode; -import org.apache.calcite.rel.core.CorrelationId; -import org.apache.calcite.rel.metadata.DefaultRelMetadataProvider; -import org.apache.calcite.rel.metadata.MetadataFactory; -import org.apache.calcite.rel.metadata.MetadataFactoryImpl; -import org.apache.calcite.rel.metadata.RelMetadataProvider; -import org.apache.calcite.rel.metadata.RelMetadataQuery; -import org.apache.calcite.rel.type.RelDataTypeFactory; -import org.apache.calcite.rex.RexBuilder; -import org.apache.calcite.rex.RexNode; - -import java.util.HashMap; -import java.util.Map; -import java.util.Objects; -import java.util.concurrent.atomic.AtomicInteger; - -/** - * This class is copied from Calcite's {@link org.apache.calcite.plan.RelOptCluster}, - * can be removed after https://issues.apache.org/jira/browse/CALCITE-2855 is accepted. - * NOTES: please make sure to synchronize with RelDecorrelator in flink planner when changing this class. - * - * Modification: - * - Make non-deprecated constructor public - */ - -/** - * An environment for related relational expressions during the - * optimization of a query. - */ -public class RelOptCluster { - //~ Instance fields -------------------------------------------------------- - - private final RelDataTypeFactory typeFactory; - private final RelOptPlanner planner; - private final AtomicInteger nextCorrel; - private final Map mapCorrelToRel; - private RexNode originalExpression; - private final RexBuilder rexBuilder; - private RelMetadataProvider metadataProvider; - private MetadataFactory metadataFactory; - private final RelTraitSet emptyTraitSet; - private RelMetadataQuery mq; - - //~ Constructors ----------------------------------------------------------- - - /** - * Creates a cluster. - */ - @Deprecated // to be removed before 2.0 - RelOptCluster( - RelOptQuery query, - RelOptPlanner planner, - RelDataTypeFactory typeFactory, - RexBuilder rexBuilder) { - this(planner, typeFactory, rexBuilder, query.nextCorrel, - query.mapCorrelToRel); - } - - /** - * Creates a cluster. - * - *

    For use only from {@link #create} and {@link RelOptQuery}. - */ - public RelOptCluster(RelOptPlanner planner, RelDataTypeFactory typeFactory, - RexBuilder rexBuilder, AtomicInteger nextCorrel, - Map mapCorrelToRel) { - this.nextCorrel = nextCorrel; - this.mapCorrelToRel = mapCorrelToRel; - this.planner = Objects.requireNonNull(planner); - this.typeFactory = Objects.requireNonNull(typeFactory); - this.rexBuilder = rexBuilder; - this.originalExpression = rexBuilder.makeLiteral("?"); - - // set up a default rel metadata provider, - // giving the planner first crack at everything - setMetadataProvider(DefaultRelMetadataProvider.INSTANCE); - this.emptyTraitSet = planner.emptyTraitSet(); - assert emptyTraitSet.size() == planner.getRelTraitDefs().size(); - } - - /** Creates a cluster. */ - public static RelOptCluster create(RelOptPlanner planner, - RexBuilder rexBuilder) { - return new RelOptCluster(planner, rexBuilder.getTypeFactory(), - rexBuilder, new AtomicInteger(0), new HashMap()); - } - - //~ Methods ---------------------------------------------------------------- - - @Deprecated // to be removed before 2.0 - public RelOptQuery getQuery() { - return new RelOptQuery(planner, nextCorrel, mapCorrelToRel); - } - - @Deprecated // to be removed before 2.0 - public RexNode getOriginalExpression() { - return originalExpression; - } - - @Deprecated // to be removed before 2.0 - public void setOriginalExpression(RexNode originalExpression) { - this.originalExpression = originalExpression; - } - - public RelOptPlanner getPlanner() { - return planner; - } - - public RelDataTypeFactory getTypeFactory() { - return typeFactory; - } - - public RexBuilder getRexBuilder() { - return rexBuilder; - } - - public RelMetadataProvider getMetadataProvider() { - return metadataProvider; - } - - /** - * Overrides the default metadata provider for this cluster. - * - * @param metadataProvider custom provider - */ - public void setMetadataProvider(RelMetadataProvider metadataProvider) { - this.metadataProvider = metadataProvider; - this.metadataFactory = new MetadataFactoryImpl(metadataProvider); - } - - public MetadataFactory getMetadataFactory() { - return metadataFactory; - } - - /** Returns the current RelMetadataQuery. - * - *

    This method might be changed or moved in future. - * If you have a {@link RelOptRuleCall} available, - * for example if you are in a {@link RelOptRule#onMatch(RelOptRuleCall)} - * method, then use {@link RelOptRuleCall#getMetadataQuery()} instead. */ - public RelMetadataQuery getMetadataQuery() { - if (mq == null) { - mq = RelMetadataQuery.instance(); - } - return mq; - } - - /** - * Should be called whenever the current {@link RelMetadataQuery} becomes - * invalid. Typically invoked from {@link RelOptRuleCall#transformTo}. - */ - public void invalidateMetadataQuery() { - mq = null; - } - - /** - * Constructs a new id for a correlating variable. It is unique within the - * whole query. - */ - public CorrelationId createCorrel() { - return new CorrelationId(nextCorrel.getAndIncrement()); - } - - /** Returns the default trait set for this cluster. */ - public RelTraitSet traitSet() { - return emptyTraitSet; - } - - /** @deprecated For {@code traitSetOf(t1, t2)}, - * use {@link #traitSet}().replace(t1).replace(t2). */ - @Deprecated // to be removed before 2.0 - public RelTraitSet traitSetOf(RelTrait... traits) { - RelTraitSet traitSet = emptyTraitSet; - for (RelTrait trait : traits) { - traitSet = traitSet.replace(trait); - } - return traitSet; - } - - public RelTraitSet traitSetOf(RelTrait trait) { - return emptyTraitSet.replace(trait); - } -} - -// End RelOptCluster.java diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/metadata/FlinkRelMetadataQuery.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/metadata/FlinkRelMetadataQuery.java index 7d70678c7a..f41954e8e0 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/metadata/FlinkRelMetadataQuery.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/metadata/FlinkRelMetadataQuery.java @@ -33,8 +33,8 @@ import org.apache.calcite.util.ImmutableBitSet; * e.g ColumnInterval, ColumnNullCount. */ public class FlinkRelMetadataQuery extends RelMetadataQuery { - - protected static final FlinkRelMetadataQuery PROTOTYPE = new FlinkRelMetadataQuery(false); + // Serves as the handlers prototype of all the FlinkRelMetadataQuery instances. + private static final Handlers HANDLERS = new Handlers(); private FlinkMetadata.ColumnInterval.Handler columnIntervalHandler; private FlinkMetadata.FilteredColumnInterval.Handler filteredColumnInterval; @@ -67,43 +67,35 @@ public class FlinkRelMetadataQuery extends RelMetadataQuery { } } - private FlinkRelMetadataQuery( - JaninoRelMetadataProvider metadataProvider, - RelMetadataQuery prototype) { - super(metadataProvider, prototype); - } - + /** + * Creates a FlinkRelMetadataQuery instance. + */ private FlinkRelMetadataQuery() { - super(RelMetadataQuery.THREAD_PROVIDERS.get(), RelMetadataQuery.EMPTY); - this.columnIntervalHandler = PROTOTYPE.columnIntervalHandler; - this.filteredColumnInterval = PROTOTYPE.filteredColumnInterval; - this.columnNullCountHandler = PROTOTYPE.columnNullCountHandler; - this.columnOriginNullCountHandler = PROTOTYPE.columnOriginNullCountHandler; - this.uniqueGroupsHandler = PROTOTYPE.uniqueGroupsHandler; - this.distributionHandler = PROTOTYPE.distributionHandler; - this.modifiedMonotonicityHandler = PROTOTYPE.modifiedMonotonicityHandler; + this.columnIntervalHandler = HANDLERS.columnIntervalHandler; + this.filteredColumnInterval = HANDLERS.filteredColumnInterval; + this.columnNullCountHandler = HANDLERS.columnNullCountHandler; + this.columnOriginNullCountHandler = HANDLERS.columnOriginNullCountHandler; + this.uniqueGroupsHandler = HANDLERS.uniqueGroupsHandler; + this.distributionHandler = HANDLERS.distributionHandler; + this.modifiedMonotonicityHandler = HANDLERS.modifiedMonotonicityHandler; } - /** - * Creates and initializes the instance that will serve as a prototype for - * all other instances. - */ - private FlinkRelMetadataQuery(boolean dummy) { - super(RelMetadataQuery.THREAD_PROVIDERS.get(), RelMetadataQuery.EMPTY); - this.columnIntervalHandler = - RelMetadataQuery.initialHandler(FlinkMetadata.ColumnInterval.Handler.class); - this.filteredColumnInterval = - RelMetadataQuery.initialHandler(FlinkMetadata.FilteredColumnInterval.Handler.class); - this.columnNullCountHandler = - RelMetadataQuery.initialHandler(FlinkMetadata.ColumnNullCount.Handler.class); - this.columnOriginNullCountHandler = - RelMetadataQuery.initialHandler(FlinkMetadata.ColumnOriginNullCount.Handler.class); - this.uniqueGroupsHandler = - RelMetadataQuery.initialHandler(FlinkMetadata.UniqueGroups.Handler.class); - this.distributionHandler = - RelMetadataQuery.initialHandler(FlinkMetadata.FlinkDistribution.Handler.class); - this.modifiedMonotonicityHandler = - RelMetadataQuery.initialHandler(FlinkMetadata.ModifiedMonotonicity.Handler.class); + /** Extended handlers. */ + private static class Handlers { + private FlinkMetadata.ColumnInterval.Handler columnIntervalHandler = + initialHandler(FlinkMetadata.ColumnInterval.Handler.class); + private FlinkMetadata.FilteredColumnInterval.Handler filteredColumnInterval = + initialHandler(FlinkMetadata.FilteredColumnInterval.Handler.class); + private FlinkMetadata.ColumnNullCount.Handler columnNullCountHandler = + initialHandler(FlinkMetadata.ColumnNullCount.Handler.class); + private FlinkMetadata.ColumnOriginNullCount.Handler columnOriginNullCountHandler = + initialHandler(FlinkMetadata.ColumnOriginNullCount.Handler.class); + private FlinkMetadata.UniqueGroups.Handler uniqueGroupsHandler = + initialHandler(FlinkMetadata.UniqueGroups.Handler.class); + private FlinkMetadata.FlinkDistribution.Handler distributionHandler = + initialHandler(FlinkMetadata.FlinkDistribution.Handler.class); + private FlinkMetadata.ModifiedMonotonicity.Handler modifiedMonotonicityHandler = + initialHandler(FlinkMetadata.ModifiedMonotonicity.Handler.class); } /** diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelOptClusterFactory.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelOptClusterFactory.scala index 629361d139..93e16c22a9 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelOptClusterFactory.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelOptClusterFactory.scala @@ -18,12 +18,14 @@ package org.apache.flink.table.planner.calcite -import org.apache.flink.table.planner.plan.metadata.FlinkDefaultRelMetadataProvider +import org.apache.flink.table.planner.plan.metadata.{FlinkDefaultRelMetadataProvider, FlinkRelMetadataQuery} import org.apache.calcite.plan.{RelOptCluster, RelOptPlanner} -import org.apache.calcite.rel.metadata.{DefaultRelMetadataProvider, JaninoRelMetadataProvider, RelMetadataQuery} +import org.apache.calcite.rel.metadata.{DefaultRelMetadataProvider, RelMetadataQuery} import org.apache.calcite.rex.RexBuilder +import java.util.function.Supplier + /** * The utility class is to create special [[RelOptCluster]] instance which use * [[FlinkDefaultRelMetadataProvider]] instead of [[DefaultRelMetadataProvider]]. @@ -31,12 +33,11 @@ import org.apache.calcite.rex.RexBuilder object FlinkRelOptClusterFactory { def create(planner: RelOptPlanner, rexBuilder: RexBuilder): RelOptCluster = { - val cluster = FlinkRelOptCluster.create(planner, rexBuilder) + val cluster = RelOptCluster.create(planner, rexBuilder) cluster.setMetadataProvider(FlinkDefaultRelMetadataProvider.INSTANCE) - // just set metadataProvider is not enough, see - // https://www.mail-archive.com/dev@calcite.apache.org/msg00930.html - RelMetadataQuery.THREAD_PROVIDERS.set( - JaninoRelMetadataProvider.of(cluster.getMetadataProvider)) + cluster.setMetadataQuerySupplier(new Supplier[RelMetadataQuery]() { + def get: FlinkRelMetadataQuery = FlinkRelMetadataQuery.instance() + }) cluster } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/AggCallSelectivityEstimatorTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/AggCallSelectivityEstimatorTest.scala index da81b360db..0fa3485799 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/AggCallSelectivityEstimatorTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/AggCallSelectivityEstimatorTest.scala @@ -31,7 +31,7 @@ import org.apache.calcite.plan.{AbstractRelOptPlanner, RelOptCluster} import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.{Aggregate, AggregateCall, TableScan} import org.apache.calcite.rel.logical.LogicalAggregate -import org.apache.calcite.rel.metadata.{JaninoRelMetadataProvider, RelMetadataQuery} +import org.apache.calcite.rel.metadata.{JaninoRelMetadataProvider, RelMetadataQueryBase} import org.apache.calcite.rex.{RexBuilder, RexInputRef, RexLiteral, RexNode} import org.apache.calcite.sql.`type`.SqlTypeName import org.apache.calcite.sql.`type`.SqlTypeName._ @@ -628,7 +628,7 @@ object AggCallSelectivityEstimatorTest { @BeforeClass def beforeAll(): Unit = { - RelMetadataQuery + RelMetadataQueryBase .THREAD_PROVIDERS .set(JaninoRelMetadataProvider.of(FlinkDefaultRelMetadataProvider.INSTANCE)) } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala index 6ac2d6811c..20d91fc889 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala @@ -57,7 +57,7 @@ import org.apache.calcite.rel._ import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFieldImpl} import org.apache.calcite.rel.core.{AggregateCall, Calc, JoinInfo, JoinRelType, Project, Window} import org.apache.calcite.rel.logical.{LogicalAggregate, LogicalProject, LogicalSort, LogicalTableScan, LogicalValues} -import org.apache.calcite.rel.metadata.{JaninoRelMetadataProvider, RelMetadataQuery} +import org.apache.calcite.rel.metadata.{JaninoRelMetadataProvider, RelMetadataQuery, RelMetadataQueryBase} import org.apache.calcite.rex._ import org.apache.calcite.schema.SchemaPlus import org.apache.calcite.sql.SqlWindow @@ -2446,7 +2446,7 @@ class TestRel( object FlinkRelMdHandlerTestBase { @BeforeClass def beforeAll(): Unit = { - RelMetadataQuery + RelMetadataQueryBase .THREAD_PROVIDERS .set(JaninoRelMetadataProvider.of(FlinkDefaultRelMetadataProvider.INSTANCE)) } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimatorTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimatorTest.scala index 7a7e034c40..911d732174 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimatorTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimatorTest.scala @@ -30,7 +30,7 @@ import org.apache.flink.util.Preconditions import org.apache.calcite.plan.{AbstractRelOptPlanner, RelOptCluster} import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.TableScan -import org.apache.calcite.rel.metadata.{JaninoRelMetadataProvider, RelMetadataQuery} +import org.apache.calcite.rel.metadata.{JaninoRelMetadataProvider, RelMetadataQueryBase} import org.apache.calcite.rex.{RexBuilder, RexInputRef, RexLiteral, RexNode} import org.apache.calcite.sql.SqlOperator import org.apache.calcite.sql.`type`.SqlTypeName._ @@ -1107,7 +1107,7 @@ object SelectivityEstimatorTest { @BeforeClass def beforeAll(): Unit = { - RelMetadataQuery + RelMetadataQueryBase .THREAD_PROVIDERS .set(JaninoRelMetadataProvider.of(FlinkDefaultRelMetadataProvider.INSTANCE)) } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/plan/RelOptCluster.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/plan/RelOptCluster.java deleted file mode 100644 index 82a2bd6d5c..0000000000 --- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/plan/RelOptCluster.java +++ /dev/null @@ -1,204 +0,0 @@ -/* - * 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.calcite.plan; - -import org.apache.calcite.rel.RelNode; -import org.apache.calcite.rel.core.CorrelationId; -import org.apache.calcite.rel.metadata.DefaultRelMetadataProvider; -import org.apache.calcite.rel.metadata.MetadataFactory; -import org.apache.calcite.rel.metadata.MetadataFactoryImpl; -import org.apache.calcite.rel.metadata.RelMetadataProvider; -import org.apache.calcite.rel.metadata.RelMetadataQuery; -import org.apache.calcite.rel.type.RelDataTypeFactory; -import org.apache.calcite.rex.RexBuilder; -import org.apache.calcite.rex.RexNode; - -import java.util.HashMap; -import java.util.Map; -import java.util.Objects; -import java.util.concurrent.atomic.AtomicInteger; - -/** - * This class is copied from Calcite's {@link org.apache.calcite.plan.RelOptCluster}, - * can be removed after https://issues.apache.org/jira/browse/CALCITE-2855 is accepted. - * NOTES: please make sure to synchronize with RelDecorrelator in blink planner when changing this class. - * - * Modification: - * - Make non-deprecated constructor public - */ - -/** - * An environment for related relational expressions during the - * optimization of a query. - */ -public class RelOptCluster { - //~ Instance fields -------------------------------------------------------- - - private final RelDataTypeFactory typeFactory; - private final RelOptPlanner planner; - private final AtomicInteger nextCorrel; - private final Map mapCorrelToRel; - private RexNode originalExpression; - private final RexBuilder rexBuilder; - private RelMetadataProvider metadataProvider; - private MetadataFactory metadataFactory; - private final RelTraitSet emptyTraitSet; - private RelMetadataQuery mq; - - //~ Constructors ----------------------------------------------------------- - - /** - * Creates a cluster. - */ - @Deprecated // to be removed before 2.0 - RelOptCluster( - RelOptQuery query, - RelOptPlanner planner, - RelDataTypeFactory typeFactory, - RexBuilder rexBuilder) { - this(planner, typeFactory, rexBuilder, query.nextCorrel, - query.mapCorrelToRel); - } - - /** - * Creates a cluster. - * - *

    For use only from {@link #create} and {@link RelOptQuery}. - */ - public RelOptCluster(RelOptPlanner planner, RelDataTypeFactory typeFactory, - RexBuilder rexBuilder, AtomicInteger nextCorrel, - Map mapCorrelToRel) { - this.nextCorrel = nextCorrel; - this.mapCorrelToRel = mapCorrelToRel; - this.planner = Objects.requireNonNull(planner); - this.typeFactory = Objects.requireNonNull(typeFactory); - this.rexBuilder = rexBuilder; - this.originalExpression = rexBuilder.makeLiteral("?"); - - // set up a default rel metadata provider, - // giving the planner first crack at everything - setMetadataProvider(DefaultRelMetadataProvider.INSTANCE); - this.emptyTraitSet = planner.emptyTraitSet(); - assert emptyTraitSet.size() == planner.getRelTraitDefs().size(); - } - - /** Creates a cluster. */ - public static RelOptCluster create(RelOptPlanner planner, - RexBuilder rexBuilder) { - return new RelOptCluster(planner, rexBuilder.getTypeFactory(), - rexBuilder, new AtomicInteger(0), new HashMap()); - } - - //~ Methods ---------------------------------------------------------------- - - @Deprecated // to be removed before 2.0 - public RelOptQuery getQuery() { - return new RelOptQuery(planner, nextCorrel, mapCorrelToRel); - } - - @Deprecated // to be removed before 2.0 - public RexNode getOriginalExpression() { - return originalExpression; - } - - @Deprecated // to be removed before 2.0 - public void setOriginalExpression(RexNode originalExpression) { - this.originalExpression = originalExpression; - } - - public RelOptPlanner getPlanner() { - return planner; - } - - public RelDataTypeFactory getTypeFactory() { - return typeFactory; - } - - public RexBuilder getRexBuilder() { - return rexBuilder; - } - - public RelMetadataProvider getMetadataProvider() { - return metadataProvider; - } - - /** - * Overrides the default metadata provider for this cluster. - * - * @param metadataProvider custom provider - */ - public void setMetadataProvider(RelMetadataProvider metadataProvider) { - this.metadataProvider = metadataProvider; - this.metadataFactory = new MetadataFactoryImpl(metadataProvider); - } - - public MetadataFactory getMetadataFactory() { - return metadataFactory; - } - - /** Returns the current RelMetadataQuery. - * - *

    This method might be changed or moved in future. - * If you have a {@link RelOptRuleCall} available, - * for example if you are in a {@link RelOptRule#onMatch(RelOptRuleCall)} - * method, then use {@link RelOptRuleCall#getMetadataQuery()} instead. */ - public RelMetadataQuery getMetadataQuery() { - if (mq == null) { - mq = RelMetadataQuery.instance(); - } - return mq; - } - - /** - * Should be called whenever the current {@link RelMetadataQuery} becomes - * invalid. Typically invoked from {@link RelOptRuleCall#transformTo}. - */ - public void invalidateMetadataQuery() { - mq = null; - } - - /** - * Constructs a new id for a correlating variable. It is unique within the - * whole query. - */ - public CorrelationId createCorrel() { - return new CorrelationId(nextCorrel.getAndIncrement()); - } - - /** Returns the default trait set for this cluster. */ - public RelTraitSet traitSet() { - return emptyTraitSet; - } - - /** @deprecated For {@code traitSetOf(t1, t2)}, - * use {@link #traitSet}().replace(t1).replace(t2). */ - @Deprecated // to be removed before 2.0 - public RelTraitSet traitSetOf(RelTrait... traits) { - RelTraitSet traitSet = emptyTraitSet; - for (RelTrait trait : traits) { - traitSet = traitSet.replace(trait); - } - return traitSet; - } - - public RelTraitSet traitSetOf(RelTrait trait) { - return emptyTraitSet.replace(trait); - } -} - -// End RelOptCluster.java diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/FlinkRelOptClusterFactory.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/FlinkRelOptClusterFactory.scala index 24fdb9e36b..c8cfd59f6f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/FlinkRelOptClusterFactory.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/FlinkRelOptClusterFactory.scala @@ -19,7 +19,7 @@ package org.apache.flink.table.calcite import org.apache.calcite.plan.{RelOptCluster, RelOptPlanner} -import org.apache.calcite.rel.metadata.{DefaultRelMetadataProvider, JaninoRelMetadataProvider, RelMetadataQuery} +import org.apache.calcite.rel.metadata.DefaultRelMetadataProvider import org.apache.calcite.rex.RexBuilder import org.apache.flink.table.plan.cost.FlinkDefaultRelMetadataProvider @@ -32,10 +32,6 @@ object FlinkRelOptClusterFactory { def create(planner: RelOptPlanner, rexBuilder: RexBuilder): RelOptCluster = { val cluster = RelOptCluster.create(planner, rexBuilder) cluster.setMetadataProvider(FlinkDefaultRelMetadataProvider.INSTANCE) - // just set metadataProvider is not enough, see - // https://www.mail-archive.com/dev@calcite.apache.org/msg00930.html - RelMetadataQuery.THREAD_PROVIDERS.set( - JaninoRelMetadataProvider.of(cluster.getMetadataProvider)) cluster } -- Gitee From ac972cd78da4ef1b972dbd2b9afc39e774a2f6fe Mon Sep 17 00:00:00 2001 From: "yuzhao.cyz" Date: Tue, 17 Mar 2020 19:27:40 +0800 Subject: [PATCH 368/885] [FLINK-14338][table-planner][table-planner-blink] Remove usage of TableScanRule and use new TableScanFactory extension * This change was introduced in CALCITE-3769 --- .../QueryOperationCatalogViewTable.java | 7 ++- .../planner/delegation/PlannerContext.java | 40 +++------------ .../plan/rules/FlinkBatchRuleSets.scala | 4 +- .../plan/rules/FlinkStreamRuleSets.scala | 4 +- .../plan/schema/CatalogSourceTable.scala | 14 +----- .../planner/plan/batch/sql/TableScanTest.xml | 50 +++++++------------ .../table/plan/rules/FlinkRuleSets.scala | 11 ++-- 7 files changed, 43 insertions(+), 87 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/QueryOperationCatalogViewTable.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/QueryOperationCatalogViewTable.java index 37bacb360f..f184be0cf0 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/QueryOperationCatalogViewTable.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/QueryOperationCatalogViewTable.java @@ -25,6 +25,7 @@ import org.apache.flink.table.planner.calcite.FlinkRelBuilder; import org.apache.flink.table.planner.plan.schema.ExpandingPreparingTable; import org.apache.flink.table.planner.plan.stats.FlinkStatistic; +import org.apache.calcite.plan.Contexts; import org.apache.calcite.plan.RelOptSchema; import org.apache.calcite.plan.RelOptTable; import org.apache.calcite.rel.RelNode; @@ -68,7 +69,11 @@ public class QueryOperationCatalogViewTable extends ExpandingPreparingTable { @Override public RelNode convertToRel(RelOptTable.ToRelContext context) { - FlinkRelBuilder relBuilder = FlinkRelBuilder.of(context.getCluster(), this.getRelOptSchema()); + FlinkRelBuilder relBuilder = new FlinkRelBuilder( + // Sets up the view expander. + Contexts.of(context, context.getCluster().getPlanner().getContext()), + context.getCluster(), + this.getRelOptSchema()); return relBuilder.queryOperation(catalogView.getQueryOperation()).build(); } diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java index 20823ea8f1..23372bf094 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java @@ -44,7 +44,6 @@ import org.apache.flink.table.planner.codegen.ExpressionReducer; import org.apache.flink.table.planner.functions.sql.FlinkSqlOperatorTable; import org.apache.flink.table.planner.plan.FlinkCalciteCatalogReader; import org.apache.flink.table.planner.plan.cost.FlinkCostFactory; -import org.apache.flink.table.planner.plan.schema.ExpandingPreparingTable; import org.apache.flink.table.planner.utils.JavaScalaConversionUtil; import org.apache.flink.table.planner.utils.TableConfigUtils; @@ -54,11 +53,8 @@ import org.apache.calcite.plan.Context; import org.apache.calcite.plan.Contexts; import org.apache.calcite.plan.RelOptCluster; import org.apache.calcite.plan.RelOptPlanner; -import org.apache.calcite.plan.RelOptTable; import org.apache.calcite.plan.RelTraitDef; -import org.apache.calcite.plan.ViewExpanders; import org.apache.calcite.plan.volcano.VolcanoPlanner; -import org.apache.calcite.rel.core.RelFactories; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeSystem; import org.apache.calcite.rex.RexBuilder; @@ -159,40 +155,19 @@ public class PlannerContext { * @return configured rel builder */ public FlinkRelBuilder createRelBuilder(String currentCatalog, String currentDatabase) { - FlinkCalciteCatalogReader relOptSchema = createCatalogReader(false, currentCatalog, currentDatabase); + FlinkCalciteCatalogReader relOptSchema = createCatalogReader( + false, + currentCatalog, + currentDatabase); - Context chain = Contexts.chain( + Context chain = Contexts.of( context, - // We need to overwrite the default scan factory, which does not - // expand views. The expandingScanFactory uses the FlinkPlanner to translate a view - // into a rel tree, before applying any subsequent rules. - Contexts.of(expandingScanFactory( - createFlinkPlanner(currentCatalog, currentDatabase).createToRelContext())) + // Sets up the ViewExpander explicitly for FlinkRelBuilder. + createFlinkPlanner(currentCatalog, currentDatabase).createToRelContext() ); return new FlinkRelBuilder(chain, cluster, relOptSchema); } - /** - * Creates a {@link RelFactories.TableScanFactory} that uses a - * {@link org.apache.calcite.plan.RelOptTable.ViewExpander} to handle - * {@link ExpandingPreparingTable} instances, and falls back to a default - * factory for other tables. - * - * @param viewExpander View expander - * @return Table scan factory - */ - private static RelFactories.TableScanFactory expandingScanFactory( - RelOptTable.ViewExpander viewExpander) { - return (cluster, table) -> { - if (table instanceof ExpandingPreparingTable) { - final RelOptTable.ToRelContext toRelContext = - ViewExpanders.toRelContext(viewExpander, cluster); - return table.toRel(toRelContext); - } - return RelFactories.DEFAULT_TABLE_SCAN_FACTORY.createScan(cluster, table); - }; - } - /** * Creates a configured {@link FlinkPlannerImpl} for a planning session. * @@ -293,7 +268,6 @@ public class PlannerContext { return JavaScalaConversionUtil.toJava(calciteConfig.getSqlToRelConverterConfig()).orElseGet( () -> SqlToRelConverter.configBuilder() .withTrimUnusedFields(false) - .withConvertTableAccess(true) .withInSubQueryThreshold(Integer.MAX_VALUE) .withExpand(false) .withRelBuilderFactory(FlinkRelFactories.FLINK_REL_BUILDER()) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkBatchRuleSets.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkBatchRuleSets.scala index 104c8f3322..77b5fc139d 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkBatchRuleSets.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkBatchRuleSets.scala @@ -55,8 +55,7 @@ object FlinkBatchRuleSets { */ val EXPAND_PLAN_RULES: RuleSet = RuleSets.ofList( LogicalCorrelateToJoinFromTemporalTableRule.WITH_FILTER, - LogicalCorrelateToJoinFromTemporalTableRule.WITHOUT_FILTER, - TableScanRule.INSTANCE) + LogicalCorrelateToJoinFromTemporalTableRule.WITHOUT_FILTER) val POST_EXPAND_CLEAN_UP_RULES: RuleSet = RuleSets.ofList( EnumerableToLogicalTableScan.INSTANCE) @@ -65,7 +64,6 @@ object FlinkBatchRuleSets { * Convert table references before query decorrelation. */ val TABLE_REF_RULES: RuleSet = RuleSets.ofList( - TableScanRule.INSTANCE, EnumerableToLogicalTableScan.INSTANCE ) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala index 03e294ede3..83d6a4122c 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala @@ -56,8 +56,7 @@ object FlinkStreamRuleSets { val EXPAND_PLAN_RULES: RuleSet = RuleSets.ofList( LogicalCorrelateToJoinFromTemporalTableRule.WITH_FILTER, LogicalCorrelateToJoinFromTemporalTableRule.WITHOUT_FILTER, - LogicalCorrelateToJoinFromTemporalTableFunctionRule.INSTANCE, - TableScanRule.INSTANCE) + LogicalCorrelateToJoinFromTemporalTableFunctionRule.INSTANCE) val POST_EXPAND_CLEAN_UP_RULES: RuleSet = RuleSets.ofList( EnumerableToLogicalTableScan.INSTANCE) @@ -66,7 +65,6 @@ object FlinkStreamRuleSets { * Convert table references before query decorrelation. */ val TABLE_REF_RULES: RuleSet = RuleSets.ofList( - TableScanRule.INSTANCE, EnumerableToLogicalTableScan.INSTANCE ) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/CatalogSourceTable.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/CatalogSourceTable.scala index 72764fe543..094de369d3 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/CatalogSourceTable.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/CatalogSourceTable.scala @@ -25,14 +25,13 @@ import org.apache.flink.table.factories.{TableFactoryUtil, TableSourceFactory, T import org.apache.flink.table.planner.calcite.{FlinkContext, FlinkRelBuilder, FlinkTypeFactory} import org.apache.flink.table.planner.catalog.CatalogSchemaTable import org.apache.flink.table.sources.{StreamTableSource, TableSource, TableSourceValidation} -import org.apache.flink.table.utils.TableConnectorUtils.generateRuntimeName + import org.apache.calcite.plan.{RelOptSchema, RelOptTable} import org.apache.calcite.rel.RelNode import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.logical.LogicalTableScan import org.apache.flink.table.types.logical.{TimestampKind, TimestampType} -import java.util import java.util.{List => JList} import scala.collection.JavaConversions._ @@ -64,15 +63,6 @@ class CatalogSourceTable[T]( .toMap } - override def getQualifiedName: JList[String] = { - // Do not explain source, we already have full names, table source should be created in toRel. - val ret = new util.ArrayList[String](names) - // Add class name to distinguish TableSourceTable. - val name = generateRuntimeName(getClass, catalogTable.getSchema.getFieldNames) - ret.add(s"catalog_source: [$name]") - ret - } - override def toRel(context: RelOptTable.ToRelContext): RelNode = { val cluster = context.getCluster val flinkContext = cluster @@ -104,7 +94,7 @@ class CatalogSourceTable[T]( .toArray // Copy this table with physical scan row type. val newRelTable = tableSourceTable.copy(tableSource, physicalFields) - val scan = LogicalTableScan.create(cluster, newRelTable) + val scan = LogicalTableScan.create(cluster, newRelTable, context.getTableHints) val relBuilder = FlinkRelBuilder.of(cluster, getRelOptSchema) relBuilder.push(scan) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableScanTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableScanTest.xml index 63bea009fe..1c75dffda0 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableScanTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableScanTest.xml @@ -19,7 +19,6 @@ limitations under the License. 1]]> - ($1, 1)]) +- LogicalTableScan(table=[[default_catalog, default_database, src, source: [CollectionTableSource(ts, a, b)]]]) ]]> - (a, 1)]) +- TableSourceScan(table=[[default_catalog, default_database, src, source: [CollectionTableSource(ts, a, b)]]], fields=[ts, a, b]) ]]> - - + - - + - - - - - - - - - - - + - - @@ -109,11 +94,10 @@ TableSourceScan(table=[[default_catalog, default_database, t1, source: [Collecti ]]> - @@ -122,11 +106,11 @@ TableSourceScan(table=[[default_catalog, default_database, t1, source: [CsvTable ]]> - @@ -136,17 +120,19 @@ Calc(select=[a, b, +(a, 1) AS c, TO_TIMESTAMP(b) AS d, my_udf(a) AS e]) ]]> - - + + + + diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala index 534abe4929..df61b48811 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala @@ -27,6 +27,8 @@ import org.apache.flink.table.plan.rules.dataSet._ import org.apache.flink.table.plan.rules.datastream._ import org.apache.flink.table.plan.rules.logical.{ExtendedAggregateExtractProjectRule, _} +import org.apache.calcite.rel.logical.{LogicalJoin, LogicalProject} + object FlinkRuleSets { /** @@ -42,8 +44,7 @@ object FlinkRuleSets { * can create new plan nodes. */ val EXPAND_PLAN_RULES: RuleSet = RuleSets.ofList( - LogicalCorrelateToTemporalTableJoinRule.INSTANCE, - TableScanRule.INSTANCE) + LogicalCorrelateToTemporalTableJoinRule.INSTANCE) val POST_EXPAND_CLEAN_UP_RULES: RuleSet = RuleSets.ofList( EnumerableToLogicalTableScan.INSTANCE) @@ -69,7 +70,11 @@ object FlinkRuleSets { FilterProjectTransposeRule.INSTANCE, // push a projection to the children of a join // push all expressions to handle the time indicator correctly - new ProjectJoinTransposeRule(PushProjector.ExprCondition.FALSE, RelFactories.LOGICAL_BUILDER), + new ProjectJoinTransposeRule( + classOf[LogicalProject], + classOf[LogicalJoin], + PushProjector.ExprCondition.FALSE, + RelFactories.LOGICAL_BUILDER), // merge projections ProjectMergeRule.INSTANCE, // remove identity project -- Gitee From 7dad5c47c92db305dc39f58acb1f8cf88b6eaed1 Mon Sep 17 00:00:00 2001 From: "yuzhao.cyz" Date: Tue, 17 Mar 2020 20:36:40 +0800 Subject: [PATCH 369/885] [FLINK-14338][table-planner][table-planner-blink] Tweak implementations due to API change * Replace RelOptUtils.createCastRel with RelOptUtil.createCastRel * Implement RelOptTable#getKeys and Statistic#getKeys * Changes logical nodes constructor for hints * Implement RelShuttle.visit(LogicalTableModify) --- .../planner/catalog/SqlCatalogViewTable.java | 4 +- .../plan/schema/FlinkPreparingTableBase.java | 5 ++ .../table/planner/plan/utils/RelOptUtils.java | 85 ------------------- .../calcite/FlinkLogicalRelFactories.scala | 17 +++- .../planner/calcite/FlinkPlannerImpl.scala | 4 + .../planner/calcite/FlinkRelOptCluster.scala | 77 ----------------- .../calcite/RelTimeIndicatorConverter.scala | 6 ++ .../plan/nodes/common/CommonCalc.scala | 6 +- .../nodes/logical/FlinkLogicalAggregate.scala | 8 +- .../FlinkLogicalDataStreamTableScan.scala | 4 +- .../plan/nodes/logical/FlinkLogicalJoin.scala | 6 +- .../logical/FlinkLogicalTableSourceScan.scala | 4 +- .../planner/plan/stats/FlinkStatistic.scala | 6 +- .../planner/plan/utils/AggregateUtil.scala | 4 - .../planner/plan/utils/RelShuttles.scala | 2 + .../table/planner/sinks/TableSinkUtils.scala | 7 +- .../plan/common/ViewsExpandingTest.scala | 1 - .../AggCallSelectivityEstimatorTest.scala | 10 +-- .../metadata/SelectivityEstimatorTest.scala | 7 +- .../program/FlinkChainedProgramTest.scala | 7 +- .../calcite/RelTimeIndicatorConverter.scala | 7 ++ .../table/plan/stats/FlinkStatistic.scala | 2 + 22 files changed, 81 insertions(+), 198 deletions(-) delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/utils/RelOptUtils.java delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelOptCluster.scala diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/SqlCatalogViewTable.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/SqlCatalogViewTable.java index a9f56e1f79..cf2fb355f8 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/SqlCatalogViewTable.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/SqlCatalogViewTable.java @@ -21,9 +21,9 @@ package org.apache.flink.table.planner.catalog; import org.apache.flink.table.catalog.CatalogView; import org.apache.flink.table.planner.plan.schema.ExpandingPreparingTable; import org.apache.flink.table.planner.plan.stats.FlinkStatistic; -import org.apache.flink.table.planner.plan.utils.RelOptUtils; import org.apache.calcite.plan.RelOptSchema; +import org.apache.calcite.plan.RelOptUtil; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.type.RelDataType; @@ -57,6 +57,6 @@ public class SqlCatalogViewTable extends ExpandingPreparingTable { RelNode original = context .expandView(rowType, view.getExpandedQuery(), viewPath, names) .project(); - return RelOptUtils.createCastRel(original, rowType); + return RelOptUtil.createCastRel(original, rowType, true); } } diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/schema/FlinkPreparingTableBase.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/schema/FlinkPreparingTableBase.java index ef4ff37b45..98d1711323 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/schema/FlinkPreparingTableBase.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/schema/FlinkPreparingTableBase.java @@ -268,6 +268,11 @@ public abstract class FlinkPreparingTableBase extends Prepare.AbstractPreparingT } } + @Override + public List getKeys() { + return statistic.getKeys(); + } + /** * Returns unique keySets of current table. */ diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/utils/RelOptUtils.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/utils/RelOptUtils.java deleted file mode 100644 index d3dfb3eccc..0000000000 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/utils/RelOptUtils.java +++ /dev/null @@ -1,85 +0,0 @@ -/* - * 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.table.planner.plan.utils; - -import org.apache.calcite.plan.RelOptUtil; -import org.apache.calcite.rel.RelNode; -import org.apache.calcite.rel.core.Project; -import org.apache.calcite.rel.core.RelFactories; -import org.apache.calcite.rel.type.RelDataType; -import org.apache.calcite.rel.type.RelDataTypeField; -import org.apache.calcite.rex.RexBuilder; -import org.apache.calcite.rex.RexNode; -import org.apache.calcite.rex.RexUtil; - -import java.util.ArrayList; -import java.util.List; - -/** - * RelOptUtils defines static utility methods for use in optimizing - * {@link RelNode}s. - * - *

    This is an extension of {@link org.apache.calcite.plan.RelOptUtil}. - */ -public class RelOptUtils { - /** - * Creates a projection which casts a rel's output to a desired row type. - * - *

    This method is inspired by {@link RelOptUtil#createCastRel}, different with that, - * we do not generate another {@link Project} if the {@code rel} is already a {@link Project}. - * - * @param rel Producer of rows to be converted - * @param castRowType Row type after cast - * @return Conversion rel with castRowType - */ - public static RelNode createCastRel(RelNode rel, RelDataType castRowType) { - RelFactories.ProjectFactory projectFactory = RelFactories.DEFAULT_PROJECT_FACTORY; - final RelDataType oriRowType = rel.getRowType(); - if (RelOptUtil.areRowTypesEqual(oriRowType, castRowType, true)) { - // nothing to do - return rel; - } - final RexBuilder rexBuilder = rel.getCluster().getRexBuilder(); - - final List fieldList = oriRowType.getFieldList(); - int n = fieldList.size(); - assert n == castRowType.getFieldCount() - : "field count: lhs [" + castRowType + "] rhs [" + oriRowType + "]"; - - final List rhsExps; - final RelNode input; - if (rel instanceof Project) { - rhsExps = ((Project) rel).getProjects(); - // Avoid to generate redundant project node. - input = rel.getInput(0); - } else { - rhsExps = new ArrayList<>(); - for (RelDataTypeField field : fieldList) { - rhsExps.add(rexBuilder.makeInputRef(field.getType(), field.getIndex())); - } - input = rel; - } - - final List castExps = - RexUtil.generateCastExpressions(rexBuilder, castRowType, rhsExps); - // Use names and types from castRowType. - return projectFactory.createProject(input, castExps, - castRowType.getFieldNames()); - } -} diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkLogicalRelFactories.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkLogicalRelFactories.scala index 99260d3396..56ddbf022d 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkLogicalRelFactories.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkLogicalRelFactories.scala @@ -25,10 +25,12 @@ import org.apache.flink.table.runtime.operators.rank.{RankRange, RankType} import org.apache.flink.table.sinks.TableSink import com.google.common.collect.ImmutableList +import org.apache.calcite.plan.RelOptTable.ToRelContext import org.apache.calcite.plan.{Contexts, RelOptCluster, RelOptTable} import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeField} import org.apache.calcite.rel.core.RelFactories._ import org.apache.calcite.rel.core._ +import org.apache.calcite.rel.hint.RelHint import org.apache.calcite.rel.logical._ import org.apache.calcite.rel.{RelCollation, RelNode} import org.apache.calcite.rex._ @@ -82,6 +84,7 @@ object FlinkLogicalRelFactories { class ProjectFactoryImpl extends ProjectFactory { def createProject( input: RelNode, + hints: util.List[RelHint], childExprs: util.List[_ <: RexNode], fieldNames: util.List[String]): RelNode = { val rexBuilder = input.getCluster.getRexBuilder @@ -134,6 +137,7 @@ object FlinkLogicalRelFactories { class AggregateFactoryImpl extends AggregateFactory { def createAggregate( input: RelNode, + hints: util.List[RelHint], groupSet: ImmutableBitSet, groupSets: ImmutableList[ImmutableBitSet], aggCalls: util.List[AggregateCall]): RelNode = { @@ -168,6 +172,7 @@ object FlinkLogicalRelFactories { def createJoin( left: RelNode, right: RelNode, + hints: util.List[RelHint], condition: RexNode, variablesSet: util.Set[CorrelationId], joinType: JoinRelType, @@ -208,14 +213,18 @@ object FlinkLogicalRelFactories { * [[FlinkLogicalTableSourceScan]] or [[FlinkLogicalDataStreamTableScan]]. */ class TableScanFactoryImpl extends TableScanFactory { - def createScan(cluster: RelOptCluster, table: RelOptTable): RelNode = { - val tableScan = LogicalTableScan.create(cluster, table) + def createScan(toRelContext: ToRelContext, table: RelOptTable): RelNode = { + val cluster = toRelContext.getCluster + val hints = toRelContext.getTableHints + val tableScan = LogicalTableScan.create(cluster, table, hints) tableScan match { case s: LogicalTableScan if FlinkLogicalTableSourceScan.isTableSourceScan(s) => - FlinkLogicalTableSourceScan.create(cluster, + FlinkLogicalTableSourceScan.create( + cluster, s.getTable.asInstanceOf[FlinkPreparingTableBase]) case s: LogicalTableScan if FlinkLogicalDataStreamTableScan.isDataStreamTableScan(s) => - FlinkLogicalDataStreamTableScan.create(cluster, + FlinkLogicalDataStreamTableScan.create( + cluster, s.getTable.asInstanceOf[FlinkPreparingTableBase]) } } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala index 6426436cef..e28e1e7866 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala @@ -22,10 +22,12 @@ import org.apache.flink.sql.parser.ExtendedSqlNode import org.apache.flink.table.api.{TableException, ValidationException} import org.apache.flink.table.planner.plan.FlinkCalciteCatalogReader +import com.google.common.collect.ImmutableList import org.apache.calcite.config.NullCollation import org.apache.calcite.plan._ import org.apache.calcite.prepare.CalciteCatalogReader import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.rel.hint.RelHint import org.apache.calcite.rel.{RelFieldCollation, RelRoot} import org.apache.calcite.sql.advise.{SqlAdvisor, SqlAdvisorValidator} import org.apache.calcite.sql.{SqlKind, SqlNode, SqlOperatorTable} @@ -190,6 +192,8 @@ class FlinkPlannerImpl( } override def getCluster: RelOptCluster = cluster + + override def getTableHints: util.List[RelHint] = ImmutableList.of() } } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelOptCluster.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelOptCluster.scala deleted file mode 100644 index d9d8b33411..0000000000 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelOptCluster.scala +++ /dev/null @@ -1,77 +0,0 @@ -/* - * 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.table.planner.calcite - -import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery - -import org.apache.calcite.plan.{RelOptCluster, RelOptPlanner, RelOptRule, RelOptRuleCall} -import org.apache.calcite.rel.RelNode -import org.apache.calcite.rel.`type`.RelDataTypeFactory -import org.apache.calcite.rel.metadata.RelMetadataQuery -import org.apache.calcite.rex.RexBuilder - -import java.util -import java.util.concurrent.atomic.AtomicInteger - -/** - * Flink specific [[RelOptCluster]] to use [[FlinkRelMetadataQuery]] - * instead of [[RelMetadataQuery]]. - */ -class FlinkRelOptCluster( - planner: RelOptPlanner, - typeFactory: RelDataTypeFactory, - rexBuilder: RexBuilder, - nextCorrel: AtomicInteger, - mapCorrelToRel: util.Map[String, RelNode]) - extends RelOptCluster(planner, typeFactory, rexBuilder, nextCorrel, mapCorrelToRel) { - - private var fmq: FlinkRelMetadataQuery = _ - - /** - * Returns the current [[FlinkRelMetadataQuery]] instead of [[RelMetadataQuery]]. - * - *

    This method might be changed or moved in future. - * If you have a [[RelOptRuleCall]] available, - * for example if you are in a [[RelOptRule#onMatch(RelOptRuleCall)]] - * method, then use [[RelOptRuleCall#getMetadataQuery()]] instead. - */ - override def getMetadataQuery: RelMetadataQuery = { - if (fmq == null) { - fmq = FlinkRelMetadataQuery.instance() - } - fmq - } - - /** - * Should be called whenever the current [[FlinkRelMetadataQuery]] becomes - * invalid. Typically invoked from [[RelOptRuleCall#transformTo]]. - */ - override def invalidateMetadataQuery(): Unit = fmq = null -} - -object FlinkRelOptCluster { - /** Creates a FlinkRelOptCluster instance. */ - def create(planner: RelOptPlanner, rexBuilder: RexBuilder): RelOptCluster = - new FlinkRelOptCluster( - planner, - rexBuilder.getTypeFactory, - rexBuilder, - new AtomicInteger(0), - new util.HashMap[String, RelNode]) -} diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.scala index fa6eb10e9c..b6bec80dbf 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.scala @@ -35,6 +35,8 @@ import org.apache.calcite.sql.`type`.SqlTypeName import org.apache.calcite.sql.fun.SqlStdOperatorTable import org.apache.calcite.sql.fun.SqlStdOperatorTable.FINAL +import java.util.{Collections => JCollections} + import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ import scala.collection.mutable @@ -464,6 +466,10 @@ class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle { updatedAggCalls) } + override def visit(modify: LogicalTableModify): RelNode = { + val input = modify.getInput.accept(this) + modify.copy(modify.getTraitSet, JCollections.singletonList(input)) + } } object RelTimeIndicatorConverter { diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonCalc.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonCalc.scala index 532e9df176..5da5d9b9cc 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonCalc.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonCalc.scala @@ -21,12 +21,16 @@ package org.apache.flink.table.planner.plan.nodes.common import org.apache.flink.table.planner.plan.nodes.ExpressionFormat.ExpressionFormat import org.apache.flink.table.planner.plan.nodes.{ExpressionFormat, FlinkRelNode} import org.apache.flink.table.planner.plan.utils.RelExplainUtil.{conditionToString, preferExpressionFormat} + import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet} import org.apache.calcite.rel.core.Calc +import org.apache.calcite.rel.hint.RelHint import org.apache.calcite.rel.metadata.RelMetadataQuery import org.apache.calcite.rel.{RelNode, RelWriter} import org.apache.calcite.rex.{RexCall, RexInputRef, RexLiteral, RexProgram} +import java.util.Collections + import scala.collection.JavaConversions._ /** @@ -37,7 +41,7 @@ abstract class CommonCalc( traitSet: RelTraitSet, input: RelNode, calcProgram: RexProgram) - extends Calc(cluster, traitSet, input, calcProgram) + extends Calc(cluster, traitSet, Collections.emptyList[RelHint](), input, calcProgram) with FlinkRelNode { override def computeSelfCost(planner: RelOptPlanner, mq: RelMetadataQuery): RelOptCost = { diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalAggregate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalAggregate.scala index 476fe4a121..2307c7282c 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalAggregate.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalAggregate.scala @@ -26,12 +26,14 @@ import org.apache.calcite.plan._ import org.apache.calcite.rel.RelNode import org.apache.calcite.rel.convert.ConverterRule import org.apache.calcite.rel.core.{Aggregate, AggregateCall} +import org.apache.calcite.rel.hint.RelHint import org.apache.calcite.rel.logical.LogicalAggregate import org.apache.calcite.rel.metadata.RelMetadataQuery import org.apache.calcite.sql.SqlKind import org.apache.calcite.util.ImmutableBitSet import java.util +import java.util.Collections import scala.collection.JavaConversions._ @@ -48,7 +50,8 @@ class FlinkLogicalAggregate( aggCalls: util.List[AggregateCall], /* flag indicating whether to skip SplitAggregateRule */ var partialFinalType: PartialFinalType = PartialFinalType.NONE) - extends Aggregate(cluster, traitSet, child, groupSet, groupSets, aggCalls) + extends Aggregate(cluster, traitSet, Collections.emptyList[RelHint](), + child, groupSet, groupSets, aggCalls) with FlinkLogicalRel { def setPartialFinalType(partialFinalType: PartialFinalType): Unit = { @@ -78,7 +81,6 @@ class FlinkLogicalAggregate( planner.getCostFactory.makeCost(rowCnt, cpuCost, rowCnt * rowSize) } } - } private class FlinkLogicalAggregateBatchConverter @@ -157,6 +159,6 @@ object FlinkLogicalAggregate { aggCalls: util.List[AggregateCall]): FlinkLogicalAggregate = { val cluster = input.getCluster val traitSet = cluster.traitSetOf(FlinkConventions.LOGICAL).simplify() - new FlinkLogicalAggregate(cluster,traitSet, input, groupSet, groupSets, aggCalls) + new FlinkLogicalAggregate(cluster, traitSet, input, groupSet, groupSets, aggCalls) } } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalDataStreamTableScan.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalDataStreamTableScan.scala index b104a66958..d630e8569f 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalDataStreamTableScan.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalDataStreamTableScan.scala @@ -25,11 +25,13 @@ import com.google.common.collect.ImmutableList import org.apache.calcite.plan._ import org.apache.calcite.rel.convert.ConverterRule import org.apache.calcite.rel.core.TableScan +import org.apache.calcite.rel.hint.RelHint import org.apache.calcite.rel.logical.LogicalTableScan import org.apache.calcite.rel.metadata.RelMetadataQuery import org.apache.calcite.rel.{RelCollation, RelCollationTraitDef, RelNode} import java.util +import java.util.Collections import java.util.function.Supplier /** @@ -40,7 +42,7 @@ class FlinkLogicalDataStreamTableScan( cluster: RelOptCluster, traitSet: RelTraitSet, table: RelOptTable) - extends TableScan(cluster, traitSet, table) + extends TableScan(cluster, traitSet, Collections.emptyList[RelHint](), table) with FlinkLogicalRel { override def copy(traitSet: RelTraitSet, inputs: util.List[RelNode]): RelNode = { diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalJoin.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalJoin.scala index cfe6f71c6c..cbc8396f5a 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalJoin.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalJoin.scala @@ -24,10 +24,13 @@ import org.apache.calcite.plan._ import org.apache.calcite.rel.RelNode import org.apache.calcite.rel.convert.ConverterRule import org.apache.calcite.rel.core.{CorrelationId, Join, JoinRelType} +import org.apache.calcite.rel.hint.RelHint import org.apache.calcite.rel.logical.LogicalJoin import org.apache.calcite.rel.metadata.RelMetadataQuery import org.apache.calcite.rex.RexNode +import java.util.Collections + import scala.collection.JavaConversions._ /** @@ -41,7 +44,8 @@ class FlinkLogicalJoin( right: RelNode, condition: RexNode, joinType: JoinRelType) - extends Join(cluster, traitSet, left, right, condition, Set.empty[CorrelationId], joinType) + extends Join(cluster, traitSet, Collections.emptyList[RelHint](), + left, right, condition, Set.empty[CorrelationId], joinType) with FlinkLogicalRel { override def copy( diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalTableSourceScan.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalTableSourceScan.scala index f28328df43..396318a771 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalTableSourceScan.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalTableSourceScan.scala @@ -28,11 +28,13 @@ import org.apache.calcite.plan._ import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.convert.ConverterRule import org.apache.calcite.rel.core.TableScan +import org.apache.calcite.rel.hint.RelHint import org.apache.calcite.rel.logical.LogicalTableScan import org.apache.calcite.rel.metadata.RelMetadataQuery import org.apache.calcite.rel.{RelCollation, RelCollationTraitDef, RelNode, RelWriter} import java.util +import java.util.Collections import java.util.function.Supplier /** @@ -43,7 +45,7 @@ class FlinkLogicalTableSourceScan( cluster: RelOptCluster, traitSet: RelTraitSet, relOptTable: TableSourceTable[_]) - extends TableScan(cluster, traitSet, relOptTable) + extends TableScan(cluster, traitSet, Collections.emptyList[RelHint](), relOptTable) with FlinkLogicalRel { lazy val tableSource: TableSource[_] = tableSourceTable.tableSource diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/stats/FlinkStatistic.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/stats/FlinkStatistic.scala index f2d6c51297..37ff25556f 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/stats/FlinkStatistic.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/stats/FlinkStatistic.scala @@ -21,11 +21,11 @@ package org.apache.flink.table.planner.plan.stats import org.apache.flink.table.plan.stats.{ColumnStats, TableStats} import org.apache.flink.table.planner.plan.`trait`.RelModifiedMonotonicity +import com.google.common.collect.ImmutableList import org.apache.calcite.rel.{RelCollation, RelDistribution, RelReferentialConstraint} import org.apache.calcite.schema.Statistic import org.apache.calcite.util.ImmutableBitSet -import java.lang.Double import java.util import scala.collection.JavaConversions._ @@ -80,7 +80,7 @@ class FlinkStatistic( * * @return The number of rows of the table. */ - override def getRowCount: Double = { + override def getRowCount: java.lang.Double = { if (tableStats != TableStats.UNKNOWN) { val rowCount = tableStats.getRowCount.toDouble // rowCount requires non-negative number @@ -136,6 +136,8 @@ class FlinkStatistic( } builder.toString() } + + override def getKeys: util.List[ImmutableBitSet] = ImmutableList.of() } /** diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala index 674fb2bf18..760306e8b7 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala @@ -128,10 +128,6 @@ object AggregateUtil extends Enumeration { require(auxGroupCalls.isEmpty, "AUXILIARY_GROUP aggCalls should be empty when groupSet is empty") } - if (agg.indicator) { - require(auxGroupCalls.isEmpty, - "AUXILIARY_GROUP aggCalls should be empty when indicator is true") - } val auxGrouping = auxGroupCalls.map(_.getArgList.head.toInt).toArray require(auxGrouping.length + otherAggCalls.length == aggCalls.length) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RelShuttles.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RelShuttles.scala index 4f74d24022..077c8c18ea 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RelShuttles.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RelShuttles.scala @@ -72,6 +72,8 @@ class DefaultRelShuttle extends RelShuttle { override def visit(join: LogicalJoin): RelNode = visit(join.asInstanceOf[RelNode]) override def visit(correlate: LogicalCorrelate): RelNode = visit(correlate.asInstanceOf[RelNode]) + + override def visit(modify: LogicalTableModify): RelNode = visit(modify.asInstanceOf[RelNode]) } /** diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sinks/TableSinkUtils.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sinks/TableSinkUtils.scala index 84397c9d5b..d4a6d3be9d 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sinks/TableSinkUtils.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sinks/TableSinkUtils.scala @@ -18,7 +18,6 @@ package org.apache.flink.table.planner.sinks -import org.apache.calcite.rel.RelNode import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2} import org.apache.flink.api.java.typeutils.{GenericTypeInfo, PojoTypeInfo, TupleTypeInfo} import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo @@ -27,7 +26,6 @@ import org.apache.flink.table.catalog.{CatalogTable, ObjectIdentifier} import org.apache.flink.table.dataformat.BaseRow import org.apache.flink.table.operations.CatalogSinkModifyOperation import org.apache.flink.table.planner.calcite.FlinkTypeFactory -import org.apache.flink.table.planner.plan.utils.RelOptUtils import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter.fromDataTypeToTypeInfo import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo import org.apache.flink.table.sinks._ @@ -41,6 +39,9 @@ import org.apache.flink.table.types.utils.TypeConversions.{fromLegacyInfoToDataT import org.apache.flink.table.utils.{TableSchemaUtils, TypeMappingUtils} import org.apache.flink.types.Row +import org.apache.calcite.plan.RelOptUtil +import org.apache.calcite.rel.RelNode + import _root_.scala.collection.JavaConversions._ object TableSinkUtils { @@ -78,7 +79,7 @@ object TableSinkUtils { val castedDataType = typeFactory.buildRelNodeRowType( sinkLogicalType.getFieldNames, sinkLogicalType.getFields.map(_.getType)) - RelOptUtils.createCastRel(query, castedDataType) + RelOptUtil.createCastRel(query, castedDataType, true) } } else { // format query and sink schema strings diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/common/ViewsExpandingTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/common/ViewsExpandingTest.scala index efed57a949..074a072f79 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/common/ViewsExpandingTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/common/ViewsExpandingTest.scala @@ -59,7 +59,6 @@ class ViewsExpandingTest(tableTestUtil: TableTestBase => TableTestUtil) extends val tableUtil = tableTestUtil(this) val tableEnv = tableUtil.tableEnv tableUtil.addDataStream[(Int, String, Int)]("t1", 'a, 'b, 'c) - val catalog = tableEnv.getCatalog(tableEnv.getCurrentCatalog).get() tableEnv.createTemporaryView("view1", tableEnv.from("t1")) tableEnv.createTemporaryView("view2", tableEnv.from("view1")) tableEnv.createTemporaryView("view3", tableEnv.from("view2")) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/AggCallSelectivityEstimatorTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/AggCallSelectivityEstimatorTest.scala index 0fa3485799..e4ddf880c3 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/AggCallSelectivityEstimatorTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/AggCallSelectivityEstimatorTest.scala @@ -19,13 +19,16 @@ package org.apache.flink.table.planner.plan.metadata import org.apache.flink.table.api.TableConfig -import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog} +import org.apache.flink.table.catalog.FunctionCatalog +import org.apache.flink.table.module.ModuleManager import org.apache.flink.table.plan.stats.{ColumnStats, TableStats} import org.apache.flink.table.planner.calcite.{FlinkContextImpl, FlinkTypeFactory, FlinkTypeSystem} import org.apache.flink.table.planner.plan.schema._ import org.apache.flink.table.planner.plan.stats.FlinkStatistic import org.apache.flink.table.planner.{JDouble, JLong} +import org.apache.flink.table.utils.CatalogManagerMocks import org.apache.flink.util.Preconditions + import com.google.common.collect.ImmutableList import org.apache.calcite.plan.{AbstractRelOptPlanner, RelOptCluster} import org.apache.calcite.rel.`type`.RelDataType @@ -45,10 +48,8 @@ import org.junit.{Before, BeforeClass, Test} import org.powermock.api.mockito.PowerMockito._ import org.powermock.core.classloader.annotations.PrepareForTest import org.powermock.modules.junit4.PowerMockRunner -import java.math.BigDecimal -import org.apache.flink.table.module.ModuleManager -import org.apache.flink.table.utils.CatalogManagerMocks +import java.math.BigDecimal import scala.collection.JavaConversions._ @@ -634,4 +635,3 @@ object AggCallSelectivityEstimatorTest { } } - diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimatorTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimatorTest.scala index 911d732174..031a668793 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimatorTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimatorTest.scala @@ -26,7 +26,9 @@ import org.apache.flink.table.planner.calcite.{FlinkContext, FlinkContextImpl, F import org.apache.flink.table.planner.plan.schema._ import org.apache.flink.table.planner.plan.stats.FlinkStatistic import org.apache.flink.table.planner.{JDouble, JLong} +import org.apache.flink.table.utils.CatalogManagerMocks import org.apache.flink.util.Preconditions + import org.apache.calcite.plan.{AbstractRelOptPlanner, RelOptCluster} import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.TableScan @@ -43,11 +45,10 @@ import org.junit.{Before, BeforeClass, Test} import org.powermock.api.mockito.PowerMockito._ import org.powermock.core.classloader.annotations.PrepareForTest import org.powermock.modules.junit4.PowerMockRunner + import java.math.BigDecimal import java.sql.{Date, Time, Timestamp} -import org.apache.flink.table.utils.CatalogManagerMocks - import scala.collection.JavaConverters._ /** @@ -1111,6 +1112,4 @@ object SelectivityEstimatorTest { .THREAD_PROVIDERS .set(JaninoRelMetadataProvider.of(FlinkDefaultRelMetadataProvider.INSTANCE)) } - } - diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChainedProgramTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChainedProgramTest.scala index 8b6039b23b..10cec795f8 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChainedProgramTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChainedProgramTest.scala @@ -26,6 +26,8 @@ import org.apache.calcite.tools.RuleSets import org.junit.Assert._ import org.junit.Test +import java.util.Collections + import scala.collection.JavaConversions._ /** @@ -49,10 +51,7 @@ class FlinkChainedProgramTest { .addRuleInstance(SubQueryRemoveRule.JOIN) .addMatchLimit(100) .addMatchOrder(HepMatchOrder.BOTTOM_UP) - .addRuleCollection(Array( - TableScanRule.INSTANCE, - ValuesReduceRule.FILTER_INSTANCE - ).toList) + .addRuleCollection(Collections.singletonList(ValuesReduceRule.FILTER_INSTANCE)) val program1 = FlinkHepProgram(builder.build()) assertTrue(programs.addFirst("o2", program1)) assertEquals(List("o2"), programs.getProgramNames.toList) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala index d3aaf7d8e6..7f8524591e 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala @@ -34,6 +34,8 @@ import org.apache.flink.table.functions.sql.ProctimeSqlFunction import org.apache.flink.table.plan.logical.rel._ import org.apache.flink.table.plan.schema.TimeIndicatorRelDataType +import java.util.{Collections => JCollections} + import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ import scala.collection.mutable @@ -377,6 +379,11 @@ class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle { rexBuilder, inputs.flatMap(_.getRowType.getFieldList.map(_.getType))) } + + override def visit(modify: LogicalTableModify): RelNode = { + val input = modify.getInput.accept(this) + modify.copy(modify.getTraitSet, JCollections.singletonList(input)) + } } object RelTimeIndicatorConverter { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/stats/FlinkStatistic.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/stats/FlinkStatistic.scala index 754509e66d..957bdf62bc 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/stats/FlinkStatistic.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/stats/FlinkStatistic.scala @@ -70,6 +70,8 @@ class FlinkStatistic(tableStats: Option[TableStats]) extends Statistic { override def getReferentialConstraints: util.List[RelReferentialConstraint] = Collections.emptyList() + + override def getKeys: util.List[ImmutableBitSet] = Collections.emptyList() } /** -- Gitee From 10c8392b9041d636f194b80492afc492eae806b7 Mon Sep 17 00:00:00 2001 From: "yuzhao.cyz" Date: Tue, 17 Mar 2020 21:02:21 +0800 Subject: [PATCH 370/885] [FLINK-14338][table-planner][table-planner-blink] Remove redundant code copy because the bugs are already fixed in Calcite * SqlValidatorImpl.java was removed because of CALCITE-2707 * ParameterScope.java was removed because of CALCITE-3476 * SqlFunction.java was removed because of CALCITE-3360 --- .../org/apache/calcite/sql/SqlFunction.java | 322 - .../calcite/sql/validate/ParameterScope.java | 72 - .../sql/validate/SqlValidatorImpl.java | 6451 ---------------- .../org/apache/calcite/sql/SqlFunction.java | 322 - .../calcite/sql/validate/ParameterScope.java | 72 - .../sql/validate/SqlValidatorImpl.java | 6452 ----------------- 6 files changed, 13691 deletions(-) delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/sql/SqlFunction.java delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/sql/validate/ParameterScope.java delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java delete mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/SqlFunction.java delete mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/ParameterScope.java delete mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/sql/SqlFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/sql/SqlFunction.java deleted file mode 100644 index fda4cd57d0..0000000000 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/sql/SqlFunction.java +++ /dev/null @@ -1,322 +0,0 @@ -/* - * 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.calcite.sql; - -import com.google.common.collect.ImmutableList; -import org.apache.calcite.linq4j.function.Functions; -import org.apache.calcite.rel.type.RelDataType; -import org.apache.calcite.sql.type.SqlOperandTypeChecker; -import org.apache.calcite.sql.type.SqlOperandTypeInference; -import org.apache.calcite.sql.type.SqlReturnTypeInference; -import org.apache.calcite.sql.validate.SqlValidator; -import org.apache.calcite.sql.validate.SqlValidatorScope; -import org.apache.calcite.sql.validate.implicit.TypeCoercion; -import org.apache.calcite.util.Util; - -import javax.annotation.Nonnull; - -import java.util.List; -import java.util.Objects; - -import static org.apache.calcite.util.Static.RESOURCE; - -/** - * A SqlFunction is a type of operator which has conventional - * function-call syntax. - * - *

    This file is copied from Apache Calcite and should be removed once CALCITE-3360 - * has been fixed. Changed lines: 279-293 - */ -public class SqlFunction extends SqlOperator { - - //~ Instance fields -------------------------------------------------------- - - private final SqlFunctionCategory category; - - private final SqlIdentifier sqlIdentifier; - - private final List paramTypes; - - //~ Constructors ----------------------------------------------------------- - - /** - * Creates a new SqlFunction for a call to a builtin function. - * - * @param name Name of builtin function - * @param kind kind of operator implemented by function - * @param returnTypeInference strategy to use for return type inference - * @param operandTypeInference strategy to use for parameter type inference - * @param operandTypeChecker strategy to use for parameter type checking - * @param category categorization for function - */ - public SqlFunction( - String name, - SqlKind kind, - SqlReturnTypeInference returnTypeInference, - SqlOperandTypeInference operandTypeInference, - SqlOperandTypeChecker operandTypeChecker, - SqlFunctionCategory category) { - // We leave sqlIdentifier as null to indicate - // that this is a builtin. Same for paramTypes. - this(name, null, kind, returnTypeInference, operandTypeInference, - operandTypeChecker, null, category); - - assert !((category == SqlFunctionCategory.USER_DEFINED_CONSTRUCTOR) - && (returnTypeInference == null)); - } - - /** - * Creates a placeholder SqlFunction for an invocation of a function with a - * possibly qualified name. This name must be resolved into either a builtin - * function or a user-defined function. - * - * @param sqlIdentifier possibly qualified identifier for function - * @param returnTypeInference strategy to use for return type inference - * @param operandTypeInference strategy to use for parameter type inference - * @param operandTypeChecker strategy to use for parameter type checking - * @param paramTypes array of parameter types - * @param funcType function category - */ - public SqlFunction( - SqlIdentifier sqlIdentifier, - SqlReturnTypeInference returnTypeInference, - SqlOperandTypeInference operandTypeInference, - SqlOperandTypeChecker operandTypeChecker, - List paramTypes, - SqlFunctionCategory funcType) { - this(Util.last(sqlIdentifier.names), sqlIdentifier, SqlKind.OTHER_FUNCTION, - returnTypeInference, operandTypeInference, operandTypeChecker, - paramTypes, funcType); - } - - /** - * Internal constructor. - */ - protected SqlFunction( - String name, - SqlIdentifier sqlIdentifier, - SqlKind kind, - SqlReturnTypeInference returnTypeInference, - SqlOperandTypeInference operandTypeInference, - SqlOperandTypeChecker operandTypeChecker, - List paramTypes, - SqlFunctionCategory category) { - super(name, kind, 100, 100, returnTypeInference, operandTypeInference, - operandTypeChecker); - - this.sqlIdentifier = sqlIdentifier; - this.category = Objects.requireNonNull(category); - this.paramTypes = - paramTypes == null ? null : ImmutableList.copyOf(paramTypes); - } - - //~ Methods ---------------------------------------------------------------- - - public SqlSyntax getSyntax() { - return SqlSyntax.FUNCTION; - } - - /** - * @return fully qualified name of function, or null for a builtin function - */ - public SqlIdentifier getSqlIdentifier() { - return sqlIdentifier; - } - - @Override public SqlIdentifier getNameAsId() { - if (sqlIdentifier != null) { - return sqlIdentifier; - } - return super.getNameAsId(); - } - - /** - * @return array of parameter types, or null for builtin function - */ - public List getParamTypes() { - return paramTypes; - } - - /** - * Returns a list of parameter names. - * - *

    The default implementation returns {@code [arg0, arg1, ..., argN]}. - */ - public List getParamNames() { - return Functions.generate(paramTypes.size(), i -> "arg" + i); - } - - public void unparse( - SqlWriter writer, - SqlCall call, - int leftPrec, - int rightPrec) { - getSyntax().unparse(writer, this, call, leftPrec, rightPrec); - } - - /** - * @return function category - */ - @Nonnull public SqlFunctionCategory getFunctionType() { - return this.category; - } - - /** - * Returns whether this function allows a DISTINCT or - * ALL quantifier. The default is false; some aggregate - * functions return true. - */ - public boolean isQuantifierAllowed() { - return false; - } - - public void validateCall( - SqlCall call, - SqlValidator validator, - SqlValidatorScope scope, - SqlValidatorScope operandScope) { - // This implementation looks for the quantifier keywords DISTINCT or - // ALL as the first operand in the list. If found then the literal is - // not called to validate itself. Further the function is checked to - // make sure that a quantifier is valid for that particular function. - // - // If the first operand does not appear to be a quantifier then the - // parent ValidateCall is invoked to do normal function validation. - - super.validateCall(call, validator, scope, operandScope); - validateQuantifier(validator, call); - } - - /** - * Throws a validation error if a DISTINCT or ALL quantifier is present but - * not allowed. - */ - protected void validateQuantifier(SqlValidator validator, SqlCall call) { - if ((null != call.getFunctionQuantifier()) && !isQuantifierAllowed()) { - throw validator.newValidationError(call.getFunctionQuantifier(), - RESOURCE.functionQuantifierNotAllowed(call.getOperator().getName())); - } - } - - public RelDataType deriveType( - SqlValidator validator, - SqlValidatorScope scope, - SqlCall call) { - return deriveType(validator, scope, call, true); - } - - private RelDataType deriveType( - SqlValidator validator, - SqlValidatorScope scope, - SqlCall call, - boolean convertRowArgToColumnList) { - // Scope for operands. Usually the same as 'scope'. - final SqlValidatorScope operandScope = scope.getOperandScope(call); - - // Indicate to the validator that we're validating a new function call - validator.pushFunctionCall(); - - final List argNames = constructArgNameList(call); - - final List args = constructOperandList(validator, call, argNames); - - final List argTypes = constructArgTypeList(validator, scope, - call, args, convertRowArgToColumnList); - - SqlFunction function = - (SqlFunction) SqlUtil.lookupRoutine(validator.getOperatorTable(), - getNameAsId(), argTypes, argNames, getFunctionType(), - SqlSyntax.FUNCTION, getKind(), - validator.getCatalogReader().nameMatcher(), - false); - try { - // if we have a match on function name and parameter count, but - // couldn't find a function with a COLUMN_LIST type, retry, but - // this time, don't convert the row argument to a COLUMN_LIST type; - // if we did find a match, go back and re-validate the row operands - // (corresponding to column references), now that we can set the - // scope to that of the source cursor referenced by that ColumnList - // type - if (convertRowArgToColumnList && containsRowArg(args)) { - if (function == null - && SqlUtil.matchRoutinesByParameterCount( - validator.getOperatorTable(), getNameAsId(), argTypes, - getFunctionType(), - validator.getCatalogReader().nameMatcher())) { - // remove the already validated node types corresponding to - // row arguments before re-validating - for (SqlNode operand : args) { - if (operand.getKind() == SqlKind.ROW) { - validator.removeValidatedNodeType(operand); - } - } - return deriveType(validator, scope, call, false); - } else if (function != null) { - validator.validateColumnListParams(function, argTypes, args); - } - } - - if (getFunctionType() == SqlFunctionCategory.USER_DEFINED_CONSTRUCTOR) { - return validator.deriveConstructorType(scope, call, this, function, - argTypes); - } - if (function == null) { - // try again if implicit type coercion is allowed. - boolean changed = false; - if (validator.isTypeCoercionEnabled()) { - function = (SqlFunction) SqlUtil.lookupRoutine(validator.getOperatorTable(), - getNameAsId(), argTypes, argNames, getFunctionType(), SqlSyntax.FUNCTION, getKind(), - validator.getCatalogReader().nameMatcher(), - true); - // try to coerce the function arguments to the declared sql type name. - // if we succeed, the arguments would be wrapped with CAST operator. - if (function != null) { - TypeCoercion typeCoercion = validator.getTypeCoercion(); - changed = typeCoercion.userDefinedFunctionCoercion(scope, call, function); - } - } - if (!changed) { - throw validator.handleUnresolvedFunction(call, this, argTypes, - argNames); - } - } - - // REVIEW jvs 25-Mar-2005: This is, in a sense, expanding - // identifiers, but we ignore shouldExpandIdentifiers() - // because otherwise later validation code will - // choke on the unresolved function. - ((SqlBasicCall) call).setOperator(function); - return function.validateOperands( - validator, - operandScope, - call); - } finally { - validator.popFunctionCall(); - } - } - - private boolean containsRowArg(List args) { - for (SqlNode operand : args) { - if (operand.getKind() == SqlKind.ROW) { - return true; - } - } - return false; - } -} - -// End SqlFunction.java diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/sql/validate/ParameterScope.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/sql/validate/ParameterScope.java deleted file mode 100644 index 414c928910..0000000000 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/sql/validate/ParameterScope.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * 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.calcite.sql.validate; - -import org.apache.calcite.rel.type.RelDataType; -import org.apache.calcite.sql.SqlCall; -import org.apache.calcite.sql.SqlIdentifier; -import org.apache.calcite.sql.SqlNode; - -import java.util.Map; - -// This class is copied from Calcite's org.apache.calcite.sql.validate.ParameterScope, -// can be removed after https://issues.apache.org/jira/browse/CALCITE-3476 is fixed. -// -// Modification: -// - L66~L69: override resolveColumn method - -/** - * A scope which contains nothing besides a few parameters. Like - * {@link EmptyScope} (which is its base class), it has no parent scope. - * - * @see ParameterNamespace - */ -public class ParameterScope extends EmptyScope { - //~ Instance fields -------------------------------------------------------- - - /** - * Map from the simple names of the parameters to types of the parameters - * ({@link RelDataType}). - */ - private final Map nameToTypeMap; - - //~ Constructors ----------------------------------------------------------- - - public ParameterScope( - SqlValidatorImpl validator, - Map nameToTypeMap) { - super(validator); - this.nameToTypeMap = nameToTypeMap; - } - - //~ Methods ---------------------------------------------------------------- - - public SqlQualified fullyQualify(SqlIdentifier identifier) { - return SqlQualified.create(this, 1, null, identifier); - } - - public SqlValidatorScope getOperandScope(SqlCall call) { - return this; - } - - @Override - public RelDataType resolveColumn(String name, SqlNode ctx) { - return nameToTypeMap.get(name); - } -} - -// End ParameterScope.java diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java deleted file mode 100644 index 05615f7e93..0000000000 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java +++ /dev/null @@ -1,6451 +0,0 @@ -/* - * 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.calcite.sql.validate; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; -import org.apache.calcite.config.NullCollation; -import org.apache.calcite.linq4j.Ord; -import org.apache.calcite.linq4j.function.Function2; -import org.apache.calcite.linq4j.function.Functions; -import org.apache.calcite.plan.RelOptTable; -import org.apache.calcite.plan.RelOptUtil; -import org.apache.calcite.prepare.Prepare; -import org.apache.calcite.rel.type.DynamicRecordType; -import org.apache.calcite.rel.type.RelDataType; -import org.apache.calcite.rel.type.RelDataTypeFactory; -import org.apache.calcite.rel.type.RelDataTypeField; -import org.apache.calcite.rel.type.RelDataTypeSystem; -import org.apache.calcite.rel.type.RelRecordType; -import org.apache.calcite.rex.RexBuilder; -import org.apache.calcite.rex.RexNode; -import org.apache.calcite.rex.RexPatternFieldRef; -import org.apache.calcite.rex.RexVisitor; -import org.apache.calcite.runtime.CalciteContextException; -import org.apache.calcite.runtime.CalciteException; -import org.apache.calcite.runtime.Feature; -import org.apache.calcite.runtime.Resources; -import org.apache.calcite.schema.ColumnStrategy; -import org.apache.calcite.schema.Table; -import org.apache.calcite.schema.impl.ModifiableViewTable; -import org.apache.calcite.sql.JoinConditionType; -import org.apache.calcite.sql.JoinType; -import org.apache.calcite.sql.SqlAccessEnum; -import org.apache.calcite.sql.SqlAccessType; -import org.apache.calcite.sql.SqlAggFunction; -import org.apache.calcite.sql.SqlBasicCall; -import org.apache.calcite.sql.SqlCall; -import org.apache.calcite.sql.SqlCallBinding; -import org.apache.calcite.sql.SqlDataTypeSpec; -import org.apache.calcite.sql.SqlDelete; -import org.apache.calcite.sql.SqlDynamicParam; -import org.apache.calcite.sql.SqlExplain; -import org.apache.calcite.sql.SqlFunction; -import org.apache.calcite.sql.SqlFunctionCategory; -import org.apache.calcite.sql.SqlIdentifier; -import org.apache.calcite.sql.SqlInsert; -import org.apache.calcite.sql.SqlIntervalLiteral; -import org.apache.calcite.sql.SqlIntervalQualifier; -import org.apache.calcite.sql.SqlJoin; -import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.sql.SqlLiteral; -import org.apache.calcite.sql.SqlMatchRecognize; -import org.apache.calcite.sql.SqlMerge; -import org.apache.calcite.sql.SqlNode; -import org.apache.calcite.sql.SqlNodeList; -import org.apache.calcite.sql.SqlOperator; -import org.apache.calcite.sql.SqlOperatorTable; -import org.apache.calcite.sql.SqlOrderBy; -import org.apache.calcite.sql.SqlSampleSpec; -import org.apache.calcite.sql.SqlSelect; -import org.apache.calcite.sql.SqlSelectKeyword; -import org.apache.calcite.sql.SqlSnapshot; -import org.apache.calcite.sql.SqlSyntax; -import org.apache.calcite.sql.SqlUnresolvedFunction; -import org.apache.calcite.sql.SqlUpdate; -import org.apache.calcite.sql.SqlUtil; -import org.apache.calcite.sql.SqlWindow; -import org.apache.calcite.sql.SqlWith; -import org.apache.calcite.sql.SqlWithItem; -import org.apache.calcite.sql.fun.SqlCase; -import org.apache.calcite.sql.fun.SqlStdOperatorTable; -import org.apache.calcite.sql.parser.SqlParserPos; -import org.apache.calcite.sql.type.AssignableOperandTypeChecker; -import org.apache.calcite.sql.type.ReturnTypes; -import org.apache.calcite.sql.type.SqlOperandTypeInference; -import org.apache.calcite.sql.type.SqlTypeName; -import org.apache.calcite.sql.type.SqlTypeUtil; -import org.apache.calcite.sql.util.SqlBasicVisitor; -import org.apache.calcite.sql.util.SqlShuttle; -import org.apache.calcite.sql.util.SqlVisitor; -import org.apache.calcite.sql.validate.implicit.TypeCoercion; -import org.apache.calcite.sql.validate.implicit.TypeCoercions; -import org.apache.calcite.sql2rel.InitializerContext; -import org.apache.calcite.util.BitString; -import org.apache.calcite.util.Bug; -import org.apache.calcite.util.ImmutableBitSet; -import org.apache.calcite.util.ImmutableIntList; -import org.apache.calcite.util.ImmutableNullableList; -import org.apache.calcite.util.Litmus; -import org.apache.calcite.util.Pair; -import org.apache.calcite.util.Static; -import org.apache.calcite.util.Util; -import org.apache.calcite.util.trace.CalciteTrace; -import org.slf4j.Logger; - -import javax.annotation.Nonnull; -import javax.annotation.Nullable; - -import java.math.BigDecimal; -import java.math.BigInteger; -import java.util.AbstractList; -import java.util.ArrayDeque; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Calendar; -import java.util.Collection; -import java.util.Collections; -import java.util.Deque; -import java.util.GregorianCalendar; -import java.util.HashMap; -import java.util.HashSet; -import java.util.IdentityHashMap; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.Objects; -import java.util.Set; -import java.util.function.Supplier; - -import static org.apache.calcite.sql.SqlUtil.stripAs; -import static org.apache.calcite.util.Static.RESOURCE; - -/* - * THIS FILE HAS BEEN COPIED FROM THE APACHE CALCITE PROJECT UNTIL CALCITE-2707 IS FIXED. - * (Added lines: 6016-6022) - */ - -/** - * Default implementation of {@link SqlValidator}. - */ -public class SqlValidatorImpl implements SqlValidatorWithHints { - //~ Static fields/initializers --------------------------------------------- - - public static final Logger TRACER = CalciteTrace.PARSER_LOGGER; - - /** - * Alias generated for the source table when rewriting UPDATE to MERGE. - */ - public static final String UPDATE_SRC_ALIAS = "SYS$SRC"; - - /** - * Alias generated for the target table when rewriting UPDATE to MERGE if no - * alias was specified by the user. - */ - public static final String UPDATE_TGT_ALIAS = "SYS$TGT"; - - /** - * Alias prefix generated for source columns when rewriting UPDATE to MERGE. - */ - public static final String UPDATE_ANON_PREFIX = "SYS$ANON"; - - //~ Instance fields -------------------------------------------------------- - - private final SqlOperatorTable opTab; - final SqlValidatorCatalogReader catalogReader; - - /** - * Maps ParsePosition strings to the {@link SqlIdentifier} identifier - * objects at these positions - */ - protected final Map idPositions = new HashMap<>(); - - /** - * Maps {@link SqlNode query node} objects to the {@link SqlValidatorScope} - * scope created from them. - */ - protected final Map scopes = - new IdentityHashMap<>(); - - /** - * Maps a {@link SqlSelect} node to the scope used by its WHERE and HAVING - * clauses. - */ - private final Map whereScopes = - new IdentityHashMap<>(); - - /** - * Maps a {@link SqlSelect} node to the scope used by its GROUP BY clause. - */ - private final Map groupByScopes = - new IdentityHashMap<>(); - - /** - * Maps a {@link SqlSelect} node to the scope used by its SELECT and HAVING - * clauses. - */ - private final Map selectScopes = - new IdentityHashMap<>(); - - /** - * Maps a {@link SqlSelect} node to the scope used by its ORDER BY clause. - */ - private final Map orderScopes = - new IdentityHashMap<>(); - - /** - * Maps a {@link SqlSelect} node that is the argument to a CURSOR - * constructor to the scope of the result of that select node - */ - private final Map cursorScopes = - new IdentityHashMap<>(); - - /** - * The name-resolution scope of a LATERAL TABLE clause. - */ - private TableScope tableScope = null; - - /** - * Maps a {@link SqlNode node} to the - * {@link SqlValidatorNamespace namespace} which describes what columns they - * contain. - */ - protected final Map namespaces = - new IdentityHashMap<>(); - - /** - * Set of select expressions used as cursor definitions. In standard SQL, - * only the top-level SELECT is a cursor; Calcite extends this with - * cursors as inputs to table functions. - */ - private final Set cursorSet = Sets.newIdentityHashSet(); - - /** - * Stack of objects that maintain information about function calls. A stack - * is needed to handle nested function calls. The function call currently - * being validated is at the top of the stack. - */ - protected final Deque functionCallStack = - new ArrayDeque<>(); - - private int nextGeneratedId; - protected final RelDataTypeFactory typeFactory; - - /** The type of dynamic parameters until a type is imposed on them. */ - protected final RelDataType unknownType; - private final RelDataType booleanType; - - /** - * Map of derived RelDataType for each node. This is an IdentityHashMap - * since in some cases (such as null literals) we need to discriminate by - * instance. - */ - private final Map nodeToTypeMap = - new IdentityHashMap<>(); - private final AggFinder aggFinder; - private final AggFinder aggOrOverFinder; - private final AggFinder aggOrOverOrGroupFinder; - private final AggFinder groupFinder; - private final AggFinder overFinder; - private final SqlConformance conformance; - private final Map originalExprs = new HashMap<>(); - - private SqlNode top; - - // REVIEW jvs 30-June-2006: subclasses may override shouldExpandIdentifiers - // in a way that ignores this; we should probably get rid of the protected - // method and always use this variable (or better, move preferences like - // this to a separate "parameter" class) - protected boolean expandIdentifiers; - - protected boolean expandColumnReferences; - - private boolean rewriteCalls; - - private NullCollation nullCollation = NullCollation.HIGH; - - // TODO jvs 11-Dec-2008: make this local to performUnconditionalRewrites - // if it's OK to expand the signature of that method. - private boolean validatingSqlMerge; - - private boolean inWindow; // Allow nested aggregates - - private final ValidationErrorFunction validationErrorFunction = - new ValidationErrorFunction(); - - // TypeCoercion instance used for implicit type coercion. - private TypeCoercion typeCoercion; - - // Flag saying if we enable the implicit type coercion. - private boolean enableTypeCoercion; - - //~ Constructors ----------------------------------------------------------- - - /** - * Creates a validator. - * - * @param opTab Operator table - * @param catalogReader Catalog reader - * @param typeFactory Type factory - * @param conformance Compatibility mode - */ - protected SqlValidatorImpl( - SqlOperatorTable opTab, - SqlValidatorCatalogReader catalogReader, - RelDataTypeFactory typeFactory, - SqlConformance conformance) { - this.opTab = Objects.requireNonNull(opTab); - this.catalogReader = Objects.requireNonNull(catalogReader); - this.typeFactory = Objects.requireNonNull(typeFactory); - this.conformance = Objects.requireNonNull(conformance); - - unknownType = typeFactory.createUnknownType(); - booleanType = typeFactory.createSqlType(SqlTypeName.BOOLEAN); - - rewriteCalls = true; - expandColumnReferences = true; - final SqlNameMatcher nameMatcher = catalogReader.nameMatcher(); - aggFinder = new AggFinder(opTab, false, true, false, null, nameMatcher); - aggOrOverFinder = new AggFinder(opTab, true, true, false, null, nameMatcher); - overFinder = new AggFinder(opTab, true, false, false, aggOrOverFinder, nameMatcher); - groupFinder = new AggFinder(opTab, false, false, true, null, nameMatcher); - aggOrOverOrGroupFinder = new AggFinder(opTab, true, true, true, null, nameMatcher); - this.enableTypeCoercion = catalogReader.getConfig() == null - || catalogReader.getConfig().typeCoercion(); - this.typeCoercion = TypeCoercions.getTypeCoercion(this, conformance); - } - - //~ Methods ---------------------------------------------------------------- - - public SqlConformance getConformance() { - return conformance; - } - - public SqlValidatorCatalogReader getCatalogReader() { - return catalogReader; - } - - public SqlOperatorTable getOperatorTable() { - return opTab; - } - - public RelDataTypeFactory getTypeFactory() { - return typeFactory; - } - - public RelDataType getUnknownType() { - return unknownType; - } - - public SqlNodeList expandStar( - SqlNodeList selectList, - SqlSelect select, - boolean includeSystemVars) { - final List list = new ArrayList<>(); - final List> types = new ArrayList<>(); - for (int i = 0; i < selectList.size(); i++) { - final SqlNode selectItem = selectList.get(i); - final RelDataType originalType = getValidatedNodeTypeIfKnown(selectItem); - expandSelectItem( - selectItem, - select, - Util.first(originalType, unknownType), - list, - catalogReader.nameMatcher().createSet(), - types, - includeSystemVars); - } - getRawSelectScope(select).setExpandedSelectList(list); - return new SqlNodeList(list, SqlParserPos.ZERO); - } - - // implement SqlValidator - public void declareCursor(SqlSelect select, SqlValidatorScope parentScope) { - cursorSet.add(select); - - // add the cursor to a map that maps the cursor to its select based on - // the position of the cursor relative to other cursors in that call - FunctionParamInfo funcParamInfo = functionCallStack.peek(); - Map cursorMap = funcParamInfo.cursorPosToSelectMap; - int numCursors = cursorMap.size(); - cursorMap.put(numCursors, select); - - // create a namespace associated with the result of the select - // that is the argument to the cursor constructor; register it - // with a scope corresponding to the cursor - SelectScope cursorScope = new SelectScope(parentScope, null, select); - cursorScopes.put(select, cursorScope); - final SelectNamespace selectNs = createSelectNamespace(select, select); - String alias = deriveAlias(select, nextGeneratedId++); - registerNamespace(cursorScope, alias, selectNs, false); - } - - // implement SqlValidator - public void pushFunctionCall() { - FunctionParamInfo funcInfo = new FunctionParamInfo(); - functionCallStack.push(funcInfo); - } - - // implement SqlValidator - public void popFunctionCall() { - functionCallStack.pop(); - } - - // implement SqlValidator - public String getParentCursor(String columnListParamName) { - FunctionParamInfo funcParamInfo = functionCallStack.peek(); - Map parentCursorMap = - funcParamInfo.columnListParamToParentCursorMap; - return parentCursorMap.get(columnListParamName); - } - - /** - * If selectItem is "*" or "TABLE.*", expands it and returns - * true; otherwise writes the unexpanded item. - * - * @param selectItem Select-list item - * @param select Containing select clause - * @param selectItems List that expanded items are written to - * @param aliases Set of aliases - * @param fields List of field names and types, in alias order - * @param includeSystemVars If true include system vars in lists - * @return Whether the node was expanded - */ - private boolean expandSelectItem( - final SqlNode selectItem, - SqlSelect select, - RelDataType targetType, - List selectItems, - Set aliases, - List> fields, - final boolean includeSystemVars) { - final SelectScope scope = (SelectScope) getWhereScope(select); - if (expandStar(selectItems, aliases, fields, includeSystemVars, scope, - selectItem)) { - return true; - } - - // Expand the select item: fully-qualify columns, and convert - // parentheses-free functions such as LOCALTIME into explicit function - // calls. - SqlNode expanded = expand(selectItem, scope); - final String alias = - deriveAlias( - selectItem, - aliases.size()); - - // If expansion has altered the natural alias, supply an explicit 'AS'. - final SqlValidatorScope selectScope = getSelectScope(select); - if (expanded != selectItem) { - String newAlias = - deriveAlias( - expanded, - aliases.size()); - if (!newAlias.equals(alias)) { - expanded = - SqlStdOperatorTable.AS.createCall( - selectItem.getParserPosition(), - expanded, - new SqlIdentifier(alias, SqlParserPos.ZERO)); - deriveTypeImpl(selectScope, expanded); - } - } - - selectItems.add(expanded); - aliases.add(alias); - - if (expanded != null) { - inferUnknownTypes(targetType, scope, expanded); - } - final RelDataType type = deriveType(selectScope, expanded); - setValidatedNodeType(expanded, type); - fields.add(Pair.of(alias, type)); - return false; - } - - private boolean expandStar(List selectItems, Set aliases, - List> fields, boolean includeSystemVars, - SelectScope scope, SqlNode node) { - if (!(node instanceof SqlIdentifier)) { - return false; - } - final SqlIdentifier identifier = (SqlIdentifier) node; - if (!identifier.isStar()) { - return false; - } - final SqlParserPos startPosition = identifier.getParserPosition(); - switch (identifier.names.size()) { - case 1: - boolean hasDynamicStruct = false; - for (ScopeChild child : scope.children) { - final int before = fields.size(); - if (child.namespace.getRowType().isDynamicStruct()) { - hasDynamicStruct = true; - // don't expand star if the underneath table is dynamic. - // Treat this star as a special field in validation/conversion and - // wait until execution time to expand this star. - final SqlNode exp = - new SqlIdentifier( - ImmutableList.of(child.name, - DynamicRecordType.DYNAMIC_STAR_PREFIX), - startPosition); - addToSelectList( - selectItems, - aliases, - fields, - exp, - scope, - includeSystemVars); - } else { - final SqlNode from = child.namespace.getNode(); - final SqlValidatorNamespace fromNs = getNamespace(from, scope); - assert fromNs != null; - final RelDataType rowType = fromNs.getRowType(); - for (RelDataTypeField field : rowType.getFieldList()) { - String columnName = field.getName(); - - // TODO: do real implicit collation here - final SqlIdentifier exp = - new SqlIdentifier( - ImmutableList.of(child.name, columnName), - startPosition); - // Don't add expanded rolled up columns - if (!isRolledUpColumn(exp, scope)) { - addOrExpandField( - selectItems, - aliases, - fields, - includeSystemVars, - scope, - exp, - field); - } - } - } - if (child.nullable) { - for (int i = before; i < fields.size(); i++) { - final Map.Entry entry = fields.get(i); - final RelDataType type = entry.getValue(); - if (!type.isNullable()) { - fields.set(i, - Pair.of(entry.getKey(), - typeFactory.createTypeWithNullability(type, true))); - } - } - } - } - // If NATURAL JOIN or USING is present, move key fields to the front of - // the list, per standard SQL. Disabled if there are dynamic fields. - if (!hasDynamicStruct || Bug.CALCITE_2400_FIXED) { - new Permute(scope.getNode().getFrom(), 0).permute(selectItems, fields); - } - return true; - - default: - final SqlIdentifier prefixId = identifier.skipLast(1); - final SqlValidatorScope.ResolvedImpl resolved = - new SqlValidatorScope.ResolvedImpl(); - final SqlNameMatcher nameMatcher = - scope.validator.catalogReader.nameMatcher(); - scope.resolve(prefixId.names, nameMatcher, true, resolved); - if (resolved.count() == 0) { - // e.g. "select s.t.* from e" - // or "select r.* from e" - throw newValidationError(prefixId, - RESOURCE.unknownIdentifier(prefixId.toString())); - } - final RelDataType rowType = resolved.only().rowType(); - if (rowType.isDynamicStruct()) { - // don't expand star if the underneath table is dynamic. - addToSelectList( - selectItems, - aliases, - fields, - prefixId.plus(DynamicRecordType.DYNAMIC_STAR_PREFIX, startPosition), - scope, - includeSystemVars); - } else if (rowType.isStruct()) { - for (RelDataTypeField field : rowType.getFieldList()) { - String columnName = field.getName(); - - // TODO: do real implicit collation here - addOrExpandField( - selectItems, - aliases, - fields, - includeSystemVars, - scope, - prefixId.plus(columnName, startPosition), - field); - } - } else { - throw newValidationError(prefixId, RESOURCE.starRequiresRecordType()); - } - return true; - } - } - - private SqlNode maybeCast(SqlNode node, RelDataType currentType, - RelDataType desiredType) { - return currentType.equals(desiredType) - || (currentType.isNullable() != desiredType.isNullable() - && typeFactory.createTypeWithNullability(currentType, - desiredType.isNullable()).equals(desiredType)) - ? node - : SqlStdOperatorTable.CAST.createCall(SqlParserPos.ZERO, - node, SqlTypeUtil.convertTypeToSpec(desiredType)); - } - - private boolean addOrExpandField(List selectItems, Set aliases, - List> fields, boolean includeSystemVars, - SelectScope scope, SqlIdentifier id, RelDataTypeField field) { - switch (field.getType().getStructKind()) { - case PEEK_FIELDS: - case PEEK_FIELDS_DEFAULT: - final SqlNode starExp = id.plusStar(); - expandStar( - selectItems, - aliases, - fields, - includeSystemVars, - scope, - starExp); - return true; - - default: - addToSelectList( - selectItems, - aliases, - fields, - id, - scope, - includeSystemVars); - } - - return false; - } - - public SqlNode validate(SqlNode topNode) { - SqlValidatorScope scope = new EmptyScope(this); - scope = new CatalogScope(scope, ImmutableList.of("CATALOG")); - final SqlNode topNode2 = validateScopedExpression(topNode, scope); - final RelDataType type = getValidatedNodeType(topNode2); - Util.discard(type); - return topNode2; - } - - public List lookupHints(SqlNode topNode, SqlParserPos pos) { - SqlValidatorScope scope = new EmptyScope(this); - SqlNode outermostNode = performUnconditionalRewrites(topNode, false); - cursorSet.add(outermostNode); - if (outermostNode.isA(SqlKind.TOP_LEVEL)) { - registerQuery( - scope, - null, - outermostNode, - outermostNode, - null, - false); - } - final SqlValidatorNamespace ns = getNamespace(outermostNode); - if (ns == null) { - throw new AssertionError("Not a query: " + outermostNode); - } - Collection hintList = Sets.newTreeSet(SqlMoniker.COMPARATOR); - lookupSelectHints(ns, pos, hintList); - return ImmutableList.copyOf(hintList); - } - - public SqlMoniker lookupQualifiedName(SqlNode topNode, SqlParserPos pos) { - final String posString = pos.toString(); - IdInfo info = idPositions.get(posString); - if (info != null) { - final SqlQualified qualified = info.scope.fullyQualify(info.id); - return new SqlIdentifierMoniker(qualified.identifier); - } else { - return null; - } - } - - /** - * Looks up completion hints for a syntactically correct select SQL that has - * been parsed into an expression tree. - * - * @param select the Select node of the parsed expression tree - * @param pos indicates the position in the sql statement we want to get - * completion hints for - * @param hintList list of {@link SqlMoniker} (sql identifiers) that can - * fill in at the indicated position - */ - void lookupSelectHints( - SqlSelect select, - SqlParserPos pos, - Collection hintList) { - IdInfo info = idPositions.get(pos.toString()); - if ((info == null) || (info.scope == null)) { - SqlNode fromNode = select.getFrom(); - final SqlValidatorScope fromScope = getFromScope(select); - lookupFromHints(fromNode, fromScope, pos, hintList); - } else { - lookupNameCompletionHints(info.scope, info.id.names, - info.id.getParserPosition(), hintList); - } - } - - private void lookupSelectHints( - SqlValidatorNamespace ns, - SqlParserPos pos, - Collection hintList) { - final SqlNode node = ns.getNode(); - if (node instanceof SqlSelect) { - lookupSelectHints((SqlSelect) node, pos, hintList); - } - } - - private void lookupFromHints( - SqlNode node, - SqlValidatorScope scope, - SqlParserPos pos, - Collection hintList) { - if (node == null) { - // This can happen in cases like "select * _suggest_", so from clause is absent - return; - } - final SqlValidatorNamespace ns = getNamespace(node); - if (ns.isWrapperFor(IdentifierNamespace.class)) { - IdentifierNamespace idNs = ns.unwrap(IdentifierNamespace.class); - final SqlIdentifier id = idNs.getId(); - for (int i = 0; i < id.names.size(); i++) { - if (pos.toString().equals( - id.getComponent(i).getParserPosition().toString())) { - final List objNames = new ArrayList<>(); - SqlValidatorUtil.getSchemaObjectMonikers( - getCatalogReader(), - id.names.subList(0, i + 1), - objNames); - for (SqlMoniker objName : objNames) { - if (objName.getType() != SqlMonikerType.FUNCTION) { - hintList.add(objName); - } - } - return; - } - } - } - switch (node.getKind()) { - case JOIN: - lookupJoinHints((SqlJoin) node, scope, pos, hintList); - break; - default: - lookupSelectHints(ns, pos, hintList); - break; - } - } - - private void lookupJoinHints( - SqlJoin join, - SqlValidatorScope scope, - SqlParserPos pos, - Collection hintList) { - SqlNode left = join.getLeft(); - SqlNode right = join.getRight(); - SqlNode condition = join.getCondition(); - lookupFromHints(left, scope, pos, hintList); - if (hintList.size() > 0) { - return; - } - lookupFromHints(right, scope, pos, hintList); - if (hintList.size() > 0) { - return; - } - final JoinConditionType conditionType = join.getConditionType(); - final SqlValidatorScope joinScope = scopes.get(join); - switch (conditionType) { - case ON: - condition.findValidOptions(this, joinScope, pos, hintList); - return; - default: - - // No suggestions. - // Not supporting hints for other types such as 'Using' yet. - return; - } - } - - /** - * Populates a list of all the valid alternatives for an identifier. - * - * @param scope Validation scope - * @param names Components of the identifier - * @param pos position - * @param hintList a list of valid options - */ - public final void lookupNameCompletionHints( - SqlValidatorScope scope, - List names, - SqlParserPos pos, - Collection hintList) { - // Remove the last part of name - it is a dummy - List subNames = Util.skipLast(names); - - if (subNames.size() > 0) { - // If there's a prefix, resolve it to a namespace. - SqlValidatorNamespace ns = null; - for (String name : subNames) { - if (ns == null) { - final SqlValidatorScope.ResolvedImpl resolved = - new SqlValidatorScope.ResolvedImpl(); - final SqlNameMatcher nameMatcher = catalogReader.nameMatcher(); - scope.resolve(ImmutableList.of(name), nameMatcher, false, resolved); - if (resolved.count() == 1) { - ns = resolved.only().namespace; - } - } else { - ns = ns.lookupChild(name); - } - if (ns == null) { - break; - } - } - if (ns != null) { - RelDataType rowType = ns.getRowType(); - if (rowType.isStruct()) { - for (RelDataTypeField field : rowType.getFieldList()) { - hintList.add( - new SqlMonikerImpl( - field.getName(), - SqlMonikerType.COLUMN)); - } - } - } - - // builtin function names are valid completion hints when the - // identifier has only 1 name part - findAllValidFunctionNames(names, this, hintList, pos); - } else { - // No prefix; use the children of the current scope (that is, - // the aliases in the FROM clause) - scope.findAliases(hintList); - - // If there's only one alias, add all child columns - SelectScope selectScope = - SqlValidatorUtil.getEnclosingSelectScope(scope); - if ((selectScope != null) - && (selectScope.getChildren().size() == 1)) { - RelDataType rowType = - selectScope.getChildren().get(0).getRowType(); - for (RelDataTypeField field : rowType.getFieldList()) { - hintList.add( - new SqlMonikerImpl( - field.getName(), - SqlMonikerType.COLUMN)); - } - } - } - - findAllValidUdfNames(names, this, hintList); - } - - private static void findAllValidUdfNames( - List names, - SqlValidator validator, - Collection result) { - final List objNames = new ArrayList<>(); - SqlValidatorUtil.getSchemaObjectMonikers( - validator.getCatalogReader(), - names, - objNames); - for (SqlMoniker objName : objNames) { - if (objName.getType() == SqlMonikerType.FUNCTION) { - result.add(objName); - } - } - } - - private static void findAllValidFunctionNames( - List names, - SqlValidator validator, - Collection result, - SqlParserPos pos) { - // a function name can only be 1 part - if (names.size() > 1) { - return; - } - for (SqlOperator op : validator.getOperatorTable().getOperatorList()) { - SqlIdentifier curOpId = - new SqlIdentifier( - op.getName(), - pos); - - final SqlCall call = validator.makeNullaryCall(curOpId); - if (call != null) { - result.add( - new SqlMonikerImpl( - op.getName(), - SqlMonikerType.FUNCTION)); - } else { - if ((op.getSyntax() == SqlSyntax.FUNCTION) - || (op.getSyntax() == SqlSyntax.PREFIX)) { - if (op.getOperandTypeChecker() != null) { - String sig = op.getAllowedSignatures(); - sig = sig.replaceAll("'", ""); - result.add( - new SqlMonikerImpl( - sig, - SqlMonikerType.FUNCTION)); - continue; - } - result.add( - new SqlMonikerImpl( - op.getName(), - SqlMonikerType.FUNCTION)); - } - } - } - } - - public SqlNode validateParameterizedExpression( - SqlNode topNode, - final Map nameToTypeMap) { - SqlValidatorScope scope = new ParameterScope(this, nameToTypeMap); - return validateScopedExpression(topNode, scope); - } - - private SqlNode validateScopedExpression( - SqlNode topNode, - SqlValidatorScope scope) { - SqlNode outermostNode = performUnconditionalRewrites(topNode, false); - cursorSet.add(outermostNode); - top = outermostNode; - TRACER.trace("After unconditional rewrite: {}", outermostNode); - if (outermostNode.isA(SqlKind.TOP_LEVEL)) { - registerQuery(scope, null, outermostNode, outermostNode, null, false); - } - outermostNode.validate(this, scope); - if (!outermostNode.isA(SqlKind.TOP_LEVEL)) { - // force type derivation so that we can provide it to the - // caller later without needing the scope - deriveType(scope, outermostNode); - } - TRACER.trace("After validation: {}", outermostNode); - return outermostNode; - } - - public void validateQuery(SqlNode node, SqlValidatorScope scope, - RelDataType targetRowType) { - final SqlValidatorNamespace ns = getNamespace(node, scope); - if (node.getKind() == SqlKind.TABLESAMPLE) { - List operands = ((SqlCall) node).getOperandList(); - SqlSampleSpec sampleSpec = SqlLiteral.sampleValue(operands.get(1)); - if (sampleSpec instanceof SqlSampleSpec.SqlTableSampleSpec) { - validateFeature(RESOURCE.sQLFeature_T613(), node.getParserPosition()); - } else if (sampleSpec - instanceof SqlSampleSpec.SqlSubstitutionSampleSpec) { - validateFeature(RESOURCE.sQLFeatureExt_T613_Substitution(), - node.getParserPosition()); - } - } - - validateNamespace(ns, targetRowType); - switch (node.getKind()) { - case EXTEND: - // Until we have a dedicated namespace for EXTEND - deriveType(scope, node); - } - if (node == top) { - validateModality(node); - } - validateAccess( - node, - ns.getTable(), - SqlAccessEnum.SELECT); - - if (node.getKind() == SqlKind.SNAPSHOT) { - SqlSnapshot snapshot = (SqlSnapshot) node; - SqlNode period = snapshot.getPeriod(); - RelDataType dataType = deriveType(scope, period); - if (dataType.getSqlTypeName() != SqlTypeName.TIMESTAMP) { - throw newValidationError(period, - Static.RESOURCE.illegalExpressionForTemporal(dataType.getSqlTypeName().getName())); - } - if (!ns.getTable().isTemporal()) { - List qualifiedName = ns.getTable().getQualifiedName(); - String tableName = qualifiedName.get(qualifiedName.size() - 1); - throw newValidationError(snapshot.getTableRef(), - Static.RESOURCE.notTemporalTable(tableName)); - } - } - } - - /** - * Validates a namespace. - * - * @param namespace Namespace - * @param targetRowType Desired row type, must not be null, may be the data - * type 'unknown'. - */ - protected void validateNamespace(final SqlValidatorNamespace namespace, - RelDataType targetRowType) { - namespace.validate(targetRowType); - if (namespace.getNode() != null) { - setValidatedNodeType(namespace.getNode(), namespace.getType()); - } - } - - @VisibleForTesting - public SqlValidatorScope getEmptyScope() { - return new EmptyScope(this); - } - - public SqlValidatorScope getCursorScope(SqlSelect select) { - return cursorScopes.get(select); - } - - public SqlValidatorScope getWhereScope(SqlSelect select) { - return whereScopes.get(select); - } - - public SqlValidatorScope getSelectScope(SqlSelect select) { - return selectScopes.get(select); - } - - public SelectScope getRawSelectScope(SqlSelect select) { - SqlValidatorScope scope = getSelectScope(select); - if (scope instanceof AggregatingSelectScope) { - scope = ((AggregatingSelectScope) scope).getParent(); - } - return (SelectScope) scope; - } - - public SqlValidatorScope getHavingScope(SqlSelect select) { - // Yes, it's the same as getSelectScope - return selectScopes.get(select); - } - - public SqlValidatorScope getGroupScope(SqlSelect select) { - // Yes, it's the same as getWhereScope - return groupByScopes.get(select); - } - - public SqlValidatorScope getFromScope(SqlSelect select) { - return scopes.get(select); - } - - public SqlValidatorScope getOrderScope(SqlSelect select) { - return orderScopes.get(select); - } - - public SqlValidatorScope getMatchRecognizeScope(SqlMatchRecognize node) { - return scopes.get(node); - } - - public SqlValidatorScope getJoinScope(SqlNode node) { - return scopes.get(stripAs(node)); - } - - public SqlValidatorScope getOverScope(SqlNode node) { - return scopes.get(node); - } - - private SqlValidatorNamespace getNamespace(SqlNode node, - SqlValidatorScope scope) { - if (node instanceof SqlIdentifier && scope instanceof DelegatingScope) { - final SqlIdentifier id = (SqlIdentifier) node; - final DelegatingScope idScope = (DelegatingScope) ((DelegatingScope) scope).getParent(); - return getNamespace(id, idScope); - } else if (node instanceof SqlCall) { - // Handle extended identifiers. - final SqlCall call = (SqlCall) node; - switch (call.getOperator().getKind()) { - case EXTEND: - final SqlIdentifier id = (SqlIdentifier) call.getOperandList().get(0); - final DelegatingScope idScope = (DelegatingScope) scope; - return getNamespace(id, idScope); - case AS: - final SqlNode nested = call.getOperandList().get(0); - switch (nested.getKind()) { - case EXTEND: - return getNamespace(nested, scope); - } - break; - } - } - return getNamespace(node); - } - - private SqlValidatorNamespace getNamespace(SqlIdentifier id, DelegatingScope scope) { - if (id.isSimple()) { - final SqlNameMatcher nameMatcher = catalogReader.nameMatcher(); - final SqlValidatorScope.ResolvedImpl resolved = - new SqlValidatorScope.ResolvedImpl(); - scope.resolve(id.names, nameMatcher, false, resolved); - if (resolved.count() == 1) { - return resolved.only().namespace; - } - } - return getNamespace(id); - } - - public SqlValidatorNamespace getNamespace(SqlNode node) { - switch (node.getKind()) { - case AS: - - // AS has a namespace if it has a column list 'AS t (c1, c2, ...)' - final SqlValidatorNamespace ns = namespaces.get(node); - if (ns != null) { - return ns; - } - // fall through - case SNAPSHOT: - case OVER: - case COLLECTION_TABLE: - case ORDER_BY: - case TABLESAMPLE: - return getNamespace(((SqlCall) node).operand(0)); - default: - return namespaces.get(node); - } - } - - private void handleOffsetFetch(SqlNode offset, SqlNode fetch) { - if (offset instanceof SqlDynamicParam) { - setValidatedNodeType(offset, - typeFactory.createSqlType(SqlTypeName.INTEGER)); - } - if (fetch instanceof SqlDynamicParam) { - setValidatedNodeType(fetch, - typeFactory.createSqlType(SqlTypeName.INTEGER)); - } - } - - /** - * Performs expression rewrites which are always used unconditionally. These - * rewrites massage the expression tree into a standard form so that the - * rest of the validation logic can be simpler. - * - * @param node expression to be rewritten - * @param underFrom whether node appears directly under a FROM clause - * @return rewritten expression - */ - protected SqlNode performUnconditionalRewrites( - SqlNode node, - boolean underFrom) { - if (node == null) { - return node; - } - - SqlNode newOperand; - - // first transform operands and invoke generic call rewrite - if (node instanceof SqlCall) { - if (node instanceof SqlMerge) { - validatingSqlMerge = true; - } - SqlCall call = (SqlCall) node; - final SqlKind kind = call.getKind(); - final List operands = call.getOperandList(); - for (int i = 0; i < operands.size(); i++) { - SqlNode operand = operands.get(i); - boolean childUnderFrom; - if (kind == SqlKind.SELECT) { - childUnderFrom = i == SqlSelect.FROM_OPERAND; - } else if (kind == SqlKind.AS && (i == 0)) { - // for an aliased expression, it is under FROM if - // the AS expression is under FROM - childUnderFrom = underFrom; - } else { - childUnderFrom = false; - } - newOperand = - performUnconditionalRewrites(operand, childUnderFrom); - if (newOperand != null && newOperand != operand) { - call.setOperand(i, newOperand); - } - } - - if (call.getOperator() instanceof SqlUnresolvedFunction) { - assert call instanceof SqlBasicCall; - final SqlUnresolvedFunction function = - (SqlUnresolvedFunction) call.getOperator(); - // This function hasn't been resolved yet. Perform - // a half-hearted resolution now in case it's a - // builtin function requiring special casing. If it's - // not, we'll handle it later during overload resolution. - final List overloads = new ArrayList<>(); - opTab.lookupOperatorOverloads(function.getNameAsId(), - function.getFunctionType(), SqlSyntax.FUNCTION, overloads, - catalogReader.nameMatcher()); - if (overloads.size() == 1) { - ((SqlBasicCall) call).setOperator(overloads.get(0)); - } - } - if (rewriteCalls) { - node = call.getOperator().rewriteCall(this, call); - } - } else if (node instanceof SqlNodeList) { - SqlNodeList list = (SqlNodeList) node; - for (int i = 0, count = list.size(); i < count; i++) { - SqlNode operand = list.get(i); - newOperand = - performUnconditionalRewrites( - operand, - false); - if (newOperand != null) { - list.getList().set(i, newOperand); - } - } - } - - // now transform node itself - final SqlKind kind = node.getKind(); - switch (kind) { - case VALUES: - // CHECKSTYLE: IGNORE 1 - if (underFrom || true) { - // leave FROM (VALUES(...)) [ AS alias ] clauses alone, - // otherwise they grow cancerously if this rewrite is invoked - // over and over - return node; - } else { - final SqlNodeList selectList = - new SqlNodeList(SqlParserPos.ZERO); - selectList.add(SqlIdentifier.star(SqlParserPos.ZERO)); - return new SqlSelect(node.getParserPosition(), null, selectList, node, - null, null, null, null, null, null, null); - } - - case ORDER_BY: { - SqlOrderBy orderBy = (SqlOrderBy) node; - handleOffsetFetch(orderBy.offset, orderBy.fetch); - if (orderBy.query instanceof SqlSelect) { - SqlSelect select = (SqlSelect) orderBy.query; - - // Don't clobber existing ORDER BY. It may be needed for - // an order-sensitive function like RANK. - if (select.getOrderList() == null) { - // push ORDER BY into existing select - select.setOrderBy(orderBy.orderList); - select.setOffset(orderBy.offset); - select.setFetch(orderBy.fetch); - return select; - } - } - if (orderBy.query instanceof SqlWith - && ((SqlWith) orderBy.query).body instanceof SqlSelect) { - SqlWith with = (SqlWith) orderBy.query; - SqlSelect select = (SqlSelect) with.body; - - // Don't clobber existing ORDER BY. It may be needed for - // an order-sensitive function like RANK. - if (select.getOrderList() == null) { - // push ORDER BY into existing select - select.setOrderBy(orderBy.orderList); - select.setOffset(orderBy.offset); - select.setFetch(orderBy.fetch); - return with; - } - } - final SqlNodeList selectList = new SqlNodeList(SqlParserPos.ZERO); - selectList.add(SqlIdentifier.star(SqlParserPos.ZERO)); - final SqlNodeList orderList; - if (getInnerSelect(node) != null && isAggregate(getInnerSelect(node))) { - orderList = SqlNode.clone(orderBy.orderList); - // We assume that ORDER BY item does not have ASC etc. - // We assume that ORDER BY item is present in SELECT list. - for (int i = 0; i < orderList.size(); i++) { - SqlNode sqlNode = orderList.get(i); - SqlNodeList selectList2 = getInnerSelect(node).getSelectList(); - for (Ord sel : Ord.zip(selectList2)) { - if (stripAs(sel.e).equalsDeep(sqlNode, Litmus.IGNORE)) { - orderList.set(i, - SqlLiteral.createExactNumeric(Integer.toString(sel.i + 1), - SqlParserPos.ZERO)); - } - } - } - } else { - orderList = orderBy.orderList; - } - return new SqlSelect(SqlParserPos.ZERO, null, selectList, orderBy.query, - null, null, null, null, orderList, orderBy.offset, - orderBy.fetch); - } - - case EXPLICIT_TABLE: { - // (TABLE t) is equivalent to (SELECT * FROM t) - SqlCall call = (SqlCall) node; - final SqlNodeList selectList = new SqlNodeList(SqlParserPos.ZERO); - selectList.add(SqlIdentifier.star(SqlParserPos.ZERO)); - return new SqlSelect(SqlParserPos.ZERO, null, selectList, call.operand(0), - null, null, null, null, null, null, null); - } - - case DELETE: { - SqlDelete call = (SqlDelete) node; - SqlSelect select = createSourceSelectForDelete(call); - call.setSourceSelect(select); - break; - } - - case UPDATE: { - SqlUpdate call = (SqlUpdate) node; - SqlSelect select = createSourceSelectForUpdate(call); - call.setSourceSelect(select); - - // See if we're supposed to rewrite UPDATE to MERGE - // (unless this is the UPDATE clause of a MERGE, - // in which case leave it alone). - if (!validatingSqlMerge) { - SqlNode selfJoinSrcExpr = - getSelfJoinExprForUpdate( - call.getTargetTable(), - UPDATE_SRC_ALIAS); - if (selfJoinSrcExpr != null) { - node = rewriteUpdateToMerge(call, selfJoinSrcExpr); - } - } - break; - } - - case MERGE: { - SqlMerge call = (SqlMerge) node; - rewriteMerge(call); - break; - } - } - return node; - } - - private SqlSelect getInnerSelect(SqlNode node) { - for (;;) { - if (node instanceof SqlSelect) { - return (SqlSelect) node; - } else if (node instanceof SqlOrderBy) { - node = ((SqlOrderBy) node).query; - } else if (node instanceof SqlWith) { - node = ((SqlWith) node).body; - } else { - return null; - } - } - } - - private void rewriteMerge(SqlMerge call) { - SqlNodeList selectList; - SqlUpdate updateStmt = call.getUpdateCall(); - if (updateStmt != null) { - // if we have an update statement, just clone the select list - // from the update statement's source since it's the same as - // what we want for the select list of the merge source -- '*' - // followed by the update set expressions - selectList = SqlNode.clone(updateStmt.getSourceSelect().getSelectList()); - } else { - // otherwise, just use select * - selectList = new SqlNodeList(SqlParserPos.ZERO); - selectList.add(SqlIdentifier.star(SqlParserPos.ZERO)); - } - SqlNode targetTable = call.getTargetTable(); - if (call.getAlias() != null) { - targetTable = - SqlValidatorUtil.addAlias( - targetTable, - call.getAlias().getSimple()); - } - - // Provided there is an insert substatement, the source select for - // the merge is a left outer join between the source in the USING - // clause and the target table; otherwise, the join is just an - // inner join. Need to clone the source table reference in order - // for validation to work - SqlNode sourceTableRef = call.getSourceTableRef(); - SqlInsert insertCall = call.getInsertCall(); - JoinType joinType = (insertCall == null) ? JoinType.INNER : JoinType.LEFT; - final SqlNode leftJoinTerm = SqlNode.clone(sourceTableRef); - SqlNode outerJoin = - new SqlJoin(SqlParserPos.ZERO, - leftJoinTerm, - SqlLiteral.createBoolean(false, SqlParserPos.ZERO), - joinType.symbol(SqlParserPos.ZERO), - targetTable, - JoinConditionType.ON.symbol(SqlParserPos.ZERO), - call.getCondition()); - SqlSelect select = - new SqlSelect(SqlParserPos.ZERO, null, selectList, outerJoin, null, - null, null, null, null, null, null); - call.setSourceSelect(select); - - // Source for the insert call is a select of the source table - // reference with the select list being the value expressions; - // note that the values clause has already been converted to a - // select on the values row constructor; so we need to extract - // that via the from clause on the select - if (insertCall != null) { - SqlCall valuesCall = (SqlCall) insertCall.getSource(); - SqlCall rowCall = valuesCall.operand(0); - selectList = - new SqlNodeList( - rowCall.getOperandList(), - SqlParserPos.ZERO); - final SqlNode insertSource = SqlNode.clone(sourceTableRef); - select = - new SqlSelect(SqlParserPos.ZERO, null, selectList, insertSource, null, - null, null, null, null, null, null); - insertCall.setSource(select); - } - } - - private SqlNode rewriteUpdateToMerge( - SqlUpdate updateCall, - SqlNode selfJoinSrcExpr) { - // Make sure target has an alias. - if (updateCall.getAlias() == null) { - updateCall.setAlias( - new SqlIdentifier(UPDATE_TGT_ALIAS, SqlParserPos.ZERO)); - } - SqlNode selfJoinTgtExpr = - getSelfJoinExprForUpdate( - updateCall.getTargetTable(), - updateCall.getAlias().getSimple()); - assert selfJoinTgtExpr != null; - - // Create join condition between source and target exprs, - // creating a conjunction with the user-level WHERE - // clause if one was supplied - SqlNode condition = updateCall.getCondition(); - SqlNode selfJoinCond = - SqlStdOperatorTable.EQUALS.createCall( - SqlParserPos.ZERO, - selfJoinSrcExpr, - selfJoinTgtExpr); - if (condition == null) { - condition = selfJoinCond; - } else { - condition = - SqlStdOperatorTable.AND.createCall( - SqlParserPos.ZERO, - selfJoinCond, - condition); - } - SqlNode target = - updateCall.getTargetTable().clone(SqlParserPos.ZERO); - - // For the source, we need to anonymize the fields, so - // that for a statement like UPDATE T SET I = I + 1, - // there's no ambiguity for the "I" in "I + 1"; - // this is OK because the source and target have - // identical values due to the self-join. - // Note that we anonymize the source rather than the - // target because downstream, the optimizer rules - // don't want to see any projection on top of the target. - IdentifierNamespace ns = - new IdentifierNamespace(this, target, null, null); - RelDataType rowType = ns.getRowType(); - SqlNode source = updateCall.getTargetTable().clone(SqlParserPos.ZERO); - final SqlNodeList selectList = new SqlNodeList(SqlParserPos.ZERO); - int i = 1; - for (RelDataTypeField field : rowType.getFieldList()) { - SqlIdentifier col = - new SqlIdentifier( - field.getName(), - SqlParserPos.ZERO); - selectList.add( - SqlValidatorUtil.addAlias(col, UPDATE_ANON_PREFIX + i)); - ++i; - } - source = - new SqlSelect(SqlParserPos.ZERO, null, selectList, source, null, null, - null, null, null, null, null); - source = SqlValidatorUtil.addAlias(source, UPDATE_SRC_ALIAS); - SqlMerge mergeCall = - new SqlMerge(updateCall.getParserPosition(), target, condition, source, - updateCall, null, null, updateCall.getAlias()); - rewriteMerge(mergeCall); - return mergeCall; - } - - /** - * Allows a subclass to provide information about how to convert an UPDATE - * into a MERGE via self-join. If this method returns null, then no such - * conversion takes place. Otherwise, this method should return a suitable - * unique identifier expression for the given table. - * - * @param table identifier for table being updated - * @param alias alias to use for qualifying columns in expression, or null - * for unqualified references; if this is equal to - * {@value #UPDATE_SRC_ALIAS}, then column references have been - * anonymized to "SYS$ANONx", where x is the 1-based column - * number. - * @return expression for unique identifier, or null to prevent conversion - */ - protected SqlNode getSelfJoinExprForUpdate( - SqlNode table, - String alias) { - return null; - } - - /** - * Creates the SELECT statement that putatively feeds rows into an UPDATE - * statement to be updated. - * - * @param call Call to the UPDATE operator - * @return select statement - */ - protected SqlSelect createSourceSelectForUpdate(SqlUpdate call) { - final SqlNodeList selectList = new SqlNodeList(SqlParserPos.ZERO); - selectList.add(SqlIdentifier.star(SqlParserPos.ZERO)); - int ordinal = 0; - for (SqlNode exp : call.getSourceExpressionList()) { - // Force unique aliases to avoid a duplicate for Y with - // SET X=Y - String alias = SqlUtil.deriveAliasFromOrdinal(ordinal); - selectList.add(SqlValidatorUtil.addAlias(exp, alias)); - ++ordinal; - } - SqlNode sourceTable = call.getTargetTable(); - if (call.getAlias() != null) { - sourceTable = - SqlValidatorUtil.addAlias( - sourceTable, - call.getAlias().getSimple()); - } - return new SqlSelect(SqlParserPos.ZERO, null, selectList, sourceTable, - call.getCondition(), null, null, null, null, null, null); - } - - /** - * Creates the SELECT statement that putatively feeds rows into a DELETE - * statement to be deleted. - * - * @param call Call to the DELETE operator - * @return select statement - */ - protected SqlSelect createSourceSelectForDelete(SqlDelete call) { - final SqlNodeList selectList = new SqlNodeList(SqlParserPos.ZERO); - selectList.add(SqlIdentifier.star(SqlParserPos.ZERO)); - SqlNode sourceTable = call.getTargetTable(); - if (call.getAlias() != null) { - sourceTable = - SqlValidatorUtil.addAlias( - sourceTable, - call.getAlias().getSimple()); - } - return new SqlSelect(SqlParserPos.ZERO, null, selectList, sourceTable, - call.getCondition(), null, null, null, null, null, null); - } - - /** - * Returns null if there is no common type. E.g. if the rows have a - * different number of columns. - */ - RelDataType getTableConstructorRowType( - SqlCall values, - SqlValidatorScope scope) { - final List rows = values.getOperandList(); - assert rows.size() >= 1; - final List rowTypes = new ArrayList<>(); - for (final SqlNode row : rows) { - assert row.getKind() == SqlKind.ROW; - SqlCall rowConstructor = (SqlCall) row; - - // REVIEW jvs 10-Sept-2003: Once we support single-row queries as - // rows, need to infer aliases from there. - final List aliasList = new ArrayList<>(); - final List typeList = new ArrayList<>(); - for (Ord column : Ord.zip(rowConstructor.getOperandList())) { - final String alias = deriveAlias(column.e, column.i); - aliasList.add(alias); - final RelDataType type = deriveType(scope, column.e); - typeList.add(type); - } - rowTypes.add(typeFactory.createStructType(typeList, aliasList)); - } - if (rows.size() == 1) { - // TODO jvs 10-Oct-2005: get rid of this workaround once - // leastRestrictive can handle all cases - return rowTypes.get(0); - } - return typeFactory.leastRestrictive(rowTypes); - } - - public RelDataType getValidatedNodeType(SqlNode node) { - RelDataType type = getValidatedNodeTypeIfKnown(node); - if (type == null) { - throw Util.needToImplement(node); - } else { - return type; - } - } - - public RelDataType getValidatedNodeTypeIfKnown(SqlNode node) { - final RelDataType type = nodeToTypeMap.get(node); - if (type != null) { - return type; - } - final SqlValidatorNamespace ns = getNamespace(node); - if (ns != null) { - return ns.getType(); - } - final SqlNode original = originalExprs.get(node); - if (original != null && original != node) { - return getValidatedNodeType(original); - } - if (node instanceof SqlIdentifier) { - return getCatalogReader().getNamedType((SqlIdentifier) node); - } - return null; - } - - /** - * Saves the type of a {@link SqlNode}, now that it has been validated. - * - *

    Unlike the base class method, this method is not deprecated. - * It is available from within Calcite, but is not part of the public API. - * - * @param node A SQL parse tree node, never null - * @param type Its type; must not be null - */ - @SuppressWarnings("deprecation") - public final void setValidatedNodeType(SqlNode node, RelDataType type) { - Objects.requireNonNull(type); - Objects.requireNonNull(node); - if (type.equals(unknownType)) { - // don't set anything until we know what it is, and don't overwrite - // a known type with the unknown type - return; - } - nodeToTypeMap.put(node, type); - } - - public void removeValidatedNodeType(SqlNode node) { - nodeToTypeMap.remove(node); - } - - @Nullable public SqlCall makeNullaryCall(SqlIdentifier id) { - if (id.names.size() == 1 && !id.isComponentQuoted(0)) { - final List list = new ArrayList<>(); - opTab.lookupOperatorOverloads(id, null, SqlSyntax.FUNCTION, list, - catalogReader.nameMatcher()); - for (SqlOperator operator : list) { - if (operator.getSyntax() == SqlSyntax.FUNCTION_ID) { - // Even though this looks like an identifier, it is a - // actually a call to a function. Construct a fake - // call to this function, so we can use the regular - // operator validation. - return new SqlBasicCall(operator, SqlNode.EMPTY_ARRAY, - id.getParserPosition(), true, null); - } - } - } - return null; - } - - public RelDataType deriveType( - SqlValidatorScope scope, - SqlNode expr) { - Objects.requireNonNull(scope); - Objects.requireNonNull(expr); - - // if we already know the type, no need to re-derive - RelDataType type = nodeToTypeMap.get(expr); - if (type != null) { - return type; - } - final SqlValidatorNamespace ns = getNamespace(expr); - if (ns != null) { - return ns.getType(); - } - type = deriveTypeImpl(scope, expr); - Preconditions.checkArgument( - type != null, - "SqlValidator.deriveTypeInternal returned null"); - setValidatedNodeType(expr, type); - return type; - } - - /** - * Derives the type of a node, never null. - */ - RelDataType deriveTypeImpl( - SqlValidatorScope scope, - SqlNode operand) { - DeriveTypeVisitor v = new DeriveTypeVisitor(scope); - final RelDataType type = operand.accept(v); - return Objects.requireNonNull(scope.nullifyType(operand, type)); - } - - public RelDataType deriveConstructorType( - SqlValidatorScope scope, - SqlCall call, - SqlFunction unresolvedConstructor, - SqlFunction resolvedConstructor, - List argTypes) { - SqlIdentifier sqlIdentifier = unresolvedConstructor.getSqlIdentifier(); - assert sqlIdentifier != null; - RelDataType type = catalogReader.getNamedType(sqlIdentifier); - if (type == null) { - // TODO jvs 12-Feb-2005: proper type name formatting - throw newValidationError(sqlIdentifier, - RESOURCE.unknownDatatypeName(sqlIdentifier.toString())); - } - - if (resolvedConstructor == null) { - if (call.operandCount() > 0) { - // This is not a default constructor invocation, and - // no user-defined constructor could be found - throw handleUnresolvedFunction(call, unresolvedConstructor, argTypes, - null); - } - } else { - SqlCall testCall = - resolvedConstructor.createCall( - call.getParserPosition(), - call.getOperandList()); - RelDataType returnType = - resolvedConstructor.validateOperands( - this, - scope, - testCall); - assert type == returnType; - } - - if (shouldExpandIdentifiers()) { - if (resolvedConstructor != null) { - ((SqlBasicCall) call).setOperator(resolvedConstructor); - } else { - // fake a fully-qualified call to the default constructor - ((SqlBasicCall) call).setOperator( - new SqlFunction( - type.getSqlIdentifier(), - ReturnTypes.explicit(type), - null, - null, - null, - SqlFunctionCategory.USER_DEFINED_CONSTRUCTOR)); - } - } - return type; - } - - public CalciteException handleUnresolvedFunction(SqlCall call, - SqlFunction unresolvedFunction, List argTypes, - List argNames) { - // For builtins, we can give a better error message - final List overloads = new ArrayList<>(); - opTab.lookupOperatorOverloads(unresolvedFunction.getNameAsId(), null, - SqlSyntax.FUNCTION, overloads, catalogReader.nameMatcher()); - if (overloads.size() == 1) { - SqlFunction fun = (SqlFunction) overloads.get(0); - if ((fun.getSqlIdentifier() == null) - && (fun.getSyntax() != SqlSyntax.FUNCTION_ID)) { - final int expectedArgCount = - fun.getOperandCountRange().getMin(); - throw newValidationError(call, - RESOURCE.invalidArgCount(call.getOperator().getName(), - expectedArgCount)); - } - } - - AssignableOperandTypeChecker typeChecking = - new AssignableOperandTypeChecker(argTypes, argNames); - String signature = - typeChecking.getAllowedSignatures( - unresolvedFunction, - unresolvedFunction.getName()); - throw newValidationError(call, - RESOURCE.validatorUnknownFunction(signature)); - } - - protected void inferUnknownTypes( - @Nonnull RelDataType inferredType, - @Nonnull SqlValidatorScope scope, - @Nonnull SqlNode node) { - Objects.requireNonNull(inferredType); - Objects.requireNonNull(scope); - Objects.requireNonNull(node); - final SqlValidatorScope newScope = scopes.get(node); - if (newScope != null) { - scope = newScope; - } - boolean isNullLiteral = SqlUtil.isNullLiteral(node, false); - if ((node instanceof SqlDynamicParam) || isNullLiteral) { - if (inferredType.equals(unknownType)) { - if (isNullLiteral) { - throw newValidationError(node, RESOURCE.nullIllegal()); - } else { - throw newValidationError(node, RESOURCE.dynamicParamIllegal()); - } - } - - // REVIEW: should dynamic parameter types always be nullable? - RelDataType newInferredType = - typeFactory.createTypeWithNullability(inferredType, true); - if (SqlTypeUtil.inCharFamily(inferredType)) { - newInferredType = - typeFactory.createTypeWithCharsetAndCollation( - newInferredType, - inferredType.getCharset(), - inferredType.getCollation()); - } - setValidatedNodeType(node, newInferredType); - } else if (node instanceof SqlNodeList) { - SqlNodeList nodeList = (SqlNodeList) node; - if (inferredType.isStruct()) { - if (inferredType.getFieldCount() != nodeList.size()) { - // this can happen when we're validating an INSERT - // where the source and target degrees are different; - // bust out, and the error will be detected higher up - return; - } - } - int i = 0; - for (SqlNode child : nodeList) { - RelDataType type; - if (inferredType.isStruct()) { - type = inferredType.getFieldList().get(i).getType(); - ++i; - } else { - type = inferredType; - } - inferUnknownTypes(type, scope, child); - } - } else if (node instanceof SqlCase) { - final SqlCase caseCall = (SqlCase) node; - - final RelDataType whenType = - caseCall.getValueOperand() == null ? booleanType : unknownType; - for (SqlNode sqlNode : caseCall.getWhenOperands().getList()) { - inferUnknownTypes(whenType, scope, sqlNode); - } - RelDataType returnType = deriveType(scope, node); - for (SqlNode sqlNode : caseCall.getThenOperands().getList()) { - inferUnknownTypes(returnType, scope, sqlNode); - } - - if (!SqlUtil.isNullLiteral(caseCall.getElseOperand(), false)) { - inferUnknownTypes( - returnType, - scope, - caseCall.getElseOperand()); - } else { - setValidatedNodeType(caseCall.getElseOperand(), returnType); - } - } else if (node.getKind() == SqlKind.AS) { - // For AS operator, only infer the operand not the alias - inferUnknownTypes(inferredType, scope, ((SqlCall) node).operand(0)); - } else if (node instanceof SqlCall) { - final SqlCall call = (SqlCall) node; - final SqlOperandTypeInference operandTypeInference = - call.getOperator().getOperandTypeInference(); - final SqlCallBinding callBinding = new SqlCallBinding(this, scope, call); - final List operands = callBinding.operands(); - final RelDataType[] operandTypes = new RelDataType[operands.size()]; - Arrays.fill(operandTypes, unknownType); - // TODO: eventually should assert(operandTypeInference != null) - // instead; for now just eat it - if (operandTypeInference != null) { - operandTypeInference.inferOperandTypes( - callBinding, - inferredType, - operandTypes); - } - for (int i = 0; i < operands.size(); ++i) { - final SqlNode operand = operands.get(i); - if (operand != null) { - inferUnknownTypes(operandTypes[i], scope, operand); - } - } - } - } - - /** - * Adds an expression to a select list, ensuring that its alias does not - * clash with any existing expressions on the list. - */ - protected void addToSelectList( - List list, - Set aliases, - List> fieldList, - SqlNode exp, - SqlValidatorScope scope, - final boolean includeSystemVars) { - String alias = SqlValidatorUtil.getAlias(exp, -1); - String uniqueAlias = - SqlValidatorUtil.uniquify( - alias, aliases, SqlValidatorUtil.EXPR_SUGGESTER); - if (!alias.equals(uniqueAlias)) { - exp = SqlValidatorUtil.addAlias(exp, uniqueAlias); - } - fieldList.add(Pair.of(uniqueAlias, deriveType(scope, exp))); - list.add(exp); - } - - public String deriveAlias( - SqlNode node, - int ordinal) { - return SqlValidatorUtil.getAlias(node, ordinal); - } - - // implement SqlValidator - public void setIdentifierExpansion(boolean expandIdentifiers) { - this.expandIdentifiers = expandIdentifiers; - } - - // implement SqlValidator - public void setColumnReferenceExpansion( - boolean expandColumnReferences) { - this.expandColumnReferences = expandColumnReferences; - } - - // implement SqlValidator - public boolean getColumnReferenceExpansion() { - return expandColumnReferences; - } - - public void setDefaultNullCollation(NullCollation nullCollation) { - this.nullCollation = Objects.requireNonNull(nullCollation); - } - - public NullCollation getDefaultNullCollation() { - return nullCollation; - } - - // implement SqlValidator - public void setCallRewrite(boolean rewriteCalls) { - this.rewriteCalls = rewriteCalls; - } - - public boolean shouldExpandIdentifiers() { - return expandIdentifiers; - } - - protected boolean shouldAllowIntermediateOrderBy() { - return true; - } - - private void registerMatchRecognize( - SqlValidatorScope parentScope, - SqlValidatorScope usingScope, - SqlMatchRecognize call, - SqlNode enclosingNode, - String alias, - boolean forceNullable) { - - final MatchRecognizeNamespace matchRecognizeNamespace = - createMatchRecognizeNameSpace(call, enclosingNode); - registerNamespace(usingScope, alias, matchRecognizeNamespace, forceNullable); - - final MatchRecognizeScope matchRecognizeScope = - new MatchRecognizeScope(parentScope, call); - scopes.put(call, matchRecognizeScope); - - // parse input query - SqlNode expr = call.getTableRef(); - SqlNode newExpr = registerFrom(usingScope, matchRecognizeScope, true, expr, - expr, null, null, forceNullable, false); - if (expr != newExpr) { - call.setOperand(0, newExpr); - } - } - - protected MatchRecognizeNamespace createMatchRecognizeNameSpace( - SqlMatchRecognize call, - SqlNode enclosingNode) { - return new MatchRecognizeNamespace(this, call, enclosingNode); - } - - /** - * Registers a new namespace, and adds it as a child of its parent scope. - * Derived class can override this method to tinker with namespaces as they - * are created. - * - * @param usingScope Parent scope (which will want to look for things in - * this namespace) - * @param alias Alias by which parent will refer to this namespace - * @param ns Namespace - * @param forceNullable Whether to force the type of namespace to be nullable - */ - protected void registerNamespace( - SqlValidatorScope usingScope, - String alias, - SqlValidatorNamespace ns, - boolean forceNullable) { - namespaces.put(ns.getNode(), ns); - if (usingScope != null) { - usingScope.addChild(ns, alias, forceNullable); - } - } - - /** - * Registers scopes and namespaces implied a relational expression in the - * FROM clause. - * - *

    {@code parentScope} and {@code usingScope} are often the same. They - * differ when the namespace are not visible within the parent. (Example - * needed.) - * - *

    Likewise, {@code enclosingNode} and {@code node} are often the same. - * {@code enclosingNode} is the topmost node within the FROM clause, from - * which any decorations like an alias (AS alias) or a table - * sample clause are stripped away to get {@code node}. Both are recorded in - * the namespace. - * - * @param parentScope Parent scope which this scope turns to in order to - * resolve objects - * @param usingScope Scope whose child list this scope should add itself to - * @param register Whether to register this scope as a child of - * {@code usingScope} - * @param node Node which namespace is based on - * @param enclosingNode Outermost node for namespace, including decorations - * such as alias and sample clause - * @param alias Alias - * @param extendList Definitions of extended columns - * @param forceNullable Whether to force the type of namespace to be - * nullable because it is in an outer join - * @param lateral Whether LATERAL is specified, so that items to the - * left of this in the JOIN tree are visible in the - * scope - * @return registered node, usually the same as {@code node} - */ - private SqlNode registerFrom( - SqlValidatorScope parentScope, - SqlValidatorScope usingScope, - boolean register, - final SqlNode node, - SqlNode enclosingNode, - String alias, - SqlNodeList extendList, - boolean forceNullable, - final boolean lateral) { - final SqlKind kind = node.getKind(); - - SqlNode expr; - SqlNode newExpr; - - // Add an alias if necessary. - SqlNode newNode = node; - if (alias == null) { - switch (kind) { - case IDENTIFIER: - case OVER: - alias = deriveAlias(node, -1); - if (alias == null) { - alias = deriveAlias(node, nextGeneratedId++); - } - if (shouldExpandIdentifiers()) { - newNode = SqlValidatorUtil.addAlias(node, alias); - } - break; - - case SELECT: - case UNION: - case INTERSECT: - case EXCEPT: - case VALUES: - case UNNEST: - case OTHER_FUNCTION: - case COLLECTION_TABLE: - case MATCH_RECOGNIZE: - - // give this anonymous construct a name since later - // query processing stages rely on it - alias = deriveAlias(node, nextGeneratedId++); - if (shouldExpandIdentifiers()) { - // Since we're expanding identifiers, we should make the - // aliases explicit too, otherwise the expanded query - // will not be consistent if we convert back to SQL, e.g. - // "select EXPR$1.EXPR$2 from values (1)". - newNode = SqlValidatorUtil.addAlias(node, alias); - } - break; - } - } - - if (lateral) { - SqlValidatorScope s = usingScope; - while (s instanceof JoinScope) { - s = ((JoinScope) s).getUsingScope(); - } - final SqlNode node2 = s != null ? s.getNode() : node; - final TableScope tableScope = new TableScope(parentScope, node2); - if (usingScope instanceof ListScope) { - for (ScopeChild child : ((ListScope) usingScope).children) { - tableScope.addChild(child.namespace, child.name, child.nullable); - } - } - parentScope = tableScope; - } - - SqlCall call; - SqlNode operand; - SqlNode newOperand; - - switch (kind) { - case AS: - call = (SqlCall) node; - if (alias == null) { - alias = call.operand(1).toString(); - } - final boolean needAlias = call.operandCount() > 2; - expr = call.operand(0); - newExpr = - registerFrom( - parentScope, - usingScope, - !needAlias, - expr, - enclosingNode, - alias, - extendList, - forceNullable, - lateral); - if (newExpr != expr) { - call.setOperand(0, newExpr); - } - - // If alias has a column list, introduce a namespace to translate - // column names. We skipped registering it just now. - if (needAlias) { - registerNamespace( - usingScope, - alias, - new AliasNamespace(this, call, enclosingNode), - forceNullable); - } - return node; - case MATCH_RECOGNIZE: - registerMatchRecognize(parentScope, usingScope, - (SqlMatchRecognize) node, enclosingNode, alias, forceNullable); - return node; - case TABLESAMPLE: - call = (SqlCall) node; - expr = call.operand(0); - newExpr = - registerFrom( - parentScope, - usingScope, - true, - expr, - enclosingNode, - alias, - extendList, - forceNullable, - lateral); - if (newExpr != expr) { - call.setOperand(0, newExpr); - } - return node; - - case JOIN: - final SqlJoin join = (SqlJoin) node; - final JoinScope joinScope = - new JoinScope(parentScope, usingScope, join); - scopes.put(join, joinScope); - final SqlNode left = join.getLeft(); - final SqlNode right = join.getRight(); - final boolean rightIsLateral = isLateral(right); - boolean forceLeftNullable = forceNullable; - boolean forceRightNullable = forceNullable; - switch (join.getJoinType()) { - case LEFT: - forceRightNullable = true; - break; - case RIGHT: - forceLeftNullable = true; - break; - case FULL: - forceLeftNullable = true; - forceRightNullable = true; - break; - } - final SqlNode newLeft = - registerFrom( - parentScope, - joinScope, - true, - left, - left, - null, - null, - forceLeftNullable, - lateral); - if (newLeft != left) { - join.setLeft(newLeft); - } - final SqlNode newRight = - registerFrom( - parentScope, - joinScope, - true, - right, - right, - null, - null, - forceRightNullable, - lateral); - if (newRight != right) { - join.setRight(newRight); - } - registerSubQueries(joinScope, join.getCondition()); - final JoinNamespace joinNamespace = new JoinNamespace(this, join); - registerNamespace(null, null, joinNamespace, forceNullable); - return join; - - case IDENTIFIER: - final SqlIdentifier id = (SqlIdentifier) node; - final IdentifierNamespace newNs = - new IdentifierNamespace( - this, id, extendList, enclosingNode, - parentScope); - registerNamespace(register ? usingScope : null, alias, newNs, - forceNullable); - if (tableScope == null) { - tableScope = new TableScope(parentScope, node); - } - tableScope.addChild(newNs, alias, forceNullable); - if (extendList != null && extendList.size() != 0) { - return enclosingNode; - } - return newNode; - - case LATERAL: - return registerFrom( - parentScope, - usingScope, - register, - ((SqlCall) node).operand(0), - enclosingNode, - alias, - extendList, - forceNullable, - true); - - case COLLECTION_TABLE: - call = (SqlCall) node; - operand = call.operand(0); - newOperand = - registerFrom( - parentScope, - usingScope, - register, - operand, - enclosingNode, - alias, - extendList, - forceNullable, lateral); - if (newOperand != operand) { - call.setOperand(0, newOperand); - } - scopes.put(node, parentScope); - return newNode; - - case UNNEST: - if (!lateral) { - return registerFrom(parentScope, usingScope, register, node, - enclosingNode, alias, extendList, forceNullable, true); - } - // fall through - case SELECT: - case UNION: - case INTERSECT: - case EXCEPT: - case VALUES: - case WITH: - case OTHER_FUNCTION: - if (alias == null) { - alias = deriveAlias(node, nextGeneratedId++); - } - registerQuery( - parentScope, - register ? usingScope : null, - node, - enclosingNode, - alias, - forceNullable); - return newNode; - - case OVER: - if (!shouldAllowOverRelation()) { - throw Util.unexpected(kind); - } - call = (SqlCall) node; - final OverScope overScope = new OverScope(usingScope, call); - scopes.put(call, overScope); - operand = call.operand(0); - newOperand = - registerFrom( - parentScope, - overScope, - true, - operand, - enclosingNode, - alias, - extendList, - forceNullable, - lateral); - if (newOperand != operand) { - call.setOperand(0, newOperand); - } - - for (ScopeChild child : overScope.children) { - registerNamespace(register ? usingScope : null, child.name, - child.namespace, forceNullable); - } - - return newNode; - - case EXTEND: - final SqlCall extend = (SqlCall) node; - return registerFrom(parentScope, - usingScope, - true, - extend.getOperandList().get(0), - extend, - alias, - (SqlNodeList) extend.getOperandList().get(1), - forceNullable, - lateral); - - case SNAPSHOT: - call = (SqlCall) node; - operand = call.operand(0); - newOperand = registerFrom( - tableScope == null ? parentScope : tableScope, - usingScope, - register, - operand, - enclosingNode, - alias, - extendList, - forceNullable, - true); - if (newOperand != operand) { - call.setOperand(0, newOperand); - } - scopes.put(node, parentScope); - return newNode; - - default: - throw Util.unexpected(kind); - } - } - - private static boolean isLateral(SqlNode node) { - switch (node.getKind()) { - case LATERAL: - case UNNEST: - // Per SQL std, UNNEST is implicitly LATERAL. - return true; - case AS: - return isLateral(((SqlCall) node).operand(0)); - default: - return false; - } - } - - protected boolean shouldAllowOverRelation() { - return false; - } - - /** - * Creates a namespace for a SELECT node. Derived class may - * override this factory method. - * - * @param select Select node - * @param enclosingNode Enclosing node - * @return Select namespace - */ - protected SelectNamespace createSelectNamespace( - SqlSelect select, - SqlNode enclosingNode) { - return new SelectNamespace(this, select, enclosingNode); - } - - /** - * Creates a namespace for a set operation (UNION, - * INTERSECT, or EXCEPT). Derived class may override - * this factory method. - * - * @param call Call to set operation - * @param enclosingNode Enclosing node - * @return Set operation namespace - */ - protected SetopNamespace createSetopNamespace( - SqlCall call, - SqlNode enclosingNode) { - return new SetopNamespace(this, call, enclosingNode); - } - - /** - * Registers a query in a parent scope. - * - * @param parentScope Parent scope which this scope turns to in order to - * resolve objects - * @param usingScope Scope whose child list this scope should add itself to - * @param node Query node - * @param alias Name of this query within its parent. Must be specified - * if usingScope != null - */ - private void registerQuery( - SqlValidatorScope parentScope, - SqlValidatorScope usingScope, - SqlNode node, - SqlNode enclosingNode, - String alias, - boolean forceNullable) { - Preconditions.checkArgument(usingScope == null || alias != null); - registerQuery( - parentScope, - usingScope, - node, - enclosingNode, - alias, - forceNullable, - true); - } - - /** - * Registers a query in a parent scope. - * - * @param parentScope Parent scope which this scope turns to in order to - * resolve objects - * @param usingScope Scope whose child list this scope should add itself to - * @param node Query node - * @param alias Name of this query within its parent. Must be specified - * if usingScope != null - * @param checkUpdate if true, validate that the update feature is supported - * if validating the update statement - */ - private void registerQuery( - SqlValidatorScope parentScope, - SqlValidatorScope usingScope, - SqlNode node, - SqlNode enclosingNode, - String alias, - boolean forceNullable, - boolean checkUpdate) { - Objects.requireNonNull(node); - Objects.requireNonNull(enclosingNode); - Preconditions.checkArgument(usingScope == null || alias != null); - - SqlCall call; - List operands; - switch (node.getKind()) { - case SELECT: - final SqlSelect select = (SqlSelect) node; - final SelectNamespace selectNs = - createSelectNamespace(select, enclosingNode); - registerNamespace(usingScope, alias, selectNs, forceNullable); - final SqlValidatorScope windowParentScope = - (usingScope != null) ? usingScope : parentScope; - SelectScope selectScope = - new SelectScope(parentScope, windowParentScope, select); - scopes.put(select, selectScope); - - // Start by registering the WHERE clause - whereScopes.put(select, selectScope); - registerOperandSubQueries( - selectScope, - select, - SqlSelect.WHERE_OPERAND); - - // Register FROM with the inherited scope 'parentScope', not - // 'selectScope', otherwise tables in the FROM clause would be - // able to see each other. - final SqlNode from = select.getFrom(); - if (from != null) { - final SqlNode newFrom = - registerFrom( - parentScope, - selectScope, - true, - from, - from, - null, - null, - false, - false); - if (newFrom != from) { - select.setFrom(newFrom); - } - } - - // If this is an aggregating query, the SELECT list and HAVING - // clause use a different scope, where you can only reference - // columns which are in the GROUP BY clause. - SqlValidatorScope aggScope = selectScope; - if (isAggregate(select)) { - aggScope = - new AggregatingSelectScope(selectScope, select, false); - selectScopes.put(select, aggScope); - } else { - selectScopes.put(select, selectScope); - } - if (select.getGroup() != null) { - GroupByScope groupByScope = - new GroupByScope(selectScope, select.getGroup(), select); - groupByScopes.put(select, groupByScope); - registerSubQueries(groupByScope, select.getGroup()); - } - registerOperandSubQueries( - aggScope, - select, - SqlSelect.HAVING_OPERAND); - registerSubQueries(aggScope, select.getSelectList()); - final SqlNodeList orderList = select.getOrderList(); - if (orderList != null) { - // If the query is 'SELECT DISTINCT', restrict the columns - // available to the ORDER BY clause. - if (select.isDistinct()) { - aggScope = - new AggregatingSelectScope(selectScope, select, true); - } - OrderByScope orderScope = - new OrderByScope(aggScope, orderList, select); - orderScopes.put(select, orderScope); - registerSubQueries(orderScope, orderList); - - if (!isAggregate(select)) { - // Since this is not an aggregating query, - // there cannot be any aggregates in the ORDER BY clause. - SqlNode agg = aggFinder.findAgg(orderList); - if (agg != null) { - throw newValidationError(agg, RESOURCE.aggregateIllegalInOrderBy()); - } - } - } - break; - - case INTERSECT: - validateFeature(RESOURCE.sQLFeature_F302(), node.getParserPosition()); - registerSetop( - parentScope, - usingScope, - node, - node, - alias, - forceNullable); - break; - - case EXCEPT: - validateFeature(RESOURCE.sQLFeature_E071_03(), node.getParserPosition()); - registerSetop( - parentScope, - usingScope, - node, - node, - alias, - forceNullable); - break; - - case UNION: - registerSetop( - parentScope, - usingScope, - node, - node, - alias, - forceNullable); - break; - - case WITH: - registerWith(parentScope, usingScope, (SqlWith) node, enclosingNode, - alias, forceNullable, checkUpdate); - break; - - case VALUES: - call = (SqlCall) node; - scopes.put(call, parentScope); - final TableConstructorNamespace tableConstructorNamespace = - new TableConstructorNamespace( - this, - call, - parentScope, - enclosingNode); - registerNamespace( - usingScope, - alias, - tableConstructorNamespace, - forceNullable); - operands = call.getOperandList(); - for (int i = 0; i < operands.size(); ++i) { - assert operands.get(i).getKind() == SqlKind.ROW; - - // FIXME jvs 9-Feb-2005: Correlation should - // be illegal in these sub-queries. Same goes for - // any non-lateral SELECT in the FROM list. - registerOperandSubQueries(parentScope, call, i); - } - break; - - case INSERT: - SqlInsert insertCall = (SqlInsert) node; - InsertNamespace insertNs = - new InsertNamespace( - this, - insertCall, - enclosingNode, - parentScope); - registerNamespace(usingScope, null, insertNs, forceNullable); - registerQuery( - parentScope, - usingScope, - insertCall.getSource(), - enclosingNode, - null, - false); - break; - - case DELETE: - SqlDelete deleteCall = (SqlDelete) node; - DeleteNamespace deleteNs = - new DeleteNamespace( - this, - deleteCall, - enclosingNode, - parentScope); - registerNamespace(usingScope, null, deleteNs, forceNullable); - registerQuery( - parentScope, - usingScope, - deleteCall.getSourceSelect(), - enclosingNode, - null, - false); - break; - - case UPDATE: - if (checkUpdate) { - validateFeature(RESOURCE.sQLFeature_E101_03(), - node.getParserPosition()); - } - SqlUpdate updateCall = (SqlUpdate) node; - UpdateNamespace updateNs = - new UpdateNamespace( - this, - updateCall, - enclosingNode, - parentScope); - registerNamespace(usingScope, null, updateNs, forceNullable); - registerQuery( - parentScope, - usingScope, - updateCall.getSourceSelect(), - enclosingNode, - null, - false); - break; - - case MERGE: - validateFeature(RESOURCE.sQLFeature_F312(), node.getParserPosition()); - SqlMerge mergeCall = (SqlMerge) node; - MergeNamespace mergeNs = - new MergeNamespace( - this, - mergeCall, - enclosingNode, - parentScope); - registerNamespace(usingScope, null, mergeNs, forceNullable); - registerQuery( - parentScope, - usingScope, - mergeCall.getSourceSelect(), - enclosingNode, - null, - false); - - // update call can reference either the source table reference - // or the target table, so set its parent scope to the merge's - // source select; when validating the update, skip the feature - // validation check - if (mergeCall.getUpdateCall() != null) { - registerQuery( - whereScopes.get(mergeCall.getSourceSelect()), - null, - mergeCall.getUpdateCall(), - enclosingNode, - null, - false, - false); - } - if (mergeCall.getInsertCall() != null) { - registerQuery( - parentScope, - null, - mergeCall.getInsertCall(), - enclosingNode, - null, - false); - } - break; - - case UNNEST: - call = (SqlCall) node; - final UnnestNamespace unnestNs = - new UnnestNamespace(this, call, parentScope, enclosingNode); - registerNamespace( - usingScope, - alias, - unnestNs, - forceNullable); - registerOperandSubQueries(parentScope, call, 0); - scopes.put(node, parentScope); - break; - - case OTHER_FUNCTION: - call = (SqlCall) node; - ProcedureNamespace procNs = - new ProcedureNamespace( - this, - parentScope, - call, - enclosingNode); - registerNamespace( - usingScope, - alias, - procNs, - forceNullable); - registerSubQueries(parentScope, call); - break; - - case MULTISET_QUERY_CONSTRUCTOR: - case MULTISET_VALUE_CONSTRUCTOR: - validateFeature(RESOURCE.sQLFeature_S271(), node.getParserPosition()); - call = (SqlCall) node; - CollectScope cs = new CollectScope(parentScope, usingScope, call); - final CollectNamespace tableConstructorNs = - new CollectNamespace(call, cs, enclosingNode); - final String alias2 = deriveAlias(node, nextGeneratedId++); - registerNamespace( - usingScope, - alias2, - tableConstructorNs, - forceNullable); - operands = call.getOperandList(); - for (int i = 0; i < operands.size(); i++) { - registerOperandSubQueries(parentScope, call, i); - } - break; - - default: - throw Util.unexpected(node.getKind()); - } - } - - private void registerSetop( - SqlValidatorScope parentScope, - SqlValidatorScope usingScope, - SqlNode node, - SqlNode enclosingNode, - String alias, - boolean forceNullable) { - SqlCall call = (SqlCall) node; - final SetopNamespace setopNamespace = - createSetopNamespace(call, enclosingNode); - registerNamespace(usingScope, alias, setopNamespace, forceNullable); - - // A setop is in the same scope as its parent. - scopes.put(call, parentScope); - for (SqlNode operand : call.getOperandList()) { - registerQuery( - parentScope, - null, - operand, - operand, - null, - false); - } - } - - private void registerWith( - SqlValidatorScope parentScope, - SqlValidatorScope usingScope, - SqlWith with, - SqlNode enclosingNode, - String alias, - boolean forceNullable, - boolean checkUpdate) { - final WithNamespace withNamespace = - new WithNamespace(this, with, enclosingNode); - registerNamespace(usingScope, alias, withNamespace, forceNullable); - - SqlValidatorScope scope = parentScope; - for (SqlNode withItem_ : with.withList) { - final SqlWithItem withItem = (SqlWithItem) withItem_; - final WithScope withScope = new WithScope(scope, withItem); - scopes.put(withItem, withScope); - - registerQuery(scope, null, withItem.query, with, - withItem.name.getSimple(), false); - registerNamespace(null, alias, - new WithItemNamespace(this, withItem, enclosingNode), - false); - scope = withScope; - } - - registerQuery(scope, null, with.body, enclosingNode, alias, forceNullable, - checkUpdate); - } - - public boolean isAggregate(SqlSelect select) { - if (getAggregate(select) != null) { - return true; - } - // Also when nested window aggregates are present - for (SqlCall call : overFinder.findAll(select.getSelectList())) { - assert call.getKind() == SqlKind.OVER; - if (isNestedAggregateWindow(call.operand(0))) { - return true; - } - if (isOverAggregateWindow(call.operand(1))) { - return true; - } - } - return false; - } - - protected boolean isNestedAggregateWindow(SqlNode node) { - AggFinder nestedAggFinder = - new AggFinder(opTab, false, false, false, aggFinder, - catalogReader.nameMatcher()); - return nestedAggFinder.findAgg(node) != null; - } - - protected boolean isOverAggregateWindow(SqlNode node) { - return aggFinder.findAgg(node) != null; - } - - /** Returns the parse tree node (GROUP BY, HAVING, or an aggregate function - * call) that causes {@code select} to be an aggregate query, or null if it - * is not an aggregate query. - * - *

    The node is useful context for error messages, - * but you cannot assume that the node is the only aggregate function. */ - protected SqlNode getAggregate(SqlSelect select) { - SqlNode node = select.getGroup(); - if (node != null) { - return node; - } - node = select.getHaving(); - if (node != null) { - return node; - } - return getAgg(select); - } - - /** If there is at least one call to an aggregate function, returns the - * first. */ - private SqlNode getAgg(SqlSelect select) { - final SelectScope selectScope = getRawSelectScope(select); - if (selectScope != null) { - final List selectList = selectScope.getExpandedSelectList(); - if (selectList != null) { - return aggFinder.findAgg(selectList); - } - } - return aggFinder.findAgg(select.getSelectList()); - } - - @SuppressWarnings("deprecation") - public boolean isAggregate(SqlNode selectNode) { - return aggFinder.findAgg(selectNode) != null; - } - - private void validateNodeFeature(SqlNode node) { - switch (node.getKind()) { - case MULTISET_VALUE_CONSTRUCTOR: - validateFeature(RESOURCE.sQLFeature_S271(), node.getParserPosition()); - break; - } - } - - private void registerSubQueries( - SqlValidatorScope parentScope, - SqlNode node) { - if (node == null) { - return; - } - if (node.getKind().belongsTo(SqlKind.QUERY) - || node.getKind() == SqlKind.MULTISET_QUERY_CONSTRUCTOR - || node.getKind() == SqlKind.MULTISET_VALUE_CONSTRUCTOR) { - registerQuery(parentScope, null, node, node, null, false); - } else if (node instanceof SqlCall) { - validateNodeFeature(node); - SqlCall call = (SqlCall) node; - for (int i = 0; i < call.operandCount(); i++) { - registerOperandSubQueries(parentScope, call, i); - } - } else if (node instanceof SqlNodeList) { - SqlNodeList list = (SqlNodeList) node; - for (int i = 0, count = list.size(); i < count; i++) { - SqlNode listNode = list.get(i); - if (listNode.getKind().belongsTo(SqlKind.QUERY)) { - listNode = - SqlStdOperatorTable.SCALAR_QUERY.createCall( - listNode.getParserPosition(), - listNode); - list.set(i, listNode); - } - registerSubQueries(parentScope, listNode); - } - } else { - // atomic node -- can be ignored - } - } - - /** - * Registers any sub-queries inside a given call operand, and converts the - * operand to a scalar sub-query if the operator requires it. - * - * @param parentScope Parent scope - * @param call Call - * @param operandOrdinal Ordinal of operand within call - * @see SqlOperator#argumentMustBeScalar(int) - */ - private void registerOperandSubQueries( - SqlValidatorScope parentScope, - SqlCall call, - int operandOrdinal) { - SqlNode operand = call.operand(operandOrdinal); - if (operand == null) { - return; - } - if (operand.getKind().belongsTo(SqlKind.QUERY) - && call.getOperator().argumentMustBeScalar(operandOrdinal)) { - operand = - SqlStdOperatorTable.SCALAR_QUERY.createCall( - operand.getParserPosition(), - operand); - call.setOperand(operandOrdinal, operand); - } - registerSubQueries(parentScope, operand); - } - - public void validateIdentifier(SqlIdentifier id, SqlValidatorScope scope) { - final SqlQualified fqId = scope.fullyQualify(id); - if (expandColumnReferences) { - // NOTE jvs 9-Apr-2007: this doesn't cover ORDER BY, which has its - // own ideas about qualification. - id.assignNamesFrom(fqId.identifier); - } else { - Util.discard(fqId); - } - } - - public void validateLiteral(SqlLiteral literal) { - switch (literal.getTypeName()) { - case DECIMAL: - // Decimal and long have the same precision (as 64-bit integers), so - // the unscaled value of a decimal must fit into a long. - - // REVIEW jvs 4-Aug-2004: This should probably be calling over to - // the available calculator implementations to see what they - // support. For now use ESP instead. - // - // jhyde 2006/12/21: I think the limits should be baked into the - // type system, not dependent on the calculator implementation. - BigDecimal bd = (BigDecimal) literal.getValue(); - BigInteger unscaled = bd.unscaledValue(); - long longValue = unscaled.longValue(); - if (!BigInteger.valueOf(longValue).equals(unscaled)) { - // overflow - throw newValidationError(literal, - RESOURCE.numberLiteralOutOfRange(bd.toString())); - } - break; - - case DOUBLE: - validateLiteralAsDouble(literal); - break; - - case BINARY: - final BitString bitString = (BitString) literal.getValue(); - if ((bitString.getBitCount() % 8) != 0) { - throw newValidationError(literal, RESOURCE.binaryLiteralOdd()); - } - break; - - case DATE: - case TIME: - case TIMESTAMP: - Calendar calendar = literal.getValueAs(Calendar.class); - final int year = calendar.get(Calendar.YEAR); - final int era = calendar.get(Calendar.ERA); - if (year < 1 || era == GregorianCalendar.BC || year > 9999) { - throw newValidationError(literal, - RESOURCE.dateLiteralOutOfRange(literal.toString())); - } - break; - - case INTERVAL_YEAR: - case INTERVAL_YEAR_MONTH: - case INTERVAL_MONTH: - case INTERVAL_DAY: - case INTERVAL_DAY_HOUR: - case INTERVAL_DAY_MINUTE: - case INTERVAL_DAY_SECOND: - case INTERVAL_HOUR: - case INTERVAL_HOUR_MINUTE: - case INTERVAL_HOUR_SECOND: - case INTERVAL_MINUTE: - case INTERVAL_MINUTE_SECOND: - case INTERVAL_SECOND: - if (literal instanceof SqlIntervalLiteral) { - SqlIntervalLiteral.IntervalValue interval = - (SqlIntervalLiteral.IntervalValue) - literal.getValue(); - SqlIntervalQualifier intervalQualifier = - interval.getIntervalQualifier(); - - // ensure qualifier is good before attempting to validate literal - validateIntervalQualifier(intervalQualifier); - String intervalStr = interval.getIntervalLiteral(); - // throws CalciteContextException if string is invalid - int[] values = intervalQualifier.evaluateIntervalLiteral(intervalStr, - literal.getParserPosition(), typeFactory.getTypeSystem()); - Util.discard(values); - } - break; - default: - // default is to do nothing - } - } - - private void validateLiteralAsDouble(SqlLiteral literal) { - BigDecimal bd = (BigDecimal) literal.getValue(); - double d = bd.doubleValue(); - if (Double.isInfinite(d) || Double.isNaN(d)) { - // overflow - throw newValidationError(literal, - RESOURCE.numberLiteralOutOfRange(Util.toScientificNotation(bd))); - } - - // REVIEW jvs 4-Aug-2004: what about underflow? - } - - public void validateIntervalQualifier(SqlIntervalQualifier qualifier) { - assert qualifier != null; - boolean startPrecisionOutOfRange = false; - boolean fractionalSecondPrecisionOutOfRange = false; - final RelDataTypeSystem typeSystem = typeFactory.getTypeSystem(); - - final int startPrecision = qualifier.getStartPrecision(typeSystem); - final int fracPrecision = - qualifier.getFractionalSecondPrecision(typeSystem); - final int maxPrecision = typeSystem.getMaxPrecision(qualifier.typeName()); - final int minPrecision = qualifier.typeName().getMinPrecision(); - final int minScale = qualifier.typeName().getMinScale(); - final int maxScale = typeSystem.getMaxScale(qualifier.typeName()); - if (qualifier.isYearMonth()) { - if (startPrecision < minPrecision || startPrecision > maxPrecision) { - startPrecisionOutOfRange = true; - } else { - if (fracPrecision < minScale || fracPrecision > maxScale) { - fractionalSecondPrecisionOutOfRange = true; - } - } - } else { - if (startPrecision < minPrecision || startPrecision > maxPrecision) { - startPrecisionOutOfRange = true; - } else { - if (fracPrecision < minScale || fracPrecision > maxScale) { - fractionalSecondPrecisionOutOfRange = true; - } - } - } - - if (startPrecisionOutOfRange) { - throw newValidationError(qualifier, - RESOURCE.intervalStartPrecisionOutOfRange(startPrecision, - "INTERVAL " + qualifier)); - } else if (fractionalSecondPrecisionOutOfRange) { - throw newValidationError(qualifier, - RESOURCE.intervalFractionalSecondPrecisionOutOfRange( - fracPrecision, - "INTERVAL " + qualifier)); - } - } - - /** - * Validates the FROM clause of a query, or (recursively) a child node of - * the FROM clause: AS, OVER, JOIN, VALUES, or sub-query. - * - * @param node Node in FROM clause, typically a table or derived - * table - * @param targetRowType Desired row type of this expression, or - * {@link #unknownType} if not fussy. Must not be null. - * @param scope Scope - */ - protected void validateFrom( - SqlNode node, - RelDataType targetRowType, - SqlValidatorScope scope) { - Objects.requireNonNull(targetRowType); - switch (node.getKind()) { - case AS: - validateFrom( - ((SqlCall) node).operand(0), - targetRowType, - scope); - break; - case VALUES: - validateValues((SqlCall) node, targetRowType, scope); - break; - case JOIN: - validateJoin((SqlJoin) node, scope); - break; - case OVER: - validateOver((SqlCall) node, scope); - break; - case UNNEST: - validateUnnest((SqlCall) node, scope, targetRowType); - break; - default: - validateQuery(node, scope, targetRowType); - break; - } - - // Validate the namespace representation of the node, just in case the - // validation did not occur implicitly. - getNamespace(node, scope).validate(targetRowType); - } - - protected void validateOver(SqlCall call, SqlValidatorScope scope) { - throw new AssertionError("OVER unexpected in this context"); - } - - protected void validateUnnest(SqlCall call, SqlValidatorScope scope, RelDataType targetRowType) { - for (int i = 0; i < call.operandCount(); i++) { - SqlNode expandedItem = expand(call.operand(i), scope); - call.setOperand(i, expandedItem); - } - validateQuery(call, scope, targetRowType); - } - - private void checkRollUpInUsing(SqlIdentifier identifier, - SqlNode leftOrRight, SqlValidatorScope scope) { - SqlValidatorNamespace namespace = getNamespace(leftOrRight, scope); - if (namespace != null) { - SqlValidatorTable sqlValidatorTable = namespace.getTable(); - if (sqlValidatorTable != null) { - Table table = sqlValidatorTable.unwrap(Table.class); - String column = Util.last(identifier.names); - - if (table.isRolledUp(column)) { - throw newValidationError(identifier, - RESOURCE.rolledUpNotAllowed(column, "USING")); - } - } - } - } - - protected void validateJoin(SqlJoin join, SqlValidatorScope scope) { - SqlNode left = join.getLeft(); - SqlNode right = join.getRight(); - SqlNode condition = join.getCondition(); - boolean natural = join.isNatural(); - final JoinType joinType = join.getJoinType(); - final JoinConditionType conditionType = join.getConditionType(); - final SqlValidatorScope joinScope = scopes.get(join); - validateFrom(left, unknownType, joinScope); - validateFrom(right, unknownType, joinScope); - - // Validate condition. - switch (conditionType) { - case NONE: - Preconditions.checkArgument(condition == null); - break; - case ON: - Preconditions.checkArgument(condition != null); - SqlNode expandedCondition = expand(condition, joinScope); - join.setOperand(5, expandedCondition); - condition = join.getCondition(); - validateWhereOrOn(joinScope, condition, "ON"); - checkRollUp(null, join, condition, joinScope, "ON"); - break; - case USING: - SqlNodeList list = (SqlNodeList) condition; - - // Parser ensures that using clause is not empty. - Preconditions.checkArgument(list.size() > 0, "Empty USING clause"); - for (SqlNode node : list) { - SqlIdentifier id = (SqlIdentifier) node; - final RelDataType leftColType = validateUsingCol(id, left); - final RelDataType rightColType = validateUsingCol(id, right); - if (!SqlTypeUtil.isComparable(leftColType, rightColType)) { - throw newValidationError(id, - RESOURCE.naturalOrUsingColumnNotCompatible(id.getSimple(), - leftColType.toString(), rightColType.toString())); - } - checkRollUpInUsing(id, left, scope); - checkRollUpInUsing(id, right, scope); - } - break; - default: - throw Util.unexpected(conditionType); - } - - // Validate NATURAL. - if (natural) { - if (condition != null) { - throw newValidationError(condition, - RESOURCE.naturalDisallowsOnOrUsing()); - } - - // Join on fields that occur exactly once on each side. Ignore - // fields that occur more than once on either side. - final RelDataType leftRowType = getNamespace(left).getRowType(); - final RelDataType rightRowType = getNamespace(right).getRowType(); - final SqlNameMatcher nameMatcher = catalogReader.nameMatcher(); - List naturalColumnNames = - SqlValidatorUtil.deriveNaturalJoinColumnList(nameMatcher, - leftRowType, rightRowType); - - // Check compatibility of the chosen columns. - for (String name : naturalColumnNames) { - final RelDataType leftColType = - nameMatcher.field(leftRowType, name).getType(); - final RelDataType rightColType = - nameMatcher.field(rightRowType, name).getType(); - if (!SqlTypeUtil.isComparable(leftColType, rightColType)) { - throw newValidationError(join, - RESOURCE.naturalOrUsingColumnNotCompatible(name, - leftColType.toString(), rightColType.toString())); - } - } - } - - // Which join types require/allow a ON/USING condition, or allow - // a NATURAL keyword? - switch (joinType) { - case LEFT_SEMI_JOIN: - if (!conformance.isLiberal()) { - throw newValidationError(join.getJoinTypeNode(), - RESOURCE.dialectDoesNotSupportFeature("LEFT SEMI JOIN")); - } - // fall through - case INNER: - case LEFT: - case RIGHT: - case FULL: - if ((condition == null) && !natural) { - throw newValidationError(join, RESOURCE.joinRequiresCondition()); - } - break; - case COMMA: - case CROSS: - if (condition != null) { - throw newValidationError(join.getConditionTypeNode(), - RESOURCE.crossJoinDisallowsCondition()); - } - if (natural) { - throw newValidationError(join.getConditionTypeNode(), - RESOURCE.crossJoinDisallowsCondition()); - } - break; - default: - throw Util.unexpected(joinType); - } - } - - /** - * Throws an error if there is an aggregate or windowed aggregate in the - * given clause. - * - * @param aggFinder Finder for the particular kind(s) of aggregate function - * @param node Parse tree - * @param clause Name of clause: "WHERE", "GROUP BY", "ON" - */ - private void validateNoAggs(AggFinder aggFinder, SqlNode node, - String clause) { - final SqlCall agg = aggFinder.findAgg(node); - if (agg == null) { - return; - } - final SqlOperator op = agg.getOperator(); - if (op == SqlStdOperatorTable.OVER) { - throw newValidationError(agg, - RESOURCE.windowedAggregateIllegalInClause(clause)); - } else if (op.isGroup() || op.isGroupAuxiliary()) { - throw newValidationError(agg, - RESOURCE.groupFunctionMustAppearInGroupByClause(op.getName())); - } else { - throw newValidationError(agg, - RESOURCE.aggregateIllegalInClause(clause)); - } - } - - private RelDataType validateUsingCol(SqlIdentifier id, SqlNode leftOrRight) { - if (id.names.size() == 1) { - String name = id.names.get(0); - final SqlValidatorNamespace namespace = getNamespace(leftOrRight); - final RelDataType rowType = namespace.getRowType(); - final SqlNameMatcher nameMatcher = catalogReader.nameMatcher(); - final RelDataTypeField field = nameMatcher.field(rowType, name); - if (field != null) { - if (nameMatcher.frequency(rowType.getFieldNames(), name) > 1) { - throw newValidationError(id, - RESOURCE.columnInUsingNotUnique(id.toString())); - } - return field.getType(); - } - } - throw newValidationError(id, RESOURCE.columnNotFound(id.toString())); - } - - /** - * Validates a SELECT statement. - * - * @param select Select statement - * @param targetRowType Desired row type, must not be null, may be the data - * type 'unknown'. - */ - protected void validateSelect( - SqlSelect select, - RelDataType targetRowType) { - assert targetRowType != null; - // Namespace is either a select namespace or a wrapper around one. - final SelectNamespace ns = - getNamespace(select).unwrap(SelectNamespace.class); - - // Its rowtype is null, meaning it hasn't been validated yet. - // This is important, because we need to take the targetRowType into - // account. - assert ns.rowType == null; - - if (select.isDistinct()) { - validateFeature(RESOURCE.sQLFeature_E051_01(), - select.getModifierNode(SqlSelectKeyword.DISTINCT) - .getParserPosition()); - } - - final SqlNodeList selectItems = select.getSelectList(); - RelDataType fromType = unknownType; - if (selectItems.size() == 1) { - final SqlNode selectItem = selectItems.get(0); - if (selectItem instanceof SqlIdentifier) { - SqlIdentifier id = (SqlIdentifier) selectItem; - if (id.isStar() && (id.names.size() == 1)) { - // Special case: for INSERT ... VALUES(?,?), the SQL - // standard says we're supposed to propagate the target - // types down. So iff the select list is an unqualified - // star (as it will be after an INSERT ... VALUES has been - // expanded), then propagate. - fromType = targetRowType; - } - } - } - - // Make sure that items in FROM clause have distinct aliases. - final SelectScope fromScope = (SelectScope) getFromScope(select); - List names = fromScope.getChildNames(); - if (!catalogReader.nameMatcher().isCaseSensitive()) { - names = Lists.transform(names, s -> s.toUpperCase(Locale.ROOT)); - } - final int duplicateAliasOrdinal = Util.firstDuplicate(names); - if (duplicateAliasOrdinal >= 0) { - final ScopeChild child = - fromScope.children.get(duplicateAliasOrdinal); - throw newValidationError(child.namespace.getEnclosingNode(), - RESOURCE.fromAliasDuplicate(child.name)); - } - - if (select.getFrom() == null) { - if (conformance.isFromRequired()) { - throw newValidationError(select, RESOURCE.selectMissingFrom()); - } - } else { - validateFrom(select.getFrom(), fromType, fromScope); - } - - validateWhereClause(select); - validateGroupClause(select); - validateHavingClause(select); - validateWindowClause(select); - handleOffsetFetch(select.getOffset(), select.getFetch()); - - // Validate the SELECT clause late, because a select item might - // depend on the GROUP BY list, or the window function might reference - // window name in the WINDOW clause etc. - final RelDataType rowType = - validateSelectList(selectItems, select, targetRowType); - ns.setType(rowType); - - // Validate ORDER BY after we have set ns.rowType because in some - // dialects you can refer to columns of the select list, e.g. - // "SELECT empno AS x FROM emp ORDER BY x" - validateOrderList(select); - - if (shouldCheckForRollUp(select.getFrom())) { - checkRollUpInSelectList(select); - checkRollUp(null, select, select.getWhere(), getWhereScope(select)); - checkRollUp(null, select, select.getHaving(), getHavingScope(select)); - checkRollUpInWindowDecl(select); - checkRollUpInGroupBy(select); - checkRollUpInOrderBy(select); - } - } - - private void checkRollUpInSelectList(SqlSelect select) { - SqlValidatorScope scope = getSelectScope(select); - for (SqlNode item : select.getSelectList()) { - checkRollUp(null, select, item, scope); - } - } - - private void checkRollUpInGroupBy(SqlSelect select) { - SqlNodeList group = select.getGroup(); - if (group != null) { - for (SqlNode node : group) { - checkRollUp(null, select, node, getGroupScope(select), "GROUP BY"); - } - } - } - - private void checkRollUpInOrderBy(SqlSelect select) { - SqlNodeList orderList = select.getOrderList(); - if (orderList != null) { - for (SqlNode node : orderList) { - checkRollUp(null, select, node, getOrderScope(select), "ORDER BY"); - } - } - } - - private void checkRollUpInWindow(SqlWindow window, SqlValidatorScope scope) { - if (window != null) { - for (SqlNode node : window.getPartitionList()) { - checkRollUp(null, window, node, scope, "PARTITION BY"); - } - - for (SqlNode node : window.getOrderList()) { - checkRollUp(null, window, node, scope, "ORDER BY"); - } - } - } - - private void checkRollUpInWindowDecl(SqlSelect select) { - for (SqlNode decl : select.getWindowList()) { - checkRollUpInWindow((SqlWindow) decl, getSelectScope(select)); - } - } - - private SqlNode stripDot(SqlNode node) { - if (node != null && node.getKind() == SqlKind.DOT) { - return stripDot(((SqlCall) node).operand(0)); - } - return node; - } - - private void checkRollUp(SqlNode grandParent, SqlNode parent, - SqlNode current, SqlValidatorScope scope, String optionalClause) { - current = stripAs(current); - if (current instanceof SqlCall && !(current instanceof SqlSelect)) { - // Validate OVER separately - checkRollUpInWindow(getWindowInOver(current), scope); - current = stripOver(current); - - List children = ((SqlCall) stripDot(current)).getOperandList(); - for (SqlNode child : children) { - checkRollUp(parent, current, child, scope, optionalClause); - } - } else if (current instanceof SqlIdentifier) { - SqlIdentifier id = (SqlIdentifier) current; - if (!id.isStar() && isRolledUpColumn(id, scope)) { - if (!isAggregation(parent.getKind()) - || !isRolledUpColumnAllowedInAgg(id, scope, (SqlCall) parent, grandParent)) { - String context = optionalClause != null ? optionalClause : parent.getKind().toString(); - throw newValidationError(id, - RESOURCE.rolledUpNotAllowed(deriveAlias(id, 0), context)); - } - } - } - } - - private void checkRollUp(SqlNode grandParent, SqlNode parent, - SqlNode current, SqlValidatorScope scope) { - checkRollUp(grandParent, parent, current, scope, null); - } - - private SqlWindow getWindowInOver(SqlNode over) { - if (over.getKind() == SqlKind.OVER) { - SqlNode window = ((SqlCall) over).getOperandList().get(1); - if (window instanceof SqlWindow) { - return (SqlWindow) window; - } - // SqlIdentifier, gets validated elsewhere - return null; - } - return null; - } - - private static SqlNode stripOver(SqlNode node) { - switch (node.getKind()) { - case OVER: - return ((SqlCall) node).getOperandList().get(0); - default: - return node; - } - } - - private Pair findTableColumnPair(SqlIdentifier identifier, - SqlValidatorScope scope) { - final SqlCall call = makeNullaryCall(identifier); - if (call != null) { - return null; - } - SqlQualified qualified = scope.fullyQualify(identifier); - List names = qualified.identifier.names; - - if (names.size() < 2) { - return null; - } - - return new Pair<>(names.get(names.size() - 2), Util.last(names)); - } - - // Returns true iff the given column is valid inside the given aggCall. - private boolean isRolledUpColumnAllowedInAgg(SqlIdentifier identifier, SqlValidatorScope scope, - SqlCall aggCall, SqlNode parent) { - Pair pair = findTableColumnPair(identifier, scope); - - if (pair == null) { - return true; - } - - String columnName = pair.right; - - SqlValidatorTable sqlValidatorTable = - scope.fullyQualify(identifier).namespace.getTable(); - if (sqlValidatorTable != null) { - Table table = sqlValidatorTable.unwrap(Table.class); - return table.rolledUpColumnValidInsideAgg(columnName, aggCall, parent, - catalogReader.getConfig()); - } - return true; - } - - - // Returns true iff the given column is actually rolled up. - private boolean isRolledUpColumn(SqlIdentifier identifier, SqlValidatorScope scope) { - Pair pair = findTableColumnPair(identifier, scope); - - if (pair == null) { - return false; - } - - String columnName = pair.right; - - SqlValidatorTable sqlValidatorTable = - scope.fullyQualify(identifier).namespace.getTable(); - if (sqlValidatorTable != null) { - Table table = sqlValidatorTable.unwrap(Table.class); - return table.isRolledUp(columnName); - } - return false; - } - - private boolean shouldCheckForRollUp(SqlNode from) { - if (from != null) { - SqlKind kind = stripAs(from).getKind(); - return kind != SqlKind.VALUES && kind != SqlKind.SELECT; - } - return false; - } - - /** Validates that a query can deliver the modality it promises. Only called - * on the top-most SELECT or set operator in the tree. */ - private void validateModality(SqlNode query) { - final SqlModality modality = deduceModality(query); - if (query instanceof SqlSelect) { - final SqlSelect select = (SqlSelect) query; - validateModality(select, modality, true); - } else if (query.getKind() == SqlKind.VALUES) { - switch (modality) { - case STREAM: - throw newValidationError(query, Static.RESOURCE.cannotStreamValues()); - } - } else { - assert query.isA(SqlKind.SET_QUERY); - final SqlCall call = (SqlCall) query; - for (SqlNode operand : call.getOperandList()) { - if (deduceModality(operand) != modality) { - throw newValidationError(operand, - Static.RESOURCE.streamSetOpInconsistentInputs()); - } - validateModality(operand); - } - } - } - - /** Return the intended modality of a SELECT or set-op. */ - private SqlModality deduceModality(SqlNode query) { - if (query instanceof SqlSelect) { - SqlSelect select = (SqlSelect) query; - return select.getModifierNode(SqlSelectKeyword.STREAM) != null - ? SqlModality.STREAM - : SqlModality.RELATION; - } else if (query.getKind() == SqlKind.VALUES) { - return SqlModality.RELATION; - } else { - assert query.isA(SqlKind.SET_QUERY); - final SqlCall call = (SqlCall) query; - return deduceModality(call.getOperandList().get(0)); - } - } - - public boolean validateModality(SqlSelect select, SqlModality modality, - boolean fail) { - final SelectScope scope = getRawSelectScope(select); - - switch (modality) { - case STREAM: - if (scope.children.size() == 1) { - for (ScopeChild child : scope.children) { - if (!child.namespace.supportsModality(modality)) { - if (fail) { - throw newValidationError(child.namespace.getNode(), - Static.RESOURCE.cannotConvertToStream(child.name)); - } else { - return false; - } - } - } - } else { - int supportsModalityCount = 0; - for (ScopeChild child : scope.children) { - if (child.namespace.supportsModality(modality)) { - ++supportsModalityCount; - } - } - - if (supportsModalityCount == 0) { - if (fail) { - String inputs = String.join(", ", scope.getChildNames()); - throw newValidationError(select, - Static.RESOURCE.cannotStreamResultsForNonStreamingInputs(inputs)); - } else { - return false; - } - } - } - break; - default: - for (ScopeChild child : scope.children) { - if (!child.namespace.supportsModality(modality)) { - if (fail) { - throw newValidationError(child.namespace.getNode(), - Static.RESOURCE.cannotConvertToRelation(child.name)); - } else { - return false; - } - } - } - } - - // Make sure that aggregation is possible. - final SqlNode aggregateNode = getAggregate(select); - if (aggregateNode != null) { - switch (modality) { - case STREAM: - SqlNodeList groupList = select.getGroup(); - if (groupList == null - || !SqlValidatorUtil.containsMonotonic(scope, groupList)) { - if (fail) { - throw newValidationError(aggregateNode, - Static.RESOURCE.streamMustGroupByMonotonic()); - } else { - return false; - } - } - } - } - - // Make sure that ORDER BY is possible. - final SqlNodeList orderList = select.getOrderList(); - if (orderList != null && orderList.size() > 0) { - switch (modality) { - case STREAM: - if (!hasSortedPrefix(scope, orderList)) { - if (fail) { - throw newValidationError(orderList.get(0), - Static.RESOURCE.streamMustOrderByMonotonic()); - } else { - return false; - } - } - } - } - return true; - } - - /** Returns whether the prefix is sorted. */ - private boolean hasSortedPrefix(SelectScope scope, SqlNodeList orderList) { - return isSortCompatible(scope, orderList.get(0), false); - } - - private boolean isSortCompatible(SelectScope scope, SqlNode node, - boolean descending) { - switch (node.getKind()) { - case DESCENDING: - return isSortCompatible(scope, ((SqlCall) node).getOperandList().get(0), - true); - } - final SqlMonotonicity monotonicity = scope.getMonotonicity(node); - switch (monotonicity) { - case INCREASING: - case STRICTLY_INCREASING: - return !descending; - case DECREASING: - case STRICTLY_DECREASING: - return descending; - default: - return false; - } - } - - protected void validateWindowClause(SqlSelect select) { - final SqlNodeList windowList = select.getWindowList(); - @SuppressWarnings("unchecked") final List windows = - (List) windowList.getList(); - if (windows.isEmpty()) { - return; - } - - final SelectScope windowScope = (SelectScope) getFromScope(select); - assert windowScope != null; - - // 1. ensure window names are simple - // 2. ensure they are unique within this scope - for (SqlWindow window : windows) { - SqlIdentifier declName = window.getDeclName(); - if (!declName.isSimple()) { - throw newValidationError(declName, RESOURCE.windowNameMustBeSimple()); - } - - if (windowScope.existingWindowName(declName.toString())) { - throw newValidationError(declName, RESOURCE.duplicateWindowName()); - } else { - windowScope.addWindowName(declName.toString()); - } - } - - // 7.10 rule 2 - // Check for pairs of windows which are equivalent. - for (int i = 0; i < windows.size(); i++) { - SqlNode window1 = windows.get(i); - for (int j = i + 1; j < windows.size(); j++) { - SqlNode window2 = windows.get(j); - if (window1.equalsDeep(window2, Litmus.IGNORE)) { - throw newValidationError(window2, RESOURCE.dupWindowSpec()); - } - } - } - - for (SqlWindow window : windows) { - final SqlNodeList expandedOrderList = - (SqlNodeList) expand(window.getOrderList(), windowScope); - window.setOrderList(expandedOrderList); - expandedOrderList.validate(this, windowScope); - - final SqlNodeList expandedPartitionList = - (SqlNodeList) expand(window.getPartitionList(), windowScope); - window.setPartitionList(expandedPartitionList); - expandedPartitionList.validate(this, windowScope); - } - - // Hand off to validate window spec components - windowList.validate(this, windowScope); - } - - public void validateWith(SqlWith with, SqlValidatorScope scope) { - final SqlValidatorNamespace namespace = getNamespace(with); - validateNamespace(namespace, unknownType); - } - - public void validateWithItem(SqlWithItem withItem) { - if (withItem.columnList != null) { - final RelDataType rowType = getValidatedNodeType(withItem.query); - final int fieldCount = rowType.getFieldCount(); - if (withItem.columnList.size() != fieldCount) { - throw newValidationError(withItem.columnList, - RESOURCE.columnCountMismatch()); - } - SqlValidatorUtil.checkIdentifierListForDuplicates( - withItem.columnList.getList(), validationErrorFunction); - } else { - // Luckily, field names have not been make unique yet. - final List fieldNames = - getValidatedNodeType(withItem.query).getFieldNames(); - final int i = Util.firstDuplicate(fieldNames); - if (i >= 0) { - throw newValidationError(withItem.query, - RESOURCE.duplicateColumnAndNoColumnList(fieldNames.get(i))); - } - } - } - - public void validateSequenceValue(SqlValidatorScope scope, SqlIdentifier id) { - // Resolve identifier as a table. - final SqlValidatorScope.ResolvedImpl resolved = - new SqlValidatorScope.ResolvedImpl(); - scope.resolveTable(id.names, catalogReader.nameMatcher(), - SqlValidatorScope.Path.EMPTY, resolved); - if (resolved.count() != 1) { - throw newValidationError(id, RESOURCE.tableNameNotFound(id.toString())); - } - // We've found a table. But is it a sequence? - final SqlValidatorNamespace ns = resolved.only().namespace; - if (ns instanceof TableNamespace) { - final Table table = ns.getTable().unwrap(Table.class); - switch (table.getJdbcTableType()) { - case SEQUENCE: - case TEMPORARY_SEQUENCE: - return; - } - } - throw newValidationError(id, RESOURCE.notASequence(id.toString())); - } - - public SqlValidatorScope getWithScope(SqlNode withItem) { - assert withItem.getKind() == SqlKind.WITH_ITEM; - return scopes.get(withItem); - } - - @Override - public SqlValidator setEnableTypeCoercion(boolean enabled) { - this.enableTypeCoercion = enabled; - return this; - } - - @Override - public boolean isTypeCoercionEnabled() { - return this.enableTypeCoercion; - } - - @Override - public void setTypeCoercion(TypeCoercion typeCoercion) { - Objects.requireNonNull(typeCoercion); - this.typeCoercion = typeCoercion; - } - - @Override - public TypeCoercion getTypeCoercion() { - assert isTypeCoercionEnabled(); - return this.typeCoercion; - } - - /** - * Validates the ORDER BY clause of a SELECT statement. - * - * @param select Select statement - */ - protected void validateOrderList(SqlSelect select) { - // ORDER BY is validated in a scope where aliases in the SELECT clause - // are visible. For example, "SELECT empno AS x FROM emp ORDER BY x" - // is valid. - SqlNodeList orderList = select.getOrderList(); - if (orderList == null) { - return; - } - if (!shouldAllowIntermediateOrderBy()) { - if (!cursorSet.contains(select)) { - throw newValidationError(select, RESOURCE.invalidOrderByPos()); - } - } - final SqlValidatorScope orderScope = getOrderScope(select); - Objects.requireNonNull(orderScope); - - List expandList = new ArrayList<>(); - for (SqlNode orderItem : orderList) { - SqlNode expandedOrderItem = expand(orderItem, orderScope); - expandList.add(expandedOrderItem); - } - - SqlNodeList expandedOrderList = new SqlNodeList( - expandList, - orderList.getParserPosition()); - select.setOrderBy(expandedOrderList); - - for (SqlNode orderItem : expandedOrderList) { - validateOrderItem(select, orderItem); - } - } - - /** - * Validates an item in the GROUP BY clause of a SELECT statement. - * - * @param select Select statement - * @param groupByItem GROUP BY clause item - */ - private void validateGroupByItem(SqlSelect select, SqlNode groupByItem) { - final SqlValidatorScope groupByScope = getGroupScope(select); - groupByScope.validateExpr(groupByItem); - } - - /** - * Validates an item in the ORDER BY clause of a SELECT statement. - * - * @param select Select statement - * @param orderItem ORDER BY clause item - */ - private void validateOrderItem(SqlSelect select, SqlNode orderItem) { - switch (orderItem.getKind()) { - case DESCENDING: - validateFeature(RESOURCE.sQLConformance_OrderByDesc(), - orderItem.getParserPosition()); - validateOrderItem(select, - ((SqlCall) orderItem).operand(0)); - return; - } - - final SqlValidatorScope orderScope = getOrderScope(select); - validateExpr(orderItem, orderScope); - } - - public SqlNode expandOrderExpr(SqlSelect select, SqlNode orderExpr) { - final SqlNode newSqlNode = - new OrderExpressionExpander(select, orderExpr).go(); - if (newSqlNode != orderExpr) { - final SqlValidatorScope scope = getOrderScope(select); - inferUnknownTypes(unknownType, scope, newSqlNode); - final RelDataType type = deriveType(scope, newSqlNode); - setValidatedNodeType(newSqlNode, type); - } - return newSqlNode; - } - - /** - * Validates the GROUP BY clause of a SELECT statement. This method is - * called even if no GROUP BY clause is present. - */ - protected void validateGroupClause(SqlSelect select) { - SqlNodeList groupList = select.getGroup(); - if (groupList == null) { - return; - } - final String clause = "GROUP BY"; - validateNoAggs(aggOrOverFinder, groupList, clause); - final SqlValidatorScope groupScope = getGroupScope(select); - inferUnknownTypes(unknownType, groupScope, groupList); - - // expand the expression in group list. - List expandedList = new ArrayList<>(); - for (SqlNode groupItem : groupList) { - SqlNode expandedItem = expandGroupByOrHavingExpr(groupItem, groupScope, select, false); - expandedList.add(expandedItem); - } - groupList = new SqlNodeList(expandedList, groupList.getParserPosition()); - select.setGroupBy(groupList); - for (SqlNode groupItem : expandedList) { - validateGroupByItem(select, groupItem); - } - - // Nodes in the GROUP BY clause are expressions except if they are calls - // to the GROUPING SETS, ROLLUP or CUBE operators; this operators are not - // expressions, because they do not have a type. - for (SqlNode node : groupList) { - switch (node.getKind()) { - case GROUPING_SETS: - case ROLLUP: - case CUBE: - node.validate(this, groupScope); - break; - default: - node.validateExpr(this, groupScope); - } - } - - // Derive the type of each GROUP BY item. We don't need the type, but - // it resolves functions, and that is necessary for deducing - // monotonicity. - final SqlValidatorScope selectScope = getSelectScope(select); - AggregatingSelectScope aggregatingScope = null; - if (selectScope instanceof AggregatingSelectScope) { - aggregatingScope = (AggregatingSelectScope) selectScope; - } - for (SqlNode groupItem : groupList) { - if (groupItem instanceof SqlNodeList - && ((SqlNodeList) groupItem).size() == 0) { - continue; - } - validateGroupItem(groupScope, aggregatingScope, groupItem); - } - - SqlNode agg = aggFinder.findAgg(groupList); - if (agg != null) { - throw newValidationError(agg, RESOURCE.aggregateIllegalInClause(clause)); - } - } - - private void validateGroupItem(SqlValidatorScope groupScope, - AggregatingSelectScope aggregatingScope, - SqlNode groupItem) { - switch (groupItem.getKind()) { - case GROUPING_SETS: - case ROLLUP: - case CUBE: - validateGroupingSets(groupScope, aggregatingScope, (SqlCall) groupItem); - break; - default: - if (groupItem instanceof SqlNodeList) { - break; - } - final RelDataType type = deriveType(groupScope, groupItem); - setValidatedNodeType(groupItem, type); - } - } - - private void validateGroupingSets(SqlValidatorScope groupScope, - AggregatingSelectScope aggregatingScope, SqlCall groupItem) { - for (SqlNode node : groupItem.getOperandList()) { - validateGroupItem(groupScope, aggregatingScope, node); - } - } - - protected void validateWhereClause(SqlSelect select) { - // validate WHERE clause - final SqlNode where = select.getWhere(); - if (where == null) { - return; - } - final SqlValidatorScope whereScope = getWhereScope(select); - final SqlNode expandedWhere = expand(where, whereScope); - select.setWhere(expandedWhere); - validateWhereOrOn(whereScope, expandedWhere, "WHERE"); - } - - protected void validateWhereOrOn( - SqlValidatorScope scope, - SqlNode condition, - String clause) { - validateNoAggs(aggOrOverOrGroupFinder, condition, clause); - inferUnknownTypes( - booleanType, - scope, - condition); - condition.validate(this, scope); - - final RelDataType type = deriveType(scope, condition); - if (!SqlTypeUtil.inBooleanFamily(type)) { - throw newValidationError(condition, RESOURCE.condMustBeBoolean(clause)); - } - } - - protected void validateHavingClause(SqlSelect select) { - // HAVING is validated in the scope after groups have been created. - // For example, in "SELECT empno FROM emp WHERE empno = 10 GROUP BY - // deptno HAVING empno = 10", the reference to 'empno' in the HAVING - // clause is illegal. - SqlNode having = select.getHaving(); - if (having == null) { - return; - } - final AggregatingScope havingScope = - (AggregatingScope) getSelectScope(select); - if (getConformance().isHavingAlias()) { - SqlNode newExpr = expandGroupByOrHavingExpr(having, havingScope, select, true); - if (having != newExpr) { - having = newExpr; - select.setHaving(newExpr); - } - } - havingScope.checkAggregateExpr(having, true); - inferUnknownTypes( - booleanType, - havingScope, - having); - having.validate(this, havingScope); - final RelDataType type = deriveType(havingScope, having); - if (!SqlTypeUtil.inBooleanFamily(type)) { - throw newValidationError(having, RESOURCE.havingMustBeBoolean()); - } - } - - protected RelDataType validateSelectList( - final SqlNodeList selectItems, - SqlSelect select, - RelDataType targetRowType) { - // First pass, ensure that aliases are unique. "*" and "TABLE.*" items - // are ignored. - - // Validate SELECT list. Expand terms of the form "*" or "TABLE.*". - final SqlValidatorScope selectScope = getSelectScope(select); - final List expandedSelectItems = new ArrayList<>(); - final Set aliases = new HashSet<>(); - final List> fieldList = new ArrayList<>(); - - for (int i = 0; i < selectItems.size(); i++) { - SqlNode selectItem = selectItems.get(i); - if (selectItem instanceof SqlSelect) { - handleScalarSubQuery( - select, - (SqlSelect) selectItem, - expandedSelectItems, - aliases, - fieldList); - } else { - expandSelectItem( - selectItem, - select, - targetRowType.isStruct() - && targetRowType.getFieldCount() >= i - ? targetRowType.getFieldList().get(i).getType() - : unknownType, - expandedSelectItems, - aliases, - fieldList, - false); - } - } - - // Create the new select list with expanded items. Pass through - // the original parser position so that any overall failures can - // still reference the original input text. - SqlNodeList newSelectList = - new SqlNodeList( - expandedSelectItems, - selectItems.getParserPosition()); - if (shouldExpandIdentifiers()) { - select.setSelectList(newSelectList); - } - getRawSelectScope(select).setExpandedSelectList(expandedSelectItems); - - // TODO: when SELECT appears as a value sub-query, should be using - // something other than unknownType for targetRowType - inferUnknownTypes(targetRowType, selectScope, newSelectList); - - for (SqlNode selectItem : expandedSelectItems) { - validateNoAggs(groupFinder, selectItem, "SELECT"); - validateExpr(selectItem, selectScope); - } - - return typeFactory.createStructType(fieldList); - } - - /** - * Validates an expression. - * - * @param expr Expression - * @param scope Scope in which expression occurs - */ - private void validateExpr(SqlNode expr, SqlValidatorScope scope) { - if (expr instanceof SqlCall) { - final SqlOperator op = ((SqlCall) expr).getOperator(); - if (op.isAggregator() && op.requiresOver()) { - throw newValidationError(expr, - RESOURCE.absentOverClause()); - } - } - - // Call on the expression to validate itself. - expr.validateExpr(this, scope); - - // Perform any validation specific to the scope. For example, an - // aggregating scope requires that expressions are valid aggregations. - scope.validateExpr(expr); - } - - /** - * Processes SubQuery found in Select list. Checks that is actually Scalar - * sub-query and makes proper entries in each of the 3 lists used to create - * the final rowType entry. - * - * @param parentSelect base SqlSelect item - * @param selectItem child SqlSelect from select list - * @param expandedSelectItems Select items after processing - * @param aliasList built from user or system values - * @param fieldList Built up entries for each select list entry - */ - private void handleScalarSubQuery( - SqlSelect parentSelect, - SqlSelect selectItem, - List expandedSelectItems, - Set aliasList, - List> fieldList) { - // A scalar sub-query only has one output column. - if (1 != selectItem.getSelectList().size()) { - throw newValidationError(selectItem, - RESOURCE.onlyScalarSubQueryAllowed()); - } - - // No expansion in this routine just append to list. - expandedSelectItems.add(selectItem); - - // Get or generate alias and add to list. - final String alias = - deriveAlias( - selectItem, - aliasList.size()); - aliasList.add(alias); - - final SelectScope scope = (SelectScope) getWhereScope(parentSelect); - final RelDataType type = deriveType(scope, selectItem); - setValidatedNodeType(selectItem, type); - - // we do not want to pass on the RelRecordType returned - // by the sub query. Just the type of the single expression - // in the sub-query select list. - assert type instanceof RelRecordType; - RelRecordType rec = (RelRecordType) type; - - RelDataType nodeType = rec.getFieldList().get(0).getType(); - nodeType = typeFactory.createTypeWithNullability(nodeType, true); - fieldList.add(Pair.of(alias, nodeType)); - } - - /** - * Derives a row-type for INSERT and UPDATE operations. - * - * @param table Target table for INSERT/UPDATE - * @param targetColumnList List of target columns, or null if not specified - * @param append Whether to append fields to those in - * baseRowType - * @return Rowtype - */ - protected RelDataType createTargetRowType( - SqlValidatorTable table, - SqlNodeList targetColumnList, - boolean append) { - RelDataType baseRowType = table.getRowType(); - if (targetColumnList == null) { - return baseRowType; - } - List targetFields = baseRowType.getFieldList(); - final List> fields = new ArrayList<>(); - if (append) { - for (RelDataTypeField targetField : targetFields) { - fields.add( - Pair.of(SqlUtil.deriveAliasFromOrdinal(fields.size()), - targetField.getType())); - } - } - final Set assignedFields = new HashSet<>(); - final RelOptTable relOptTable = table instanceof RelOptTable - ? ((RelOptTable) table) : null; - for (SqlNode node : targetColumnList) { - SqlIdentifier id = (SqlIdentifier) node; - RelDataTypeField targetField = - SqlValidatorUtil.getTargetField( - baseRowType, typeFactory, id, catalogReader, relOptTable); - if (targetField == null) { - throw newValidationError(id, - RESOURCE.unknownTargetColumn(id.toString())); - } - if (!assignedFields.add(targetField.getIndex())) { - throw newValidationError(id, - RESOURCE.duplicateTargetColumn(targetField.getName())); - } - fields.add(targetField); - } - return typeFactory.createStructType(fields); - } - - public void validateInsert(SqlInsert insert) { - final SqlValidatorNamespace targetNamespace = getNamespace(insert); - validateNamespace(targetNamespace, unknownType); - final RelOptTable relOptTable = SqlValidatorUtil.getRelOptTable( - targetNamespace, catalogReader.unwrap(Prepare.CatalogReader.class), null, null); - final SqlValidatorTable table = relOptTable == null - ? targetNamespace.getTable() - : relOptTable.unwrap(SqlValidatorTable.class); - - // INSERT has an optional column name list. If present then - // reduce the rowtype to the columns specified. If not present - // then the entire target rowtype is used. - final RelDataType targetRowType = - createTargetRowType( - table, - insert.getTargetColumnList(), - false); - - final SqlNode source = insert.getSource(); - if (source instanceof SqlSelect) { - final SqlSelect sqlSelect = (SqlSelect) source; - validateSelect(sqlSelect, targetRowType); - } else { - final SqlValidatorScope scope = scopes.get(source); - validateQuery(source, scope, targetRowType); - } - - // REVIEW jvs 4-Dec-2008: In FRG-365, this namespace row type is - // discarding the type inferred by inferUnknownTypes (which was invoked - // from validateSelect above). It would be better if that information - // were used here so that we never saw any untyped nulls during - // checkTypeAssignment. - final RelDataType sourceRowType = getNamespace(source).getRowType(); - final RelDataType logicalTargetRowType = - getLogicalTargetRowType(targetRowType, insert); - setValidatedNodeType(insert, logicalTargetRowType); - final RelDataType logicalSourceRowType = - getLogicalSourceRowType(sourceRowType, insert); - - checkFieldCount(insert.getTargetTable(), table, source, - logicalSourceRowType, logicalTargetRowType); - - checkTypeAssignment(logicalSourceRowType, logicalTargetRowType, insert); - - checkConstraint(table, source, logicalTargetRowType); - - validateAccess(insert.getTargetTable(), table, SqlAccessEnum.INSERT); - } - - /** - * Validates insert values against the constraint of a modifiable view. - * - * @param validatorTable Table that may wrap a ModifiableViewTable - * @param source The values being inserted - * @param targetRowType The target type for the view - */ - private void checkConstraint( - SqlValidatorTable validatorTable, - SqlNode source, - RelDataType targetRowType) { - final ModifiableViewTable modifiableViewTable = - validatorTable.unwrap(ModifiableViewTable.class); - if (modifiableViewTable != null && source instanceof SqlCall) { - final Table table = modifiableViewTable.unwrap(Table.class); - final RelDataType tableRowType = table.getRowType(typeFactory); - final List tableFields = tableRowType.getFieldList(); - - // Get the mapping from column indexes of the underlying table - // to the target columns and view constraints. - final Map tableIndexToTargetField = - SqlValidatorUtil.getIndexToFieldMap(tableFields, targetRowType); - final Map projectMap = - RelOptUtil.getColumnConstraints(modifiableViewTable, targetRowType, typeFactory); - - // Determine columns (indexed to the underlying table) that need - // to be validated against the view constraint. - final ImmutableBitSet targetColumns = - ImmutableBitSet.of(tableIndexToTargetField.keySet()); - final ImmutableBitSet constrainedColumns = - ImmutableBitSet.of(projectMap.keySet()); - final ImmutableBitSet constrainedTargetColumns = - targetColumns.intersect(constrainedColumns); - - // Validate insert values against the view constraint. - final List values = ((SqlCall) source).getOperandList(); - for (final int colIndex : constrainedTargetColumns.asList()) { - final String colName = tableFields.get(colIndex).getName(); - final RelDataTypeField targetField = tableIndexToTargetField.get(colIndex); - for (SqlNode row : values) { - final SqlCall call = (SqlCall) row; - final SqlNode sourceValue = call.operand(targetField.getIndex()); - final ValidationError validationError = - new ValidationError(sourceValue, - RESOURCE.viewConstraintNotSatisfied(colName, - Util.last(validatorTable.getQualifiedName()))); - RelOptUtil.validateValueAgainstConstraint(sourceValue, - projectMap.get(colIndex), validationError); - } - } - } - } - - /** - * Validates updates against the constraint of a modifiable view. - * - * @param validatorTable A {@link SqlValidatorTable} that may wrap a - * ModifiableViewTable - * @param update The UPDATE parse tree node - * @param targetRowType The target type - */ - private void checkConstraint( - SqlValidatorTable validatorTable, - SqlUpdate update, - RelDataType targetRowType) { - final ModifiableViewTable modifiableViewTable = - validatorTable.unwrap(ModifiableViewTable.class); - if (modifiableViewTable != null) { - final Table table = modifiableViewTable.unwrap(Table.class); - final RelDataType tableRowType = table.getRowType(typeFactory); - - final Map projectMap = - RelOptUtil.getColumnConstraints(modifiableViewTable, targetRowType, - typeFactory); - final Map nameToIndex = - SqlValidatorUtil.mapNameToIndex(tableRowType.getFieldList()); - - // Validate update values against the view constraint. - final List targets = update.getTargetColumnList().getList(); - final List sources = update.getSourceExpressionList().getList(); - for (final Pair column : Pair.zip(targets, sources)) { - final String columnName = ((SqlIdentifier) column.left).getSimple(); - final Integer columnIndex = nameToIndex.get(columnName); - if (projectMap.containsKey(columnIndex)) { - final RexNode columnConstraint = projectMap.get(columnIndex); - final ValidationError validationError = - new ValidationError(column.right, - RESOURCE.viewConstraintNotSatisfied(columnName, - Util.last(validatorTable.getQualifiedName()))); - RelOptUtil.validateValueAgainstConstraint(column.right, - columnConstraint, validationError); - } - } - } - } - - private void checkFieldCount(SqlNode node, SqlValidatorTable table, - SqlNode source, RelDataType logicalSourceRowType, - RelDataType logicalTargetRowType) { - final int sourceFieldCount = logicalSourceRowType.getFieldCount(); - final int targetFieldCount = logicalTargetRowType.getFieldCount(); - if (sourceFieldCount != targetFieldCount) { - throw newValidationError(node, - RESOURCE.unmatchInsertColumn(targetFieldCount, sourceFieldCount)); - } - // Ensure that non-nullable fields are targeted. - final InitializerContext rexBuilder = - new InitializerContext() { - public RexBuilder getRexBuilder() { - return new RexBuilder(typeFactory); - } - - public RexNode convertExpression(SqlNode e) { - throw new UnsupportedOperationException(); - } - }; - final List strategies = - table.unwrap(RelOptTable.class).getColumnStrategies(); - for (final RelDataTypeField field : table.getRowType().getFieldList()) { - final RelDataTypeField targetField = - logicalTargetRowType.getField(field.getName(), true, false); - switch (strategies.get(field.getIndex())) { - case NOT_NULLABLE: - assert !field.getType().isNullable(); - if (targetField == null) { - throw newValidationError(node, - RESOURCE.columnNotNullable(field.getName())); - } - break; - case NULLABLE: - assert field.getType().isNullable(); - break; - case VIRTUAL: - case STORED: - if (targetField != null - && !isValuesWithDefault(source, targetField.getIndex())) { - throw newValidationError(node, - RESOURCE.insertIntoAlwaysGenerated(field.getName())); - } - } - } - } - - /** Returns whether a query uses {@code DEFAULT} to populate a given - * column. */ - private boolean isValuesWithDefault(SqlNode source, int column) { - switch (source.getKind()) { - case VALUES: - for (SqlNode operand : ((SqlCall) source).getOperandList()) { - if (!isRowWithDefault(operand, column)) { - return false; - } - } - return true; - } - return false; - } - - private boolean isRowWithDefault(SqlNode operand, int column) { - switch (operand.getKind()) { - case ROW: - final SqlCall row = (SqlCall) operand; - return row.getOperandList().size() >= column - && row.getOperandList().get(column).getKind() == SqlKind.DEFAULT; - } - return false; - } - - protected RelDataType getLogicalTargetRowType( - RelDataType targetRowType, - SqlInsert insert) { - if (insert.getTargetColumnList() == null - && conformance.isInsertSubsetColumnsAllowed()) { - // Target an implicit subset of columns. - final SqlNode source = insert.getSource(); - final RelDataType sourceRowType = getNamespace(source).getRowType(); - final RelDataType logicalSourceRowType = - getLogicalSourceRowType(sourceRowType, insert); - final RelDataType implicitTargetRowType = - typeFactory.createStructType( - targetRowType.getFieldList() - .subList(0, logicalSourceRowType.getFieldCount())); - final SqlValidatorNamespace targetNamespace = getNamespace(insert); - validateNamespace(targetNamespace, implicitTargetRowType); - return implicitTargetRowType; - } else { - // Either the set of columns are explicitly targeted, or target the full - // set of columns. - return targetRowType; - } - } - - protected RelDataType getLogicalSourceRowType( - RelDataType sourceRowType, - SqlInsert insert) { - return sourceRowType; - } - - protected void checkTypeAssignment( - RelDataType sourceRowType, - RelDataType targetRowType, - final SqlNode query) { - // NOTE jvs 23-Feb-2006: subclasses may allow for extra targets - // representing system-maintained columns, so stop after all sources - // matched - List sourceFields = sourceRowType.getFieldList(); - List targetFields = targetRowType.getFieldList(); - final int sourceCount = sourceFields.size(); - for (int i = 0; i < sourceCount; ++i) { - RelDataType sourceType = sourceFields.get(i).getType(); - RelDataType targetType = targetFields.get(i).getType(); - if (!SqlTypeUtil.canAssignFrom(targetType, sourceType)) { - // FRG-255: account for UPDATE rewrite; there's - // probably a better way to do this. - int iAdjusted = i; - if (query instanceof SqlUpdate) { - int nUpdateColumns = - ((SqlUpdate) query).getTargetColumnList().size(); - assert sourceFields.size() >= nUpdateColumns; - iAdjusted -= sourceFields.size() - nUpdateColumns; - } - SqlNode node = getNthExpr(query, iAdjusted, sourceCount); - String targetTypeString; - String sourceTypeString; - if (SqlTypeUtil.areCharacterSetsMismatched( - sourceType, - targetType)) { - sourceTypeString = sourceType.getFullTypeString(); - targetTypeString = targetType.getFullTypeString(); - } else { - sourceTypeString = sourceType.toString(); - targetTypeString = targetType.toString(); - } - throw newValidationError(node, - RESOURCE.typeNotAssignable( - targetFields.get(i).getName(), targetTypeString, - sourceFields.get(i).getName(), sourceTypeString)); - } - } - } - - /** - * Locates the n'th expression in an INSERT or UPDATE query. - * - * @param query Query - * @param ordinal Ordinal of expression - * @param sourceCount Number of expressions - * @return Ordinal'th expression, never null - */ - private SqlNode getNthExpr(SqlNode query, int ordinal, int sourceCount) { - if (query instanceof SqlInsert) { - SqlInsert insert = (SqlInsert) query; - if (insert.getTargetColumnList() != null) { - return insert.getTargetColumnList().get(ordinal); - } else { - return getNthExpr( - insert.getSource(), - ordinal, - sourceCount); - } - } else if (query instanceof SqlUpdate) { - SqlUpdate update = (SqlUpdate) query; - if (update.getTargetColumnList() != null) { - return update.getTargetColumnList().get(ordinal); - } else if (update.getSourceExpressionList() != null) { - return update.getSourceExpressionList().get(ordinal); - } else { - return getNthExpr( - update.getSourceSelect(), - ordinal, - sourceCount); - } - } else if (query instanceof SqlSelect) { - SqlSelect select = (SqlSelect) query; - if (select.getSelectList().size() == sourceCount) { - return select.getSelectList().get(ordinal); - } else { - return query; // give up - } - } else { - return query; // give up - } - } - - public void validateDelete(SqlDelete call) { - final SqlSelect sqlSelect = call.getSourceSelect(); - validateSelect(sqlSelect, unknownType); - - final SqlValidatorNamespace targetNamespace = getNamespace(call); - validateNamespace(targetNamespace, unknownType); - final SqlValidatorTable table = targetNamespace.getTable(); - - validateAccess(call.getTargetTable(), table, SqlAccessEnum.DELETE); - } - - public void validateUpdate(SqlUpdate call) { - final SqlValidatorNamespace targetNamespace = getNamespace(call); - validateNamespace(targetNamespace, unknownType); - final RelOptTable relOptTable = SqlValidatorUtil.getRelOptTable( - targetNamespace, catalogReader.unwrap(Prepare.CatalogReader.class), null, null); - final SqlValidatorTable table = relOptTable == null - ? targetNamespace.getTable() - : relOptTable.unwrap(SqlValidatorTable.class); - - final RelDataType targetRowType = - createTargetRowType( - table, - call.getTargetColumnList(), - true); - - final SqlSelect select = call.getSourceSelect(); - validateSelect(select, targetRowType); - - final RelDataType sourceRowType = getNamespace(call).getRowType(); - checkTypeAssignment(sourceRowType, targetRowType, call); - - checkConstraint(table, call, targetRowType); - - validateAccess(call.getTargetTable(), table, SqlAccessEnum.UPDATE); - } - - public void validateMerge(SqlMerge call) { - SqlSelect sqlSelect = call.getSourceSelect(); - // REVIEW zfong 5/25/06 - Does an actual type have to be passed into - // validateSelect()? - - // REVIEW jvs 6-June-2006: In general, passing unknownType like - // this means we won't be able to correctly infer the types - // for dynamic parameter markers (SET x = ?). But - // maybe validateUpdate and validateInsert below will do - // the job? - - // REVIEW ksecretan 15-July-2011: They didn't get a chance to - // since validateSelect() would bail. - // Let's use the update/insert targetRowType when available. - IdentifierNamespace targetNamespace = - (IdentifierNamespace) getNamespace(call.getTargetTable()); - validateNamespace(targetNamespace, unknownType); - - SqlValidatorTable table = targetNamespace.getTable(); - validateAccess(call.getTargetTable(), table, SqlAccessEnum.UPDATE); - - RelDataType targetRowType = unknownType; - - if (call.getUpdateCall() != null) { - targetRowType = createTargetRowType( - table, - call.getUpdateCall().getTargetColumnList(), - true); - } - if (call.getInsertCall() != null) { - targetRowType = createTargetRowType( - table, - call.getInsertCall().getTargetColumnList(), - false); - } - - validateSelect(sqlSelect, targetRowType); - - if (call.getUpdateCall() != null) { - validateUpdate(call.getUpdateCall()); - } - if (call.getInsertCall() != null) { - validateInsert(call.getInsertCall()); - } - } - - /** - * Validates access to a table. - * - * @param table Table - * @param requiredAccess Access requested on table - */ - private void validateAccess( - SqlNode node, - SqlValidatorTable table, - SqlAccessEnum requiredAccess) { - if (table != null) { - SqlAccessType access = table.getAllowedAccess(); - if (!access.allowsAccess(requiredAccess)) { - throw newValidationError(node, - RESOURCE.accessNotAllowed(requiredAccess.name(), - table.getQualifiedName().toString())); - } - } - } - - /** - * Validates a VALUES clause. - * - * @param node Values clause - * @param targetRowType Row type which expression must conform to - * @param scope Scope within which clause occurs - */ - protected void validateValues( - SqlCall node, - RelDataType targetRowType, - final SqlValidatorScope scope) { - assert node.getKind() == SqlKind.VALUES; - - final List operands = node.getOperandList(); - for (SqlNode operand : operands) { - if (!(operand.getKind() == SqlKind.ROW)) { - throw Util.needToImplement( - "Values function where operands are scalars"); - } - - SqlCall rowConstructor = (SqlCall) operand; - if (conformance.isInsertSubsetColumnsAllowed() && targetRowType.isStruct() - && rowConstructor.operandCount() < targetRowType.getFieldCount()) { - targetRowType = - typeFactory.createStructType( - targetRowType.getFieldList() - .subList(0, rowConstructor.operandCount())); - } else if (targetRowType.isStruct() - && rowConstructor.operandCount() != targetRowType.getFieldCount()) { - return; - } - - inferUnknownTypes( - targetRowType, - scope, - rowConstructor); - - if (targetRowType.isStruct()) { - for (Pair pair - : Pair.zip(rowConstructor.getOperandList(), - targetRowType.getFieldList())) { - if (!pair.right.getType().isNullable() - && SqlUtil.isNullLiteral(pair.left, false)) { - throw newValidationError(node, - RESOURCE.columnNotNullable(pair.right.getName())); - } - } - } - } - - for (SqlNode operand : operands) { - operand.validate(this, scope); - } - - // validate that all row types have the same number of columns - // and that expressions in each column are compatible. - // A values expression is turned into something that looks like - // ROW(type00, type01,...), ROW(type11,...),... - final int rowCount = operands.size(); - if (rowCount >= 2) { - SqlCall firstRow = (SqlCall) operands.get(0); - final int columnCount = firstRow.operandCount(); - - // 1. check that all rows have the same cols length - for (SqlNode operand : operands) { - SqlCall thisRow = (SqlCall) operand; - if (columnCount != thisRow.operandCount()) { - throw newValidationError(node, - RESOURCE.incompatibleValueType( - SqlStdOperatorTable.VALUES.getName())); - } - } - - // 2. check if types at i:th position in each row are compatible - for (int col = 0; col < columnCount; col++) { - final int c = col; - final RelDataType type = - typeFactory.leastRestrictive( - new AbstractList() { - public RelDataType get(int row) { - SqlCall thisRow = (SqlCall) operands.get(row); - return deriveType(scope, thisRow.operand(c)); - } - - public int size() { - return rowCount; - } - }); - - if (null == type) { - throw newValidationError(node, - RESOURCE.incompatibleValueType( - SqlStdOperatorTable.VALUES.getName())); - } - } - } - } - - public void validateDataType(SqlDataTypeSpec dataType) { - } - - public void validateDynamicParam(SqlDynamicParam dynamicParam) { - } - - /** - * Throws a validator exception with access to the validator context. - * The exception is determined when an instance is created. - */ - private class ValidationError implements Supplier { - private final SqlNode sqlNode; - private final Resources.ExInst validatorException; - - ValidationError(SqlNode sqlNode, - Resources.ExInst validatorException) { - this.sqlNode = sqlNode; - this.validatorException = validatorException; - } - - public CalciteContextException get() { - return newValidationError(sqlNode, validatorException); - } - } - - /** - * Throws a validator exception with access to the validator context. - * The exception is determined when the function is applied. - */ - class ValidationErrorFunction - implements Function2, - CalciteContextException> { - @Override public CalciteContextException apply( - SqlNode v0, Resources.ExInst v1) { - return newValidationError(v0, v1); - } - } - - public ValidationErrorFunction getValidationErrorFunction() { - return validationErrorFunction; - } - - public CalciteContextException newValidationError(SqlNode node, - Resources.ExInst e) { - assert node != null; - final SqlParserPos pos = node.getParserPosition(); - return SqlUtil.newContextException(pos, e); - } - - protected SqlWindow getWindowByName( - SqlIdentifier id, - SqlValidatorScope scope) { - SqlWindow window = null; - if (id.isSimple()) { - final String name = id.getSimple(); - window = scope.lookupWindow(name); - } - if (window == null) { - throw newValidationError(id, RESOURCE.windowNotFound(id.toString())); - } - return window; - } - - public SqlWindow resolveWindow( - SqlNode windowOrRef, - SqlValidatorScope scope, - boolean populateBounds) { - SqlWindow window; - if (windowOrRef instanceof SqlIdentifier) { - window = getWindowByName((SqlIdentifier) windowOrRef, scope); - } else { - window = (SqlWindow) windowOrRef; - } - while (true) { - final SqlIdentifier refId = window.getRefName(); - if (refId == null) { - break; - } - final String refName = refId.getSimple(); - SqlWindow refWindow = scope.lookupWindow(refName); - if (refWindow == null) { - throw newValidationError(refId, RESOURCE.windowNotFound(refName)); - } - window = window.overlay(refWindow, this); - } - - if (populateBounds) { - window.populateBounds(); - } - return window; - } - - public SqlNode getOriginal(SqlNode expr) { - SqlNode original = originalExprs.get(expr); - if (original == null) { - original = expr; - } - return original; - } - - public void setOriginal(SqlNode expr, SqlNode original) { - // Don't overwrite the original original. - originalExprs.putIfAbsent(expr, original); - } - - SqlValidatorNamespace lookupFieldNamespace(RelDataType rowType, String name) { - final SqlNameMatcher nameMatcher = catalogReader.nameMatcher(); - final RelDataTypeField field = nameMatcher.field(rowType, name); - if (field == null) { - return null; - } - return new FieldNamespace(this, field.getType()); - } - - public void validateWindow( - SqlNode windowOrId, - SqlValidatorScope scope, - SqlCall call) { - // Enable nested aggregates with window aggregates (OVER operator) - inWindow = true; - - final SqlWindow targetWindow; - switch (windowOrId.getKind()) { - case IDENTIFIER: - // Just verify the window exists in this query. It will validate - // when the definition is processed - targetWindow = getWindowByName((SqlIdentifier) windowOrId, scope); - break; - case WINDOW: - targetWindow = (SqlWindow) windowOrId; - break; - default: - throw Util.unexpected(windowOrId.getKind()); - } - - assert targetWindow.getWindowCall() == null; - targetWindow.setWindowCall(call); - targetWindow.validate(this, scope); - targetWindow.setWindowCall(null); - call.validate(this, scope); - - validateAggregateParams(call, null, null, scope); - - // Disable nested aggregates post validation - inWindow = false; - } - - @Override public void validateMatchRecognize(SqlCall call) { - final SqlMatchRecognize matchRecognize = (SqlMatchRecognize) call; - final MatchRecognizeScope scope = - (MatchRecognizeScope) getMatchRecognizeScope(matchRecognize); - - final MatchRecognizeNamespace ns = - getNamespace(call).unwrap(MatchRecognizeNamespace.class); - assert ns.rowType == null; - - // rows per match - final SqlLiteral rowsPerMatch = matchRecognize.getRowsPerMatch(); - final boolean allRows = rowsPerMatch != null - && rowsPerMatch.getValue() - == SqlMatchRecognize.RowsPerMatchOption.ALL_ROWS; - - final RelDataTypeFactory.Builder typeBuilder = typeFactory.builder(); - - // parse PARTITION BY column - SqlNodeList partitionBy = matchRecognize.getPartitionList(); - if (partitionBy != null) { - for (SqlNode node : partitionBy) { - SqlIdentifier identifier = (SqlIdentifier) node; - identifier.validate(this, scope); - RelDataType type = deriveType(scope, identifier); - String name = identifier.names.get(1); - typeBuilder.add(name, type); - } - } - - // parse ORDER BY column - SqlNodeList orderBy = matchRecognize.getOrderList(); - if (orderBy != null) { - for (SqlNode node : orderBy) { - node.validate(this, scope); - SqlIdentifier identifier; - if (node instanceof SqlBasicCall) { - identifier = (SqlIdentifier) ((SqlBasicCall) node).getOperands()[0]; - } else { - identifier = (SqlIdentifier) node; - } - - if (allRows) { - RelDataType type = deriveType(scope, identifier); - String name = identifier.names.get(1); - if (!typeBuilder.nameExists(name)) { - typeBuilder.add(name, type); - } - } - } - } - - if (allRows) { - final SqlValidatorNamespace sqlNs = - getNamespace(matchRecognize.getTableRef()); - final RelDataType inputDataType = sqlNs.getRowType(); - for (RelDataTypeField fs : inputDataType.getFieldList()) { - if (!typeBuilder.nameExists(fs.getName())) { - typeBuilder.add(fs); - } - } - } - - // retrieve pattern variables used in pattern and subset - SqlNode pattern = matchRecognize.getPattern(); - PatternVarVisitor visitor = new PatternVarVisitor(scope); - pattern.accept(visitor); - - SqlLiteral interval = matchRecognize.getInterval(); - if (interval != null) { - interval.validate(this, scope); - if (((SqlIntervalLiteral) interval).signum() < 0) { - throw newValidationError(interval, - RESOURCE.intervalMustBeNonNegative(interval.toValue())); - } - if (orderBy == null || orderBy.size() == 0) { - throw newValidationError(interval, - RESOURCE.cannotUseWithinWithoutOrderBy()); - } - - SqlNode firstOrderByColumn = orderBy.getList().get(0); - SqlIdentifier identifier; - if (firstOrderByColumn instanceof SqlBasicCall) { - identifier = (SqlIdentifier) ((SqlBasicCall) firstOrderByColumn).getOperands()[0]; - } else { - identifier = (SqlIdentifier) firstOrderByColumn; - } - RelDataType firstOrderByColumnType = deriveType(scope, identifier); - if (firstOrderByColumnType.getSqlTypeName() != SqlTypeName.TIMESTAMP) { - throw newValidationError(interval, - RESOURCE.firstColumnOfOrderByMustBeTimestamp()); - } - - SqlNode expand = expand(interval, scope); - RelDataType type = deriveType(scope, expand); - setValidatedNodeType(interval, type); - } - - validateDefinitions(matchRecognize, scope); - - SqlNodeList subsets = matchRecognize.getSubsetList(); - if (subsets != null && subsets.size() > 0) { - for (SqlNode node : subsets) { - List operands = ((SqlCall) node).getOperandList(); - String leftString = ((SqlIdentifier) operands.get(0)).getSimple(); - if (scope.getPatternVars().contains(leftString)) { - throw newValidationError(operands.get(0), - RESOURCE.patternVarAlreadyDefined(leftString)); - } - scope.addPatternVar(leftString); - for (SqlNode right : (SqlNodeList) operands.get(1)) { - SqlIdentifier id = (SqlIdentifier) right; - if (!scope.getPatternVars().contains(id.getSimple())) { - throw newValidationError(id, - RESOURCE.unknownPattern(id.getSimple())); - } - scope.addPatternVar(id.getSimple()); - } - } - } - - // validate AFTER ... SKIP TO - final SqlNode skipTo = matchRecognize.getAfter(); - if (skipTo instanceof SqlCall) { - final SqlCall skipToCall = (SqlCall) skipTo; - final SqlIdentifier id = skipToCall.operand(0); - if (!scope.getPatternVars().contains(id.getSimple())) { - throw newValidationError(id, - RESOURCE.unknownPattern(id.getSimple())); - } - } - - List> measureColumns = - validateMeasure(matchRecognize, scope, allRows); - for (Map.Entry c : measureColumns) { - if (!typeBuilder.nameExists(c.getKey())) { - typeBuilder.add(c.getKey(), c.getValue()); - } - } - - final RelDataType rowType = typeBuilder.build(); - if (matchRecognize.getMeasureList().size() == 0) { - ns.setType(getNamespace(matchRecognize.getTableRef()).getRowType()); - } else { - ns.setType(rowType); - } - } - - private List> validateMeasure(SqlMatchRecognize mr, - MatchRecognizeScope scope, boolean allRows) { - final List aliases = new ArrayList<>(); - final List sqlNodes = new ArrayList<>(); - final SqlNodeList measures = mr.getMeasureList(); - final List> fields = new ArrayList<>(); - - for (SqlNode measure : measures) { - assert measure instanceof SqlCall; - final String alias = deriveAlias(measure, aliases.size()); - aliases.add(alias); - - SqlNode expand = expand(measure, scope); - expand = navigationInMeasure(expand, allRows); - setOriginal(expand, measure); - - inferUnknownTypes(unknownType, scope, expand); - final RelDataType type = deriveType(scope, expand); - setValidatedNodeType(measure, type); - - fields.add(Pair.of(alias, type)); - sqlNodes.add( - SqlStdOperatorTable.AS.createCall(SqlParserPos.ZERO, expand, - new SqlIdentifier(alias, SqlParserPos.ZERO))); - } - - SqlNodeList list = new SqlNodeList(sqlNodes, measures.getParserPosition()); - inferUnknownTypes(unknownType, scope, list); - - for (SqlNode node : list) { - validateExpr(node, scope); - } - - mr.setOperand(SqlMatchRecognize.OPERAND_MEASURES, list); - - return fields; - } - - private SqlNode navigationInMeasure(SqlNode node, boolean allRows) { - final Set prefix = node.accept(new PatternValidator(true)); - Util.discard(prefix); - final List ops = ((SqlCall) node).getOperandList(); - - final SqlOperator defaultOp = - allRows ? SqlStdOperatorTable.RUNNING : SqlStdOperatorTable.FINAL; - final SqlNode op0 = ops.get(0); - if (!isRunningOrFinal(op0.getKind()) - || !allRows && op0.getKind() == SqlKind.RUNNING) { - SqlNode newNode = defaultOp.createCall(SqlParserPos.ZERO, op0); - node = SqlStdOperatorTable.AS.createCall(SqlParserPos.ZERO, newNode, ops.get(1)); - } - - node = new NavigationExpander().go(node); - return node; - } - - private void validateDefinitions(SqlMatchRecognize mr, - MatchRecognizeScope scope) { - final Set aliases = catalogReader.nameMatcher().createSet(); - for (SqlNode item : mr.getPatternDefList().getList()) { - final String alias = alias(item); - if (!aliases.add(alias)) { - throw newValidationError(item, - Static.RESOURCE.patternVarAlreadyDefined(alias)); - } - scope.addPatternVar(alias); - } - - final List sqlNodes = new ArrayList<>(); - for (SqlNode item : mr.getPatternDefList().getList()) { - final String alias = alias(item); - SqlNode expand = expand(item, scope); - expand = navigationInDefine(expand, alias); - setOriginal(expand, item); - - inferUnknownTypes(booleanType, scope, expand); - expand.validate(this, scope); - - // Some extra work need required here. - // In PREV, NEXT, FINAL and LAST, only one pattern variable is allowed. - sqlNodes.add( - SqlStdOperatorTable.AS.createCall(SqlParserPos.ZERO, expand, - new SqlIdentifier(alias, SqlParserPos.ZERO))); - - final RelDataType type = deriveType(scope, expand); - if (!SqlTypeUtil.inBooleanFamily(type)) { - throw newValidationError(expand, RESOURCE.condMustBeBoolean("DEFINE")); - } - setValidatedNodeType(item, type); - } - - SqlNodeList list = - new SqlNodeList(sqlNodes, mr.getPatternDefList().getParserPosition()); - inferUnknownTypes(unknownType, scope, list); - for (SqlNode node : list) { - validateExpr(node, scope); - } - mr.setOperand(SqlMatchRecognize.OPERAND_PATTERN_DEFINES, list); - } - - /** Returns the alias of a "expr AS alias" expression. */ - private static String alias(SqlNode item) { - assert item instanceof SqlCall; - assert item.getKind() == SqlKind.AS; - final SqlIdentifier identifier = ((SqlCall) item).operand(1); - return identifier.getSimple(); - } - - /** Checks that all pattern variables within a function are the same, - * and canonizes expressions such as {@code PREV(B.price)} to - * {@code LAST(B.price, 0)}. */ - private SqlNode navigationInDefine(SqlNode node, String alpha) { - Set prefix = node.accept(new PatternValidator(false)); - Util.discard(prefix); - node = new NavigationExpander().go(node); - node = new NavigationReplacer(alpha).go(node); - return node; - } - - public void validateAggregateParams(SqlCall aggCall, SqlNode filter, - SqlNodeList orderList, SqlValidatorScope scope) { - // For "agg(expr)", expr cannot itself contain aggregate function - // invocations. For example, "SUM(2 * MAX(x))" is illegal; when - // we see it, we'll report the error for the SUM (not the MAX). - // For more than one level of nesting, the error which results - // depends on the traversal order for validation. - // - // For a windowed aggregate "agg(expr)", expr can contain an aggregate - // function. For example, - // SELECT AVG(2 * MAX(x)) OVER (PARTITION BY y) - // FROM t - // GROUP BY y - // is legal. Only one level of nesting is allowed since non-windowed - // aggregates cannot nest aggregates. - - // Store nesting level of each aggregate. If an aggregate is found at an invalid - // nesting level, throw an assert. - final AggFinder a; - if (inWindow) { - a = overFinder; - } else { - a = aggOrOverFinder; - } - - for (SqlNode param : aggCall.getOperandList()) { - if (a.findAgg(param) != null) { - throw newValidationError(aggCall, RESOURCE.nestedAggIllegal()); - } - } - if (filter != null) { - if (a.findAgg(filter) != null) { - throw newValidationError(filter, RESOURCE.aggregateInFilterIllegal()); - } - } - if (orderList != null) { - for (SqlNode param : orderList) { - if (a.findAgg(param) != null) { - throw newValidationError(aggCall, - RESOURCE.aggregateInWithinGroupIllegal()); - } - } - } - - final SqlAggFunction op = (SqlAggFunction) aggCall.getOperator(); - switch (op.requiresGroupOrder()) { - case MANDATORY: - if (orderList == null || orderList.size() == 0) { - throw newValidationError(aggCall, - RESOURCE.aggregateMissingWithinGroupClause(op.getName())); - } - break; - case OPTIONAL: - break; - case IGNORED: - // rewrite the order list to empty - if (orderList != null) { - orderList.getList().clear(); - } - break; - case FORBIDDEN: - if (orderList != null && orderList.size() != 0) { - throw newValidationError(aggCall, - RESOURCE.withinGroupClauseIllegalInAggregate(op.getName())); - } - break; - default: - throw new AssertionError(op); - } - } - - public void validateCall( - SqlCall call, - SqlValidatorScope scope) { - final SqlOperator operator = call.getOperator(); - if ((call.operandCount() == 0) - && (operator.getSyntax() == SqlSyntax.FUNCTION_ID) - && !call.isExpanded() - && !conformance.allowNiladicParentheses()) { - // For example, "LOCALTIME()" is illegal. (It should be - // "LOCALTIME", which would have been handled as a - // SqlIdentifier.) - throw handleUnresolvedFunction(call, (SqlFunction) operator, - ImmutableList.of(), null); - } - - SqlValidatorScope operandScope = scope.getOperandScope(call); - - if (operator instanceof SqlFunction - && ((SqlFunction) operator).getFunctionType() - == SqlFunctionCategory.MATCH_RECOGNIZE - && !(operandScope instanceof MatchRecognizeScope)) { - throw newValidationError(call, - Static.RESOURCE.functionMatchRecognizeOnly(call.toString())); - } - // Delegate validation to the operator. - operator.validateCall(call, this, scope, operandScope); - } - - /** - * Validates that a particular feature is enabled. By default, all features - * are enabled; subclasses may override this method to be more - * discriminating. - * - * @param feature feature being used, represented as a resource instance - * @param context parser position context for error reporting, or null if - */ - protected void validateFeature( - Feature feature, - SqlParserPos context) { - // By default, do nothing except to verify that the resource - // represents a real feature definition. - assert feature.getProperties().get("FeatureDefinition") != null; - } - - public SqlNode expand(SqlNode expr, SqlValidatorScope scope) { - final Expander expander = new Expander(this, scope); - SqlNode newExpr = expr.accept(expander); - if (expr != newExpr) { - setOriginal(newExpr, expr); - } - return newExpr; - } - - public SqlNode expandGroupByOrHavingExpr(SqlNode expr, - SqlValidatorScope scope, SqlSelect select, boolean havingExpression) { - final Expander expander = new ExtendedExpander(this, scope, select, expr, - havingExpression); - SqlNode newExpr = expr.accept(expander); - if (expr != newExpr) { - setOriginal(newExpr, expr); - } - return newExpr; - } - - public boolean isSystemField(RelDataTypeField field) { - return false; - } - - public List> getFieldOrigins(SqlNode sqlQuery) { - if (sqlQuery instanceof SqlExplain) { - return Collections.emptyList(); - } - final RelDataType rowType = getValidatedNodeType(sqlQuery); - final int fieldCount = rowType.getFieldCount(); - if (!sqlQuery.isA(SqlKind.QUERY)) { - return Collections.nCopies(fieldCount, null); - } - final List> list = new ArrayList<>(); - for (int i = 0; i < fieldCount; i++) { - list.add(getFieldOrigin(sqlQuery, i)); - } - return ImmutableNullableList.copyOf(list); - } - - private List getFieldOrigin(SqlNode sqlQuery, int i) { - if (sqlQuery instanceof SqlSelect) { - SqlSelect sqlSelect = (SqlSelect) sqlQuery; - final SelectScope scope = getRawSelectScope(sqlSelect); - final List selectList = scope.getExpandedSelectList(); - final SqlNode selectItem = stripAs(selectList.get(i)); - if (selectItem instanceof SqlIdentifier) { - final SqlQualified qualified = - scope.fullyQualify((SqlIdentifier) selectItem); - SqlValidatorNamespace namespace = qualified.namespace; - final SqlValidatorTable table = namespace.getTable(); - if (table == null) { - return null; - } - final List origin = - new ArrayList<>(table.getQualifiedName()); - for (String name : qualified.suffix()) { - namespace = namespace.lookupChild(name); - if (namespace == null) { - return null; - } - origin.add(name); - } - return origin; - } - return null; - } else if (sqlQuery instanceof SqlOrderBy) { - return getFieldOrigin(((SqlOrderBy) sqlQuery).query, i); - } else { - return null; - } - } - - public RelDataType getParameterRowType(SqlNode sqlQuery) { - // NOTE: We assume that bind variables occur in depth-first tree - // traversal in the same order that they occurred in the SQL text. - final List types = new ArrayList<>(); - // NOTE: but parameters on fetch/offset would be counted twice - // as they are counted in the SqlOrderBy call and the inner SqlSelect call - final Set alreadyVisited = new HashSet<>(); - sqlQuery.accept( - new SqlShuttle() { - - @Override public SqlNode visit(SqlDynamicParam param) { - if (alreadyVisited.add(param)) { - RelDataType type = getValidatedNodeType(param); - types.add(type); - } - return param; - } - }); - return typeFactory.createStructType( - types, - new AbstractList() { - @Override public String get(int index) { - return "?" + index; - } - - @Override public int size() { - return types.size(); - } - }); - } - - public void validateColumnListParams( - SqlFunction function, - List argTypes, - List operands) { - throw new UnsupportedOperationException(); - } - - private static boolean isPhysicalNavigation(SqlKind kind) { - return kind == SqlKind.PREV || kind == SqlKind.NEXT; - } - - private static boolean isLogicalNavigation(SqlKind kind) { - return kind == SqlKind.FIRST || kind == SqlKind.LAST; - } - - private static boolean isAggregation(SqlKind kind) { - return kind == SqlKind.SUM || kind == SqlKind.SUM0 - || kind == SqlKind.AVG || kind == SqlKind.COUNT - || kind == SqlKind.MAX || kind == SqlKind.MIN; - } - - private static boolean isRunningOrFinal(SqlKind kind) { - return kind == SqlKind.RUNNING || kind == SqlKind.FINAL; - } - - private static boolean isSingleVarRequired(SqlKind kind) { - return isPhysicalNavigation(kind) - || isLogicalNavigation(kind) - || isAggregation(kind); - } - - //~ Inner Classes ---------------------------------------------------------- - - /** - * Common base class for DML statement namespaces. - */ - public static class DmlNamespace extends IdentifierNamespace { - protected DmlNamespace(SqlValidatorImpl validator, SqlNode id, - SqlNode enclosingNode, SqlValidatorScope parentScope) { - super(validator, id, enclosingNode, parentScope); - } - } - - /** - * Namespace for an INSERT statement. - */ - private static class InsertNamespace extends DmlNamespace { - private final SqlInsert node; - - InsertNamespace(SqlValidatorImpl validator, SqlInsert node, - SqlNode enclosingNode, SqlValidatorScope parentScope) { - super(validator, node.getTargetTable(), enclosingNode, parentScope); - this.node = Objects.requireNonNull(node); - } - - public SqlInsert getNode() { - return node; - } - } - - /** - * Namespace for an UPDATE statement. - */ - private static class UpdateNamespace extends DmlNamespace { - private final SqlUpdate node; - - UpdateNamespace(SqlValidatorImpl validator, SqlUpdate node, - SqlNode enclosingNode, SqlValidatorScope parentScope) { - super(validator, node.getTargetTable(), enclosingNode, parentScope); - this.node = Objects.requireNonNull(node); - } - - public SqlUpdate getNode() { - return node; - } - } - - /** - * Namespace for a DELETE statement. - */ - private static class DeleteNamespace extends DmlNamespace { - private final SqlDelete node; - - DeleteNamespace(SqlValidatorImpl validator, SqlDelete node, - SqlNode enclosingNode, SqlValidatorScope parentScope) { - super(validator, node.getTargetTable(), enclosingNode, parentScope); - this.node = Objects.requireNonNull(node); - } - - public SqlDelete getNode() { - return node; - } - } - - /** - * Namespace for a MERGE statement. - */ - private static class MergeNamespace extends DmlNamespace { - private final SqlMerge node; - - MergeNamespace(SqlValidatorImpl validator, SqlMerge node, - SqlNode enclosingNode, SqlValidatorScope parentScope) { - super(validator, node.getTargetTable(), enclosingNode, parentScope); - this.node = Objects.requireNonNull(node); - } - - public SqlMerge getNode() { - return node; - } - } - - /** - * retrieve pattern variables defined - */ - private class PatternVarVisitor implements SqlVisitor { - private MatchRecognizeScope scope; - PatternVarVisitor(MatchRecognizeScope scope) { - this.scope = scope; - } - - @Override public Void visit(SqlLiteral literal) { - return null; - } - - @Override public Void visit(SqlCall call) { - for (int i = 0; i < call.getOperandList().size(); i++) { - call.getOperandList().get(i).accept(this); - } - return null; - } - - @Override public Void visit(SqlNodeList nodeList) { - throw Util.needToImplement(nodeList); - } - - @Override public Void visit(SqlIdentifier id) { - Preconditions.checkArgument(id.isSimple()); - scope.addPatternVar(id.getSimple()); - return null; - } - - @Override public Void visit(SqlDataTypeSpec type) { - throw Util.needToImplement(type); - } - - @Override public Void visit(SqlDynamicParam param) { - throw Util.needToImplement(param); - } - - @Override public Void visit(SqlIntervalQualifier intervalQualifier) { - throw Util.needToImplement(intervalQualifier); - } - } - - /** - * Visitor which derives the type of a given {@link SqlNode}. - * - *

    Each method must return the derived type. This visitor is basically a - * single-use dispatcher; the visit is never recursive. - */ - private class DeriveTypeVisitor implements SqlVisitor { - private final SqlValidatorScope scope; - - DeriveTypeVisitor(SqlValidatorScope scope) { - this.scope = scope; - } - - public RelDataType visit(SqlLiteral literal) { - return literal.createSqlType(typeFactory); - } - - public RelDataType visit(SqlCall call) { - final SqlOperator operator = call.getOperator(); - return operator.deriveType(SqlValidatorImpl.this, scope, call); - } - - public RelDataType visit(SqlNodeList nodeList) { - // Operand is of a type that we can't derive a type for. If the - // operand is of a peculiar type, such as a SqlNodeList, then you - // should override the operator's validateCall() method so that it - // doesn't try to validate that operand as an expression. - throw Util.needToImplement(nodeList); - } - - public RelDataType visit(SqlIdentifier id) { - // First check for builtin functions which don't have parentheses, - // like "LOCALTIME". - final SqlCall call = makeNullaryCall(id); - if (call != null) { - return call.getOperator().validateOperands( - SqlValidatorImpl.this, - scope, - call); - } - - RelDataType type = null; - if (!(scope instanceof EmptyScope)) { - id = scope.fullyQualify(id).identifier; - } - - // Resolve the longest prefix of id that we can - int i; - for (i = id.names.size() - 1; i > 0; i--) { - // REVIEW jvs 9-June-2005: The name resolution rules used - // here are supposed to match SQL:2003 Part 2 Section 6.6 - // (identifier chain), but we don't currently have enough - // information to get everything right. In particular, - // routine parameters are currently looked up via resolve; - // we could do a better job if they were looked up via - // resolveColumn. - - final SqlNameMatcher nameMatcher = catalogReader.nameMatcher(); - final SqlValidatorScope.ResolvedImpl resolved = - new SqlValidatorScope.ResolvedImpl(); - scope.resolve(id.names.subList(0, i), nameMatcher, false, resolved); - if (resolved.count() == 1) { - // There's a namespace with the name we seek. - final SqlValidatorScope.Resolve resolve = resolved.only(); - type = resolve.rowType(); - for (SqlValidatorScope.Step p : Util.skip(resolve.path.steps())) { - type = type.getFieldList().get(p.i).getType(); - } - break; - } - } - - // Give precedence to namespace found, unless there - // are no more identifier components. - if (type == null || id.names.size() == 1) { - // See if there's a column with the name we seek in - // precisely one of the namespaces in this scope. - RelDataType colType = scope.resolveColumn(id.names.get(0), id); - if (colType != null) { - type = colType; - } - ++i; - } - - if (type == null) { - final SqlIdentifier last = id.getComponent(i - 1, i); - throw newValidationError(last, - RESOURCE.unknownIdentifier(last.toString())); - } - - // Resolve rest of identifier - for (; i < id.names.size(); i++) { - String name = id.names.get(i); - final RelDataTypeField field; - if (name.equals("")) { - // The wildcard "*" is represented as an empty name. It never - // resolves to a field. - name = "*"; - field = null; - } else { - final SqlNameMatcher nameMatcher = catalogReader.nameMatcher(); - field = nameMatcher.field(type, name); - } - if (field == null) { - throw newValidationError(id.getComponent(i), - RESOURCE.unknownField(name)); - } - type = field.getType(); - } - type = - SqlTypeUtil.addCharsetAndCollation( - type, - getTypeFactory()); - return type; - } - - public RelDataType visit(SqlDataTypeSpec dataType) { - // Q. How can a data type have a type? - // A. When it appears in an expression. (Say as the 2nd arg to the - // CAST operator.) - validateDataType(dataType); - return dataType.deriveType(SqlValidatorImpl.this); - } - - public RelDataType visit(SqlDynamicParam param) { - return unknownType; - } - - public RelDataType visit(SqlIntervalQualifier intervalQualifier) { - return typeFactory.createSqlIntervalType(intervalQualifier); - } - } - - /** - * Converts an expression into canonical form by fully-qualifying any - * identifiers. - */ - private static class Expander extends SqlScopedShuttle { - protected final SqlValidatorImpl validator; - - Expander(SqlValidatorImpl validator, SqlValidatorScope scope) { - super(scope); - this.validator = validator; - } - - @Override public SqlNode visit(SqlIdentifier id) { - // First check for builtin functions which don't have - // parentheses, like "LOCALTIME". - final SqlCall call = validator.makeNullaryCall(id); - if (call != null) { - return call.accept(this); - } - final SqlIdentifier fqId = getScope().fullyQualify(id).identifier; - SqlNode expandedExpr = expandDynamicStar(id, fqId); - validator.setOriginal(expandedExpr, id); - return expandedExpr; - } - - @Override protected SqlNode visitScoped(SqlCall call) { - switch (call.getKind()) { - case SCALAR_QUERY: - case CURRENT_VALUE: - case NEXT_VALUE: - case WITH: - return call; - } - // Only visits arguments which are expressions. We don't want to - // qualify non-expressions such as 'x' in 'empno * 5 AS x'. - ArgHandler argHandler = - new CallCopyingArgHandler(call, false); - call.getOperator().acceptCall(this, call, true, argHandler); - final SqlNode result = argHandler.result(); - validator.setOriginal(result, call); - return result; - } - - protected SqlNode expandDynamicStar(SqlIdentifier id, SqlIdentifier fqId) { - if (DynamicRecordType.isDynamicStarColName(Util.last(fqId.names)) - && !DynamicRecordType.isDynamicStarColName(Util.last(id.names))) { - // Convert a column ref into ITEM(*, 'col_name') - // for a dynamic star field in dynTable's rowType. - SqlNode[] inputs = new SqlNode[2]; - inputs[0] = fqId; - inputs[1] = SqlLiteral.createCharString( - Util.last(id.names), - id.getParserPosition()); - return new SqlBasicCall( - SqlStdOperatorTable.ITEM, - inputs, - id.getParserPosition()); - } - return fqId; - } - } - - /** - * Shuttle which walks over an expression in the ORDER BY clause, replacing - * usages of aliases with the underlying expression. - */ - class OrderExpressionExpander extends SqlScopedShuttle { - private final List aliasList; - private final SqlSelect select; - private final SqlNode root; - - OrderExpressionExpander(SqlSelect select, SqlNode root) { - super(getOrderScope(select)); - this.select = select; - this.root = root; - this.aliasList = getNamespace(select).getRowType().getFieldNames(); - } - - public SqlNode go() { - return root.accept(this); - } - - public SqlNode visit(SqlLiteral literal) { - // Ordinal markers, e.g. 'select a, b from t order by 2'. - // Only recognize them if they are the whole expression, - // and if the dialect permits. - if (literal == root && getConformance().isSortByOrdinal()) { - switch (literal.getTypeName()) { - case DECIMAL: - case DOUBLE: - final int intValue = literal.intValue(false); - if (intValue >= 0) { - if (intValue < 1 || intValue > aliasList.size()) { - throw newValidationError( - literal, RESOURCE.orderByOrdinalOutOfRange()); - } - - // SQL ordinals are 1-based, but Sort's are 0-based - int ordinal = intValue - 1; - return nthSelectItem(ordinal, literal.getParserPosition()); - } - break; - } - } - - return super.visit(literal); - } - - /** - * Returns the ordinalth item in the select list. - */ - private SqlNode nthSelectItem(int ordinal, final SqlParserPos pos) { - // TODO: Don't expand the list every time. Maybe keep an expanded - // version of each expression -- select lists and identifiers -- in - // the validator. - - SqlNodeList expandedSelectList = - expandStar( - select.getSelectList(), - select, - false); - SqlNode expr = expandedSelectList.get(ordinal); - expr = stripAs(expr); - if (expr instanceof SqlIdentifier) { - expr = getScope().fullyQualify((SqlIdentifier) expr).identifier; - } - - // Create a copy of the expression with the position of the order - // item. - return expr.clone(pos); - } - - public SqlNode visit(SqlIdentifier id) { - // Aliases, e.g. 'select a as x, b from t order by x'. - if (id.isSimple() - && getConformance().isSortByAlias()) { - String alias = id.getSimple(); - final SqlValidatorNamespace selectNs = getNamespace(select); - final RelDataType rowType = - selectNs.getRowTypeSansSystemColumns(); - final SqlNameMatcher nameMatcher = catalogReader.nameMatcher(); - RelDataTypeField field = nameMatcher.field(rowType, alias); - if (field != null) { - return nthSelectItem( - field.getIndex(), - id.getParserPosition()); - } - } - - // No match. Return identifier unchanged. - return getScope().fullyQualify(id).identifier; - } - - protected SqlNode visitScoped(SqlCall call) { - // Don't attempt to expand sub-queries. We haven't implemented - // these yet. - if (call instanceof SqlSelect) { - return call; - } - return super.visitScoped(call); - } - } - - /** - * Shuttle which walks over an expression in the GROUP BY/HAVING clause, replacing - * usages of aliases or ordinals with the underlying expression. - */ - static class ExtendedExpander extends Expander { - final SqlSelect select; - final SqlNode root; - final boolean havingExpr; - - ExtendedExpander(SqlValidatorImpl validator, SqlValidatorScope scope, - SqlSelect select, SqlNode root, boolean havingExpr) { - super(validator, scope); - this.select = select; - this.root = root; - this.havingExpr = havingExpr; - } - - @Override public SqlNode visit(SqlIdentifier id) { - if (id.isSimple() - && (havingExpr - ? validator.getConformance().isHavingAlias() - : validator.getConformance().isGroupByAlias())) { - String name = id.getSimple(); - SqlNode expr = null; - final SqlNameMatcher nameMatcher = - validator.catalogReader.nameMatcher(); - int n = 0; - for (SqlNode s : select.getSelectList()) { - final String alias = SqlValidatorUtil.getAlias(s, -1); - if (alias != null && nameMatcher.matches(alias, name)) { - expr = s; - n++; - } - } - if (n == 0) { - return super.visit(id); - } else if (n > 1) { - // More than one column has this alias. - throw validator.newValidationError(id, - RESOURCE.columnAmbiguous(name)); - } - if (havingExpr && validator.isAggregate(root)) { - return super.visit(id); - } - expr = stripAs(expr); - if (expr instanceof SqlIdentifier) { - SqlIdentifier sid = (SqlIdentifier) expr; - final SqlIdentifier fqId = getScope().fullyQualify(sid).identifier; - expr = expandDynamicStar(sid, fqId); - } - return expr; - } - return super.visit(id); - } - - public SqlNode visit(SqlLiteral literal) { - if (havingExpr || !validator.getConformance().isGroupByOrdinal()) { - return super.visit(literal); - } - boolean isOrdinalLiteral = literal == root; - switch (root.getKind()) { - case GROUPING_SETS: - case ROLLUP: - case CUBE: - if (root instanceof SqlBasicCall) { - List operandList = ((SqlBasicCall) root).getOperandList(); - for (SqlNode node : operandList) { - if (node.equals(literal)) { - isOrdinalLiteral = true; - break; - } - } - } - break; - } - if (isOrdinalLiteral) { - switch (literal.getTypeName()) { - case DECIMAL: - case DOUBLE: - final int intValue = literal.intValue(false); - if (intValue >= 0) { - if (intValue < 1 || intValue > select.getSelectList().size()) { - throw validator.newValidationError(literal, - RESOURCE.orderByOrdinalOutOfRange()); - } - - // SQL ordinals are 1-based, but Sort's are 0-based - int ordinal = intValue - 1; - return SqlUtil.stripAs(select.getSelectList().get(ordinal)); - } - break; - } - } - - return super.visit(literal); - } - } - - /** Information about an identifier in a particular scope. */ - protected static class IdInfo { - public final SqlValidatorScope scope; - public final SqlIdentifier id; - - public IdInfo(SqlValidatorScope scope, SqlIdentifier id) { - this.scope = scope; - this.id = id; - } - } - - /** - * Utility object used to maintain information about the parameters in a - * function call. - */ - protected static class FunctionParamInfo { - /** - * Maps a cursor (based on its position relative to other cursor - * parameters within a function call) to the SELECT associated with the - * cursor. - */ - public final Map cursorPosToSelectMap; - - /** - * Maps a column list parameter to the parent cursor parameter it - * references. The parameters are id'd by their names. - */ - public final Map columnListParamToParentCursorMap; - - public FunctionParamInfo() { - cursorPosToSelectMap = new HashMap<>(); - columnListParamToParentCursorMap = new HashMap<>(); - } - } - - /** - * Modify the nodes in navigation function - * such as FIRST, LAST, PREV AND NEXT. - */ - private static class NavigationModifier extends SqlShuttle { - public SqlNode go(SqlNode node) { - return node.accept(this); - } - } - - /** - * Shuttle that expands navigation expressions in a MATCH_RECOGNIZE clause. - * - *

    Examples: - * - *

      - *
    • {@code PREV(A.price + A.amount)} → - * {@code PREV(A.price) + PREV(A.amount)} - * - *
    • {@code FIRST(A.price * 2)} → {@code FIRST(A.PRICE) * 2} - *
    - */ - private static class NavigationExpander extends NavigationModifier { - final SqlOperator op; - final SqlNode offset; - - NavigationExpander() { - this(null, null); - } - - NavigationExpander(SqlOperator operator, SqlNode offset) { - this.offset = offset; - this.op = operator; - } - - @Override public SqlNode visit(SqlCall call) { - SqlKind kind = call.getKind(); - List operands = call.getOperandList(); - List newOperands = new ArrayList<>(); - - // This code is a workaround for CALCITE-2707 - if (call.getFunctionQuantifier() != null - && call.getFunctionQuantifier().getValue() == SqlSelectKeyword.DISTINCT) { - final SqlParserPos pos = call.getParserPosition(); - throw SqlUtil.newContextException(pos, Static.RESOURCE.functionQuantifierNotAllowed(call.toString())); - } - // This code is a workaround for CALCITE-2707 - - if (isLogicalNavigation(kind) || isPhysicalNavigation(kind)) { - SqlNode inner = operands.get(0); - SqlNode offset = operands.get(1); - - // merge two straight prev/next, update offset - if (isPhysicalNavigation(kind)) { - SqlKind innerKind = inner.getKind(); - if (isPhysicalNavigation(innerKind)) { - List innerOperands = ((SqlCall) inner).getOperandList(); - SqlNode innerOffset = innerOperands.get(1); - SqlOperator newOperator = innerKind == kind - ? SqlStdOperatorTable.PLUS : SqlStdOperatorTable.MINUS; - offset = newOperator.createCall(SqlParserPos.ZERO, - offset, innerOffset); - inner = call.getOperator().createCall(SqlParserPos.ZERO, - innerOperands.get(0), offset); - } - } - SqlNode newInnerNode = - inner.accept(new NavigationExpander(call.getOperator(), offset)); - if (op != null) { - newInnerNode = op.createCall(SqlParserPos.ZERO, newInnerNode, - this.offset); - } - return newInnerNode; - } - - if (operands.size() > 0) { - for (SqlNode node : operands) { - if (node != null) { - SqlNode newNode = node.accept(new NavigationExpander()); - if (op != null) { - newNode = op.createCall(SqlParserPos.ZERO, newNode, offset); - } - newOperands.add(newNode); - } else { - newOperands.add(null); - } - } - return call.getOperator().createCall(SqlParserPos.ZERO, newOperands); - } else { - if (op == null) { - return call; - } else { - return op.createCall(SqlParserPos.ZERO, call, offset); - } - } - } - - @Override public SqlNode visit(SqlIdentifier id) { - if (op == null) { - return id; - } else { - return op.createCall(SqlParserPos.ZERO, id, offset); - } - } - } - - /** - * Shuttle that replaces {@code A as A.price > PREV(B.price)} with - * {@code PREV(A.price, 0) > LAST(B.price, 0)}. - * - *

    Replacing {@code A.price} with {@code PREV(A.price, 0)} makes the - * implementation of - * {@link RexVisitor#visitPatternFieldRef(RexPatternFieldRef)} more unified. - * Otherwise, it's difficult to implement this method. If it returns the - * specified field, then the navigation such as {@code PREV(A.price, 1)} - * becomes impossible; if not, then comparisons such as - * {@code A.price > PREV(A.price, 1)} become meaningless. - */ - private static class NavigationReplacer extends NavigationModifier { - private final String alpha; - - NavigationReplacer(String alpha) { - this.alpha = alpha; - } - - @Override public SqlNode visit(SqlCall call) { - SqlKind kind = call.getKind(); - if (isLogicalNavigation(kind) - || isAggregation(kind) - || isRunningOrFinal(kind)) { - return call; - } - - switch (kind) { - case PREV: - final List operands = call.getOperandList(); - if (operands.get(0) instanceof SqlIdentifier) { - String name = ((SqlIdentifier) operands.get(0)).names.get(0); - return name.equals(alpha) ? call - : SqlStdOperatorTable.LAST.createCall(SqlParserPos.ZERO, operands); - } - } - return super.visit(call); - } - - @Override public SqlNode visit(SqlIdentifier id) { - if (id.isSimple()) { - return id; - } - SqlOperator operator = id.names.get(0).equals(alpha) - ? SqlStdOperatorTable.PREV : SqlStdOperatorTable.LAST; - - return operator.createCall(SqlParserPos.ZERO, id, - SqlLiteral.createExactNumeric("0", SqlParserPos.ZERO)); - } - } - - /** - * Within one navigation function, the pattern var should be same - */ - private class PatternValidator extends SqlBasicVisitor> { - private final boolean isMeasure; - int firstLastCount; - int prevNextCount; - int aggregateCount; - - PatternValidator(boolean isMeasure) { - this(isMeasure, 0, 0, 0); - } - - PatternValidator(boolean isMeasure, int firstLastCount, int prevNextCount, - int aggregateCount) { - this.isMeasure = isMeasure; - this.firstLastCount = firstLastCount; - this.prevNextCount = prevNextCount; - this.aggregateCount = aggregateCount; - } - - @Override public Set visit(SqlCall call) { - boolean isSingle = false; - Set vars = new HashSet<>(); - SqlKind kind = call.getKind(); - List operands = call.getOperandList(); - - if (isSingleVarRequired(kind)) { - isSingle = true; - if (isPhysicalNavigation(kind)) { - if (isMeasure) { - throw newValidationError(call, - Static.RESOURCE.patternPrevFunctionInMeasure(call.toString())); - } - if (firstLastCount != 0) { - throw newValidationError(call, - Static.RESOURCE.patternPrevFunctionOrder(call.toString())); - } - prevNextCount++; - } else if (isLogicalNavigation(kind)) { - if (firstLastCount != 0) { - throw newValidationError(call, - Static.RESOURCE.patternPrevFunctionOrder(call.toString())); - } - firstLastCount++; - } else if (isAggregation(kind)) { - // cannot apply aggregation in PREV/NEXT, FIRST/LAST - if (firstLastCount != 0 || prevNextCount != 0) { - throw newValidationError(call, - Static.RESOURCE.patternAggregationInNavigation(call.toString())); - } - if (kind == SqlKind.COUNT && call.getOperandList().size() > 1) { - throw newValidationError(call, - Static.RESOURCE.patternCountFunctionArg()); - } - aggregateCount++; - } - } - - if (isRunningOrFinal(kind) && !isMeasure) { - throw newValidationError(call, - Static.RESOURCE.patternRunningFunctionInDefine(call.toString())); - } - - for (SqlNode node : operands) { - if (node != null) { - vars.addAll( - node.accept( - new PatternValidator(isMeasure, firstLastCount, prevNextCount, - aggregateCount))); - } - } - - if (isSingle) { - switch (kind) { - case COUNT: - if (vars.size() > 1) { - throw newValidationError(call, - Static.RESOURCE.patternCountFunctionArg()); - } - break; - default: - if (operands.size() == 0 - || !(operands.get(0) instanceof SqlCall) - || ((SqlCall) operands.get(0)).getOperator() != SqlStdOperatorTable.CLASSIFIER) { - if (vars.isEmpty()) { - throw newValidationError(call, - Static.RESOURCE.patternFunctionNullCheck(call.toString())); - } - if (vars.size() != 1) { - throw newValidationError(call, - Static.RESOURCE.patternFunctionVariableCheck(call.toString())); - } - } - break; - } - } - return vars; - } - - @Override public Set visit(SqlIdentifier identifier) { - boolean check = prevNextCount > 0 || firstLastCount > 0 || aggregateCount > 0; - Set vars = new HashSet<>(); - if (identifier.names.size() > 1 && check) { - vars.add(identifier.names.get(0)); - } - return vars; - } - - @Override public Set visit(SqlLiteral literal) { - return ImmutableSet.of(); - } - - @Override public Set visit(SqlIntervalQualifier qualifier) { - return ImmutableSet.of(); - } - - @Override public Set visit(SqlDataTypeSpec type) { - return ImmutableSet.of(); - } - - @Override public Set visit(SqlDynamicParam param) { - return ImmutableSet.of(); - } - } - - /** Permutation of fields in NATURAL JOIN or USING. */ - private class Permute { - final List sources; - final RelDataType rowType; - final boolean trivial; - - Permute(SqlNode from, int offset) { - switch (from.getKind()) { - case JOIN: - final SqlJoin join = (SqlJoin) from; - final Permute left = new Permute(join.getLeft(), offset); - final int fieldCount = - getValidatedNodeType(join.getLeft()).getFieldList().size(); - final Permute right = - new Permute(join.getRight(), offset + fieldCount); - final List names = usingNames(join); - final List sources = new ArrayList<>(); - final Set sourceSet = new HashSet<>(); - final RelDataTypeFactory.Builder b = typeFactory.builder(); - if (names != null) { - for (String name : names) { - final RelDataTypeField f = left.field(name); - final ImmutableIntList source = left.sources.get(f.getIndex()); - sourceSet.add(source); - final RelDataTypeField f2 = right.field(name); - final ImmutableIntList source2 = right.sources.get(f2.getIndex()); - sourceSet.add(source2); - sources.add(source.appendAll(source2)); - final boolean nullable = - (f.getType().isNullable() - || join.getJoinType().generatesNullsOnLeft()) - && (f2.getType().isNullable() - || join.getJoinType().generatesNullsOnRight()); - b.add(f).nullable(nullable); - } - } - for (RelDataTypeField f : left.rowType.getFieldList()) { - final ImmutableIntList source = left.sources.get(f.getIndex()); - if (sourceSet.add(source)) { - sources.add(source); - b.add(f); - } - } - for (RelDataTypeField f : right.rowType.getFieldList()) { - final ImmutableIntList source = right.sources.get(f.getIndex()); - if (sourceSet.add(source)) { - sources.add(source); - b.add(f); - } - } - rowType = b.build(); - this.sources = ImmutableList.copyOf(sources); - this.trivial = left.trivial - && right.trivial - && (names == null || names.isEmpty()); - break; - - default: - rowType = getValidatedNodeType(from); - this.sources = Functions.generate(rowType.getFieldCount(), - i -> ImmutableIntList.of(offset + i)); - this.trivial = true; - } - } - - private RelDataTypeField field(String name) { - return catalogReader.nameMatcher().field(rowType, name); - } - - /** Returns the set of field names in the join condition specified by USING - * or implicitly by NATURAL, de-duplicated and in order. */ - private List usingNames(SqlJoin join) { - switch (join.getConditionType()) { - case USING: - final ImmutableList.Builder list = ImmutableList.builder(); - final Set names = catalogReader.nameMatcher().createSet(); - for (SqlNode node : (SqlNodeList) join.getCondition()) { - final String name = ((SqlIdentifier) node).getSimple(); - if (names.add(name)) { - list.add(name); - } - } - return list.build(); - case NONE: - if (join.isNatural()) { - final RelDataType t0 = getValidatedNodeType(join.getLeft()); - final RelDataType t1 = getValidatedNodeType(join.getRight()); - return SqlValidatorUtil.deriveNaturalJoinColumnList( - catalogReader.nameMatcher(), t0, t1); - } - } - return null; - } - - /** Moves fields according to the permutation. */ - public void permute(List selectItems, - List> fields) { - if (trivial) { - return; - } - - final List oldSelectItems = ImmutableList.copyOf(selectItems); - selectItems.clear(); - final List> oldFields = - ImmutableList.copyOf(fields); - fields.clear(); - for (ImmutableIntList source : sources) { - final int p0 = source.get(0); - Map.Entry field = oldFields.get(p0); - final String name = field.getKey(); - RelDataType type = field.getValue(); - SqlNode selectItem = oldSelectItems.get(p0); - for (int p1 : Util.skip(source)) { - final Map.Entry field1 = oldFields.get(p1); - final SqlNode selectItem1 = oldSelectItems.get(p1); - final RelDataType type1 = field1.getValue(); - // output is nullable only if both inputs are - final boolean nullable = type.isNullable() && type1.isNullable(); - final RelDataType type2 = - SqlTypeUtil.leastRestrictiveForComparison(typeFactory, type, - type1); - selectItem = - SqlStdOperatorTable.AS.createCall(SqlParserPos.ZERO, - SqlStdOperatorTable.COALESCE.createCall(SqlParserPos.ZERO, - maybeCast(selectItem, type, type2), - maybeCast(selectItem1, type1, type2)), - new SqlIdentifier(name, SqlParserPos.ZERO)); - type = typeFactory.createTypeWithNullability(type2, nullable); - } - fields.add(Pair.of(name, type)); - selectItems.add(selectItem); - } - } - } - - //~ Enums ------------------------------------------------------------------ - - /** - * Validation status. - */ - public enum Status { - /** - * Validation has not started for this scope. - */ - UNVALIDATED, - - /** - * Validation is in progress for this scope. - */ - IN_PROGRESS, - - /** - * Validation has completed (perhaps unsuccessfully). - */ - VALID - } - -} - -// End SqlValidatorImpl.java diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/SqlFunction.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/SqlFunction.java deleted file mode 100644 index 5bb2bf5f38..0000000000 --- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/SqlFunction.java +++ /dev/null @@ -1,322 +0,0 @@ -/* - * 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.calcite.sql; - -import org.apache.calcite.linq4j.function.Functions; -import org.apache.calcite.rel.type.RelDataType; -import org.apache.calcite.sql.type.SqlOperandTypeChecker; -import org.apache.calcite.sql.type.SqlOperandTypeInference; -import org.apache.calcite.sql.type.SqlReturnTypeInference; -import org.apache.calcite.sql.validate.SqlValidator; -import org.apache.calcite.sql.validate.SqlValidatorScope; -import org.apache.calcite.sql.validate.implicit.TypeCoercion; -import org.apache.calcite.util.Util; - -import com.google.common.collect.ImmutableList; - -import java.util.List; -import java.util.Objects; -import javax.annotation.Nonnull; - -import static org.apache.calcite.util.Static.RESOURCE; - -/** - * A SqlFunction is a type of operator which has conventional - * function-call syntax. - * - *

    This file is copied from Apache Calcite and should be removed once CALCITE-3360 - * has been fixed. Changed lines: 279-293 - */ -public class SqlFunction extends SqlOperator { - - //~ Instance fields -------------------------------------------------------- - - private final SqlFunctionCategory category; - - private final SqlIdentifier sqlIdentifier; - - private final List paramTypes; - - //~ Constructors ----------------------------------------------------------- - - /** - * Creates a new SqlFunction for a call to a builtin function. - * - * @param name Name of builtin function - * @param kind kind of operator implemented by function - * @param returnTypeInference strategy to use for return type inference - * @param operandTypeInference strategy to use for parameter type inference - * @param operandTypeChecker strategy to use for parameter type checking - * @param category categorization for function - */ - public SqlFunction( - String name, - SqlKind kind, - SqlReturnTypeInference returnTypeInference, - SqlOperandTypeInference operandTypeInference, - SqlOperandTypeChecker operandTypeChecker, - SqlFunctionCategory category) { - // We leave sqlIdentifier as null to indicate - // that this is a builtin. Same for paramTypes. - this(name, null, kind, returnTypeInference, operandTypeInference, - operandTypeChecker, null, category); - - assert !((category == SqlFunctionCategory.USER_DEFINED_CONSTRUCTOR) - && (returnTypeInference == null)); - } - - /** - * Creates a placeholder SqlFunction for an invocation of a function with a - * possibly qualified name. This name must be resolved into either a builtin - * function or a user-defined function. - * - * @param sqlIdentifier possibly qualified identifier for function - * @param returnTypeInference strategy to use for return type inference - * @param operandTypeInference strategy to use for parameter type inference - * @param operandTypeChecker strategy to use for parameter type checking - * @param paramTypes array of parameter types - * @param funcType function category - */ - public SqlFunction( - SqlIdentifier sqlIdentifier, - SqlReturnTypeInference returnTypeInference, - SqlOperandTypeInference operandTypeInference, - SqlOperandTypeChecker operandTypeChecker, - List paramTypes, - SqlFunctionCategory funcType) { - this(Util.last(sqlIdentifier.names), sqlIdentifier, SqlKind.OTHER_FUNCTION, - returnTypeInference, operandTypeInference, operandTypeChecker, - paramTypes, funcType); - } - - /** - * Internal constructor. - */ - protected SqlFunction( - String name, - SqlIdentifier sqlIdentifier, - SqlKind kind, - SqlReturnTypeInference returnTypeInference, - SqlOperandTypeInference operandTypeInference, - SqlOperandTypeChecker operandTypeChecker, - List paramTypes, - SqlFunctionCategory category) { - super(name, kind, 100, 100, returnTypeInference, operandTypeInference, - operandTypeChecker); - - this.sqlIdentifier = sqlIdentifier; - this.category = Objects.requireNonNull(category); - this.paramTypes = - paramTypes == null ? null : ImmutableList.copyOf(paramTypes); - } - - //~ Methods ---------------------------------------------------------------- - - public SqlSyntax getSyntax() { - return SqlSyntax.FUNCTION; - } - - /** - * @return fully qualified name of function, or null for a builtin function - */ - public SqlIdentifier getSqlIdentifier() { - return sqlIdentifier; - } - - @Override public SqlIdentifier getNameAsId() { - if (sqlIdentifier != null) { - return sqlIdentifier; - } - return super.getNameAsId(); - } - - /** - * @return array of parameter types, or null for builtin function - */ - public List getParamTypes() { - return paramTypes; - } - - /** - * Returns a list of parameter names. - * - *

    The default implementation returns {@code [arg0, arg1, ..., argN]}. - */ - public List getParamNames() { - return Functions.generate(paramTypes.size(), i -> "arg" + i); - } - - public void unparse( - SqlWriter writer, - SqlCall call, - int leftPrec, - int rightPrec) { - getSyntax().unparse(writer, this, call, leftPrec, rightPrec); - } - - /** - * @return function category - */ - @Nonnull public SqlFunctionCategory getFunctionType() { - return this.category; - } - - /** - * Returns whether this function allows a DISTINCT or - * ALL quantifier. The default is false; some aggregate - * functions return true. - */ - public boolean isQuantifierAllowed() { - return false; - } - - public void validateCall( - SqlCall call, - SqlValidator validator, - SqlValidatorScope scope, - SqlValidatorScope operandScope) { - // This implementation looks for the quantifier keywords DISTINCT or - // ALL as the first operand in the list. If found then the literal is - // not called to validate itself. Further the function is checked to - // make sure that a quantifier is valid for that particular function. - // - // If the first operand does not appear to be a quantifier then the - // parent ValidateCall is invoked to do normal function validation. - - super.validateCall(call, validator, scope, operandScope); - validateQuantifier(validator, call); - } - - /** - * Throws a validation error if a DISTINCT or ALL quantifier is present but - * not allowed. - */ - protected void validateQuantifier(SqlValidator validator, SqlCall call) { - if ((null != call.getFunctionQuantifier()) && !isQuantifierAllowed()) { - throw validator.newValidationError(call.getFunctionQuantifier(), - RESOURCE.functionQuantifierNotAllowed(call.getOperator().getName())); - } - } - - public RelDataType deriveType( - SqlValidator validator, - SqlValidatorScope scope, - SqlCall call) { - return deriveType(validator, scope, call, true); - } - - private RelDataType deriveType( - SqlValidator validator, - SqlValidatorScope scope, - SqlCall call, - boolean convertRowArgToColumnList) { - // Scope for operands. Usually the same as 'scope'. - final SqlValidatorScope operandScope = scope.getOperandScope(call); - - // Indicate to the validator that we're validating a new function call - validator.pushFunctionCall(); - - final List argNames = constructArgNameList(call); - - final List args = constructOperandList(validator, call, argNames); - - final List argTypes = constructArgTypeList(validator, scope, - call, args, convertRowArgToColumnList); - - SqlFunction function = - (SqlFunction) SqlUtil.lookupRoutine(validator.getOperatorTable(), - getNameAsId(), argTypes, argNames, getFunctionType(), - SqlSyntax.FUNCTION, getKind(), - validator.getCatalogReader().nameMatcher(), - false); - try { - // if we have a match on function name and parameter count, but - // couldn't find a function with a COLUMN_LIST type, retry, but - // this time, don't convert the row argument to a COLUMN_LIST type; - // if we did find a match, go back and re-validate the row operands - // (corresponding to column references), now that we can set the - // scope to that of the source cursor referenced by that ColumnList - // type - if (convertRowArgToColumnList && containsRowArg(args)) { - if (function == null - && SqlUtil.matchRoutinesByParameterCount( - validator.getOperatorTable(), getNameAsId(), argTypes, - getFunctionType(), - validator.getCatalogReader().nameMatcher())) { - // remove the already validated node types corresponding to - // row arguments before re-validating - for (SqlNode operand : args) { - if (operand.getKind() == SqlKind.ROW) { - validator.removeValidatedNodeType(operand); - } - } - return deriveType(validator, scope, call, false); - } else if (function != null) { - validator.validateColumnListParams(function, argTypes, args); - } - } - - if (getFunctionType() == SqlFunctionCategory.USER_DEFINED_CONSTRUCTOR) { - return validator.deriveConstructorType(scope, call, this, function, - argTypes); - } - if (function == null) { - // try again if implicit type coercion is allowed. - boolean changed = false; - if (validator.isTypeCoercionEnabled()) { - function = (SqlFunction) SqlUtil.lookupRoutine(validator.getOperatorTable(), - getNameAsId(), argTypes, argNames, getFunctionType(), SqlSyntax.FUNCTION, getKind(), - validator.getCatalogReader().nameMatcher(), - true); - // try to coerce the function arguments to the declared sql type name. - // if we succeed, the arguments would be wrapped with CAST operator. - if (function != null) { - TypeCoercion typeCoercion = validator.getTypeCoercion(); - changed = typeCoercion.userDefinedFunctionCoercion(scope, call, function); - } - } - if (!changed) { - throw validator.handleUnresolvedFunction(call, this, argTypes, - argNames); - } - } - - // REVIEW jvs 25-Mar-2005: This is, in a sense, expanding - // identifiers, but we ignore shouldExpandIdentifiers() - // because otherwise later validation code will - // choke on the unresolved function. - ((SqlBasicCall) call).setOperator(function); - return function.validateOperands( - validator, - operandScope, - call); - } finally { - validator.popFunctionCall(); - } - } - - private boolean containsRowArg(List args) { - for (SqlNode operand : args) { - if (operand.getKind() == SqlKind.ROW) { - return true; - } - } - return false; - } -} - -// End SqlFunction.java diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/ParameterScope.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/ParameterScope.java deleted file mode 100644 index 414c928910..0000000000 --- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/ParameterScope.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * 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.calcite.sql.validate; - -import org.apache.calcite.rel.type.RelDataType; -import org.apache.calcite.sql.SqlCall; -import org.apache.calcite.sql.SqlIdentifier; -import org.apache.calcite.sql.SqlNode; - -import java.util.Map; - -// This class is copied from Calcite's org.apache.calcite.sql.validate.ParameterScope, -// can be removed after https://issues.apache.org/jira/browse/CALCITE-3476 is fixed. -// -// Modification: -// - L66~L69: override resolveColumn method - -/** - * A scope which contains nothing besides a few parameters. Like - * {@link EmptyScope} (which is its base class), it has no parent scope. - * - * @see ParameterNamespace - */ -public class ParameterScope extends EmptyScope { - //~ Instance fields -------------------------------------------------------- - - /** - * Map from the simple names of the parameters to types of the parameters - * ({@link RelDataType}). - */ - private final Map nameToTypeMap; - - //~ Constructors ----------------------------------------------------------- - - public ParameterScope( - SqlValidatorImpl validator, - Map nameToTypeMap) { - super(validator); - this.nameToTypeMap = nameToTypeMap; - } - - //~ Methods ---------------------------------------------------------------- - - public SqlQualified fullyQualify(SqlIdentifier identifier) { - return SqlQualified.create(this, 1, null, identifier); - } - - public SqlValidatorScope getOperandScope(SqlCall call) { - return this; - } - - @Override - public RelDataType resolveColumn(String name, SqlNode ctx) { - return nameToTypeMap.get(name); - } -} - -// End ParameterScope.java diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java deleted file mode 100644 index c3ccc3f9b9..0000000000 --- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java +++ /dev/null @@ -1,6452 +0,0 @@ -/* - * 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.calcite.sql.validate; - -import org.apache.calcite.config.NullCollation; -import org.apache.calcite.linq4j.Ord; -import org.apache.calcite.linq4j.function.Function2; -import org.apache.calcite.linq4j.function.Functions; -import org.apache.calcite.plan.RelOptTable; -import org.apache.calcite.plan.RelOptUtil; -import org.apache.calcite.prepare.Prepare; -import org.apache.calcite.rel.type.DynamicRecordType; -import org.apache.calcite.rel.type.RelDataType; -import org.apache.calcite.rel.type.RelDataTypeFactory; -import org.apache.calcite.rel.type.RelDataTypeField; -import org.apache.calcite.rel.type.RelDataTypeSystem; -import org.apache.calcite.rel.type.RelRecordType; -import org.apache.calcite.rex.RexBuilder; -import org.apache.calcite.rex.RexNode; -import org.apache.calcite.rex.RexPatternFieldRef; -import org.apache.calcite.rex.RexVisitor; -import org.apache.calcite.runtime.CalciteContextException; -import org.apache.calcite.runtime.CalciteException; -import org.apache.calcite.runtime.Feature; -import org.apache.calcite.runtime.Resources; -import org.apache.calcite.schema.ColumnStrategy; -import org.apache.calcite.schema.Table; -import org.apache.calcite.schema.impl.ModifiableViewTable; -import org.apache.calcite.sql.JoinConditionType; -import org.apache.calcite.sql.JoinType; -import org.apache.calcite.sql.SqlAccessEnum; -import org.apache.calcite.sql.SqlAccessType; -import org.apache.calcite.sql.SqlAggFunction; -import org.apache.calcite.sql.SqlBasicCall; -import org.apache.calcite.sql.SqlCall; -import org.apache.calcite.sql.SqlCallBinding; -import org.apache.calcite.sql.SqlDataTypeSpec; -import org.apache.calcite.sql.SqlDelete; -import org.apache.calcite.sql.SqlDynamicParam; -import org.apache.calcite.sql.SqlExplain; -import org.apache.calcite.sql.SqlFunction; -import org.apache.calcite.sql.SqlFunctionCategory; -import org.apache.calcite.sql.SqlIdentifier; -import org.apache.calcite.sql.SqlInsert; -import org.apache.calcite.sql.SqlIntervalLiteral; -import org.apache.calcite.sql.SqlIntervalQualifier; -import org.apache.calcite.sql.SqlJoin; -import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.sql.SqlLiteral; -import org.apache.calcite.sql.SqlMatchRecognize; -import org.apache.calcite.sql.SqlMerge; -import org.apache.calcite.sql.SqlNode; -import org.apache.calcite.sql.SqlNodeList; -import org.apache.calcite.sql.SqlOperator; -import org.apache.calcite.sql.SqlOperatorTable; -import org.apache.calcite.sql.SqlOrderBy; -import org.apache.calcite.sql.SqlSampleSpec; -import org.apache.calcite.sql.SqlSelect; -import org.apache.calcite.sql.SqlSelectKeyword; -import org.apache.calcite.sql.SqlSnapshot; -import org.apache.calcite.sql.SqlSyntax; -import org.apache.calcite.sql.SqlUnresolvedFunction; -import org.apache.calcite.sql.SqlUpdate; -import org.apache.calcite.sql.SqlUtil; -import org.apache.calcite.sql.SqlWindow; -import org.apache.calcite.sql.SqlWith; -import org.apache.calcite.sql.SqlWithItem; -import org.apache.calcite.sql.fun.SqlCase; -import org.apache.calcite.sql.fun.SqlStdOperatorTable; -import org.apache.calcite.sql.parser.SqlParserPos; -import org.apache.calcite.sql.type.AssignableOperandTypeChecker; -import org.apache.calcite.sql.type.ReturnTypes; -import org.apache.calcite.sql.type.SqlOperandTypeInference; -import org.apache.calcite.sql.type.SqlTypeName; -import org.apache.calcite.sql.type.SqlTypeUtil; -import org.apache.calcite.sql.util.SqlBasicVisitor; -import org.apache.calcite.sql.util.SqlShuttle; -import org.apache.calcite.sql.util.SqlVisitor; -import org.apache.calcite.sql.validate.implicit.TypeCoercion; -import org.apache.calcite.sql.validate.implicit.TypeCoercions; -import org.apache.calcite.sql2rel.InitializerContext; -import org.apache.calcite.util.BitString; -import org.apache.calcite.util.Bug; -import org.apache.calcite.util.ImmutableBitSet; -import org.apache.calcite.util.ImmutableIntList; -import org.apache.calcite.util.ImmutableNullableList; -import org.apache.calcite.util.Litmus; -import org.apache.calcite.util.Pair; -import org.apache.calcite.util.Static; -import org.apache.calcite.util.Util; -import org.apache.calcite.util.trace.CalciteTrace; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; - -import org.slf4j.Logger; - -import java.math.BigDecimal; -import java.math.BigInteger; -import java.util.AbstractList; -import java.util.ArrayDeque; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Calendar; -import java.util.Collection; -import java.util.Collections; -import java.util.Deque; -import java.util.GregorianCalendar; -import java.util.HashMap; -import java.util.HashSet; -import java.util.IdentityHashMap; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.Objects; -import java.util.Set; -import java.util.function.Supplier; -import javax.annotation.Nonnull; -import javax.annotation.Nullable; - -import static org.apache.calcite.sql.SqlUtil.stripAs; -import static org.apache.calcite.util.Static.RESOURCE; - -/* - * THIS FILE HAS BEEN COPIED FROM THE APACHE CALCITE PROJECT UNTIL CALCITE-2707 IS FIXED. - * (Added lines: 6016-6022) - */ - -/** - * Default implementation of {@link SqlValidator}. - */ -public class SqlValidatorImpl implements SqlValidatorWithHints { - //~ Static fields/initializers --------------------------------------------- - - public static final Logger TRACER = CalciteTrace.PARSER_LOGGER; - - /** - * Alias generated for the source table when rewriting UPDATE to MERGE. - */ - public static final String UPDATE_SRC_ALIAS = "SYS$SRC"; - - /** - * Alias generated for the target table when rewriting UPDATE to MERGE if no - * alias was specified by the user. - */ - public static final String UPDATE_TGT_ALIAS = "SYS$TGT"; - - /** - * Alias prefix generated for source columns when rewriting UPDATE to MERGE. - */ - public static final String UPDATE_ANON_PREFIX = "SYS$ANON"; - - //~ Instance fields -------------------------------------------------------- - - private final SqlOperatorTable opTab; - final SqlValidatorCatalogReader catalogReader; - - /** - * Maps ParsePosition strings to the {@link SqlIdentifier} identifier - * objects at these positions - */ - protected final Map idPositions = new HashMap<>(); - - /** - * Maps {@link SqlNode query node} objects to the {@link SqlValidatorScope} - * scope created from them. - */ - protected final Map scopes = - new IdentityHashMap<>(); - - /** - * Maps a {@link SqlSelect} node to the scope used by its WHERE and HAVING - * clauses. - */ - private final Map whereScopes = - new IdentityHashMap<>(); - - /** - * Maps a {@link SqlSelect} node to the scope used by its GROUP BY clause. - */ - private final Map groupByScopes = - new IdentityHashMap<>(); - - /** - * Maps a {@link SqlSelect} node to the scope used by its SELECT and HAVING - * clauses. - */ - private final Map selectScopes = - new IdentityHashMap<>(); - - /** - * Maps a {@link SqlSelect} node to the scope used by its ORDER BY clause. - */ - private final Map orderScopes = - new IdentityHashMap<>(); - - /** - * Maps a {@link SqlSelect} node that is the argument to a CURSOR - * constructor to the scope of the result of that select node - */ - private final Map cursorScopes = - new IdentityHashMap<>(); - - /** - * The name-resolution scope of a LATERAL TABLE clause. - */ - private TableScope tableScope = null; - - /** - * Maps a {@link SqlNode node} to the - * {@link SqlValidatorNamespace namespace} which describes what columns they - * contain. - */ - protected final Map namespaces = - new IdentityHashMap<>(); - - /** - * Set of select expressions used as cursor definitions. In standard SQL, - * only the top-level SELECT is a cursor; Calcite extends this with - * cursors as inputs to table functions. - */ - private final Set cursorSet = Sets.newIdentityHashSet(); - - /** - * Stack of objects that maintain information about function calls. A stack - * is needed to handle nested function calls. The function call currently - * being validated is at the top of the stack. - */ - protected final Deque functionCallStack = - new ArrayDeque<>(); - - private int nextGeneratedId; - protected final RelDataTypeFactory typeFactory; - - /** The type of dynamic parameters until a type is imposed on them. */ - protected final RelDataType unknownType; - private final RelDataType booleanType; - - /** - * Map of derived RelDataType for each node. This is an IdentityHashMap - * since in some cases (such as null literals) we need to discriminate by - * instance. - */ - private final Map nodeToTypeMap = - new IdentityHashMap<>(); - private final AggFinder aggFinder; - private final AggFinder aggOrOverFinder; - private final AggFinder aggOrOverOrGroupFinder; - private final AggFinder groupFinder; - private final AggFinder overFinder; - private final SqlConformance conformance; - private final Map originalExprs = new HashMap<>(); - - private SqlNode top; - - // REVIEW jvs 30-June-2006: subclasses may override shouldExpandIdentifiers - // in a way that ignores this; we should probably get rid of the protected - // method and always use this variable (or better, move preferences like - // this to a separate "parameter" class) - protected boolean expandIdentifiers; - - protected boolean expandColumnReferences; - - private boolean rewriteCalls; - - private NullCollation nullCollation = NullCollation.HIGH; - - // TODO jvs 11-Dec-2008: make this local to performUnconditionalRewrites - // if it's OK to expand the signature of that method. - private boolean validatingSqlMerge; - - private boolean inWindow; // Allow nested aggregates - - private final SqlValidatorImpl.ValidationErrorFunction validationErrorFunction = - new SqlValidatorImpl.ValidationErrorFunction(); - - // TypeCoercion instance used for implicit type coercion. - private TypeCoercion typeCoercion; - - // Flag saying if we enable the implicit type coercion. - private boolean enableTypeCoercion; - - //~ Constructors ----------------------------------------------------------- - - /** - * Creates a validator. - * - * @param opTab Operator table - * @param catalogReader Catalog reader - * @param typeFactory Type factory - * @param conformance Compatibility mode - */ - protected SqlValidatorImpl( - SqlOperatorTable opTab, - SqlValidatorCatalogReader catalogReader, - RelDataTypeFactory typeFactory, - SqlConformance conformance) { - this.opTab = Objects.requireNonNull(opTab); - this.catalogReader = Objects.requireNonNull(catalogReader); - this.typeFactory = Objects.requireNonNull(typeFactory); - this.conformance = Objects.requireNonNull(conformance); - - unknownType = typeFactory.createUnknownType(); - booleanType = typeFactory.createSqlType(SqlTypeName.BOOLEAN); - - rewriteCalls = true; - expandColumnReferences = true; - final SqlNameMatcher nameMatcher = catalogReader.nameMatcher(); - aggFinder = new AggFinder(opTab, false, true, false, null, nameMatcher); - aggOrOverFinder = new AggFinder(opTab, true, true, false, null, nameMatcher); - overFinder = new AggFinder(opTab, true, false, false, aggOrOverFinder, nameMatcher); - groupFinder = new AggFinder(opTab, false, false, true, null, nameMatcher); - aggOrOverOrGroupFinder = new AggFinder(opTab, true, true, true, null, nameMatcher); - this.enableTypeCoercion = catalogReader.getConfig() == null - || catalogReader.getConfig().typeCoercion(); - this.typeCoercion = TypeCoercions.getTypeCoercion(this, conformance); - } - - //~ Methods ---------------------------------------------------------------- - - public SqlConformance getConformance() { - return conformance; - } - - public SqlValidatorCatalogReader getCatalogReader() { - return catalogReader; - } - - public SqlOperatorTable getOperatorTable() { - return opTab; - } - - public RelDataTypeFactory getTypeFactory() { - return typeFactory; - } - - public RelDataType getUnknownType() { - return unknownType; - } - - public SqlNodeList expandStar( - SqlNodeList selectList, - SqlSelect select, - boolean includeSystemVars) { - final List list = new ArrayList<>(); - final List> types = new ArrayList<>(); - for (int i = 0; i < selectList.size(); i++) { - final SqlNode selectItem = selectList.get(i); - final RelDataType originalType = getValidatedNodeTypeIfKnown(selectItem); - expandSelectItem( - selectItem, - select, - Util.first(originalType, unknownType), - list, - catalogReader.nameMatcher().createSet(), - types, - includeSystemVars); - } - getRawSelectScope(select).setExpandedSelectList(list); - return new SqlNodeList(list, SqlParserPos.ZERO); - } - - // implement SqlValidator - public void declareCursor(SqlSelect select, SqlValidatorScope parentScope) { - cursorSet.add(select); - - // add the cursor to a map that maps the cursor to its select based on - // the position of the cursor relative to other cursors in that call - FunctionParamInfo funcParamInfo = functionCallStack.peek(); - Map cursorMap = funcParamInfo.cursorPosToSelectMap; - int numCursors = cursorMap.size(); - cursorMap.put(numCursors, select); - - // create a namespace associated with the result of the select - // that is the argument to the cursor constructor; register it - // with a scope corresponding to the cursor - SelectScope cursorScope = new SelectScope(parentScope, null, select); - cursorScopes.put(select, cursorScope); - final SelectNamespace selectNs = createSelectNamespace(select, select); - String alias = deriveAlias(select, nextGeneratedId++); - registerNamespace(cursorScope, alias, selectNs, false); - } - - // implement SqlValidator - public void pushFunctionCall() { - FunctionParamInfo funcInfo = new FunctionParamInfo(); - functionCallStack.push(funcInfo); - } - - // implement SqlValidator - public void popFunctionCall() { - functionCallStack.pop(); - } - - // implement SqlValidator - public String getParentCursor(String columnListParamName) { - FunctionParamInfo funcParamInfo = functionCallStack.peek(); - Map parentCursorMap = - funcParamInfo.columnListParamToParentCursorMap; - return parentCursorMap.get(columnListParamName); - } - - /** - * If selectItem is "*" or "TABLE.*", expands it and returns - * true; otherwise writes the unexpanded item. - * - * @param selectItem Select-list item - * @param select Containing select clause - * @param selectItems List that expanded items are written to - * @param aliases Set of aliases - * @param fields List of field names and types, in alias order - * @param includeSystemVars If true include system vars in lists - * @return Whether the node was expanded - */ - private boolean expandSelectItem( - final SqlNode selectItem, - SqlSelect select, - RelDataType targetType, - List selectItems, - Set aliases, - List> fields, - final boolean includeSystemVars) { - final SelectScope scope = (SelectScope) getWhereScope(select); - if (expandStar(selectItems, aliases, fields, includeSystemVars, scope, - selectItem)) { - return true; - } - - // Expand the select item: fully-qualify columns, and convert - // parentheses-free functions such as LOCALTIME into explicit function - // calls. - SqlNode expanded = expand(selectItem, scope); - final String alias = - deriveAlias( - selectItem, - aliases.size()); - - // If expansion has altered the natural alias, supply an explicit 'AS'. - final SqlValidatorScope selectScope = getSelectScope(select); - if (expanded != selectItem) { - String newAlias = - deriveAlias( - expanded, - aliases.size()); - if (!newAlias.equals(alias)) { - expanded = - SqlStdOperatorTable.AS.createCall( - selectItem.getParserPosition(), - expanded, - new SqlIdentifier(alias, SqlParserPos.ZERO)); - deriveTypeImpl(selectScope, expanded); - } - } - - selectItems.add(expanded); - aliases.add(alias); - - if (expanded != null) { - inferUnknownTypes(targetType, scope, expanded); - } - final RelDataType type = deriveType(selectScope, expanded); - setValidatedNodeType(expanded, type); - fields.add(Pair.of(alias, type)); - return false; - } - - private boolean expandStar(List selectItems, Set aliases, - List> fields, boolean includeSystemVars, - SelectScope scope, SqlNode node) { - if (!(node instanceof SqlIdentifier)) { - return false; - } - final SqlIdentifier identifier = (SqlIdentifier) node; - if (!identifier.isStar()) { - return false; - } - final SqlParserPos startPosition = identifier.getParserPosition(); - switch (identifier.names.size()) { - case 1: - boolean hasDynamicStruct = false; - for (ScopeChild child : scope.children) { - final int before = fields.size(); - if (child.namespace.getRowType().isDynamicStruct()) { - hasDynamicStruct = true; - // don't expand star if the underneath table is dynamic. - // Treat this star as a special field in validation/conversion and - // wait until execution time to expand this star. - final SqlNode exp = - new SqlIdentifier( - ImmutableList.of(child.name, - DynamicRecordType.DYNAMIC_STAR_PREFIX), - startPosition); - addToSelectList( - selectItems, - aliases, - fields, - exp, - scope, - includeSystemVars); - } else { - final SqlNode from = child.namespace.getNode(); - final SqlValidatorNamespace fromNs = getNamespace(from, scope); - assert fromNs != null; - final RelDataType rowType = fromNs.getRowType(); - for (RelDataTypeField field : rowType.getFieldList()) { - String columnName = field.getName(); - - // TODO: do real implicit collation here - final SqlIdentifier exp = - new SqlIdentifier( - ImmutableList.of(child.name, columnName), - startPosition); - // Don't add expanded rolled up columns - if (!isRolledUpColumn(exp, scope)) { - addOrExpandField( - selectItems, - aliases, - fields, - includeSystemVars, - scope, - exp, - field); - } - } - } - if (child.nullable) { - for (int i = before; i < fields.size(); i++) { - final Map.Entry entry = fields.get(i); - final RelDataType type = entry.getValue(); - if (!type.isNullable()) { - fields.set(i, - Pair.of(entry.getKey(), - typeFactory.createTypeWithNullability(type, true))); - } - } - } - } - // If NATURAL JOIN or USING is present, move key fields to the front of - // the list, per standard SQL. Disabled if there are dynamic fields. - if (!hasDynamicStruct || Bug.CALCITE_2400_FIXED) { - new Permute(scope.getNode().getFrom(), 0).permute(selectItems, fields); - } - return true; - - default: - final SqlIdentifier prefixId = identifier.skipLast(1); - final SqlValidatorScope.ResolvedImpl resolved = - new SqlValidatorScope.ResolvedImpl(); - final SqlNameMatcher nameMatcher = - scope.validator.catalogReader.nameMatcher(); - scope.resolve(prefixId.names, nameMatcher, true, resolved); - if (resolved.count() == 0) { - // e.g. "select s.t.* from e" - // or "select r.* from e" - throw newValidationError(prefixId, - RESOURCE.unknownIdentifier(prefixId.toString())); - } - final RelDataType rowType = resolved.only().rowType(); - if (rowType.isDynamicStruct()) { - // don't expand star if the underneath table is dynamic. - addToSelectList( - selectItems, - aliases, - fields, - prefixId.plus(DynamicRecordType.DYNAMIC_STAR_PREFIX, startPosition), - scope, - includeSystemVars); - } else if (rowType.isStruct()) { - for (RelDataTypeField field : rowType.getFieldList()) { - String columnName = field.getName(); - - // TODO: do real implicit collation here - addOrExpandField( - selectItems, - aliases, - fields, - includeSystemVars, - scope, - prefixId.plus(columnName, startPosition), - field); - } - } else { - throw newValidationError(prefixId, RESOURCE.starRequiresRecordType()); - } - return true; - } - } - - private SqlNode maybeCast(SqlNode node, RelDataType currentType, - RelDataType desiredType) { - return currentType.equals(desiredType) - || (currentType.isNullable() != desiredType.isNullable() - && typeFactory.createTypeWithNullability(currentType, - desiredType.isNullable()).equals(desiredType)) - ? node - : SqlStdOperatorTable.CAST.createCall(SqlParserPos.ZERO, - node, SqlTypeUtil.convertTypeToSpec(desiredType)); - } - - private boolean addOrExpandField(List selectItems, Set aliases, - List> fields, boolean includeSystemVars, - SelectScope scope, SqlIdentifier id, RelDataTypeField field) { - switch (field.getType().getStructKind()) { - case PEEK_FIELDS: - case PEEK_FIELDS_DEFAULT: - final SqlNode starExp = id.plusStar(); - expandStar( - selectItems, - aliases, - fields, - includeSystemVars, - scope, - starExp); - return true; - - default: - addToSelectList( - selectItems, - aliases, - fields, - id, - scope, - includeSystemVars); - } - - return false; - } - - public SqlNode validate(SqlNode topNode) { - SqlValidatorScope scope = new EmptyScope(this); - scope = new CatalogScope(scope, ImmutableList.of("CATALOG")); - final SqlNode topNode2 = validateScopedExpression(topNode, scope); - final RelDataType type = getValidatedNodeType(topNode2); - Util.discard(type); - return topNode2; - } - - public List lookupHints(SqlNode topNode, SqlParserPos pos) { - SqlValidatorScope scope = new EmptyScope(this); - SqlNode outermostNode = performUnconditionalRewrites(topNode, false); - cursorSet.add(outermostNode); - if (outermostNode.isA(SqlKind.TOP_LEVEL)) { - registerQuery( - scope, - null, - outermostNode, - outermostNode, - null, - false); - } - final SqlValidatorNamespace ns = getNamespace(outermostNode); - if (ns == null) { - throw new AssertionError("Not a query: " + outermostNode); - } - Collection hintList = Sets.newTreeSet(SqlMoniker.COMPARATOR); - lookupSelectHints(ns, pos, hintList); - return ImmutableList.copyOf(hintList); - } - - public SqlMoniker lookupQualifiedName(SqlNode topNode, SqlParserPos pos) { - final String posString = pos.toString(); - IdInfo info = idPositions.get(posString); - if (info != null) { - final SqlQualified qualified = info.scope.fullyQualify(info.id); - return new SqlIdentifierMoniker(qualified.identifier); - } else { - return null; - } - } - - /** - * Looks up completion hints for a syntactically correct select SQL that has - * been parsed into an expression tree. - * - * @param select the Select node of the parsed expression tree - * @param pos indicates the position in the sql statement we want to get - * completion hints for - * @param hintList list of {@link SqlMoniker} (sql identifiers) that can - * fill in at the indicated position - */ - void lookupSelectHints( - SqlSelect select, - SqlParserPos pos, - Collection hintList) { - IdInfo info = idPositions.get(pos.toString()); - if ((info == null) || (info.scope == null)) { - SqlNode fromNode = select.getFrom(); - final SqlValidatorScope fromScope = getFromScope(select); - lookupFromHints(fromNode, fromScope, pos, hintList); - } else { - lookupNameCompletionHints(info.scope, info.id.names, - info.id.getParserPosition(), hintList); - } - } - - private void lookupSelectHints( - SqlValidatorNamespace ns, - SqlParserPos pos, - Collection hintList) { - final SqlNode node = ns.getNode(); - if (node instanceof SqlSelect) { - lookupSelectHints((SqlSelect) node, pos, hintList); - } - } - - private void lookupFromHints( - SqlNode node, - SqlValidatorScope scope, - SqlParserPos pos, - Collection hintList) { - if (node == null) { - // This can happen in cases like "select * _suggest_", so from clause is absent - return; - } - final SqlValidatorNamespace ns = getNamespace(node); - if (ns.isWrapperFor(IdentifierNamespace.class)) { - IdentifierNamespace idNs = ns.unwrap(IdentifierNamespace.class); - final SqlIdentifier id = idNs.getId(); - for (int i = 0; i < id.names.size(); i++) { - if (pos.toString().equals( - id.getComponent(i).getParserPosition().toString())) { - final List objNames = new ArrayList<>(); - SqlValidatorUtil.getSchemaObjectMonikers( - getCatalogReader(), - id.names.subList(0, i + 1), - objNames); - for (SqlMoniker objName : objNames) { - if (objName.getType() != SqlMonikerType.FUNCTION) { - hintList.add(objName); - } - } - return; - } - } - } - switch (node.getKind()) { - case JOIN: - lookupJoinHints((SqlJoin) node, scope, pos, hintList); - break; - default: - lookupSelectHints(ns, pos, hintList); - break; - } - } - - private void lookupJoinHints( - SqlJoin join, - SqlValidatorScope scope, - SqlParserPos pos, - Collection hintList) { - SqlNode left = join.getLeft(); - SqlNode right = join.getRight(); - SqlNode condition = join.getCondition(); - lookupFromHints(left, scope, pos, hintList); - if (hintList.size() > 0) { - return; - } - lookupFromHints(right, scope, pos, hintList); - if (hintList.size() > 0) { - return; - } - final JoinConditionType conditionType = join.getConditionType(); - final SqlValidatorScope joinScope = scopes.get(join); - switch (conditionType) { - case ON: - condition.findValidOptions(this, joinScope, pos, hintList); - return; - default: - - // No suggestions. - // Not supporting hints for other types such as 'Using' yet. - return; - } - } - - /** - * Populates a list of all the valid alternatives for an identifier. - * - * @param scope Validation scope - * @param names Components of the identifier - * @param pos position - * @param hintList a list of valid options - */ - public final void lookupNameCompletionHints( - SqlValidatorScope scope, - List names, - SqlParserPos pos, - Collection hintList) { - // Remove the last part of name - it is a dummy - List subNames = Util.skipLast(names); - - if (subNames.size() > 0) { - // If there's a prefix, resolve it to a namespace. - SqlValidatorNamespace ns = null; - for (String name : subNames) { - if (ns == null) { - final SqlValidatorScope.ResolvedImpl resolved = - new SqlValidatorScope.ResolvedImpl(); - final SqlNameMatcher nameMatcher = catalogReader.nameMatcher(); - scope.resolve(ImmutableList.of(name), nameMatcher, false, resolved); - if (resolved.count() == 1) { - ns = resolved.only().namespace; - } - } else { - ns = ns.lookupChild(name); - } - if (ns == null) { - break; - } - } - if (ns != null) { - RelDataType rowType = ns.getRowType(); - if (rowType.isStruct()) { - for (RelDataTypeField field : rowType.getFieldList()) { - hintList.add( - new SqlMonikerImpl( - field.getName(), - SqlMonikerType.COLUMN)); - } - } - } - - // builtin function names are valid completion hints when the - // identifier has only 1 name part - findAllValidFunctionNames(names, this, hintList, pos); - } else { - // No prefix; use the children of the current scope (that is, - // the aliases in the FROM clause) - scope.findAliases(hintList); - - // If there's only one alias, add all child columns - SelectScope selectScope = - SqlValidatorUtil.getEnclosingSelectScope(scope); - if ((selectScope != null) - && (selectScope.getChildren().size() == 1)) { - RelDataType rowType = - selectScope.getChildren().get(0).getRowType(); - for (RelDataTypeField field : rowType.getFieldList()) { - hintList.add( - new SqlMonikerImpl( - field.getName(), - SqlMonikerType.COLUMN)); - } - } - } - - findAllValidUdfNames(names, this, hintList); - } - - private static void findAllValidUdfNames( - List names, - SqlValidator validator, - Collection result) { - final List objNames = new ArrayList<>(); - SqlValidatorUtil.getSchemaObjectMonikers( - validator.getCatalogReader(), - names, - objNames); - for (SqlMoniker objName : objNames) { - if (objName.getType() == SqlMonikerType.FUNCTION) { - result.add(objName); - } - } - } - - private static void findAllValidFunctionNames( - List names, - SqlValidator validator, - Collection result, - SqlParserPos pos) { - // a function name can only be 1 part - if (names.size() > 1) { - return; - } - for (SqlOperator op : validator.getOperatorTable().getOperatorList()) { - SqlIdentifier curOpId = - new SqlIdentifier( - op.getName(), - pos); - - final SqlCall call = validator.makeNullaryCall(curOpId); - if (call != null) { - result.add( - new SqlMonikerImpl( - op.getName(), - SqlMonikerType.FUNCTION)); - } else { - if ((op.getSyntax() == SqlSyntax.FUNCTION) - || (op.getSyntax() == SqlSyntax.PREFIX)) { - if (op.getOperandTypeChecker() != null) { - String sig = op.getAllowedSignatures(); - sig = sig.replaceAll("'", ""); - result.add( - new SqlMonikerImpl( - sig, - SqlMonikerType.FUNCTION)); - continue; - } - result.add( - new SqlMonikerImpl( - op.getName(), - SqlMonikerType.FUNCTION)); - } - } - } - } - - public SqlNode validateParameterizedExpression( - SqlNode topNode, - final Map nameToTypeMap) { - SqlValidatorScope scope = new ParameterScope(this, nameToTypeMap); - return validateScopedExpression(topNode, scope); - } - - private SqlNode validateScopedExpression( - SqlNode topNode, - SqlValidatorScope scope) { - SqlNode outermostNode = performUnconditionalRewrites(topNode, false); - cursorSet.add(outermostNode); - top = outermostNode; - TRACER.trace("After unconditional rewrite: {}", outermostNode); - if (outermostNode.isA(SqlKind.TOP_LEVEL)) { - registerQuery(scope, null, outermostNode, outermostNode, null, false); - } - outermostNode.validate(this, scope); - if (!outermostNode.isA(SqlKind.TOP_LEVEL)) { - // force type derivation so that we can provide it to the - // caller later without needing the scope - deriveType(scope, outermostNode); - } - TRACER.trace("After validation: {}", outermostNode); - return outermostNode; - } - - public void validateQuery(SqlNode node, SqlValidatorScope scope, - RelDataType targetRowType) { - final SqlValidatorNamespace ns = getNamespace(node, scope); - if (node.getKind() == SqlKind.TABLESAMPLE) { - List operands = ((SqlCall) node).getOperandList(); - SqlSampleSpec sampleSpec = SqlLiteral.sampleValue(operands.get(1)); - if (sampleSpec instanceof SqlSampleSpec.SqlTableSampleSpec) { - validateFeature(RESOURCE.sQLFeature_T613(), node.getParserPosition()); - } else if (sampleSpec - instanceof SqlSampleSpec.SqlSubstitutionSampleSpec) { - validateFeature(RESOURCE.sQLFeatureExt_T613_Substitution(), - node.getParserPosition()); - } - } - - validateNamespace(ns, targetRowType); - switch (node.getKind()) { - case EXTEND: - // Until we have a dedicated namespace for EXTEND - deriveType(scope, node); - } - if (node == top) { - validateModality(node); - } - validateAccess( - node, - ns.getTable(), - SqlAccessEnum.SELECT); - - if (node.getKind() == SqlKind.SNAPSHOT) { - SqlSnapshot snapshot = (SqlSnapshot) node; - SqlNode period = snapshot.getPeriod(); - RelDataType dataType = deriveType(scope, period); - if (dataType.getSqlTypeName() != SqlTypeName.TIMESTAMP) { - throw newValidationError(period, - Static.RESOURCE.illegalExpressionForTemporal(dataType.getSqlTypeName().getName())); - } - if (!ns.getTable().isTemporal()) { - List qualifiedName = ns.getTable().getQualifiedName(); - String tableName = qualifiedName.get(qualifiedName.size() - 1); - throw newValidationError(snapshot.getTableRef(), - Static.RESOURCE.notTemporalTable(tableName)); - } - } - } - - /** - * Validates a namespace. - * - * @param namespace Namespace - * @param targetRowType Desired row type, must not be null, may be the data - * type 'unknown'. - */ - protected void validateNamespace(final SqlValidatorNamespace namespace, - RelDataType targetRowType) { - namespace.validate(targetRowType); - if (namespace.getNode() != null) { - setValidatedNodeType(namespace.getNode(), namespace.getType()); - } - } - - @VisibleForTesting - public SqlValidatorScope getEmptyScope() { - return new EmptyScope(this); - } - - public SqlValidatorScope getCursorScope(SqlSelect select) { - return cursorScopes.get(select); - } - - public SqlValidatorScope getWhereScope(SqlSelect select) { - return whereScopes.get(select); - } - - public SqlValidatorScope getSelectScope(SqlSelect select) { - return selectScopes.get(select); - } - - public SelectScope getRawSelectScope(SqlSelect select) { - SqlValidatorScope scope = getSelectScope(select); - if (scope instanceof AggregatingSelectScope) { - scope = ((AggregatingSelectScope) scope).getParent(); - } - return (SelectScope) scope; - } - - public SqlValidatorScope getHavingScope(SqlSelect select) { - // Yes, it's the same as getSelectScope - return selectScopes.get(select); - } - - public SqlValidatorScope getGroupScope(SqlSelect select) { - // Yes, it's the same as getWhereScope - return groupByScopes.get(select); - } - - public SqlValidatorScope getFromScope(SqlSelect select) { - return scopes.get(select); - } - - public SqlValidatorScope getOrderScope(SqlSelect select) { - return orderScopes.get(select); - } - - public SqlValidatorScope getMatchRecognizeScope(SqlMatchRecognize node) { - return scopes.get(node); - } - - public SqlValidatorScope getJoinScope(SqlNode node) { - return scopes.get(stripAs(node)); - } - - public SqlValidatorScope getOverScope(SqlNode node) { - return scopes.get(node); - } - - private SqlValidatorNamespace getNamespace(SqlNode node, - SqlValidatorScope scope) { - if (node instanceof SqlIdentifier && scope instanceof DelegatingScope) { - final SqlIdentifier id = (SqlIdentifier) node; - final DelegatingScope idScope = (DelegatingScope) ((DelegatingScope) scope).getParent(); - return getNamespace(id, idScope); - } else if (node instanceof SqlCall) { - // Handle extended identifiers. - final SqlCall call = (SqlCall) node; - switch (call.getOperator().getKind()) { - case EXTEND: - final SqlIdentifier id = (SqlIdentifier) call.getOperandList().get(0); - final DelegatingScope idScope = (DelegatingScope) scope; - return getNamespace(id, idScope); - case AS: - final SqlNode nested = call.getOperandList().get(0); - switch (nested.getKind()) { - case EXTEND: - return getNamespace(nested, scope); - } - break; - } - } - return getNamespace(node); - } - - private SqlValidatorNamespace getNamespace(SqlIdentifier id, DelegatingScope scope) { - if (id.isSimple()) { - final SqlNameMatcher nameMatcher = catalogReader.nameMatcher(); - final SqlValidatorScope.ResolvedImpl resolved = - new SqlValidatorScope.ResolvedImpl(); - scope.resolve(id.names, nameMatcher, false, resolved); - if (resolved.count() == 1) { - return resolved.only().namespace; - } - } - return getNamespace(id); - } - - public SqlValidatorNamespace getNamespace(SqlNode node) { - switch (node.getKind()) { - case AS: - - // AS has a namespace if it has a column list 'AS t (c1, c2, ...)' - final SqlValidatorNamespace ns = namespaces.get(node); - if (ns != null) { - return ns; - } - // fall through - case SNAPSHOT: - case OVER: - case COLLECTION_TABLE: - case ORDER_BY: - case TABLESAMPLE: - return getNamespace(((SqlCall) node).operand(0)); - default: - return namespaces.get(node); - } - } - - private void handleOffsetFetch(SqlNode offset, SqlNode fetch) { - if (offset instanceof SqlDynamicParam) { - setValidatedNodeType(offset, - typeFactory.createSqlType(SqlTypeName.INTEGER)); - } - if (fetch instanceof SqlDynamicParam) { - setValidatedNodeType(fetch, - typeFactory.createSqlType(SqlTypeName.INTEGER)); - } - } - - /** - * Performs expression rewrites which are always used unconditionally. These - * rewrites massage the expression tree into a standard form so that the - * rest of the validation logic can be simpler. - * - * @param node expression to be rewritten - * @param underFrom whether node appears directly under a FROM clause - * @return rewritten expression - */ - protected SqlNode performUnconditionalRewrites( - SqlNode node, - boolean underFrom) { - if (node == null) { - return node; - } - - SqlNode newOperand; - - // first transform operands and invoke generic call rewrite - if (node instanceof SqlCall) { - if (node instanceof SqlMerge) { - validatingSqlMerge = true; - } - SqlCall call = (SqlCall) node; - final SqlKind kind = call.getKind(); - final List operands = call.getOperandList(); - for (int i = 0; i < operands.size(); i++) { - SqlNode operand = operands.get(i); - boolean childUnderFrom; - if (kind == SqlKind.SELECT) { - childUnderFrom = i == SqlSelect.FROM_OPERAND; - } else if (kind == SqlKind.AS && (i == 0)) { - // for an aliased expression, it is under FROM if - // the AS expression is under FROM - childUnderFrom = underFrom; - } else { - childUnderFrom = false; - } - newOperand = - performUnconditionalRewrites(operand, childUnderFrom); - if (newOperand != null && newOperand != operand) { - call.setOperand(i, newOperand); - } - } - - if (call.getOperator() instanceof SqlUnresolvedFunction) { - assert call instanceof SqlBasicCall; - final SqlUnresolvedFunction function = - (SqlUnresolvedFunction) call.getOperator(); - // This function hasn't been resolved yet. Perform - // a half-hearted resolution now in case it's a - // builtin function requiring special casing. If it's - // not, we'll handle it later during overload resolution. - final List overloads = new ArrayList<>(); - opTab.lookupOperatorOverloads(function.getNameAsId(), - function.getFunctionType(), SqlSyntax.FUNCTION, overloads, - catalogReader.nameMatcher()); - if (overloads.size() == 1) { - ((SqlBasicCall) call).setOperator(overloads.get(0)); - } - } - if (rewriteCalls) { - node = call.getOperator().rewriteCall(this, call); - } - } else if (node instanceof SqlNodeList) { - SqlNodeList list = (SqlNodeList) node; - for (int i = 0, count = list.size(); i < count; i++) { - SqlNode operand = list.get(i); - newOperand = - performUnconditionalRewrites( - operand, - false); - if (newOperand != null) { - list.getList().set(i, newOperand); - } - } - } - - // now transform node itself - final SqlKind kind = node.getKind(); - switch (kind) { - case VALUES: - // CHECKSTYLE: IGNORE 1 - if (underFrom || true) { - // leave FROM (VALUES(...)) [ AS alias ] clauses alone, - // otherwise they grow cancerously if this rewrite is invoked - // over and over - return node; - } else { - final SqlNodeList selectList = - new SqlNodeList(SqlParserPos.ZERO); - selectList.add(SqlIdentifier.star(SqlParserPos.ZERO)); - return new SqlSelect(node.getParserPosition(), null, selectList, node, - null, null, null, null, null, null, null); - } - - case ORDER_BY: { - SqlOrderBy orderBy = (SqlOrderBy) node; - handleOffsetFetch(orderBy.offset, orderBy.fetch); - if (orderBy.query instanceof SqlSelect) { - SqlSelect select = (SqlSelect) orderBy.query; - - // Don't clobber existing ORDER BY. It may be needed for - // an order-sensitive function like RANK. - if (select.getOrderList() == null) { - // push ORDER BY into existing select - select.setOrderBy(orderBy.orderList); - select.setOffset(orderBy.offset); - select.setFetch(orderBy.fetch); - return select; - } - } - if (orderBy.query instanceof SqlWith - && ((SqlWith) orderBy.query).body instanceof SqlSelect) { - SqlWith with = (SqlWith) orderBy.query; - SqlSelect select = (SqlSelect) with.body; - - // Don't clobber existing ORDER BY. It may be needed for - // an order-sensitive function like RANK. - if (select.getOrderList() == null) { - // push ORDER BY into existing select - select.setOrderBy(orderBy.orderList); - select.setOffset(orderBy.offset); - select.setFetch(orderBy.fetch); - return with; - } - } - final SqlNodeList selectList = new SqlNodeList(SqlParserPos.ZERO); - selectList.add(SqlIdentifier.star(SqlParserPos.ZERO)); - final SqlNodeList orderList; - if (getInnerSelect(node) != null && isAggregate(getInnerSelect(node))) { - orderList = SqlNode.clone(orderBy.orderList); - // We assume that ORDER BY item does not have ASC etc. - // We assume that ORDER BY item is present in SELECT list. - for (int i = 0; i < orderList.size(); i++) { - SqlNode sqlNode = orderList.get(i); - SqlNodeList selectList2 = getInnerSelect(node).getSelectList(); - for (Ord sel : Ord.zip(selectList2)) { - if (stripAs(sel.e).equalsDeep(sqlNode, Litmus.IGNORE)) { - orderList.set(i, - SqlLiteral.createExactNumeric(Integer.toString(sel.i + 1), - SqlParserPos.ZERO)); - } - } - } - } else { - orderList = orderBy.orderList; - } - return new SqlSelect(SqlParserPos.ZERO, null, selectList, orderBy.query, - null, null, null, null, orderList, orderBy.offset, - orderBy.fetch); - } - - case EXPLICIT_TABLE: { - // (TABLE t) is equivalent to (SELECT * FROM t) - SqlCall call = (SqlCall) node; - final SqlNodeList selectList = new SqlNodeList(SqlParserPos.ZERO); - selectList.add(SqlIdentifier.star(SqlParserPos.ZERO)); - return new SqlSelect(SqlParserPos.ZERO, null, selectList, call.operand(0), - null, null, null, null, null, null, null); - } - - case DELETE: { - SqlDelete call = (SqlDelete) node; - SqlSelect select = createSourceSelectForDelete(call); - call.setSourceSelect(select); - break; - } - - case UPDATE: { - SqlUpdate call = (SqlUpdate) node; - SqlSelect select = createSourceSelectForUpdate(call); - call.setSourceSelect(select); - - // See if we're supposed to rewrite UPDATE to MERGE - // (unless this is the UPDATE clause of a MERGE, - // in which case leave it alone). - if (!validatingSqlMerge) { - SqlNode selfJoinSrcExpr = - getSelfJoinExprForUpdate( - call.getTargetTable(), - UPDATE_SRC_ALIAS); - if (selfJoinSrcExpr != null) { - node = rewriteUpdateToMerge(call, selfJoinSrcExpr); - } - } - break; - } - - case MERGE: { - SqlMerge call = (SqlMerge) node; - rewriteMerge(call); - break; - } - } - return node; - } - - private SqlSelect getInnerSelect(SqlNode node) { - for (;;) { - if (node instanceof SqlSelect) { - return (SqlSelect) node; - } else if (node instanceof SqlOrderBy) { - node = ((SqlOrderBy) node).query; - } else if (node instanceof SqlWith) { - node = ((SqlWith) node).body; - } else { - return null; - } - } - } - - private void rewriteMerge(SqlMerge call) { - SqlNodeList selectList; - SqlUpdate updateStmt = call.getUpdateCall(); - if (updateStmt != null) { - // if we have an update statement, just clone the select list - // from the update statement's source since it's the same as - // what we want for the select list of the merge source -- '*' - // followed by the update set expressions - selectList = SqlNode.clone(updateStmt.getSourceSelect().getSelectList()); - } else { - // otherwise, just use select * - selectList = new SqlNodeList(SqlParserPos.ZERO); - selectList.add(SqlIdentifier.star(SqlParserPos.ZERO)); - } - SqlNode targetTable = call.getTargetTable(); - if (call.getAlias() != null) { - targetTable = - SqlValidatorUtil.addAlias( - targetTable, - call.getAlias().getSimple()); - } - - // Provided there is an insert substatement, the source select for - // the merge is a left outer join between the source in the USING - // clause and the target table; otherwise, the join is just an - // inner join. Need to clone the source table reference in order - // for validation to work - SqlNode sourceTableRef = call.getSourceTableRef(); - SqlInsert insertCall = call.getInsertCall(); - JoinType joinType = (insertCall == null) ? JoinType.INNER : JoinType.LEFT; - final SqlNode leftJoinTerm = SqlNode.clone(sourceTableRef); - SqlNode outerJoin = - new SqlJoin(SqlParserPos.ZERO, - leftJoinTerm, - SqlLiteral.createBoolean(false, SqlParserPos.ZERO), - joinType.symbol(SqlParserPos.ZERO), - targetTable, - JoinConditionType.ON.symbol(SqlParserPos.ZERO), - call.getCondition()); - SqlSelect select = - new SqlSelect(SqlParserPos.ZERO, null, selectList, outerJoin, null, - null, null, null, null, null, null); - call.setSourceSelect(select); - - // Source for the insert call is a select of the source table - // reference with the select list being the value expressions; - // note that the values clause has already been converted to a - // select on the values row constructor; so we need to extract - // that via the from clause on the select - if (insertCall != null) { - SqlCall valuesCall = (SqlCall) insertCall.getSource(); - SqlCall rowCall = valuesCall.operand(0); - selectList = - new SqlNodeList( - rowCall.getOperandList(), - SqlParserPos.ZERO); - final SqlNode insertSource = SqlNode.clone(sourceTableRef); - select = - new SqlSelect(SqlParserPos.ZERO, null, selectList, insertSource, null, - null, null, null, null, null, null); - insertCall.setSource(select); - } - } - - private SqlNode rewriteUpdateToMerge( - SqlUpdate updateCall, - SqlNode selfJoinSrcExpr) { - // Make sure target has an alias. - if (updateCall.getAlias() == null) { - updateCall.setAlias( - new SqlIdentifier(UPDATE_TGT_ALIAS, SqlParserPos.ZERO)); - } - SqlNode selfJoinTgtExpr = - getSelfJoinExprForUpdate( - updateCall.getTargetTable(), - updateCall.getAlias().getSimple()); - assert selfJoinTgtExpr != null; - - // Create join condition between source and target exprs, - // creating a conjunction with the user-level WHERE - // clause if one was supplied - SqlNode condition = updateCall.getCondition(); - SqlNode selfJoinCond = - SqlStdOperatorTable.EQUALS.createCall( - SqlParserPos.ZERO, - selfJoinSrcExpr, - selfJoinTgtExpr); - if (condition == null) { - condition = selfJoinCond; - } else { - condition = - SqlStdOperatorTable.AND.createCall( - SqlParserPos.ZERO, - selfJoinCond, - condition); - } - SqlNode target = - updateCall.getTargetTable().clone(SqlParserPos.ZERO); - - // For the source, we need to anonymize the fields, so - // that for a statement like UPDATE T SET I = I + 1, - // there's no ambiguity for the "I" in "I + 1"; - // this is OK because the source and target have - // identical values due to the self-join. - // Note that we anonymize the source rather than the - // target because downstream, the optimizer rules - // don't want to see any projection on top of the target. - IdentifierNamespace ns = - new IdentifierNamespace(this, target, null, null); - RelDataType rowType = ns.getRowType(); - SqlNode source = updateCall.getTargetTable().clone(SqlParserPos.ZERO); - final SqlNodeList selectList = new SqlNodeList(SqlParserPos.ZERO); - int i = 1; - for (RelDataTypeField field : rowType.getFieldList()) { - SqlIdentifier col = - new SqlIdentifier( - field.getName(), - SqlParserPos.ZERO); - selectList.add( - SqlValidatorUtil.addAlias(col, UPDATE_ANON_PREFIX + i)); - ++i; - } - source = - new SqlSelect(SqlParserPos.ZERO, null, selectList, source, null, null, - null, null, null, null, null); - source = SqlValidatorUtil.addAlias(source, UPDATE_SRC_ALIAS); - SqlMerge mergeCall = - new SqlMerge(updateCall.getParserPosition(), target, condition, source, - updateCall, null, null, updateCall.getAlias()); - rewriteMerge(mergeCall); - return mergeCall; - } - - /** - * Allows a subclass to provide information about how to convert an UPDATE - * into a MERGE via self-join. If this method returns null, then no such - * conversion takes place. Otherwise, this method should return a suitable - * unique identifier expression for the given table. - * - * @param table identifier for table being updated - * @param alias alias to use for qualifying columns in expression, or null - * for unqualified references; if this is equal to - * {@value #UPDATE_SRC_ALIAS}, then column references have been - * anonymized to "SYS$ANONx", where x is the 1-based column - * number. - * @return expression for unique identifier, or null to prevent conversion - */ - protected SqlNode getSelfJoinExprForUpdate( - SqlNode table, - String alias) { - return null; - } - - /** - * Creates the SELECT statement that putatively feeds rows into an UPDATE - * statement to be updated. - * - * @param call Call to the UPDATE operator - * @return select statement - */ - protected SqlSelect createSourceSelectForUpdate(SqlUpdate call) { - final SqlNodeList selectList = new SqlNodeList(SqlParserPos.ZERO); - selectList.add(SqlIdentifier.star(SqlParserPos.ZERO)); - int ordinal = 0; - for (SqlNode exp : call.getSourceExpressionList()) { - // Force unique aliases to avoid a duplicate for Y with - // SET X=Y - String alias = SqlUtil.deriveAliasFromOrdinal(ordinal); - selectList.add(SqlValidatorUtil.addAlias(exp, alias)); - ++ordinal; - } - SqlNode sourceTable = call.getTargetTable(); - if (call.getAlias() != null) { - sourceTable = - SqlValidatorUtil.addAlias( - sourceTable, - call.getAlias().getSimple()); - } - return new SqlSelect(SqlParserPos.ZERO, null, selectList, sourceTable, - call.getCondition(), null, null, null, null, null, null); - } - - /** - * Creates the SELECT statement that putatively feeds rows into a DELETE - * statement to be deleted. - * - * @param call Call to the DELETE operator - * @return select statement - */ - protected SqlSelect createSourceSelectForDelete(SqlDelete call) { - final SqlNodeList selectList = new SqlNodeList(SqlParserPos.ZERO); - selectList.add(SqlIdentifier.star(SqlParserPos.ZERO)); - SqlNode sourceTable = call.getTargetTable(); - if (call.getAlias() != null) { - sourceTable = - SqlValidatorUtil.addAlias( - sourceTable, - call.getAlias().getSimple()); - } - return new SqlSelect(SqlParserPos.ZERO, null, selectList, sourceTable, - call.getCondition(), null, null, null, null, null, null); - } - - /** - * Returns null if there is no common type. E.g. if the rows have a - * different number of columns. - */ - RelDataType getTableConstructorRowType( - SqlCall values, - SqlValidatorScope scope) { - final List rows = values.getOperandList(); - assert rows.size() >= 1; - final List rowTypes = new ArrayList<>(); - for (final SqlNode row : rows) { - assert row.getKind() == SqlKind.ROW; - SqlCall rowConstructor = (SqlCall) row; - - // REVIEW jvs 10-Sept-2003: Once we support single-row queries as - // rows, need to infer aliases from there. - final List aliasList = new ArrayList<>(); - final List typeList = new ArrayList<>(); - for (Ord column : Ord.zip(rowConstructor.getOperandList())) { - final String alias = deriveAlias(column.e, column.i); - aliasList.add(alias); - final RelDataType type = deriveType(scope, column.e); - typeList.add(type); - } - rowTypes.add(typeFactory.createStructType(typeList, aliasList)); - } - if (rows.size() == 1) { - // TODO jvs 10-Oct-2005: get rid of this workaround once - // leastRestrictive can handle all cases - return rowTypes.get(0); - } - return typeFactory.leastRestrictive(rowTypes); - } - - public RelDataType getValidatedNodeType(SqlNode node) { - RelDataType type = getValidatedNodeTypeIfKnown(node); - if (type == null) { - throw Util.needToImplement(node); - } else { - return type; - } - } - - public RelDataType getValidatedNodeTypeIfKnown(SqlNode node) { - final RelDataType type = nodeToTypeMap.get(node); - if (type != null) { - return type; - } - final SqlValidatorNamespace ns = getNamespace(node); - if (ns != null) { - return ns.getType(); - } - final SqlNode original = originalExprs.get(node); - if (original != null && original != node) { - return getValidatedNodeType(original); - } - if (node instanceof SqlIdentifier) { - return getCatalogReader().getNamedType((SqlIdentifier) node); - } - return null; - } - - /** - * Saves the type of a {@link SqlNode}, now that it has been validated. - * - *

    Unlike the base class method, this method is not deprecated. - * It is available from within Calcite, but is not part of the public API. - * - * @param node A SQL parse tree node, never null - * @param type Its type; must not be null - */ - @SuppressWarnings("deprecation") - public final void setValidatedNodeType(SqlNode node, RelDataType type) { - Objects.requireNonNull(type); - Objects.requireNonNull(node); - if (type.equals(unknownType)) { - // don't set anything until we know what it is, and don't overwrite - // a known type with the unknown type - return; - } - nodeToTypeMap.put(node, type); - } - - public void removeValidatedNodeType(SqlNode node) { - nodeToTypeMap.remove(node); - } - - @Nullable public SqlCall makeNullaryCall(SqlIdentifier id) { - if (id.names.size() == 1 && !id.isComponentQuoted(0)) { - final List list = new ArrayList<>(); - opTab.lookupOperatorOverloads(id, null, SqlSyntax.FUNCTION, list, - catalogReader.nameMatcher()); - for (SqlOperator operator : list) { - if (operator.getSyntax() == SqlSyntax.FUNCTION_ID) { - // Even though this looks like an identifier, it is a - // actually a call to a function. Construct a fake - // call to this function, so we can use the regular - // operator validation. - return new SqlBasicCall(operator, SqlNode.EMPTY_ARRAY, - id.getParserPosition(), true, null); - } - } - } - return null; - } - - public RelDataType deriveType( - SqlValidatorScope scope, - SqlNode expr) { - Objects.requireNonNull(scope); - Objects.requireNonNull(expr); - - // if we already know the type, no need to re-derive - RelDataType type = nodeToTypeMap.get(expr); - if (type != null) { - return type; - } - final SqlValidatorNamespace ns = getNamespace(expr); - if (ns != null) { - return ns.getType(); - } - type = deriveTypeImpl(scope, expr); - Preconditions.checkArgument( - type != null, - "SqlValidator.deriveTypeInternal returned null"); - setValidatedNodeType(expr, type); - return type; - } - - /** - * Derives the type of a node, never null. - */ - RelDataType deriveTypeImpl( - SqlValidatorScope scope, - SqlNode operand) { - DeriveTypeVisitor v = new DeriveTypeVisitor(scope); - final RelDataType type = operand.accept(v); - return Objects.requireNonNull(scope.nullifyType(operand, type)); - } - - public RelDataType deriveConstructorType( - SqlValidatorScope scope, - SqlCall call, - SqlFunction unresolvedConstructor, - SqlFunction resolvedConstructor, - List argTypes) { - SqlIdentifier sqlIdentifier = unresolvedConstructor.getSqlIdentifier(); - assert sqlIdentifier != null; - RelDataType type = catalogReader.getNamedType(sqlIdentifier); - if (type == null) { - // TODO jvs 12-Feb-2005: proper type name formatting - throw newValidationError(sqlIdentifier, - RESOURCE.unknownDatatypeName(sqlIdentifier.toString())); - } - - if (resolvedConstructor == null) { - if (call.operandCount() > 0) { - // This is not a default constructor invocation, and - // no user-defined constructor could be found - throw handleUnresolvedFunction(call, unresolvedConstructor, argTypes, - null); - } - } else { - SqlCall testCall = - resolvedConstructor.createCall( - call.getParserPosition(), - call.getOperandList()); - RelDataType returnType = - resolvedConstructor.validateOperands( - this, - scope, - testCall); - assert type == returnType; - } - - if (shouldExpandIdentifiers()) { - if (resolvedConstructor != null) { - ((SqlBasicCall) call).setOperator(resolvedConstructor); - } else { - // fake a fully-qualified call to the default constructor - ((SqlBasicCall) call).setOperator( - new SqlFunction( - type.getSqlIdentifier(), - ReturnTypes.explicit(type), - null, - null, - null, - SqlFunctionCategory.USER_DEFINED_CONSTRUCTOR)); - } - } - return type; - } - - public CalciteException handleUnresolvedFunction(SqlCall call, - SqlFunction unresolvedFunction, List argTypes, - List argNames) { - // For builtins, we can give a better error message - final List overloads = new ArrayList<>(); - opTab.lookupOperatorOverloads(unresolvedFunction.getNameAsId(), null, - SqlSyntax.FUNCTION, overloads, catalogReader.nameMatcher()); - if (overloads.size() == 1) { - SqlFunction fun = (SqlFunction) overloads.get(0); - if ((fun.getSqlIdentifier() == null) - && (fun.getSyntax() != SqlSyntax.FUNCTION_ID)) { - final int expectedArgCount = - fun.getOperandCountRange().getMin(); - throw newValidationError(call, - RESOURCE.invalidArgCount(call.getOperator().getName(), - expectedArgCount)); - } - } - - AssignableOperandTypeChecker typeChecking = - new AssignableOperandTypeChecker(argTypes, argNames); - String signature = - typeChecking.getAllowedSignatures( - unresolvedFunction, - unresolvedFunction.getName()); - throw newValidationError(call, - RESOURCE.validatorUnknownFunction(signature)); - } - - protected void inferUnknownTypes( - @Nonnull RelDataType inferredType, - @Nonnull SqlValidatorScope scope, - @Nonnull SqlNode node) { - Objects.requireNonNull(inferredType); - Objects.requireNonNull(scope); - Objects.requireNonNull(node); - final SqlValidatorScope newScope = scopes.get(node); - if (newScope != null) { - scope = newScope; - } - boolean isNullLiteral = SqlUtil.isNullLiteral(node, false); - if ((node instanceof SqlDynamicParam) || isNullLiteral) { - if (inferredType.equals(unknownType)) { - if (isNullLiteral) { - throw newValidationError(node, RESOURCE.nullIllegal()); - } else { - throw newValidationError(node, RESOURCE.dynamicParamIllegal()); - } - } - - // REVIEW: should dynamic parameter types always be nullable? - RelDataType newInferredType = - typeFactory.createTypeWithNullability(inferredType, true); - if (SqlTypeUtil.inCharFamily(inferredType)) { - newInferredType = - typeFactory.createTypeWithCharsetAndCollation( - newInferredType, - inferredType.getCharset(), - inferredType.getCollation()); - } - setValidatedNodeType(node, newInferredType); - } else if (node instanceof SqlNodeList) { - SqlNodeList nodeList = (SqlNodeList) node; - if (inferredType.isStruct()) { - if (inferredType.getFieldCount() != nodeList.size()) { - // this can happen when we're validating an INSERT - // where the source and target degrees are different; - // bust out, and the error will be detected higher up - return; - } - } - int i = 0; - for (SqlNode child : nodeList) { - RelDataType type; - if (inferredType.isStruct()) { - type = inferredType.getFieldList().get(i).getType(); - ++i; - } else { - type = inferredType; - } - inferUnknownTypes(type, scope, child); - } - } else if (node instanceof SqlCase) { - final SqlCase caseCall = (SqlCase) node; - - final RelDataType whenType = - caseCall.getValueOperand() == null ? booleanType : unknownType; - for (SqlNode sqlNode : caseCall.getWhenOperands().getList()) { - inferUnknownTypes(whenType, scope, sqlNode); - } - RelDataType returnType = deriveType(scope, node); - for (SqlNode sqlNode : caseCall.getThenOperands().getList()) { - inferUnknownTypes(returnType, scope, sqlNode); - } - - if (!SqlUtil.isNullLiteral(caseCall.getElseOperand(), false)) { - inferUnknownTypes( - returnType, - scope, - caseCall.getElseOperand()); - } else { - setValidatedNodeType(caseCall.getElseOperand(), returnType); - } - } else if (node.getKind() == SqlKind.AS) { - // For AS operator, only infer the operand not the alias - inferUnknownTypes(inferredType, scope, ((SqlCall) node).operand(0)); - } else if (node instanceof SqlCall) { - final SqlCall call = (SqlCall) node; - final SqlOperandTypeInference operandTypeInference = - call.getOperator().getOperandTypeInference(); - final SqlCallBinding callBinding = new SqlCallBinding(this, scope, call); - final List operands = callBinding.operands(); - final RelDataType[] operandTypes = new RelDataType[operands.size()]; - Arrays.fill(operandTypes, unknownType); - // TODO: eventually should assert(operandTypeInference != null) - // instead; for now just eat it - if (operandTypeInference != null) { - operandTypeInference.inferOperandTypes( - callBinding, - inferredType, - operandTypes); - } - for (int i = 0; i < operands.size(); ++i) { - final SqlNode operand = operands.get(i); - if (operand != null) { - inferUnknownTypes(operandTypes[i], scope, operand); - } - } - } - } - - /** - * Adds an expression to a select list, ensuring that its alias does not - * clash with any existing expressions on the list. - */ - protected void addToSelectList( - List list, - Set aliases, - List> fieldList, - SqlNode exp, - SqlValidatorScope scope, - final boolean includeSystemVars) { - String alias = SqlValidatorUtil.getAlias(exp, -1); - String uniqueAlias = - SqlValidatorUtil.uniquify( - alias, aliases, SqlValidatorUtil.EXPR_SUGGESTER); - if (!alias.equals(uniqueAlias)) { - exp = SqlValidatorUtil.addAlias(exp, uniqueAlias); - } - fieldList.add(Pair.of(uniqueAlias, deriveType(scope, exp))); - list.add(exp); - } - - public String deriveAlias( - SqlNode node, - int ordinal) { - return SqlValidatorUtil.getAlias(node, ordinal); - } - - // implement SqlValidator - public void setIdentifierExpansion(boolean expandIdentifiers) { - this.expandIdentifiers = expandIdentifiers; - } - - // implement SqlValidator - public void setColumnReferenceExpansion( - boolean expandColumnReferences) { - this.expandColumnReferences = expandColumnReferences; - } - - // implement SqlValidator - public boolean getColumnReferenceExpansion() { - return expandColumnReferences; - } - - public void setDefaultNullCollation(NullCollation nullCollation) { - this.nullCollation = Objects.requireNonNull(nullCollation); - } - - public NullCollation getDefaultNullCollation() { - return nullCollation; - } - - // implement SqlValidator - public void setCallRewrite(boolean rewriteCalls) { - this.rewriteCalls = rewriteCalls; - } - - public boolean shouldExpandIdentifiers() { - return expandIdentifiers; - } - - protected boolean shouldAllowIntermediateOrderBy() { - return true; - } - - private void registerMatchRecognize( - SqlValidatorScope parentScope, - SqlValidatorScope usingScope, - SqlMatchRecognize call, - SqlNode enclosingNode, - String alias, - boolean forceNullable) { - - final MatchRecognizeNamespace matchRecognizeNamespace = - createMatchRecognizeNameSpace(call, enclosingNode); - registerNamespace(usingScope, alias, matchRecognizeNamespace, forceNullable); - - final MatchRecognizeScope matchRecognizeScope = - new MatchRecognizeScope(parentScope, call); - scopes.put(call, matchRecognizeScope); - - // parse input query - SqlNode expr = call.getTableRef(); - SqlNode newExpr = registerFrom(usingScope, matchRecognizeScope, true, expr, - expr, null, null, forceNullable, false); - if (expr != newExpr) { - call.setOperand(0, newExpr); - } - } - - protected MatchRecognizeNamespace createMatchRecognizeNameSpace( - SqlMatchRecognize call, - SqlNode enclosingNode) { - return new MatchRecognizeNamespace(this, call, enclosingNode); - } - - /** - * Registers a new namespace, and adds it as a child of its parent scope. - * Derived class can override this method to tinker with namespaces as they - * are created. - * - * @param usingScope Parent scope (which will want to look for things in - * this namespace) - * @param alias Alias by which parent will refer to this namespace - * @param ns Namespace - * @param forceNullable Whether to force the type of namespace to be nullable - */ - protected void registerNamespace( - SqlValidatorScope usingScope, - String alias, - SqlValidatorNamespace ns, - boolean forceNullable) { - namespaces.put(ns.getNode(), ns); - if (usingScope != null) { - usingScope.addChild(ns, alias, forceNullable); - } - } - - /** - * Registers scopes and namespaces implied a relational expression in the - * FROM clause. - * - *

    {@code parentScope} and {@code usingScope} are often the same. They - * differ when the namespace are not visible within the parent. (Example - * needed.) - * - *

    Likewise, {@code enclosingNode} and {@code node} are often the same. - * {@code enclosingNode} is the topmost node within the FROM clause, from - * which any decorations like an alias (AS alias) or a table - * sample clause are stripped away to get {@code node}. Both are recorded in - * the namespace. - * - * @param parentScope Parent scope which this scope turns to in order to - * resolve objects - * @param usingScope Scope whose child list this scope should add itself to - * @param register Whether to register this scope as a child of - * {@code usingScope} - * @param node Node which namespace is based on - * @param enclosingNode Outermost node for namespace, including decorations - * such as alias and sample clause - * @param alias Alias - * @param extendList Definitions of extended columns - * @param forceNullable Whether to force the type of namespace to be - * nullable because it is in an outer join - * @param lateral Whether LATERAL is specified, so that items to the - * left of this in the JOIN tree are visible in the - * scope - * @return registered node, usually the same as {@code node} - */ - private SqlNode registerFrom( - SqlValidatorScope parentScope, - SqlValidatorScope usingScope, - boolean register, - final SqlNode node, - SqlNode enclosingNode, - String alias, - SqlNodeList extendList, - boolean forceNullable, - final boolean lateral) { - final SqlKind kind = node.getKind(); - - SqlNode expr; - SqlNode newExpr; - - // Add an alias if necessary. - SqlNode newNode = node; - if (alias == null) { - switch (kind) { - case IDENTIFIER: - case OVER: - alias = deriveAlias(node, -1); - if (alias == null) { - alias = deriveAlias(node, nextGeneratedId++); - } - if (shouldExpandIdentifiers()) { - newNode = SqlValidatorUtil.addAlias(node, alias); - } - break; - - case SELECT: - case UNION: - case INTERSECT: - case EXCEPT: - case VALUES: - case UNNEST: - case OTHER_FUNCTION: - case COLLECTION_TABLE: - case MATCH_RECOGNIZE: - - // give this anonymous construct a name since later - // query processing stages rely on it - alias = deriveAlias(node, nextGeneratedId++); - if (shouldExpandIdentifiers()) { - // Since we're expanding identifiers, we should make the - // aliases explicit too, otherwise the expanded query - // will not be consistent if we convert back to SQL, e.g. - // "select EXPR$1.EXPR$2 from values (1)". - newNode = SqlValidatorUtil.addAlias(node, alias); - } - break; - } - } - - if (lateral) { - SqlValidatorScope s = usingScope; - while (s instanceof JoinScope) { - s = ((JoinScope) s).getUsingScope(); - } - final SqlNode node2 = s != null ? s.getNode() : node; - final TableScope tableScope = new TableScope(parentScope, node2); - if (usingScope instanceof ListScope) { - for (ScopeChild child : ((ListScope) usingScope).children) { - tableScope.addChild(child.namespace, child.name, child.nullable); - } - } - parentScope = tableScope; - } - - SqlCall call; - SqlNode operand; - SqlNode newOperand; - - switch (kind) { - case AS: - call = (SqlCall) node; - if (alias == null) { - alias = call.operand(1).toString(); - } - final boolean needAlias = call.operandCount() > 2; - expr = call.operand(0); - newExpr = - registerFrom( - parentScope, - usingScope, - !needAlias, - expr, - enclosingNode, - alias, - extendList, - forceNullable, - lateral); - if (newExpr != expr) { - call.setOperand(0, newExpr); - } - - // If alias has a column list, introduce a namespace to translate - // column names. We skipped registering it just now. - if (needAlias) { - registerNamespace( - usingScope, - alias, - new AliasNamespace(this, call, enclosingNode), - forceNullable); - } - return node; - case MATCH_RECOGNIZE: - registerMatchRecognize(parentScope, usingScope, - (SqlMatchRecognize) node, enclosingNode, alias, forceNullable); - return node; - case TABLESAMPLE: - call = (SqlCall) node; - expr = call.operand(0); - newExpr = - registerFrom( - parentScope, - usingScope, - true, - expr, - enclosingNode, - alias, - extendList, - forceNullable, - lateral); - if (newExpr != expr) { - call.setOperand(0, newExpr); - } - return node; - - case JOIN: - final SqlJoin join = (SqlJoin) node; - final JoinScope joinScope = - new JoinScope(parentScope, usingScope, join); - scopes.put(join, joinScope); - final SqlNode left = join.getLeft(); - final SqlNode right = join.getRight(); - final boolean rightIsLateral = isLateral(right); - boolean forceLeftNullable = forceNullable; - boolean forceRightNullable = forceNullable; - switch (join.getJoinType()) { - case LEFT: - forceRightNullable = true; - break; - case RIGHT: - forceLeftNullable = true; - break; - case FULL: - forceLeftNullable = true; - forceRightNullable = true; - break; - } - final SqlNode newLeft = - registerFrom( - parentScope, - joinScope, - true, - left, - left, - null, - null, - forceLeftNullable, - lateral); - if (newLeft != left) { - join.setLeft(newLeft); - } - final SqlNode newRight = - registerFrom( - parentScope, - joinScope, - true, - right, - right, - null, - null, - forceRightNullable, - lateral); - if (newRight != right) { - join.setRight(newRight); - } - registerSubQueries(joinScope, join.getCondition()); - final JoinNamespace joinNamespace = new JoinNamespace(this, join); - registerNamespace(null, null, joinNamespace, forceNullable); - return join; - - case IDENTIFIER: - final SqlIdentifier id = (SqlIdentifier) node; - final IdentifierNamespace newNs = - new IdentifierNamespace( - this, id, extendList, enclosingNode, - parentScope); - registerNamespace(register ? usingScope : null, alias, newNs, - forceNullable); - if (tableScope == null) { - tableScope = new TableScope(parentScope, node); - } - tableScope.addChild(newNs, alias, forceNullable); - if (extendList != null && extendList.size() != 0) { - return enclosingNode; - } - return newNode; - - case LATERAL: - return registerFrom( - parentScope, - usingScope, - register, - ((SqlCall) node).operand(0), - enclosingNode, - alias, - extendList, - forceNullable, - true); - - case COLLECTION_TABLE: - call = (SqlCall) node; - operand = call.operand(0); - newOperand = - registerFrom( - parentScope, - usingScope, - register, - operand, - enclosingNode, - alias, - extendList, - forceNullable, lateral); - if (newOperand != operand) { - call.setOperand(0, newOperand); - } - scopes.put(node, parentScope); - return newNode; - - case UNNEST: - if (!lateral) { - return registerFrom(parentScope, usingScope, register, node, - enclosingNode, alias, extendList, forceNullable, true); - } - // fall through - case SELECT: - case UNION: - case INTERSECT: - case EXCEPT: - case VALUES: - case WITH: - case OTHER_FUNCTION: - if (alias == null) { - alias = deriveAlias(node, nextGeneratedId++); - } - registerQuery( - parentScope, - register ? usingScope : null, - node, - enclosingNode, - alias, - forceNullable); - return newNode; - - case OVER: - if (!shouldAllowOverRelation()) { - throw Util.unexpected(kind); - } - call = (SqlCall) node; - final OverScope overScope = new OverScope(usingScope, call); - scopes.put(call, overScope); - operand = call.operand(0); - newOperand = - registerFrom( - parentScope, - overScope, - true, - operand, - enclosingNode, - alias, - extendList, - forceNullable, - lateral); - if (newOperand != operand) { - call.setOperand(0, newOperand); - } - - for (ScopeChild child : overScope.children) { - registerNamespace(register ? usingScope : null, child.name, - child.namespace, forceNullable); - } - - return newNode; - - case EXTEND: - final SqlCall extend = (SqlCall) node; - return registerFrom(parentScope, - usingScope, - true, - extend.getOperandList().get(0), - extend, - alias, - (SqlNodeList) extend.getOperandList().get(1), - forceNullable, - lateral); - - case SNAPSHOT: - call = (SqlCall) node; - operand = call.operand(0); - newOperand = registerFrom( - tableScope == null ? parentScope : tableScope, - usingScope, - register, - operand, - enclosingNode, - alias, - extendList, - forceNullable, - true); - if (newOperand != operand) { - call.setOperand(0, newOperand); - } - scopes.put(node, parentScope); - return newNode; - - default: - throw Util.unexpected(kind); - } - } - - private static boolean isLateral(SqlNode node) { - switch (node.getKind()) { - case LATERAL: - case UNNEST: - // Per SQL std, UNNEST is implicitly LATERAL. - return true; - case AS: - return isLateral(((SqlCall) node).operand(0)); - default: - return false; - } - } - - protected boolean shouldAllowOverRelation() { - return false; - } - - /** - * Creates a namespace for a SELECT node. Derived class may - * override this factory method. - * - * @param select Select node - * @param enclosingNode Enclosing node - * @return Select namespace - */ - protected SelectNamespace createSelectNamespace( - SqlSelect select, - SqlNode enclosingNode) { - return new SelectNamespace(this, select, enclosingNode); - } - - /** - * Creates a namespace for a set operation (UNION, - * INTERSECT, or EXCEPT). Derived class may override - * this factory method. - * - * @param call Call to set operation - * @param enclosingNode Enclosing node - * @return Set operation namespace - */ - protected SetopNamespace createSetopNamespace( - SqlCall call, - SqlNode enclosingNode) { - return new SetopNamespace(this, call, enclosingNode); - } - - /** - * Registers a query in a parent scope. - * - * @param parentScope Parent scope which this scope turns to in order to - * resolve objects - * @param usingScope Scope whose child list this scope should add itself to - * @param node Query node - * @param alias Name of this query within its parent. Must be specified - * if usingScope != null - */ - private void registerQuery( - SqlValidatorScope parentScope, - SqlValidatorScope usingScope, - SqlNode node, - SqlNode enclosingNode, - String alias, - boolean forceNullable) { - Preconditions.checkArgument(usingScope == null || alias != null); - registerQuery( - parentScope, - usingScope, - node, - enclosingNode, - alias, - forceNullable, - true); - } - - /** - * Registers a query in a parent scope. - * - * @param parentScope Parent scope which this scope turns to in order to - * resolve objects - * @param usingScope Scope whose child list this scope should add itself to - * @param node Query node - * @param alias Name of this query within its parent. Must be specified - * if usingScope != null - * @param checkUpdate if true, validate that the update feature is supported - * if validating the update statement - */ - private void registerQuery( - SqlValidatorScope parentScope, - SqlValidatorScope usingScope, - SqlNode node, - SqlNode enclosingNode, - String alias, - boolean forceNullable, - boolean checkUpdate) { - Objects.requireNonNull(node); - Objects.requireNonNull(enclosingNode); - Preconditions.checkArgument(usingScope == null || alias != null); - - SqlCall call; - List operands; - switch (node.getKind()) { - case SELECT: - final SqlSelect select = (SqlSelect) node; - final SelectNamespace selectNs = - createSelectNamespace(select, enclosingNode); - registerNamespace(usingScope, alias, selectNs, forceNullable); - final SqlValidatorScope windowParentScope = - (usingScope != null) ? usingScope : parentScope; - SelectScope selectScope = - new SelectScope(parentScope, windowParentScope, select); - scopes.put(select, selectScope); - - // Start by registering the WHERE clause - whereScopes.put(select, selectScope); - registerOperandSubQueries( - selectScope, - select, - SqlSelect.WHERE_OPERAND); - - // Register FROM with the inherited scope 'parentScope', not - // 'selectScope', otherwise tables in the FROM clause would be - // able to see each other. - final SqlNode from = select.getFrom(); - if (from != null) { - final SqlNode newFrom = - registerFrom( - parentScope, - selectScope, - true, - from, - from, - null, - null, - false, - false); - if (newFrom != from) { - select.setFrom(newFrom); - } - } - - // If this is an aggregating query, the SELECT list and HAVING - // clause use a different scope, where you can only reference - // columns which are in the GROUP BY clause. - SqlValidatorScope aggScope = selectScope; - if (isAggregate(select)) { - aggScope = - new AggregatingSelectScope(selectScope, select, false); - selectScopes.put(select, aggScope); - } else { - selectScopes.put(select, selectScope); - } - if (select.getGroup() != null) { - GroupByScope groupByScope = - new GroupByScope(selectScope, select.getGroup(), select); - groupByScopes.put(select, groupByScope); - registerSubQueries(groupByScope, select.getGroup()); - } - registerOperandSubQueries( - aggScope, - select, - SqlSelect.HAVING_OPERAND); - registerSubQueries(aggScope, select.getSelectList()); - final SqlNodeList orderList = select.getOrderList(); - if (orderList != null) { - // If the query is 'SELECT DISTINCT', restrict the columns - // available to the ORDER BY clause. - if (select.isDistinct()) { - aggScope = - new AggregatingSelectScope(selectScope, select, true); - } - OrderByScope orderScope = - new OrderByScope(aggScope, orderList, select); - orderScopes.put(select, orderScope); - registerSubQueries(orderScope, orderList); - - if (!isAggregate(select)) { - // Since this is not an aggregating query, - // there cannot be any aggregates in the ORDER BY clause. - SqlNode agg = aggFinder.findAgg(orderList); - if (agg != null) { - throw newValidationError(agg, RESOURCE.aggregateIllegalInOrderBy()); - } - } - } - break; - - case INTERSECT: - validateFeature(RESOURCE.sQLFeature_F302(), node.getParserPosition()); - registerSetop( - parentScope, - usingScope, - node, - node, - alias, - forceNullable); - break; - - case EXCEPT: - validateFeature(RESOURCE.sQLFeature_E071_03(), node.getParserPosition()); - registerSetop( - parentScope, - usingScope, - node, - node, - alias, - forceNullable); - break; - - case UNION: - registerSetop( - parentScope, - usingScope, - node, - node, - alias, - forceNullable); - break; - - case WITH: - registerWith(parentScope, usingScope, (SqlWith) node, enclosingNode, - alias, forceNullable, checkUpdate); - break; - - case VALUES: - call = (SqlCall) node; - scopes.put(call, parentScope); - final TableConstructorNamespace tableConstructorNamespace = - new TableConstructorNamespace( - this, - call, - parentScope, - enclosingNode); - registerNamespace( - usingScope, - alias, - tableConstructorNamespace, - forceNullable); - operands = call.getOperandList(); - for (int i = 0; i < operands.size(); ++i) { - assert operands.get(i).getKind() == SqlKind.ROW; - - // FIXME jvs 9-Feb-2005: Correlation should - // be illegal in these sub-queries. Same goes for - // any non-lateral SELECT in the FROM list. - registerOperandSubQueries(parentScope, call, i); - } - break; - - case INSERT: - SqlInsert insertCall = (SqlInsert) node; - InsertNamespace insertNs = - new InsertNamespace( - this, - insertCall, - enclosingNode, - parentScope); - registerNamespace(usingScope, null, insertNs, forceNullable); - registerQuery( - parentScope, - usingScope, - insertCall.getSource(), - enclosingNode, - null, - false); - break; - - case DELETE: - SqlDelete deleteCall = (SqlDelete) node; - DeleteNamespace deleteNs = - new DeleteNamespace( - this, - deleteCall, - enclosingNode, - parentScope); - registerNamespace(usingScope, null, deleteNs, forceNullable); - registerQuery( - parentScope, - usingScope, - deleteCall.getSourceSelect(), - enclosingNode, - null, - false); - break; - - case UPDATE: - if (checkUpdate) { - validateFeature(RESOURCE.sQLFeature_E101_03(), - node.getParserPosition()); - } - SqlUpdate updateCall = (SqlUpdate) node; - UpdateNamespace updateNs = - new UpdateNamespace( - this, - updateCall, - enclosingNode, - parentScope); - registerNamespace(usingScope, null, updateNs, forceNullable); - registerQuery( - parentScope, - usingScope, - updateCall.getSourceSelect(), - enclosingNode, - null, - false); - break; - - case MERGE: - validateFeature(RESOURCE.sQLFeature_F312(), node.getParserPosition()); - SqlMerge mergeCall = (SqlMerge) node; - MergeNamespace mergeNs = - new MergeNamespace( - this, - mergeCall, - enclosingNode, - parentScope); - registerNamespace(usingScope, null, mergeNs, forceNullable); - registerQuery( - parentScope, - usingScope, - mergeCall.getSourceSelect(), - enclosingNode, - null, - false); - - // update call can reference either the source table reference - // or the target table, so set its parent scope to the merge's - // source select; when validating the update, skip the feature - // validation check - if (mergeCall.getUpdateCall() != null) { - registerQuery( - whereScopes.get(mergeCall.getSourceSelect()), - null, - mergeCall.getUpdateCall(), - enclosingNode, - null, - false, - false); - } - if (mergeCall.getInsertCall() != null) { - registerQuery( - parentScope, - null, - mergeCall.getInsertCall(), - enclosingNode, - null, - false); - } - break; - - case UNNEST: - call = (SqlCall) node; - final UnnestNamespace unnestNs = - new UnnestNamespace(this, call, parentScope, enclosingNode); - registerNamespace( - usingScope, - alias, - unnestNs, - forceNullable); - registerOperandSubQueries(parentScope, call, 0); - scopes.put(node, parentScope); - break; - - case OTHER_FUNCTION: - call = (SqlCall) node; - ProcedureNamespace procNs = - new ProcedureNamespace( - this, - parentScope, - call, - enclosingNode); - registerNamespace( - usingScope, - alias, - procNs, - forceNullable); - registerSubQueries(parentScope, call); - break; - - case MULTISET_QUERY_CONSTRUCTOR: - case MULTISET_VALUE_CONSTRUCTOR: - validateFeature(RESOURCE.sQLFeature_S271(), node.getParserPosition()); - call = (SqlCall) node; - CollectScope cs = new CollectScope(parentScope, usingScope, call); - final CollectNamespace tableConstructorNs = - new CollectNamespace(call, cs, enclosingNode); - final String alias2 = deriveAlias(node, nextGeneratedId++); - registerNamespace( - usingScope, - alias2, - tableConstructorNs, - forceNullable); - operands = call.getOperandList(); - for (int i = 0; i < operands.size(); i++) { - registerOperandSubQueries(parentScope, call, i); - } - break; - - default: - throw Util.unexpected(node.getKind()); - } - } - - private void registerSetop( - SqlValidatorScope parentScope, - SqlValidatorScope usingScope, - SqlNode node, - SqlNode enclosingNode, - String alias, - boolean forceNullable) { - SqlCall call = (SqlCall) node; - final SetopNamespace setopNamespace = - createSetopNamespace(call, enclosingNode); - registerNamespace(usingScope, alias, setopNamespace, forceNullable); - - // A setop is in the same scope as its parent. - scopes.put(call, parentScope); - for (SqlNode operand : call.getOperandList()) { - registerQuery( - parentScope, - null, - operand, - operand, - null, - false); - } - } - - private void registerWith( - SqlValidatorScope parentScope, - SqlValidatorScope usingScope, - SqlWith with, - SqlNode enclosingNode, - String alias, - boolean forceNullable, - boolean checkUpdate) { - final WithNamespace withNamespace = - new WithNamespace(this, with, enclosingNode); - registerNamespace(usingScope, alias, withNamespace, forceNullable); - - SqlValidatorScope scope = parentScope; - for (SqlNode withItem_ : with.withList) { - final SqlWithItem withItem = (SqlWithItem) withItem_; - final WithScope withScope = new WithScope(scope, withItem); - scopes.put(withItem, withScope); - - registerQuery(scope, null, withItem.query, with, - withItem.name.getSimple(), false); - registerNamespace(null, alias, - new WithItemNamespace(this, withItem, enclosingNode), - false); - scope = withScope; - } - - registerQuery(scope, null, with.body, enclosingNode, alias, forceNullable, - checkUpdate); - } - - public boolean isAggregate(SqlSelect select) { - if (getAggregate(select) != null) { - return true; - } - // Also when nested window aggregates are present - for (SqlCall call : overFinder.findAll(select.getSelectList())) { - assert call.getKind() == SqlKind.OVER; - if (isNestedAggregateWindow(call.operand(0))) { - return true; - } - if (isOverAggregateWindow(call.operand(1))) { - return true; - } - } - return false; - } - - protected boolean isNestedAggregateWindow(SqlNode node) { - AggFinder nestedAggFinder = - new AggFinder(opTab, false, false, false, aggFinder, - catalogReader.nameMatcher()); - return nestedAggFinder.findAgg(node) != null; - } - - protected boolean isOverAggregateWindow(SqlNode node) { - return aggFinder.findAgg(node) != null; - } - - /** Returns the parse tree node (GROUP BY, HAVING, or an aggregate function - * call) that causes {@code select} to be an aggregate query, or null if it - * is not an aggregate query. - * - *

    The node is useful context for error messages, - * but you cannot assume that the node is the only aggregate function. */ - protected SqlNode getAggregate(SqlSelect select) { - SqlNode node = select.getGroup(); - if (node != null) { - return node; - } - node = select.getHaving(); - if (node != null) { - return node; - } - return getAgg(select); - } - - /** If there is at least one call to an aggregate function, returns the - * first. */ - private SqlNode getAgg(SqlSelect select) { - final SelectScope selectScope = getRawSelectScope(select); - if (selectScope != null) { - final List selectList = selectScope.getExpandedSelectList(); - if (selectList != null) { - return aggFinder.findAgg(selectList); - } - } - return aggFinder.findAgg(select.getSelectList()); - } - - @SuppressWarnings("deprecation") - public boolean isAggregate(SqlNode selectNode) { - return aggFinder.findAgg(selectNode) != null; - } - - private void validateNodeFeature(SqlNode node) { - switch (node.getKind()) { - case MULTISET_VALUE_CONSTRUCTOR: - validateFeature(RESOURCE.sQLFeature_S271(), node.getParserPosition()); - break; - } - } - - private void registerSubQueries( - SqlValidatorScope parentScope, - SqlNode node) { - if (node == null) { - return; - } - if (node.getKind().belongsTo(SqlKind.QUERY) - || node.getKind() == SqlKind.MULTISET_QUERY_CONSTRUCTOR - || node.getKind() == SqlKind.MULTISET_VALUE_CONSTRUCTOR) { - registerQuery(parentScope, null, node, node, null, false); - } else if (node instanceof SqlCall) { - validateNodeFeature(node); - SqlCall call = (SqlCall) node; - for (int i = 0; i < call.operandCount(); i++) { - registerOperandSubQueries(parentScope, call, i); - } - } else if (node instanceof SqlNodeList) { - SqlNodeList list = (SqlNodeList) node; - for (int i = 0, count = list.size(); i < count; i++) { - SqlNode listNode = list.get(i); - if (listNode.getKind().belongsTo(SqlKind.QUERY)) { - listNode = - SqlStdOperatorTable.SCALAR_QUERY.createCall( - listNode.getParserPosition(), - listNode); - list.set(i, listNode); - } - registerSubQueries(parentScope, listNode); - } - } else { - // atomic node -- can be ignored - } - } - - /** - * Registers any sub-queries inside a given call operand, and converts the - * operand to a scalar sub-query if the operator requires it. - * - * @param parentScope Parent scope - * @param call Call - * @param operandOrdinal Ordinal of operand within call - * @see SqlOperator#argumentMustBeScalar(int) - */ - private void registerOperandSubQueries( - SqlValidatorScope parentScope, - SqlCall call, - int operandOrdinal) { - SqlNode operand = call.operand(operandOrdinal); - if (operand == null) { - return; - } - if (operand.getKind().belongsTo(SqlKind.QUERY) - && call.getOperator().argumentMustBeScalar(operandOrdinal)) { - operand = - SqlStdOperatorTable.SCALAR_QUERY.createCall( - operand.getParserPosition(), - operand); - call.setOperand(operandOrdinal, operand); - } - registerSubQueries(parentScope, operand); - } - - public void validateIdentifier(SqlIdentifier id, SqlValidatorScope scope) { - final SqlQualified fqId = scope.fullyQualify(id); - if (expandColumnReferences) { - // NOTE jvs 9-Apr-2007: this doesn't cover ORDER BY, which has its - // own ideas about qualification. - id.assignNamesFrom(fqId.identifier); - } else { - Util.discard(fqId); - } - } - - public void validateLiteral(SqlLiteral literal) { - switch (literal.getTypeName()) { - case DECIMAL: - // Decimal and long have the same precision (as 64-bit integers), so - // the unscaled value of a decimal must fit into a long. - - // REVIEW jvs 4-Aug-2004: This should probably be calling over to - // the available calculator implementations to see what they - // support. For now use ESP instead. - // - // jhyde 2006/12/21: I think the limits should be baked into the - // type system, not dependent on the calculator implementation. - BigDecimal bd = (BigDecimal) literal.getValue(); - BigInteger unscaled = bd.unscaledValue(); - long longValue = unscaled.longValue(); - if (!BigInteger.valueOf(longValue).equals(unscaled)) { - // overflow - throw newValidationError(literal, - RESOURCE.numberLiteralOutOfRange(bd.toString())); - } - break; - - case DOUBLE: - validateLiteralAsDouble(literal); - break; - - case BINARY: - final BitString bitString = (BitString) literal.getValue(); - if ((bitString.getBitCount() % 8) != 0) { - throw newValidationError(literal, RESOURCE.binaryLiteralOdd()); - } - break; - - case DATE: - case TIME: - case TIMESTAMP: - Calendar calendar = literal.getValueAs(Calendar.class); - final int year = calendar.get(Calendar.YEAR); - final int era = calendar.get(Calendar.ERA); - if (year < 1 || era == GregorianCalendar.BC || year > 9999) { - throw newValidationError(literal, - RESOURCE.dateLiteralOutOfRange(literal.toString())); - } - break; - - case INTERVAL_YEAR: - case INTERVAL_YEAR_MONTH: - case INTERVAL_MONTH: - case INTERVAL_DAY: - case INTERVAL_DAY_HOUR: - case INTERVAL_DAY_MINUTE: - case INTERVAL_DAY_SECOND: - case INTERVAL_HOUR: - case INTERVAL_HOUR_MINUTE: - case INTERVAL_HOUR_SECOND: - case INTERVAL_MINUTE: - case INTERVAL_MINUTE_SECOND: - case INTERVAL_SECOND: - if (literal instanceof SqlIntervalLiteral) { - SqlIntervalLiteral.IntervalValue interval = - (SqlIntervalLiteral.IntervalValue) - literal.getValue(); - SqlIntervalQualifier intervalQualifier = - interval.getIntervalQualifier(); - - // ensure qualifier is good before attempting to validate literal - validateIntervalQualifier(intervalQualifier); - String intervalStr = interval.getIntervalLiteral(); - // throws CalciteContextException if string is invalid - int[] values = intervalQualifier.evaluateIntervalLiteral(intervalStr, - literal.getParserPosition(), typeFactory.getTypeSystem()); - Util.discard(values); - } - break; - default: - // default is to do nothing - } - } - - private void validateLiteralAsDouble(SqlLiteral literal) { - BigDecimal bd = (BigDecimal) literal.getValue(); - double d = bd.doubleValue(); - if (Double.isInfinite(d) || Double.isNaN(d)) { - // overflow - throw newValidationError(literal, - RESOURCE.numberLiteralOutOfRange(Util.toScientificNotation(bd))); - } - - // REVIEW jvs 4-Aug-2004: what about underflow? - } - - public void validateIntervalQualifier(SqlIntervalQualifier qualifier) { - assert qualifier != null; - boolean startPrecisionOutOfRange = false; - boolean fractionalSecondPrecisionOutOfRange = false; - final RelDataTypeSystem typeSystem = typeFactory.getTypeSystem(); - - final int startPrecision = qualifier.getStartPrecision(typeSystem); - final int fracPrecision = - qualifier.getFractionalSecondPrecision(typeSystem); - final int maxPrecision = typeSystem.getMaxPrecision(qualifier.typeName()); - final int minPrecision = qualifier.typeName().getMinPrecision(); - final int minScale = qualifier.typeName().getMinScale(); - final int maxScale = typeSystem.getMaxScale(qualifier.typeName()); - if (qualifier.isYearMonth()) { - if (startPrecision < minPrecision || startPrecision > maxPrecision) { - startPrecisionOutOfRange = true; - } else { - if (fracPrecision < minScale || fracPrecision > maxScale) { - fractionalSecondPrecisionOutOfRange = true; - } - } - } else { - if (startPrecision < minPrecision || startPrecision > maxPrecision) { - startPrecisionOutOfRange = true; - } else { - if (fracPrecision < minScale || fracPrecision > maxScale) { - fractionalSecondPrecisionOutOfRange = true; - } - } - } - - if (startPrecisionOutOfRange) { - throw newValidationError(qualifier, - RESOURCE.intervalStartPrecisionOutOfRange(startPrecision, - "INTERVAL " + qualifier)); - } else if (fractionalSecondPrecisionOutOfRange) { - throw newValidationError(qualifier, - RESOURCE.intervalFractionalSecondPrecisionOutOfRange( - fracPrecision, - "INTERVAL " + qualifier)); - } - } - - /** - * Validates the FROM clause of a query, or (recursively) a child node of - * the FROM clause: AS, OVER, JOIN, VALUES, or sub-query. - * - * @param node Node in FROM clause, typically a table or derived - * table - * @param targetRowType Desired row type of this expression, or - * {@link #unknownType} if not fussy. Must not be null. - * @param scope Scope - */ - protected void validateFrom( - SqlNode node, - RelDataType targetRowType, - SqlValidatorScope scope) { - Objects.requireNonNull(targetRowType); - switch (node.getKind()) { - case AS: - validateFrom( - ((SqlCall) node).operand(0), - targetRowType, - scope); - break; - case VALUES: - validateValues((SqlCall) node, targetRowType, scope); - break; - case JOIN: - validateJoin((SqlJoin) node, scope); - break; - case OVER: - validateOver((SqlCall) node, scope); - break; - case UNNEST: - validateUnnest((SqlCall) node, scope, targetRowType); - break; - default: - validateQuery(node, scope, targetRowType); - break; - } - - // Validate the namespace representation of the node, just in case the - // validation did not occur implicitly. - getNamespace(node, scope).validate(targetRowType); - } - - protected void validateOver(SqlCall call, SqlValidatorScope scope) { - throw new AssertionError("OVER unexpected in this context"); - } - - protected void validateUnnest(SqlCall call, SqlValidatorScope scope, RelDataType targetRowType) { - for (int i = 0; i < call.operandCount(); i++) { - SqlNode expandedItem = expand(call.operand(i), scope); - call.setOperand(i, expandedItem); - } - validateQuery(call, scope, targetRowType); - } - - private void checkRollUpInUsing(SqlIdentifier identifier, - SqlNode leftOrRight, SqlValidatorScope scope) { - SqlValidatorNamespace namespace = getNamespace(leftOrRight, scope); - if (namespace != null) { - SqlValidatorTable sqlValidatorTable = namespace.getTable(); - if (sqlValidatorTable != null) { - Table table = sqlValidatorTable.unwrap(Table.class); - String column = Util.last(identifier.names); - - if (table.isRolledUp(column)) { - throw newValidationError(identifier, - RESOURCE.rolledUpNotAllowed(column, "USING")); - } - } - } - } - - protected void validateJoin(SqlJoin join, SqlValidatorScope scope) { - SqlNode left = join.getLeft(); - SqlNode right = join.getRight(); - SqlNode condition = join.getCondition(); - boolean natural = join.isNatural(); - final JoinType joinType = join.getJoinType(); - final JoinConditionType conditionType = join.getConditionType(); - final SqlValidatorScope joinScope = scopes.get(join); - validateFrom(left, unknownType, joinScope); - validateFrom(right, unknownType, joinScope); - - // Validate condition. - switch (conditionType) { - case NONE: - Preconditions.checkArgument(condition == null); - break; - case ON: - Preconditions.checkArgument(condition != null); - SqlNode expandedCondition = expand(condition, joinScope); - join.setOperand(5, expandedCondition); - condition = join.getCondition(); - validateWhereOrOn(joinScope, condition, "ON"); - checkRollUp(null, join, condition, joinScope, "ON"); - break; - case USING: - SqlNodeList list = (SqlNodeList) condition; - - // Parser ensures that using clause is not empty. - Preconditions.checkArgument(list.size() > 0, "Empty USING clause"); - for (SqlNode node : list) { - SqlIdentifier id = (SqlIdentifier) node; - final RelDataType leftColType = validateUsingCol(id, left); - final RelDataType rightColType = validateUsingCol(id, right); - if (!SqlTypeUtil.isComparable(leftColType, rightColType)) { - throw newValidationError(id, - RESOURCE.naturalOrUsingColumnNotCompatible(id.getSimple(), - leftColType.toString(), rightColType.toString())); - } - checkRollUpInUsing(id, left, scope); - checkRollUpInUsing(id, right, scope); - } - break; - default: - throw Util.unexpected(conditionType); - } - - // Validate NATURAL. - if (natural) { - if (condition != null) { - throw newValidationError(condition, - RESOURCE.naturalDisallowsOnOrUsing()); - } - - // Join on fields that occur exactly once on each side. Ignore - // fields that occur more than once on either side. - final RelDataType leftRowType = getNamespace(left).getRowType(); - final RelDataType rightRowType = getNamespace(right).getRowType(); - final SqlNameMatcher nameMatcher = catalogReader.nameMatcher(); - List naturalColumnNames = - SqlValidatorUtil.deriveNaturalJoinColumnList(nameMatcher, - leftRowType, rightRowType); - - // Check compatibility of the chosen columns. - for (String name : naturalColumnNames) { - final RelDataType leftColType = - nameMatcher.field(leftRowType, name).getType(); - final RelDataType rightColType = - nameMatcher.field(rightRowType, name).getType(); - if (!SqlTypeUtil.isComparable(leftColType, rightColType)) { - throw newValidationError(join, - RESOURCE.naturalOrUsingColumnNotCompatible(name, - leftColType.toString(), rightColType.toString())); - } - } - } - - // Which join types require/allow a ON/USING condition, or allow - // a NATURAL keyword? - switch (joinType) { - case LEFT_SEMI_JOIN: - if (!conformance.isLiberal()) { - throw newValidationError(join.getJoinTypeNode(), - RESOURCE.dialectDoesNotSupportFeature("LEFT SEMI JOIN")); - } - // fall through - case INNER: - case LEFT: - case RIGHT: - case FULL: - if ((condition == null) && !natural) { - throw newValidationError(join, RESOURCE.joinRequiresCondition()); - } - break; - case COMMA: - case CROSS: - if (condition != null) { - throw newValidationError(join.getConditionTypeNode(), - RESOURCE.crossJoinDisallowsCondition()); - } - if (natural) { - throw newValidationError(join.getConditionTypeNode(), - RESOURCE.crossJoinDisallowsCondition()); - } - break; - default: - throw Util.unexpected(joinType); - } - } - - /** - * Throws an error if there is an aggregate or windowed aggregate in the - * given clause. - * - * @param aggFinder Finder for the particular kind(s) of aggregate function - * @param node Parse tree - * @param clause Name of clause: "WHERE", "GROUP BY", "ON" - */ - private void validateNoAggs(AggFinder aggFinder, SqlNode node, - String clause) { - final SqlCall agg = aggFinder.findAgg(node); - if (agg == null) { - return; - } - final SqlOperator op = agg.getOperator(); - if (op == SqlStdOperatorTable.OVER) { - throw newValidationError(agg, - RESOURCE.windowedAggregateIllegalInClause(clause)); - } else if (op.isGroup() || op.isGroupAuxiliary()) { - throw newValidationError(agg, - RESOURCE.groupFunctionMustAppearInGroupByClause(op.getName())); - } else { - throw newValidationError(agg, - RESOURCE.aggregateIllegalInClause(clause)); - } - } - - private RelDataType validateUsingCol(SqlIdentifier id, SqlNode leftOrRight) { - if (id.names.size() == 1) { - String name = id.names.get(0); - final SqlValidatorNamespace namespace = getNamespace(leftOrRight); - final RelDataType rowType = namespace.getRowType(); - final SqlNameMatcher nameMatcher = catalogReader.nameMatcher(); - final RelDataTypeField field = nameMatcher.field(rowType, name); - if (field != null) { - if (nameMatcher.frequency(rowType.getFieldNames(), name) > 1) { - throw newValidationError(id, - RESOURCE.columnInUsingNotUnique(id.toString())); - } - return field.getType(); - } - } - throw newValidationError(id, RESOURCE.columnNotFound(id.toString())); - } - - /** - * Validates a SELECT statement. - * - * @param select Select statement - * @param targetRowType Desired row type, must not be null, may be the data - * type 'unknown'. - */ - protected void validateSelect( - SqlSelect select, - RelDataType targetRowType) { - assert targetRowType != null; - // Namespace is either a select namespace or a wrapper around one. - final SelectNamespace ns = - getNamespace(select).unwrap(SelectNamespace.class); - - // Its rowtype is null, meaning it hasn't been validated yet. - // This is important, because we need to take the targetRowType into - // account. - assert ns.rowType == null; - - if (select.isDistinct()) { - validateFeature(RESOURCE.sQLFeature_E051_01(), - select.getModifierNode(SqlSelectKeyword.DISTINCT) - .getParserPosition()); - } - - final SqlNodeList selectItems = select.getSelectList(); - RelDataType fromType = unknownType; - if (selectItems.size() == 1) { - final SqlNode selectItem = selectItems.get(0); - if (selectItem instanceof SqlIdentifier) { - SqlIdentifier id = (SqlIdentifier) selectItem; - if (id.isStar() && (id.names.size() == 1)) { - // Special case: for INSERT ... VALUES(?,?), the SQL - // standard says we're supposed to propagate the target - // types down. So iff the select list is an unqualified - // star (as it will be after an INSERT ... VALUES has been - // expanded), then propagate. - fromType = targetRowType; - } - } - } - - // Make sure that items in FROM clause have distinct aliases. - final SelectScope fromScope = (SelectScope) getFromScope(select); - List names = fromScope.getChildNames(); - if (!catalogReader.nameMatcher().isCaseSensitive()) { - names = Lists.transform(names, s -> s.toUpperCase(Locale.ROOT)); - } - final int duplicateAliasOrdinal = Util.firstDuplicate(names); - if (duplicateAliasOrdinal >= 0) { - final ScopeChild child = - fromScope.children.get(duplicateAliasOrdinal); - throw newValidationError(child.namespace.getEnclosingNode(), - RESOURCE.fromAliasDuplicate(child.name)); - } - - if (select.getFrom() == null) { - if (conformance.isFromRequired()) { - throw newValidationError(select, RESOURCE.selectMissingFrom()); - } - } else { - validateFrom(select.getFrom(), fromType, fromScope); - } - - validateWhereClause(select); - validateGroupClause(select); - validateHavingClause(select); - validateWindowClause(select); - handleOffsetFetch(select.getOffset(), select.getFetch()); - - // Validate the SELECT clause late, because a select item might - // depend on the GROUP BY list, or the window function might reference - // window name in the WINDOW clause etc. - final RelDataType rowType = - validateSelectList(selectItems, select, targetRowType); - ns.setType(rowType); - - // Validate ORDER BY after we have set ns.rowType because in some - // dialects you can refer to columns of the select list, e.g. - // "SELECT empno AS x FROM emp ORDER BY x" - validateOrderList(select); - - if (shouldCheckForRollUp(select.getFrom())) { - checkRollUpInSelectList(select); - checkRollUp(null, select, select.getWhere(), getWhereScope(select)); - checkRollUp(null, select, select.getHaving(), getHavingScope(select)); - checkRollUpInWindowDecl(select); - checkRollUpInGroupBy(select); - checkRollUpInOrderBy(select); - } - } - - private void checkRollUpInSelectList(SqlSelect select) { - SqlValidatorScope scope = getSelectScope(select); - for (SqlNode item : select.getSelectList()) { - checkRollUp(null, select, item, scope); - } - } - - private void checkRollUpInGroupBy(SqlSelect select) { - SqlNodeList group = select.getGroup(); - if (group != null) { - for (SqlNode node : group) { - checkRollUp(null, select, node, getGroupScope(select), "GROUP BY"); - } - } - } - - private void checkRollUpInOrderBy(SqlSelect select) { - SqlNodeList orderList = select.getOrderList(); - if (orderList != null) { - for (SqlNode node : orderList) { - checkRollUp(null, select, node, getOrderScope(select), "ORDER BY"); - } - } - } - - private void checkRollUpInWindow(SqlWindow window, SqlValidatorScope scope) { - if (window != null) { - for (SqlNode node : window.getPartitionList()) { - checkRollUp(null, window, node, scope, "PARTITION BY"); - } - - for (SqlNode node : window.getOrderList()) { - checkRollUp(null, window, node, scope, "ORDER BY"); - } - } - } - - private void checkRollUpInWindowDecl(SqlSelect select) { - for (SqlNode decl : select.getWindowList()) { - checkRollUpInWindow((SqlWindow) decl, getSelectScope(select)); - } - } - - private SqlNode stripDot(SqlNode node) { - if (node != null && node.getKind() == SqlKind.DOT) { - return stripDot(((SqlCall) node).operand(0)); - } - return node; - } - - private void checkRollUp(SqlNode grandParent, SqlNode parent, - SqlNode current, SqlValidatorScope scope, String optionalClause) { - current = stripAs(current); - if (current instanceof SqlCall && !(current instanceof SqlSelect)) { - // Validate OVER separately - checkRollUpInWindow(getWindowInOver(current), scope); - current = stripOver(current); - - List children = ((SqlCall) stripDot(current)).getOperandList(); - for (SqlNode child : children) { - checkRollUp(parent, current, child, scope, optionalClause); - } - } else if (current instanceof SqlIdentifier) { - SqlIdentifier id = (SqlIdentifier) current; - if (!id.isStar() && isRolledUpColumn(id, scope)) { - if (!isAggregation(parent.getKind()) - || !isRolledUpColumnAllowedInAgg(id, scope, (SqlCall) parent, grandParent)) { - String context = optionalClause != null ? optionalClause : parent.getKind().toString(); - throw newValidationError(id, - RESOURCE.rolledUpNotAllowed(deriveAlias(id, 0), context)); - } - } - } - } - - private void checkRollUp(SqlNode grandParent, SqlNode parent, - SqlNode current, SqlValidatorScope scope) { - checkRollUp(grandParent, parent, current, scope, null); - } - - private SqlWindow getWindowInOver(SqlNode over) { - if (over.getKind() == SqlKind.OVER) { - SqlNode window = ((SqlCall) over).getOperandList().get(1); - if (window instanceof SqlWindow) { - return (SqlWindow) window; - } - // SqlIdentifier, gets validated elsewhere - return null; - } - return null; - } - - private static SqlNode stripOver(SqlNode node) { - switch (node.getKind()) { - case OVER: - return ((SqlCall) node).getOperandList().get(0); - default: - return node; - } - } - - private Pair findTableColumnPair(SqlIdentifier identifier, - SqlValidatorScope scope) { - final SqlCall call = makeNullaryCall(identifier); - if (call != null) { - return null; - } - SqlQualified qualified = scope.fullyQualify(identifier); - List names = qualified.identifier.names; - - if (names.size() < 2) { - return null; - } - - return new Pair<>(names.get(names.size() - 2), Util.last(names)); - } - - // Returns true iff the given column is valid inside the given aggCall. - private boolean isRolledUpColumnAllowedInAgg(SqlIdentifier identifier, SqlValidatorScope scope, - SqlCall aggCall, SqlNode parent) { - Pair pair = findTableColumnPair(identifier, scope); - - if (pair == null) { - return true; - } - - String columnName = pair.right; - - SqlValidatorTable sqlValidatorTable = - scope.fullyQualify(identifier).namespace.getTable(); - if (sqlValidatorTable != null) { - Table table = sqlValidatorTable.unwrap(Table.class); - return table.rolledUpColumnValidInsideAgg(columnName, aggCall, parent, - catalogReader.getConfig()); - } - return true; - } - - - // Returns true iff the given column is actually rolled up. - private boolean isRolledUpColumn(SqlIdentifier identifier, SqlValidatorScope scope) { - Pair pair = findTableColumnPair(identifier, scope); - - if (pair == null) { - return false; - } - - String columnName = pair.right; - - SqlValidatorTable sqlValidatorTable = - scope.fullyQualify(identifier).namespace.getTable(); - if (sqlValidatorTable != null) { - Table table = sqlValidatorTable.unwrap(Table.class); - return table.isRolledUp(columnName); - } - return false; - } - - private boolean shouldCheckForRollUp(SqlNode from) { - if (from != null) { - SqlKind kind = stripAs(from).getKind(); - return kind != SqlKind.VALUES && kind != SqlKind.SELECT; - } - return false; - } - - /** Validates that a query can deliver the modality it promises. Only called - * on the top-most SELECT or set operator in the tree. */ - private void validateModality(SqlNode query) { - final SqlModality modality = deduceModality(query); - if (query instanceof SqlSelect) { - final SqlSelect select = (SqlSelect) query; - validateModality(select, modality, true); - } else if (query.getKind() == SqlKind.VALUES) { - switch (modality) { - case STREAM: - throw newValidationError(query, Static.RESOURCE.cannotStreamValues()); - } - } else { - assert query.isA(SqlKind.SET_QUERY); - final SqlCall call = (SqlCall) query; - for (SqlNode operand : call.getOperandList()) { - if (deduceModality(operand) != modality) { - throw newValidationError(operand, - Static.RESOURCE.streamSetOpInconsistentInputs()); - } - validateModality(operand); - } - } - } - - /** Return the intended modality of a SELECT or set-op. */ - private SqlModality deduceModality(SqlNode query) { - if (query instanceof SqlSelect) { - SqlSelect select = (SqlSelect) query; - return select.getModifierNode(SqlSelectKeyword.STREAM) != null - ? SqlModality.STREAM - : SqlModality.RELATION; - } else if (query.getKind() == SqlKind.VALUES) { - return SqlModality.RELATION; - } else { - assert query.isA(SqlKind.SET_QUERY); - final SqlCall call = (SqlCall) query; - return deduceModality(call.getOperandList().get(0)); - } - } - - public boolean validateModality(SqlSelect select, SqlModality modality, - boolean fail) { - final SelectScope scope = getRawSelectScope(select); - - switch (modality) { - case STREAM: - if (scope.children.size() == 1) { - for (ScopeChild child : scope.children) { - if (!child.namespace.supportsModality(modality)) { - if (fail) { - throw newValidationError(child.namespace.getNode(), - Static.RESOURCE.cannotConvertToStream(child.name)); - } else { - return false; - } - } - } - } else { - int supportsModalityCount = 0; - for (ScopeChild child : scope.children) { - if (child.namespace.supportsModality(modality)) { - ++supportsModalityCount; - } - } - - if (supportsModalityCount == 0) { - if (fail) { - String inputs = String.join(", ", scope.getChildNames()); - throw newValidationError(select, - Static.RESOURCE.cannotStreamResultsForNonStreamingInputs(inputs)); - } else { - return false; - } - } - } - break; - default: - for (ScopeChild child : scope.children) { - if (!child.namespace.supportsModality(modality)) { - if (fail) { - throw newValidationError(child.namespace.getNode(), - Static.RESOURCE.cannotConvertToRelation(child.name)); - } else { - return false; - } - } - } - } - - // Make sure that aggregation is possible. - final SqlNode aggregateNode = getAggregate(select); - if (aggregateNode != null) { - switch (modality) { - case STREAM: - SqlNodeList groupList = select.getGroup(); - if (groupList == null - || !SqlValidatorUtil.containsMonotonic(scope, groupList)) { - if (fail) { - throw newValidationError(aggregateNode, - Static.RESOURCE.streamMustGroupByMonotonic()); - } else { - return false; - } - } - } - } - - // Make sure that ORDER BY is possible. - final SqlNodeList orderList = select.getOrderList(); - if (orderList != null && orderList.size() > 0) { - switch (modality) { - case STREAM: - if (!hasSortedPrefix(scope, orderList)) { - if (fail) { - throw newValidationError(orderList.get(0), - Static.RESOURCE.streamMustOrderByMonotonic()); - } else { - return false; - } - } - } - } - return true; - } - - /** Returns whether the prefix is sorted. */ - private boolean hasSortedPrefix(SelectScope scope, SqlNodeList orderList) { - return isSortCompatible(scope, orderList.get(0), false); - } - - private boolean isSortCompatible(SelectScope scope, SqlNode node, - boolean descending) { - switch (node.getKind()) { - case DESCENDING: - return isSortCompatible(scope, ((SqlCall) node).getOperandList().get(0), - true); - } - final SqlMonotonicity monotonicity = scope.getMonotonicity(node); - switch (monotonicity) { - case INCREASING: - case STRICTLY_INCREASING: - return !descending; - case DECREASING: - case STRICTLY_DECREASING: - return descending; - default: - return false; - } - } - - protected void validateWindowClause(SqlSelect select) { - final SqlNodeList windowList = select.getWindowList(); - @SuppressWarnings("unchecked") final List windows = - (List) windowList.getList(); - if (windows.isEmpty()) { - return; - } - - final SelectScope windowScope = (SelectScope) getFromScope(select); - assert windowScope != null; - - // 1. ensure window names are simple - // 2. ensure they are unique within this scope - for (SqlWindow window : windows) { - SqlIdentifier declName = window.getDeclName(); - if (!declName.isSimple()) { - throw newValidationError(declName, RESOURCE.windowNameMustBeSimple()); - } - - if (windowScope.existingWindowName(declName.toString())) { - throw newValidationError(declName, RESOURCE.duplicateWindowName()); - } else { - windowScope.addWindowName(declName.toString()); - } - } - - // 7.10 rule 2 - // Check for pairs of windows which are equivalent. - for (int i = 0; i < windows.size(); i++) { - SqlNode window1 = windows.get(i); - for (int j = i + 1; j < windows.size(); j++) { - SqlNode window2 = windows.get(j); - if (window1.equalsDeep(window2, Litmus.IGNORE)) { - throw newValidationError(window2, RESOURCE.dupWindowSpec()); - } - } - } - - for (SqlWindow window : windows) { - final SqlNodeList expandedOrderList = - (SqlNodeList) expand(window.getOrderList(), windowScope); - window.setOrderList(expandedOrderList); - expandedOrderList.validate(this, windowScope); - - final SqlNodeList expandedPartitionList = - (SqlNodeList) expand(window.getPartitionList(), windowScope); - window.setPartitionList(expandedPartitionList); - expandedPartitionList.validate(this, windowScope); - } - - // Hand off to validate window spec components - windowList.validate(this, windowScope); - } - - public void validateWith(SqlWith with, SqlValidatorScope scope) { - final SqlValidatorNamespace namespace = getNamespace(with); - validateNamespace(namespace, unknownType); - } - - public void validateWithItem(SqlWithItem withItem) { - if (withItem.columnList != null) { - final RelDataType rowType = getValidatedNodeType(withItem.query); - final int fieldCount = rowType.getFieldCount(); - if (withItem.columnList.size() != fieldCount) { - throw newValidationError(withItem.columnList, - RESOURCE.columnCountMismatch()); - } - SqlValidatorUtil.checkIdentifierListForDuplicates( - withItem.columnList.getList(), validationErrorFunction); - } else { - // Luckily, field names have not been make unique yet. - final List fieldNames = - getValidatedNodeType(withItem.query).getFieldNames(); - final int i = Util.firstDuplicate(fieldNames); - if (i >= 0) { - throw newValidationError(withItem.query, - RESOURCE.duplicateColumnAndNoColumnList(fieldNames.get(i))); - } - } - } - - public void validateSequenceValue(SqlValidatorScope scope, SqlIdentifier id) { - // Resolve identifier as a table. - final SqlValidatorScope.ResolvedImpl resolved = - new SqlValidatorScope.ResolvedImpl(); - scope.resolveTable(id.names, catalogReader.nameMatcher(), - SqlValidatorScope.Path.EMPTY, resolved); - if (resolved.count() != 1) { - throw newValidationError(id, RESOURCE.tableNameNotFound(id.toString())); - } - // We've found a table. But is it a sequence? - final SqlValidatorNamespace ns = resolved.only().namespace; - if (ns instanceof TableNamespace) { - final Table table = ns.getTable().unwrap(Table.class); - switch (table.getJdbcTableType()) { - case SEQUENCE: - case TEMPORARY_SEQUENCE: - return; - } - } - throw newValidationError(id, RESOURCE.notASequence(id.toString())); - } - - public SqlValidatorScope getWithScope(SqlNode withItem) { - assert withItem.getKind() == SqlKind.WITH_ITEM; - return scopes.get(withItem); - } - - @Override - public SqlValidator setEnableTypeCoercion(boolean enabled) { - this.enableTypeCoercion = enabled; - return this; - } - - @Override - public boolean isTypeCoercionEnabled() { - return this.enableTypeCoercion; - } - - @Override - public void setTypeCoercion(TypeCoercion typeCoercion) { - Objects.requireNonNull(typeCoercion); - this.typeCoercion = typeCoercion; - } - - @Override - public TypeCoercion getTypeCoercion() { - assert isTypeCoercionEnabled(); - return this.typeCoercion; - } - - /** - * Validates the ORDER BY clause of a SELECT statement. - * - * @param select Select statement - */ - protected void validateOrderList(SqlSelect select) { - // ORDER BY is validated in a scope where aliases in the SELECT clause - // are visible. For example, "SELECT empno AS x FROM emp ORDER BY x" - // is valid. - SqlNodeList orderList = select.getOrderList(); - if (orderList == null) { - return; - } - if (!shouldAllowIntermediateOrderBy()) { - if (!cursorSet.contains(select)) { - throw newValidationError(select, RESOURCE.invalidOrderByPos()); - } - } - final SqlValidatorScope orderScope = getOrderScope(select); - Objects.requireNonNull(orderScope); - - List expandList = new ArrayList<>(); - for (SqlNode orderItem : orderList) { - SqlNode expandedOrderItem = expand(orderItem, orderScope); - expandList.add(expandedOrderItem); - } - - SqlNodeList expandedOrderList = new SqlNodeList( - expandList, - orderList.getParserPosition()); - select.setOrderBy(expandedOrderList); - - for (SqlNode orderItem : expandedOrderList) { - validateOrderItem(select, orderItem); - } - } - - /** - * Validates an item in the GROUP BY clause of a SELECT statement. - * - * @param select Select statement - * @param groupByItem GROUP BY clause item - */ - private void validateGroupByItem(SqlSelect select, SqlNode groupByItem) { - final SqlValidatorScope groupByScope = getGroupScope(select); - groupByScope.validateExpr(groupByItem); - } - - /** - * Validates an item in the ORDER BY clause of a SELECT statement. - * - * @param select Select statement - * @param orderItem ORDER BY clause item - */ - private void validateOrderItem(SqlSelect select, SqlNode orderItem) { - switch (orderItem.getKind()) { - case DESCENDING: - validateFeature(RESOURCE.sQLConformance_OrderByDesc(), - orderItem.getParserPosition()); - validateOrderItem(select, - ((SqlCall) orderItem).operand(0)); - return; - } - - final SqlValidatorScope orderScope = getOrderScope(select); - validateExpr(orderItem, orderScope); - } - - public SqlNode expandOrderExpr(SqlSelect select, SqlNode orderExpr) { - final SqlNode newSqlNode = - new OrderExpressionExpander(select, orderExpr).go(); - if (newSqlNode != orderExpr) { - final SqlValidatorScope scope = getOrderScope(select); - inferUnknownTypes(unknownType, scope, newSqlNode); - final RelDataType type = deriveType(scope, newSqlNode); - setValidatedNodeType(newSqlNode, type); - } - return newSqlNode; - } - - /** - * Validates the GROUP BY clause of a SELECT statement. This method is - * called even if no GROUP BY clause is present. - */ - protected void validateGroupClause(SqlSelect select) { - SqlNodeList groupList = select.getGroup(); - if (groupList == null) { - return; - } - final String clause = "GROUP BY"; - validateNoAggs(aggOrOverFinder, groupList, clause); - final SqlValidatorScope groupScope = getGroupScope(select); - inferUnknownTypes(unknownType, groupScope, groupList); - - // expand the expression in group list. - List expandedList = new ArrayList<>(); - for (SqlNode groupItem : groupList) { - SqlNode expandedItem = expandGroupByOrHavingExpr(groupItem, groupScope, select, false); - expandedList.add(expandedItem); - } - groupList = new SqlNodeList(expandedList, groupList.getParserPosition()); - select.setGroupBy(groupList); - for (SqlNode groupItem : expandedList) { - validateGroupByItem(select, groupItem); - } - - // Nodes in the GROUP BY clause are expressions except if they are calls - // to the GROUPING SETS, ROLLUP or CUBE operators; this operators are not - // expressions, because they do not have a type. - for (SqlNode node : groupList) { - switch (node.getKind()) { - case GROUPING_SETS: - case ROLLUP: - case CUBE: - node.validate(this, groupScope); - break; - default: - node.validateExpr(this, groupScope); - } - } - - // Derive the type of each GROUP BY item. We don't need the type, but - // it resolves functions, and that is necessary for deducing - // monotonicity. - final SqlValidatorScope selectScope = getSelectScope(select); - AggregatingSelectScope aggregatingScope = null; - if (selectScope instanceof AggregatingSelectScope) { - aggregatingScope = (AggregatingSelectScope) selectScope; - } - for (SqlNode groupItem : groupList) { - if (groupItem instanceof SqlNodeList - && ((SqlNodeList) groupItem).size() == 0) { - continue; - } - validateGroupItem(groupScope, aggregatingScope, groupItem); - } - - SqlNode agg = aggFinder.findAgg(groupList); - if (agg != null) { - throw newValidationError(agg, RESOURCE.aggregateIllegalInClause(clause)); - } - } - - private void validateGroupItem(SqlValidatorScope groupScope, - AggregatingSelectScope aggregatingScope, - SqlNode groupItem) { - switch (groupItem.getKind()) { - case GROUPING_SETS: - case ROLLUP: - case CUBE: - validateGroupingSets(groupScope, aggregatingScope, (SqlCall) groupItem); - break; - default: - if (groupItem instanceof SqlNodeList) { - break; - } - final RelDataType type = deriveType(groupScope, groupItem); - setValidatedNodeType(groupItem, type); - } - } - - private void validateGroupingSets(SqlValidatorScope groupScope, - AggregatingSelectScope aggregatingScope, SqlCall groupItem) { - for (SqlNode node : groupItem.getOperandList()) { - validateGroupItem(groupScope, aggregatingScope, node); - } - } - - protected void validateWhereClause(SqlSelect select) { - // validate WHERE clause - final SqlNode where = select.getWhere(); - if (where == null) { - return; - } - final SqlValidatorScope whereScope = getWhereScope(select); - final SqlNode expandedWhere = expand(where, whereScope); - select.setWhere(expandedWhere); - validateWhereOrOn(whereScope, expandedWhere, "WHERE"); - } - - protected void validateWhereOrOn( - SqlValidatorScope scope, - SqlNode condition, - String clause) { - validateNoAggs(aggOrOverOrGroupFinder, condition, clause); - inferUnknownTypes( - booleanType, - scope, - condition); - condition.validate(this, scope); - - final RelDataType type = deriveType(scope, condition); - if (!SqlTypeUtil.inBooleanFamily(type)) { - throw newValidationError(condition, RESOURCE.condMustBeBoolean(clause)); - } - } - - protected void validateHavingClause(SqlSelect select) { - // HAVING is validated in the scope after groups have been created. - // For example, in "SELECT empno FROM emp WHERE empno = 10 GROUP BY - // deptno HAVING empno = 10", the reference to 'empno' in the HAVING - // clause is illegal. - SqlNode having = select.getHaving(); - if (having == null) { - return; - } - final AggregatingScope havingScope = - (AggregatingScope) getSelectScope(select); - if (getConformance().isHavingAlias()) { - SqlNode newExpr = expandGroupByOrHavingExpr(having, havingScope, select, true); - if (having != newExpr) { - having = newExpr; - select.setHaving(newExpr); - } - } - havingScope.checkAggregateExpr(having, true); - inferUnknownTypes( - booleanType, - havingScope, - having); - having.validate(this, havingScope); - final RelDataType type = deriveType(havingScope, having); - if (!SqlTypeUtil.inBooleanFamily(type)) { - throw newValidationError(having, RESOURCE.havingMustBeBoolean()); - } - } - - protected RelDataType validateSelectList( - final SqlNodeList selectItems, - SqlSelect select, - RelDataType targetRowType) { - // First pass, ensure that aliases are unique. "*" and "TABLE.*" items - // are ignored. - - // Validate SELECT list. Expand terms of the form "*" or "TABLE.*". - final SqlValidatorScope selectScope = getSelectScope(select); - final List expandedSelectItems = new ArrayList<>(); - final Set aliases = new HashSet<>(); - final List> fieldList = new ArrayList<>(); - - for (int i = 0; i < selectItems.size(); i++) { - SqlNode selectItem = selectItems.get(i); - if (selectItem instanceof SqlSelect) { - handleScalarSubQuery( - select, - (SqlSelect) selectItem, - expandedSelectItems, - aliases, - fieldList); - } else { - expandSelectItem( - selectItem, - select, - targetRowType.isStruct() - && targetRowType.getFieldCount() >= i - ? targetRowType.getFieldList().get(i).getType() - : unknownType, - expandedSelectItems, - aliases, - fieldList, - false); - } - } - - // Create the new select list with expanded items. Pass through - // the original parser position so that any overall failures can - // still reference the original input text. - SqlNodeList newSelectList = - new SqlNodeList( - expandedSelectItems, - selectItems.getParserPosition()); - if (shouldExpandIdentifiers()) { - select.setSelectList(newSelectList); - } - getRawSelectScope(select).setExpandedSelectList(expandedSelectItems); - - // TODO: when SELECT appears as a value sub-query, should be using - // something other than unknownType for targetRowType - inferUnknownTypes(targetRowType, selectScope, newSelectList); - - for (SqlNode selectItem : expandedSelectItems) { - validateNoAggs(groupFinder, selectItem, "SELECT"); - validateExpr(selectItem, selectScope); - } - - return typeFactory.createStructType(fieldList); - } - - /** - * Validates an expression. - * - * @param expr Expression - * @param scope Scope in which expression occurs - */ - private void validateExpr(SqlNode expr, SqlValidatorScope scope) { - if (expr instanceof SqlCall) { - final SqlOperator op = ((SqlCall) expr).getOperator(); - if (op.isAggregator() && op.requiresOver()) { - throw newValidationError(expr, - RESOURCE.absentOverClause()); - } - } - - // Call on the expression to validate itself. - expr.validateExpr(this, scope); - - // Perform any validation specific to the scope. For example, an - // aggregating scope requires that expressions are valid aggregations. - scope.validateExpr(expr); - } - - /** - * Processes SubQuery found in Select list. Checks that is actually Scalar - * sub-query and makes proper entries in each of the 3 lists used to create - * the final rowType entry. - * - * @param parentSelect base SqlSelect item - * @param selectItem child SqlSelect from select list - * @param expandedSelectItems Select items after processing - * @param aliasList built from user or system values - * @param fieldList Built up entries for each select list entry - */ - private void handleScalarSubQuery( - SqlSelect parentSelect, - SqlSelect selectItem, - List expandedSelectItems, - Set aliasList, - List> fieldList) { - // A scalar sub-query only has one output column. - if (1 != selectItem.getSelectList().size()) { - throw newValidationError(selectItem, - RESOURCE.onlyScalarSubQueryAllowed()); - } - - // No expansion in this routine just append to list. - expandedSelectItems.add(selectItem); - - // Get or generate alias and add to list. - final String alias = - deriveAlias( - selectItem, - aliasList.size()); - aliasList.add(alias); - - final SelectScope scope = (SelectScope) getWhereScope(parentSelect); - final RelDataType type = deriveType(scope, selectItem); - setValidatedNodeType(selectItem, type); - - // we do not want to pass on the RelRecordType returned - // by the sub query. Just the type of the single expression - // in the sub-query select list. - assert type instanceof RelRecordType; - RelRecordType rec = (RelRecordType) type; - - RelDataType nodeType = rec.getFieldList().get(0).getType(); - nodeType = typeFactory.createTypeWithNullability(nodeType, true); - fieldList.add(Pair.of(alias, nodeType)); - } - - /** - * Derives a row-type for INSERT and UPDATE operations. - * - * @param table Target table for INSERT/UPDATE - * @param targetColumnList List of target columns, or null if not specified - * @param append Whether to append fields to those in - * baseRowType - * @return Rowtype - */ - protected RelDataType createTargetRowType( - SqlValidatorTable table, - SqlNodeList targetColumnList, - boolean append) { - RelDataType baseRowType = table.getRowType(); - if (targetColumnList == null) { - return baseRowType; - } - List targetFields = baseRowType.getFieldList(); - final List> fields = new ArrayList<>(); - if (append) { - for (RelDataTypeField targetField : targetFields) { - fields.add( - Pair.of(SqlUtil.deriveAliasFromOrdinal(fields.size()), - targetField.getType())); - } - } - final Set assignedFields = new HashSet<>(); - final RelOptTable relOptTable = table instanceof RelOptTable - ? ((RelOptTable) table) : null; - for (SqlNode node : targetColumnList) { - SqlIdentifier id = (SqlIdentifier) node; - RelDataTypeField targetField = - SqlValidatorUtil.getTargetField( - baseRowType, typeFactory, id, catalogReader, relOptTable); - if (targetField == null) { - throw newValidationError(id, - RESOURCE.unknownTargetColumn(id.toString())); - } - if (!assignedFields.add(targetField.getIndex())) { - throw newValidationError(id, - RESOURCE.duplicateTargetColumn(targetField.getName())); - } - fields.add(targetField); - } - return typeFactory.createStructType(fields); - } - - public void validateInsert(SqlInsert insert) { - final SqlValidatorNamespace targetNamespace = getNamespace(insert); - validateNamespace(targetNamespace, unknownType); - final RelOptTable relOptTable = SqlValidatorUtil.getRelOptTable( - targetNamespace, catalogReader.unwrap(Prepare.CatalogReader.class), null, null); - final SqlValidatorTable table = relOptTable == null - ? targetNamespace.getTable() - : relOptTable.unwrap(SqlValidatorTable.class); - - // INSERT has an optional column name list. If present then - // reduce the rowtype to the columns specified. If not present - // then the entire target rowtype is used. - final RelDataType targetRowType = - createTargetRowType( - table, - insert.getTargetColumnList(), - false); - - final SqlNode source = insert.getSource(); - if (source instanceof SqlSelect) { - final SqlSelect sqlSelect = (SqlSelect) source; - validateSelect(sqlSelect, targetRowType); - } else { - final SqlValidatorScope scope = scopes.get(source); - validateQuery(source, scope, targetRowType); - } - - // REVIEW jvs 4-Dec-2008: In FRG-365, this namespace row type is - // discarding the type inferred by inferUnknownTypes (which was invoked - // from validateSelect above). It would be better if that information - // were used here so that we never saw any untyped nulls during - // checkTypeAssignment. - final RelDataType sourceRowType = getNamespace(source).getRowType(); - final RelDataType logicalTargetRowType = - getLogicalTargetRowType(targetRowType, insert); - setValidatedNodeType(insert, logicalTargetRowType); - final RelDataType logicalSourceRowType = - getLogicalSourceRowType(sourceRowType, insert); - - checkFieldCount(insert.getTargetTable(), table, source, - logicalSourceRowType, logicalTargetRowType); - - checkTypeAssignment(logicalSourceRowType, logicalTargetRowType, insert); - - checkConstraint(table, source, logicalTargetRowType); - - validateAccess(insert.getTargetTable(), table, SqlAccessEnum.INSERT); - } - - /** - * Validates insert values against the constraint of a modifiable view. - * - * @param validatorTable Table that may wrap a ModifiableViewTable - * @param source The values being inserted - * @param targetRowType The target type for the view - */ - private void checkConstraint( - SqlValidatorTable validatorTable, - SqlNode source, - RelDataType targetRowType) { - final ModifiableViewTable modifiableViewTable = - validatorTable.unwrap(ModifiableViewTable.class); - if (modifiableViewTable != null && source instanceof SqlCall) { - final Table table = modifiableViewTable.unwrap(Table.class); - final RelDataType tableRowType = table.getRowType(typeFactory); - final List tableFields = tableRowType.getFieldList(); - - // Get the mapping from column indexes of the underlying table - // to the target columns and view constraints. - final Map tableIndexToTargetField = - SqlValidatorUtil.getIndexToFieldMap(tableFields, targetRowType); - final Map projectMap = - RelOptUtil.getColumnConstraints(modifiableViewTable, targetRowType, typeFactory); - - // Determine columns (indexed to the underlying table) that need - // to be validated against the view constraint. - final ImmutableBitSet targetColumns = - ImmutableBitSet.of(tableIndexToTargetField.keySet()); - final ImmutableBitSet constrainedColumns = - ImmutableBitSet.of(projectMap.keySet()); - final ImmutableBitSet constrainedTargetColumns = - targetColumns.intersect(constrainedColumns); - - // Validate insert values against the view constraint. - final List values = ((SqlCall) source).getOperandList(); - for (final int colIndex : constrainedTargetColumns.asList()) { - final String colName = tableFields.get(colIndex).getName(); - final RelDataTypeField targetField = tableIndexToTargetField.get(colIndex); - for (SqlNode row : values) { - final SqlCall call = (SqlCall) row; - final SqlNode sourceValue = call.operand(targetField.getIndex()); - final ValidationError validationError = - new ValidationError(sourceValue, - RESOURCE.viewConstraintNotSatisfied(colName, - Util.last(validatorTable.getQualifiedName()))); - RelOptUtil.validateValueAgainstConstraint(sourceValue, - projectMap.get(colIndex), validationError); - } - } - } - } - - /** - * Validates updates against the constraint of a modifiable view. - * - * @param validatorTable A {@link SqlValidatorTable} that may wrap a - * ModifiableViewTable - * @param update The UPDATE parse tree node - * @param targetRowType The target type - */ - private void checkConstraint( - SqlValidatorTable validatorTable, - SqlUpdate update, - RelDataType targetRowType) { - final ModifiableViewTable modifiableViewTable = - validatorTable.unwrap(ModifiableViewTable.class); - if (modifiableViewTable != null) { - final Table table = modifiableViewTable.unwrap(Table.class); - final RelDataType tableRowType = table.getRowType(typeFactory); - - final Map projectMap = - RelOptUtil.getColumnConstraints(modifiableViewTable, targetRowType, - typeFactory); - final Map nameToIndex = - SqlValidatorUtil.mapNameToIndex(tableRowType.getFieldList()); - - // Validate update values against the view constraint. - final List targets = update.getTargetColumnList().getList(); - final List sources = update.getSourceExpressionList().getList(); - for (final Pair column : Pair.zip(targets, sources)) { - final String columnName = ((SqlIdentifier) column.left).getSimple(); - final Integer columnIndex = nameToIndex.get(columnName); - if (projectMap.containsKey(columnIndex)) { - final RexNode columnConstraint = projectMap.get(columnIndex); - final ValidationError validationError = - new ValidationError(column.right, - RESOURCE.viewConstraintNotSatisfied(columnName, - Util.last(validatorTable.getQualifiedName()))); - RelOptUtil.validateValueAgainstConstraint(column.right, - columnConstraint, validationError); - } - } - } - } - - private void checkFieldCount(SqlNode node, SqlValidatorTable table, - SqlNode source, RelDataType logicalSourceRowType, - RelDataType logicalTargetRowType) { - final int sourceFieldCount = logicalSourceRowType.getFieldCount(); - final int targetFieldCount = logicalTargetRowType.getFieldCount(); - if (sourceFieldCount != targetFieldCount) { - throw newValidationError(node, - RESOURCE.unmatchInsertColumn(targetFieldCount, sourceFieldCount)); - } - // Ensure that non-nullable fields are targeted. - final InitializerContext rexBuilder = - new InitializerContext() { - public RexBuilder getRexBuilder() { - return new RexBuilder(typeFactory); - } - - public RexNode convertExpression(SqlNode e) { - throw new UnsupportedOperationException(); - } - }; - final List strategies = - table.unwrap(RelOptTable.class).getColumnStrategies(); - for (final RelDataTypeField field : table.getRowType().getFieldList()) { - final RelDataTypeField targetField = - logicalTargetRowType.getField(field.getName(), true, false); - switch (strategies.get(field.getIndex())) { - case NOT_NULLABLE: - assert !field.getType().isNullable(); - if (targetField == null) { - throw newValidationError(node, - RESOURCE.columnNotNullable(field.getName())); - } - break; - case NULLABLE: - assert field.getType().isNullable(); - break; - case VIRTUAL: - case STORED: - if (targetField != null - && !isValuesWithDefault(source, targetField.getIndex())) { - throw newValidationError(node, - RESOURCE.insertIntoAlwaysGenerated(field.getName())); - } - } - } - } - - /** Returns whether a query uses {@code DEFAULT} to populate a given - * column. */ - private boolean isValuesWithDefault(SqlNode source, int column) { - switch (source.getKind()) { - case VALUES: - for (SqlNode operand : ((SqlCall) source).getOperandList()) { - if (!isRowWithDefault(operand, column)) { - return false; - } - } - return true; - } - return false; - } - - private boolean isRowWithDefault(SqlNode operand, int column) { - switch (operand.getKind()) { - case ROW: - final SqlCall row = (SqlCall) operand; - return row.getOperandList().size() >= column - && row.getOperandList().get(column).getKind() == SqlKind.DEFAULT; - } - return false; - } - - protected RelDataType getLogicalTargetRowType( - RelDataType targetRowType, - SqlInsert insert) { - if (insert.getTargetColumnList() == null - && conformance.isInsertSubsetColumnsAllowed()) { - // Target an implicit subset of columns. - final SqlNode source = insert.getSource(); - final RelDataType sourceRowType = getNamespace(source).getRowType(); - final RelDataType logicalSourceRowType = - getLogicalSourceRowType(sourceRowType, insert); - final RelDataType implicitTargetRowType = - typeFactory.createStructType( - targetRowType.getFieldList() - .subList(0, logicalSourceRowType.getFieldCount())); - final SqlValidatorNamespace targetNamespace = getNamespace(insert); - validateNamespace(targetNamespace, implicitTargetRowType); - return implicitTargetRowType; - } else { - // Either the set of columns are explicitly targeted, or target the full - // set of columns. - return targetRowType; - } - } - - protected RelDataType getLogicalSourceRowType( - RelDataType sourceRowType, - SqlInsert insert) { - return sourceRowType; - } - - protected void checkTypeAssignment( - RelDataType sourceRowType, - RelDataType targetRowType, - final SqlNode query) { - // NOTE jvs 23-Feb-2006: subclasses may allow for extra targets - // representing system-maintained columns, so stop after all sources - // matched - List sourceFields = sourceRowType.getFieldList(); - List targetFields = targetRowType.getFieldList(); - final int sourceCount = sourceFields.size(); - for (int i = 0; i < sourceCount; ++i) { - RelDataType sourceType = sourceFields.get(i).getType(); - RelDataType targetType = targetFields.get(i).getType(); - if (!SqlTypeUtil.canAssignFrom(targetType, sourceType)) { - // FRG-255: account for UPDATE rewrite; there's - // probably a better way to do this. - int iAdjusted = i; - if (query instanceof SqlUpdate) { - int nUpdateColumns = - ((SqlUpdate) query).getTargetColumnList().size(); - assert sourceFields.size() >= nUpdateColumns; - iAdjusted -= sourceFields.size() - nUpdateColumns; - } - SqlNode node = getNthExpr(query, iAdjusted, sourceCount); - String targetTypeString; - String sourceTypeString; - if (SqlTypeUtil.areCharacterSetsMismatched( - sourceType, - targetType)) { - sourceTypeString = sourceType.getFullTypeString(); - targetTypeString = targetType.getFullTypeString(); - } else { - sourceTypeString = sourceType.toString(); - targetTypeString = targetType.toString(); - } - throw newValidationError(node, - RESOURCE.typeNotAssignable( - targetFields.get(i).getName(), targetTypeString, - sourceFields.get(i).getName(), sourceTypeString)); - } - } - } - - /** - * Locates the n'th expression in an INSERT or UPDATE query. - * - * @param query Query - * @param ordinal Ordinal of expression - * @param sourceCount Number of expressions - * @return Ordinal'th expression, never null - */ - private SqlNode getNthExpr(SqlNode query, int ordinal, int sourceCount) { - if (query instanceof SqlInsert) { - SqlInsert insert = (SqlInsert) query; - if (insert.getTargetColumnList() != null) { - return insert.getTargetColumnList().get(ordinal); - } else { - return getNthExpr( - insert.getSource(), - ordinal, - sourceCount); - } - } else if (query instanceof SqlUpdate) { - SqlUpdate update = (SqlUpdate) query; - if (update.getTargetColumnList() != null) { - return update.getTargetColumnList().get(ordinal); - } else if (update.getSourceExpressionList() != null) { - return update.getSourceExpressionList().get(ordinal); - } else { - return getNthExpr( - update.getSourceSelect(), - ordinal, - sourceCount); - } - } else if (query instanceof SqlSelect) { - SqlSelect select = (SqlSelect) query; - if (select.getSelectList().size() == sourceCount) { - return select.getSelectList().get(ordinal); - } else { - return query; // give up - } - } else { - return query; // give up - } - } - - public void validateDelete(SqlDelete call) { - final SqlSelect sqlSelect = call.getSourceSelect(); - validateSelect(sqlSelect, unknownType); - - final SqlValidatorNamespace targetNamespace = getNamespace(call); - validateNamespace(targetNamespace, unknownType); - final SqlValidatorTable table = targetNamespace.getTable(); - - validateAccess(call.getTargetTable(), table, SqlAccessEnum.DELETE); - } - - public void validateUpdate(SqlUpdate call) { - final SqlValidatorNamespace targetNamespace = getNamespace(call); - validateNamespace(targetNamespace, unknownType); - final RelOptTable relOptTable = SqlValidatorUtil.getRelOptTable( - targetNamespace, catalogReader.unwrap(Prepare.CatalogReader.class), null, null); - final SqlValidatorTable table = relOptTable == null - ? targetNamespace.getTable() - : relOptTable.unwrap(SqlValidatorTable.class); - - final RelDataType targetRowType = - createTargetRowType( - table, - call.getTargetColumnList(), - true); - - final SqlSelect select = call.getSourceSelect(); - validateSelect(select, targetRowType); - - final RelDataType sourceRowType = getNamespace(call).getRowType(); - checkTypeAssignment(sourceRowType, targetRowType, call); - - checkConstraint(table, call, targetRowType); - - validateAccess(call.getTargetTable(), table, SqlAccessEnum.UPDATE); - } - - public void validateMerge(SqlMerge call) { - SqlSelect sqlSelect = call.getSourceSelect(); - // REVIEW zfong 5/25/06 - Does an actual type have to be passed into - // validateSelect()? - - // REVIEW jvs 6-June-2006: In general, passing unknownType like - // this means we won't be able to correctly infer the types - // for dynamic parameter markers (SET x = ?). But - // maybe validateUpdate and validateInsert below will do - // the job? - - // REVIEW ksecretan 15-July-2011: They didn't get a chance to - // since validateSelect() would bail. - // Let's use the update/insert targetRowType when available. - IdentifierNamespace targetNamespace = - (IdentifierNamespace) getNamespace(call.getTargetTable()); - validateNamespace(targetNamespace, unknownType); - - SqlValidatorTable table = targetNamespace.getTable(); - validateAccess(call.getTargetTable(), table, SqlAccessEnum.UPDATE); - - RelDataType targetRowType = unknownType; - - if (call.getUpdateCall() != null) { - targetRowType = createTargetRowType( - table, - call.getUpdateCall().getTargetColumnList(), - true); - } - if (call.getInsertCall() != null) { - targetRowType = createTargetRowType( - table, - call.getInsertCall().getTargetColumnList(), - false); - } - - validateSelect(sqlSelect, targetRowType); - - if (call.getUpdateCall() != null) { - validateUpdate(call.getUpdateCall()); - } - if (call.getInsertCall() != null) { - validateInsert(call.getInsertCall()); - } - } - - /** - * Validates access to a table. - * - * @param table Table - * @param requiredAccess Access requested on table - */ - private void validateAccess( - SqlNode node, - SqlValidatorTable table, - SqlAccessEnum requiredAccess) { - if (table != null) { - SqlAccessType access = table.getAllowedAccess(); - if (!access.allowsAccess(requiredAccess)) { - throw newValidationError(node, - RESOURCE.accessNotAllowed(requiredAccess.name(), - table.getQualifiedName().toString())); - } - } - } - - /** - * Validates a VALUES clause. - * - * @param node Values clause - * @param targetRowType Row type which expression must conform to - * @param scope Scope within which clause occurs - */ - protected void validateValues( - SqlCall node, - RelDataType targetRowType, - final SqlValidatorScope scope) { - assert node.getKind() == SqlKind.VALUES; - - final List operands = node.getOperandList(); - for (SqlNode operand : operands) { - if (!(operand.getKind() == SqlKind.ROW)) { - throw Util.needToImplement( - "Values function where operands are scalars"); - } - - SqlCall rowConstructor = (SqlCall) operand; - if (conformance.isInsertSubsetColumnsAllowed() && targetRowType.isStruct() - && rowConstructor.operandCount() < targetRowType.getFieldCount()) { - targetRowType = - typeFactory.createStructType( - targetRowType.getFieldList() - .subList(0, rowConstructor.operandCount())); - } else if (targetRowType.isStruct() - && rowConstructor.operandCount() != targetRowType.getFieldCount()) { - return; - } - - inferUnknownTypes( - targetRowType, - scope, - rowConstructor); - - if (targetRowType.isStruct()) { - for (Pair pair - : Pair.zip(rowConstructor.getOperandList(), - targetRowType.getFieldList())) { - if (!pair.right.getType().isNullable() - && SqlUtil.isNullLiteral(pair.left, false)) { - throw newValidationError(node, - RESOURCE.columnNotNullable(pair.right.getName())); - } - } - } - } - - for (SqlNode operand : operands) { - operand.validate(this, scope); - } - - // validate that all row types have the same number of columns - // and that expressions in each column are compatible. - // A values expression is turned into something that looks like - // ROW(type00, type01,...), ROW(type11,...),... - final int rowCount = operands.size(); - if (rowCount >= 2) { - SqlCall firstRow = (SqlCall) operands.get(0); - final int columnCount = firstRow.operandCount(); - - // 1. check that all rows have the same cols length - for (SqlNode operand : operands) { - SqlCall thisRow = (SqlCall) operand; - if (columnCount != thisRow.operandCount()) { - throw newValidationError(node, - RESOURCE.incompatibleValueType( - SqlStdOperatorTable.VALUES.getName())); - } - } - - // 2. check if types at i:th position in each row are compatible - for (int col = 0; col < columnCount; col++) { - final int c = col; - final RelDataType type = - typeFactory.leastRestrictive( - new AbstractList() { - public RelDataType get(int row) { - SqlCall thisRow = (SqlCall) operands.get(row); - return deriveType(scope, thisRow.operand(c)); - } - - public int size() { - return rowCount; - } - }); - - if (null == type) { - throw newValidationError(node, - RESOURCE.incompatibleValueType( - SqlStdOperatorTable.VALUES.getName())); - } - } - } - } - - public void validateDataType(SqlDataTypeSpec dataType) { - } - - public void validateDynamicParam(SqlDynamicParam dynamicParam) { - } - - /** - * Throws a validator exception with access to the validator context. - * The exception is determined when an instance is created. - */ - private class ValidationError implements Supplier { - private final SqlNode sqlNode; - private final Resources.ExInst validatorException; - - ValidationError(SqlNode sqlNode, - Resources.ExInst validatorException) { - this.sqlNode = sqlNode; - this.validatorException = validatorException; - } - - public CalciteContextException get() { - return newValidationError(sqlNode, validatorException); - } - } - - /** - * Throws a validator exception with access to the validator context. - * The exception is determined when the function is applied. - */ - class ValidationErrorFunction - implements Function2, - CalciteContextException> { - @Override public CalciteContextException apply( - SqlNode v0, Resources.ExInst v1) { - return newValidationError(v0, v1); - } - } - - public ValidationErrorFunction getValidationErrorFunction() { - return validationErrorFunction; - } - - public CalciteContextException newValidationError(SqlNode node, - Resources.ExInst e) { - assert node != null; - final SqlParserPos pos = node.getParserPosition(); - return SqlUtil.newContextException(pos, e); - } - - protected SqlWindow getWindowByName( - SqlIdentifier id, - SqlValidatorScope scope) { - SqlWindow window = null; - if (id.isSimple()) { - final String name = id.getSimple(); - window = scope.lookupWindow(name); - } - if (window == null) { - throw newValidationError(id, RESOURCE.windowNotFound(id.toString())); - } - return window; - } - - public SqlWindow resolveWindow( - SqlNode windowOrRef, - SqlValidatorScope scope, - boolean populateBounds) { - SqlWindow window; - if (windowOrRef instanceof SqlIdentifier) { - window = getWindowByName((SqlIdentifier) windowOrRef, scope); - } else { - window = (SqlWindow) windowOrRef; - } - while (true) { - final SqlIdentifier refId = window.getRefName(); - if (refId == null) { - break; - } - final String refName = refId.getSimple(); - SqlWindow refWindow = scope.lookupWindow(refName); - if (refWindow == null) { - throw newValidationError(refId, RESOURCE.windowNotFound(refName)); - } - window = window.overlay(refWindow, this); - } - - if (populateBounds) { - window.populateBounds(); - } - return window; - } - - public SqlNode getOriginal(SqlNode expr) { - SqlNode original = originalExprs.get(expr); - if (original == null) { - original = expr; - } - return original; - } - - public void setOriginal(SqlNode expr, SqlNode original) { - // Don't overwrite the original original. - originalExprs.putIfAbsent(expr, original); - } - - SqlValidatorNamespace lookupFieldNamespace(RelDataType rowType, String name) { - final SqlNameMatcher nameMatcher = catalogReader.nameMatcher(); - final RelDataTypeField field = nameMatcher.field(rowType, name); - if (field == null) { - return null; - } - return new FieldNamespace(this, field.getType()); - } - - public void validateWindow( - SqlNode windowOrId, - SqlValidatorScope scope, - SqlCall call) { - // Enable nested aggregates with window aggregates (OVER operator) - inWindow = true; - - final SqlWindow targetWindow; - switch (windowOrId.getKind()) { - case IDENTIFIER: - // Just verify the window exists in this query. It will validate - // when the definition is processed - targetWindow = getWindowByName((SqlIdentifier) windowOrId, scope); - break; - case WINDOW: - targetWindow = (SqlWindow) windowOrId; - break; - default: - throw Util.unexpected(windowOrId.getKind()); - } - - assert targetWindow.getWindowCall() == null; - targetWindow.setWindowCall(call); - targetWindow.validate(this, scope); - targetWindow.setWindowCall(null); - call.validate(this, scope); - - validateAggregateParams(call, null, null, scope); - - // Disable nested aggregates post validation - inWindow = false; - } - - @Override public void validateMatchRecognize(SqlCall call) { - final SqlMatchRecognize matchRecognize = (SqlMatchRecognize) call; - final MatchRecognizeScope scope = - (MatchRecognizeScope) getMatchRecognizeScope(matchRecognize); - - final MatchRecognizeNamespace ns = - getNamespace(call).unwrap(MatchRecognizeNamespace.class); - assert ns.rowType == null; - - // rows per match - final SqlLiteral rowsPerMatch = matchRecognize.getRowsPerMatch(); - final boolean allRows = rowsPerMatch != null - && rowsPerMatch.getValue() - == SqlMatchRecognize.RowsPerMatchOption.ALL_ROWS; - - final RelDataTypeFactory.Builder typeBuilder = typeFactory.builder(); - - // parse PARTITION BY column - SqlNodeList partitionBy = matchRecognize.getPartitionList(); - if (partitionBy != null) { - for (SqlNode node : partitionBy) { - SqlIdentifier identifier = (SqlIdentifier) node; - identifier.validate(this, scope); - RelDataType type = deriveType(scope, identifier); - String name = identifier.names.get(1); - typeBuilder.add(name, type); - } - } - - // parse ORDER BY column - SqlNodeList orderBy = matchRecognize.getOrderList(); - if (orderBy != null) { - for (SqlNode node : orderBy) { - node.validate(this, scope); - SqlIdentifier identifier; - if (node instanceof SqlBasicCall) { - identifier = (SqlIdentifier) ((SqlBasicCall) node).getOperands()[0]; - } else { - identifier = (SqlIdentifier) node; - } - - if (allRows) { - RelDataType type = deriveType(scope, identifier); - String name = identifier.names.get(1); - if (!typeBuilder.nameExists(name)) { - typeBuilder.add(name, type); - } - } - } - } - - if (allRows) { - final SqlValidatorNamespace sqlNs = - getNamespace(matchRecognize.getTableRef()); - final RelDataType inputDataType = sqlNs.getRowType(); - for (RelDataTypeField fs : inputDataType.getFieldList()) { - if (!typeBuilder.nameExists(fs.getName())) { - typeBuilder.add(fs); - } - } - } - - // retrieve pattern variables used in pattern and subset - SqlNode pattern = matchRecognize.getPattern(); - PatternVarVisitor visitor = new PatternVarVisitor(scope); - pattern.accept(visitor); - - SqlLiteral interval = matchRecognize.getInterval(); - if (interval != null) { - interval.validate(this, scope); - if (((SqlIntervalLiteral) interval).signum() < 0) { - throw newValidationError(interval, - RESOURCE.intervalMustBeNonNegative(interval.toValue())); - } - if (orderBy == null || orderBy.size() == 0) { - throw newValidationError(interval, - RESOURCE.cannotUseWithinWithoutOrderBy()); - } - - SqlNode firstOrderByColumn = orderBy.getList().get(0); - SqlIdentifier identifier; - if (firstOrderByColumn instanceof SqlBasicCall) { - identifier = (SqlIdentifier) ((SqlBasicCall) firstOrderByColumn).getOperands()[0]; - } else { - identifier = (SqlIdentifier) firstOrderByColumn; - } - RelDataType firstOrderByColumnType = deriveType(scope, identifier); - if (firstOrderByColumnType.getSqlTypeName() != SqlTypeName.TIMESTAMP) { - throw newValidationError(interval, - RESOURCE.firstColumnOfOrderByMustBeTimestamp()); - } - - SqlNode expand = expand(interval, scope); - RelDataType type = deriveType(scope, expand); - setValidatedNodeType(interval, type); - } - - validateDefinitions(matchRecognize, scope); - - SqlNodeList subsets = matchRecognize.getSubsetList(); - if (subsets != null && subsets.size() > 0) { - for (SqlNode node : subsets) { - List operands = ((SqlCall) node).getOperandList(); - String leftString = ((SqlIdentifier) operands.get(0)).getSimple(); - if (scope.getPatternVars().contains(leftString)) { - throw newValidationError(operands.get(0), - RESOURCE.patternVarAlreadyDefined(leftString)); - } - scope.addPatternVar(leftString); - for (SqlNode right : (SqlNodeList) operands.get(1)) { - SqlIdentifier id = (SqlIdentifier) right; - if (!scope.getPatternVars().contains(id.getSimple())) { - throw newValidationError(id, - RESOURCE.unknownPattern(id.getSimple())); - } - scope.addPatternVar(id.getSimple()); - } - } - } - - // validate AFTER ... SKIP TO - final SqlNode skipTo = matchRecognize.getAfter(); - if (skipTo instanceof SqlCall) { - final SqlCall skipToCall = (SqlCall) skipTo; - final SqlIdentifier id = skipToCall.operand(0); - if (!scope.getPatternVars().contains(id.getSimple())) { - throw newValidationError(id, - RESOURCE.unknownPattern(id.getSimple())); - } - } - - List> measureColumns = - validateMeasure(matchRecognize, scope, allRows); - for (Map.Entry c : measureColumns) { - if (!typeBuilder.nameExists(c.getKey())) { - typeBuilder.add(c.getKey(), c.getValue()); - } - } - - final RelDataType rowType = typeBuilder.build(); - if (matchRecognize.getMeasureList().size() == 0) { - ns.setType(getNamespace(matchRecognize.getTableRef()).getRowType()); - } else { - ns.setType(rowType); - } - } - - private List> validateMeasure(SqlMatchRecognize mr, - MatchRecognizeScope scope, boolean allRows) { - final List aliases = new ArrayList<>(); - final List sqlNodes = new ArrayList<>(); - final SqlNodeList measures = mr.getMeasureList(); - final List> fields = new ArrayList<>(); - - for (SqlNode measure : measures) { - assert measure instanceof SqlCall; - final String alias = deriveAlias(measure, aliases.size()); - aliases.add(alias); - - SqlNode expand = expand(measure, scope); - expand = navigationInMeasure(expand, allRows); - setOriginal(expand, measure); - - inferUnknownTypes(unknownType, scope, expand); - final RelDataType type = deriveType(scope, expand); - setValidatedNodeType(measure, type); - - fields.add(Pair.of(alias, type)); - sqlNodes.add( - SqlStdOperatorTable.AS.createCall(SqlParserPos.ZERO, expand, - new SqlIdentifier(alias, SqlParserPos.ZERO))); - } - - SqlNodeList list = new SqlNodeList(sqlNodes, measures.getParserPosition()); - inferUnknownTypes(unknownType, scope, list); - - for (SqlNode node : list) { - validateExpr(node, scope); - } - - mr.setOperand(SqlMatchRecognize.OPERAND_MEASURES, list); - - return fields; - } - - private SqlNode navigationInMeasure(SqlNode node, boolean allRows) { - final Set prefix = node.accept(new PatternValidator(true)); - Util.discard(prefix); - final List ops = ((SqlCall) node).getOperandList(); - - final SqlOperator defaultOp = - allRows ? SqlStdOperatorTable.RUNNING : SqlStdOperatorTable.FINAL; - final SqlNode op0 = ops.get(0); - if (!isRunningOrFinal(op0.getKind()) - || !allRows && op0.getKind() == SqlKind.RUNNING) { - SqlNode newNode = defaultOp.createCall(SqlParserPos.ZERO, op0); - node = SqlStdOperatorTable.AS.createCall(SqlParserPos.ZERO, newNode, ops.get(1)); - } - - node = new NavigationExpander().go(node); - return node; - } - - private void validateDefinitions(SqlMatchRecognize mr, - MatchRecognizeScope scope) { - final Set aliases = catalogReader.nameMatcher().createSet(); - for (SqlNode item : mr.getPatternDefList().getList()) { - final String alias = alias(item); - if (!aliases.add(alias)) { - throw newValidationError(item, - Static.RESOURCE.patternVarAlreadyDefined(alias)); - } - scope.addPatternVar(alias); - } - - final List sqlNodes = new ArrayList<>(); - for (SqlNode item : mr.getPatternDefList().getList()) { - final String alias = alias(item); - SqlNode expand = expand(item, scope); - expand = navigationInDefine(expand, alias); - setOriginal(expand, item); - - inferUnknownTypes(booleanType, scope, expand); - expand.validate(this, scope); - - // Some extra work need required here. - // In PREV, NEXT, FINAL and LAST, only one pattern variable is allowed. - sqlNodes.add( - SqlStdOperatorTable.AS.createCall(SqlParserPos.ZERO, expand, - new SqlIdentifier(alias, SqlParserPos.ZERO))); - - final RelDataType type = deriveType(scope, expand); - if (!SqlTypeUtil.inBooleanFamily(type)) { - throw newValidationError(expand, RESOURCE.condMustBeBoolean("DEFINE")); - } - setValidatedNodeType(item, type); - } - - SqlNodeList list = - new SqlNodeList(sqlNodes, mr.getPatternDefList().getParserPosition()); - inferUnknownTypes(unknownType, scope, list); - for (SqlNode node : list) { - validateExpr(node, scope); - } - mr.setOperand(SqlMatchRecognize.OPERAND_PATTERN_DEFINES, list); - } - - /** Returns the alias of a "expr AS alias" expression. */ - private static String alias(SqlNode item) { - assert item instanceof SqlCall; - assert item.getKind() == SqlKind.AS; - final SqlIdentifier identifier = ((SqlCall) item).operand(1); - return identifier.getSimple(); - } - - /** Checks that all pattern variables within a function are the same, - * and canonizes expressions such as {@code PREV(B.price)} to - * {@code LAST(B.price, 0)}. */ - private SqlNode navigationInDefine(SqlNode node, String alpha) { - Set prefix = node.accept(new PatternValidator(false)); - Util.discard(prefix); - node = new NavigationExpander().go(node); - node = new NavigationReplacer(alpha).go(node); - return node; - } - - public void validateAggregateParams(SqlCall aggCall, SqlNode filter, - SqlNodeList orderList, SqlValidatorScope scope) { - // For "agg(expr)", expr cannot itself contain aggregate function - // invocations. For example, "SUM(2 * MAX(x))" is illegal; when - // we see it, we'll report the error for the SUM (not the MAX). - // For more than one level of nesting, the error which results - // depends on the traversal order for validation. - // - // For a windowed aggregate "agg(expr)", expr can contain an aggregate - // function. For example, - // SELECT AVG(2 * MAX(x)) OVER (PARTITION BY y) - // FROM t - // GROUP BY y - // is legal. Only one level of nesting is allowed since non-windowed - // aggregates cannot nest aggregates. - - // Store nesting level of each aggregate. If an aggregate is found at an invalid - // nesting level, throw an assert. - final AggFinder a; - if (inWindow) { - a = overFinder; - } else { - a = aggOrOverFinder; - } - - for (SqlNode param : aggCall.getOperandList()) { - if (a.findAgg(param) != null) { - throw newValidationError(aggCall, RESOURCE.nestedAggIllegal()); - } - } - if (filter != null) { - if (a.findAgg(filter) != null) { - throw newValidationError(filter, RESOURCE.aggregateInFilterIllegal()); - } - } - if (orderList != null) { - for (SqlNode param : orderList) { - if (a.findAgg(param) != null) { - throw newValidationError(aggCall, - RESOURCE.aggregateInWithinGroupIllegal()); - } - } - } - - final SqlAggFunction op = (SqlAggFunction) aggCall.getOperator(); - switch (op.requiresGroupOrder()) { - case MANDATORY: - if (orderList == null || orderList.size() == 0) { - throw newValidationError(aggCall, - RESOURCE.aggregateMissingWithinGroupClause(op.getName())); - } - break; - case OPTIONAL: - break; - case IGNORED: - // rewrite the order list to empty - if (orderList != null) { - orderList.getList().clear(); - } - break; - case FORBIDDEN: - if (orderList != null && orderList.size() != 0) { - throw newValidationError(aggCall, - RESOURCE.withinGroupClauseIllegalInAggregate(op.getName())); - } - break; - default: - throw new AssertionError(op); - } - } - - public void validateCall( - SqlCall call, - SqlValidatorScope scope) { - final SqlOperator operator = call.getOperator(); - if ((call.operandCount() == 0) - && (operator.getSyntax() == SqlSyntax.FUNCTION_ID) - && !call.isExpanded() - && !conformance.allowNiladicParentheses()) { - // For example, "LOCALTIME()" is illegal. (It should be - // "LOCALTIME", which would have been handled as a - // SqlIdentifier.) - throw handleUnresolvedFunction(call, (SqlFunction) operator, - ImmutableList.of(), null); - } - - SqlValidatorScope operandScope = scope.getOperandScope(call); - - if (operator instanceof SqlFunction - && ((SqlFunction) operator).getFunctionType() - == SqlFunctionCategory.MATCH_RECOGNIZE - && !(operandScope instanceof MatchRecognizeScope)) { - throw newValidationError(call, - Static.RESOURCE.functionMatchRecognizeOnly(call.toString())); - } - // Delegate validation to the operator. - operator.validateCall(call, this, scope, operandScope); - } - - /** - * Validates that a particular feature is enabled. By default, all features - * are enabled; subclasses may override this method to be more - * discriminating. - * - * @param feature feature being used, represented as a resource instance - * @param context parser position context for error reporting, or null if - */ - protected void validateFeature( - Feature feature, - SqlParserPos context) { - // By default, do nothing except to verify that the resource - // represents a real feature definition. - assert feature.getProperties().get("FeatureDefinition") != null; - } - - public SqlNode expand(SqlNode expr, SqlValidatorScope scope) { - final Expander expander = new Expander(this, scope); - SqlNode newExpr = expr.accept(expander); - if (expr != newExpr) { - setOriginal(newExpr, expr); - } - return newExpr; - } - - public SqlNode expandGroupByOrHavingExpr(SqlNode expr, - SqlValidatorScope scope, SqlSelect select, boolean havingExpression) { - final Expander expander = new ExtendedExpander(this, scope, select, expr, - havingExpression); - SqlNode newExpr = expr.accept(expander); - if (expr != newExpr) { - setOriginal(newExpr, expr); - } - return newExpr; - } - - public boolean isSystemField(RelDataTypeField field) { - return false; - } - - public List> getFieldOrigins(SqlNode sqlQuery) { - if (sqlQuery instanceof SqlExplain) { - return Collections.emptyList(); - } - final RelDataType rowType = getValidatedNodeType(sqlQuery); - final int fieldCount = rowType.getFieldCount(); - if (!sqlQuery.isA(SqlKind.QUERY)) { - return Collections.nCopies(fieldCount, null); - } - final List> list = new ArrayList<>(); - for (int i = 0; i < fieldCount; i++) { - list.add(getFieldOrigin(sqlQuery, i)); - } - return ImmutableNullableList.copyOf(list); - } - - private List getFieldOrigin(SqlNode sqlQuery, int i) { - if (sqlQuery instanceof SqlSelect) { - SqlSelect sqlSelect = (SqlSelect) sqlQuery; - final SelectScope scope = getRawSelectScope(sqlSelect); - final List selectList = scope.getExpandedSelectList(); - final SqlNode selectItem = stripAs(selectList.get(i)); - if (selectItem instanceof SqlIdentifier) { - final SqlQualified qualified = - scope.fullyQualify((SqlIdentifier) selectItem); - SqlValidatorNamespace namespace = qualified.namespace; - final SqlValidatorTable table = namespace.getTable(); - if (table == null) { - return null; - } - final List origin = - new ArrayList<>(table.getQualifiedName()); - for (String name : qualified.suffix()) { - namespace = namespace.lookupChild(name); - if (namespace == null) { - return null; - } - origin.add(name); - } - return origin; - } - return null; - } else if (sqlQuery instanceof SqlOrderBy) { - return getFieldOrigin(((SqlOrderBy) sqlQuery).query, i); - } else { - return null; - } - } - - public RelDataType getParameterRowType(SqlNode sqlQuery) { - // NOTE: We assume that bind variables occur in depth-first tree - // traversal in the same order that they occurred in the SQL text. - final List types = new ArrayList<>(); - // NOTE: but parameters on fetch/offset would be counted twice - // as they are counted in the SqlOrderBy call and the inner SqlSelect call - final Set alreadyVisited = new HashSet<>(); - sqlQuery.accept( - new SqlShuttle() { - - @Override public SqlNode visit(SqlDynamicParam param) { - if (alreadyVisited.add(param)) { - RelDataType type = getValidatedNodeType(param); - types.add(type); - } - return param; - } - }); - return typeFactory.createStructType( - types, - new AbstractList() { - @Override public String get(int index) { - return "?" + index; - } - - @Override public int size() { - return types.size(); - } - }); - } - - public void validateColumnListParams( - SqlFunction function, - List argTypes, - List operands) { - throw new UnsupportedOperationException(); - } - - private static boolean isPhysicalNavigation(SqlKind kind) { - return kind == SqlKind.PREV || kind == SqlKind.NEXT; - } - - private static boolean isLogicalNavigation(SqlKind kind) { - return kind == SqlKind.FIRST || kind == SqlKind.LAST; - } - - private static boolean isAggregation(SqlKind kind) { - return kind == SqlKind.SUM || kind == SqlKind.SUM0 - || kind == SqlKind.AVG || kind == SqlKind.COUNT - || kind == SqlKind.MAX || kind == SqlKind.MIN; - } - - private static boolean isRunningOrFinal(SqlKind kind) { - return kind == SqlKind.RUNNING || kind == SqlKind.FINAL; - } - - private static boolean isSingleVarRequired(SqlKind kind) { - return isPhysicalNavigation(kind) - || isLogicalNavigation(kind) - || isAggregation(kind); - } - - //~ Inner Classes ---------------------------------------------------------- - - /** - * Common base class for DML statement namespaces. - */ - public static class DmlNamespace extends IdentifierNamespace { - protected DmlNamespace(SqlValidatorImpl validator, SqlNode id, - SqlNode enclosingNode, SqlValidatorScope parentScope) { - super(validator, id, enclosingNode, parentScope); - } - } - - /** - * Namespace for an INSERT statement. - */ - private static class InsertNamespace extends DmlNamespace { - private final SqlInsert node; - - InsertNamespace(SqlValidatorImpl validator, SqlInsert node, - SqlNode enclosingNode, SqlValidatorScope parentScope) { - super(validator, node.getTargetTable(), enclosingNode, parentScope); - this.node = Objects.requireNonNull(node); - } - - public SqlInsert getNode() { - return node; - } - } - - /** - * Namespace for an UPDATE statement. - */ - private static class UpdateNamespace extends DmlNamespace { - private final SqlUpdate node; - - UpdateNamespace(SqlValidatorImpl validator, SqlUpdate node, - SqlNode enclosingNode, SqlValidatorScope parentScope) { - super(validator, node.getTargetTable(), enclosingNode, parentScope); - this.node = Objects.requireNonNull(node); - } - - public SqlUpdate getNode() { - return node; - } - } - - /** - * Namespace for a DELETE statement. - */ - private static class DeleteNamespace extends DmlNamespace { - private final SqlDelete node; - - DeleteNamespace(SqlValidatorImpl validator, SqlDelete node, - SqlNode enclosingNode, SqlValidatorScope parentScope) { - super(validator, node.getTargetTable(), enclosingNode, parentScope); - this.node = Objects.requireNonNull(node); - } - - public SqlDelete getNode() { - return node; - } - } - - /** - * Namespace for a MERGE statement. - */ - private static class MergeNamespace extends DmlNamespace { - private final SqlMerge node; - - MergeNamespace(SqlValidatorImpl validator, SqlMerge node, - SqlNode enclosingNode, SqlValidatorScope parentScope) { - super(validator, node.getTargetTable(), enclosingNode, parentScope); - this.node = Objects.requireNonNull(node); - } - - public SqlMerge getNode() { - return node; - } - } - - /** - * retrieve pattern variables defined - */ - private class PatternVarVisitor implements SqlVisitor { - private MatchRecognizeScope scope; - PatternVarVisitor(MatchRecognizeScope scope) { - this.scope = scope; - } - - @Override public Void visit(SqlLiteral literal) { - return null; - } - - @Override public Void visit(SqlCall call) { - for (int i = 0; i < call.getOperandList().size(); i++) { - call.getOperandList().get(i).accept(this); - } - return null; - } - - @Override public Void visit(SqlNodeList nodeList) { - throw Util.needToImplement(nodeList); - } - - @Override public Void visit(SqlIdentifier id) { - Preconditions.checkArgument(id.isSimple()); - scope.addPatternVar(id.getSimple()); - return null; - } - - @Override public Void visit(SqlDataTypeSpec type) { - throw Util.needToImplement(type); - } - - @Override public Void visit(SqlDynamicParam param) { - throw Util.needToImplement(param); - } - - @Override public Void visit(SqlIntervalQualifier intervalQualifier) { - throw Util.needToImplement(intervalQualifier); - } - } - - /** - * Visitor which derives the type of a given {@link SqlNode}. - * - *

    Each method must return the derived type. This visitor is basically a - * single-use dispatcher; the visit is never recursive. - */ - private class DeriveTypeVisitor implements SqlVisitor { - private final SqlValidatorScope scope; - - DeriveTypeVisitor(SqlValidatorScope scope) { - this.scope = scope; - } - - public RelDataType visit(SqlLiteral literal) { - return literal.createSqlType(typeFactory); - } - - public RelDataType visit(SqlCall call) { - final SqlOperator operator = call.getOperator(); - return operator.deriveType(SqlValidatorImpl.this, scope, call); - } - - public RelDataType visit(SqlNodeList nodeList) { - // Operand is of a type that we can't derive a type for. If the - // operand is of a peculiar type, such as a SqlNodeList, then you - // should override the operator's validateCall() method so that it - // doesn't try to validate that operand as an expression. - throw Util.needToImplement(nodeList); - } - - public RelDataType visit(SqlIdentifier id) { - // First check for builtin functions which don't have parentheses, - // like "LOCALTIME". - final SqlCall call = makeNullaryCall(id); - if (call != null) { - return call.getOperator().validateOperands( - SqlValidatorImpl.this, - scope, - call); - } - - RelDataType type = null; - if (!(scope instanceof EmptyScope)) { - id = scope.fullyQualify(id).identifier; - } - - // Resolve the longest prefix of id that we can - int i; - for (i = id.names.size() - 1; i > 0; i--) { - // REVIEW jvs 9-June-2005: The name resolution rules used - // here are supposed to match SQL:2003 Part 2 Section 6.6 - // (identifier chain), but we don't currently have enough - // information to get everything right. In particular, - // routine parameters are currently looked up via resolve; - // we could do a better job if they were looked up via - // resolveColumn. - - final SqlNameMatcher nameMatcher = catalogReader.nameMatcher(); - final SqlValidatorScope.ResolvedImpl resolved = - new SqlValidatorScope.ResolvedImpl(); - scope.resolve(id.names.subList(0, i), nameMatcher, false, resolved); - if (resolved.count() == 1) { - // There's a namespace with the name we seek. - final SqlValidatorScope.Resolve resolve = resolved.only(); - type = resolve.rowType(); - for (SqlValidatorScope.Step p : Util.skip(resolve.path.steps())) { - type = type.getFieldList().get(p.i).getType(); - } - break; - } - } - - // Give precedence to namespace found, unless there - // are no more identifier components. - if (type == null || id.names.size() == 1) { - // See if there's a column with the name we seek in - // precisely one of the namespaces in this scope. - RelDataType colType = scope.resolveColumn(id.names.get(0), id); - if (colType != null) { - type = colType; - } - ++i; - } - - if (type == null) { - final SqlIdentifier last = id.getComponent(i - 1, i); - throw newValidationError(last, - RESOURCE.unknownIdentifier(last.toString())); - } - - // Resolve rest of identifier - for (; i < id.names.size(); i++) { - String name = id.names.get(i); - final RelDataTypeField field; - if (name.equals("")) { - // The wildcard "*" is represented as an empty name. It never - // resolves to a field. - name = "*"; - field = null; - } else { - final SqlNameMatcher nameMatcher = catalogReader.nameMatcher(); - field = nameMatcher.field(type, name); - } - if (field == null) { - throw newValidationError(id.getComponent(i), - RESOURCE.unknownField(name)); - } - type = field.getType(); - } - type = - SqlTypeUtil.addCharsetAndCollation( - type, - getTypeFactory()); - return type; - } - - public RelDataType visit(SqlDataTypeSpec dataType) { - // Q. How can a data type have a type? - // A. When it appears in an expression. (Say as the 2nd arg to the - // CAST operator.) - validateDataType(dataType); - return dataType.deriveType(SqlValidatorImpl.this); - } - - public RelDataType visit(SqlDynamicParam param) { - return unknownType; - } - - public RelDataType visit(SqlIntervalQualifier intervalQualifier) { - return typeFactory.createSqlIntervalType(intervalQualifier); - } - } - - /** - * Converts an expression into canonical form by fully-qualifying any - * identifiers. - */ - private static class Expander extends SqlScopedShuttle { - protected final SqlValidatorImpl validator; - - Expander(SqlValidatorImpl validator, SqlValidatorScope scope) { - super(scope); - this.validator = validator; - } - - @Override public SqlNode visit(SqlIdentifier id) { - // First check for builtin functions which don't have - // parentheses, like "LOCALTIME". - final SqlCall call = validator.makeNullaryCall(id); - if (call != null) { - return call.accept(this); - } - final SqlIdentifier fqId = getScope().fullyQualify(id).identifier; - SqlNode expandedExpr = expandDynamicStar(id, fqId); - validator.setOriginal(expandedExpr, id); - return expandedExpr; - } - - @Override protected SqlNode visitScoped(SqlCall call) { - switch (call.getKind()) { - case SCALAR_QUERY: - case CURRENT_VALUE: - case NEXT_VALUE: - case WITH: - return call; - } - // Only visits arguments which are expressions. We don't want to - // qualify non-expressions such as 'x' in 'empno * 5 AS x'. - ArgHandler argHandler = - new CallCopyingArgHandler(call, false); - call.getOperator().acceptCall(this, call, true, argHandler); - final SqlNode result = argHandler.result(); - validator.setOriginal(result, call); - return result; - } - - protected SqlNode expandDynamicStar(SqlIdentifier id, SqlIdentifier fqId) { - if (DynamicRecordType.isDynamicStarColName(Util.last(fqId.names)) - && !DynamicRecordType.isDynamicStarColName(Util.last(id.names))) { - // Convert a column ref into ITEM(*, 'col_name') - // for a dynamic star field in dynTable's rowType. - SqlNode[] inputs = new SqlNode[2]; - inputs[0] = fqId; - inputs[1] = SqlLiteral.createCharString( - Util.last(id.names), - id.getParserPosition()); - return new SqlBasicCall( - SqlStdOperatorTable.ITEM, - inputs, - id.getParserPosition()); - } - return fqId; - } - } - - /** - * Shuttle which walks over an expression in the ORDER BY clause, replacing - * usages of aliases with the underlying expression. - */ - class OrderExpressionExpander extends SqlScopedShuttle { - private final List aliasList; - private final SqlSelect select; - private final SqlNode root; - - OrderExpressionExpander(SqlSelect select, SqlNode root) { - super(getOrderScope(select)); - this.select = select; - this.root = root; - this.aliasList = getNamespace(select).getRowType().getFieldNames(); - } - - public SqlNode go() { - return root.accept(this); - } - - public SqlNode visit(SqlLiteral literal) { - // Ordinal markers, e.g. 'select a, b from t order by 2'. - // Only recognize them if they are the whole expression, - // and if the dialect permits. - if (literal == root && getConformance().isSortByOrdinal()) { - switch (literal.getTypeName()) { - case DECIMAL: - case DOUBLE: - final int intValue = literal.intValue(false); - if (intValue >= 0) { - if (intValue < 1 || intValue > aliasList.size()) { - throw newValidationError( - literal, RESOURCE.orderByOrdinalOutOfRange()); - } - - // SQL ordinals are 1-based, but Sort's are 0-based - int ordinal = intValue - 1; - return nthSelectItem(ordinal, literal.getParserPosition()); - } - break; - } - } - - return super.visit(literal); - } - - /** - * Returns the ordinalth item in the select list. - */ - private SqlNode nthSelectItem(int ordinal, final SqlParserPos pos) { - // TODO: Don't expand the list every time. Maybe keep an expanded - // version of each expression -- select lists and identifiers -- in - // the validator. - - SqlNodeList expandedSelectList = - expandStar( - select.getSelectList(), - select, - false); - SqlNode expr = expandedSelectList.get(ordinal); - expr = stripAs(expr); - if (expr instanceof SqlIdentifier) { - expr = getScope().fullyQualify((SqlIdentifier) expr).identifier; - } - - // Create a copy of the expression with the position of the order - // item. - return expr.clone(pos); - } - - public SqlNode visit(SqlIdentifier id) { - // Aliases, e.g. 'select a as x, b from t order by x'. - if (id.isSimple() - && getConformance().isSortByAlias()) { - String alias = id.getSimple(); - final SqlValidatorNamespace selectNs = getNamespace(select); - final RelDataType rowType = - selectNs.getRowTypeSansSystemColumns(); - final SqlNameMatcher nameMatcher = catalogReader.nameMatcher(); - RelDataTypeField field = nameMatcher.field(rowType, alias); - if (field != null) { - return nthSelectItem( - field.getIndex(), - id.getParserPosition()); - } - } - - // No match. Return identifier unchanged. - return getScope().fullyQualify(id).identifier; - } - - protected SqlNode visitScoped(SqlCall call) { - // Don't attempt to expand sub-queries. We haven't implemented - // these yet. - if (call instanceof SqlSelect) { - return call; - } - return super.visitScoped(call); - } - } - - /** - * Shuttle which walks over an expression in the GROUP BY/HAVING clause, replacing - * usages of aliases or ordinals with the underlying expression. - */ - static class ExtendedExpander extends Expander { - final SqlSelect select; - final SqlNode root; - final boolean havingExpr; - - ExtendedExpander(SqlValidatorImpl validator, SqlValidatorScope scope, - SqlSelect select, SqlNode root, boolean havingExpr) { - super(validator, scope); - this.select = select; - this.root = root; - this.havingExpr = havingExpr; - } - - @Override public SqlNode visit(SqlIdentifier id) { - if (id.isSimple() - && (havingExpr - ? validator.getConformance().isHavingAlias() - : validator.getConformance().isGroupByAlias())) { - String name = id.getSimple(); - SqlNode expr = null; - final SqlNameMatcher nameMatcher = - validator.catalogReader.nameMatcher(); - int n = 0; - for (SqlNode s : select.getSelectList()) { - final String alias = SqlValidatorUtil.getAlias(s, -1); - if (alias != null && nameMatcher.matches(alias, name)) { - expr = s; - n++; - } - } - if (n == 0) { - return super.visit(id); - } else if (n > 1) { - // More than one column has this alias. - throw validator.newValidationError(id, - RESOURCE.columnAmbiguous(name)); - } - if (havingExpr && validator.isAggregate(root)) { - return super.visit(id); - } - expr = stripAs(expr); - if (expr instanceof SqlIdentifier) { - SqlIdentifier sid = (SqlIdentifier) expr; - final SqlIdentifier fqId = getScope().fullyQualify(sid).identifier; - expr = expandDynamicStar(sid, fqId); - } - return expr; - } - return super.visit(id); - } - - public SqlNode visit(SqlLiteral literal) { - if (havingExpr || !validator.getConformance().isGroupByOrdinal()) { - return super.visit(literal); - } - boolean isOrdinalLiteral = literal == root; - switch (root.getKind()) { - case GROUPING_SETS: - case ROLLUP: - case CUBE: - if (root instanceof SqlBasicCall) { - List operandList = ((SqlBasicCall) root).getOperandList(); - for (SqlNode node : operandList) { - if (node.equals(literal)) { - isOrdinalLiteral = true; - break; - } - } - } - break; - } - if (isOrdinalLiteral) { - switch (literal.getTypeName()) { - case DECIMAL: - case DOUBLE: - final int intValue = literal.intValue(false); - if (intValue >= 0) { - if (intValue < 1 || intValue > select.getSelectList().size()) { - throw validator.newValidationError(literal, - RESOURCE.orderByOrdinalOutOfRange()); - } - - // SQL ordinals are 1-based, but Sort's are 0-based - int ordinal = intValue - 1; - return SqlUtil.stripAs(select.getSelectList().get(ordinal)); - } - break; - } - } - - return super.visit(literal); - } - } - - /** Information about an identifier in a particular scope. */ - protected static class IdInfo { - public final SqlValidatorScope scope; - public final SqlIdentifier id; - - public IdInfo(SqlValidatorScope scope, SqlIdentifier id) { - this.scope = scope; - this.id = id; - } - } - - /** - * Utility object used to maintain information about the parameters in a - * function call. - */ - protected static class FunctionParamInfo { - /** - * Maps a cursor (based on its position relative to other cursor - * parameters within a function call) to the SELECT associated with the - * cursor. - */ - public final Map cursorPosToSelectMap; - - /** - * Maps a column list parameter to the parent cursor parameter it - * references. The parameters are id'd by their names. - */ - public final Map columnListParamToParentCursorMap; - - public FunctionParamInfo() { - cursorPosToSelectMap = new HashMap<>(); - columnListParamToParentCursorMap = new HashMap<>(); - } - } - - /** - * Modify the nodes in navigation function - * such as FIRST, LAST, PREV AND NEXT. - */ - private static class NavigationModifier extends SqlShuttle { - public SqlNode go(SqlNode node) { - return node.accept(this); - } - } - - /** - * Shuttle that expands navigation expressions in a MATCH_RECOGNIZE clause. - * - *

    Examples: - * - *

      - *
    • {@code PREV(A.price + A.amount)} → - * {@code PREV(A.price) + PREV(A.amount)} - * - *
    • {@code FIRST(A.price * 2)} → {@code FIRST(A.PRICE) * 2} - *
    - */ - private static class NavigationExpander extends NavigationModifier { - final SqlOperator op; - final SqlNode offset; - - NavigationExpander() { - this(null, null); - } - - NavigationExpander(SqlOperator operator, SqlNode offset) { - this.offset = offset; - this.op = operator; - } - - @Override public SqlNode visit(SqlCall call) { - SqlKind kind = call.getKind(); - List operands = call.getOperandList(); - List newOperands = new ArrayList<>(); - - // This code is a workaround for CALCITE-2707 - if (call.getFunctionQuantifier() != null - && call.getFunctionQuantifier().getValue() == SqlSelectKeyword.DISTINCT) { - final SqlParserPos pos = call.getParserPosition(); - throw SqlUtil.newContextException(pos, Static.RESOURCE.functionQuantifierNotAllowed(call.toString())); - } - // This code is a workaround for CALCITE-2707 - - if (isLogicalNavigation(kind) || isPhysicalNavigation(kind)) { - SqlNode inner = operands.get(0); - SqlNode offset = operands.get(1); - - // merge two straight prev/next, update offset - if (isPhysicalNavigation(kind)) { - SqlKind innerKind = inner.getKind(); - if (isPhysicalNavigation(innerKind)) { - List innerOperands = ((SqlCall) inner).getOperandList(); - SqlNode innerOffset = innerOperands.get(1); - SqlOperator newOperator = innerKind == kind - ? SqlStdOperatorTable.PLUS : SqlStdOperatorTable.MINUS; - offset = newOperator.createCall(SqlParserPos.ZERO, - offset, innerOffset); - inner = call.getOperator().createCall(SqlParserPos.ZERO, - innerOperands.get(0), offset); - } - } - SqlNode newInnerNode = - inner.accept(new NavigationExpander(call.getOperator(), offset)); - if (op != null) { - newInnerNode = op.createCall(SqlParserPos.ZERO, newInnerNode, - this.offset); - } - return newInnerNode; - } - - if (operands.size() > 0) { - for (SqlNode node : operands) { - if (node != null) { - SqlNode newNode = node.accept(new NavigationExpander()); - if (op != null) { - newNode = op.createCall(SqlParserPos.ZERO, newNode, offset); - } - newOperands.add(newNode); - } else { - newOperands.add(null); - } - } - return call.getOperator().createCall(SqlParserPos.ZERO, newOperands); - } else { - if (op == null) { - return call; - } else { - return op.createCall(SqlParserPos.ZERO, call, offset); - } - } - } - - @Override public SqlNode visit(SqlIdentifier id) { - if (op == null) { - return id; - } else { - return op.createCall(SqlParserPos.ZERO, id, offset); - } - } - } - - /** - * Shuttle that replaces {@code A as A.price > PREV(B.price)} with - * {@code PREV(A.price, 0) > LAST(B.price, 0)}. - * - *

    Replacing {@code A.price} with {@code PREV(A.price, 0)} makes the - * implementation of - * {@link RexVisitor#visitPatternFieldRef(RexPatternFieldRef)} more unified. - * Otherwise, it's difficult to implement this method. If it returns the - * specified field, then the navigation such as {@code PREV(A.price, 1)} - * becomes impossible; if not, then comparisons such as - * {@code A.price > PREV(A.price, 1)} become meaningless. - */ - private static class NavigationReplacer extends NavigationModifier { - private final String alpha; - - NavigationReplacer(String alpha) { - this.alpha = alpha; - } - - @Override public SqlNode visit(SqlCall call) { - SqlKind kind = call.getKind(); - if (isLogicalNavigation(kind) - || isAggregation(kind) - || isRunningOrFinal(kind)) { - return call; - } - - switch (kind) { - case PREV: - final List operands = call.getOperandList(); - if (operands.get(0) instanceof SqlIdentifier) { - String name = ((SqlIdentifier) operands.get(0)).names.get(0); - return name.equals(alpha) ? call - : SqlStdOperatorTable.LAST.createCall(SqlParserPos.ZERO, operands); - } - } - return super.visit(call); - } - - @Override public SqlNode visit(SqlIdentifier id) { - if (id.isSimple()) { - return id; - } - SqlOperator operator = id.names.get(0).equals(alpha) - ? SqlStdOperatorTable.PREV : SqlStdOperatorTable.LAST; - - return operator.createCall(SqlParserPos.ZERO, id, - SqlLiteral.createExactNumeric("0", SqlParserPos.ZERO)); - } - } - - /** - * Within one navigation function, the pattern var should be same - */ - private class PatternValidator extends SqlBasicVisitor> { - private final boolean isMeasure; - int firstLastCount; - int prevNextCount; - int aggregateCount; - - PatternValidator(boolean isMeasure) { - this(isMeasure, 0, 0, 0); - } - - PatternValidator(boolean isMeasure, int firstLastCount, int prevNextCount, - int aggregateCount) { - this.isMeasure = isMeasure; - this.firstLastCount = firstLastCount; - this.prevNextCount = prevNextCount; - this.aggregateCount = aggregateCount; - } - - @Override public Set visit(SqlCall call) { - boolean isSingle = false; - Set vars = new HashSet<>(); - SqlKind kind = call.getKind(); - List operands = call.getOperandList(); - - if (isSingleVarRequired(kind)) { - isSingle = true; - if (isPhysicalNavigation(kind)) { - if (isMeasure) { - throw newValidationError(call, - Static.RESOURCE.patternPrevFunctionInMeasure(call.toString())); - } - if (firstLastCount != 0) { - throw newValidationError(call, - Static.RESOURCE.patternPrevFunctionOrder(call.toString())); - } - prevNextCount++; - } else if (isLogicalNavigation(kind)) { - if (firstLastCount != 0) { - throw newValidationError(call, - Static.RESOURCE.patternPrevFunctionOrder(call.toString())); - } - firstLastCount++; - } else if (isAggregation(kind)) { - // cannot apply aggregation in PREV/NEXT, FIRST/LAST - if (firstLastCount != 0 || prevNextCount != 0) { - throw newValidationError(call, - Static.RESOURCE.patternAggregationInNavigation(call.toString())); - } - if (kind == SqlKind.COUNT && call.getOperandList().size() > 1) { - throw newValidationError(call, - Static.RESOURCE.patternCountFunctionArg()); - } - aggregateCount++; - } - } - - if (isRunningOrFinal(kind) && !isMeasure) { - throw newValidationError(call, - Static.RESOURCE.patternRunningFunctionInDefine(call.toString())); - } - - for (SqlNode node : operands) { - if (node != null) { - vars.addAll( - node.accept( - new PatternValidator(isMeasure, firstLastCount, prevNextCount, - aggregateCount))); - } - } - - if (isSingle) { - switch (kind) { - case COUNT: - if (vars.size() > 1) { - throw newValidationError(call, - Static.RESOURCE.patternCountFunctionArg()); - } - break; - default: - if (operands.size() == 0 - || !(operands.get(0) instanceof SqlCall) - || ((SqlCall) operands.get(0)).getOperator() != SqlStdOperatorTable.CLASSIFIER) { - if (vars.isEmpty()) { - throw newValidationError(call, - Static.RESOURCE.patternFunctionNullCheck(call.toString())); - } - if (vars.size() != 1) { - throw newValidationError(call, - Static.RESOURCE.patternFunctionVariableCheck(call.toString())); - } - } - break; - } - } - return vars; - } - - @Override public Set visit(SqlIdentifier identifier) { - boolean check = prevNextCount > 0 || firstLastCount > 0 || aggregateCount > 0; - Set vars = new HashSet<>(); - if (identifier.names.size() > 1 && check) { - vars.add(identifier.names.get(0)); - } - return vars; - } - - @Override public Set visit(SqlLiteral literal) { - return ImmutableSet.of(); - } - - @Override public Set visit(SqlIntervalQualifier qualifier) { - return ImmutableSet.of(); - } - - @Override public Set visit(SqlDataTypeSpec type) { - return ImmutableSet.of(); - } - - @Override public Set visit(SqlDynamicParam param) { - return ImmutableSet.of(); - } - } - - /** Permutation of fields in NATURAL JOIN or USING. */ - private class Permute { - final List sources; - final RelDataType rowType; - final boolean trivial; - - Permute(SqlNode from, int offset) { - switch (from.getKind()) { - case JOIN: - final SqlJoin join = (SqlJoin) from; - final Permute left = new Permute(join.getLeft(), offset); - final int fieldCount = - getValidatedNodeType(join.getLeft()).getFieldList().size(); - final Permute right = - new Permute(join.getRight(), offset + fieldCount); - final List names = usingNames(join); - final List sources = new ArrayList<>(); - final Set sourceSet = new HashSet<>(); - final RelDataTypeFactory.Builder b = typeFactory.builder(); - if (names != null) { - for (String name : names) { - final RelDataTypeField f = left.field(name); - final ImmutableIntList source = left.sources.get(f.getIndex()); - sourceSet.add(source); - final RelDataTypeField f2 = right.field(name); - final ImmutableIntList source2 = right.sources.get(f2.getIndex()); - sourceSet.add(source2); - sources.add(source.appendAll(source2)); - final boolean nullable = - (f.getType().isNullable() - || join.getJoinType().generatesNullsOnLeft()) - && (f2.getType().isNullable() - || join.getJoinType().generatesNullsOnRight()); - b.add(f).nullable(nullable); - } - } - for (RelDataTypeField f : left.rowType.getFieldList()) { - final ImmutableIntList source = left.sources.get(f.getIndex()); - if (sourceSet.add(source)) { - sources.add(source); - b.add(f); - } - } - for (RelDataTypeField f : right.rowType.getFieldList()) { - final ImmutableIntList source = right.sources.get(f.getIndex()); - if (sourceSet.add(source)) { - sources.add(source); - b.add(f); - } - } - rowType = b.build(); - this.sources = ImmutableList.copyOf(sources); - this.trivial = left.trivial - && right.trivial - && (names == null || names.isEmpty()); - break; - - default: - rowType = getValidatedNodeType(from); - this.sources = Functions.generate(rowType.getFieldCount(), - i -> ImmutableIntList.of(offset + i)); - this.trivial = true; - } - } - - private RelDataTypeField field(String name) { - return catalogReader.nameMatcher().field(rowType, name); - } - - /** Returns the set of field names in the join condition specified by USING - * or implicitly by NATURAL, de-duplicated and in order. */ - private List usingNames(SqlJoin join) { - switch (join.getConditionType()) { - case USING: - final ImmutableList.Builder list = ImmutableList.builder(); - final Set names = catalogReader.nameMatcher().createSet(); - for (SqlNode node : (SqlNodeList) join.getCondition()) { - final String name = ((SqlIdentifier) node).getSimple(); - if (names.add(name)) { - list.add(name); - } - } - return list.build(); - case NONE: - if (join.isNatural()) { - final RelDataType t0 = getValidatedNodeType(join.getLeft()); - final RelDataType t1 = getValidatedNodeType(join.getRight()); - return SqlValidatorUtil.deriveNaturalJoinColumnList( - catalogReader.nameMatcher(), t0, t1); - } - } - return null; - } - - /** Moves fields according to the permutation. */ - public void permute(List selectItems, - List> fields) { - if (trivial) { - return; - } - - final List oldSelectItems = ImmutableList.copyOf(selectItems); - selectItems.clear(); - final List> oldFields = - ImmutableList.copyOf(fields); - fields.clear(); - for (ImmutableIntList source : sources) { - final int p0 = source.get(0); - Map.Entry field = oldFields.get(p0); - final String name = field.getKey(); - RelDataType type = field.getValue(); - SqlNode selectItem = oldSelectItems.get(p0); - for (int p1 : Util.skip(source)) { - final Map.Entry field1 = oldFields.get(p1); - final SqlNode selectItem1 = oldSelectItems.get(p1); - final RelDataType type1 = field1.getValue(); - // output is nullable only if both inputs are - final boolean nullable = type.isNullable() && type1.isNullable(); - final RelDataType type2 = - SqlTypeUtil.leastRestrictiveForComparison(typeFactory, type, - type1); - selectItem = - SqlStdOperatorTable.AS.createCall(SqlParserPos.ZERO, - SqlStdOperatorTable.COALESCE.createCall(SqlParserPos.ZERO, - maybeCast(selectItem, type, type2), - maybeCast(selectItem1, type1, type2)), - new SqlIdentifier(name, SqlParserPos.ZERO)); - type = typeFactory.createTypeWithNullability(type2, nullable); - } - fields.add(Pair.of(name, type)); - selectItems.add(selectItem); - } - } - } - - //~ Enums ------------------------------------------------------------------ - - /** - * Validation status. - */ - public enum Status { - /** - * Validation has not started for this scope. - */ - UNVALIDATED, - - /** - * Validation is in progress for this scope. - */ - IN_PROGRESS, - - /** - * Validation has completed (perhaps unsuccessfully). - */ - VALID - } - -} - -// End SqlValidatorImpl.java -- Gitee From 06489d1982e83e4890f69cfa3b492def38bb3dfd Mon Sep 17 00:00:00 2001 From: "yuzhao.cyz" Date: Tue, 17 Mar 2020 21:11:38 +0800 Subject: [PATCH 371/885] [FLINK-14338][table-planner-blink] Plan verify changes from DIGEST to EXPLAIN * Because of CALCITE-3713, the project names was removed from plan DIGEST, thus, the DIGEST plan has less info that EXPLAIN, we switch to EXPLAIN for plan verification; * The CALC window fields name was also changes with CALCITE-3713, which is acceptable because the old name was also un-readable, the new name was based on field index, not bad. --- .../QueryOperationCatalogViewTable.java | 6 +- .../planner/calcite/FlinkRelBuilder.scala | 9 +++ .../planner/plan/utils/FlinkRelOptUtil.scala | 2 +- .../testGetDigestWithDynamicFunction.out | 12 +-- .../testGetDigestWithDynamicFunctionView.out | 12 +-- .../table/planner/plan/batch/sql/RankTest.xml | 16 ++-- ...inkLogicalRankRuleForConstantRangeTest.xml | 32 ++++---- .../FlinkLogicalRankRuleForRangeEndTest.xml | 2 +- .../plan/stream/sql/DeduplicateTest.xml | 76 +++++++++---------- .../planner/plan/stream/sql/RankTest.xml | 10 +-- .../table/planner/utils/TableTestBase.scala | 28 +++---- 11 files changed, 106 insertions(+), 99 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/QueryOperationCatalogViewTable.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/QueryOperationCatalogViewTable.java index f184be0cf0..4c8b07b540 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/QueryOperationCatalogViewTable.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/QueryOperationCatalogViewTable.java @@ -25,7 +25,6 @@ import org.apache.flink.table.planner.calcite.FlinkRelBuilder; import org.apache.flink.table.planner.plan.schema.ExpandingPreparingTable; import org.apache.flink.table.planner.plan.stats.FlinkStatistic; -import org.apache.calcite.plan.Contexts; import org.apache.calcite.plan.RelOptSchema; import org.apache.calcite.plan.RelOptTable; import org.apache.calcite.rel.RelNode; @@ -69,9 +68,8 @@ public class QueryOperationCatalogViewTable extends ExpandingPreparingTable { @Override public RelNode convertToRel(RelOptTable.ToRelContext context) { - FlinkRelBuilder relBuilder = new FlinkRelBuilder( - // Sets up the view expander. - Contexts.of(context, context.getCluster().getPlanner().getContext()), + FlinkRelBuilder relBuilder = FlinkRelBuilder.of( + context, context.getCluster(), this.getRelOptSchema()); diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelBuilder.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelBuilder.scala index 66b736bb55..47f1ab8723 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelBuilder.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelBuilder.scala @@ -188,4 +188,13 @@ object FlinkRelBuilder { cluster, relOptSchema) } + + def of(contextVar: Object, cluster: RelOptCluster, relOptSchema: RelOptSchema) + : FlinkRelBuilder = { + val mergedContext = Contexts.of(contextVar, cluster.getPlanner.getContext) + new FlinkRelBuilder( + mergedContext, + cluster, + relOptSchema) + } } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala index 2a05b74af6..80e5945fba 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala @@ -65,7 +65,7 @@ object FlinkRelOptUtil { */ def toString( rel: RelNode, - detailLevel: SqlExplainLevel = SqlExplainLevel.DIGEST_ATTRIBUTES, + detailLevel: SqlExplainLevel = SqlExplainLevel.EXPPLAN_ATTRIBUTES, withIdPrefix: Boolean = false, withRetractTraits: Boolean = false, withRowType: Boolean = false): String = { diff --git a/flink-table/flink-table-planner-blink/src/test/resources/digest/testGetDigestWithDynamicFunction.out b/flink-table/flink-table-planner-blink/src/test/resources/digest/testGetDigestWithDynamicFunction.out index 92691679c4..152f3a4444 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/digest/testGetDigestWithDynamicFunction.out +++ b/flink-table/flink-table-planner-blink/src/test/resources/digest/testGetDigestWithDynamicFunction.out @@ -1,14 +1,14 @@ LogicalIntersect(all=[false]), rowType=[RecordType(INTEGER random)] LogicalIntersect(all=[false]), rowType=[RecordType(INTEGER random)] -LogicalProject(random=[$0]), rowType=[RecordType(INTEGER random)] +LogicalProject(inputs=[0]), rowType=[RecordType(INTEGER random)] LogicalSort(sort0=[$1], dir0=[ASC-nulls-first], fetch=[1]), rowType=[RecordType(INTEGER random, DOUBLE EXPR$1)] -LogicalProject(random=[$1], EXPR$1=[RAND()]), rowType=[RecordType(INTEGER random, DOUBLE EXPR$1)] +LogicalProject(exprs=[[$1, RAND()]]), rowType=[RecordType(INTEGER random, DOUBLE EXPR$1)] LogicalTableScan(table=[[default_catalog, default_database, MyTable]]), rowType=[RecordType(VARCHAR(2147483647) first, INTEGER id, DOUBLE score, VARCHAR(2147483647) last)] -LogicalProject(random=[$0]), rowType=[RecordType(INTEGER random)] +LogicalProject(inputs=[0]), rowType=[RecordType(INTEGER random)] LogicalSort(sort0=[$1], dir0=[ASC-nulls-first], fetch=[1]), rowType=[RecordType(INTEGER random, DOUBLE EXPR$1)] -LogicalProject(random=[$1], EXPR$1=[RAND()]), rowType=[RecordType(INTEGER random, DOUBLE EXPR$1)] +LogicalProject(exprs=[[$1, RAND()]]), rowType=[RecordType(INTEGER random, DOUBLE EXPR$1)] LogicalTableScan(table=[[default_catalog, default_database, MyTable]]), rowType=[RecordType(VARCHAR(2147483647) first, INTEGER id, DOUBLE score, VARCHAR(2147483647) last)] -LogicalProject(random=[$0]), rowType=[RecordType(INTEGER random)] +LogicalProject(inputs=[0]), rowType=[RecordType(INTEGER random)] LogicalSort(sort0=[$1], dir0=[ASC-nulls-first], fetch=[1]), rowType=[RecordType(INTEGER random, DOUBLE EXPR$1)] -LogicalProject(random=[$1], EXPR$1=[RAND()]), rowType=[RecordType(INTEGER random, DOUBLE EXPR$1)] +LogicalProject(exprs=[[$1, RAND()]]), rowType=[RecordType(INTEGER random, DOUBLE EXPR$1)] LogicalTableScan(table=[[default_catalog, default_database, MyTable]]), rowType=[RecordType(VARCHAR(2147483647) first, INTEGER id, DOUBLE score, VARCHAR(2147483647) last)] diff --git a/flink-table/flink-table-planner-blink/src/test/resources/digest/testGetDigestWithDynamicFunctionView.out b/flink-table/flink-table-planner-blink/src/test/resources/digest/testGetDigestWithDynamicFunctionView.out index 92691679c4..152f3a4444 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/digest/testGetDigestWithDynamicFunctionView.out +++ b/flink-table/flink-table-planner-blink/src/test/resources/digest/testGetDigestWithDynamicFunctionView.out @@ -1,14 +1,14 @@ LogicalIntersect(all=[false]), rowType=[RecordType(INTEGER random)] LogicalIntersect(all=[false]), rowType=[RecordType(INTEGER random)] -LogicalProject(random=[$0]), rowType=[RecordType(INTEGER random)] +LogicalProject(inputs=[0]), rowType=[RecordType(INTEGER random)] LogicalSort(sort0=[$1], dir0=[ASC-nulls-first], fetch=[1]), rowType=[RecordType(INTEGER random, DOUBLE EXPR$1)] -LogicalProject(random=[$1], EXPR$1=[RAND()]), rowType=[RecordType(INTEGER random, DOUBLE EXPR$1)] +LogicalProject(exprs=[[$1, RAND()]]), rowType=[RecordType(INTEGER random, DOUBLE EXPR$1)] LogicalTableScan(table=[[default_catalog, default_database, MyTable]]), rowType=[RecordType(VARCHAR(2147483647) first, INTEGER id, DOUBLE score, VARCHAR(2147483647) last)] -LogicalProject(random=[$0]), rowType=[RecordType(INTEGER random)] +LogicalProject(inputs=[0]), rowType=[RecordType(INTEGER random)] LogicalSort(sort0=[$1], dir0=[ASC-nulls-first], fetch=[1]), rowType=[RecordType(INTEGER random, DOUBLE EXPR$1)] -LogicalProject(random=[$1], EXPR$1=[RAND()]), rowType=[RecordType(INTEGER random, DOUBLE EXPR$1)] +LogicalProject(exprs=[[$1, RAND()]]), rowType=[RecordType(INTEGER random, DOUBLE EXPR$1)] LogicalTableScan(table=[[default_catalog, default_database, MyTable]]), rowType=[RecordType(VARCHAR(2147483647) first, INTEGER id, DOUBLE score, VARCHAR(2147483647) last)] -LogicalProject(random=[$0]), rowType=[RecordType(INTEGER random)] +LogicalProject(inputs=[0]), rowType=[RecordType(INTEGER random)] LogicalSort(sort0=[$1], dir0=[ASC-nulls-first], fetch=[1]), rowType=[RecordType(INTEGER random, DOUBLE EXPR$1)] -LogicalProject(random=[$1], EXPR$1=[RAND()]), rowType=[RecordType(INTEGER random, DOUBLE EXPR$1)] +LogicalProject(exprs=[[$1, RAND()]]), rowType=[RecordType(INTEGER random, DOUBLE EXPR$1)] LogicalTableScan(table=[[default_catalog, default_database, MyTable]]), rowType=[RecordType(VARCHAR(2147483647) first, INTEGER id, DOUBLE score, VARCHAR(2147483647) last)] diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/RankTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/RankTest.xml index 1d41877b83..05bdc9e0d0 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/RankTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/RankTest.xml @@ -34,8 +34,8 @@ LogicalProject(a=[$0], b=[$1], rk=[$2]) (a, 10)]) -+- Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=2], partitionBy=[b], orderBy=[a ASC], global=[true], select=[a, b, c, w0$o0]) +Calc(select=[a, b, $2], where=[>(a, 10)]) ++- Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=2], partitionBy=[b], orderBy=[a ASC], global=[true], select=[a, b, c, $2]) +- Sort(orderBy=[b ASC, a ASC]) +- Exchange(distribution=[hash[b]]) +- Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=2], partitionBy=[b], orderBy=[a ASC], global=[false], select=[a, b, c]) @@ -204,8 +204,8 @@ LogicalProject(a=[$0], b=[$1], rk=[$2]) @@ -60,7 +60,7 @@ LogicalProject(a=[$0], b=[$1], rk=[$2], rn=[$3]) @@ -86,7 +86,7 @@ LogicalProject(a=[$0], b=[$1], rk1=[$2], rk2=[$3]) @@ -160,7 +160,7 @@ LogicalProject(a=[$0], b=[$1], rk=[$2]) (w0$o0, 2)]) +FlinkLogicalCalc(select=[a, b, w0$o0 AS $2], where=[>(w0$o0, 2)]) +- FlinkLogicalOverAggregate(window#0=[window(partition {1} order by [0 ASC-nulls-first, 2 ASC-nulls-first] range between UNBOUNDED PRECEDING and CURRENT ROW aggs [RANK()])]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -184,8 +184,8 @@ LogicalProject(a=[$0], b=[$1], rk=[$2]) @@ -208,8 +208,8 @@ LogicalProject(a=[$0], b=[$1], rk=[$2]) (a, 10)]) -+- FlinkLogicalRank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=2], partitionBy=[b], orderBy=[a ASC], select=[a, b, c, w0$o0]) +FlinkLogicalCalc(select=[a, b, $2], where=[>(a, 10)]) ++- FlinkLogicalRank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=2], partitionBy=[b], orderBy=[a ASC], select=[a, b, c, $2]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -232,7 +232,7 @@ LogicalProject(a=[$0], b=[$1], rn=[$2]) @@ -256,7 +256,7 @@ LogicalProject(a=[$0], b=[$1], rk=[$2]) @@ -280,7 +280,7 @@ LogicalProject(a=[$0], b=[$1], rk=[$2]) (w0$o0, a)]) +FlinkLogicalCalc(select=[a, b, w0$o0 AS $2], where=[>(w0$o0, a)]) +- FlinkLogicalOverAggregate(window#0=[window(partition {1} order by [2 ASC-nulls-first] range between UNBOUNDED PRECEDING and CURRENT ROW aggs [RANK()])]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -304,7 +304,7 @@ LogicalProject(a=[$0], b=[$1], rk=[$2]) (CAST(b), 5:BIGINT))]) +FlinkLogicalCalc(select=[a, b, w0$o0 AS $2], where=[AND(<(w0$o0, a), >(CAST(b), 5:BIGINT))]) +- FlinkLogicalOverAggregate(window#0=[window(partition {1} order by [2 ASC-nulls-first] range between UNBOUNDED PRECEDING and CURRENT ROW aggs [RANK()])]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -328,7 +328,7 @@ LogicalProject(a=[$0], b=[$1], rk=[$2]) @@ -352,8 +352,8 @@ LogicalProject(a=[$0], b=[$1], rk=[$2]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkLogicalRankRuleForRangeEndTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkLogicalRankRuleForRangeEndTest.xml index 1ec496fc4d..599337496b 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkLogicalRankRuleForRangeEndTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkLogicalRankRuleForRangeEndTest.xml @@ -113,7 +113,7 @@ LogicalProject(a=[$0], b=[$1], rk=[$2]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DeduplicateTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DeduplicateTest.xml index 6db3a2584e..a7a80d3db5 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DeduplicateTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DeduplicateTest.xml @@ -36,7 +36,7 @@ LogicalProject(a=[$0], rank_num=[$1]) - + - + - + - + @@ -670,7 +670,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], row_num=[$3]) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala index a93b5a2505..c9d232cff6 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala @@ -261,7 +261,7 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) def verifyPlan(sql: String): Unit = { doVerifyPlan( sql, - SqlExplainLevel.DIGEST_ATTRIBUTES, + SqlExplainLevel.EXPPLAN_ATTRIBUTES, withRowType = false, printPlanBefore = true) } @@ -269,7 +269,7 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) def verifyPlan(table: Table): Unit = { doVerifyPlan( table, - SqlExplainLevel.DIGEST_ATTRIBUTES, + SqlExplainLevel.EXPPLAN_ATTRIBUTES, withRowType = false, printPlanBefore = true) } @@ -277,7 +277,7 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) def verifyPlanWithType(sql: String): Unit = { doVerifyPlan( sql, - explainLevel = SqlExplainLevel.DIGEST_ATTRIBUTES, + explainLevel = SqlExplainLevel.EXPPLAN_ATTRIBUTES, withRowType = true, printPlanBefore = true) } @@ -285,7 +285,7 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) def verifyPlanWithType(table: Table): Unit = { doVerifyPlan( table, - explainLevel = SqlExplainLevel.DIGEST_ATTRIBUTES, + explainLevel = SqlExplainLevel.EXPPLAN_ATTRIBUTES, withRowType = true, printPlanBefore = true) } @@ -299,7 +299,7 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) val relNode = TableTestUtil.toRelNode(table) val optimizedPlan = getOptimizedPlan( Array(relNode), - explainLevel = SqlExplainLevel.DIGEST_ATTRIBUTES, + explainLevel = SqlExplainLevel.EXPPLAN_ATTRIBUTES, withRetractTraits = false, withRowType = false) val result = notExpected.forall(!optimizedPlan.contains(_)) @@ -357,7 +357,7 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) val planBefore = SystemUtils.LINE_SEPARATOR + FlinkRelOptUtil.toString( relNode, - SqlExplainLevel.DIGEST_ATTRIBUTES, + SqlExplainLevel.EXPPLAN_ATTRIBUTES, withRowType = withRowType) assertEqualsOrExpand("planBefore", planBefore) } @@ -371,7 +371,7 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) val table = getTableEnv.sqlQuery(sql) doVerifyPlan( table, - explainLevel = SqlExplainLevel.DIGEST_ATTRIBUTES, + explainLevel = SqlExplainLevel.EXPPLAN_ATTRIBUTES, withRowType = false, withRetractTraits = false, printResource = true, @@ -410,7 +410,7 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) val planBefore = SystemUtils.LINE_SEPARATOR + FlinkRelOptUtil.toString( relNode, - SqlExplainLevel.DIGEST_ATTRIBUTES, + SqlExplainLevel.EXPPLAN_ATTRIBUTES, withRowType = withRowType) assertEqualsOrExpand("planBefore", planBefore) } @@ -595,7 +595,7 @@ abstract class TableTestUtil( def verifySqlUpdate(sql: String): Unit = { doVerifySqlUpdate( sql, - SqlExplainLevel.DIGEST_ATTRIBUTES, + SqlExplainLevel.EXPPLAN_ATTRIBUTES, withRowType = false, withRetractTraits = false, printPlanBefore = true) @@ -603,7 +603,7 @@ abstract class TableTestUtil( def verifyPlan(): Unit = { doVerifyPlan( - SqlExplainLevel.DIGEST_ATTRIBUTES, + SqlExplainLevel.EXPPLAN_ATTRIBUTES, withRowType = false, withRetractTraits = false, printPlanBefore = true) @@ -644,7 +644,7 @@ abstract class TableTestUtil( val planBefore = new StringBuilder relNodes.foreach { sink => planBefore.append(System.lineSeparator) - planBefore.append(FlinkRelOptUtil.toString(sink, SqlExplainLevel.DIGEST_ATTRIBUTES)) + planBefore.append(FlinkRelOptUtil.toString(sink, SqlExplainLevel.EXPPLAN_ATTRIBUTES)) } assertEqualsOrExpand("planBefore", planBefore.toString()) } @@ -770,7 +770,7 @@ case class StreamTableTestUtil( def verifyPlanWithTrait(): Unit = { doVerifyPlan( - SqlExplainLevel.DIGEST_ATTRIBUTES, + SqlExplainLevel.EXPPLAN_ATTRIBUTES, withRetractTraits = true, withRowType = false, printPlanBefore = true) @@ -779,7 +779,7 @@ case class StreamTableTestUtil( def verifyPlanWithTrait(sql: String): Unit = { doVerifyPlan( sql, - SqlExplainLevel.DIGEST_ATTRIBUTES, + SqlExplainLevel.EXPPLAN_ATTRIBUTES, withRetractTraits = true, withRowType = false, printPlanBefore = true) @@ -788,7 +788,7 @@ case class StreamTableTestUtil( def verifyPlanWithTrait(table: Table): Unit = { doVerifyPlan( table, - SqlExplainLevel.DIGEST_ATTRIBUTES, + SqlExplainLevel.EXPPLAN_ATTRIBUTES, withRetractTraits = true, withRowType = false, printPlanBefore = true) -- Gitee From c41887df9e92572ef4c9b3a0884b1df7e01d7299 Mon Sep 17 00:00:00 2001 From: "yuzhao.cyz" Date: Tue, 17 Mar 2020 21:16:46 +0800 Subject: [PATCH 372/885] [FLINK-14338][table-planner-blink] Plan verify changes from DIGEST to EXPLAIN: change plan for expand node * The explain node does not show each projects names anymore --- .../batch/sql/agg/DistinctAggregateTest.xml | 312 ++++++++--------- .../plan/batch/sql/agg/GroupingSetsTest.xml | 153 ++++---- ...regateExpandDistinctAggregatesRuleTest.xml | 271 +++++++------- .../logical/FlinkAggregateRemoveRuleTest.xml | 30 +- .../rules/logical/SplitAggregateRuleTest.xml | 74 +--- .../batch/EnforceLocalHashAggRuleTest.xml | 12 +- .../batch/EnforceLocalSortAggRuleTest.xml | 18 +- .../plan/stream/sql/DagOptimizationTest.xml | 12 +- .../planner/plan/stream/sql/SinkTest.xml | 6 +- .../stream/sql/agg/DistinctAggregateTest.xml | 330 +++++++++--------- .../plan/stream/sql/agg/GroupingSetsTest.xml | 146 ++++---- .../sql/agg/IncrementalAggregateTest.xml | 178 +++++----- 12 files changed, 753 insertions(+), 789 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/DistinctAggregateTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/DistinctAggregateTest.xml index 577f89a430..a6b15d5abe 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/DistinctAggregateTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/DistinctAggregateTest.xml @@ -37,7 +37,7 @@ Calc(select=[a, CAST(EXPR$1) AS EXPR$1, EXPR$2, EXPR$3]) +- HashAggregate(isMerge=[true], groupBy=[a, b, $e], select=[a, b, $e, Final_COUNT(count$0) AS EXPR$1, Final_SUM(sum$1) AS EXPR$2]) +- Exchange(distribution=[hash[a, b, $e]]) +- LocalHashAggregate(groupBy=[a, b, $e], select=[a, b, $e, Partial_COUNT(a) AS count$0, Partial_SUM(b_0) AS sum$1]) - +- Expand(projects=[{a=[$0], b=[$1], $e=[0], b_0=[$1]}, {a=[$0], b=[null], $e=[1], b_0=[$1]}], projects=[{a, b, 0 AS $e, b AS b_0}, {a, null AS b, 1 AS $e, b AS b_0}]) + +- Expand(projects=[a, b, $e, b_0], projects=[{a, b, 0 AS $e, b AS b_0}, {a, null AS b, 1 AS $e, b AS b_0}]) +- Calc(select=[a, b]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -65,7 +65,7 @@ Calc(select=[EXPR$0, EXPR$1]) +- HashAggregate(isMerge=[true], groupBy=[a, b, c, $e], select=[a, b, c, $e]) +- Exchange(distribution=[hash[a, b, c, $e]]) +- LocalHashAggregate(groupBy=[a, b, c, $e], select=[a, b, c, $e]) - +- Expand(projects=[{a=[$0], b=[$1], c=[null], $e=[1]}, {a=[null], b=[$1], c=[$2], $e=[4]}], projects=[{a, b, null AS c, 1 AS $e}, {null AS a, b, c, 4 AS $e}]) + +- Expand(projects=[a, b, c, $e], projects=[{a, b, null AS c, 1 AS $e}, {null AS a, b, c, 4 AS $e}]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -90,7 +90,7 @@ SortAggregate(isMerge=[true], select=[Final_COUNT(count$0) AS EXPR$0, Final_SUM( +- HashAggregate(isMerge=[true], groupBy=[a, b, $e], select=[a, b, $e, Final_MAX(max$0) AS EXPR$2, Final_MIN(min$1) AS EXPR$3]) +- Exchange(distribution=[hash[a, b, $e]]) +- LocalHashAggregate(groupBy=[a, b, $e], select=[a, b, $e, Partial_MAX(a_0) AS max$0, Partial_MIN(a_0) AS min$1]) - +- Expand(projects=[{a=[$0], b=[null], $e=[1], a_0=[$0]}, {a=[null], b=[$1], $e=[2], a_0=[$0]}, {a=[null], b=[null], $e=[3], a_0=[$0]}], projects=[{a, null AS b, 1 AS $e, a AS a_0}, {null AS a, b, 2 AS $e, a AS a_0}, {null AS a, null AS b, 3 AS $e, a AS a_0}]) + +- Expand(projects=[a, b, $e, a_0], projects=[{a, null AS b, 1 AS $e, a AS a_0}, {null AS a, b, 2 AS $e, a AS a_0}, {null AS a, null AS b, 3 AS $e, a AS a_0}]) +- Calc(select=[a, b]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -118,7 +118,7 @@ Calc(select=[EXPR$0, EXPR$1, EXPR$2, EXPR$3]) +- HashAggregate(isMerge=[true], groupBy=[a, b, c, $e], select=[a, b, c, $e, Final_MAX(max$0) AS EXPR$2, Final_MIN(min$1) AS EXPR$3]) +- Exchange(distribution=[hash[a, b, c, $e]]) +- LocalHashAggregate(groupBy=[a, b, c, $e], select=[a, b, c, $e, Partial_MAX(a_0) AS max$0, Partial_MIN(a_0) AS min$1]) - +- Expand(projects=[{a=[$0], b=[null], c=[$2], $e=[2], a_0=[$0]}, {a=[null], b=[$1], c=[$2], $e=[4], a_0=[$0]}, {a=[null], b=[null], c=[$2], $e=[6], a_0=[$0]}], projects=[{a, null AS b, c, 2 AS $e, a AS a_0}, {null AS a, b, c, 4 AS $e, a AS a_0}, {null AS a, null AS b, c, 6 AS $e, a AS a_0}]) + +- Expand(projects=[a, b, c, $e, a_0], projects=[{a, null AS b, c, 2 AS $e, a AS a_0}, {null AS a, b, c, 4 AS $e, a AS a_0}, {null AS a, null AS b, c, 6 AS $e, a AS a_0}]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -140,11 +140,11 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT(DISTINCT $0)], EXPR$1=[SUM(DISTINCT $ SortAggregate(isMerge=[true], select=[Final_COUNT(count$0) AS EXPR$0, Final_SUM(sum$1) AS EXPR$1, Final_COUNT(count$2) AS EXPR$2]) +- Exchange(distribution=[single]) +- LocalSortAggregate(select=[Partial_COUNT(a) FILTER $g_7 AS count$0, Partial_SUM(b) FILTER $g_11 AS sum$1, Partial_COUNT(c) FILTER $g_12 AS count$2]) - +- Calc(select=[a, b, c, CAST($f3) AS $f3, =(CASE(=($e, 7:BIGINT), 7:BIGINT, =($e, 11:BIGINT), 11:BIGINT, 12:BIGINT), 7) AS $g_7, =(CASE(=($e, 7:BIGINT), 7:BIGINT, =($e, 11:BIGINT), 11:BIGINT, 12:BIGINT), 11) AS $g_11, AND(=(CASE(=($e, 7:BIGINT), 7:BIGINT, =($e, 11:BIGINT), 11:BIGINT, 12:BIGINT), 12), IS TRUE(CAST($f3))) AS $g_12]) + +- Calc(select=[a, b, c, =(CASE(=($e, 7:BIGINT), 7:BIGINT, =($e, 11:BIGINT), 11:BIGINT, 12:BIGINT), 7) AS $g_7, =(CASE(=($e, 7:BIGINT), 7:BIGINT, =($e, 11:BIGINT), 11:BIGINT, 12:BIGINT), 11) AS $g_11, AND(=(CASE(=($e, 7:BIGINT), 7:BIGINT, =($e, 11:BIGINT), 11:BIGINT, 12:BIGINT), 12), IS TRUE(CAST($f3))) AS $g_12]) +- HashAggregate(isMerge=[true], groupBy=[a, b, c, $f3, $e], select=[a, b, c, $f3, $e]) +- Exchange(distribution=[hash[a, b, c, $f3, $e]]) +- LocalHashAggregate(groupBy=[a, b, c, $f3, $e], select=[a, b, c, $f3, $e]) - +- Expand(projects=[{a=[$0], b=[null], c=[null], $f3=[null], $e=[7]}, {a=[null], b=[$1], c=[null], $f3=[null], $e=[11]}, {a=[null], b=[null], c=[$2], $f3=[$3], $e=[12]}], projects=[{a, null AS b, null AS c, null AS $f3, 7 AS $e}, {null AS a, b, null AS c, null AS $f3, 11 AS $e}, {null AS a, null AS b, c, $f3, 12 AS $e}]) + +- Expand(projects=[a, b, c, $f3, $e], projects=[{a, null AS b, null AS c, null AS $f3, 7 AS $e}, {null AS a, b, null AS c, null AS $f3, 11 AS $e}, {null AS a, null AS b, c, $f3, 12 AS $e}]) +- Calc(select=[a, b, c, IS TRUE(>(a, 5)) AS $f3]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -172,8 +172,8 @@ Calc(select=[EXPR$0, EXPR$1]) +- HashAggregate(isMerge=[true], groupBy=[c, d, a, b, $e, $e_0], select=[c, d, a, b, $e, $e_0]) +- Exchange(distribution=[hash[c, d, a, b, $e, $e_0]]) +- LocalHashAggregate(groupBy=[c, d, a, b, $e, $e_0], select=[c, d, a, b, $e, $e_0]) - +- Expand(projects=[{c=[$0], d=[$1], a=[$2], b=[null], $e=[$4], $e_0=[2]}, {c=[$0], d=[$1], a=[null], b=[$3], $e=[$4], $e_0=[4]}], projects=[{c, d, a, null AS b, $e, 2 AS $e_0}, {c, d, null AS a, b, $e, 4 AS $e_0}]) - +- Expand(projects=[{c=[$0], d=[null], a=[$2], b=[$3], $e=[1]}, {c=[null], d=[$1], a=[$2], b=[$3], $e=[2]}], projects=[{c, null AS d, a, b, 1 AS $e}, {null AS c, d, a, b, 2 AS $e}]) + +- Expand(projects=[c, d, a, b, $e, $e_0], projects=[{c, d, a, null AS b, $e, 2 AS $e_0}, {c, d, null AS a, b, $e, 4 AS $e_0}]) + +- Expand(projects=[c, d, a, b, $e], projects=[{c, null AS d, a, b, 1 AS $e}, {null AS c, d, a, b, 2 AS $e}]) +- Calc(select=[c, d, a, b]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) ]]> @@ -196,11 +196,11 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT(DISTINCT $0)], EXPR$1=[SUM(DISTINCT $ SortAggregate(isMerge=[true], select=[Final_COUNT(count$0) AS EXPR$0, Final_SUM(sum$1) AS EXPR$1, Final_MIN(min$2) AS EXPR$2]) +- Exchange(distribution=[single]) +- LocalSortAggregate(select=[Partial_COUNT(a) FILTER $g_1 AS count$0, Partial_SUM(a) FILTER $g_0 AS sum$1, Partial_MIN(EXPR$2) FILTER $g_3 AS min$2]) - +- Calc(select=[a, CAST($f1) AS $f1, EXPR$2, AND(=(CASE(=($e, 0:BIGINT), 0:BIGINT, =($e, 1:BIGINT), 1:BIGINT, 3:BIGINT), 0), IS TRUE(CAST($f1))) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, =($e, 1:BIGINT), 1:BIGINT, 3:BIGINT), 1) AS $g_1, =(CASE(=($e, 0:BIGINT), 0:BIGINT, =($e, 1:BIGINT), 1:BIGINT, 3:BIGINT), 3) AS $g_3]) + +- Calc(select=[a, EXPR$2, AND(=(CASE(=($e, 0:BIGINT), 0:BIGINT, =($e, 1:BIGINT), 1:BIGINT, 3:BIGINT), 0), IS TRUE(CAST($f1))) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, =($e, 1:BIGINT), 1:BIGINT, 3:BIGINT), 1) AS $g_1, =(CASE(=($e, 0:BIGINT), 0:BIGINT, =($e, 1:BIGINT), 1:BIGINT, 3:BIGINT), 3) AS $g_3]) +- HashAggregate(isMerge=[true], groupBy=[a, $f1, $e], select=[a, $f1, $e, Final_MAX(max$0) AS EXPR$2]) +- Exchange(distribution=[hash[a, $f1, $e]]) +- LocalHashAggregate(groupBy=[a, $f1, $e], select=[a, $f1, $e, Partial_MAX(a_0) AS max$0]) - +- Expand(projects=[{a=[$0], $f1=[$1], $e=[0], a_0=[$0]}, {a=[$0], $f1=[null], $e=[1], a_0=[$0]}, {a=[null], $f1=[null], $e=[3], a_0=[$0]}], projects=[{a, $f1, 0 AS $e, a AS a_0}, {a, null AS $f1, 1 AS $e, a AS a_0}, {null AS a, null AS $f1, 3 AS $e, a AS a_0}]) + +- Expand(projects=[a, $f1, $e, a_0], projects=[{a, $f1, 0 AS $e, a AS a_0}, {a, null AS $f1, 1 AS $e, a AS a_0}, {null AS a, null AS $f1, 3 AS $e, a AS a_0}]) +- Calc(select=[a, IS TRUE(>(b, 0)) AS $f1]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -227,7 +227,7 @@ Calc(select=[EXPR$0, EXPR$1, EXPR$2]) +- HashAggregate(isMerge=[true], groupBy=[b, c, a, $e], select=[b, c, a, $e]) +- Exchange(distribution=[hash[b, c, a, $e]]) +- LocalHashAggregate(groupBy=[b, c, a, $e], select=[b, c, a, $e]) - +- Expand(projects=[{b=[$0], c=[null], a=[$2], $e=[1]}, {b=[null], c=[$1], a=[$2], $e=[2]}], projects=[{b, null AS c, a, 1 AS $e}, {null AS b, c, a, 2 AS $e}]) + +- Expand(projects=[b, c, a, $e], projects=[{b, null AS c, a, 1 AS $e}, {null AS b, c, a, 2 AS $e}]) +- Calc(select=[b, c, a]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) ]]> @@ -247,15 +247,15 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT(DISTINCT $0) FILTER $1], EXPR$1=[SUM( (c, 0)) AS $f1, b, c]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -283,9 +283,122 @@ Calc(select=[EXPR$0, EXPR$1, CAST(EXPR$2) AS EXPR$2]) +- HashAggregate(isMerge=[true], groupBy=[a, b, d, e, $e, $e_0], select=[a, b, d, e, $e, $e_0, Final_COUNT(count$0) AS EXPR$2]) +- Exchange(distribution=[hash[a, b, d, e, $e, $e_0]]) +- LocalHashAggregate(groupBy=[a, b, d, e, $e, $e_0], select=[a, b, d, e, $e, $e_0, Partial_COUNT(c) AS count$0]) - +- Expand(projects=[{a=[$0], b=[null], c=[$2], d=[$3], e=[$4], $e=[$5], $e_0=[8]}, {a=[null], b=[$1], c=[$2], d=[$3], e=[$4], $e=[$5], $e_0=[16]}, {a=[null], b=[null], c=[$2], d=[$3], e=[$4], $e=[$5], $e_0=[24]}], projects=[{a, null AS b, c, d, e, $e, 8 AS $e_0}, {null AS a, b, c, d, e, $e, 16 AS $e_0}, {null AS a, null AS b, c, d, e, $e, 24 AS $e_0}]) - +- Expand(projects=[{a=[$0], b=[$1], c=[$2], d=[$3], e=[null], $e=[1]}, {a=[$0], b=[$1], c=[$2], d=[null], e=[$4], $e=[2]}], projects=[{a, b, c, d, null AS e, 1 AS $e}, {a, b, c, null AS d, e, 2 AS $e}]) + +- Expand(projects=[a, b, c, d, e, $e, $e_0], projects=[{a, null AS b, c, d, e, $e, 8 AS $e_0}, {null AS a, b, c, d, e, $e, 16 AS $e_0}, {null AS a, null AS b, c, d, e, $e, 24 AS $e_0}]) + +- Expand(projects=[a, b, c, d, e, $e], projects=[{a, b, c, d, null AS e, 1 AS $e}, {a, b, c, null AS d, e, 2 AS $e}]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) +]]> + + + + + 5), COUNT(DISTINCT b) FILTER (WHERE b > 3) +FROM MyTable2 GROUP BY d]]> + + + ($0, 5))], b=[$1], $f6=[IS TRUE(>($1, 3))]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) +]]> + + + (a, 5)) AS $f4, b, IS TRUE(>(b, 3)) AS $f6]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) +]]> + + + + + 0), +COUNT(DISTINCT b) FILTER (WHERE b > 1) FROM MyTable2 GROUP BY d]]> + + + ($0, 0))], b=[$1], $f4=[IS TRUE(>($1, 1))]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) +]]> + + + (a, 0)) AS $f2, b, IS TRUE(>(b, 1)) AS $f4]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) +]]> + + + + + 10), +COUNT(DISTINCT c) FILTER (WHERE a < 10) FROM MyTable2 GROUP BY d]]> + + + ($0, 10))], $f3=[IS TRUE(<($0, 10))]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) +]]> + + + (a, 10)) AS $f2, IS TRUE(<(a, 10)) AS $f3]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) +]]> + + + + + 0), +MAX(e), MIN(e) FROM MyTable2 GROUP BY d]]> + + + ($0, 0))], e=[$4]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) +]]> + + + (a, 0)) AS $f2, e]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) ]]> @@ -329,11 +442,11 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT(DISTINCT $0) FILTER $1], EXPR$1=[SUM( SortAggregate(isMerge=[true], select=[Final_COUNT(count$0) AS EXPR$0, Final_MIN(min$1) AS EXPR$1]) +- Exchange(distribution=[single]) +- LocalSortAggregate(select=[Partial_COUNT(a) FILTER $g_0 AS count$0, Partial_MIN(EXPR$1) FILTER $g_3 AS min$1]) - +- Calc(select=[a, CAST($f1) AS $f1, EXPR$1, AND(=(CASE(=($e, 0:BIGINT), 0:BIGINT, 3:BIGINT), 0), IS TRUE(CAST($f1))) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 3:BIGINT), 3) AS $g_3]) + +- Calc(select=[a, EXPR$1, AND(=(CASE(=($e, 0:BIGINT), 0:BIGINT, 3:BIGINT), 0), IS TRUE(CAST($f1))) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 3:BIGINT), 3) AS $g_3]) +- HashAggregate(isMerge=[true], groupBy=[a, $f1, $e], select=[a, $f1, $e, Final_SUM(sum$0) AS EXPR$1]) +- Exchange(distribution=[hash[a, $f1, $e]]) +- LocalHashAggregate(groupBy=[a, $f1, $e], select=[a, $f1, $e, Partial_SUM(b) AS sum$0]) - +- Expand(projects=[{a=[$0], $f1=[$1], b=[$2], $e=[0]}, {a=[null], $f1=[null], b=[$2], $e=[3]}], projects=[{a, $f1, b, 0 AS $e}, {null AS a, null AS $f1, b, 3 AS $e}]) + +- Expand(projects=[a, $f1, b, $e], projects=[{a, $f1, b, 0 AS $e}, {null AS a, null AS $f1, b, 3 AS $e}]) +- Calc(select=[a, IS TRUE(>(a, 0)) AS $f1, b]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -352,15 +465,15 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT($0) FILTER $1], EXPR$1=[SUM(DISTINCT (a, 0)) AS $f1, b]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -388,8 +501,8 @@ Calc(select=[EXPR$0, EXPR$1]) +- HashAggregate(isMerge=[true], groupBy=[b, c, a, $e, $e_0], select=[b, c, a, $e, $e_0, Final_SUM(sum$0) AS EXPR$1]) +- Exchange(distribution=[hash[b, c, a, $e, $e_0]]) +- LocalHashAggregate(groupBy=[b, c, a, $e, $e_0], select=[b, c, a, $e, $e_0, Partial_SUM(b_0) AS sum$0]) - +- Expand(projects=[{b=[$0], c=[$1], a=[$2], $e=[$3], b_0=[$4], $e_0=[0]}, {b=[$0], c=[$1], a=[null], $e=[$3], b_0=[$4], $e_0=[2]}], projects=[{b, c, a, $e, b_0, 0 AS $e_0}, {b, c, null AS a, $e, b_0, 2 AS $e_0}]) - +- Expand(projects=[{b=[$0], c=[null], a=[$2], $e=[1], b_0=[$0]}, {b=[null], c=[$1], a=[$2], $e=[2], b_0=[$0]}], projects=[{b, null AS c, a, 1 AS $e, b AS b_0}, {null AS b, c, a, 2 AS $e, b AS b_0}]) + +- Expand(projects=[b, c, a, $e, b_0, $e_0], projects=[{b, c, a, $e, b_0, 0 AS $e_0}, {b, c, null AS a, $e, b_0, 2 AS $e_0}]) + +- Expand(projects=[b, c, a, $e, b_0], projects=[{b, null AS c, a, 1 AS $e, b AS b_0}, {null AS b, c, a, 2 AS $e, b AS b_0}]) +- Calc(select=[b, c, a]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) ]]> @@ -417,8 +530,8 @@ Calc(select=[CAST(EXPR$0) AS EXPR$0, EXPR$1]) +- HashAggregate(isMerge=[true], groupBy=[c, d, b, $e, $e_0], select=[c, d, b, $e, $e_0, Final_COUNT(count$0) AS EXPR$0]) +- Exchange(distribution=[hash[c, d, b, $e, $e_0]]) +- LocalHashAggregate(groupBy=[c, d, b, $e, $e_0], select=[c, d, b, $e, $e_0, Partial_COUNT(a) AS count$0]) - +- Expand(projects=[{c=[$0], d=[$1], a=[$2], b=[$3], $e=[$4], $e_0=[0]}, {c=[$0], d=[$1], a=[$2], b=[null], $e=[$4], $e_0=[2]}], projects=[{c, d, a, b, $e, 0 AS $e_0}, {c, d, a, null AS b, $e, 2 AS $e_0}]) - +- Expand(projects=[{c=[$0], d=[null], a=[$2], b=[$3], $e=[1]}, {c=[null], d=[$1], a=[$2], b=[$3], $e=[2]}], projects=[{c, null AS d, a, b, 1 AS $e}, {null AS c, d, a, b, 2 AS $e}]) + +- Expand(projects=[c, d, a, b, $e, $e_0], projects=[{c, d, a, b, $e, 0 AS $e_0}, {c, d, a, null AS b, $e, 2 AS $e_0}]) + +- Expand(projects=[c, d, a, b, $e], projects=[{c, null AS d, a, b, 1 AS $e}, {null AS c, d, a, b, 2 AS $e}]) +- Calc(select=[c, d, a, b]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) ]]> @@ -438,14 +551,14 @@ LogicalAggregate(group=[{0}], EXPR$1=[COUNT()], EXPR$2=[SUM(DISTINCT $1)], EXPR$ @@ -472,7 +585,7 @@ Calc(select=[a, CAST(EXPR$1) AS EXPR$1, EXPR$2]) +- HashAggregate(isMerge=[true], groupBy=[a, b, $e], select=[a, b, $e, Final_COUNT(count$0) AS EXPR$1]) +- Exchange(distribution=[hash[a, b, $e]]) +- LocalHashAggregate(groupBy=[a, b, $e], select=[a, b, $e, Partial_COUNT(a) AS count$0]) - +- Expand(projects=[{a=[$0], b=[$1], $e=[0]}, {a=[$0], b=[null], $e=[1]}], projects=[{a, b, 0 AS $e}, {a, null AS b, 1 AS $e}]) + +- Expand(projects=[a, b, $e], projects=[{a, b, 0 AS $e}, {a, null AS b, 1 AS $e}]) +- Calc(select=[a, b]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -499,117 +612,9 @@ Calc(select=[a, CAST(EXPR$1) AS EXPR$1, EXPR$2]) +- HashAggregate(isMerge=[true], groupBy=[a, b, $e], select=[a, b, $e, Final_COUNT(count1$0) AS EXPR$1]) +- Exchange(distribution=[hash[a, b, $e]]) +- LocalHashAggregate(groupBy=[a, b, $e], select=[a, b, $e, Partial_COUNT(*) AS count1$0]) - +- Expand(projects=[{a=[$0], b=[$1], $e=[0]}, {a=[$0], b=[null], $e=[1]}], projects=[{a, b, 0 AS $e}, {a, null AS b, 1 AS $e}]) + +- Expand(projects=[a, b, $e], projects=[{a, b, 0 AS $e}, {a, null AS b, 1 AS $e}]) +- Calc(select=[a, b]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) -]]> - - - - - - - - - - - - - - - - 0), -COUNT(DISTINCT b) FILTER (WHERE b > 1) FROM MyTable2 GROUP BY d]]> - - - ($0, 0))], b=[$1], $f4=[IS TRUE(>($1, 1))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) -]]> - - - (a, 0)) AS $f2, b, IS TRUE(>(b, 1)) AS $f4]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) -]]> - - - - - 0), -MAX(e), MIN(e) FROM MyTable2 GROUP BY d]]> - - - ($0, 0))], e=[$4]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) -]]> - - - (a, 0)) AS $f2, e]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) -]]> - - - - - 10), -COUNT(DISTINCT c) FILTER (WHERE a < 10) FROM MyTable2 GROUP BY d]]> - - - ($0, 10))], $f3=[IS TRUE(<($0, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) -]]> - - - (a, 10)) AS $f2, IS TRUE(<(a, 10)) AS $f3]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) ]]> @@ -637,33 +642,28 @@ HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS EX ]]> - + - 5), COUNT(DISTINCT b) FILTER (WHERE b > 3) -FROM MyTable2 GROUP BY d]]> + ($0, 5))], b=[$1], $f6=[IS TRUE(>($1, 3))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) +LogicalAggregate(group=[{0}], EXPR$1=[COUNT()], EXPR$2=[SUM(DISTINCT $1)], EXPR$3=[COUNT(DISTINCT $2)]) ++- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) ]]> (a, 5)) AS $f4, b, IS TRUE(>(b, 3)) AS $f6]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) +Calc(select=[a, CAST(EXPR$1) AS EXPR$1, EXPR$2, EXPR$3]) ++- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_MIN(min$0) AS EXPR$1, Final_SUM(sum$1) AS EXPR$2, Final_COUNT(count$2) AS EXPR$3]) + +- Exchange(distribution=[hash[a]]) + +- LocalHashAggregate(groupBy=[a], select=[a, Partial_MIN(EXPR$1) FILTER $g_3 AS min$0, Partial_SUM(b) FILTER $g_1 AS sum$1, Partial_COUNT(c) FILTER $g_2 AS count$2]) + +- Calc(select=[a, b, c, EXPR$1, =(CASE(=($e, 1:BIGINT), 1:BIGINT, =($e, 2:BIGINT), 2:BIGINT, 3:BIGINT), 1) AS $g_1, =(CASE(=($e, 1:BIGINT), 1:BIGINT, =($e, 2:BIGINT), 2:BIGINT, 3:BIGINT), 2) AS $g_2, =(CASE(=($e, 1:BIGINT), 1:BIGINT, =($e, 2:BIGINT), 2:BIGINT, 3:BIGINT), 3) AS $g_3]) + +- HashAggregate(isMerge=[true], groupBy=[a, b, c, $e], select=[a, b, c, $e, Final_COUNT(count1$0) AS EXPR$1]) + +- Exchange(distribution=[hash[a, b, c, $e]]) + +- LocalHashAggregate(groupBy=[a, b, c, $e], select=[a, b, c, $e, Partial_COUNT(*) AS count1$0]) + +- Expand(projects=[a, b, c, $e], projects=[{a, b, null AS c, 1 AS $e}, {a, null AS b, c, 2 AS $e}, {a, null AS b, null AS c, 3 AS $e}]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/GroupingSetsTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/GroupingSetsTest.xml index d90e9fcd84..2af1a76301 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/GroupingSetsTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/GroupingSetsTest.xml @@ -37,7 +37,7 @@ Calc(select=[d, c, g]) +- HashAggregate(isMerge=[true], groupBy=[d, g, $e], select=[d, g, $e, Final_COUNT(count1$0) AS c]) +- Exchange(distribution=[hash[d, g, $e]]) +- LocalHashAggregate(groupBy=[d, g, $e], select=[d, g, $e, Partial_COUNT(*) AS count1$0]) - +- Expand(projects=[{d=[$0], g=[$1], $e=[0]}, {d=[$0], g=[null], $e=[1]}, {d=[null], g=[$1], $e=[2]}, {d=[null], g=[null], $e=[3]}], projects=[{d, g, 0 AS $e}, {d, null AS g, 1 AS $e}, {null AS d, g, 2 AS $e}, {null AS d, null AS g, 3 AS $e}]) + +- Expand(projects=[d, g, $e], projects=[{d, g, 0 AS $e}, {d, null AS g, 1 AS $e}, {null AS d, g, 2 AS $e}, {null AS d, null AS g, 3 AS $e}]) +- Calc(select=[MOD(deptno, 20) AS d, gender AS g]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -61,7 +61,7 @@ Calc(select=[c]) +- HashAggregate(isMerge=[true], groupBy=[$f0, $e], select=[$f0, $e, Final_COUNT(count1$0) AS c]) +- Exchange(distribution=[hash[$f0, $e]]) +- LocalHashAggregate(groupBy=[$f0, $e], select=[$f0, $e, Partial_COUNT(*) AS count1$0]) - +- Expand(projects=[{$f0=[$0], $e=[0]}, {$f0=[null], $e=[1]}], projects=[{$f0, 0 AS $e}, {null AS $f0, 1 AS $e}]) + +- Expand(projects=[$f0, $e], projects=[{$f0, 0 AS $e}, {null AS $f0, 1 AS $e}]) +- Calc(select=[1 AS $f0]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -88,7 +88,7 @@ Calc(select=[d1, -(d1, 1) AS d0, c]) +- HashAggregate(isMerge=[true], groupBy=[d1, $e], select=[d1, $e, Final_COUNT(count1$0) AS c]) +- Exchange(distribution=[hash[d1, $e]]) +- LocalHashAggregate(groupBy=[d1, $e], select=[d1, $e, Partial_COUNT(*) AS count1$0]) - +- Expand(projects=[{d1=[$0], $e=[0]}, {d1=[null], $e=[1]}], projects=[{d1, 0 AS $e}, {null AS d1, 1 AS $e}]) + +- Expand(projects=[d1, $e], projects=[{d1, 0 AS $e}, {null AS d1, 1 AS $e}]) +- Calc(select=[+(deptno, 1) AS d1]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -115,7 +115,7 @@ Calc(select=[d, c, g]) +- HashAggregate(isMerge=[true], groupBy=[d, g, $e], select=[d, g, $e, Final_COUNT(count1$0) AS c]) +- Exchange(distribution=[hash[d, g, $e]]) +- LocalHashAggregate(groupBy=[d, g, $e], select=[d, g, $e, Partial_COUNT(*) AS count1$0]) - +- Expand(projects=[{d=[$0], g=[$1], $e=[0]}, {d=[$0], g=[null], $e=[1]}, {d=[null], g=[null], $e=[3]}], projects=[{d, g, 0 AS $e}, {d, null AS g, 1 AS $e}, {null AS d, null AS g, 3 AS $e}]) + +- Expand(projects=[d, g, $e], projects=[{d, g, 0 AS $e}, {d, null AS g, 1 AS $e}, {null AS d, null AS g, 3 AS $e}]) +- Calc(select=[MOD(deptno, 20) AS d, gender AS g]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -139,7 +139,7 @@ Calc(select=[c]) +- HashAggregate(isMerge=[true], groupBy=[$f0, $e], select=[$f0, $e, Final_COUNT(count1$0) AS c]) +- Exchange(distribution=[hash[$f0, $e]]) +- LocalHashAggregate(groupBy=[$f0, $e], select=[$f0, $e, Partial_COUNT(*) AS count1$0]) - +- Expand(projects=[{$f0=[$0], $e=[0]}, {$f0=[null], $e=[1]}], projects=[{$f0, 0 AS $e}, {null AS $f0, 1 AS $e}]) + +- Expand(projects=[$f0, $e], projects=[{$f0, 0 AS $e}, {null AS $f0, 1 AS $e}]) +- Calc(select=[1 AS $f0]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -162,7 +162,7 @@ Calc(select=[deptno, c]) +- HashAggregate(isMerge=[true], groupBy=[deptno, $e], select=[deptno, $e, Final_COUNT(count1$0) AS c]) +- Exchange(distribution=[hash[deptno, $e]]) +- LocalHashAggregate(groupBy=[deptno, $e], select=[deptno, $e, Partial_COUNT(*) AS count1$0]) - +- Expand(projects=[{deptno=[$0], $e=[0]}, {deptno=[null], $e=[1]}], projects=[{deptno, 0 AS $e}, {null AS deptno, 1 AS $e}]) + +- Expand(projects=[deptno, $e], projects=[{deptno, 0 AS $e}, {null AS deptno, 1 AS $e}]) +- Calc(select=[deptno]) +- TableSourceScan(table=[[default_catalog, default_database, emps, source: [TestTableSource(empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat)]]], fields=[empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat]) ]]> @@ -177,20 +177,32 @@ FROM scott_emp GROUP BY GROUPING SETS (deptno, (), ()) @@ -211,9 +223,10 @@ FROM MyTable @@ -222,7 +235,7 @@ Calc(select=[b, c, a, 0:BIGINT AS g, CASE(OR(=($e, 0:BIGINT), =($e, 1:BIGINT)), +- HashAggregate(isMerge=[true], groupBy=[b, c, $e], select=[b, c, $e, Final_AVG(sum$0, count$1) AS a]) +- Exchange(distribution=[hash[b, c, $e]]) +- LocalHashAggregate(groupBy=[b, c, $e], select=[b, c, $e, Partial_AVG(a) AS (sum$0, count$1)]) - +- Expand(projects=[{a=[$0], b=[$1], c=[$2], $e=[0]}, {a=[$0], b=[$1], c=[null], $e=[1]}, {a=[$0], b=[null], c=[$2], $e=[2]}, {a=[$0], b=[null], c=[null], $e=[3]}], projects=[{a, b, c, 0 AS $e}, {a, b, null AS c, 1 AS $e}, {a, null AS b, c, 2 AS $e}, {a, null AS b, null AS c, 3 AS $e}]) + +- Expand(projects=[a, b, c, $e], projects=[{a, b, c, 0 AS $e}, {a, b, null AS c, 1 AS $e}, {a, null AS b, c, 2 AS $e}, {a, null AS b, null AS c, 3 AS $e}]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -249,7 +262,7 @@ HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0]) +- HashAggregate(isMerge=[true], groupBy=[deptno, gender, $e], select=[deptno, gender, $e, Final_COUNT(count1$0) AS EXPR$0]) +- Exchange(distribution=[hash[deptno, gender, $e]]) +- LocalHashAggregate(groupBy=[deptno, gender, $e], select=[deptno, gender, $e, Partial_COUNT(*) AS count1$0]) - +- Expand(projects=[{deptno=[$0], gender=[$1], $e=[0]}, {deptno=[$0], gender=[null], $e=[1]}, {deptno=[null], gender=[$1], $e=[2]}, {deptno=[null], gender=[null], $e=[3]}], projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, gender, 2 AS $e}, {null AS deptno, null AS gender, 3 AS $e}]) + +- Expand(projects=[deptno, gender, $e], projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, gender, 2 AS $e}, {null AS deptno, null AS gender, 3 AS $e}]) +- Calc(select=[deptno, gender]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -284,7 +297,7 @@ Calc(select=[deptno, gender, min_name], where=[OR(>($f5, 2), AND(=(gender, _UTF- +- Exchange(distribution=[hash[deptno, gender, deptno0, $e]]) +- LocalSortAggregate(groupBy=[deptno, gender, deptno0, $e], select=[deptno, gender, deptno0, $e, Partial_MIN(ename) AS min$0, Partial_COUNT(*) AS count1$1]) +- Sort(orderBy=[deptno ASC, gender ASC, deptno0 ASC, $e ASC]) - +- Expand(projects=[{ename=[$0], deptno=[$1], gender=[$2], deptno0=[$3], $e=[0]}, {ename=[$0], deptno=[$1], gender=[$2], deptno0=[null], $e=[1]}, {ename=[$0], deptno=[$1], gender=[null], deptno0=[$3], $e=[2]}, {ename=[$0], deptno=[$1], gender=[null], deptno0=[null], $e=[3]}, {ename=[$0], deptno=[null], gender=[$2], deptno0=[$3], $e=[4]}, {ename=[$0], deptno=[null], gender=[$2], deptno0=[null], $e=[5]}, {ename=[$0], deptno=[null], gender=[null], deptno0=[$3], $e=[6]}, {ename=[$0], deptno=[null], gender=[null], deptno0=[null], $e=[7]}], projects=[{ename, deptno, gender, deptno0, 0 AS $e}, {ename, deptno, gender, null AS deptno0, 1 AS $e}, {ename, deptno, null AS gender, deptno0, 2 AS $e}, {ename, deptno, null AS gender, null AS deptno0, 3 AS $e}, {ename, null AS deptno, gender, deptno0, 4 AS $e}, {ename, null AS deptno, gender, null AS deptno0, 5 AS $e}, {ename, null AS deptno, null AS gender, deptno0, 6 AS $e}, {ename, null AS deptno, null AS gender, null AS deptno0, 7 AS $e}]) + +- Expand(projects=[ename, deptno, gender, deptno0, $e], projects=[{ename, deptno, gender, deptno0, 0 AS $e}, {ename, deptno, gender, null AS deptno0, 1 AS $e}, {ename, deptno, null AS gender, deptno0, 2 AS $e}, {ename, deptno, null AS gender, null AS deptno0, 3 AS $e}, {ename, null AS deptno, gender, deptno0, 4 AS $e}, {ename, null AS deptno, gender, null AS deptno0, 5 AS $e}, {ename, null AS deptno, null AS gender, deptno0, 6 AS $e}, {ename, null AS deptno, null AS gender, null AS deptno0, 7 AS $e}]) +- HashJoin(joinType=[InnerJoin], where=[=(deptno, deptno0)], select=[ename, deptno, gender, deptno0], build=[right]) :- Exchange(distribution=[hash[deptno]]) : +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) @@ -311,9 +324,10 @@ FROM MyTable @@ -322,7 +336,7 @@ Calc(select=[b, c, a, 0:BIGINT AS g, CASE(OR(=($e, 0:BIGINT), =($e, 1:BIGINT)), +- HashAggregate(isMerge=[true], groupBy=[b, c, $e], select=[b, c, $e, Final_AVG(sum$0, count$1) AS a]) +- Exchange(distribution=[hash[b, c, $e]]) +- LocalHashAggregate(groupBy=[b, c, $e], select=[b, c, $e, Partial_AVG(a) AS (sum$0, count$1)]) - +- Expand(projects=[{a=[$0], b=[$1], c=[$2], $e=[0]}, {a=[$0], b=[$1], c=[null], $e=[1]}, {a=[$0], b=[null], c=[$2], $e=[2]}, {a=[$0], b=[null], c=[null], $e=[3]}], projects=[{a, b, c, 0 AS $e}, {a, b, null AS c, 1 AS $e}, {a, null AS b, c, 2 AS $e}, {a, null AS b, null AS c, 3 AS $e}]) + +- Expand(projects=[a, b, c, $e], projects=[{a, b, c, 0 AS $e}, {a, b, null AS c, 1 AS $e}, {a, null AS b, c, 2 AS $e}, {a, null AS b, null AS c, 3 AS $e}]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -352,7 +366,7 @@ Calc(select=[deptno, gender, CASE(=($e, 0:BIGINT), 0:BIGINT, =($e, 1:BIGINT), 5: +- HashAggregate(isMerge=[true], groupBy=[gender, deptno, $e], select=[gender, deptno, $e, Final_COUNT(count1$0) AS c]) +- Exchange(distribution=[hash[gender, deptno, $e]]) +- LocalHashAggregate(groupBy=[gender, deptno, $e], select=[gender, deptno, $e, Partial_COUNT(*) AS count1$0]) - +- Expand(projects=[{gender=[$0], deptno=[$1], $e=[0]}, {gender=[$0], deptno=[null], $e=[1]}, {gender=[null], deptno=[null], $e=[3]}], projects=[{gender, deptno, 0 AS $e}, {gender, null AS deptno, 1 AS $e}, {null AS gender, null AS deptno, 3 AS $e}]) + +- Expand(projects=[gender, deptno, $e], projects=[{gender, deptno, 0 AS $e}, {gender, null AS deptno, 1 AS $e}, {null AS gender, null AS deptno, 3 AS $e}]) +- Calc(select=[gender, CAST(10) AS deptno], where=[=(deptno, 10)]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -376,7 +390,7 @@ Calc(select=[+(deptno, 1) AS EXPR$0, c]) +- HashAggregate(isMerge=[true], groupBy=[deptno, gender, $e], select=[deptno, gender, $e, Final_COUNT(count1$0) AS c]) +- Exchange(distribution=[hash[deptno, gender, $e]]) +- LocalHashAggregate(groupBy=[deptno, gender, $e], select=[deptno, gender, $e, Partial_COUNT(*) AS count1$0]) - +- Expand(projects=[{deptno=[$0], gender=[$1], $e=[0]}, {deptno=[$0], gender=[null], $e=[1]}, {deptno=[null], gender=[$1], $e=[2]}, {deptno=[null], gender=[null], $e=[3]}], projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, gender, 2 AS $e}, {null AS deptno, null AS gender, 3 AS $e}]) + +- Expand(projects=[deptno, gender, $e], projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, gender, 2 AS $e}, {null AS deptno, null AS gender, 3 AS $e}]) +- Calc(select=[deptno, gender]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -414,7 +428,7 @@ Sort(orderBy=[deptno ASC, job ASC, empno ASC]) +- HashAggregate(isMerge=[true], groupBy=[deptno, job, empno, ename, $e], select=[deptno, job, empno, ename, $e, Final_SUM(sum$0) AS sumsal]) +- Exchange(distribution=[hash[deptno, job, empno, ename, $e]]) +- LocalHashAggregate(groupBy=[deptno, job, empno, ename, $e], select=[deptno, job, empno, ename, $e, Partial_SUM(sal) AS sum$0]) - +- Expand(projects=[{deptno=[$0], job=[$1], empno=[$2], ename=[$3], sal=[$4], $e=[0]}, {deptno=[$0], job=[$1], empno=[null], ename=[null], sal=[$4], $e=[3]}, {deptno=[$0], job=[null], empno=[null], ename=[null], sal=[$4], $e=[7]}, {deptno=[null], job=[null], empno=[null], ename=[null], sal=[$4], $e=[15]}], projects=[{deptno, job, empno, ename, sal, 0 AS $e}, {deptno, job, null AS empno, null AS ename, sal, 3 AS $e}, {deptno, null AS job, null AS empno, null AS ename, sal, 7 AS $e}, {null AS deptno, null AS job, null AS empno, null AS ename, sal, 15 AS $e}]) + +- Expand(projects=[deptno, job, empno, ename, sal, $e], projects=[{deptno, job, empno, ename, sal, 0 AS $e}, {deptno, job, null AS empno, null AS ename, sal, 3 AS $e}, {deptno, null AS job, null AS empno, null AS ename, sal, 7 AS $e}, {null AS deptno, null AS job, null AS empno, null AS ename, sal, 15 AS $e}]) +- Calc(select=[deptno, job, empno, ename, sal]) +- TableSourceScan(table=[[default_catalog, default_database, scott_emp, source: [TestTableSource(empno, ename, job, mgr, hiredate, sal, comm, deptno)]]], fields=[empno, ename, job, mgr, hiredate, sal, comm, deptno]) ]]> @@ -449,7 +463,7 @@ Calc(select=[c, CASE(OR(=($e, 0:BIGINT), =($e, 1:BIGINT)), 0:BIGINT, 1:BIGINT) A +- Exchange(distribution=[hash[deptno, gender, $e]]) +- LocalHashAggregate(groupBy=[deptno, gender, $e], select=[deptno, gender, $e, Partial_COUNT(*) AS count1$0]) +- Calc(select=[deptno, gender, $e], where=[<=(CASE(OR(=($e, 0:BIGINT), =($e, 1:BIGINT)), 0:BIGINT, 1:BIGINT), CASE(=($e, 0:BIGINT), 0:BIGINT, =($e, 1:BIGINT), 2:BIGINT, 7:BIGINT))]) - +- Expand(projects=[{deptno=[$0], gender=[$1], $e=[0]}, {deptno=[$0], gender=[null], $e=[1]}, {deptno=[null], gender=[null], $e=[3]}], projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, null AS gender, 3 AS $e}]) + +- Expand(projects=[deptno, gender, $e], projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, null AS gender, 3 AS $e}]) +- Calc(select=[deptno, gender]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -470,8 +484,8 @@ FROM emp GROUP BY ROLLUP(deptno, gender) @@ -482,7 +496,7 @@ Calc(select=[c, CASE(OR(=($e, 0:BIGINT), =($e, 1:BIGINT)), 0:BIGINT, 1:BIGINT) A +- HashAggregate(isMerge=[true], groupBy=[deptno, gender, $e], select=[deptno, gender, $e, Final_COUNT(count1$0) AS c]) +- Exchange(distribution=[hash[deptno, gender, $e]]) +- LocalHashAggregate(groupBy=[deptno, gender, $e], select=[deptno, gender, $e, Partial_COUNT(*) AS count1$0]) - +- Expand(projects=[{deptno=[$0], gender=[$1], $e=[0]}, {deptno=[$0], gender=[null], $e=[1]}, {deptno=[null], gender=[null], $e=[3]}], projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, null AS gender, 3 AS $e}]) + +- Expand(projects=[deptno, gender, $e], projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, null AS gender, 3 AS $e}]) +- Calc(select=[deptno, gender]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -504,8 +518,8 @@ FROM emp @@ -516,7 +530,7 @@ Calc(select=[deptno, gender, CASE(OR(=($e, 0:BIGINT), =($e, 1:BIGINT)), 0:BIGINT +- HashAggregate(isMerge=[true], groupBy=[deptno, gender, $e], select=[deptno, gender, $e, Final_COUNT(count1$0) AS c]) +- Exchange(distribution=[hash[deptno, gender, $e]]) +- LocalHashAggregate(groupBy=[deptno, gender, $e], select=[deptno, gender, $e, Partial_COUNT(*) AS count1$0]) - +- Expand(projects=[{deptno=[$0], gender=[$1], $e=[0]}, {deptno=[$0], gender=[null], $e=[1]}, {deptno=[null], gender=[$1], $e=[2]}, {deptno=[null], gender=[null], $e=[3]}], projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, gender, 2 AS $e}, {null AS deptno, null AS gender, 3 AS $e}]) + +- Expand(projects=[deptno, gender, $e], projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, gender, 2 AS $e}, {null AS deptno, null AS gender, 3 AS $e}]) +- Calc(select=[deptno, gender]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -547,7 +561,7 @@ Calc(select=[deptno, job, c, CASE(OR(=($e, 0:BIGINT), =($e, 1:BIGINT)), 0:BIGINT +- HashAggregate(isMerge=[true], groupBy=[deptno, job, $e], select=[deptno, job, $e, Final_COUNT(count1$0) AS c]) +- Exchange(distribution=[hash[deptno, job, $e]]) +- LocalHashAggregate(groupBy=[deptno, job, $e], select=[deptno, job, $e, Partial_COUNT(*) AS count1$0]) - +- Expand(projects=[{deptno=[$0], job=[$1], $e=[0]}, {deptno=[$0], job=[null], $e=[1]}, {deptno=[null], job=[$1], $e=[2]}, {deptno=[null], job=[null], $e=[3]}], projects=[{deptno, job, 0 AS $e}, {deptno, null AS job, 1 AS $e}, {null AS deptno, job, 2 AS $e}, {null AS deptno, null AS job, 3 AS $e}]) + +- Expand(projects=[deptno, job, $e], projects=[{deptno, job, 0 AS $e}, {deptno, null AS job, 1 AS $e}, {null AS deptno, job, 2 AS $e}, {null AS deptno, null AS job, 3 AS $e}]) +- Calc(select=[deptno, job]) +- TableSourceScan(table=[[default_catalog, default_database, scott_emp, source: [TestTableSource(empno, ename, job, mgr, hiredate, sal, comm, deptno)]]], fields=[empno, ename, job, mgr, hiredate, sal, comm, deptno]) ]]> @@ -596,7 +610,7 @@ Calc(select=[c, deptno, CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT) AS g]) +- HashAggregate(isMerge=[true], groupBy=[deptno, $e], select=[deptno, $e, Final_COUNT(count1$0) AS c]) +- Exchange(distribution=[hash[deptno, $e]]) +- LocalHashAggregate(groupBy=[deptno, $e], select=[deptno, $e, Partial_COUNT(*) AS count1$0]) - +- Expand(projects=[{deptno=[$0], $e=[0]}, {deptno=[null], $e=[1]}], projects=[{deptno, 0 AS $e}, {null AS deptno, 1 AS $e}]) + +- Expand(projects=[deptno, $e], projects=[{deptno, 0 AS $e}, {null AS deptno, 1 AS $e}]) +- Calc(select=[deptno]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -611,9 +625,10 @@ GROUP BY GROUPING SETS (b, c) @@ -622,7 +637,7 @@ Calc(select=[b, c, a, 0:BIGINT AS g]) +- HashAggregate(isMerge=[true], groupBy=[b, c, $e], select=[b, c, $e, Final_AVG(sum$0, count$1) AS a]) +- Exchange(distribution=[hash[b, c, $e]]) +- LocalHashAggregate(groupBy=[b, c, $e], select=[b, c, $e, Partial_AVG(a) AS (sum$0, count$1)]) - +- Expand(projects=[{a=[$0], b=[$1], c=[null], $e=[1]}, {a=[$0], b=[null], c=[$2], $e=[2]}], projects=[{a, b, null AS c, 1 AS $e}, {a, null AS b, c, 2 AS $e}]) + +- Expand(projects=[a, b, c, $e], projects=[{a, b, null AS c, 1 AS $e}, {a, null AS b, c, 2 AS $e}]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -644,7 +659,7 @@ Calc(select=[b, c, a]) +- HashAggregate(isMerge=[true], groupBy=[b, c, $e], select=[b, c, $e, Final_AVG(sum$0, count$1) AS a]) +- Exchange(distribution=[hash[b, c, $e]]) +- LocalHashAggregate(groupBy=[b, c, $e], select=[b, c, $e, Partial_AVG(a) AS (sum$0, count$1)]) - +- Expand(projects=[{a=[$0], b=[$1], c=[null], $e=[1]}, {a=[$0], b=[null], c=[$2], $e=[2]}, {a=[$0], b=[null], c=[null], $e=[3]}], projects=[{a, b, null AS c, 1 AS $e}, {a, null AS b, c, 2 AS $e}, {a, null AS b, null AS c, 3 AS $e}]) + +- Expand(projects=[a, b, c, $e], projects=[{a, b, null AS c, 1 AS $e}, {a, null AS b, c, 2 AS $e}, {a, null AS b, null AS c, 3 AS $e}]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -667,9 +682,10 @@ FROM MyTable @@ -678,7 +694,7 @@ Calc(select=[b, c, a, 0:BIGINT AS g, CASE(=($e, 1:BIGINT), 0:BIGINT, 1:BIGINT) A +- HashAggregate(isMerge=[true], groupBy=[b, c, $e], select=[b, c, $e, Final_AVG(sum$0, count$1) AS a, Final_COUNT(count1$2) AS cnt]) +- Exchange(distribution=[hash[b, c, $e]]) +- LocalHashAggregate(groupBy=[b, c, $e], select=[b, c, $e, Partial_AVG(a) AS (sum$0, count$1), Partial_COUNT(*) AS count1$2]) - +- Expand(projects=[{a=[$0], b=[$1], c=[null], $e=[1]}, {a=[$0], b=[null], c=[$2], $e=[2]}, {a=[$0], b=[null], c=[null], $e=[3]}], projects=[{a, b, null AS c, 1 AS $e}, {a, null AS b, c, 2 AS $e}, {a, null AS b, null AS c, 3 AS $e}]) + +- Expand(projects=[a, b, c, $e], projects=[{a, b, null AS c, 1 AS $e}, {a, null AS b, c, 2 AS $e}, {a, null AS b, null AS c, 3 AS $e}]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -702,7 +718,7 @@ Calc(select=[EXPR$0, c]) +- HashAggregate(isMerge=[true], groupBy=[EXPR$0, $e], select=[EXPR$0, $e, Final_COUNT(count1$0) AS c]) +- Exchange(distribution=[hash[EXPR$0, $e]]) +- LocalHashAggregate(groupBy=[EXPR$0, $e], select=[EXPR$0, $e, Partial_COUNT(*) AS count1$0]) - +- Expand(projects=[{EXPR$0=[$0], $e=[0]}, {EXPR$0=[null], $e=[1]}], projects=[{EXPR$0, 0 AS $e}, {null AS EXPR$0, 1 AS $e}]) + +- Expand(projects=[EXPR$0, $e], projects=[{EXPR$0, 0 AS $e}, {null AS EXPR$0, 1 AS $e}]) +- Calc(select=[+(deptno, 1) AS EXPR$0]) +- TableSourceScan(table=[[default_catalog, default_database, emps, source: [TestTableSource(empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat)]]], fields=[empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat]) ]]> @@ -733,7 +749,7 @@ Calc(select=[c]) +- HashAggregate(isMerge=[true], groupBy=[deptno, $e], select=[deptno, $e, Final_COUNT(count1$0) AS c]) +- Exchange(distribution=[hash[deptno, $e]]) +- LocalHashAggregate(groupBy=[deptno, $e], select=[deptno, $e, Partial_COUNT(*) AS count1$0]) - +- Expand(projects=[{deptno=[$0], $e=[0]}, {deptno=[null], $e=[1]}], projects=[{deptno, 0 AS $e}, {null AS deptno, 1 AS $e}]) + +- Expand(projects=[deptno, $e], projects=[{deptno, 0 AS $e}, {null AS deptno, 1 AS $e}]) +- Calc(select=[deptno]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -752,9 +768,10 @@ FROM emps GROUP BY GROUPING SETS (deptno) @@ -785,9 +802,10 @@ FROM MyTable @@ -796,7 +814,7 @@ Calc(select=[b, c, a, 0:BIGINT AS g, CASE(OR(=($e, 0:BIGINT), =($e, 1:BIGINT)), +- HashAggregate(isMerge=[true], groupBy=[b, c, $e], select=[b, c, $e, Final_AVG(sum$0, count$1) AS a]) +- Exchange(distribution=[hash[b, c, $e]]) +- LocalHashAggregate(groupBy=[b, c, $e], select=[b, c, $e, Partial_AVG(a) AS (sum$0, count$1)]) - +- Expand(projects=[{a=[$0], b=[$1], c=[$2], $e=[0]}, {a=[$0], b=[$1], c=[null], $e=[1]}, {a=[$0], b=[null], c=[null], $e=[3]}], projects=[{a, b, c, 0 AS $e}, {a, b, null AS c, 1 AS $e}, {a, null AS b, null AS c, 3 AS $e}]) + +- Expand(projects=[a, b, c, $e], projects=[{a, b, c, 0 AS $e}, {a, b, null AS c, 1 AS $e}, {a, null AS b, null AS c, 3 AS $e}]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -818,9 +836,10 @@ FROM MyTable @@ -829,7 +848,7 @@ Calc(select=[b, c, a, 0:BIGINT AS g, CASE(OR(=($e, 0:BIGINT), =($e, 1:BIGINT)), +- HashAggregate(isMerge=[true], groupBy=[b, c, $e], select=[b, c, $e, Final_AVG(sum$0, count$1) AS a]) +- Exchange(distribution=[hash[b, c, $e]]) +- LocalHashAggregate(groupBy=[b, c, $e], select=[b, c, $e, Partial_AVG(a) AS (sum$0, count$1)]) - +- Expand(projects=[{a=[$0], b=[$1], c=[$2], $e=[0]}, {a=[$0], b=[$1], c=[null], $e=[1]}, {a=[$0], b=[null], c=[null], $e=[3]}], projects=[{a, b, c, 0 AS $e}, {a, b, null AS c, 1 AS $e}, {a, null AS b, null AS c, 3 AS $e}]) + +- Expand(projects=[a, b, c, $e], projects=[{a, b, c, 0 AS $e}, {a, b, null AS c, 1 AS $e}, {a, null AS b, null AS c, 3 AS $e}]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -852,7 +871,7 @@ Calc(select=[deptno, c]) +- HashAggregate(isMerge=[true], groupBy=[deptno, gender, $e], select=[deptno, gender, $e, Final_COUNT(count1$0) AS c]) +- Exchange(distribution=[hash[deptno, gender, $e]]) +- LocalHashAggregate(groupBy=[deptno, gender, $e], select=[deptno, gender, $e, Partial_COUNT(*) AS count1$0]) - +- Expand(projects=[{deptno=[$0], gender=[$1], $e=[0]}, {deptno=[$0], gender=[null], $e=[1]}, {deptno=[null], gender=[$1], $e=[2]}, {deptno=[null], gender=[null], $e=[3]}], projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, gender, 2 AS $e}, {null AS deptno, null AS gender, 3 AS $e}]) + +- Expand(projects=[deptno, gender, $e], projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, gender, 2 AS $e}, {null AS deptno, null AS gender, 3 AS $e}]) +- Calc(select=[deptno, gender]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -879,7 +898,7 @@ Calc(select=[+($f0, 1) AS half1, c]) +- HashAggregate(isMerge=[true], groupBy=[$f0, gender, $f2, $e], select=[$f0, gender, $f2, $e, Final_COUNT(count1$0) AS c]) +- Exchange(distribution=[hash[$f0, gender, $f2, $e]]) +- LocalHashAggregate(groupBy=[$f0, gender, $f2, $e], select=[$f0, gender, $f2, $e, Partial_COUNT(*) AS count1$0]) - +- Expand(projects=[{$f0=[$0], gender=[$1], $f2=[$2], $e=[0]}, {$f0=[$0], gender=[$1], $f2=[null], $e=[1]}, {$f0=[$0], gender=[null], $f2=[$2], $e=[2]}, {$f0=[$0], gender=[null], $f2=[null], $e=[3]}, {$f0=[null], gender=[null], $f2=[$2], $e=[6]}, {$f0=[null], gender=[null], $f2=[null], $e=[7]}], projects=[{$f0, gender, $f2, 0 AS $e}, {$f0, gender, null AS $f2, 1 AS $e}, {$f0, null AS gender, $f2, 2 AS $e}, {$f0, null AS gender, null AS $f2, 3 AS $e}, {null AS $f0, null AS gender, $f2, 6 AS $e}, {null AS $f0, null AS gender, null AS $f2, 7 AS $e}]) + +- Expand(projects=[$f0, gender, $f2, $e], projects=[{$f0, gender, $f2, 0 AS $e}, {$f0, gender, null AS $f2, 1 AS $e}, {$f0, null AS gender, $f2, 2 AS $e}, {$f0, null AS gender, null AS $f2, 3 AS $e}, {null AS $f0, null AS gender, $f2, 6 AS $e}, {null AS $f0, null AS gender, null AS $f2, 7 AS $e}]) +- Calc(select=[/(deptno, 2) AS $f0, gender, SUBSTRING(ename, 1, 1) AS $f2]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -905,7 +924,7 @@ Calc(select=[gender, +(deptno, 1) AS EXPR$1, c]) +- HashAggregate(isMerge=[true], groupBy=[deptno, gender, $e], select=[deptno, gender, $e, Final_COUNT(count1$0) AS c]) +- Exchange(distribution=[hash[deptno, gender, $e]]) +- LocalHashAggregate(groupBy=[deptno, gender, $e], select=[deptno, gender, $e, Partial_COUNT(*) AS count1$0]) - +- Expand(projects=[{deptno=[$0], gender=[$1], $e=[0]}, {deptno=[$0], gender=[null], $e=[1]}, {deptno=[null], gender=[null], $e=[3]}], projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, null AS gender, 3 AS $e}]) + +- Expand(projects=[deptno, gender, $e], projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, null AS gender, 3 AS $e}]) +- Calc(select=[deptno, gender]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -928,7 +947,7 @@ Calc(select=[gender, c]) +- HashAggregate(isMerge=[true], groupBy=[gender, $e], select=[gender, $e, Final_COUNT(count1$0) AS c]) +- Exchange(distribution=[hash[gender, $e]]) +- LocalHashAggregate(groupBy=[gender, $e], select=[gender, $e, Partial_COUNT(*) AS count1$0]) - +- Expand(projects=[{gender=[$0], $e=[0]}, {gender=[null], $e=[1]}], projects=[{gender, 0 AS $e}, {null AS gender, 1 AS $e}]) + +- Expand(projects=[gender, $e], projects=[{gender, 0 AS $e}, {null AS gender, 1 AS $e}]) +- Calc(select=[gender]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -954,7 +973,7 @@ Sort(orderBy=[c DESC]) +- HashAggregate(isMerge=[true], groupBy=[gender, $e], select=[gender, $e, Final_COUNT(count1$0) AS c]) +- Exchange(distribution=[hash[gender, $e]]) +- LocalHashAggregate(groupBy=[gender, $e], select=[gender, $e, Partial_COUNT(*) AS count1$0]) - +- Expand(projects=[{gender=[$0], $e=[0]}, {gender=[null], $e=[1]}], projects=[{gender, 0 AS $e}, {null AS gender, 1 AS $e}]) + +- Expand(projects=[gender, $e], projects=[{gender, 0 AS $e}, {null AS gender, 1 AS $e}]) +- Calc(select=[gender]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -981,7 +1000,7 @@ Calc(select=[+(deptno, 1) AS d1, c], where=[>(c, 3)]) +- HashAggregate(isMerge=[true], groupBy=[deptno, $e], select=[deptno, $e, Final_COUNT(count1$0) AS c]) +- Exchange(distribution=[hash[deptno, $e]]) +- LocalHashAggregate(groupBy=[deptno, $e], select=[deptno, $e, Partial_COUNT(*) AS count1$0]) - +- Expand(projects=[{deptno=[$0], $e=[0]}, {deptno=[null], $e=[1]}], projects=[{deptno, 0 AS $e}, {null AS deptno, 1 AS $e}]) + +- Expand(projects=[deptno, $e], projects=[{deptno, 0 AS $e}, {null AS deptno, 1 AS $e}]) +- Calc(select=[deptno]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -1005,7 +1024,7 @@ Calc(select=[+(deptno, 1) AS EXPR$0, c]) +- HashAggregate(isMerge=[true], groupBy=[deptno, $e], select=[deptno, $e, Final_COUNT(count1$0) AS c]) +- Exchange(distribution=[hash[deptno, $e]]) +- LocalHashAggregate(groupBy=[deptno, $e], select=[deptno, $e, Partial_COUNT(*) AS count1$0]) - +- Expand(projects=[{deptno=[$0], $e=[0]}, {deptno=[null], $e=[1]}], projects=[{deptno, 0 AS $e}, {null AS deptno, 1 AS $e}]) + +- Expand(projects=[deptno, $e], projects=[{deptno, 0 AS $e}, {null AS deptno, 1 AS $e}]) +- Calc(select=[deptno]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateExpandDistinctAggregatesRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateExpandDistinctAggregatesRuleTest.xml index 02685051cc..62581e9dcd 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateExpandDistinctAggregatesRuleTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateExpandDistinctAggregatesRuleTest.xml @@ -33,7 +33,7 @@ FlinkLogicalCalc(select=[a, CAST(EXPR$1) AS EXPR$1, EXPR$2, EXPR$3]) +- FlinkLogicalAggregate(group=[{0}], EXPR$1=[MIN($2) FILTER $5], EXPR$2=[MIN($3) FILTER $5], EXPR$3=[SUM($1) FILTER $4]) +- FlinkLogicalCalc(select=[a, b, EXPR$1, EXPR$2, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1]) +- FlinkLogicalAggregate(group=[{0, 1, 2}], EXPR$1=[COUNT($0)], EXPR$2=[SUM($3)]) - +- FlinkLogicalExpand(projects=[{a=[$0], b=[$1], $e=[0], b_0=[$1]}, {a=[$0], b=[null], $e=[1], b_0=[$1]}]) + +- FlinkLogicalExpand(projects=[a, b, $e, b_0]) +- FlinkLogicalCalc(select=[a, b]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -57,7 +57,7 @@ FlinkLogicalCalc(select=[EXPR$0, EXPR$1]) +- FlinkLogicalAggregate(group=[{0}], EXPR$0=[SUM($1) FILTER $3], EXPR$1=[COUNT($2) FILTER $4]) +- FlinkLogicalCalc(select=[b, a, c, =(CASE(=($e, 1:BIGINT), 1:BIGINT, 2:BIGINT), 1) AS $g_1, =(CASE(=($e, 1:BIGINT), 1:BIGINT, 2:BIGINT), 2) AS $g_2]) +- FlinkLogicalAggregate(group=[{0, 1, 2, 3}]) - +- FlinkLogicalExpand(projects=[{a=[$0], b=[$1], c=[null], $e=[1]}, {a=[null], b=[$1], c=[$2], $e=[4]}]) + +- FlinkLogicalExpand(projects=[a, b, c, $e]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -78,7 +78,7 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT(DISTINCT $0)], EXPR$1=[SUM(DISTINCT $ FlinkLogicalAggregate(group=[{}], EXPR$0=[COUNT($0) FILTER $4], EXPR$1=[SUM($1) FILTER $5], EXPR$2=[MIN($2) FILTER $6], EXPR$3=[MIN($3) FILTER $6]) +- FlinkLogicalCalc(select=[a, b, EXPR$2, EXPR$3, =(CASE(=($e, 1:BIGINT), 1:BIGINT, =($e, 2:BIGINT), 2:BIGINT, 3:BIGINT), 1) AS $g_1, =(CASE(=($e, 1:BIGINT), 1:BIGINT, =($e, 2:BIGINT), 2:BIGINT, 3:BIGINT), 2) AS $g_2, =(CASE(=($e, 1:BIGINT), 1:BIGINT, =($e, 2:BIGINT), 2:BIGINT, 3:BIGINT), 3) AS $g_3]) +- FlinkLogicalAggregate(group=[{0, 1, 2}], EXPR$2=[MAX($3)], EXPR$3=[MIN($3)]) - +- FlinkLogicalExpand(projects=[{a=[$0], b=[null], $e=[1], a_0=[$0]}, {a=[null], b=[$1], $e=[2], a_0=[$0]}, {a=[null], b=[null], $e=[3], a_0=[$0]}]) + +- FlinkLogicalExpand(projects=[a, b, $e, a_0]) +- FlinkLogicalCalc(select=[a, b]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -102,7 +102,7 @@ FlinkLogicalCalc(select=[EXPR$0, EXPR$1, EXPR$2, EXPR$3]) +- FlinkLogicalAggregate(group=[{0}], EXPR$0=[COUNT($1) FILTER $5], EXPR$1=[SUM($2) FILTER $6], EXPR$2=[MIN($3) FILTER $7], EXPR$3=[MIN($4) FILTER $7]) +- FlinkLogicalCalc(select=[c, a, b, EXPR$2, EXPR$3, =(CASE(=($e, 2:BIGINT), 1:BIGINT, =($e, 4:BIGINT), 2:BIGINT, 3:BIGINT), 1) AS $g_1, =(CASE(=($e, 2:BIGINT), 1:BIGINT, =($e, 4:BIGINT), 2:BIGINT, 3:BIGINT), 2) AS $g_2, =(CASE(=($e, 2:BIGINT), 1:BIGINT, =($e, 4:BIGINT), 2:BIGINT, 3:BIGINT), 3) AS $g_3]) +- FlinkLogicalAggregate(group=[{0, 1, 2, 3}], EXPR$2=[MAX($4)], EXPR$3=[MIN($4)]) - +- FlinkLogicalExpand(projects=[{a=[$0], b=[null], c=[$2], $e=[2], a_0=[$0]}, {a=[null], b=[$1], c=[$2], $e=[4], a_0=[$0]}, {a=[null], b=[null], c=[$2], $e=[6], a_0=[$0]}]) + +- FlinkLogicalExpand(projects=[a, b, c, $e, a_0]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -121,10 +121,10 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT(DISTINCT $0)], EXPR$1=[SUM(DISTINCT $ (a, 5)) AS $f3]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -148,8 +148,8 @@ FlinkLogicalCalc(select=[EXPR$0, EXPR$1]) +- FlinkLogicalAggregate(group=[{0, 1, 4}], EXPR$0=[COUNT($2) FILTER $5], EXPR$1=[SUM($3) FILTER $6]) +- FlinkLogicalCalc(select=[c, d, a, b, $e, =(CASE(=($e_0, 2:BIGINT), 2:BIGINT, 4:BIGINT), 2) AS $g_2, =(CASE(=($e_0, 2:BIGINT), 2:BIGINT, 4:BIGINT), 4) AS $g_4]) +- FlinkLogicalAggregate(group=[{0, 1, 2, 3, 4, 5}]) - +- FlinkLogicalExpand(projects=[{c=[$0], d=[$1], a=[$2], b=[null], $e=[$4], $e_0=[2]}, {c=[$0], d=[$1], a=[null], b=[$3], $e=[$4], $e_0=[4]}]) - +- FlinkLogicalExpand(projects=[{c=[$0], d=[null], a=[$2], b=[$3], $e=[1]}, {c=[null], d=[$1], a=[$2], b=[$3], $e=[2]}]) + +- FlinkLogicalExpand(projects=[c, d, a, b, $e, $e_0]) + +- FlinkLogicalExpand(projects=[c, d, a, b, $e]) +- FlinkLogicalCalc(select=[c, d, a, b]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) ]]> @@ -169,10 +169,10 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT(DISTINCT $0)], EXPR$1=[SUM(DISTINCT $ (b, 0)) AS $f1]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -195,7 +195,7 @@ LogicalProject(EXPR$0=[$2], EXPR$1=[$3], EXPR$2=[$4]) FlinkLogicalCalc(select=[EXPR$0, EXPR$1, EXPR$2]) +- FlinkLogicalAggregate(group=[{0, 1, 3}], EXPR$0=[COUNT($2)], EXPR$1=[SUM($2)], EXPR$2=[MAX($2)]) +- FlinkLogicalAggregate(group=[{0, 1, 2, 3}]) - +- FlinkLogicalExpand(projects=[{b=[$0], c=[null], a=[$2], $e=[1]}, {b=[null], c=[$1], a=[$2], $e=[2]}]) + +- FlinkLogicalExpand(projects=[b, c, a, $e]) +- FlinkLogicalCalc(select=[b, c, a]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) ]]> @@ -215,11 +215,11 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT(DISTINCT $0) FILTER $1], EXPR$1=[SUM( (c, 0)) AS $f1, b, c]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -243,9 +243,102 @@ FlinkLogicalCalc(select=[EXPR$0, EXPR$1, CAST(EXPR$2) AS EXPR$2]) +- FlinkLogicalAggregate(group=[{2, 3, 4}], EXPR$0=[COUNT($0) FILTER $6], EXPR$1=[SUM($1) FILTER $7], EXPR$2=[MIN($5) FILTER $8]) +- FlinkLogicalCalc(select=[a, b, d, e, $e, EXPR$2, =(CASE(=($e_0, 8:BIGINT), 8:BIGINT, =($e_0, 16:BIGINT), 16:BIGINT, 24:BIGINT), 8) AS $g_8, =(CASE(=($e_0, 8:BIGINT), 8:BIGINT, =($e_0, 16:BIGINT), 16:BIGINT, 24:BIGINT), 16) AS $g_16, =(CASE(=($e_0, 8:BIGINT), 8:BIGINT, =($e_0, 16:BIGINT), 16:BIGINT, 24:BIGINT), 24) AS $g_24]) +- FlinkLogicalAggregate(group=[{0, 1, 3, 4, 5, 6}], EXPR$2=[COUNT($2)]) - +- FlinkLogicalExpand(projects=[{a=[$0], b=[null], c=[$2], d=[$3], e=[$4], $e=[$5], $e_0=[8]}, {a=[null], b=[$1], c=[$2], d=[$3], e=[$4], $e=[$5], $e_0=[16]}, {a=[null], b=[null], c=[$2], d=[$3], e=[$4], $e=[$5], $e_0=[24]}]) - +- FlinkLogicalExpand(projects=[{a=[$0], b=[$1], c=[$2], d=[$3], e=[null], $e=[1]}, {a=[$0], b=[$1], c=[$2], d=[null], e=[$4], $e=[2]}]) + +- FlinkLogicalExpand(projects=[a, b, c, d, e, $e, $e_0]) + +- FlinkLogicalExpand(projects=[a, b, c, d, e, $e]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) +]]> + + + + + 5), COUNT(DISTINCT b) FILTER (WHERE b > 3) +FROM MyTable2 GROUP BY d]]> + + + ($0, 5))], b=[$1], $f6=[IS TRUE(>($1, 3))]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) +]]> + + + (a, 5)) AS $f4, b, IS TRUE(>(b, 3)) AS $f6]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) +]]> + + + + + 0), +COUNT(DISTINCT b) FILTER (WHERE b > 1) FROM MyTable2 GROUP BY d]]> + + + ($0, 0))], b=[$1], $f4=[IS TRUE(>($1, 1))]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) +]]> + + + (a, 0)) AS $f2, b, IS TRUE(>(b, 1)) AS $f4]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) +]]> + + + + + 10), +COUNT(DISTINCT c) FILTER (WHERE a < 10) FROM MyTable2 GROUP BY d]]> + + + ($0, 10))], $f3=[IS TRUE(<($0, 10))]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) +]]> + + + (a, 10)) AS $f2, IS TRUE(<(a, 10)) AS $f3]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) +]]> + + + + + 0), +MAX(e), MIN(e) FROM MyTable2 GROUP BY d]]> + + + ($0, 0))], e=[$4]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) +]]> + + + (a, 0)) AS $f2, e]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) ]]> @@ -282,10 +375,10 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT(DISTINCT $0) FILTER $1], EXPR$1=[SUM( (a, 0)) AS $f1, b]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -304,11 +397,11 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT($0) FILTER $1], EXPR$1=[SUM(DISTINCT (a, 0)) AS $f1, b]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -332,8 +425,8 @@ FlinkLogicalCalc(select=[EXPR$0, EXPR$1]) +- FlinkLogicalAggregate(group=[{0, 1, 3}], EXPR$0=[COUNT($2) FILTER $5], EXPR$1=[MIN($4) FILTER $6]) +- FlinkLogicalCalc(select=[b, c, a, $e, EXPR$1, =(CASE(=($e_0, 0:BIGINT), 0:BIGINT, 2:BIGINT), 0) AS $g_0, =(CASE(=($e_0, 0:BIGINT), 0:BIGINT, 2:BIGINT), 2) AS $g_2]) +- FlinkLogicalAggregate(group=[{0, 1, 2, 3, 5}], EXPR$1=[SUM($4)]) - +- FlinkLogicalExpand(projects=[{b=[$0], c=[$1], a=[$2], $e=[$3], b_0=[$4], $e_0=[0]}, {b=[$0], c=[$1], a=[null], $e=[$3], b_0=[$4], $e_0=[2]}]) - +- FlinkLogicalExpand(projects=[{b=[$0], c=[null], a=[$2], $e=[1], b_0=[$0]}, {b=[null], c=[$1], a=[$2], $e=[2], b_0=[$0]}]) + +- FlinkLogicalExpand(projects=[b, c, a, $e, b_0, $e_0]) + +- FlinkLogicalExpand(projects=[b, c, a, $e, b_0]) +- FlinkLogicalCalc(select=[b, c, a]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) ]]> @@ -357,8 +450,8 @@ FlinkLogicalCalc(select=[CAST(EXPR$0) AS EXPR$0, EXPR$1]) +- FlinkLogicalAggregate(group=[{0, 1, 3}], EXPR$0=[MIN($4) FILTER $6], EXPR$1=[SUM($2) FILTER $5]) +- FlinkLogicalCalc(select=[c, d, b, $e, EXPR$0, =(CASE(=($e_0, 0:BIGINT), 0:BIGINT, 2:BIGINT), 0) AS $g_0, =(CASE(=($e_0, 0:BIGINT), 0:BIGINT, 2:BIGINT), 2) AS $g_2]) +- FlinkLogicalAggregate(group=[{0, 1, 3, 4, 5}], EXPR$0=[COUNT($2)]) - +- FlinkLogicalExpand(projects=[{c=[$0], d=[$1], a=[$2], b=[$3], $e=[$4], $e_0=[0]}, {c=[$0], d=[$1], a=[$2], b=[null], $e=[$4], $e_0=[2]}]) - +- FlinkLogicalExpand(projects=[{c=[$0], d=[null], a=[$2], b=[$3], $e=[1]}, {c=[null], d=[$1], a=[$2], b=[$3], $e=[2]}]) + +- FlinkLogicalExpand(projects=[c, d, a, b, $e, $e_0]) + +- FlinkLogicalExpand(projects=[c, d, a, b, $e]) +- FlinkLogicalCalc(select=[c, d, a, b]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) ]]> @@ -401,7 +494,7 @@ LogicalProject(EXPR$0=[$2]) FlinkLogicalCalc(select=[EXPR$0]) +- FlinkLogicalAggregate(group=[{0, 1, 4}], EXPR$0=[COUNT($2, $3)]) +- FlinkLogicalAggregate(group=[{0, 1, 2, 3, 4}]) - +- FlinkLogicalExpand(projects=[{c=[$0], d=[null], a=[$2], b=[$3], $e=[1]}, {c=[null], d=[$1], a=[$2], b=[$3], $e=[2]}]) + +- FlinkLogicalExpand(projects=[c, d, a, b, $e]) +- FlinkLogicalCalc(select=[c, d, a, b]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) ]]> @@ -424,7 +517,7 @@ FlinkLogicalCalc(select=[a, CAST(EXPR$1) AS EXPR$1, EXPR$2, EXPR$3]) +- FlinkLogicalAggregate(group=[{0}], EXPR$1=[MIN($2) FILTER $4], EXPR$2=[SUM($1) FILTER $3], EXPR$3=[COUNT($1) FILTER $3]) +- FlinkLogicalCalc(select=[a, b, EXPR$1, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1]) +- FlinkLogicalAggregate(group=[{0, 1, 2}], EXPR$1=[COUNT()]) - +- FlinkLogicalExpand(projects=[{a=[$0], b=[$1], $e=[0]}, {a=[$0], b=[null], $e=[1]}]) + +- FlinkLogicalExpand(projects=[a, b, $e]) +- FlinkLogicalCalc(select=[a, b]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -447,7 +540,7 @@ FlinkLogicalCalc(select=[a, CAST(EXPR$1) AS EXPR$1, EXPR$2]) +- FlinkLogicalAggregate(group=[{0}], EXPR$1=[MIN($2) FILTER $4], EXPR$2=[SUM($1) FILTER $3]) +- FlinkLogicalCalc(select=[a, b, EXPR$1, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1]) +- FlinkLogicalAggregate(group=[{0, 1, 2}], EXPR$1=[COUNT($0)]) - +- FlinkLogicalExpand(projects=[{a=[$0], b=[$1], $e=[0]}, {a=[$0], b=[null], $e=[1]}]) + +- FlinkLogicalExpand(projects=[a, b, $e]) +- FlinkLogicalCalc(select=[a, b]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -470,99 +563,9 @@ FlinkLogicalCalc(select=[a, CAST(EXPR$1) AS EXPR$1, EXPR$2]) +- FlinkLogicalAggregate(group=[{0}], EXPR$1=[MIN($2) FILTER $4], EXPR$2=[SUM($1) FILTER $3]) +- FlinkLogicalCalc(select=[a, b, EXPR$1, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1]) +- FlinkLogicalAggregate(group=[{0, 1, 2}], EXPR$1=[COUNT()]) - +- FlinkLogicalExpand(projects=[{a=[$0], b=[$1], $e=[0]}, {a=[$0], b=[null], $e=[1]}]) + +- FlinkLogicalExpand(projects=[a, b, $e]) +- FlinkLogicalCalc(select=[a, b]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) -]]> - - - - - - - - - - - - - - - - 0), -COUNT(DISTINCT b) FILTER (WHERE b > 1) FROM MyTable2 GROUP BY d]]> - - - ($0, 0))], b=[$1], $f4=[IS TRUE(>($1, 1))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) -]]> - - - (a, 0)) AS $f2, b, IS TRUE(>(b, 1)) AS $f4]) - +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) -]]> - - - - - 0), -MAX(e), MIN(e) FROM MyTable2 GROUP BY d]]> - - - ($0, 0))], e=[$4]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) -]]> - - - (a, 0)) AS $f2, e]) - +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) -]]> - - - - - 10), -COUNT(DISTINCT c) FILTER (WHERE a < 10) FROM MyTable2 GROUP BY d]]> - - - ($0, 10))], $f3=[IS TRUE(<($0, 10))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) -]]> - - - (a, 10)) AS $f2, IS TRUE(<(a, 10)) AS $f3]) - +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) ]]> @@ -586,29 +589,25 @@ FlinkLogicalAggregate(group=[{0}], EXPR$1=[COUNT($2)]) ]]> - + - 5), COUNT(DISTINCT b) FILTER (WHERE b > 3) -FROM MyTable2 GROUP BY d]]> + ($0, 5))], b=[$1], $f6=[IS TRUE(>($1, 3))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) +LogicalAggregate(group=[{0}], EXPR$1=[COUNT()], EXPR$2=[SUM(DISTINCT $1)], EXPR$3=[COUNT(DISTINCT $2)]) ++- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) ]]> (a, 5)) AS $f4, b, IS TRUE(>(b, 3)) AS $f6]) - +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) +FlinkLogicalCalc(select=[a, CAST(EXPR$1) AS EXPR$1, EXPR$2, EXPR$3]) ++- FlinkLogicalAggregate(group=[{0}], EXPR$1=[MIN($3) FILTER $6], EXPR$2=[SUM($1) FILTER $4], EXPR$3=[COUNT($2) FILTER $5]) + +- FlinkLogicalCalc(select=[a, b, c, EXPR$1, =(CASE(=($e, 1:BIGINT), 1:BIGINT, =($e, 2:BIGINT), 2:BIGINT, 3:BIGINT), 1) AS $g_1, =(CASE(=($e, 1:BIGINT), 1:BIGINT, =($e, 2:BIGINT), 2:BIGINT, 3:BIGINT), 2) AS $g_2, =(CASE(=($e, 1:BIGINT), 1:BIGINT, =($e, 2:BIGINT), 2:BIGINT, 3:BIGINT), 3) AS $g_3]) + +- FlinkLogicalAggregate(group=[{0, 1, 2, 3}], EXPR$1=[COUNT()]) + +- FlinkLogicalExpand(projects=[a, b, c, $e]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> - diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateRemoveRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateRemoveRuleTest.xml index 266f14beec..7c04c79d60 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateRemoveRuleTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateRemoveRuleTest.xml @@ -70,7 +70,7 @@ LogicalProject(a=[$0], s=[$3]) @@ -110,7 +110,7 @@ LogicalProject(a=[$0], s=[$3]) @@ -132,7 +132,7 @@ LogicalProject(a=[$0], s=[$2]) @@ -174,7 +174,7 @@ LogicalProject(a=[$0], EXPR$1=[$2], EXPR$2=[$3]) FlinkLogicalCalc(select=[a, EXPR$1, EXPR$2]) +- FlinkLogicalAggregate(group=[{0}], d=[AUXILIARY_GROUP($1)], EXPR$1=[COUNT($2) FILTER $4], EXPR$2=[SUM($3) FILTER $5]) +- FlinkLogicalCalc(select=[a, d, c, b, =(CASE(=($e, 1:BIGINT), 1:BIGINT, 2:BIGINT), 1) AS $g_1, =(CASE(=($e, 1:BIGINT), 1:BIGINT, 2:BIGINT), 2) AS $g_2]) - +- FlinkLogicalExpand(projects=[{a=[$0], d=[$1], c=[$2], b=[null], $e=[1]}, {a=[$0], d=[$1], c=[null], b=[$3], $e=[2]}]) + +- FlinkLogicalExpand(projects=[a, d, c, b, $e]) +- FlinkLogicalCalc(select=[a, d, c, b]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) ]]> @@ -214,7 +214,7 @@ LogicalProject(a=[$0], EXPR$1=[$2], EXPR$2=[$3]) FlinkLogicalCalc(select=[a, EXPR$1, EXPR$2]) +- FlinkLogicalAggregate(group=[{0}], d=[AUXILIARY_GROUP($1)], EXPR$1=[COUNT($2) FILTER $4], EXPR$2=[MIN($3) FILTER $5]) +- FlinkLogicalCalc(select=[a, d, c, b AS EXPR$2, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1]) - +- FlinkLogicalExpand(projects=[{a=[$0], d=[$1], c=[$2], b=[$3], $e=[0]}, {a=[$0], d=[$1], c=[null], b=[$3], $e=[1]}]) + +- FlinkLogicalExpand(projects=[a, d, c, b, $e]) +- FlinkLogicalCalc(select=[a, d, c, b]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) ]]> @@ -236,7 +236,7 @@ LogicalProject(a=[$0], s=[$3]) @@ -295,7 +295,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1)], EXPR$2=[SUM($1)]) @@ -318,7 +318,7 @@ LogicalProject(a=[$0], EXPR$1=[$2], EXPR$2=[$3]) FlinkLogicalCalc(select=[a, EXPR$1, EXPR$2]) +- FlinkLogicalAggregate(group=[{0}], c=[AUXILIARY_GROUP($1)], EXPR$1=[COUNT($2) FILTER $4], EXPR$2=[MIN($3) FILTER $5]) +- FlinkLogicalCalc(select=[a, c, b, b_0 AS EXPR$2, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1]) - +- FlinkLogicalExpand(projects=[{a=[$0], c=[$1], b=[$2], $e=[0], b_0=[$2]}, {a=[$0], c=[$1], b=[null], $e=[1], b_0=[$2]}]) + +- FlinkLogicalExpand(projects=[a, c, b, $e, b_0]) +- FlinkLogicalCalc(select=[a, c, b]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -339,7 +339,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1)], EXPR$2=[SUM($2)]) @@ -362,7 +362,7 @@ LogicalProject(a=[$0], EXPR$1=[$2], EXPR$2=[$3]) FlinkLogicalCalc(select=[a, EXPR$1, EXPR$2]) +- FlinkLogicalAggregate(group=[{0}], d=[AUXILIARY_GROUP($1)], EXPR$1=[COUNT($2) FILTER $4], EXPR$2=[MIN($3) FILTER $5]) +- FlinkLogicalCalc(select=[a, d, c, b AS EXPR$2, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1]) - +- FlinkLogicalExpand(projects=[{a=[$0], d=[$1], c=[$2], b=[$3], $e=[0]}, {a=[$0], d=[$1], c=[null], b=[$3], $e=[1]}]) + +- FlinkLogicalExpand(projects=[a, d, c, b, $e]) +- FlinkLogicalCalc(select=[a, d, c, b]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) ]]> @@ -436,7 +436,7 @@ LogicalAggregate(group=[{0}], s=[SUM($1)]) @@ -465,13 +465,13 @@ FlinkLogicalAggregate(group=[{}], EXPR$0=[MAX($0)], EXPR$1=[SUM($1)], EXPR$2=[MI @@ -501,7 +501,7 @@ FlinkLogicalCalc(select=[a, b, c]) +- FlinkLogicalCalc(select=[true AS i]) +- FlinkLogicalAggregate(group=[{}], EXPR$0=[SUM($0)]) +- FlinkLogicalCalc(select=[a]) - +- FlinkLogicalValues(type=[RecordType(INTEGER a, INTEGER b, VARCHAR(2147483647) c)], tuples=[[]]) + +- FlinkLogicalValues(tuples=[[]]) ]]> @@ -521,7 +521,7 @@ LogicalAggregate(group=[{}], EXPR$0=[SUM($0)]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRuleTest.xml index f2becf185d..f0c6efa57d 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRuleTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRuleTest.xml @@ -40,7 +40,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1) FILTER $2], EXPR$2=[SUM FlinkLogicalAggregate(group=[{0}], agg#0=[$SUM0($2)], agg#1=[SUM($3)], agg#2=[SUM($4)]) +- FlinkLogicalAggregate(group=[{0, 4}], agg#0=[COUNT(DISTINCT $1) FILTER $5], agg#1=[SUM($1) FILTER $6], agg#2=[SUM($1) FILTER $7]) +- FlinkLogicalCalc(select=[a, b, $f2, $f3, $f4, AND(=($e, 0), $f2) AS $g_0, AND(=($e, 1), $f3) AS $g_1, AND(=($e, 1), $f2) AS $g_10]) - +- FlinkLogicalExpand(projects=[{a=[$0], b=[$1], $f2=[$2], $f3=[$3], $f4=[$4], $e=[0]}, {a=[$0], b=[$1], $f2=[$2], $f3=[$3], $f4=[null], $e=[1]}]) + +- FlinkLogicalExpand(projects=[a, b, $f2, $f3, $f4, $e]) +- FlinkLogicalCalc(select=[a, b, IS TRUE(<>(b, 2)) AS $f2, IS TRUE(<>(b, 5)) AS $f3, MOD(HASH_CODE(b), 1024) AS $f4]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -82,12 +82,12 @@ FlinkLogicalJoin(condition=[=($1, $4)], joinType=[inner]) : +- FlinkLogicalAggregate(group=[{0}], agg#0=[$SUM0($2)], agg#1=[$SUM0($3)]) : +- FlinkLogicalAggregate(group=[{2, 3}], agg#0=[$SUM0($1) FILTER $4], agg#1=[COUNT(DISTINCT $0) FILTER $5]) : +- FlinkLogicalCalc(select=[a, $f1, $f2, $f3, =($e, 1) AS $g_1, =($e, 0) AS $g_0]) -: +- FlinkLogicalExpand(projects=[{a=[$0], $f1=[$1], $f2=[$2], $f3=[$3], $e=[0]}, {a=[$0], $f1=[$1], $f2=[$2], $f3=[null], $e=[1]}]) +: +- FlinkLogicalExpand(projects=[a, $f1, $f2, $f3, $e]) : +- FlinkLogicalCalc(select=[a, $f1, $f2, MOD(HASH_CODE(a), 1024) AS $f3]) : +- FlinkLogicalAggregate(group=[{0}], agg#0=[$SUM0($2)], agg#1=[SUM($3)]) : +- FlinkLogicalAggregate(group=[{0, 2}], agg#0=[COUNT(DISTINCT $1) FILTER $3], agg#1=[SUM($1) FILTER $4]) : +- FlinkLogicalCalc(select=[a, b, $f2, =($e, 0) AS $g_0, =($e, 1) AS $g_1]) -: +- FlinkLogicalExpand(projects=[{a=[$0], b=[$1], $f2=[$2], $e=[0]}, {a=[$0], b=[$1], $f2=[null], $e=[1]}]) +: +- FlinkLogicalExpand(projects=[a, b, $f2, $e]) : +- FlinkLogicalCalc(select=[a, b, MOD(HASH_CODE(b), 1024) AS $f2]) : +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) @@ -130,7 +130,7 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT(DISTINCT $0)], EXPR$1=[SUM(DISTINCT $ FlinkLogicalAggregate(group=[{}], agg#0=[$SUM0($2)], agg#1=[SUM($3)]) +- FlinkLogicalAggregate(group=[{2, 3}], agg#0=[COUNT(DISTINCT $0) FILTER $4], agg#1=[SUM(DISTINCT $1) FILTER $5]) +- FlinkLogicalCalc(select=[a, b, $f2, $f3, =($e, 1) AS $g_1, =($e, 2) AS $g_2]) - +- FlinkLogicalExpand(projects=[{a=[$0], b=[$1], $f2=[$2], $f3=[null], $e=[1]}, {a=[$0], b=[$1], $f2=[null], $f3=[$3], $e=[2]}]) + +- FlinkLogicalExpand(projects=[a, b, $f2, $f3, $e]) +- FlinkLogicalCalc(select=[a, b, MOD(HASH_CODE(a), 1024) AS $f2, MOD(HASH_CODE(b), 1024) AS $f3]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -162,32 +162,10 @@ LogicalAggregate(group=[{0}], EXPR$1=[MIN($1)], EXPR$2=[MAX($1)], EXPR$3=[SUM($1 FlinkLogicalAggregate(group=[{0}], agg#0=[MIN($3)], agg#1=[MAX($4)], agg#2=[SUM($5)], agg#3=[$SUM0($6)], agg#4=[$SUM0($7)]) +- FlinkLogicalAggregate(group=[{2, 3, 4}], agg#0=[MIN($1) FILTER $5], agg#1=[MAX($1) FILTER $5], agg#2=[SUM($1) FILTER $6], agg#3=[COUNT() FILTER $6], agg#4=[COUNT(DISTINCT $0) FILTER $7]) +- FlinkLogicalCalc(select=[a, b, c, $f3, $f4, =($e, 2) AS $g_2, =($e, 3) AS $g_3, =($e, 1) AS $g_1]) - +- FlinkLogicalExpand(projects=[{a=[$0], b=[$1], c=[$2], $f3=[$3], $f4=[null], $e=[1]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[$4], $e=[2]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[null], $e=[3]}]) + +- FlinkLogicalExpand(projects=[a, b, c, $f3, $f4, $e]) +- FlinkLogicalCalc(select=[a, b, c, MOD(HASH_CODE(a), 1024) AS $f3, MOD(HASH_CODE(b), 1024) AS $f4]) +- FlinkLogicalAggregate(group=[{0}], b=[AVG($1)], c=[MAX($2)]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) -]]> - - - - - - - - - - - @@ -232,7 +210,7 @@ FlinkLogicalCalc(select=[a, EXPR$1, CASE(=($f3, 0), null:INTEGER, EXPR$2) AS EXP +- FlinkLogicalAggregate(group=[{0}], agg#0=[$SUM0($3)], agg#1=[$SUM0($4)], agg#2=[$SUM0($5)], agg#3=[MAX($6)], agg#4=[MIN($7)], agg#5=[$SUM0($8)]) +- FlinkLogicalAggregate(group=[{0, 3, 4}], agg#0=[COUNT(DISTINCT $2) FILTER $5], agg#1=[$SUM0($1) FILTER $6], agg#2=[COUNT($1) FILTER $6], agg#3=[MAX($1) FILTER $7], agg#4=[MIN($1) FILTER $7], agg#5=[COUNT() FILTER $6]) +- FlinkLogicalCalc(select=[a, b, c, $f3, $f4, =($e, 2) AS $g_2, =($e, 3) AS $g_3, =($e, 1) AS $g_1]) - +- FlinkLogicalExpand(projects=[{a=[$0], b=[$1], c=[$2], $f3=[$3], $f4=[null], $e=[1]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[$4], $e=[2]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[null], $e=[3]}]) + +- FlinkLogicalExpand(projects=[a, b, c, $f3, $f4, $e]) +- FlinkLogicalCalc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, MOD(HASH_CODE(c), 1024) AS $f4]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -255,7 +233,7 @@ FlinkLogicalCalc(select=[a, $f1, $f2, CAST(IF(=($f4, 0:BIGINT), null:INTEGER, /( +- FlinkLogicalAggregate(group=[{0}], agg#0=[$SUM0($2)], agg#1=[SUM($3)], agg#2=[$SUM0($4)], agg#3=[$SUM0($5)]) +- FlinkLogicalAggregate(group=[{0, 2}], agg#0=[COUNT(DISTINCT $1) FILTER $3], agg#1=[SUM($1) FILTER $4], agg#2=[$SUM0($1) FILTER $4], agg#3=[COUNT($1) FILTER $4]) +- FlinkLogicalCalc(select=[a, b, $f2, =($e, 0) AS $g_0, =($e, 1) AS $g_1]) - +- FlinkLogicalExpand(projects=[{a=[$0], b=[$1], $f2=[$2], $e=[0]}, {a=[$0], b=[$1], $f2=[null], $e=[1]}]) + +- FlinkLogicalExpand(projects=[a, b, $f2, $e]) +- FlinkLogicalCalc(select=[a, b, MOD(HASH_CODE(b), 1024) AS $f2]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -318,27 +296,9 @@ LogicalAggregate(group=[{0}], EXPR$1=[FIRST_VALUE($1)], EXPR$2=[COUNT(DISTINCT $ FlinkLogicalAggregate(group=[{0}], agg#0=[FIRST_VALUE($3)], agg#1=[$SUM0($4)]) +- FlinkLogicalAggregate(group=[{0, 3, 4}], agg#0=[FIRST_VALUE($2) FILTER $5], agg#1=[COUNT(DISTINCT $1) FILTER $6]) +- FlinkLogicalCalc(select=[a, b, c, $f3, $f4, =($e, 2) AS $g_2, =($e, 1) AS $g_1]) - +- FlinkLogicalExpand(projects=[{a=[$0], b=[$1], c=[$2], $f3=[$3], $f4=[null], $e=[1]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[$4], $e=[2]}]) + +- FlinkLogicalExpand(projects=[a, b, c, $f3, $f4, $e]) +- FlinkLogicalCalc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, MOD(HASH_CODE(c), 1024) AS $f4]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) -]]> - - - - - - - - - - - @@ -358,7 +318,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[LAST_VALUE($1)], EXPR$2=[COUNT(DISTINCT $2 FlinkLogicalAggregate(group=[{0}], agg#0=[LAST_VALUE($3)], agg#1=[$SUM0($4)]) +- FlinkLogicalAggregate(group=[{0, 3, 4}], agg#0=[LAST_VALUE($2) FILTER $5], agg#1=[COUNT(DISTINCT $1) FILTER $6]) +- FlinkLogicalCalc(select=[a, b, c, $f3, $f4, =($e, 2) AS $g_2, =($e, 1) AS $g_1]) - +- FlinkLogicalExpand(projects=[{a=[$0], b=[$1], c=[$2], $f3=[$3], $f4=[null], $e=[1]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[$4], $e=[2]}]) + +- FlinkLogicalExpand(projects=[a, b, c, $f3, $f4, $e]) +- FlinkLogicalCalc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, MOD(HASH_CODE(c), 1024) AS $f4]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -383,21 +343,25 @@ FlinkLogicalAggregate(group=[{}], EXPR$0=[MIN($0)]) ]]> - + - + @@ -420,7 +384,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1)], EXPR$2=[MAX($2)]) FlinkLogicalAggregate(group=[{0}], agg#0=[$SUM0($3)], agg#1=[MAX($4)]) +- FlinkLogicalAggregate(group=[{0, 3, 4}], agg#0=[COUNT(DISTINCT $1) FILTER $5], agg#1=[MAX($2) FILTER $6]) +- FlinkLogicalCalc(select=[a, b, c, $f3, $f4, =($e, 1) AS $g_1, =($e, 2) AS $g_2]) - +- FlinkLogicalExpand(projects=[{a=[$0], b=[$1], c=[$2], $f3=[$3], $f4=[null], $e=[1]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[$4], $e=[2]}]) + +- FlinkLogicalExpand(projects=[a, b, c, $f3, $f4, $e]) +- FlinkLogicalCalc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, MOD(HASH_CODE(c), 1024) AS $f4]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalHashAggRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalHashAggRuleTest.xml index bda4ba3766..58ee711822 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalHashAggRuleTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalHashAggRuleTest.xml @@ -34,7 +34,7 @@ Calc(select=[EXPR$0]) +- HashAggregate(isMerge=[true], groupBy=[a, c, $e], select=[a, c, $e, Final_AVG(sum$0, count$1) AS EXPR$0]) +- Exchange(distribution=[hash[a, c, $e]]) +- LocalHashAggregate(groupBy=[a, c, $e], select=[a, c, $e, Partial_AVG(d) AS (sum$0, count$1)]) - +- Expand(projects=[{a=[$0], c=[$1], d=[$2], $e=[0]}, {a=[$0], c=[null], d=[$2], $e=[1]}, {a=[null], c=[$1], d=[$2], $e=[2]}, {a=[null], c=[null], d=[$2], $e=[3]}], projects=[{a, c, d, 0 AS $e}, {a, null AS c, d, 1 AS $e}, {null AS a, c, d, 2 AS $e}, {null AS a, null AS c, d, 3 AS $e}]) + +- Expand(projects=[a, c, d, $e], projects=[{a, c, d, 0 AS $e}, {a, null AS c, d, 1 AS $e}, {null AS a, c, d, 2 AS $e}, {null AS a, null AS c, d, 3 AS $e}]) +- Calc(select=[a, c, d]) +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) ]]> @@ -58,7 +58,7 @@ Calc(select=[EXPR$0]) +- HashAggregate(isMerge=[true], groupBy=[b, c, $e], select=[b, c, $e, Final_COUNT(count$0) AS EXPR$0]) +- Exchange(distribution=[hash[b, c, $e]]) +- LocalHashAggregate(groupBy=[b, c, $e], select=[b, c, $e, Partial_COUNT(a) AS count$0]) - +- Expand(projects=[{b=[$0], c=[$1], a=[$2], $e=[0]}, {b=[$0], c=[null], a=[$2], $e=[1]}, {b=[null], c=[$1], a=[$2], $e=[2]}, {b=[null], c=[null], a=[$2], $e=[3]}], projects=[{b, c, a, 0 AS $e}, {b, null AS c, a, 1 AS $e}, {null AS b, c, a, 2 AS $e}, {null AS b, null AS c, a, 3 AS $e}]) + +- Expand(projects=[b, c, a, $e], projects=[{b, c, a, 0 AS $e}, {b, null AS c, a, 1 AS $e}, {null AS b, c, a, 2 AS $e}, {null AS b, null AS c, a, 3 AS $e}]) +- Calc(select=[b, c, a]) +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) ]]> @@ -81,7 +81,7 @@ LogicalProject(EXPR$0=[$3]) Calc(select=[EXPR$0]) +- HashAggregate(isMerge=[false], groupBy=[b, c, d, $e], select=[b, c, d, $e, AVG(a) AS EXPR$0]) +- Exchange(distribution=[hash[b, c, d, $e]]) - +- Expand(projects=[{a=[$0], b=[$1], c=[$2], d=[null], $e=[1]}, {a=[$0], b=[$1], c=[null], d=[$3], $e=[2]}], projects=[{a, b, c, null AS d, 1 AS $e}, {a, b, null AS c, d, 2 AS $e}]) + +- Expand(projects=[a, b, c, d, $e], projects=[{a, b, c, null AS d, 1 AS $e}, {a, b, null AS c, d, 2 AS $e}]) +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) ]]> @@ -103,7 +103,7 @@ LogicalProject(EXPR$0=[$3]) Calc(select=[EXPR$0]) +- HashAggregate(isMerge=[false], groupBy=[b, c, d, $e], select=[b, c, d, $e, COUNT(*) AS EXPR$0]) +- Exchange(distribution=[hash[b, c, d, $e]]) - +- Expand(projects=[{b=[$0], c=[$1], d=[null], $e=[1]}, {b=[$0], c=[null], d=[$2], $e=[2]}], projects=[{b, c, null AS d, 1 AS $e}, {b, null AS c, d, 2 AS $e}]) + +- Expand(projects=[b, c, d, $e], projects=[{b, c, null AS d, 1 AS $e}, {b, null AS c, d, 2 AS $e}]) +- Calc(select=[b, c, d]) +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) ]]> @@ -127,7 +127,7 @@ Calc(select=[EXPR$0]) +- HashAggregate(isMerge=[true], groupBy=[b, c, $e], select=[b, c, $e, Final_AVG(sum$0, count$1) AS EXPR$0]) +- Exchange(distribution=[hash[b, c, $e]]) +- LocalHashAggregate(groupBy=[b, c, $e], select=[b, c, $e, Partial_AVG(a) AS (sum$0, count$1)]) - +- Expand(projects=[{b=[$0], c=[$1], a=[$2], $e=[0]}, {b=[$0], c=[null], a=[$2], $e=[1]}, {b=[null], c=[null], a=[$2], $e=[3]}], projects=[{b, c, a, 0 AS $e}, {b, null AS c, a, 1 AS $e}, {null AS b, null AS c, a, 3 AS $e}]) + +- Expand(projects=[b, c, a, $e], projects=[{b, c, a, 0 AS $e}, {b, null AS c, a, 1 AS $e}, {null AS b, null AS c, a, 3 AS $e}]) +- Calc(select=[b, c, a]) +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) ]]> @@ -151,7 +151,7 @@ Calc(select=[EXPR$0]) +- HashAggregate(isMerge=[true], groupBy=[b, c, $e], select=[b, c, $e, Final_COUNT(count1$0) AS EXPR$0]) +- Exchange(distribution=[hash[b, c, $e]]) +- LocalHashAggregate(groupBy=[b, c, $e], select=[b, c, $e, Partial_COUNT(*) AS count1$0]) - +- Expand(projects=[{b=[$0], c=[$1], $e=[0]}, {b=[$0], c=[null], $e=[1]}, {b=[null], c=[null], $e=[3]}], projects=[{b, c, 0 AS $e}, {b, null AS c, 1 AS $e}, {null AS b, null AS c, 3 AS $e}]) + +- Expand(projects=[b, c, $e], projects=[{b, c, 0 AS $e}, {b, null AS c, 1 AS $e}, {null AS b, null AS c, 3 AS $e}]) +- Calc(select=[b, c]) +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalSortAggRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalSortAggRuleTest.xml index 0150db1de6..55f57b94f4 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalSortAggRuleTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalSortAggRuleTest.xml @@ -36,7 +36,7 @@ Calc(select=[EXPR$0]) +- Exchange(distribution=[hash[a, c, $e]]) +- LocalSortAggregate(groupBy=[a, c, $e], select=[a, c, $e, Partial_AVG(d) AS (sum$0, count$1)]) +- Sort(orderBy=[a ASC, c ASC, $e ASC]) - +- Expand(projects=[{a=[$0], c=[$1], d=[$2], $e=[0]}, {a=[$0], c=[null], d=[$2], $e=[1]}, {a=[null], c=[$1], d=[$2], $e=[2]}, {a=[null], c=[null], d=[$2], $e=[3]}], projects=[{a, c, d, 0 AS $e}, {a, null AS c, d, 1 AS $e}, {null AS a, c, d, 2 AS $e}, {null AS a, null AS c, d, 3 AS $e}]) + +- Expand(projects=[a, c, d, $e], projects=[{a, c, d, 0 AS $e}, {a, null AS c, d, 1 AS $e}, {null AS a, c, d, 2 AS $e}, {null AS a, null AS c, d, 3 AS $e}]) +- Calc(select=[a, c, d]) +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) ]]> @@ -62,7 +62,7 @@ Calc(select=[EXPR$0]) +- Exchange(distribution=[hash[b, c, $e]]) +- LocalSortAggregate(groupBy=[b, c, $e], select=[b, c, $e, Partial_COUNT(a) AS count$0]) +- Sort(orderBy=[b ASC, c ASC, $e ASC]) - +- Expand(projects=[{b=[$0], c=[$1], a=[$2], $e=[0]}, {b=[$0], c=[null], a=[$2], $e=[1]}, {b=[null], c=[$1], a=[$2], $e=[2]}, {b=[null], c=[null], a=[$2], $e=[3]}], projects=[{b, c, a, 0 AS $e}, {b, null AS c, a, 1 AS $e}, {null AS b, c, a, 2 AS $e}, {null AS b, null AS c, a, 3 AS $e}]) + +- Expand(projects=[b, c, a, $e], projects=[{b, c, a, 0 AS $e}, {b, null AS c, a, 1 AS $e}, {null AS b, c, a, 2 AS $e}, {null AS b, null AS c, a, 3 AS $e}]) +- Calc(select=[b, c, a]) +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) ]]> @@ -86,7 +86,7 @@ Calc(select=[EXPR$0]) +- SortAggregate(isMerge=[false], groupBy=[a, b, $e], select=[a, b, $e, weightedAvg(d, $f3) AS EXPR$0]) +- Sort(orderBy=[a ASC, b ASC, $e ASC]) +- Exchange(distribution=[hash[a, b, $e]]) - +- Expand(projects=[{a=[$0], b=[$1], d=[$2], $f3=[$3], $e=[0]}, {a=[$0], b=[null], d=[$2], $f3=[$3], $e=[1]}, {a=[null], b=[$1], d=[$2], $f3=[$3], $e=[2]}, {a=[null], b=[null], d=[$2], $f3=[$3], $e=[3]}], projects=[{a, b, d, $f3, 0 AS $e}, {a, null AS b, d, $f3, 1 AS $e}, {null AS a, b, d, $f3, 2 AS $e}, {null AS a, null AS b, d, $f3, 3 AS $e}]) + +- Expand(projects=[a, b, d, $f3, $e], projects=[{a, b, d, $f3, 0 AS $e}, {a, null AS b, d, $f3, 1 AS $e}, {null AS a, b, d, $f3, 2 AS $e}, {null AS a, null AS b, d, $f3, 3 AS $e}]) +- Calc(select=[a, b, d, 1 AS $f3]) +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) ]]> @@ -110,7 +110,7 @@ Calc(select=[EXPR$0]) +- SortAggregate(isMerge=[false], groupBy=[b, c, d, $e], select=[b, c, d, $e, AVG(a) AS EXPR$0]) +- Sort(orderBy=[b ASC, c ASC, d ASC, $e ASC]) +- Exchange(distribution=[hash[b, c, d, $e]]) - +- Expand(projects=[{a=[$0], b=[$1], c=[$2], d=[null], $e=[1]}, {a=[$0], b=[$1], c=[null], d=[$3], $e=[2]}], projects=[{a, b, c, null AS d, 1 AS $e}, {a, b, null AS c, d, 2 AS $e}]) + +- Expand(projects=[a, b, c, d, $e], projects=[{a, b, c, null AS d, 1 AS $e}, {a, b, null AS c, d, 2 AS $e}]) +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) ]]> @@ -133,7 +133,7 @@ Calc(select=[EXPR$0]) +- SortAggregate(isMerge=[false], groupBy=[b, c, d, $e], select=[b, c, d, $e, COUNT(*) AS EXPR$0]) +- Sort(orderBy=[b ASC, c ASC, d ASC, $e ASC]) +- Exchange(distribution=[hash[b, c, d, $e]]) - +- Expand(projects=[{b=[$0], c=[$1], d=[null], $e=[1]}, {b=[$0], c=[null], d=[$2], $e=[2]}], projects=[{b, c, null AS d, 1 AS $e}, {b, null AS c, d, 2 AS $e}]) + +- Expand(projects=[b, c, d, $e], projects=[{b, c, null AS d, 1 AS $e}, {b, null AS c, d, 2 AS $e}]) +- Calc(select=[b, c, d]) +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) ]]> @@ -157,7 +157,7 @@ Calc(select=[EXPR$0]) +- SortAggregate(isMerge=[false], groupBy=[b, c, $e], select=[b, c, $e, weightedAvg(a, $f3) AS EXPR$0]) +- Sort(orderBy=[b ASC, c ASC, $e ASC]) +- Exchange(distribution=[hash[b, c, $e]]) - +- Expand(projects=[{b=[$0], c=[$1], a=[$2], $f3=[$3], $e=[0]}, {b=[$0], c=[null], a=[$2], $f3=[$3], $e=[1]}, {b=[null], c=[null], a=[$2], $f3=[$3], $e=[3]}], projects=[{b, c, a, $f3, 0 AS $e}, {b, null AS c, a, $f3, 1 AS $e}, {null AS b, null AS c, a, $f3, 3 AS $e}]) + +- Expand(projects=[b, c, a, $f3, $e], projects=[{b, c, a, $f3, 0 AS $e}, {b, null AS c, a, $f3, 1 AS $e}, {null AS b, null AS c, a, $f3, 3 AS $e}]) +- Calc(select=[b, c, a, 1 AS $f3]) +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) ]]> @@ -181,7 +181,7 @@ Calc(select=[EXPR$0]) +- SortAggregate(isMerge=[false], groupBy=[b, c, d, $e], select=[b, c, d, $e, weightedAvg(a, $f4) AS EXPR$0]) +- Sort(orderBy=[b ASC, c ASC, d ASC, $e ASC]) +- Exchange(distribution=[hash[b, c, d, $e]]) - +- Expand(projects=[{b=[$0], c=[$1], d=[null], a=[$3], $f4=[$4], $e=[1]}, {b=[$0], c=[null], d=[$2], a=[$3], $f4=[$4], $e=[2]}], projects=[{b, c, null AS d, a, $f4, 1 AS $e}, {b, null AS c, d, a, $f4, 2 AS $e}]) + +- Expand(projects=[b, c, d, a, $f4, $e], projects=[{b, c, null AS d, a, $f4, 1 AS $e}, {b, null AS c, d, a, $f4, 2 AS $e}]) +- Calc(select=[b, c, d, a, 1 AS $f4]) +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) ]]> @@ -207,7 +207,7 @@ Calc(select=[EXPR$0]) +- Exchange(distribution=[hash[b, c, $e]]) +- LocalSortAggregate(groupBy=[b, c, $e], select=[b, c, $e, Partial_AVG(a) AS (sum$0, count$1)]) +- Sort(orderBy=[b ASC, c ASC, $e ASC]) - +- Expand(projects=[{b=[$0], c=[$1], a=[$2], $e=[0]}, {b=[$0], c=[null], a=[$2], $e=[1]}, {b=[null], c=[null], a=[$2], $e=[3]}], projects=[{b, c, a, 0 AS $e}, {b, null AS c, a, 1 AS $e}, {null AS b, null AS c, a, 3 AS $e}]) + +- Expand(projects=[b, c, a, $e], projects=[{b, c, a, 0 AS $e}, {b, null AS c, a, 1 AS $e}, {null AS b, null AS c, a, 3 AS $e}]) +- Calc(select=[b, c, a]) +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) ]]> @@ -233,7 +233,7 @@ Calc(select=[EXPR$0]) +- Exchange(distribution=[hash[b, c, $e]]) +- LocalSortAggregate(groupBy=[b, c, $e], select=[b, c, $e, Partial_COUNT(*) AS count1$0]) +- Sort(orderBy=[b ASC, c ASC, $e ASC]) - +- Expand(projects=[{b=[$0], c=[$1], $e=[0]}, {b=[$0], c=[null], $e=[1]}, {b=[null], c=[null], $e=[3]}], projects=[{b, c, 0 AS $e}, {b, null AS c, 1 AS $e}, {null AS b, null AS c, 3 AS $e}]) + +- Expand(projects=[b, c, $e], projects=[{b, c, 0 AS $e}, {b, null AS c, 1 AS $e}, {null AS b, null AS c, 3 AS $e}]) +- Calc(select=[b, c]) +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.xml index 8325559aa3..b6632274db 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.xml @@ -263,9 +263,9 @@ Calc(select=[a, b, c], where=[>=(a, 0)], updateAsRetraction=[true], accMode=[Acc +- Reused(reference_id=[1]) Calc(select=[a1, b, c AS c2], updateAsRetraction=[true], accMode=[Acc], reuse_id=[3]) -+- Join(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b, a2, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], updateAsRetraction=[true], accMode=[Acc]) - :- Exchange(distribution=[hash[a1]], updateAsRetraction=[true], accMode=[Acc]) - : +- Calc(select=[a AS a1, b], where=[<=(a, 10)], updateAsRetraction=[true], accMode=[Acc]) ++- Join(joinType=[InnerJoin], where=[=(a, a2)], select=[a, b, a2, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], updateAsRetraction=[true], accMode=[Acc]) + :- Exchange(distribution=[hash[a]], updateAsRetraction=[true], accMode=[Acc]) + : +- Calc(select=[a, b], where=[<=(a, 10)], updateAsRetraction=[true], accMode=[Acc]) : +- Reused(reference_id=[1]) +- Exchange(distribution=[hash[a2]], updateAsRetraction=[true], accMode=[Acc]) +- Calc(select=[a AS a2, c], where=[>=(b, 5)], updateAsRetraction=[true], accMode=[Acc]) @@ -749,9 +749,9 @@ Sink(name=[`default_catalog`.`default_database`.`appendSink`], fields=[a1, b, c1 : +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc], reuse_id=[1]) +- Exchange(distribution=[hash[a1]], updateAsRetraction=[true], accMode=[Acc]) +- Calc(select=[a1, b, c AS c2], updateAsRetraction=[true], accMode=[Acc]) - +- Join(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b, a2, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], updateAsRetraction=[true], accMode=[Acc]) - :- Exchange(distribution=[hash[a1]], updateAsRetraction=[true], accMode=[Acc]) - : +- Calc(select=[a AS a1, b], where=[<=(a, 10)], updateAsRetraction=[true], accMode=[Acc]) + +- Join(joinType=[InnerJoin], where=[=(a, a2)], select=[a, b, a2, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], updateAsRetraction=[true], accMode=[Acc]) + :- Exchange(distribution=[hash[a]], updateAsRetraction=[true], accMode=[Acc]) + : +- Calc(select=[a, b], where=[<=(a, 10)], updateAsRetraction=[true], accMode=[Acc]) : +- Reused(reference_id=[1]) +- Exchange(distribution=[hash[a2]], updateAsRetraction=[true], accMode=[Acc]) +- Calc(select=[a AS a2, c], where=[AND(>=(a, 0), >=(b, 5))], updateAsRetraction=[true], accMode=[Acc]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SinkTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SinkTest.xml index 6bf68913ab..f26937c098 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SinkTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SinkTest.xml @@ -271,9 +271,9 @@ Sink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[a1, b, c1 : +- DataStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc], reuse_id=[1]) +- Exchange(distribution=[hash[a1]], updateAsRetraction=[true], accMode=[Acc]) +- Calc(select=[a1, b, c AS c2], updateAsRetraction=[true], accMode=[Acc]) - +- Join(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b, a2, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], updateAsRetraction=[true], accMode=[Acc]) - :- Exchange(distribution=[hash[a1]], updateAsRetraction=[true], accMode=[Acc]) - : +- Calc(select=[a AS a1, b], where=[<=(a, 10)], updateAsRetraction=[true], accMode=[Acc]) + +- Join(joinType=[InnerJoin], where=[=(a, a2)], select=[a, b, a2, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], updateAsRetraction=[true], accMode=[Acc]) + :- Exchange(distribution=[hash[a]], updateAsRetraction=[true], accMode=[Acc]) + : +- Calc(select=[a, b], where=[<=(a, 10)], updateAsRetraction=[true], accMode=[Acc]) : +- Reused(reference_id=[1]) +- Exchange(distribution=[hash[a2]], updateAsRetraction=[true], accMode=[Acc]) +- Calc(select=[a AS a2, c], where=[AND(>=(a, 0), >=(b, 5))], updateAsRetraction=[true], accMode=[Acc]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.xml index 4884c5fbbd..9d9f23e9bd 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.xml @@ -101,7 +101,7 @@ GroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0_RETRACT($ +- GroupAggregate(groupBy=[a, $f4], partialFinalType=[PARTIAL], select=[a, $f4, COUNT(DISTINCT b) FILTER $g_0 AS $f2, SUM(b) FILTER $g_1 AS $f3, SUM(b) FILTER $g_10 AS $f4_0]) +- Exchange(distribution=[hash[a, $f4]]) +- Calc(select=[a, b, $f2, $f3, $f4, AND(=($e, 0), $f2) AS $g_0, AND(=($e, 1), $f3) AS $g_1, AND(=($e, 1), $f2) AS $g_10]) - +- Expand(projects=[{a=[$0], b=[$1], $f2=[$2], $f3=[$3], $f4=[$4], $e=[0]}, {a=[$0], b=[$1], $f2=[$2], $f3=[$3], $f4=[null], $e=[1]}]) + +- Expand(projects=[a, b, $f2, $f3, $f4, $e]) +- Calc(select=[a, b, IS TRUE(<>(b, 2:BIGINT)) AS $f2, IS TRUE(<>(b, 5:BIGINT)) AS $f3, MOD(HASH_CODE(b), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) @@ -136,7 +136,7 @@ GlobalGroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0_RET +- Exchange(distribution=[hash[a, $f4]]) +- LocalGroupAggregate(groupBy=[a, $f4], partialFinalType=[PARTIAL], select=[a, $f4, COUNT(distinct$0 b) FILTER $g_0 AS count$0, SUM(b) FILTER $g_1 AS sum$1, SUM(b) FILTER $g_10 AS sum$2, DISTINCT(b) AS distinct$0]) +- Calc(select=[a, b, $f2, $f3, $f4, AND(=($e, 0), $f2) AS $g_0, AND(=($e, 1), $f3) AS $g_1, AND(=($e, 1), $f2) AS $g_10]) - +- Expand(projects=[{a=[$0], b=[$1], $f2=[$2], $f3=[$3], $f4=[$4], $e=[0]}, {a=[$0], b=[$1], $f2=[$2], $f3=[$3], $f4=[null], $e=[1]}]) + +- Expand(projects=[a, b, $f2, $f3, $f4, $e]) +- Calc(select=[a, b, IS TRUE(<>(b, 2:BIGINT)) AS $f2, IS TRUE(<>(b, 5:BIGINT)) AS $f3, MOD(HASH_CODE(b), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) @@ -388,7 +388,7 @@ GroupAggregate(groupBy=[c], partialFinalType=[FINAL], select=[c, MIN_RETRACT($f3 +- GroupAggregate(groupBy=[c, $f3, $f4], partialFinalType=[PARTIAL], select=[c, $f3, $f4, MIN_RETRACT(b) FILTER $g_2 AS $f3_0, MAX_RETRACT(b) FILTER $g_2 AS $f4_0, SUM_RETRACT(b) FILTER $g_3 AS $f5, COUNT_RETRACT(*) FILTER $g_3 AS $f6, COUNT_RETRACT(DISTINCT a) FILTER $g_1 AS $f7], updateAsRetraction=[true], accMode=[AccRetract]) +- Exchange(distribution=[hash[c, $f3, $f4]], updateAsRetraction=[true], accMode=[AccRetract]) +- Calc(select=[a, b, c, $f3, $f4, =($e, 2) AS $g_2, =($e, 3) AS $g_3, =($e, 1) AS $g_1], updateAsRetraction=[true], accMode=[AccRetract]) - +- Expand(projects=[{a=[$0], b=[$1], c=[$2], $f3=[$3], $f4=[null], $e=[1]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[$4], $e=[2]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[null], $e=[3]}], updateAsRetraction=[true], accMode=[AccRetract]) + +- Expand(projects=[a, b, c, $f3, $f4, $e], updateAsRetraction=[true], accMode=[AccRetract]) +- Calc(select=[a, b, c, MOD(HASH_CODE(a), 1024) AS $f3, MOD(HASH_CODE(b), 1024) AS $f4], updateAsRetraction=[true], accMode=[AccRetract]) +- GroupAggregate(groupBy=[a], select=[a, AVG(b) AS b, MAX(c) AS c], updateAsRetraction=[true], accMode=[AccRetract]) +- Exchange(distribution=[hash[a]], updateAsRetraction=[true], accMode=[Acc]) @@ -427,7 +427,7 @@ GlobalGroupAggregate(groupBy=[c], partialFinalType=[FINAL], select=[c, MIN_RETRA +- Exchange(distribution=[hash[c, $f3, $f4]], updateAsRetraction=[true], accMode=[Acc]) +- LocalGroupAggregate(groupBy=[c, $f3, $f4], partialFinalType=[PARTIAL], select=[c, $f3, $f4, MIN_RETRACT(b) FILTER $g_2 AS min$0, MAX_RETRACT(b) FILTER $g_2 AS max$1, SUM_RETRACT(b) FILTER $g_3 AS (sum$2, count$3), COUNT_RETRACT(*) FILTER $g_3 AS count1$4, COUNT_RETRACT(distinct$0 a) FILTER $g_1 AS count$5, COUNT_RETRACT(*) AS count1$6, DISTINCT(a) AS distinct$0], updateAsRetraction=[true], accMode=[Acc]) +- Calc(select=[a, b, c, $f3, $f4, =($e, 2) AS $g_2, =($e, 3) AS $g_3, =($e, 1) AS $g_1], updateAsRetraction=[true], accMode=[AccRetract]) - +- Expand(projects=[{a=[$0], b=[$1], c=[$2], $f3=[$3], $f4=[null], $e=[1]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[$4], $e=[2]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[null], $e=[3]}], updateAsRetraction=[true], accMode=[AccRetract]) + +- Expand(projects=[a, b, c, $f3, $f4, $e], updateAsRetraction=[true], accMode=[AccRetract]) +- Calc(select=[a, b, c, MOD(HASH_CODE(a), 1024) AS $f3, MOD(HASH_CODE(b), 1024) AS $f4], updateAsRetraction=[true], accMode=[AccRetract]) +- GlobalGroupAggregate(groupBy=[a], select=[a, AVG((sum$0, count$1)) AS b, MAX(max$2) AS c], updateAsRetraction=[true], accMode=[AccRetract]) +- Exchange(distribution=[hash[a]], updateAsRetraction=[true], accMode=[Acc]) @@ -498,7 +498,7 @@ GroupAggregate(partialFinalType=[FINAL], select=[$SUM0_RETRACT($f2_0) AS $f0, SU +- GroupAggregate(groupBy=[$f2, $f3], partialFinalType=[PARTIAL], select=[$f2, $f3, COUNT(DISTINCT a) FILTER $g_1 AS $f2_0, SUM(DISTINCT b) FILTER $g_2 AS $f3_0]) +- Exchange(distribution=[hash[$f2, $f3]]) +- Calc(select=[a, b, $f2, $f3, =($e, 1) AS $g_1, =($e, 2) AS $g_2]) - +- Expand(projects=[{a=[$0], b=[$1], $f2=[$2], $f3=[null], $e=[1]}, {a=[$0], b=[$1], $f2=[null], $f3=[$3], $e=[2]}]) + +- Expand(projects=[a, b, $f2, $f3, $e]) +- Calc(select=[a, b, MOD(HASH_CODE(a), 1024) AS $f2, MOD(HASH_CODE(b), 1024) AS $f3]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) @@ -525,7 +525,7 @@ GlobalGroupAggregate(partialFinalType=[FINAL], select=[$SUM0_RETRACT(sum$0) AS $ +- Exchange(distribution=[hash[$f2, $f3]]) +- LocalGroupAggregate(groupBy=[$f2, $f3], partialFinalType=[PARTIAL], select=[$f2, $f3, COUNT(distinct$0 a) FILTER $g_1 AS count$0, SUM(distinct$1 b) FILTER $g_2 AS sum$1, DISTINCT(a) AS distinct$0, DISTINCT(b) AS distinct$1]) +- Calc(select=[a, b, $f2, $f3, =($e, 1) AS $g_1, =($e, 2) AS $g_2]) - +- Expand(projects=[{a=[$0], b=[$1], $f2=[$2], $f3=[null], $e=[1]}, {a=[$0], b=[$1], $f2=[null], $f3=[$3], $e=[2]}]) + +- Expand(projects=[a, b, $f2, $f3, $e]) +- Calc(select=[a, b, MOD(HASH_CODE(a), 1024) AS $f2, MOD(HASH_CODE(b), 1024) AS $f3]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) @@ -626,7 +626,7 @@ GroupAggregate(groupBy=[c], partialFinalType=[FINAL], select=[c, MIN_RETRACT($f3 +- GroupAggregate(groupBy=[c, $f3, $f4], partialFinalType=[PARTIAL], select=[c, $f3, $f4, MIN_RETRACT(b) FILTER $g_2 AS $f3_0, MAX_RETRACT(b) FILTER $g_2 AS $f4_0, SUM_RETRACT(b) FILTER $g_3 AS $f5, COUNT_RETRACT(*) FILTER $g_3 AS $f6, COUNT_RETRACT(DISTINCT a) FILTER $g_1 AS $f7]) +- Exchange(distribution=[hash[c, $f3, $f4]]) +- Calc(select=[a, b, c, $f3, $f4, =($e, 2) AS $g_2, =($e, 3) AS $g_3, =($e, 1) AS $g_1]) - +- Expand(projects=[{a=[$0], b=[$1], c=[$2], $f3=[$3], $f4=[null], $e=[1]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[$4], $e=[2]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[null], $e=[3]}]) + +- Expand(projects=[a, b, c, $f3, $f4, $e]) +- Calc(select=[a, b, c, MOD(HASH_CODE(a), 1024) AS $f3, MOD(HASH_CODE(b), 1024) AS $f4]) +- GroupAggregate(groupBy=[a], select=[a, AVG(b) AS b, MAX(c) AS c]) +- Exchange(distribution=[hash[a]]) @@ -665,129 +665,13 @@ GlobalGroupAggregate(groupBy=[c], partialFinalType=[FINAL], select=[c, MIN_RETRA +- Exchange(distribution=[hash[c, $f3, $f4]]) +- LocalGroupAggregate(groupBy=[c, $f3, $f4], partialFinalType=[PARTIAL], select=[c, $f3, $f4, MIN_RETRACT(b) FILTER $g_2 AS min$0, MAX_RETRACT(b) FILTER $g_2 AS max$1, SUM_RETRACT(b) FILTER $g_3 AS (sum$2, count$3), COUNT_RETRACT(*) FILTER $g_3 AS count1$4, COUNT_RETRACT(distinct$0 a) FILTER $g_1 AS count$5, COUNT_RETRACT(*) AS count1$6, DISTINCT(a) AS distinct$0]) +- Calc(select=[a, b, c, $f3, $f4, =($e, 2) AS $g_2, =($e, 3) AS $g_3, =($e, 1) AS $g_1]) - +- Expand(projects=[{a=[$0], b=[$1], c=[$2], $f3=[$3], $f4=[null], $e=[1]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[$4], $e=[2]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[null], $e=[3]}]) + +- Expand(projects=[a, b, c, $f3, $f4, $e]) +- Calc(select=[a, b, c, MOD(HASH_CODE(a), 1024) AS $f3, MOD(HASH_CODE(b), 1024) AS $f4]) +- GlobalGroupAggregate(groupBy=[a], select=[a, AVG((sum$0, count$1)) AS b, MAX(max$2) AS c]) +- Exchange(distribution=[hash[a]]) +- LocalGroupAggregate(groupBy=[a], select=[a, AVG(b) AS (sum$0, count$1), MAX(c) AS max$2]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) -]]> - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - @@ -956,7 +840,7 @@ Calc(select=[a, EXPR$1, CASE(=($f3, 0), null:BIGINT, EXPR$2) AS EXPR$2, /(CASE(= +- GroupAggregate(groupBy=[a, $f3, $f4], partialFinalType=[PARTIAL], select=[a, $f3, $f4, COUNT(DISTINCT c) FILTER $g_2 AS $f3_0, $SUM0(b) FILTER $g_3 AS $f4_0, COUNT(b) FILTER $g_3 AS $f5, MAX(b) FILTER $g_1 AS $f6, MIN(b) FILTER $g_1 AS $f7, COUNT(*) FILTER $g_3 AS $f8]) +- Exchange(distribution=[hash[a, $f3, $f4]]) +- Calc(select=[a, b, c, $f3, $f4, =($e, 2) AS $g_2, =($e, 3) AS $g_3, =($e, 1) AS $g_1]) - +- Expand(projects=[{a=[$0], b=[$1], c=[$2], $f3=[$3], $f4=[null], $e=[1]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[$4], $e=[2]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[null], $e=[3]}]) + +- Expand(projects=[a, b, c, $f3, $f4, $e]) +- Calc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, MOD(HASH_CODE(c), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) @@ -988,7 +872,7 @@ Calc(select=[a, EXPR$1, CASE(=($f3, 0), null:BIGINT, EXPR$2) AS EXPR$2, /(CASE(= +- Exchange(distribution=[hash[a, $f3, $f4]]) +- LocalGroupAggregate(groupBy=[a, $f3, $f4], partialFinalType=[PARTIAL], select=[a, $f3, $f4, COUNT(distinct$0 c) FILTER $g_2 AS count$0, $SUM0(b) FILTER $g_3 AS sum$1, COUNT(b) FILTER $g_3 AS count$2, MAX(b) FILTER $g_1 AS max$3, MIN(b) FILTER $g_1 AS min$4, COUNT(*) FILTER $g_3 AS count1$5, DISTINCT(c) AS distinct$0]) +- Calc(select=[a, b, c, $f3, $f4, =($e, 2) AS $g_2, =($e, 3) AS $g_3, =($e, 1) AS $g_1]) - +- Expand(projects=[{a=[$0], b=[$1], c=[$2], $f3=[$3], $f4=[null], $e=[1]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[$4], $e=[2]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[null], $e=[3]}]) + +- Expand(projects=[a, b, c, $f3, $f4, $e]) +- Calc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, MOD(HASH_CODE(c), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) @@ -1035,6 +919,29 @@ GlobalGroupAggregate(groupBy=[a], select=[a, COUNT(distinct$0 count$0) AS EXPR$1 +- Calc(select=[a, b]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +]]> + + + + + + + + + + + @@ -1057,7 +964,7 @@ Calc(select=[a, $f1, $f2, IF(=($f4, 0:BIGINT), null:BIGINT, /($f3, $f4)) AS $f3] +- GroupAggregate(groupBy=[a, $f2], partialFinalType=[PARTIAL], select=[a, $f2, COUNT(DISTINCT b) FILTER $g_0 AS $f2_0, SUM(b) FILTER $g_1 AS $f3, $SUM0(b) FILTER $g_1 AS $f4, COUNT(b) FILTER $g_1 AS $f5]) +- Exchange(distribution=[hash[a, $f2]]) +- Calc(select=[a, b, $f2, =($e, 0) AS $g_0, =($e, 1) AS $g_1]) - +- Expand(projects=[{a=[$0], b=[$1], $f2=[$2], $e=[0]}, {a=[$0], b=[$1], $f2=[null], $e=[1]}]) + +- Expand(projects=[a, b, $f2, $e]) +- Calc(select=[a, b, MOD(HASH_CODE(b), 1024) AS $f2]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) @@ -1085,7 +992,7 @@ Calc(select=[a, $f1, $f2, IF(=($f4, 0:BIGINT), null:BIGINT, /($f3, $f4)) AS $f3] +- Exchange(distribution=[hash[a, $f2]]) +- LocalGroupAggregate(groupBy=[a, $f2], partialFinalType=[PARTIAL], select=[a, $f2, COUNT(distinct$0 b) FILTER $g_0 AS count$0, SUM(b) FILTER $g_1 AS sum$1, $SUM0(b) FILTER $g_1 AS sum$2, COUNT(b) FILTER $g_1 AS count$3, DISTINCT(b) AS distinct$0]) +- Calc(select=[a, b, $f2, =($e, 0) AS $g_0, =($e, 1) AS $g_1]) - +- Expand(projects=[{a=[$0], b=[$1], $f2=[$2], $e=[0]}, {a=[$0], b=[$1], $f2=[null], $e=[1]}]) + +- Expand(projects=[a, b, $f2, $e]) +- Calc(select=[a, b, MOD(HASH_CODE(b), 1024) AS $f2]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) @@ -1253,7 +1160,7 @@ GroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0_RETRACT($ +- GroupAggregate(groupBy=[a, $f3], partialFinalType=[PARTIAL], select=[a, $f3, COUNT_RETRACT(DISTINCT b) FILTER $g_0 AS $f2, COUNT_RETRACT(*) FILTER $g_1 AS $f3_0], updateAsRetraction=[true], accMode=[AccRetract]) +- Exchange(distribution=[hash[a, $f3]], updateAsRetraction=[true], accMode=[AccRetract]) +- Calc(select=[a, b, $f2, $f3, =($e, 0) AS $g_0, =($e, 1) AS $g_1], updateAsRetraction=[true], accMode=[AccRetract]) - +- Expand(projects=[{a=[$0], b=[$1], $f2=[$2], $f3=[$3], $e=[0]}, {a=[$0], b=[$1], $f2=[$2], $f3=[null], $e=[1]}], updateAsRetraction=[true], accMode=[AccRetract]) + +- Expand(projects=[a, b, $f2, $f3, $e], updateAsRetraction=[true], accMode=[AccRetract]) +- Calc(select=[a, b, 1 AS $f2, MOD(HASH_CODE(b), 1024) AS $f3], updateAsRetraction=[true], accMode=[AccRetract]) +- GroupAggregate(groupBy=[c], select=[c, AVG(a) AS a, AVG(b) AS b], updateAsRetraction=[true], accMode=[AccRetract]) +- Exchange(distribution=[hash[c]], updateAsRetraction=[true], accMode=[Acc]) @@ -1291,7 +1198,7 @@ GlobalGroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0_RET +- Exchange(distribution=[hash[a, $f3]], updateAsRetraction=[true], accMode=[Acc]) +- LocalGroupAggregate(groupBy=[a, $f3], partialFinalType=[PARTIAL], select=[a, $f3, COUNT_RETRACT(distinct$0 b) FILTER $g_0 AS count$0, COUNT_RETRACT(*) FILTER $g_1 AS count1$1, COUNT_RETRACT(*) AS count1$2, DISTINCT(b) AS distinct$0], updateAsRetraction=[true], accMode=[Acc]) +- Calc(select=[a, b, $f2, $f3, =($e, 0) AS $g_0, =($e, 1) AS $g_1], updateAsRetraction=[true], accMode=[AccRetract]) - +- Expand(projects=[{a=[$0], b=[$1], $f2=[$2], $f3=[$3], $e=[0]}, {a=[$0], b=[$1], $f2=[$2], $f3=[null], $e=[1]}], updateAsRetraction=[true], accMode=[AccRetract]) + +- Expand(projects=[a, b, $f2, $f3, $e], updateAsRetraction=[true], accMode=[AccRetract]) +- Calc(select=[a, b, 1 AS $f2, MOD(HASH_CODE(b), 1024) AS $f3], updateAsRetraction=[true], accMode=[AccRetract]) +- GlobalGroupAggregate(groupBy=[c], select=[c, AVG((sum$0, count$1)) AS a, AVG((sum$2, count$3)) AS b], updateAsRetraction=[true], accMode=[AccRetract]) +- Exchange(distribution=[hash[c]], updateAsRetraction=[true], accMode=[Acc]) @@ -1338,34 +1245,6 @@ GroupAggregate(groupBy=[a], select=[a, FIRST_VALUE(c) AS EXPR$1, COUNT(DISTINCT +- Exchange(distribution=[hash[a]]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) -]]> - - - - - - - - - - - @@ -1387,7 +1266,7 @@ GroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, FIRST_VALUE_RET +- GroupAggregate(groupBy=[a, $f3, $f4], partialFinalType=[PARTIAL], select=[a, $f3, $f4, FIRST_VALUE(c) FILTER $g_2 AS $f3_0, COUNT(DISTINCT b) FILTER $g_1 AS $f4_0]) +- Exchange(distribution=[hash[a, $f3, $f4]]) +- Calc(select=[a, b, c, $f3, $f4, =($e, 2) AS $g_2, =($e, 1) AS $g_1]) - +- Expand(projects=[{a=[$0], b=[$1], c=[$2], $f3=[$3], $f4=[null], $e=[1]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[$4], $e=[2]}]) + +- Expand(projects=[a, b, c, $f3, $f4, $e]) +- Calc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, MOD(HASH_CODE(c), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) @@ -1412,7 +1291,7 @@ GroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, FIRST_VALUE_RET +- GroupAggregate(groupBy=[a, $f3, $f4], partialFinalType=[PARTIAL], select=[a, $f3, $f4, FIRST_VALUE(c) FILTER $g_2 AS $f3_0, COUNT(DISTINCT b) FILTER $g_1 AS $f4_0]) +- Exchange(distribution=[hash[a, $f3, $f4]]) +- Calc(select=[a, b, c, $f3, $f4, =($e, 2) AS $g_2, =($e, 1) AS $g_1]) - +- Expand(projects=[{a=[$0], b=[$1], c=[$2], $f3=[$3], $f4=[null], $e=[1]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[$4], $e=[2]}]) + +- Expand(projects=[a, b, c, $f3, $f4, $e]) +- Calc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, MOD(HASH_CODE(c), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) @@ -1477,7 +1356,7 @@ GroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, LAST_VALUE_RETR +- GroupAggregate(groupBy=[a, $f3, $f4], partialFinalType=[PARTIAL], select=[a, $f3, $f4, LAST_VALUE(c) FILTER $g_2 AS $f3_0, COUNT(DISTINCT b) FILTER $g_1 AS $f4_0]) +- Exchange(distribution=[hash[a, $f3, $f4]]) +- Calc(select=[a, b, c, $f3, $f4, =($e, 2) AS $g_2, =($e, 1) AS $g_1]) - +- Expand(projects=[{a=[$0], b=[$1], c=[$2], $f3=[$3], $f4=[null], $e=[1]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[$4], $e=[2]}]) + +- Expand(projects=[a, b, c, $f3, $f4, $e]) +- Calc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, MOD(HASH_CODE(c), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) @@ -1502,10 +1381,103 @@ GroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, LAST_VALUE_RETR +- GroupAggregate(groupBy=[a, $f3, $f4], partialFinalType=[PARTIAL], select=[a, $f3, $f4, LAST_VALUE(c) FILTER $g_2 AS $f3_0, COUNT(DISTINCT b) FILTER $g_1 AS $f4_0]) +- Exchange(distribution=[hash[a, $f3, $f4]]) +- Calc(select=[a, b, c, $f3, $f4, =($e, 2) AS $g_2, =($e, 1) AS $g_1]) - +- Expand(projects=[{a=[$0], b=[$1], c=[$2], $f3=[$3], $f4=[null], $e=[1]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[$4], $e=[2]}]) + +- Expand(projects=[a, b, c, $f3, $f4, $e]) +- Calc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, MOD(HASH_CODE(c), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + @@ -1553,6 +1525,34 @@ GlobalGroupAggregate(groupBy=[a], select=[a, COUNT(distinct$0 count$0) AS EXPR$1 +- LocalGroupAggregate(groupBy=[a], select=[a, COUNT(distinct$0 b) AS count$0, MAX(c) AS max$1, DISTINCT(b) AS distinct$0]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +]]> + + + + + + + + + + + @@ -1579,7 +1579,7 @@ GlobalGroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0_RET +- Exchange(distribution=[hash[a, $f3, $f4]]) +- LocalGroupAggregate(groupBy=[a, $f3, $f4], partialFinalType=[PARTIAL], select=[a, $f3, $f4, COUNT(distinct$0 b) FILTER $g_1 AS count$0, MAX(c) FILTER $g_2 AS max$1, DISTINCT(b) AS distinct$0]) +- Calc(select=[a, b, c, $f3, $f4, =($e, 1) AS $g_1, =($e, 2) AS $g_2]) - +- Expand(projects=[{a=[$0], b=[$1], c=[$2], $f3=[$3], $f4=[null], $e=[1]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[$4], $e=[2]}]) + +- Expand(projects=[a, b, c, $f3, $f4, $e]) +- Calc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, MOD(HASH_CODE(c), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) @@ -1670,7 +1670,7 @@ GroupAggregate(groupBy=[c], partialFinalType=[FINAL], select=[c, SUM_RETRACT($f3 +- GroupAggregate(groupBy=[c, $f3, $f4], partialFinalType=[PARTIAL], select=[c, $f3, $f4, SUM(DISTINCT a) FILTER $g_1 AS $f3_0, SUM(a) FILTER $g_3 AS $f4_0, COUNT(DISTINCT b) FILTER $g_2 AS $f5]) +- Exchange(distribution=[hash[c, $f3, $f4]]) +- Calc(select=[a, b, c, $f3, $f4, =($e, 1) AS $g_1, =($e, 3) AS $g_3, =($e, 2) AS $g_2]) - +- Expand(projects=[{a=[$0], b=[$1], c=[$2], $f3=[$3], $f4=[null], $e=[1]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[$4], $e=[2]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[null], $e=[3]}]) + +- Expand(projects=[a, b, c, $f3, $f4, $e]) +- Calc(select=[a, b, c, MOD(HASH_CODE(a), 1024) AS $f3, MOD(HASH_CODE(b), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) @@ -1743,7 +1743,7 @@ GlobalGroupAggregate(groupBy=[c], partialFinalType=[FINAL], select=[c, SUM_RETRA +- Exchange(distribution=[hash[c, $f3, $f4]]) +- LocalGroupAggregate(groupBy=[c, $f3, $f4], partialFinalType=[PARTIAL], select=[c, $f3, $f4, SUM(distinct$0 a) FILTER $g_1 AS sum$0, SUM(a) FILTER $g_3 AS sum$1, COUNT(distinct$1 b) FILTER $g_2 AS count$2, DISTINCT(a) AS distinct$0, DISTINCT(b) AS distinct$1]) +- Calc(select=[a, b, c, $f3, $f4, =($e, 1) AS $g_1, =($e, 3) AS $g_3, =($e, 2) AS $g_2]) - +- Expand(projects=[{a=[$0], b=[$1], c=[$2], $f3=[$3], $f4=[null], $e=[1]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[$4], $e=[2]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[null], $e=[3]}]) + +- Expand(projects=[a, b, c, $f3, $f4, $e]) +- Calc(select=[a, b, c, MOD(HASH_CODE(a), 1024) AS $f3, MOD(HASH_CODE(b), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/GroupingSetsTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/GroupingSetsTest.xml index 0fed2c259f..f97ac5fee5 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/GroupingSetsTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/GroupingSetsTest.xml @@ -36,7 +36,7 @@ LogicalProject(d=[$0], c=[$2], g=[$1]) Calc(select=[d, c, g]) +- GroupAggregate(groupBy=[d, g, $e], select=[d, g, $e, COUNT(*) AS c]) +- Exchange(distribution=[hash[d, g, $e]]) - +- Expand(projects=[{d=[$0], g=[$1], $e=[0]}, {d=[$0], g=[null], $e=[1]}, {d=[null], g=[$1], $e=[2]}, {d=[null], g=[null], $e=[3]}]) + +- Expand(projects=[d, g, $e]) +- Calc(select=[MOD(deptno, 20) AS d, gender AS g]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -59,7 +59,7 @@ LogicalProject(c=[$1]) Calc(select=[c]) +- GroupAggregate(groupBy=[$f0, $e], select=[$f0, $e, COUNT(*) AS c]) +- Exchange(distribution=[hash[$f0, $e]]) - +- Expand(projects=[{$f0=[$0], $e=[0]}, {$f0=[null], $e=[1]}]) + +- Expand(projects=[$f0, $e]) +- Calc(select=[1 AS $f0]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -85,7 +85,7 @@ LogicalProject(d1=[$0], d0=[-($0, 1)], c=[$1]) Calc(select=[d1, -(d1, 1) AS d0, c]) +- GroupAggregate(groupBy=[d1, $e], select=[d1, $e, COUNT(*) AS c]) +- Exchange(distribution=[hash[d1, $e]]) - +- Expand(projects=[{d1=[$0], $e=[0]}, {d1=[null], $e=[1]}]) + +- Expand(projects=[d1, $e]) +- Calc(select=[+(deptno, 1) AS d1]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -111,7 +111,7 @@ LogicalProject(d=[$0], c=[$2], g=[$1]) Calc(select=[d, c, g]) +- GroupAggregate(groupBy=[d, g, $e], select=[d, g, $e, COUNT(*) AS c]) +- Exchange(distribution=[hash[d, g, $e]]) - +- Expand(projects=[{d=[$0], g=[$1], $e=[0]}, {d=[$0], g=[null], $e=[1]}, {d=[null], g=[null], $e=[3]}]) + +- Expand(projects=[d, g, $e]) +- Calc(select=[MOD(deptno, 20) AS d, gender AS g]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -134,7 +134,7 @@ LogicalProject(c=[$1]) Calc(select=[c]) +- GroupAggregate(groupBy=[$f0, $e], select=[$f0, $e, COUNT(*) AS c]) +- Exchange(distribution=[hash[$f0, $e]]) - +- Expand(projects=[{$f0=[$0], $e=[0]}, {$f0=[null], $e=[1]}]) + +- Expand(projects=[$f0, $e]) +- Calc(select=[1 AS $f0]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -156,34 +156,9 @@ LogicalAggregate(group=[{0}], groups=[[{0}, {}]], c=[COUNT()]) Calc(select=[deptno, c]) +- GroupAggregate(groupBy=[deptno, $e], select=[deptno, $e, COUNT(*) AS c]) +- Exchange(distribution=[hash[deptno, $e]]) - +- Expand(projects=[{deptno=[$0], $e=[0]}, {deptno=[null], $e=[1]}]) + +- Expand(projects=[deptno, $e]) +- Calc(select=[deptno]) +- TableSourceScan(table=[[default_catalog, default_database, emps, source: [TestTableSource(empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat)]]], fields=[empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat]) -]]> - - - - - - - - - - - @@ -204,9 +179,10 @@ FROM MyTable @@ -214,7 +190,7 @@ LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {1}, {}]], a=[AVG($2)], g Calc(select=[b, c, a, 0:BIGINT AS g, CASE(OR(=($e, 0:BIGINT), =($e, 1:BIGINT)), 0:BIGINT, 1:BIGINT) AS gb, CASE(=($e, 0:BIGINT), 0:BIGINT, =($e, 1:BIGINT), 1:BIGINT, =($e, 2:BIGINT), 0:BIGINT, 1:BIGINT) AS gc, CASE(OR(=($e, 0:BIGINT), =($e, 1:BIGINT)), 0:BIGINT, 1:BIGINT) AS gib, CASE(=($e, 0:BIGINT), 0:BIGINT, =($e, 1:BIGINT), 1:BIGINT, =($e, 2:BIGINT), 0:BIGINT, 1:BIGINT) AS gic, CASE(=($e, 0:BIGINT), 0:BIGINT, =($e, 1:BIGINT), 1:BIGINT, =($e, 2:BIGINT), 2:BIGINT, 3:BIGINT) AS gid]) +- GroupAggregate(groupBy=[b, c, $e], select=[b, c, $e, AVG(a) AS a]) +- Exchange(distribution=[hash[b, c, $e]]) - +- Expand(projects=[{a=[$0], b=[$1], c=[$2], $e=[0]}, {a=[$0], b=[$1], c=[null], $e=[1]}, {a=[$0], b=[null], c=[$2], $e=[2]}, {a=[$0], b=[null], c=[null], $e=[3]}]) + +- Expand(projects=[a, b, c, $e]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -239,7 +215,7 @@ GroupAggregate(groupBy=[EXPR$0], select=[EXPR$0]) +- Calc(select=[EXPR$0]) +- GroupAggregate(groupBy=[deptno, gender, $e], select=[deptno, gender, $e, COUNT(*) AS EXPR$0]) +- Exchange(distribution=[hash[deptno, gender, $e]]) - +- Expand(projects=[{deptno=[$0], gender=[$1], $e=[0]}, {deptno=[$0], gender=[null], $e=[1]}, {deptno=[null], gender=[$1], $e=[2]}, {deptno=[null], gender=[null], $e=[3]}]) + +- Expand(projects=[deptno, gender, $e]) +- Calc(select=[deptno, gender]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -271,7 +247,7 @@ LogicalProject(deptno=[$0], gender=[$2], min_name=[$3]) Calc(select=[deptno, gender, min_name], where=[OR(>($f5, 2), AND(=(gender, _UTF-16LE'M':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"), =(deptno, 10)))]) +- GroupAggregate(groupBy=[deptno, deptno0, gender, $e], select=[deptno, deptno0, gender, $e, MIN(ename) AS min_name, COUNT(*) AS $f5]) +- Exchange(distribution=[hash[deptno, deptno0, gender, $e]]) - +- Expand(projects=[{deptno=[$0], deptno0=[$1], gender=[$2], ename=[$3], $e=[0]}, {deptno=[$0], deptno0=[$1], gender=[null], ename=[$3], $e=[1]}, {deptno=[$0], deptno0=[null], gender=[$2], ename=[$3], $e=[2]}, {deptno=[$0], deptno0=[null], gender=[null], ename=[$3], $e=[3]}, {deptno=[null], deptno0=[$1], gender=[$2], ename=[$3], $e=[4]}, {deptno=[null], deptno0=[$1], gender=[null], ename=[$3], $e=[5]}, {deptno=[null], deptno0=[null], gender=[$2], ename=[$3], $e=[6]}, {deptno=[null], deptno0=[null], gender=[null], ename=[$3], $e=[7]}]) + +- Expand(projects=[deptno, deptno0, gender, ename, $e]) +- Calc(select=[deptno, deptno0, gender, ename]) +- Join(joinType=[InnerJoin], where=[=(deptno, deptno0)], select=[ename, deptno, gender, deptno0, dname], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[deptno]]) @@ -298,9 +274,10 @@ FROM MyTable @@ -308,7 +285,7 @@ LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {1}, {}]], a=[AVG($2)], g Calc(select=[b, c, a, 0:BIGINT AS g, CASE(OR(=($e, 0:BIGINT), =($e, 1:BIGINT)), 0:BIGINT, 1:BIGINT) AS gb, CASE(=($e, 0:BIGINT), 0:BIGINT, =($e, 1:BIGINT), 1:BIGINT, =($e, 2:BIGINT), 0:BIGINT, 1:BIGINT) AS gc, CASE(OR(=($e, 0:BIGINT), =($e, 1:BIGINT)), 0:BIGINT, 1:BIGINT) AS gib, CASE(=($e, 0:BIGINT), 0:BIGINT, =($e, 1:BIGINT), 1:BIGINT, =($e, 2:BIGINT), 0:BIGINT, 1:BIGINT) AS gic, CASE(=($e, 0:BIGINT), 0:BIGINT, =($e, 1:BIGINT), 1:BIGINT, =($e, 2:BIGINT), 2:BIGINT, 3:BIGINT) AS gid]) +- GroupAggregate(groupBy=[b, c, $e], select=[b, c, $e, AVG(a) AS a]) +- Exchange(distribution=[hash[b, c, $e]]) - +- Expand(projects=[{a=[$0], b=[$1], c=[$2], $e=[0]}, {a=[$0], b=[$1], c=[null], $e=[1]}, {a=[$0], b=[null], c=[$2], $e=[2]}, {a=[$0], b=[null], c=[null], $e=[3]}]) + +- Expand(projects=[a, b, c, $e]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -337,7 +314,7 @@ LogicalProject(deptno=[$1], gender=[$0], EXPR$2=[$2], c=[$3]) Calc(select=[deptno, gender, CASE(=($e, 0:BIGINT), 0:BIGINT, =($e, 1:BIGINT), 5:BIGINT, 7:BIGINT) AS EXPR$2, c]) +- GroupAggregate(groupBy=[gender, deptno, $e], select=[gender, deptno, $e, COUNT(*) AS c]) +- Exchange(distribution=[hash[gender, deptno, $e]]) - +- Expand(projects=[{gender=[$0], deptno=[$1], $e=[0]}, {gender=[$0], deptno=[null], $e=[1]}, {gender=[null], deptno=[null], $e=[3]}]) + +- Expand(projects=[gender, deptno, $e]) +- Calc(select=[gender, CAST(10) AS deptno], where=[=(deptno, 10)]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -360,7 +337,7 @@ LogicalProject(EXPR$0=[+($0, 1)], c=[$2]) Calc(select=[+(deptno, 1) AS EXPR$0, c]) +- GroupAggregate(groupBy=[deptno, gender, $e], select=[deptno, gender, $e, COUNT(*) AS c]) +- Exchange(distribution=[hash[deptno, gender, $e]]) - +- Expand(projects=[{deptno=[$0], gender=[$1], $e=[0]}, {deptno=[$0], gender=[null], $e=[1]}, {deptno=[null], gender=[$1], $e=[2]}, {deptno=[null], gender=[null], $e=[3]}]) + +- Expand(projects=[deptno, gender, $e]) +- Calc(select=[deptno, gender]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -397,7 +374,7 @@ Calc(select=[deptno, job, empno, ename, sumsal, CASE(=(CASE(=($e, 0:BIGINT), 0:B +- Exchange(distribution=[single]) +- GroupAggregate(groupBy=[deptno, job, empno, ename, $e], select=[deptno, job, empno, ename, $e, SUM(sal) AS sumsal]) +- Exchange(distribution=[hash[deptno, job, empno, ename, $e]]) - +- Expand(projects=[{deptno=[$0], job=[$1], empno=[$2], ename=[$3], sal=[$4], $e=[0]}, {deptno=[$0], job=[$1], empno=[null], ename=[null], sal=[$4], $e=[3]}, {deptno=[$0], job=[null], empno=[null], ename=[null], sal=[$4], $e=[7]}, {deptno=[null], job=[null], empno=[null], ename=[null], sal=[$4], $e=[15]}]) + +- Expand(projects=[deptno, job, empno, ename, sal, $e]) +- Calc(select=[deptno, job, empno, ename, sal]) +- TableSourceScan(table=[[default_catalog, default_database, scott_emp, source: [TestTableSource(empno, ename, job, mgr, hiredate, sal, comm, deptno)]]], fields=[empno, ename, job, mgr, hiredate, sal, comm, deptno]) ]]> @@ -431,7 +408,7 @@ Calc(select=[c, CASE(OR(=($e, 0:BIGINT), =($e, 1:BIGINT)), 0:BIGINT, 1:BIGINT) A +- GroupAggregate(groupBy=[deptno, gender, $e], select=[deptno, gender, $e, COUNT(*) AS c]) +- Exchange(distribution=[hash[deptno, gender, $e]]) +- Calc(select=[deptno, gender, $e], where=[<=(CASE(OR(=($e, 0:BIGINT), =($e, 1:BIGINT)), 0:BIGINT, 1:BIGINT), CASE(=($e, 0:BIGINT), 0:BIGINT, =($e, 1:BIGINT), 2:BIGINT, 7:BIGINT))]) - +- Expand(projects=[{deptno=[$0], gender=[$1], $e=[0]}, {deptno=[$0], gender=[null], $e=[1]}, {deptno=[null], gender=[null], $e=[3]}]) + +- Expand(projects=[deptno, gender, $e]) +- Calc(select=[deptno, gender]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -452,8 +429,8 @@ FROM emp GROUP BY ROLLUP(deptno, gender) @@ -463,7 +440,7 @@ LogicalProject(c=[$2], g=[$3], gid=[$4], gd=[$5], gg=[$6], ggd=[$7], gdg=[$8]) Calc(select=[c, CASE(OR(=($e, 0:BIGINT), =($e, 1:BIGINT)), 0:BIGINT, 1:BIGINT) AS g, 0:BIGINT AS gid, CASE(OR(=($e, 0:BIGINT), =($e, 1:BIGINT)), 0:BIGINT, 1:BIGINT) AS gd, CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT) AS gg, CASE(=($e, 0:BIGINT), 0:BIGINT, =($e, 1:BIGINT), 2:BIGINT, 3:BIGINT) AS ggd, CASE(=($e, 0:BIGINT), 0:BIGINT, =($e, 1:BIGINT), 1:BIGINT, 3:BIGINT) AS gdg]) +- GroupAggregate(groupBy=[deptno, gender, $e], select=[deptno, gender, $e, COUNT(*) AS c]) +- Exchange(distribution=[hash[deptno, gender, $e]]) - +- Expand(projects=[{deptno=[$0], gender=[$1], $e=[0]}, {deptno=[$0], gender=[null], $e=[1]}, {deptno=[null], gender=[null], $e=[3]}]) + +- Expand(projects=[deptno, gender, $e]) +- Calc(select=[deptno, gender]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -485,8 +462,8 @@ FROM emp @@ -496,7 +473,7 @@ LogicalProject(deptno=[$0], gender=[$1], gd=[$2], gg=[$3], dg=[$4], gd0=[$5], gi Calc(select=[deptno, gender, CASE(OR(=($e, 0:BIGINT), =($e, 1:BIGINT)), 0:BIGINT, 1:BIGINT) AS gd, CASE(=($e, 0:BIGINT), 0:BIGINT, =($e, 1:BIGINT), 1:BIGINT, =($e, 2:BIGINT), 0:BIGINT, 1:BIGINT) AS gg, CASE(=($e, 0:BIGINT), 0:BIGINT, =($e, 1:BIGINT), 1:BIGINT, =($e, 2:BIGINT), 2:BIGINT, 3:BIGINT) AS dg, CASE(=($e, 0:BIGINT), 0:BIGINT, =($e, 1:BIGINT), 2:BIGINT, =($e, 2:BIGINT), 1:BIGINT, 3:BIGINT) AS gd0, 0:BIGINT AS gid, c]) +- GroupAggregate(groupBy=[deptno, gender, $e], select=[deptno, gender, $e, COUNT(*) AS c]) +- Exchange(distribution=[hash[deptno, gender, $e]]) - +- Expand(projects=[{deptno=[$0], gender=[$1], $e=[0]}, {deptno=[$0], gender=[null], $e=[1]}, {deptno=[null], gender=[$1], $e=[2]}, {deptno=[null], gender=[null], $e=[3]}]) + +- Expand(projects=[deptno, gender, $e]) +- Calc(select=[deptno, gender]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -526,7 +503,7 @@ LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {1}, {}]], c=[COUNT()], d Calc(select=[deptno, job, c, CASE(OR(=($e, 0:BIGINT), =($e, 1:BIGINT)), 0:BIGINT, 1:BIGINT) AS d, CASE(=($e, 0:BIGINT), 0:BIGINT, =($e, 1:BIGINT), 1:BIGINT, =($e, 2:BIGINT), 0:BIGINT, 1:BIGINT) AS j, CASE(=($e, 0:BIGINT), 0:BIGINT, =($e, 1:BIGINT), 1:BIGINT, =($e, 2:BIGINT), 2:BIGINT, 3:BIGINT) AS x]) +- GroupAggregate(groupBy=[deptno, job, $e], select=[deptno, job, $e, COUNT(*) AS c]) +- Exchange(distribution=[hash[deptno, job, $e]]) - +- Expand(projects=[{deptno=[$0], job=[$1], $e=[0]}, {deptno=[$0], job=[null], $e=[1]}, {deptno=[null], job=[$1], $e=[2]}, {deptno=[null], job=[null], $e=[3]}]) + +- Expand(projects=[deptno, job, $e]) +- Calc(select=[deptno, job]) +- TableSourceScan(table=[[default_catalog, default_database, scott_emp, source: [TestTableSource(empno, ename, job, mgr, hiredate, sal, comm, deptno)]]], fields=[empno, ename, job, mgr, hiredate, sal, comm, deptno]) ]]> @@ -573,7 +550,7 @@ LogicalProject(c=[$1], deptno=[$0], g=[$2]) Calc(select=[c, deptno, CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT) AS g]) +- GroupAggregate(groupBy=[deptno, $e], select=[deptno, $e, COUNT(*) AS c]) +- Exchange(distribution=[hash[deptno, $e]]) - +- Expand(projects=[{deptno=[$0], $e=[0]}, {deptno=[null], $e=[1]}]) + +- Expand(projects=[deptno, $e]) +- Calc(select=[deptno]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -588,9 +565,10 @@ GROUP BY GROUPING SETS (b, c) @@ -598,7 +576,7 @@ LogicalAggregate(group=[{0, 1}], groups=[[{0}, {1}]], a=[AVG($2)], g=[GROUP_ID() Calc(select=[b, c, a, 0:BIGINT AS g]) +- GroupAggregate(groupBy=[b, c, $e], select=[b, c, $e, AVG(a) AS a]) +- Exchange(distribution=[hash[b, c, $e]]) - +- Expand(projects=[{a=[$0], b=[$1], c=[null], $e=[1]}, {a=[$0], b=[null], c=[$2], $e=[2]}]) + +- Expand(projects=[a, b, c, $e]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -619,7 +597,7 @@ LogicalAggregate(group=[{0, 1}], groups=[[{0}, {1}, {}]], a=[AVG($2)]) Calc(select=[b, c, a]) +- GroupAggregate(groupBy=[b, c, $e], select=[b, c, $e, AVG(a) AS a]) +- Exchange(distribution=[hash[b, c, $e]]) - +- Expand(projects=[{a=[$0], b=[$1], c=[null], $e=[1]}, {a=[$0], b=[null], c=[$2], $e=[2]}, {a=[$0], b=[null], c=[null], $e=[3]}]) + +- Expand(projects=[a, b, c, $e]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -642,9 +620,10 @@ FROM MyTable @@ -652,7 +631,7 @@ LogicalAggregate(group=[{0, 1}], groups=[[{0}, {1}, {}]], a=[AVG($2)], g=[GROUP_ Calc(select=[b, c, a, 0:BIGINT AS g, CASE(=($e, 1:BIGINT), 0:BIGINT, 1:BIGINT) AS gb, CASE(=($e, 1:BIGINT), 1:BIGINT, =($e, 2:BIGINT), 0:BIGINT, 1:BIGINT) AS gc, CASE(=($e, 1:BIGINT), 0:BIGINT, 1:BIGINT) AS gib, CASE(=($e, 1:BIGINT), 1:BIGINT, =($e, 2:BIGINT), 0:BIGINT, 1:BIGINT) AS gic, CASE(=($e, 1:BIGINT), 1:BIGINT, =($e, 2:BIGINT), 2:BIGINT, 3:BIGINT) AS gid, cnt]) +- GroupAggregate(groupBy=[b, c, $e], select=[b, c, $e, AVG(a) AS a, COUNT(*) AS cnt]) +- Exchange(distribution=[hash[b, c, $e]]) - +- Expand(projects=[{a=[$0], b=[$1], c=[null], $e=[1]}, {a=[$0], b=[null], c=[$2], $e=[2]}, {a=[$0], b=[null], c=[null], $e=[3]}]) + +- Expand(projects=[a, b, c, $e]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -675,7 +654,7 @@ LogicalAggregate(group=[{0}], groups=[[{0}, {}]], c=[COUNT()]) Calc(select=[EXPR$0, c]) +- GroupAggregate(groupBy=[EXPR$0, $e], select=[EXPR$0, $e, COUNT(*) AS c]) +- Exchange(distribution=[hash[EXPR$0, $e]]) - +- Expand(projects=[{EXPR$0=[$0], $e=[0]}, {EXPR$0=[null], $e=[1]}]) + +- Expand(projects=[EXPR$0, $e]) +- Calc(select=[+(deptno, 1) AS EXPR$0]) +- TableSourceScan(table=[[default_catalog, default_database, emps, source: [TestTableSource(empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat)]]], fields=[empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat]) ]]> @@ -705,7 +684,7 @@ Calc(select=[c]) +- Calc(select=[c, CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT) AS EXPR$1]) +- GroupAggregate(groupBy=[deptno, $e], select=[deptno, $e, COUNT(*) AS c]) +- Exchange(distribution=[hash[deptno, $e]]) - +- Expand(projects=[{deptno=[$0], $e=[0]}, {deptno=[null], $e=[1]}]) + +- Expand(projects=[deptno, $e]) +- Calc(select=[deptno]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -724,9 +703,10 @@ FROM emps GROUP BY GROUPING SETS (deptno) @@ -756,9 +736,10 @@ FROM MyTable @@ -766,7 +747,7 @@ LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], a=[AVG($2)], g=[GRO Calc(select=[b, c, a, 0:BIGINT AS g, CASE(OR(=($e, 0:BIGINT), =($e, 1:BIGINT)), 0:BIGINT, 1:BIGINT) AS gb, CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT) AS gc, CASE(OR(=($e, 0:BIGINT), =($e, 1:BIGINT)), 0:BIGINT, 1:BIGINT) AS gib, CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT) AS gic, CASE(=($e, 0:BIGINT), 0:BIGINT, =($e, 1:BIGINT), 1:BIGINT, 3:BIGINT) AS gid]) +- GroupAggregate(groupBy=[b, c, $e], select=[b, c, $e, AVG(a) AS a]) +- Exchange(distribution=[hash[b, c, $e]]) - +- Expand(projects=[{a=[$0], b=[$1], c=[$2], $e=[0]}, {a=[$0], b=[$1], c=[null], $e=[1]}, {a=[$0], b=[null], c=[null], $e=[3]}]) + +- Expand(projects=[a, b, c, $e]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -788,9 +769,10 @@ FROM MyTable @@ -798,7 +780,7 @@ LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {0}, {}]], a=[AVG($2)], g=[GRO Calc(select=[b, c, a, 0:BIGINT AS g, CASE(OR(=($e, 0:BIGINT), =($e, 1:BIGINT)), 0:BIGINT, 1:BIGINT) AS gb, CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT) AS gc, CASE(OR(=($e, 0:BIGINT), =($e, 1:BIGINT)), 0:BIGINT, 1:BIGINT) AS gib, CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT) AS gic, CASE(=($e, 0:BIGINT), 0:BIGINT, =($e, 1:BIGINT), 1:BIGINT, 3:BIGINT) AS gid]) +- GroupAggregate(groupBy=[b, c, $e], select=[b, c, $e, AVG(a) AS a]) +- Exchange(distribution=[hash[b, c, $e]]) - +- Expand(projects=[{a=[$0], b=[$1], c=[$2], $e=[0]}, {a=[$0], b=[$1], c=[null], $e=[1]}, {a=[$0], b=[null], c=[null], $e=[3]}]) + +- Expand(projects=[a, b, c, $e]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -820,7 +802,7 @@ LogicalProject(deptno=[$0], c=[$2]) Calc(select=[deptno, c]) +- GroupAggregate(groupBy=[deptno, gender, $e], select=[deptno, gender, $e, COUNT(*) AS c]) +- Exchange(distribution=[hash[deptno, gender, $e]]) - +- Expand(projects=[{deptno=[$0], gender=[$1], $e=[0]}, {deptno=[$0], gender=[null], $e=[1]}, {deptno=[null], gender=[$1], $e=[2]}, {deptno=[null], gender=[null], $e=[3]}]) + +- Expand(projects=[deptno, gender, $e]) +- Calc(select=[deptno, gender]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -846,7 +828,7 @@ LogicalProject(half1=[+($0, 1)], c=[$3]) Calc(select=[+($f0, 1) AS half1, c]) +- GroupAggregate(groupBy=[$f0, gender, $f2, $e], select=[$f0, gender, $f2, $e, COUNT(*) AS c]) +- Exchange(distribution=[hash[$f0, gender, $f2, $e]]) - +- Expand(projects=[{$f0=[$0], gender=[$1], $f2=[$2], $e=[0]}, {$f0=[$0], gender=[$1], $f2=[null], $e=[1]}, {$f0=[$0], gender=[null], $f2=[$2], $e=[2]}, {$f0=[$0], gender=[null], $f2=[null], $e=[3]}, {$f0=[null], gender=[null], $f2=[$2], $e=[6]}, {$f0=[null], gender=[null], $f2=[null], $e=[7]}]) + +- Expand(projects=[$f0, gender, $f2, $e]) +- Calc(select=[/(deptno, 2) AS $f0, gender, SUBSTRING(ename, 1, 1) AS $f2]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -871,7 +853,7 @@ LogicalProject(gender=[$1], EXPR$1=[+($0, 1)], c=[$2]) Calc(select=[gender, +(deptno, 1) AS EXPR$1, c]) +- GroupAggregate(groupBy=[deptno, gender, $e], select=[deptno, gender, $e, COUNT(*) AS c]) +- Exchange(distribution=[hash[deptno, gender, $e]]) - +- Expand(projects=[{deptno=[$0], gender=[$1], $e=[0]}, {deptno=[$0], gender=[null], $e=[1]}, {deptno=[null], gender=[null], $e=[3]}]) + +- Expand(projects=[deptno, gender, $e]) +- Calc(select=[deptno, gender]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -893,7 +875,7 @@ LogicalAggregate(group=[{0}], groups=[[{0}, {}]], c=[COUNT()]) Calc(select=[gender, c]) +- GroupAggregate(groupBy=[gender, $e], select=[gender, $e, COUNT(*) AS c]) +- Exchange(distribution=[hash[gender, $e]]) - +- Expand(projects=[{gender=[$0], $e=[0]}, {gender=[null], $e=[1]}]) + +- Expand(projects=[gender, $e]) +- Calc(select=[gender]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -918,7 +900,7 @@ Calc(select=[gender, c]) +- Exchange(distribution=[single]) +- GroupAggregate(groupBy=[gender, $e], select=[gender, $e, COUNT(*) AS c]) +- Exchange(distribution=[hash[gender, $e]]) - +- Expand(projects=[{gender=[$0], $e=[0]}, {gender=[null], $e=[1]}]) + +- Expand(projects=[gender, $e]) +- Calc(select=[gender]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -944,7 +926,7 @@ LogicalProject(d1=[+($0, 1)], c=[$1]) Calc(select=[+(deptno, 1) AS d1, c], where=[>(c, 3)]) +- GroupAggregate(groupBy=[deptno, $e], select=[deptno, $e, COUNT(*) AS c]) +- Exchange(distribution=[hash[deptno, $e]]) - +- Expand(projects=[{deptno=[$0], $e=[0]}, {deptno=[null], $e=[1]}]) + +- Expand(projects=[deptno, $e]) +- Calc(select=[deptno]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> @@ -967,7 +949,7 @@ LogicalProject(EXPR$0=[+($0, 1)], c=[$1]) Calc(select=[+(deptno, 1) AS EXPR$0, c]) +- GroupAggregate(groupBy=[deptno, $e], select=[deptno, $e, COUNT(*) AS c]) +- Exchange(distribution=[hash[deptno, $e]]) - +- Expand(projects=[{deptno=[$0], $e=[0]}, {deptno=[null], $e=[1]}]) + +- Expand(projects=[deptno, $e]) +- Calc(select=[deptno]) +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/IncrementalAggregateTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/IncrementalAggregateTest.xml index d81592c8e1..be012bca9d 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/IncrementalAggregateTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/IncrementalAggregateTest.xml @@ -43,7 +43,7 @@ GlobalGroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0(cou +- Exchange(distribution=[hash[a, $f4]]) +- LocalGroupAggregate(groupBy=[a, $f4], partialFinalType=[PARTIAL], select=[a, $f4, COUNT(distinct$0 b) FILTER $g_0 AS count$0, SUM(b) FILTER $g_1 AS sum$1, SUM(b) FILTER $g_10 AS sum$2, DISTINCT(b) AS distinct$0]) +- Calc(select=[a, b, $f2, $f3, $f4, AND(=($e, 0), $f2) AS $g_0, AND(=($e, 1), $f3) AS $g_1, AND(=($e, 1), $f2) AS $g_10]) - +- Expand(projects=[{a=[$0], b=[$1], $f2=[$2], $f3=[$3], $f4=[$4], $e=[0]}, {a=[$0], b=[$1], $f2=[$2], $f3=[$3], $f4=[null], $e=[1]}]) + +- Expand(projects=[a, b, $f2, $f3, $f4, $e]) +- Calc(select=[a, b, IS TRUE(<>(b, 2:BIGINT)) AS $f2, IS TRUE(<>(b, 5:BIGINT)) AS $f3, MOD(HASH_CODE(b), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) @@ -119,7 +119,7 @@ GlobalGroupAggregate(groupBy=[c], partialFinalType=[FINAL], select=[c, MIN_RETRA +- Exchange(distribution=[hash[c, $f3, $f4]], updateAsRetraction=[true], accMode=[Acc]) +- LocalGroupAggregate(groupBy=[c, $f3, $f4], partialFinalType=[PARTIAL], select=[c, $f3, $f4, MIN_RETRACT(b) FILTER $g_2 AS min$0, MAX_RETRACT(b) FILTER $g_2 AS max$1, SUM_RETRACT(b) FILTER $g_3 AS (sum$2, count$3), COUNT_RETRACT(*) FILTER $g_3 AS count1$4, COUNT_RETRACT(distinct$0 a) FILTER $g_1 AS count$5, COUNT_RETRACT(*) AS count1$6, DISTINCT(a) AS distinct$0], updateAsRetraction=[true], accMode=[Acc]) +- Calc(select=[a, b, c, $f3, $f4, =($e, 2) AS $g_2, =($e, 3) AS $g_3, =($e, 1) AS $g_1], updateAsRetraction=[true], accMode=[AccRetract]) - +- Expand(projects=[{a=[$0], b=[$1], c=[$2], $f3=[$3], $f4=[null], $e=[1]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[$4], $e=[2]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[null], $e=[3]}], updateAsRetraction=[true], accMode=[AccRetract]) + +- Expand(projects=[a, b, c, $f3, $f4, $e], updateAsRetraction=[true], accMode=[AccRetract]) +- Calc(select=[a, b, c, MOD(HASH_CODE(a), 1024) AS $f3, MOD(HASH_CODE(b), 1024) AS $f4], updateAsRetraction=[true], accMode=[AccRetract]) +- GlobalGroupAggregate(groupBy=[a], select=[a, AVG((sum$0, count$1)) AS b, MAX(max$2) AS c], updateAsRetraction=[true], accMode=[AccRetract]) +- Exchange(distribution=[hash[a]], updateAsRetraction=[true], accMode=[Acc]) @@ -148,7 +148,7 @@ GlobalGroupAggregate(partialFinalType=[FINAL], select=[$SUM0(count$0) AS $f0, SU +- Exchange(distribution=[hash[$f2, $f3]]) +- LocalGroupAggregate(groupBy=[$f2, $f3], partialFinalType=[PARTIAL], select=[$f2, $f3, COUNT(distinct$0 a) FILTER $g_1 AS count$0, SUM(distinct$1 b) FILTER $g_2 AS sum$1, DISTINCT(a) AS distinct$0, DISTINCT(b) AS distinct$1]) +- Calc(select=[a, b, $f2, $f3, =($e, 1) AS $g_1, =($e, 2) AS $g_2]) - +- Expand(projects=[{a=[$0], b=[$1], $f2=[$2], $f3=[null], $e=[1]}, {a=[$0], b=[$1], $f2=[null], $f3=[$3], $e=[2]}]) + +- Expand(projects=[a, b, $f2, $f3, $e]) +- Calc(select=[a, b, MOD(HASH_CODE(a), 1024) AS $f2, MOD(HASH_CODE(b), 1024) AS $f3]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) @@ -184,63 +184,13 @@ GlobalGroupAggregate(groupBy=[c], partialFinalType=[FINAL], select=[c, MIN_RETRA +- Exchange(distribution=[hash[c, $f3, $f4]]) +- LocalGroupAggregate(groupBy=[c, $f3, $f4], partialFinalType=[PARTIAL], select=[c, $f3, $f4, MIN_RETRACT(b) FILTER $g_2 AS min$0, MAX_RETRACT(b) FILTER $g_2 AS max$1, SUM_RETRACT(b) FILTER $g_3 AS (sum$2, count$3), COUNT_RETRACT(*) FILTER $g_3 AS count1$4, COUNT_RETRACT(distinct$0 a) FILTER $g_1 AS count$5, COUNT_RETRACT(*) AS count1$6, DISTINCT(a) AS distinct$0]) +- Calc(select=[a, b, c, $f3, $f4, =($e, 2) AS $g_2, =($e, 3) AS $g_3, =($e, 1) AS $g_1]) - +- Expand(projects=[{a=[$0], b=[$1], c=[$2], $f3=[$3], $f4=[null], $e=[1]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[$4], $e=[2]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[null], $e=[3]}]) + +- Expand(projects=[a, b, c, $f3, $f4, $e]) +- Calc(select=[a, b, c, MOD(HASH_CODE(a), 1024) AS $f3, MOD(HASH_CODE(b), 1024) AS $f4]) +- GlobalGroupAggregate(groupBy=[a], select=[a, AVG((sum$0, count$1)) AS b, MAX(max$2) AS c]) +- Exchange(distribution=[hash[a]]) +- LocalGroupAggregate(groupBy=[a], select=[a, AVG(b) AS (sum$0, count$1), MAX(c) AS max$2]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) -]]> - - - - - - - - - - - - - - - - - - - - - - @@ -292,10 +242,34 @@ Calc(select=[a, EXPR$1, CASE(=($f3, 0), null:BIGINT, EXPR$2) AS EXPR$2, /(CASE(= +- Exchange(distribution=[hash[a, $f3, $f4]]) +- LocalGroupAggregate(groupBy=[a, $f3, $f4], partialFinalType=[PARTIAL], select=[a, $f3, $f4, COUNT(distinct$0 c) FILTER $g_2 AS count$0, $SUM0(b) FILTER $g_3 AS sum$1, COUNT(b) FILTER $g_3 AS count$2, MAX(b) FILTER $g_1 AS max$3, MIN(b) FILTER $g_1 AS min$4, COUNT(*) FILTER $g_3 AS count1$5, DISTINCT(c) AS distinct$0]) +- Calc(select=[a, b, c, $f3, $f4, =($e, 2) AS $g_2, =($e, 3) AS $g_3, =($e, 1) AS $g_1]) - +- Expand(projects=[{a=[$0], b=[$1], c=[$2], $f3=[$3], $f4=[null], $e=[1]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[$4], $e=[2]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[null], $e=[3]}]) + +- Expand(projects=[a, b, c, $f3, $f4, $e]) +- Calc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, MOD(HASH_CODE(c), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +]]> + + + + + + + + + + + @@ -319,7 +293,7 @@ Calc(select=[a, $f1, $f2, IF(=($f4, 0:BIGINT), null:BIGINT, /($f3, $f4)) AS $f3] +- Exchange(distribution=[hash[a, $f2]]) +- LocalGroupAggregate(groupBy=[a, $f2], partialFinalType=[PARTIAL], select=[a, $f2, COUNT(distinct$0 b) FILTER $g_0 AS count$0, SUM(b) FILTER $g_1 AS sum$1, $SUM0(b) FILTER $g_1 AS sum$2, COUNT(b) FILTER $g_1 AS count$3, DISTINCT(b) AS distinct$0]) +- Calc(select=[a, b, $f2, =($e, 0) AS $g_0, =($e, 1) AS $g_1]) - +- Expand(projects=[{a=[$0], b=[$1], $f2=[$2], $e=[0]}, {a=[$0], b=[$1], $f2=[null], $e=[1]}]) + +- Expand(projects=[a, b, $f2, $e]) +- Calc(select=[a, b, MOD(HASH_CODE(b), 1024) AS $f2]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) @@ -354,42 +328,13 @@ GlobalGroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0_RET +- Exchange(distribution=[hash[a, $f3]], updateAsRetraction=[true], accMode=[Acc]) +- LocalGroupAggregate(groupBy=[a, $f3], partialFinalType=[PARTIAL], select=[a, $f3, COUNT_RETRACT(distinct$0 b) FILTER $g_0 AS count$0, COUNT_RETRACT(*) FILTER $g_1 AS count1$1, COUNT_RETRACT(*) AS count1$2, DISTINCT(b) AS distinct$0], updateAsRetraction=[true], accMode=[Acc]) +- Calc(select=[a, b, $f2, $f3, =($e, 0) AS $g_0, =($e, 1) AS $g_1], updateAsRetraction=[true], accMode=[AccRetract]) - +- Expand(projects=[{a=[$0], b=[$1], $f2=[$2], $f3=[$3], $e=[0]}, {a=[$0], b=[$1], $f2=[$2], $f3=[null], $e=[1]}], updateAsRetraction=[true], accMode=[AccRetract]) + +- Expand(projects=[a, b, $f2, $f3, $e], updateAsRetraction=[true], accMode=[AccRetract]) +- Calc(select=[a, b, 1 AS $f2, MOD(HASH_CODE(b), 1024) AS $f3], updateAsRetraction=[true], accMode=[AccRetract]) +- GlobalGroupAggregate(groupBy=[c], select=[c, AVG((sum$0, count$1)) AS a, AVG((sum$2, count$3)) AS b], updateAsRetraction=[true], accMode=[AccRetract]) +- Exchange(distribution=[hash[c]], updateAsRetraction=[true], accMode=[Acc]) +- LocalGroupAggregate(groupBy=[c], select=[c, AVG(a) AS (sum$0, count$1), AVG(b) AS (sum$2, count$3)], updateAsRetraction=[true], accMode=[Acc]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime], updateAsRetraction=[true], accMode=[Acc]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc]) -]]> - - - - - - - - - - - @@ -411,7 +356,7 @@ GroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, FIRST_VALUE_RET +- GroupAggregate(groupBy=[a, $f3, $f4], partialFinalType=[PARTIAL], select=[a, $f3, $f4, FIRST_VALUE(c) FILTER $g_2 AS $f3_0, COUNT(DISTINCT b) FILTER $g_1 AS $f4_0]) +- Exchange(distribution=[hash[a, $f3, $f4]]) +- Calc(select=[a, b, c, $f3, $f4, =($e, 2) AS $g_2, =($e, 1) AS $g_1]) - +- Expand(projects=[{a=[$0], b=[$1], c=[$2], $f3=[$3], $f4=[null], $e=[1]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[$4], $e=[2]}]) + +- Expand(projects=[a, b, c, $f3, $f4, $e]) +- Calc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, MOD(HASH_CODE(c), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) @@ -436,10 +381,65 @@ GroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, LAST_VALUE_RETR +- GroupAggregate(groupBy=[a, $f3, $f4], partialFinalType=[PARTIAL], select=[a, $f3, $f4, LAST_VALUE(c) FILTER $g_2 AS $f3_0, COUNT(DISTINCT b) FILTER $g_1 AS $f4_0]) +- Exchange(distribution=[hash[a, $f3, $f4]]) +- Calc(select=[a, b, c, $f3, $f4, =($e, 2) AS $g_2, =($e, 1) AS $g_1]) - +- Expand(projects=[{a=[$0], b=[$1], c=[$2], $f3=[$3], $f4=[null], $e=[1]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[$4], $e=[2]}]) + +- Expand(projects=[a, b, c, $f3, $f4, $e]) +- Calc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, MOD(HASH_CODE(c), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +]]> + + + + + + + + + + + + + + + + + + + + + + @@ -486,7 +486,7 @@ GlobalGroupAggregate(groupBy=[c], partialFinalType=[FINAL], select=[c, SUM(sum$0 +- Exchange(distribution=[hash[c, $f3, $f4]]) +- LocalGroupAggregate(groupBy=[c, $f3, $f4], partialFinalType=[PARTIAL], select=[c, $f3, $f4, SUM(distinct$0 a) FILTER $g_1 AS sum$0, SUM(a) FILTER $g_3 AS sum$1, COUNT(distinct$1 b) FILTER $g_2 AS count$2, DISTINCT(a) AS distinct$0, DISTINCT(b) AS distinct$1]) +- Calc(select=[a, b, c, $f3, $f4, =($e, 1) AS $g_1, =($e, 3) AS $g_3, =($e, 2) AS $g_2]) - +- Expand(projects=[{a=[$0], b=[$1], c=[$2], $f3=[$3], $f4=[null], $e=[1]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[$4], $e=[2]}, {a=[$0], b=[$1], c=[$2], $f3=[null], $f4=[null], $e=[3]}]) + +- Expand(projects=[a, b, c, $f3, $f4, $e]) +- Calc(select=[a, b, c, MOD(HASH_CODE(a), 1024) AS $f3, MOD(HASH_CODE(b), 1024) AS $f4]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) -- Gitee From d5f45c3539835f8c2dda4007eb5f69091a9913d1 Mon Sep 17 00:00:00 2001 From: "yuzhao.cyz" Date: Tue, 17 Mar 2020 21:19:41 +0800 Subject: [PATCH 373/885] [FLINK-14338][table-planner-blink] Plan verify changes from DIGEST to EXPLAIN: change plan for values node * The values node does not print row type anymore --- .../table/planner/plan/batch/sql/CalcTest.xml | 4 +- .../planner/plan/batch/sql/LimitTest.xml | 100 +++++++++--------- .../plan/batch/sql/SetOperatorsTest.xml | 6 +- .../planner/plan/batch/sql/SortLimitTest.xml | 8 +- .../planner/plan/batch/sql/UnnestTest.xml | 22 ++-- .../planner/plan/batch/sql/ValuesTest.xml | 22 ++-- .../batch/sql/join/BroadcastHashJoinTest.xml | 2 +- .../batch/sql/join/NestedLoopJoinTest.xml | 6 +- .../plan/batch/sql/join/SortMergeJoinTest.xml | 2 +- .../CalcPruneAggregateCallRuleTest.xml | 10 +- .../logical/FlinkLimit0RemoveRuleTest.xml | 18 ++-- .../logical/FlinkPruneEmptyRulesTest.xml | 2 +- .../rules/logical/LogicalUnnestRuleTest.xml | 24 ++--- .../ProjectPruneAggregateCallRuleTest.xml | 4 +- .../logical/subquery/SubQuerySemiJoinTest.xml | 4 +- .../planner/plan/stream/sql/LimitTest.xml | 24 ++--- .../plan/stream/sql/SetOperatorsTest.xml | 6 +- .../planner/plan/stream/sql/SortLimitTest.xml | 32 +++--- .../planner/plan/stream/sql/UnnestTest.xml | 22 ++-- .../planner/plan/stream/sql/ValuesTest.xml | 22 ++-- .../planner/plan/stream/sql/join/JoinTest.xml | 2 +- .../plan/stream/sql/join/WindowJoinTest.xml | 6 +- 22 files changed, 174 insertions(+), 174 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/CalcTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/CalcTest.xml index 9eb1c4d758..09e443adc8 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/CalcTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/CalcTest.xml @@ -40,13 +40,13 @@ Calc(select=[ARRAY(_UTF-16LE'Hi', _UTF-16LE'Hello', c) AS EXPR$0]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/LimitTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/LimitTest.xml index de3fd4b08c..4456145837 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/LimitTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/LimitTest.xml @@ -29,7 +29,7 @@ LogicalSort(fetch=[0]) @@ -130,28 +130,7 @@ LogicalSort(offset=[10], fetch=[0]) - - - - - - - - - - - @@ -168,7 +147,7 @@ LogicalSort(offset=[0], fetch=[0]) @@ -185,7 +164,7 @@ LogicalSort(fetch=[0]) @@ -210,34 +189,34 @@ Calc(select=[a, c]) ]]> - + - + - + - + @@ -245,30 +224,31 @@ LogicalSort(offset=[10]) - + - + @@ -292,13 +272,33 @@ Limit(offset=[0], fetch=[5], global=[true]) ]]> - + - + + + + + + + + + + + + @@ -306,10 +306,10 @@ LogicalSort(sort0=[$1], dir0=[ASC-nulls-first], fetch=[10]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.xml index 3a062b267b..8b3ae12b97 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.xml @@ -87,7 +87,7 @@ LogicalIntersect(all=[false]) @@ -107,7 +107,7 @@ LogicalIntersect(all=[false]) @@ -210,7 +210,7 @@ LogicalMinus(all=[false]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SortLimitTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SortLimitTest.xml index fb45f0c8cf..74112afadc 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SortLimitTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SortLimitTest.xml @@ -49,7 +49,7 @@ LogicalSort(sort0=[$0], sort1=[$1], dir0=[DESC-nulls-last], dir1=[ASC-nulls-firs @@ -85,7 +85,7 @@ LogicalSort(sort0=[$0], dir0=[DESC-nulls-last], fetch=[0]) @@ -141,7 +141,7 @@ LogicalSort(sort0=[$0], dir0=[DESC-nulls-last], fetch=[0]) @@ -198,7 +198,7 @@ LogicalSort(sort0=[$0], sort1=[$1], dir0=[DESC-nulls-last], dir1=[ASC-nulls-firs diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml index 0074eb94af..6da75ba99f 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml @@ -28,7 +28,7 @@ LogicalProject(a=[$0], s=[$3]) +- LogicalProject(s=[$0]) +- Uncollect +- LogicalProject(c=[$cor0.c]) - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + +- LogicalValues(tuples=[[{ 0 }]]) ]]> @@ -56,12 +56,12 @@ LogicalProject(b=[$0], s=[$2]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) :- LogicalProject(b=[$0], set=[$2]) : +- LogicalAggregate(group=[{0, 1}], set=[COLLECT($0)]) - : +- LogicalProject(b=[$1], $f1=[TUMBLE($3, 3000:INTERVAL SECOND)]) + : +- LogicalProject(b=[$1], $f1=[$TUMBLE($3, 3000:INTERVAL SECOND)]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(s=[$0]) +- Uncollect +- LogicalProject(set=[$cor0.set]) - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + +- LogicalValues(tuples=[[{ 0 }]]) ]]> @@ -90,7 +90,7 @@ LogicalProject(a=[$0], b=[$1], v=[$4]) +- LogicalProject(k=[$0], v=[$1]) +- Uncollect +- LogicalProject(c=[$cor0.c]) - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + +- LogicalValues(tuples=[[{ 0 }]]) ]]> @@ -121,7 +121,7 @@ LogicalProject(a=[$0], b=[$1], x=[$2], y=[$3]) +- LogicalProject(x=[$0], y=[$1]) +- Uncollect +- LogicalProject(b=[$cor0.b]) - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + +- LogicalValues(tuples=[[{ 0 }]]) ]]> @@ -151,7 +151,7 @@ LogicalProject(a=[$0], s=[$2]) +- LogicalProject(s=[$0]) +- Uncollect +- LogicalProject(set=[$cor0.set]) - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + +- LogicalValues(tuples=[[{ 0 }]]) ]]> @@ -180,7 +180,7 @@ LogicalProject(a=[$0], s=[$3]) +- LogicalProject(s=[$0]) +- Uncollect +- LogicalProject(c=[$cor0.c]) - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + +- LogicalValues(tuples=[[{ 0 }]]) ]]> @@ -209,7 +209,7 @@ LogicalProject(b=[$0], id=[$2], point=[$3]) +- LogicalProject(id=[$0], point=[$1]) +- Uncollect +- LogicalProject(set=[$cor0.set]) - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + +- LogicalValues(tuples=[[{ 0 }]]) ]]> @@ -238,7 +238,7 @@ LogicalProject(a=[$0], b=[$1], s=[$3]) +- LogicalProject(s=[$0]) +- Uncollect +- LogicalProject(b=[$cor0.b]) - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + +- LogicalValues(tuples=[[{ 0 }]]) ]]> @@ -262,7 +262,7 @@ LogicalProject(a=[$0], b=[$1], s=[$2], t=[$3]) +- LogicalProject(s=[$0], t=[$1]) +- Uncollect +- LogicalProject(b=[$cor0.b]) - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + +- LogicalValues(tuples=[[{ 0 }]]) ]]> @@ -284,7 +284,7 @@ LogicalProject(a=[$0], b=[$1], _1=[$2], _2=[$3]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]]) +- Uncollect +- LogicalProject(b=[$cor0.b]) - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + +- LogicalValues(tuples=[[{ 0 }]]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/ValuesTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/ValuesTest.xml index fa537d3784..c517550b77 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/ValuesTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/ValuesTest.xml @@ -25,16 +25,16 @@ limitations under the License. LogicalProject(a=[$0], b=[$1]), rowType=[RecordType(INTEGER a, DECIMAL(20, 1) b)] +- LogicalUnion(all=[true]), rowType=[RecordType(INTEGER EXPR$0, DECIMAL(20, 1) EXPR$1)] :- LogicalProject(EXPR$0=[1], EXPR$1=[2.0:DECIMAL(2, 1)]), rowType=[RecordType(INTEGER EXPR$0, DECIMAL(2, 1) EXPR$1)] - : +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]), rowType=[RecordType(INTEGER ZERO)] + : +- LogicalValues(tuples=[[{ 0 }]]), rowType=[RecordType(INTEGER ZERO)] +- LogicalProject(EXPR$0=[3], EXPR$1=[4:BIGINT]), rowType=[RecordType(INTEGER EXPR$0, BIGINT EXPR$1)] - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]), rowType=[RecordType(INTEGER ZERO)] + +- LogicalValues(tuples=[[{ 0 }]]), rowType=[RecordType(INTEGER ZERO)] ]]> @@ -47,12 +47,12 @@ Union(all=[true], union=[EXPR$0, EXPR$1]), rowType=[RecordType(INTEGER EXPR$0, D @@ -65,18 +65,18 @@ Values(type=[RecordType(INTEGER a, INTEGER b, INTEGER c)], tuples=[[{ 1, 2, 3 }] LogicalProject(a=[$0], b=[$1]) +- LogicalUnion(all=[true]) :- LogicalProject(EXPR$0=[1], EXPR$1=[2]) - : +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + : +- LogicalValues(tuples=[[{ 0 }]]) :- LogicalProject(EXPR$0=[3], EXPR$1=[null:INTEGER]) - : +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + : +- LogicalValues(tuples=[[{ 0 }]]) +- LogicalProject(EXPR$0=[4], EXPR$1=[5]) - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + +- LogicalValues(tuples=[[{ 0 }]]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.xml index 9af0bbc644..fffdc6fdde 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.xml @@ -350,7 +350,7 @@ HashJoin(joinType=[LeftOuterJoin], where=[=(k, k0)], select=[k, v, k0, v0], isBr :- Calc(select=[CAST(0:BIGINT) AS k, v], where=[=(k, 0:BIGINT)]) : +- TableSourceScan(table=[[default_catalog, default_database, src, source: [TestTableSource(k, v)]]], fields=[k, v]) +- Exchange(distribution=[broadcast]) - +- Values(type=[RecordType(BIGINT k, VARCHAR(2147483647) v)], tuples=[[]], values=[k, v]) + +- Values(tuples=[[]], values=[k, v]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.xml index 73dbc0f76f..a4d534c55e 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.xml @@ -427,7 +427,7 @@ LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7]) NestedLoopJoin(joinType=[LeftOuterJoin], where=[true], select=[d, e, f, g, h, a, b, c], build=[right]) :- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h]) +- Exchange(distribution=[broadcast]) - +- Values(type=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c)], tuples=[[]], values=[a, b, c]) + +- Values(tuples=[[]], values=[a, b, c]) ]]> @@ -541,7 +541,7 @@ LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7]) @@ -685,7 +685,7 @@ NestedLoopJoin(joinType=[LeftOuterJoin], where=[=(k, k0)], select=[k, v, k0, v0] :- Calc(select=[CAST(0:BIGINT) AS k, v], where=[=(k, 0:BIGINT)]) : +- TableSourceScan(table=[[default_catalog, default_database, src, source: [TestTableSource(k, v)]]], fields=[k, v]) +- Exchange(distribution=[broadcast]) - +- Values(type=[RecordType(BIGINT k, VARCHAR(2147483647) v)], tuples=[[]], values=[k, v]) + +- Values(tuples=[[]], values=[k, v]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.xml index f9ac1e6cfe..b9dc2afba7 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.xml @@ -463,7 +463,7 @@ SortMergeJoin(joinType=[LeftOuterJoin], where=[=(k, k0)], select=[k, v, k0, v0]) : +- Calc(select=[CAST(0:BIGINT) AS k, v], where=[=(k, 0:BIGINT)]) : +- TableSourceScan(table=[[default_catalog, default_database, src, source: [TestTableSource(k, v)]]], fields=[k, v]) +- Exchange(distribution=[hash[k]]) - +- Values(type=[RecordType(BIGINT k, VARCHAR(2147483647) v)], tuples=[[]], values=[k, v]) + +- Values(tuples=[[]], values=[k, v]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/CalcPruneAggregateCallRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/CalcPruneAggregateCallRuleTest.xml index 2ca68f6a2b..86fa1c9d31 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/CalcPruneAggregateCallRuleTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/CalcPruneAggregateCallRuleTest.xml @@ -89,7 +89,7 @@ LogicalJoin(condition=[$4], joinType=[semi]) +- LogicalAggregate(group=[{}], m=[MIN($0)]) +- LogicalCalc(expr#0=[{inputs}], expr#1=[true], i=[$t1]) +- LogicalAggregate(group=[{}], EXPR$0=[SUM($0)]) - +- LogicalValues(type=[RecordType(INTEGER a1)], tuples=[[]]) + +- LogicalValues(tuples=[[]]) ]]> @@ -166,7 +166,7 @@ LogicalJoin(condition=[$4], joinType=[semi]) +- LogicalAggregate(group=[{}], m=[MIN($0)]) +- LogicalCalc(expr#0=[{inputs}], expr#1=[true], i=[$t1]) +- LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) - +- LogicalValues(type=[RecordType(INTEGER $f0)], tuples=[[]]) + +- LogicalValues(tuples=[[]]) ]]> @@ -188,7 +188,7 @@ LogicalProject(a1=[$0], c1=[$2]) ($t1, $t2)], a1=[$t0], c1=[$t1], $condition=[$t3]) +LogicalCalc(expr#0..1=[{inputs}], expr#2=[10], expr#3=[>($t1, $t2)], proj#0..1=[{exprs}], $condition=[$t3]) +- LogicalAggregate(group=[{0}], c1=[COUNT($2)]) +- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]]) ]]> @@ -279,7 +279,7 @@ LogicalProject(a2=[$0], b2=[$1], d2=[$3]) ($t2, $t3)], a2=[$t0], b2=[$t1], d2=[$t2], $condition=[$t4]) +LogicalCalc(expr#0..2=[{inputs}], expr#3=[0], expr#4=[>($t2, $t3)], proj#0..2=[{exprs}], $condition=[$t4]) +- LogicalAggregate(group=[{0, 1}], d2=[SUM($3)]) +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2, d2)]]]) ]]> @@ -368,7 +368,7 @@ LogicalProject(a=[$0], b2=[$1], d2=[$3]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkLimit0RemoveRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkLimit0RemoveRuleTest.xml index 1102b0aef4..649de95ccc 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkLimit0RemoveRuleTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkLimit0RemoveRuleTest.xml @@ -39,7 +39,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalProject($f0=[IS NOT NULL($0)]) +- LogicalAggregate(group=[{}], m=[MIN($0)]) +- LogicalProject(i=[true]) - +- LogicalValues(type=[RecordType(INTEGER a)], tuples=[[]]) + +- LogicalValues(tuples=[[]]) ]]> @@ -63,7 +63,7 @@ LogicalSort(fetch=[0]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[OR(=($0, $3), IS NULL($0), IS NULL($3))], joinType=[anti]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalValues(type=[RecordType(INTEGER a)], tuples=[[]]) + +- LogicalValues(tuples=[[]]) ]]> @@ -87,7 +87,7 @@ LogicalSort(fetch=[0]) LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalJoin(condition=[=($0, $3)], joinType=[semi]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalValues(type=[RecordType(INTEGER a)], tuples=[[]]) + +- LogicalValues(tuples=[[]]) ]]> @@ -110,7 +110,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], a0=[$3], b0=[$4], c0=[$5]) LogicalProject(a=[$0], b=[$1], c=[$2], a0=[$3], b0=[$4], c0=[$5]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalValues(type=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c)], tuples=[[]]) + +- LogicalValues(tuples=[[]]) ]]> @@ -137,7 +137,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalProject($f0=[IS NOT NULL($0)]) +- LogicalAggregate(group=[{}], m=[MIN($0)]) +- LogicalProject(i=[true]) - +- LogicalValues(type=[RecordType(INTEGER a)], tuples=[[]]) + +- LogicalValues(tuples=[[]]) ]]> @@ -154,7 +154,7 @@ LogicalSort(fetch=[0]) @@ -171,7 +171,7 @@ LogicalSort(sort0=[$0], dir0=[ASC-nulls-first], offset=[10], fetch=[0]) @@ -190,7 +190,7 @@ LogicalProject(a=[$0]) @@ -207,7 +207,7 @@ LogicalSort(sort0=[$0], dir0=[ASC-nulls-first], fetch=[0]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkPruneEmptyRulesTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkPruneEmptyRulesTest.xml index cc063ddea7..d32afbb4c2 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkPruneEmptyRulesTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkPruneEmptyRulesTest.xml @@ -34,7 +34,7 @@ LogicalProject(d=[$0]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.xml index adb53b9a35..0a353383ae 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.xml @@ -28,7 +28,7 @@ LogicalProject(a=[$0], s=[$3]) +- LogicalProject(s=[$0]) +- Uncollect +- LogicalProject(c=[$cor0.c]) - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + +- LogicalValues(tuples=[[{ 0 }]]) ]]> @@ -58,12 +58,12 @@ LogicalProject(b=[$0], s=[$2]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) :- LogicalProject(b=[$0], set=[$2]) : +- LogicalAggregate(group=[{0, 1}], set=[COLLECT($0)]) - : +- LogicalProject(b=[$1], $f1=[TUMBLE($3, 3000:INTERVAL SECOND)]) + : +- LogicalProject(b=[$1], $f1=[$TUMBLE($3, 3000:INTERVAL SECOND)]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(s=[$0]) +- Uncollect +- LogicalProject(set=[$cor0.set]) - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + +- LogicalValues(tuples=[[{ 0 }]]) ]]> @@ -73,7 +73,7 @@ LogicalProject(b=[$0], s=[$2]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) :- LogicalProject(b=[$0], set=[$2]) : +- LogicalAggregate(group=[{0, 1}], set=[COLLECT($0)]) - : +- LogicalProject(b=[$1], $f1=[TUMBLE($3, 3000:INTERVAL SECOND)]) + : +- LogicalProject(b=[$1], $f1=[$TUMBLE($3, 3000:INTERVAL SECOND)]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(s=[$0]) +- LogicalTableFunctionScan(invocation=[explode($cor0.set)], rowType=[RecordType(BIGINT f0)], elementType=[class [Ljava.lang.Object;]) @@ -92,7 +92,7 @@ LogicalProject(a=[$0], b=[$1], v=[$4]) +- LogicalProject(k=[$0], v=[$1]) +- Uncollect +- LogicalProject(c=[$cor0.c]) - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + +- LogicalValues(tuples=[[{ 0 }]]) ]]> @@ -125,7 +125,7 @@ LogicalProject(a=[$0], b=[$1], x=[$2], y=[$3]) +- LogicalProject(x=[$0], y=[$1]) +- Uncollect +- LogicalProject(b=[$cor0.b]) - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + +- LogicalValues(tuples=[[{ 0 }]]) ]]> @@ -159,7 +159,7 @@ LogicalProject(a=[$0], s=[$2]) +- LogicalProject(s=[$0]) +- Uncollect +- LogicalProject(set=[$cor0.set]) - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + +- LogicalValues(tuples=[[{ 0 }]]) ]]> @@ -187,7 +187,7 @@ LogicalProject(a=[$0], s=[$3]) +- LogicalProject(s=[$0]) +- Uncollect +- LogicalProject(c=[$cor0.c]) - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + +- LogicalValues(tuples=[[{ 0 }]]) ]]> @@ -218,7 +218,7 @@ LogicalProject(b=[$0], id=[$2], point=[$3]) +- LogicalProject(id=[$0], point=[$1]) +- Uncollect +- LogicalProject(set=[$cor0.set]) - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + +- LogicalValues(tuples=[[{ 0 }]]) ]]> @@ -246,7 +246,7 @@ LogicalProject(a=[$0], b=[$1], s=[$3]) +- LogicalProject(s=[$0]) +- Uncollect +- LogicalProject(b=[$cor0.b]) - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + +- LogicalValues(tuples=[[{ 0 }]]) ]]> @@ -272,7 +272,7 @@ LogicalProject(a=[$0], b=[$1], s=[$2], t=[$3]) +- LogicalProject(s=[$0], t=[$1]) +- Uncollect +- LogicalProject(b=[$cor0.b]) - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + +- LogicalValues(tuples=[[{ 0 }]]) ]]> @@ -298,7 +298,7 @@ LogicalProject(a=[$0], b=[$1], _1=[$2], _2=[$3]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]]) +- Uncollect +- LogicalProject(b=[$cor0.b]) - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + +- LogicalValues(tuples=[[{ 0 }]]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ProjectPruneAggregateCallRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ProjectPruneAggregateCallRuleTest.xml index 3ded214ade..3df36ebbe4 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ProjectPruneAggregateCallRuleTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ProjectPruneAggregateCallRuleTest.xml @@ -89,7 +89,7 @@ LogicalJoin(condition=[$4], joinType=[semi]) +- LogicalAggregate(group=[{}], m=[MIN($0)]) +- LogicalProject(i=[true]) +- LogicalAggregate(group=[{}], EXPR$0=[SUM($0)]) - +- LogicalValues(type=[RecordType(INTEGER a1)], tuples=[[]]) + +- LogicalValues(tuples=[[]]) ]]> @@ -166,7 +166,7 @@ LogicalJoin(condition=[$4], joinType=[semi]) +- LogicalAggregate(group=[{}], m=[MIN($0)]) +- LogicalProject(i=[true]) +- LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) - +- LogicalValues(type=[RecordType(INTEGER $f0)], tuples=[[]]) + +- LogicalValues(tuples=[[]]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQuerySemiJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQuerySemiJoinTest.xml index 97b419b754..2a96d4debd 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQuerySemiJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQuerySemiJoinTest.xml @@ -586,7 +586,7 @@ LogicalFilter(condition=[=($0, $cor0.a)]) LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) })], variablesSet=[[$cor0]]) +- LogicalProject(EXPR$0=[$0], EXPR$1=[$1]) - +- LogicalValues(type=[RecordType(INTEGER EXPR$0, INTEGER EXPR$1)], tuples=[[{ 1, 2 }]]) + +- LogicalValues(tuples=[[{ 1, 2 }]]) ]]> @@ -594,7 +594,7 @@ LogicalFilter(condition=[=($0, $cor0.a)]) LogicalProject(a=[$0], b=[$1]) +- LogicalJoin(condition=[=($2, $0)], joinType=[semi]) :- LogicalProject(EXPR$0=[$0], EXPR$1=[$1]) - : +- LogicalValues(type=[RecordType(INTEGER EXPR$0, INTEGER EXPR$1)], tuples=[[{ 1, 2 }]]) + : +- LogicalValues(tuples=[[{ 1, 2 }]]) +- LogicalProject(c=[$0]) +- LogicalFilter(condition=[true]) +- LogicalTableScan(table=[[default_catalog, default_database, y, source: [TestTableSource(c, d)]]]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/LimitTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/LimitTest.xml index d9904a3f1a..607f9f4527 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/LimitTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/LimitTest.xml @@ -29,7 +29,7 @@ LogicalSort(fetch=[0]) @@ -86,7 +86,7 @@ LogicalSort(offset=[10], fetch=[0]) @@ -103,7 +103,7 @@ LogicalSort(offset=[0], fetch=[0]) @@ -120,7 +120,7 @@ LogicalSort(fetch=[0]) @@ -137,7 +137,7 @@ LogicalSort(fetch=[0]) @@ -154,7 +154,7 @@ LogicalSort(fetch=[0]) @@ -171,7 +171,7 @@ LogicalSort(fetch=[0]) @@ -188,7 +188,7 @@ LogicalSort(fetch=[0]) @@ -205,7 +205,7 @@ LogicalSort(fetch=[0]) @@ -222,7 +222,7 @@ LogicalSort(fetch=[0]) @@ -239,7 +239,7 @@ LogicalSort(fetch=[0]) @@ -256,7 +256,7 @@ LogicalSort(fetch=[0]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SetOperatorsTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SetOperatorsTest.xml index 2249b4ae59..8c4a2f395b 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SetOperatorsTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SetOperatorsTest.xml @@ -87,7 +87,7 @@ LogicalIntersect(all=[false]) @@ -107,7 +107,7 @@ LogicalIntersect(all=[false]) @@ -211,7 +211,7 @@ LogicalMinus(all=[false]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SortLimitTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SortLimitTest.xml index b9ce5197a6..03d9b69589 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SortLimitTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SortLimitTest.xml @@ -217,7 +217,7 @@ LogicalProject(a=[$0]) @@ -256,7 +256,7 @@ LogicalProject(a=[$0]) @@ -295,7 +295,7 @@ LogicalProject(a=[$0]) @@ -418,7 +418,7 @@ LogicalProject(a=[$0]) @@ -562,7 +562,7 @@ LogicalProject(a=[$0]) @@ -601,7 +601,7 @@ LogicalProject(a=[$0]) @@ -640,7 +640,7 @@ LogicalProject(a=[$0]) @@ -763,7 +763,7 @@ LogicalProject(a=[$0]) @@ -802,7 +802,7 @@ LogicalProject(a=[$0]) @@ -841,7 +841,7 @@ LogicalProject(a=[$0]) @@ -880,7 +880,7 @@ LogicalProject(a=[$0]) @@ -919,7 +919,7 @@ LogicalProject(a=[$0]) @@ -958,7 +958,7 @@ LogicalProject(a=[$0]) @@ -997,7 +997,7 @@ LogicalProject(a=[$0]) @@ -1036,7 +1036,7 @@ LogicalProject(a=[$0]) @@ -1180,7 +1180,7 @@ LogicalProject(a=[$0]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnnestTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnnestTest.xml index 09ef6c8c9e..b7aba4dd83 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnnestTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnnestTest.xml @@ -28,7 +28,7 @@ LogicalProject(a=[$0], s=[$3]) +- LogicalProject(s=[$0]) +- Uncollect +- LogicalProject(c=[$cor0.c]) - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + +- LogicalValues(tuples=[[{ 0 }]]) ]]> @@ -56,12 +56,12 @@ LogicalProject(b=[$0], s=[$2]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1}]) :- LogicalProject(b=[$0], set=[$2]) : +- LogicalAggregate(group=[{0, 1}], set=[COLLECT($0)]) - : +- LogicalProject(b=[$1], $f1=[TUMBLE($3, 3000:INTERVAL SECOND)]) + : +- LogicalProject(b=[$1], $f1=[$TUMBLE($3, 3000:INTERVAL SECOND)]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(s=[$0]) +- Uncollect +- LogicalProject(set=[$cor0.set]) - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + +- LogicalValues(tuples=[[{ 0 }]]) ]]> @@ -87,7 +87,7 @@ LogicalProject(a=[$0], b=[$1], v=[$4]) +- LogicalProject(k=[$0], v=[$1]) +- Uncollect +- LogicalProject(c=[$cor0.c]) - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + +- LogicalValues(tuples=[[{ 0 }]]) ]]> @@ -118,7 +118,7 @@ LogicalProject(a=[$0], b=[$1], x=[$2], y=[$3]) +- LogicalProject(x=[$0], y=[$1]) +- Uncollect +- LogicalProject(b=[$cor0.b]) - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + +- LogicalValues(tuples=[[{ 0 }]]) ]]> @@ -148,7 +148,7 @@ LogicalProject(a=[$0], s=[$2]) +- LogicalProject(s=[$0]) +- Uncollect +- LogicalProject(set=[$cor0.set]) - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + +- LogicalValues(tuples=[[{ 0 }]]) ]]> @@ -174,7 +174,7 @@ LogicalProject(a=[$0], s=[$3]) +- LogicalProject(s=[$0]) +- Uncollect +- LogicalProject(c=[$cor0.c]) - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + +- LogicalValues(tuples=[[{ 0 }]]) ]]> @@ -203,7 +203,7 @@ LogicalProject(b=[$0], id=[$2], point=[$3]) +- LogicalProject(id=[$0], point=[$1]) +- Uncollect +- LogicalProject(set=[$cor0.set]) - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + +- LogicalValues(tuples=[[{ 0 }]]) ]]> @@ -229,7 +229,7 @@ LogicalProject(a=[$0], b=[$1], s=[$3]) +- LogicalProject(s=[$0]) +- Uncollect +- LogicalProject(b=[$cor0.b]) - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + +- LogicalValues(tuples=[[{ 0 }]]) ]]> @@ -253,7 +253,7 @@ LogicalProject(a=[$0], b=[$1], s=[$2], t=[$3]) +- LogicalProject(s=[$0], t=[$1]) +- Uncollect +- LogicalProject(b=[$cor0.b]) - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + +- LogicalValues(tuples=[[{ 0 }]]) ]]> @@ -275,7 +275,7 @@ LogicalProject(a=[$0], b=[$1], _1=[$2], _2=[$3]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]]) +- Uncollect +- LogicalProject(b=[$cor0.b]) - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + +- LogicalValues(tuples=[[{ 0 }]]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/ValuesTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/ValuesTest.xml index 6f7026f8d3..d2ec0f4e05 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/ValuesTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/ValuesTest.xml @@ -25,16 +25,16 @@ limitations under the License. LogicalProject(a=[$0], b=[$1]), rowType=[RecordType(INTEGER a, DECIMAL(20, 1) b)] +- LogicalUnion(all=[true]), rowType=[RecordType(INTEGER EXPR$0, DECIMAL(20, 1) EXPR$1)] :- LogicalProject(EXPR$0=[1], EXPR$1=[2.0:DECIMAL(2, 1)]), rowType=[RecordType(INTEGER EXPR$0, DECIMAL(2, 1) EXPR$1)] - : +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]), rowType=[RecordType(INTEGER ZERO)] + : +- LogicalValues(tuples=[[{ 0 }]]), rowType=[RecordType(INTEGER ZERO)] +- LogicalProject(EXPR$0=[3], EXPR$1=[4:BIGINT]), rowType=[RecordType(INTEGER EXPR$0, BIGINT EXPR$1)] - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]), rowType=[RecordType(INTEGER ZERO)] + +- LogicalValues(tuples=[[{ 0 }]]), rowType=[RecordType(INTEGER ZERO)] ]]> @@ -47,12 +47,12 @@ Union(all=[true], union=[EXPR$0, EXPR$1]), rowType=[RecordType(INTEGER EXPR$0, D @@ -65,18 +65,18 @@ Values(type=[RecordType(INTEGER a, INTEGER b, INTEGER c)], tuples=[[{ 1, 2, 3 }] LogicalProject(a=[$0], b=[$1]) +- LogicalUnion(all=[true]) :- LogicalProject(EXPR$0=[1], EXPR$1=[2]) - : +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + : +- LogicalValues(tuples=[[{ 0 }]]) :- LogicalProject(EXPR$0=[3], EXPR$1=[null:INTEGER]) - : +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + : +- LogicalValues(tuples=[[{ 0 }]]) +- LogicalProject(EXPR$0=[4], EXPR$1=[5]) - +- LogicalValues(type=[RecordType(INTEGER ZERO)], tuples=[[{ 0 }]]) + +- LogicalValues(tuples=[[{ 0 }]]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.xml index 49b6a40683..a1d54d3bc0 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.xml @@ -1075,7 +1075,7 @@ Join(joinType=[LeftOuterJoin], where=[=(key, key0)], select=[key, v, key0, v0], : +- Calc(select=[CAST(0:BIGINT) AS key, v], where=[=(key, 0:BIGINT)], updateAsRetraction=[true], accMode=[Acc]) : +- TableSourceScan(table=[[default_catalog, default_database, src, source: [TestTableSource(key, v)]]], fields=[key, v], updateAsRetraction=[true], accMode=[Acc]) +- Exchange(distribution=[hash[key]], updateAsRetraction=[true], accMode=[Acc]) - +- Values(type=[RecordType(BIGINT key, VARCHAR(2147483647) v)], tuples=[[]], updateAsRetraction=[true], accMode=[Acc]) + +- Values(tuples=[[]], updateAsRetraction=[true], accMode=[Acc]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/WindowJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/WindowJoinTest.xml index ba5ea525da..334444bb8d 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/WindowJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/WindowJoinTest.xml @@ -99,7 +99,7 @@ LogicalProject(a=[$5], c=[$7], c0=[$2]) @@ -208,7 +208,7 @@ GROUP BY TUMBLE(t1.rowtime, INTERVAL '6' HOUR), t1.b =($4, -($9, 600000:INTERVAL MINUTE)), <=($4, +($9, 3600000:INTERVAL HOUR)))]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) @@ -394,7 +394,7 @@ GROUP BY TUMBLE(t2.rowtime, INTERVAL '6' HOUR), t2.b =($4, -($9, 600000:INTERVAL MINUTE)), <=($4, +($9, 3600000:INTERVAL HOUR)))]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) -- Gitee From 333885a5014bdd09bfe8ea6decfcd2c785c7603c Mon Sep 17 00:00:00 2001 From: "yuzhao.cyz" Date: Tue, 17 Mar 2020 21:22:30 +0800 Subject: [PATCH 374/885] [FLINK-14338][table-planner][table-planner-blink] Update files due to builtin TUMBLE operator name changes to $Tumble * This change was introduced in CALCITE-3382 --- .../functions/sql/FlinkSqlOperatorTable.java | 3 +- .../flink/table/api/stream/ExplainTest.xml | 4 +- .../plan/batch/sql/DagOptimizationTest.xml | 4 +- .../sql/agg/AggregateReduceGroupingTest.xml | 38 +- .../batch/sql/agg/WindowAggregateTest.xml | 452 +++++++++--------- .../AggregateReduceGroupingRuleTest.xml | 42 +- .../stream/sql/MiniBatchIntervalInferTest.xml | 16 +- .../sql/RelTimeIndicatorConverterTest.xml | 12 +- .../plan/stream/sql/TableSourceTest.xml | 2 +- .../stream/sql/agg/WindowAggregateTest.xml | 216 ++++----- .../table/catalog/BasicOperatorTable.scala | 2 + 11 files changed, 376 insertions(+), 415 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java index 6cae5de44c..7ce4bd493d 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java @@ -776,7 +776,8 @@ public class FlinkSqlOperatorTable extends ReflectiveSqlOperatorTable { * We need custom group auxiliary functions in order to support nested windows. */ public static final SqlGroupedWindowFunction TUMBLE = new SqlGroupedWindowFunction( - SqlKind.TUMBLE, null, + // The TUMBLE group function was hard code to $TUMBLE in CALCITE-3382. + "$TUMBLE", SqlKind.TUMBLE, null, OperandTypes.or(OperandTypes.DATETIME_INTERVAL, OperandTypes.DATETIME_INTERVAL_TIME)) { @Override public List getAuxiliaryFunctions() { diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/stream/ExplainTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/stream/ExplainTest.xml index b90451f696..7bdd8b836f 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/stream/ExplainTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/stream/ExplainTest.xml @@ -638,7 +638,7 @@ SortLimit(orderBy=[a ASC], offset=[0], fetch=[5], updateAsRetraction=[false], ac LogicalSink(name=[`default_catalog`.`default_database`.`appendSink1`], fields=[a, b]) +- LogicalProject(id1=[$0], EXPR$1=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$1=[LISTAGG($2, $3)]) - +- LogicalProject(id1=[$0], $f1=[TUMBLE($2, 8000:INTERVAL SECOND)], text=[$1], $f3=[_UTF-16LE'#']) + +- LogicalProject(id1=[$0], $f1=[$TUMBLE($2, 8000:INTERVAL SECOND)], text=[$1], $f3=[_UTF-16LE'#']) +- LogicalFilter(condition=[AND(=($0, $3), >($2, -($7, 300000:INTERVAL MINUTE)), <($2, +($7, 180000:INTERVAL MINUTE)))]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($2, 0:INTERVAL MILLISECOND)]) @@ -778,7 +778,7 @@ Union(all=[true], union=[a, b, c]) LogicalSink(name=[`default_catalog`.`default_database`.`appendSink1`], fields=[a, b]) +- LogicalProject(id1=[$0], EXPR$1=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$1=[LISTAGG($2, $3)]) - +- LogicalProject(id1=[$0], $f1=[TUMBLE($2, 8000:INTERVAL SECOND)], text=[$1], $f3=[_UTF-16LE'#']) + +- LogicalProject(id1=[$0], $f1=[$TUMBLE($2, 8000:INTERVAL SECOND)], text=[$1], $f3=[_UTF-16LE'#']) +- LogicalFilter(condition=[AND(=($0, $3), >($2, -($7, 300000:INTERVAL MINUTE)), <($2, +($7, 180000:INTERVAL MINUTE)))]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($2, 0:INTERVAL MILLISECOND)]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DagOptimizationTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DagOptimizationTest.xml index 7fe88c8613..f48686c4fb 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DagOptimizationTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DagOptimizationTest.xml @@ -570,13 +570,13 @@ Sink(name=[`default_catalog`.`default_database`.`sink2`], fields=[total_min]) LogicalSink(name=[`default_catalog`.`default_database`.`sink1`], fields=[a, sum_c, time, window_start, window_end]) +- LogicalProject(a=[$1], sum_c=[$2], time=[TUMBLE_END($0)], window_start=[TUMBLE_START($0)], window_end=[TUMBLE_END($0)]) +- LogicalAggregate(group=[{0, 1}], sum_c=[SUM($2)]) - +- LogicalProject($f0=[TUMBLE($3, 15000:INTERVAL SECOND)], a=[$0], $f2=[CAST($2):DOUBLE]) + +- LogicalProject($f0=[$TUMBLE($3, 15000:INTERVAL SECOND)], a=[$0], $f2=[CAST($2):DOUBLE]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, ts)]]]) LogicalSink(name=[`default_catalog`.`default_database`.`sink2`], fields=[a, sum_c, time]) +- LogicalProject(a=[$1], sum_c=[$2], time=[TUMBLE_END($0)]) +- LogicalAggregate(group=[{0, 1}], sum_c=[SUM($2)]) - +- LogicalProject($f0=[TUMBLE($3, 15000:INTERVAL SECOND)], a=[$0], $f2=[CAST($2):DOUBLE]) + +- LogicalProject($f0=[$TUMBLE($3, 15000:INTERVAL SECOND)], a=[$0], $f2=[CAST($2):DOUBLE]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, ts)]]]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/AggregateReduceGroupingTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/AggregateReduceGroupingTest.xml index aa9e8ccd77..3dab6fa988 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/AggregateReduceGroupingTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/AggregateReduceGroupingTest.xml @@ -359,7 +359,7 @@ HashAggregate(isMerge=[true], groupBy=[a3, b3, a1], auxGrouping=[b1], select=[a3 @@ -379,7 +379,7 @@ HashWindowAggregate(groupBy=[a4], auxGrouping=[b4], window=[TumblingGroupWindow( @@ -400,7 +400,7 @@ HashWindowAggregate(groupBy=[a4], auxGrouping=[c4], window=[TumblingGroupWindow( LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT($3)]) +- LogicalProject(a4=[$0], c4=[$1], s=[TUMBLE_START($2)], b4=[$3]) +- LogicalAggregate(group=[{0, 1, 2}], b4=[VAR_POP($3)]) - +- LogicalProject(a4=[$0], c4=[$2], $f2=[TUMBLE($3, 900000:INTERVAL MINUTE)], b4=[$1]) + +- LogicalProject(a4=[$0], c4=[$2], $f2=[$TUMBLE($3, 900000:INTERVAL MINUTE)], b4=[$1]) +- LogicalTableScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4, d4)]]]) ]]> @@ -427,7 +427,7 @@ Calc(select=[a4, c4, s, EXPR$3]) LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT($3)]) +- LogicalProject(a4=[$0], c4=[$1], e=[TUMBLE_END($2)], b4=[$3]) +- LogicalAggregate(group=[{0, 1, 2}], b4=[VAR_POP($3)]) - +- LogicalProject(a4=[$0], c4=[$2], $f2=[TUMBLE($3, 900000:INTERVAL MINUTE)], b4=[$1]) + +- LogicalProject(a4=[$0], c4=[$2], $f2=[$TUMBLE($3, 900000:INTERVAL MINUTE)], b4=[$1]) +- LogicalTableScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4, d4)]]]) ]]> @@ -454,7 +454,7 @@ Calc(select=[a4, c4, e, EXPR$3]) LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT()]) +- LogicalProject(a4=[$0], b4=[$3], c4=[$1]) +- LogicalAggregate(group=[{0, 1, 2}], b4=[VAR_POP($3)]) - +- LogicalProject(a4=[$0], c4=[$2], $f2=[TUMBLE($3, 900000:INTERVAL MINUTE)], b4=[$1]) + +- LogicalProject(a4=[$0], c4=[$2], $f2=[$TUMBLE($3, 900000:INTERVAL MINUTE)], b4=[$1]) +- LogicalTableScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4, d4)]]]) ]]> @@ -487,7 +487,7 @@ Calc(select=[a1, b1, c1, EXPR$3]) +- HashAggregate(isMerge=[true], groupBy=[a1, b1, c1, $e], select=[a1, b1, c1, $e, Final_COUNT(count$0) AS EXPR$3]) +- Exchange(distribution=[hash[a1, b1, c1, $e]]) +- LocalHashAggregate(groupBy=[a1, b1, c1, $e], select=[a1, b1, c1, $e, Partial_COUNT(d1) AS count$0]) - +- Expand(projects=[{a1=[$0], b1=[$1], c1=[$2], d1=[$3], $e=[0]}, {a1=[$0], b1=[$1], c1=[null], d1=[$3], $e=[1]}, {a1=[$0], b1=[null], c1=[$2], d1=[$3], $e=[2]}, {a1=[$0], b1=[null], c1=[null], d1=[$3], $e=[3]}, {a1=[null], b1=[$1], c1=[$2], d1=[$3], $e=[4]}, {a1=[null], b1=[$1], c1=[null], d1=[$3], $e=[5]}, {a1=[null], b1=[null], c1=[$2], d1=[$3], $e=[6]}, {a1=[null], b1=[null], c1=[null], d1=[$3], $e=[7]}], projects=[{a1, b1, c1, d1, 0 AS $e}, {a1, b1, null AS c1, d1, 1 AS $e}, {a1, null AS b1, c1, d1, 2 AS $e}, {a1, null AS b1, null AS c1, d1, 3 AS $e}, {null AS a1, b1, c1, d1, 4 AS $e}, {null AS a1, b1, null AS c1, d1, 5 AS $e}, {null AS a1, null AS b1, c1, d1, 6 AS $e}, {null AS a1, null AS b1, null AS c1, d1, 7 AS $e}]) + +- Expand(projects=[a1, b1, c1, d1, $e], projects=[{a1, b1, c1, d1, 0 AS $e}, {a1, b1, null AS c1, d1, 1 AS $e}, {a1, null AS b1, c1, d1, 2 AS $e}, {a1, null AS b1, null AS c1, d1, 3 AS $e}, {null AS a1, b1, c1, d1, 4 AS $e}, {null AS a1, b1, null AS c1, d1, 5 AS $e}, {null AS a1, null AS b1, c1, d1, 6 AS $e}, {null AS a1, null AS b1, null AS c1, d1, 7 AS $e}]) +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1]) ]]> @@ -507,7 +507,7 @@ LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1}, {0, 2}]], EXPR$3=[COUNT($3) Calc(select=[a1, b1, c1, EXPR$3]) +- HashAggregate(isMerge=[false], groupBy=[a1, b1, c1, $e], select=[a1, b1, c1, $e, COUNT(d1) AS EXPR$3]) +- Exchange(distribution=[hash[a1, b1, c1, $e]]) - +- Expand(projects=[{a1=[$0], b1=[$1], c1=[null], d1=[$3], $e=[1]}, {a1=[$0], b1=[null], c1=[$2], d1=[$3], $e=[2]}], projects=[{a1, b1, null AS c1, d1, 1 AS $e}, {a1, null AS b1, c1, d1, 2 AS $e}]) + +- Expand(projects=[a1, b1, c1, d1, $e], projects=[{a1, b1, null AS c1, d1, 1 AS $e}, {a1, null AS b1, c1, d1, 2 AS $e}]) +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1]) ]]> @@ -530,7 +530,7 @@ Calc(select=[a1, s]) +- HashAggregate(isMerge=[true], groupBy=[a1, c1, $e], select=[a1, c1, $e, Final_SUM(sum$0) AS s]) +- Exchange(distribution=[hash[a1, c1, $e]]) +- LocalHashAggregate(groupBy=[a1, c1, $e], select=[a1, c1, $e, Partial_SUM(b1) AS sum$0]) - +- Expand(projects=[{a1=[$0], c1=[$1], b1=[$2], $e=[0]}, {a1=[$0], c1=[null], b1=[$2], $e=[1]}, {a1=[null], c1=[null], b1=[$2], $e=[3]}], projects=[{a1, c1, b1, 0 AS $e}, {a1, null AS c1, b1, 1 AS $e}, {null AS a1, null AS c1, b1, 3 AS $e}]) + +- Expand(projects=[a1, c1, b1, $e], projects=[{a1, c1, b1, 0 AS $e}, {a1, null AS c1, b1, 1 AS $e}, {null AS a1, null AS c1, b1, 3 AS $e}]) +- Calc(select=[a1, c1, b1]) +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1]) ]]> @@ -552,7 +552,7 @@ LogicalProject(a1=[$0], b1=[$1], c1=[$2], EXPR$3=[$4]) Calc(select=[a1, b1, c1, EXPR$3]) +- HashAggregate(isMerge=[false], groupBy=[a1, c1, d1, $e], auxGrouping=[b1], select=[a1, c1, d1, $e, b1, COUNT(d1_0) AS EXPR$3]) +- Exchange(distribution=[hash[a1, c1, d1, $e]]) - +- Expand(projects=[{a1=[$0], b1=[$1], c1=[$2], d1=[null], $e=[1], d1_0=[$3]}, {a1=[$0], b1=[$1], c1=[null], d1=[$3], $e=[2], d1_0=[$3]}], projects=[{a1, b1, c1, null AS d1, 1 AS $e, d1 AS d1_0}, {a1, b1, null AS c1, d1, 2 AS $e, d1 AS d1_0}]) + +- Expand(projects=[a1, b1, c1, d1, $e, d1_0], projects=[{a1, b1, c1, null AS d1, 1 AS $e, d1 AS d1_0}, {a1, b1, null AS c1, d1, 2 AS $e, d1 AS d1_0}]) +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1]) ]]> @@ -614,7 +614,7 @@ Calc(select=[a1, b1, c1, EXPR$3]) +- HashAggregate(isMerge=[true], groupBy=[a1, b1, c1, $e], select=[a1, b1, c1, $e, Final_COUNT(count$0) AS EXPR$3]) +- Exchange(distribution=[hash[a1, b1, c1, $e]]) +- LocalHashAggregate(groupBy=[a1, b1, c1, $e], select=[a1, b1, c1, $e, Partial_COUNT(d1) AS count$0]) - +- Expand(projects=[{a1=[$0], b1=[$1], c1=[$2], d1=[$3], $e=[0]}, {a1=[$0], b1=[$1], c1=[null], d1=[$3], $e=[1]}, {a1=[$0], b1=[null], c1=[null], d1=[$3], $e=[3]}, {a1=[null], b1=[null], c1=[null], d1=[$3], $e=[7]}], projects=[{a1, b1, c1, d1, 0 AS $e}, {a1, b1, null AS c1, d1, 1 AS $e}, {a1, null AS b1, null AS c1, d1, 3 AS $e}, {null AS a1, null AS b1, null AS c1, d1, 7 AS $e}]) + +- Expand(projects=[a1, b1, c1, d1, $e], projects=[{a1, b1, c1, d1, 0 AS $e}, {a1, b1, null AS c1, d1, 1 AS $e}, {a1, null AS b1, null AS c1, d1, 3 AS $e}, {null AS a1, null AS b1, null AS c1, d1, 7 AS $e}]) +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1]) ]]> @@ -697,7 +697,7 @@ LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT(DISTINCT $2)], EXPR$3=[SUM(DISTIN HashAggregate(isMerge=[false], groupBy=[a1], auxGrouping=[d1], select=[a1, d1, COUNT(c1) FILTER $g_4 AS EXPR$2, SUM(b1) FILTER $g_2 AS EXPR$3]) +- Exchange(distribution=[hash[a1]]) +- Calc(select=[a1, b1, c1, d1, =(CASE(=($e, 2:BIGINT), 2:BIGINT, 4:BIGINT), 2) AS $g_2, =(CASE(=($e, 2:BIGINT), 2:BIGINT, 4:BIGINT), 4) AS $g_4]) - +- Expand(projects=[{a1=[$0], b1=[$1], c1=[null], d1=[$3], $e=[2]}, {a1=[$0], b1=[null], c1=[$2], d1=[$3], $e=[4]}], projects=[{a1, b1, null AS c1, d1, 2 AS $e}, {a1, null AS b1, c1, d1, 4 AS $e}]) + +- Expand(projects=[a1, b1, c1, d1, $e], projects=[{a1, b1, null AS c1, d1, 2 AS $e}, {a1, null AS b1, c1, d1, 4 AS $e}]) +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1]) ]]> @@ -715,7 +715,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[SUM(DISTINCT $1)], EXPR$2=[MAX($1)], EXPR$ @@ -737,7 +737,7 @@ SortAggregate(isMerge=[false], groupBy=[a1], select=[a1, MIN(d1) FILTER $g_1 AS +- Calc(select=[a1, c1, d1, b1 AS EXPR$3, b1 AS EXPR$4, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1]) +- Sort(orderBy=[a1 ASC]) +- Exchange(distribution=[hash[a1]]) - +- Expand(projects=[{a1=[$0], b1=[$1], c1=[$2], d1=[$3], $e=[0]}, {a1=[$0], b1=[$1], c1=[null], d1=[$3], $e=[1]}], projects=[{a1, b1, c1, d1, 0 AS $e}, {a1, b1, null AS c1, d1, 1 AS $e}]) + +- Expand(projects=[a1, b1, c1, d1, $e], projects=[{a1, b1, c1, d1, 0 AS $e}, {a1, b1, null AS c1, d1, 1 AS $e}]) +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1]) ]]> @@ -760,7 +760,7 @@ Calc(select=[EXPR$0]) +- HashAggregate(isMerge=[true], groupBy=[$f0], select=[$f0, Final_COUNT(count$0) AS EXPR$0]) +- Exchange(distribution=[hash[$f0]]) +- LocalHashAggregate(groupBy=[$f0], select=[$f0, Partial_COUNT(c1) AS count$0]) - +- Calc(select=[1 AS $f0, true AS $f1, c1]) + +- Calc(select=[1 AS $f0, c1]) +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1]) ]]> @@ -841,7 +841,7 @@ LogicalProject(a1=[$0], b1=[$1], EXPR$2=[$4]) @@ -902,7 +902,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1)], EXPR$2=[SUM($1)]) HashAggregate(isMerge=[false], groupBy=[a1], select=[a1, COUNT(b1) FILTER $g_0 AS EXPR$1, MIN(EXPR$2) FILTER $g_1 AS EXPR$2]) +- Exchange(distribution=[hash[a1]]) +- Calc(select=[a1, b1, b1_0 AS EXPR$2, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1]) - +- Expand(projects=[{a1=[$0], b1=[$1], c1=[$2], d1=[$3], $e=[0], b1_0=[$1]}, {a1=[$0], b1=[null], c1=[$2], d1=[$3], $e=[1], b1_0=[$1]}], projects=[{a1, b1, c1, d1, 0 AS $e, b1 AS b1_0}, {a1, null AS b1, c1, d1, 1 AS $e, b1 AS b1_0}]) + +- Expand(projects=[a1, b1, c1, d1, $e, b1_0], projects=[{a1, b1, c1, d1, 0 AS $e, b1 AS b1_0}, {a1, null AS b1, c1, d1, 1 AS $e, b1 AS b1_0}]) +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1]) ]]> @@ -924,7 +924,7 @@ SortAggregate(isMerge=[false], groupBy=[a1], select=[a1, MIN(c1) FILTER $g_1 AS +- Sort(orderBy=[a1 ASC]) +- Exchange(distribution=[hash[a1]]) +- Calc(select=[a1, b1, c1, b1_0 AS EXPR$3, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1]) - +- Expand(projects=[{a1=[$0], b1=[$1], c1=[$2], d1=[$3], $e=[0], b1_0=[$1]}, {a1=[$0], b1=[null], c1=[$2], d1=[$3], $e=[1], b1_0=[$1]}], projects=[{a1, b1, c1, d1, 0 AS $e, b1 AS b1_0}, {a1, null AS b1, c1, d1, 1 AS $e, b1 AS b1_0}]) + +- Expand(projects=[a1, b1, c1, d1, $e, b1_0], projects=[{a1, b1, c1, d1, 0 AS $e, b1 AS b1_0}, {a1, null AS b1, c1, d1, 1 AS $e, b1 AS b1_0}]) +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1]) ]]> @@ -945,7 +945,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1)], EXPR$2=[SUM($2)]) HashAggregate(isMerge=[false], groupBy=[a1], select=[a1, COUNT(c1) FILTER $g_0 AS EXPR$1, MIN(EXPR$2) FILTER $g_1 AS EXPR$2]) +- Exchange(distribution=[hash[a1]]) +- Calc(select=[a1, c1, b1 AS EXPR$2, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1]) - +- Expand(projects=[{a1=[$0], b1=[$1], c1=[$2], d1=[$3], $e=[0]}, {a1=[$0], b1=[$1], c1=[null], d1=[$3], $e=[1]}], projects=[{a1, b1, c1, d1, 0 AS $e}, {a1, b1, null AS c1, d1, 1 AS $e}]) + +- Expand(projects=[a1, b1, c1, d1, $e], projects=[{a1, b1, c1, d1, 0 AS $e}, {a1, b1, null AS c1, d1, 1 AS $e}]) +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1]) ]]> @@ -967,7 +967,7 @@ SortAggregate(isMerge=[false], groupBy=[a1], select=[a1, MIN(d1) FILTER $g_1 AS +- Sort(orderBy=[a1 ASC]) +- Exchange(distribution=[hash[a1]]) +- Calc(select=[a1, c1, d1, b1 AS EXPR$3, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1]) - +- Expand(projects=[{a1=[$0], b1=[$1], c1=[$2], d1=[$3], $e=[0]}, {a1=[$0], b1=[$1], c1=[null], d1=[$3], $e=[1]}], projects=[{a1, b1, c1, d1, 0 AS $e}, {a1, b1, null AS c1, d1, 1 AS $e}]) + +- Expand(projects=[a1, b1, c1, d1, $e], projects=[{a1, b1, c1, d1, 0 AS $e}, {a1, b1, null AS c1, d1, 1 AS $e}]) +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/WindowAggregateTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/WindowAggregateTest.xml index 07b7b71096..58fc85d89b 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/WindowAggregateTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/WindowAggregateTest.xml @@ -33,7 +33,7 @@ FROM MyTable1 @@ -65,7 +65,7 @@ FROM MyTable1 @@ -96,7 +96,7 @@ FROM MyTable1 @@ -108,154 +108,6 @@ Calc(select=[CAST(/(-($f0, /(*($f1, $f1), $f2)), $f2)) AS EXPR$0, CAST(/(-($f0, +- LocalHashWindowAggregate(window=[TumblingGroupWindow('w$, ts, 900000)], properties=[w$start, w$end, w$rowtime], select=[Partial_SUM($f2) AS sum$0, Partial_SUM(b) AS sum$1, Partial_COUNT(b) AS count$2]) +- Calc(select=[ts, b, *(b, b) AS $f2]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c]) -]]> - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - @@ -471,7 +323,7 @@ Calc(select=[EXPR$0, w$start AS EXPR$1, w$end AS EXPR$2]) @@ -493,7 +345,7 @@ HashWindowAggregate(window=[TumblingGroupWindow('w$, b, 3000)], select=[Final_AV @@ -514,7 +366,7 @@ HashWindowAggregate(window=[TumblingGroupWindow('w$, b, 3000)], select=[AVG(c) A @@ -580,7 +432,7 @@ HashWindowAggregate(window=[SlidingGroupWindow('w$, ts, 5400000, 900000)], selec @@ -602,7 +454,7 @@ HashWindowAggregate(window=[TumblingGroupWindow('w$, ts, 7200000)], select=[Fina @@ -623,7 +475,7 @@ HashWindowAggregate(window=[TumblingGroupWindow('w$, ts, 7200000)], select=[SUM( @@ -742,7 +594,7 @@ FROM MyTable2 @@ -774,7 +626,7 @@ FROM MyTable2 @@ -805,7 +657,7 @@ FROM MyTable2 @@ -841,7 +693,7 @@ GROUP BY TUMBLE(b, INTERVAL '15' MINUTE) @@ -877,7 +729,7 @@ GROUP BY TUMBLE(b, INTERVAL '15' MINUTE) @@ -912,7 +764,7 @@ GROUP BY TUMBLE(b, INTERVAL '15' MINUTE) @@ -1260,7 +1112,7 @@ Calc(select=[EXPR$0]) @@ -1283,7 +1135,7 @@ Calc(select=[EXPR$0]) @@ -1305,7 +1157,7 @@ Calc(select=[EXPR$0]) @@ -1328,7 +1180,7 @@ Calc(select=[EXPR$0]) @@ -1350,7 +1202,7 @@ Calc(select=[EXPR$0, EXPR$1]) @@ -1371,7 +1223,7 @@ Calc(select=[EXPR$0, EXPR$1]) @@ -1382,30 +1234,6 @@ Calc(select=[EXPR$0, EXPR$1]) +- Exchange(distribution=[hash[a, d]]) +- LocalHashWindowAggregate(groupBy=[a, d], window=[TumblingGroupWindow('w$, b, 3000)], select=[a, d, Partial_AVG(c) AS (sum$0, count$1), Partial_COUNT(a) AS count$2]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) -]]> - - - - - - - - - - - @@ -1417,7 +1245,7 @@ SortWindowAggregate(window=[TumblingGroupWindow('w$, ts, 240000)], select=[Final @@ -1442,7 +1270,7 @@ Calc(select=[EXPR$0]) @@ -1465,7 +1293,7 @@ Calc(select=[EXPR$0]) @@ -1490,7 +1318,7 @@ Calc(select=[EXPR$0]) @@ -1512,7 +1340,7 @@ Calc(select=[EXPR$0, EXPR$1]) @@ -1534,7 +1362,7 @@ Calc(select=[EXPR$0, EXPR$1]) @@ -1550,26 +1378,27 @@ Calc(select=[EXPR$0, EXPR$1]) ]]> - + - + @@ -1581,7 +1410,7 @@ Calc(select=[w$end AS EXPR$0]) @@ -1592,6 +1421,151 @@ SortWindowAggregate(window=[TumblingGroupWindow('w$, ts, 240000)], select=[weigh +- Exchange(distribution=[single]) +- Calc(select=[ts, b, a]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + @@ -1603,7 +1577,7 @@ SortWindowAggregate(window=[TumblingGroupWindow('w$, ts, 240000)], select=[weigh @@ -1617,27 +1591,53 @@ Calc(select=[w$end AS EXPR$0]) ]]> - + - + @@ -1649,7 +1649,7 @@ SortWindowAggregate(window=[TumblingGroupWindow('w$, ts, 240000)], select=[Final diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/AggregateReduceGroupingRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/AggregateReduceGroupingRuleTest.xml index 14b9c6ed9b..09639d399e 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/AggregateReduceGroupingRuleTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/AggregateReduceGroupingRuleTest.xml @@ -324,7 +324,7 @@ FlinkLogicalAggregate(group=[{0, 1, 2}], b1=[AUXILIARY_GROUP($3)], EXPR$4=[COUNT @@ -343,7 +343,7 @@ FlinkLogicalWindowAggregate(group=[{0}], b4=[AUXILIARY_GROUP($1)], EXPR$2=[COUNT @@ -363,7 +363,7 @@ FlinkLogicalWindowAggregate(group=[{0}], c4=[AUXILIARY_GROUP($2)], EXPR$2=[COUNT LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT($3)]) +- LogicalProject(a4=[$0], c4=[$1], s=[TUMBLE_START($2)], b4=[$3]) +- LogicalAggregate(group=[{0, 1, 2}], b4=[VAR_POP($3)]) - +- LogicalProject(a4=[$0], c4=[$2], $f2=[TUMBLE($3, 900000:INTERVAL MINUTE)], b4=[$1]) + +- LogicalProject(a4=[$0], c4=[$2], $f2=[$TUMBLE($3, 900000:INTERVAL MINUTE)], b4=[$1]) +- LogicalTableScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4, d4)]]]) ]]> @@ -387,7 +387,7 @@ FlinkLogicalCalc(select=[a4, c4, s, EXPR$3]) LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT($3)]) +- LogicalProject(a4=[$0], c4=[$1], e=[TUMBLE_END($2)], b4=[$3]) +- LogicalAggregate(group=[{0, 1, 2}], b4=[VAR_POP($3)]) - +- LogicalProject(a4=[$0], c4=[$2], $f2=[TUMBLE($3, 900000:INTERVAL MINUTE)], b4=[$1]) + +- LogicalProject(a4=[$0], c4=[$2], $f2=[$TUMBLE($3, 900000:INTERVAL MINUTE)], b4=[$1]) +- LogicalTableScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4, d4)]]]) ]]> @@ -411,7 +411,7 @@ FlinkLogicalCalc(select=[a4, c4, e, EXPR$3]) LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT()]) +- LogicalProject(a4=[$0], b4=[$3], c4=[$1]) +- LogicalAggregate(group=[{0, 1, 2}], b4=[VAR_POP($3)]) - +- LogicalProject(a4=[$0], c4=[$2], $f2=[TUMBLE($3, 900000:INTERVAL MINUTE)], b4=[$1]) + +- LogicalProject(a4=[$0], c4=[$2], $f2=[$TUMBLE($3, 900000:INTERVAL MINUTE)], b4=[$1]) +- LogicalTableScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4, d4)]]]) ]]> @@ -439,7 +439,7 @@ LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1, 2}, {0, 1}, {0, 2}, {0}, {1, @@ -458,7 +458,7 @@ LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1}, {0, 2}]], EXPR$3=[COUNT($3) @@ -479,7 +479,7 @@ LogicalProject(a1=[$0], s=[$2]) @@ -500,7 +500,7 @@ LogicalProject(a1=[$0], b1=[$1], c1=[$2], EXPR$3=[$4]) @@ -557,7 +557,7 @@ LogicalAggregate(group=[{0, 1, 2}], groups=[[{0, 1, 2}, {0, 1}, {0}, {}]], EXPR$ @@ -635,7 +635,7 @@ LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT(DISTINCT $2)], EXPR$3=[SUM(DISTIN @@ -656,7 +656,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[SUM(DISTINCT $1)], EXPR$2=[MAX($1)], EXPR$ FlinkLogicalAggregate(group=[{0}], EXPR$1=[SUM($1) FILTER $4], EXPR$2=[MIN($2) FILTER $5], EXPR$3=[MIN($3) FILTER $5]) +- FlinkLogicalCalc(select=[a1, b1, EXPR$2, EXPR$3, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1]) +- FlinkLogicalAggregate(group=[{0, 1, 3}], EXPR$2=[MAX($4)], EXPR$3=[MIN($2)]) - +- FlinkLogicalExpand(projects=[{a1=[$0], b1=[$1], c1=[$2], $e=[0], b1_0=[$1]}, {a1=[$0], b1=[null], c1=[$2], $e=[1], b1_0=[$1]}]) + +- FlinkLogicalExpand(projects=[a1, b1, c1, $e, b1_0]) +- FlinkLogicalCalc(select=[a1, b1, c1]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1]) ]]> @@ -678,7 +678,7 @@ LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT(DISTINCT $2)], EXPR$3=[MAX($3)], FlinkLogicalAggregate(group=[{0}], d1=[MIN($2) FILTER $6], EXPR$2=[COUNT($1) FILTER $5], EXPR$3=[MIN($3) FILTER $6], EXPR$4=[MIN($4) FILTER $6]) +- FlinkLogicalCalc(select=[a1, c1, d1, EXPR$3, EXPR$4, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1]) +- FlinkLogicalAggregate(group=[{0, 2, 4}], d1=[AUXILIARY_GROUP($3)], EXPR$3=[MAX($1)], EXPR$4=[SUM($1)]) - +- FlinkLogicalExpand(projects=[{a1=[$0], b1=[$1], c1=[$2], d1=[$3], $e=[0]}, {a1=[$0], b1=[$1], c1=[null], d1=[$3], $e=[1]}]) + +- FlinkLogicalExpand(projects=[a1, b1, c1, d1, $e]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1]) ]]> @@ -698,8 +698,8 @@ LogicalProject(EXPR$0=[$2]) @@ -774,8 +774,8 @@ LogicalProject(a1=[$0], b1=[$1], EXPR$2=[$4]) @@ -836,7 +836,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1)], EXPR$2=[SUM($1)]) FlinkLogicalAggregate(group=[{0}], EXPR$1=[COUNT($1) FILTER $3], EXPR$2=[MIN($2) FILTER $4]) +- FlinkLogicalCalc(select=[a1, b1, EXPR$2, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1]) +- FlinkLogicalAggregate(group=[{0, 1, 2}], EXPR$2=[SUM($3)]) - +- FlinkLogicalExpand(projects=[{a1=[$0], b1=[$1], $e=[0], b1_0=[$1]}, {a1=[$0], b1=[null], $e=[1], b1_0=[$1]}]) + +- FlinkLogicalExpand(projects=[a1, b1, $e, b1_0]) +- FlinkLogicalCalc(select=[a1, b1]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1]) ]]> @@ -858,7 +858,7 @@ LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT(DISTINCT $2)], EXPR$3=[SUM($2)]) FlinkLogicalAggregate(group=[{0}], c1=[AUXILIARY_GROUP($1)], EXPR$2=[COUNT($2) FILTER $4], EXPR$3=[MIN($3) FILTER $5]) +- FlinkLogicalCalc(select=[a1, c1, b1, EXPR$3, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1]) +- FlinkLogicalAggregate(group=[{0, 2, 3}], c1=[AUXILIARY_GROUP($1)], EXPR$3=[SUM($4)]) - +- FlinkLogicalExpand(projects=[{a1=[$0], c1=[$1], b1=[$2], $e=[0], b1_0=[$2]}, {a1=[$0], c1=[$1], b1=[null], $e=[1], b1_0=[$2]}]) + +- FlinkLogicalExpand(projects=[a1, c1, b1, $e, b1_0]) +- FlinkLogicalCalc(select=[a1, c1, b1]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1]) ]]> @@ -880,7 +880,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1)], EXPR$2=[SUM($2)]) FlinkLogicalAggregate(group=[{0}], EXPR$1=[COUNT($1) FILTER $3], EXPR$2=[MIN($2) FILTER $4]) +- FlinkLogicalCalc(select=[a1, c1, EXPR$2, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1]) +- FlinkLogicalAggregate(group=[{0, 1, 3}], EXPR$2=[SUM($2)]) - +- FlinkLogicalExpand(projects=[{a1=[$0], c1=[$1], b1=[$2], $e=[0]}, {a1=[$0], c1=[null], b1=[$2], $e=[1]}]) + +- FlinkLogicalExpand(projects=[a1, c1, b1, $e]) +- FlinkLogicalCalc(select=[a1, c1, b1]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1]) ]]> @@ -902,7 +902,7 @@ LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT(DISTINCT $2)], EXPR$3=[SUM($3)]) FlinkLogicalAggregate(group=[{0}], d1=[MIN($2) FILTER $5], EXPR$2=[COUNT($1) FILTER $4], EXPR$3=[MIN($3) FILTER $5]) +- FlinkLogicalCalc(select=[a1, c1, d1, EXPR$3, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1]) +- FlinkLogicalAggregate(group=[{0, 2, 4}], d1=[AUXILIARY_GROUP($3)], EXPR$3=[SUM($1)]) - +- FlinkLogicalExpand(projects=[{a1=[$0], b1=[$1], c1=[$2], d1=[$3], $e=[0]}, {a1=[$0], b1=[$1], c1=[null], d1=[$3], $e=[1]}]) + +- FlinkLogicalExpand(projects=[a1, b1, c1, d1, $e]) +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.xml index c2a132a0d3..02c6a23cb5 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.xml @@ -30,7 +30,7 @@ GROUP BY long, TUMBLE(rowtime, INTERVAL '10' SECOND) @@ -88,7 +88,7 @@ GlobalGroupAggregate(groupBy=[b], select=[b, COUNT(distinct$0 count$0) AS EXPR$1 =($4, -($9, 5000:INTERVAL SECOND)), <=($4, +($9, 10000:INTERVAL SECOND)))], joinType=[inner]) :- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 0:INTERVAL MILLISECOND)]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) @@ -149,7 +149,7 @@ LogicalProject(b=[$0], EXPR$1=[COUNT($1) OVER (PARTITION BY $0 ORDER BY $2 NULLS +- LogicalJoin(condition=[AND(=($1, $4), >=($2, -($5, 5000:INTERVAL SECOND)), <=($2, +($5, 10000:INTERVAL SECOND)))], joinType=[inner]) :- LogicalProject(b=[$0], a=[$2], rt=[TUMBLE_ROWTIME($1)]) : +- LogicalAggregate(group=[{0, 1}], a=[COUNT($2)]) - : +- LogicalProject(b=[$1], $f1=[TUMBLE($4, 5000:INTERVAL SECOND)], a=[$0]) + : +- LogicalProject(b=[$1], $f1=[$TUMBLE($4, 5000:INTERVAL SECOND)], a=[$0]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 0:INTERVAL MILLISECOND)]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) +- LogicalProject(b=[$0], a=[$2], rt=[HOP_ROWTIME($1)]) @@ -251,7 +251,7 @@ LogicalSink(name=[`default_catalog`.`default_database`.`appendSink1`], fields=[a +- LogicalAggregate(group=[{0, 1}], EXPR$1=[LISTAGG($2, $3)]) +- LogicalProject($f0=[HOP(TUMBLE_ROWTIME($0), 12000:INTERVAL SECOND, 4000:INTERVAL SECOND)], id1=[$1], text=[$2], $f3=[_UTF-16LE'*']) +- LogicalAggregate(group=[{0, 1}], text=[LISTAGG($2, $3)]) - +- LogicalProject($f0=[TUMBLE($1, 6000:INTERVAL SECOND)], id1=[$0], text=[$2], $f3=[_UTF-16LE'#']) + +- LogicalProject($f0=[$TUMBLE($1, 6000:INTERVAL SECOND)], id1=[$0], text=[$2], $f3=[_UTF-16LE'#']) +- LogicalFilter(condition=[AND(=($0, $3), >($1, -($4, 300000:INTERVAL MINUTE)), <($1, +($4, 180000:INTERVAL MINUTE)))]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($1, 0:INTERVAL MILLISECOND)]) @@ -262,7 +262,7 @@ LogicalSink(name=[`default_catalog`.`default_database`.`appendSink1`], fields=[a LogicalSink(name=[`default_catalog`.`default_database`.`appendSink2`], fields=[a, b]) +- LogicalProject(id1=[$1], EXPR$1=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$1=[LISTAGG($2, $3)]) - +- LogicalProject($f0=[TUMBLE($1, 9000:INTERVAL SECOND)], id1=[$0], text=[$2], $f3=[_UTF-16LE'-']) + +- LogicalProject($f0=[$TUMBLE($1, 9000:INTERVAL SECOND)], id1=[$0], text=[$2], $f3=[_UTF-16LE'-']) +- LogicalFilter(condition=[AND(=($0, $3), >($1, -($4, 300000:INTERVAL MINUTE)), <($1, +($4, 180000:INTERVAL MINUTE)))]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($1, 0:INTERVAL MILLISECOND)]) @@ -274,7 +274,7 @@ LogicalSink(name=[`default_catalog`.`default_database`.`appendSink3`], fields=[a +- LogicalAggregate(group=[{0}], EXPR$1=[COUNT($1)]) +- LogicalProject(id1=[$1], text=[$2]) +- LogicalAggregate(group=[{0, 1}], text=[LISTAGG($2, $3)]) - +- LogicalProject($f0=[TUMBLE($1, 6000:INTERVAL SECOND)], id1=[$0], text=[$2], $f3=[_UTF-16LE'#']) + +- LogicalProject($f0=[$TUMBLE($1, 6000:INTERVAL SECOND)], id1=[$0], text=[$2], $f3=[_UTF-16LE'#']) +- LogicalFilter(condition=[AND(=($0, $3), >($1, -($4, 300000:INTERVAL MINUTE)), <($1, +($4, 180000:INTERVAL MINUTE)))]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($1, 0:INTERVAL MILLISECOND)]) @@ -610,9 +610,9 @@ GlobalGroupAggregate(groupBy=[b], select=[b, COUNT(count$0) AS EXPR$1]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/RelTimeIndicatorConverterTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/RelTimeIndicatorConverterTest.xml index 6c0fafe0cc..39d6c48835 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/RelTimeIndicatorConverterTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/RelTimeIndicatorConverterTest.xml @@ -139,9 +139,9 @@ SELECT TUMBLE_END(newrowtime, INTERVAL '30' SECOND), long, sum(`int`) FROM ( @@ -229,7 +229,7 @@ GROUP BY long, TUMBLE(rowtime, INTERVAL '0.1' SECOND) @@ -279,7 +279,7 @@ FROM MyTable1 @@ -305,7 +305,7 @@ HAVING QUARTER(TUMBLE_END(rowtime, INTERVAL '1' SECOND)) = 1 LogicalProject(EXPR$0=[$2], long=[$0]) +- LogicalFilter(condition=[=(EXTRACT(FLAG(QUARTER), TUMBLE_END($1)), 1)]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[MIN($2)]) - +- LogicalProject(long=[$1], $f1=[TUMBLE($0, 1000:INTERVAL SECOND)], rowtime=[$0]) + +- LogicalProject(long=[$1], $f1=[$TUMBLE($0, 1000:INTERVAL SECOND)], rowtime=[$0]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> @@ -334,7 +334,7 @@ FROM MyTable1 diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.xml index 837fa0829a..209eab364e 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.xml @@ -117,7 +117,7 @@ FROM rowTimeT WHERE val > 100 ($2, 100)]) +- LogicalTableScan(table=[[default_catalog, default_database, rowTimeT]]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/WindowAggregateTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/WindowAggregateTest.xml index db86af3ede..13553e4665 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/WindowAggregateTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/WindowAggregateTest.xml @@ -34,7 +34,7 @@ FROM MyTable @@ -45,53 +45,6 @@ Calc(select=[/(-($f0, /(*($f1, $f1), $f2)), $f2) AS EXPR$0, /(-($f0, /(*($f1, $f +- Exchange(distribution=[single]) +- Calc(select=[rowtime, c, *(c, c) AS $f2]) +- DataStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, proctime, rowtime]) -]]> - - - - - - - - - - - @@ -108,7 +61,7 @@ FROM MyTable @@ -213,7 +166,7 @@ GroupWindowAggregate(window=[SlidingGroupWindow('w$, $f2, 60000, 1000)], select= @@ -243,7 +196,7 @@ LogicalProject(EXPR$0=[$1]) +- LogicalAggregate(group=[{0}], EXPR$0=[weightedAvg($1, $2)]) +- LogicalProject(b=[$1], c=[$2], a=[$0]) +- LogicalAggregate(group=[{0, 1, 2, 3}]) - +- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[TUMBLE($4, 900000:INTERVAL MINUTE)]) + +- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[$TUMBLE($4, 900000:INTERVAL MINUTE)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -380,9 +333,9 @@ GROUP BY TUMBLE(zzzzz, INTERVAL '0.004' SECOND) @@ -416,7 +369,7 @@ LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[weightedAvg($2, $3)]) +- LogicalProject(b=[$1], d=[$4], c=[$2], a=[$0]) +- LogicalAggregate(group=[{0, 1, 2, 3}], d=[COUNT()]) - +- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[TUMBLE($4, 900000:INTERVAL MINUTE)]) + +- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[$TUMBLE($4, 900000:INTERVAL MINUTE)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -453,7 +406,7 @@ GROUP BY TUMBLE(rowtime, INTERVAL '15' MINUTE) @@ -520,37 +473,32 @@ GroupWindowAggregate(window=[SessionGroupWindow('w$, $f2, 60000)], select=[SUM(a ]]> - + @@ -572,7 +520,7 @@ LogicalProject(EXPR$0=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[weightedAvg($2, $3)]) +- LogicalProject(b=[$1], ping_start=[TUMBLE_START($3)], c=[$2], a=[$0]) +- LogicalAggregate(group=[{0, 1, 2, 3}]) - +- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[TUMBLE($4, 900000:INTERVAL MINUTE)]) + +- LogicalProject(a=[$0], b=[$1], c=[$2], $f3=[$TUMBLE($4, 900000:INTERVAL MINUTE)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> @@ -589,94 +537,104 @@ Calc(select=[EXPR$0]) ]]> - + - + - + - + + +(SELECT * FROM window_1h) +UNION ALL +(SELECT * FROM window_2h) +]]> diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/catalog/BasicOperatorTable.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/catalog/BasicOperatorTable.scala index 11d9a4b19e..5ec1fc7584 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/catalog/BasicOperatorTable.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/catalog/BasicOperatorTable.scala @@ -260,6 +260,8 @@ object BasicOperatorTable { */ val TUMBLE: SqlGroupedWindowFunction = new SqlGroupedWindowFunction( + // The TUMBLE group function was hard code to $TUMBLE in CALCITE-3382. + "$TUMBLE", SqlKind.TUMBLE, null, OperandTypes.or(OperandTypes.DATETIME_INTERVAL, OperandTypes.DATETIME_INTERVAL_TIME)) { -- Gitee From 8a6877d14eb8f15c9f5124e400f2f3e351f42c13 Mon Sep 17 00:00:00 2001 From: "yuzhao.cyz" Date: Tue, 17 Mar 2020 21:31:45 +0800 Subject: [PATCH 375/885] [FLINK-14338][table-planner][table-planner-blink] Update files due to CALCITE-3763 * CALCITE-3763 prunes useless fields of input project --- .../planner/calcite/FlinkRelBuilder.scala | 20 ++- .../WindowAggregateReduceFunctionsRule.scala | 15 ++- .../physical/batch/BatchExecHashAggRule.scala | 13 +- .../physical/batch/BatchExecSortAggRule.scala | 17 ++- .../batch/BatchExecWindowAggregateRule.scala | 18 ++- .../RemoveRedundantLocalHashAggRule.scala | 4 +- .../RemoveRedundantLocalSortAggRule.scala | 4 +- .../planner/plan/batch/table/CalcTest.xml | 12 +- .../FlinkAggregateJoinTransposeRuleTest.xml | 8 +- .../logical/SimplifyJoinConditionRuleTest.xml | 2 +- .../RemoveRedundantLocalHashAggRuleTest.xml | 26 ++++ .../RemoveRedundantLocalRankRuleTest.xml | 6 +- .../RemoveRedundantLocalSortAggRuleTest.xml | 30 +++++ .../plan/stream/table/AggregateTest.xml | 6 +- .../plan/stream/table/TableAggregateTest.xml | 6 +- .../stream/table/TwoStageAggregateTest.xml | 6 +- .../RemoveRedundantLocalHashAggRuleTest.scala | 13 ++ .../RemoveRedundantLocalSortAggRuleTest.scala | 11 ++ .../ExtendedAggregateExtractProjectRule.java | 8 +- .../flink/table/calcite/FlinkRelBuilder.scala | 21 ++- .../common/LogicalWindowAggregateRule.scala | 126 +++++++++++++++++- .../WindowAggregateReduceFunctionsRule.scala | 14 +- 22 files changed, 337 insertions(+), 49 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelBuilder.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelBuilder.scala index 47f1ab8723..3b8eb0ea9f 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelBuilder.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelBuilder.scala @@ -28,6 +28,7 @@ import org.apache.flink.table.planner.plan.nodes.calcite.{LogicalTableAggregate, import org.apache.flink.table.planner.plan.utils.AggregateUtil import org.apache.flink.table.runtime.operators.rank.{RankRange, RankType} import org.apache.flink.table.sinks.TableSink + import org.apache.calcite.plan._ import org.apache.calcite.rel.RelCollation import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeField} @@ -40,6 +41,7 @@ import org.apache.calcite.util.{ImmutableBitSet, Util} import java.lang.Iterable import java.util import java.util.List +import java.util.function.UnaryOperator import scala.collection.JavaConversions._ @@ -132,7 +134,23 @@ class FlinkRelBuilder( namedProperties: List[PlannerNamedWindowProperty], aggCalls: Iterable[AggCall]): RelBuilder = { // build logical aggregate - val aggregate = super.aggregate(groupKey, aggCalls).build().asInstanceOf[LogicalAggregate] + + // Because of: + // [CALCITE-3763] RelBuilder.aggregate should prune unused fields from the input, + // if the input is a Project. + // + // the field can not be pruned if it is referenced by other expressions + // of the window aggregation(i.e. the TUMBLE_START/END). + // To solve this, we config the RelBuilder to forbidden this feature. + val aggregate = transform( + new UnaryOperator[RelBuilder.Config] { + override def apply(t: RelBuilder.Config) + : RelBuilder.Config = t.withPruneInputOfAggregate(false) + }) + .push(build()) + .aggregate(groupKey, aggCalls) + .build() + .asInstanceOf[LogicalAggregate] // build logical window aggregate from it aggregate match { diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/WindowAggregateReduceFunctionsRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/WindowAggregateReduceFunctionsRule.scala index 90dbdc3123..f6d7f9e4b4 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/WindowAggregateReduceFunctionsRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/WindowAggregateReduceFunctionsRule.scala @@ -20,6 +20,7 @@ package org.apache.flink.table.planner.plan.rules.logical import org.apache.flink.table.planner.plan.nodes.calcite.LogicalWindowAggregate +import org.apache.calcite.plan.Contexts import org.apache.calcite.plan.RelOptRule._ import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.{Aggregate, AggregateCall, RelFactories} @@ -39,7 +40,11 @@ import scala.collection.JavaConversions._ class WindowAggregateReduceFunctionsRule extends AggregateReduceFunctionsRule( operand(classOf[LogicalWindowAggregate], any()), - RelFactories.LOGICAL_BUILDER) { + RelBuilder.proto( + Contexts.of( + RelFactories.DEFAULT_STRUCT, + RelBuilder.Config.DEFAULT + .withPruneInputOfAggregate(false)))) { override def newAggregateRel( relBuilder: RelBuilder, @@ -47,6 +52,14 @@ class WindowAggregateReduceFunctionsRule newCalls: util.List[AggregateCall]): Unit = { // create a LogicalAggregate with simpler aggregation functions + + // Because of: + // [CALCITE-3763] RelBuilder.aggregate should prune unused fields from the input, + // if the input is a Project. + // + // the field can not be pruned if it is referenced by other expressions + // of the window aggregation(i.e. the TUMBLE_START/END). + // To solve this, we config the RelBuilder to forbidden this feature. super.newAggregateRel(relBuilder, oldAgg, newCalls) // pop LogicalAggregate from RelBuilder val newAgg = relBuilder.build().asInstanceOf[LogicalAggregate] diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecHashAggRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecHashAggRule.scala index 704d267160..cc615f0510 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecHashAggRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecHashAggRule.scala @@ -118,6 +118,17 @@ class BatchExecHashAggRule } else { Seq(FlinkRelDistribution.SINGLETON) } + // Remove the global agg call filters because the + // filter is already done by local aggregation. + val aggCallsWithoutFilter = aggCallsWithoutAuxGroupCalls.map { + aggCall => + if (aggCall.filterArg > 0) { + aggCall.copy(aggCall.getArgList, -1, aggCall.getCollation) + } else { + aggCall + } + } + val globalAggCallToAggFunction = aggCallsWithoutFilter.zip(aggFunctions) globalDistributions.foreach { globalDistribution => val requiredTraitSet = localHashAgg.getTraitSet.replace(globalDistribution) val newLocalHashAgg = RelOptRule.convert(localHashAgg, requiredTraitSet) @@ -131,7 +142,7 @@ class BatchExecHashAggRule inputRowType, globalGroupSet, globalAuxGroupSet, - aggCallToAggFunction, + globalAggCallToAggFunction, isMerge = true) call.transformTo(globalHashAgg) } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecSortAggRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecSortAggRule.scala index 426e9dfff4..bf18c59279 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecSortAggRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecSortAggRule.scala @@ -74,10 +74,6 @@ class BatchExecSortAggRule val input: RelNode = call.rel(1) val inputRowType = input.getRowType - if (agg.indicator) { - throw new UnsupportedOperationException("Not support group sets aggregate now.") - } - val (auxGroupSet, aggCallsWithoutAuxGroupCalls) = AggregateUtil.checkAndSplitAggCalls(agg) val (_, aggBufferTypes, aggFunctions) = AggregateUtil.transformToBatchAggregateFunctions( @@ -124,6 +120,17 @@ class BatchExecSortAggRule } else { (Seq(FlinkRelDistribution.SINGLETON), RelCollations.EMPTY) } + // Remove the global agg call filters because the + // filter is already done by local aggregation. + val aggCallsWithoutFilter = aggCallsWithoutAuxGroupCalls.map { + aggCall => + if (aggCall.filterArg > 0) { + aggCall.copy(aggCall.getArgList, -1, aggCall.getCollation) + } else { + aggCall + } + } + val globalAggCallToAggFunction = aggCallsWithoutFilter.zip(aggFunctions) globalDistributions.foreach { globalDistribution => val requiredTraitSet = localSortAgg.getTraitSet .replace(globalDistribution) @@ -140,7 +147,7 @@ class BatchExecSortAggRule newLocalInput.getRowType, globalGroupSet, globalAuxGroupSet, - aggCallToAggFunction, + globalAggCallToAggFunction, isMerge = true) call.transformTo(globalSortAgg) } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecWindowAggregateRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecWindowAggregateRule.scala index 7920a1a802..bf430cb9f5 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecWindowAggregateRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecWindowAggregateRule.scala @@ -34,12 +34,13 @@ import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDat import org.apache.flink.table.types.logical.{BigIntType, IntType, LogicalType} import org.apache.calcite.plan.RelOptRule._ -import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall} +import org.apache.calcite.plan.{Contexts, RelOptRule, RelOptRuleCall} import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.Aggregate.Group -import org.apache.calcite.rel.core.{Aggregate, AggregateCall} +import org.apache.calcite.rel.core.{Aggregate, AggregateCall, RelFactories} import org.apache.calcite.rel.{RelCollations, RelNode} import org.apache.calcite.sql.`type`.SqlTypeName +import org.apache.calcite.tools.RelBuilder import org.apache.commons.math3.util.ArithmeticUtils import scala.collection.JavaConversions._ @@ -70,6 +71,11 @@ class BatchExecWindowAggregateRule extends RelOptRule( operand(classOf[FlinkLogicalWindowAggregate], operand(classOf[RelNode], any)), + RelBuilder.proto( + Contexts.of( + RelFactories.DEFAULT_STRUCT, + RelBuilder.Config.DEFAULT + .withPruneInputOfAggregate(false))), "BatchExecWindowAggregateRule") with BatchExecAggRuleBase { @@ -156,6 +162,14 @@ class BatchExecWindowAggregateRule // TODO aggregate include projection now, so do not provide new trait will be safe val aggProvidedTraitSet = input.getTraitSet.replace(FlinkConventions.BATCH_PHYSICAL) + + // Because of: + // [CALCITE-3763] RelBuilder.aggregate should prune unused fields from the input, + // if the input is a Project. + // + // the field can not be pruned if it is referenced by other expressions + // of the window aggregation(i.e. the TUMBLE_START/END). + // To solve this, we config the RelBuilder to forbidden this feature. val inputTimeFieldIndex = AggregateUtil.timeFieldIndex( input.getRowType, call.builder(), window.timeAttribute) val inputTimeFieldType = agg.getInput.getRowType.getFieldList.get(inputTimeFieldIndex).getType diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalHashAggRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalHashAggRule.scala index c538dac11e..33a2bb2bde 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalHashAggRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalHashAggRule.scala @@ -49,7 +49,9 @@ class RemoveRedundantLocalHashAggRule extends RelOptRule( inputOfLocalAgg.getRowType, localAgg.getGrouping, localAgg.getAuxGrouping, - globalAgg.getAggCallToAggFunction, + // Use the localAgg agg calls because the global agg call filters was removed, + // see BatchExecHashAggRule for details. + localAgg.getAggCallToAggFunction, isMerge = false) call.transformTo(newGlobalAgg) } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalSortAggRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalSortAggRule.scala index 615d082fff..a0ff75ed1e 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalSortAggRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalSortAggRule.scala @@ -48,7 +48,9 @@ abstract class RemoveRedundantLocalSortAggRule( inputOfLocalAgg.getRowType, localAgg.getGrouping, localAgg.getAuxGrouping, - globalAgg.getAggCallToAggFunction, + // Use the localAgg agg calls because the global agg call filters was removed, + // see BatchExecSortAggRule for details. + localAgg.getAggCallToAggFunction, isMerge = false) call.transformTo(newGlobalAgg) } diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/table/CalcTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/table/CalcTest.xml index d44892ba90..4c1176422a 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/table/CalcTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/table/CalcTest.xml @@ -168,8 +168,8 @@ Calc(select=[a]) @@ -179,7 +179,7 @@ Calc(select=[EXPR$0]) +- HashAggregate(isMerge=[true], groupBy=[k], select=[k, Final_SUM(sum$0) AS EXPR$0]) +- Exchange(distribution=[hash[k]]) +- LocalHashAggregate(groupBy=[k], select=[k, Partial_SUM(a) AS sum$0]) - +- Calc(select=[a, b, c, d, MyHashCode$(c) AS k]) + +- Calc(select=[a, MyHashCode$(c) AS k]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) ]]> @@ -188,8 +188,8 @@ Calc(select=[EXPR$0]) @@ -199,7 +199,7 @@ Calc(select=[EXPR$0]) +- HashAggregate(isMerge=[true], groupBy=[k], select=[k, Final_SUM(sum$0) AS EXPR$0]) +- Exchange(distribution=[hash[k]]) +- LocalHashAggregate(groupBy=[k], select=[k, Partial_SUM(a) AS sum$0]) - +- Calc(select=[a, b, c, d, UPPER(c) AS k]) + +- Calc(select=[a, UPPER(c) AS k]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateJoinTransposeRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateJoinTransposeRuleTest.xml index 0d58e8b36b..35d417a49f 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateJoinTransposeRuleTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateJoinTransposeRuleTest.xml @@ -216,8 +216,8 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT($0)]) + + + + + 5), COUNT(DISTINCT b) FILTER (WHERE b > 3) +FROM z GROUP BY d]]> + + + ($0, 5))], b=[$1], $f6=[IS TRUE(>($1, 3))]) + +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(a, b, c, d, e)]]]) +]]> + + + (a, 5)) AS $f4, b, IS TRUE(>(b, 3)) AS $f6]) + +- TableSourceScan(table=[[default_catalog, default_database, z, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalRankRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalRankRuleTest.xml index 4e7b5c5def..df384408a0 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalRankRuleTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalRankRuleTest.xml @@ -100,9 +100,9 @@ LogicalProject(a=[$0], b=[$1], rk=[$2], rk1=[$3]) + + + + + 5), COUNT(DISTINCT b) FILTER (WHERE b > 3) +FROM z GROUP BY d]]> + + + ($0, 5))], b=[$1], $f6=[IS TRUE(>($1, 3))]) + +- LogicalTableScan(table=[[default_catalog, default_database, z, source: [TestTableSource(a, b, c, d, e)]]]) +]]> + + + (a, 5)) AS $f4, b, IS TRUE(>(b, 3)) AS $f6]) + +- TableSourceScan(table=[[default_catalog, default_database, z, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/AggregateTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/AggregateTest.xml index b7c4a6ec78..fd6047b216 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/AggregateTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/AggregateTest.xml @@ -102,8 +102,8 @@ Calc(select=[EXPR$0]) @@ -111,7 +111,7 @@ LogicalProject(b=[$0], EXPR$0=[$1]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TableAggregateTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TableAggregateTest.xml index 5cbc8cf92d..1cd0641c69 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TableAggregateTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TableAggregateTest.xml @@ -53,8 +53,8 @@ Calc(select=[f0 AS a, f1 AS b]) @@ -63,7 +63,7 @@ LogicalProject(bb=[AS($0, _UTF-16LE'bb')], _c1=[+(AS($1, _UTF-16LE'x'), 1)], y=[ Calc(select=[bb, +(f0, 1) AS _c1, f1 AS y]) +- GroupTableAggregate(groupBy=[bb], select=[bb, EmptyTableAggFunc(a, b) AS (f0, f1)]) +- Exchange(distribution=[hash[bb]]) - +- Calc(select=[a, b, c, d, e, MOD(b, 5) AS bb]) + +- Calc(select=[a, b, MOD(b, 5) AS bb]) +- TableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TwoStageAggregateTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TwoStageAggregateTest.xml index be5a7e3733..8b595885f6 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TwoStageAggregateTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TwoStageAggregateTest.xml @@ -39,8 +39,8 @@ Calc(select=[EXPR$0]) @@ -49,7 +49,7 @@ LogicalProject(b=[$0], EXPR$0=[$1]) GlobalGroupAggregate(groupBy=[b], select=[b, AVG((sum$0, count$1)) AS EXPR$0]) +- Exchange(distribution=[hash[b]]) +- LocalGroupAggregate(groupBy=[b], select=[b, AVG(a0) AS (sum$0, count$1)]) - +- Calc(select=[a, b, c, CAST(a) AS a0]) + +- Calc(select=[b, CAST(a) AS a0]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) +- TableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala index cad9754512..d155b5cb62 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala @@ -35,6 +35,7 @@ class RemoveRedundantLocalHashAggRuleTest extends TableTestBase { def setup(): Unit = { util.addTableSource[(Int, Long, String)]("x", 'a, 'b, 'c) util.addTableSource[(Int, Long, String)]("y", 'd, 'e, 'f) + util.addTableSource[(Int, Long, Long, Long, Long)]("z", 'a, 'b, 'c, 'd, 'e) } @Test @@ -69,4 +70,16 @@ class RemoveRedundantLocalHashAggRuleTest extends TableTestBase { util.verifyPlan(sqlQuery) } + @Test + def testUsingLocalAggCallFilters(): Unit = { + util.tableEnv.getConfig.getConfiguration.setString( + ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg") + util.tableEnv.getConfig.getConfiguration.setBoolean( + BatchExecJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, true) + val sqlQuery = "SELECT d, MAX(e), MAX(e) FILTER (WHERE a < 10), COUNT(DISTINCT c),\n" + + "COUNT(DISTINCT c) FILTER (WHERE a > 5), COUNT(DISTINCT b) FILTER (WHERE b > 3)\n" + + "FROM z GROUP BY d" + util.verifyPlan(sqlQuery) + } + } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.scala index a7f72c91cc..2957254014 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.scala @@ -35,6 +35,7 @@ class RemoveRedundantLocalSortAggRuleTest extends TableTestBase { def setup(): Unit = { util.addTableSource[(Int, Long, String)]("x", 'a, 'b, 'c) util.addTableSource[(Int, Long, String)]("y", 'd, 'e, 'f) + util.addTableSource[(Int, Long, Long, Long, Long)]("z", 'a, 'b, 'c, 'd, 'e) } @Test @@ -64,4 +65,14 @@ class RemoveRedundantLocalSortAggRuleTest extends TableTestBase { util.verifyPlan(sqlQuery) } + @Test + def testUsingLocalAggCallFilters(): Unit = { + util.tableEnv.getConfig.getConfiguration.setString( + ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg") + val sqlQuery = "SELECT d, MAX(e), MAX(e) FILTER (WHERE a < 10), COUNT(DISTINCT c),\n" + + "COUNT(DISTINCT c) FILTER (WHERE a > 5), COUNT(DISTINCT b) FILTER (WHERE b > 3)\n" + + "FROM z GROUP BY d" + util.verifyPlan(sqlQuery) + } + } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/plan/rules/logical/ExtendedAggregateExtractProjectRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/plan/rules/logical/ExtendedAggregateExtractProjectRule.java index 80c297e37f..ba703a4acc 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/plan/rules/logical/ExtendedAggregateExtractProjectRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/plan/rules/logical/ExtendedAggregateExtractProjectRule.java @@ -25,6 +25,7 @@ import org.apache.flink.table.plan.logical.rel.LogicalWindowAggregate; import org.apache.flink.table.plan.logical.rel.LogicalWindowTableAggregate; import org.apache.flink.table.plan.logical.rel.TableAggregate; +import org.apache.calcite.plan.Contexts; import org.apache.calcite.plan.RelOptRuleCall; import org.apache.calcite.plan.RelOptRuleOperand; import org.apache.calcite.rel.RelNode; @@ -63,7 +64,12 @@ public class ExtendedAggregateExtractProjectRule extends AggregateExtractProject public static final ExtendedAggregateExtractProjectRule INSTANCE = new ExtendedAggregateExtractProjectRule( operand(SingleRel.class, - operand(RelNode.class, any())), RelFactories.LOGICAL_BUILDER); + operand(RelNode.class, any())), + RelBuilder.proto( + Contexts.of( + RelFactories.DEFAULT_STRUCT, + RelBuilder.Config.DEFAULT + .withPruneInputOfAggregate(false)))); public ExtendedAggregateExtractProjectRule( RelOptRuleOperand operand, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/FlinkRelBuilder.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/FlinkRelBuilder.scala index 75a57b3dac..838c99a493 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/FlinkRelBuilder.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/FlinkRelBuilder.scala @@ -20,7 +20,6 @@ package org.apache.flink.table.calcite import java.lang.Iterable import java.util.{List => JList} - import org.apache.calcite.plan._ import org.apache.calcite.rel.logical.LogicalAggregate import org.apache.calcite.tools.RelBuilder @@ -33,6 +32,8 @@ import org.apache.flink.table.plan.logical.LogicalWindow import org.apache.flink.table.plan.logical.rel.{LogicalTableAggregate, LogicalWindowAggregate, LogicalWindowTableAggregate} import org.apache.flink.table.runtime.aggregate.AggregateUtil +import java.util.function.UnaryOperator + import scala.collection.JavaConverters._ /** @@ -86,7 +87,23 @@ class FlinkRelBuilder( aggCalls: Iterable[AggCall]) : RelBuilder = { // build logical aggregate - val aggregate = super.aggregate(groupKey, aggCalls).build().asInstanceOf[LogicalAggregate] + + // Because of: + // [CALCITE-3763] RelBuilder.aggregate should prune unused fields + // from the input, if the input is a Project. + // + // the field can not be pruned if it is referenced by other expressions + // of the window aggregation(i.e. the TUMBLE_START/END). + // To solve this, we config the RelBuilder to forbidden this feature. + val aggregate = transform( + new UnaryOperator[RelBuilder.Config] { + override def apply(t: RelBuilder.Config) + : RelBuilder.Config = t.withPruneInputOfAggregate(false) + }) + .push(build()) + .aggregate(groupKey, aggCalls) + .build() + .asInstanceOf[LogicalAggregate] val namedProperties = windowProperties.asScala.map { case Alias(p: WindowProperty, name, _) => diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/common/LogicalWindowAggregateRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/common/LogicalWindowAggregateRule.scala index 431fe9e487..a866b659db 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/common/LogicalWindowAggregateRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/common/LogicalWindowAggregateRule.scala @@ -21,23 +21,33 @@ import com.google.common.collect.ImmutableList import org.apache.calcite.plan._ import org.apache.calcite.plan.hep.HepRelVertex import org.apache.calcite.rel.`type`.RelDataType -import org.apache.calcite.rel.core.{Aggregate, AggregateCall} +import org.apache.calcite.rel.core.{Aggregate, AggregateCall, Project, RelFactories} import org.apache.calcite.rel.logical.{LogicalAggregate, LogicalProject} import org.apache.calcite.rex._ import org.apache.calcite.sql.`type`.SqlTypeUtil import org.apache.calcite.util.ImmutableBitSet import org.apache.flink.table.api._ import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty +import org.apache.flink.table.calcite.FlinkTypeFactory import org.apache.flink.table.catalog.BasicOperatorTable import org.apache.flink.table.plan.logical.LogicalWindow import org.apache.flink.table.plan.logical.rel.LogicalWindowAggregate +import org.apache.calcite.rel.RelNode +import org.apache.calcite.tools.RelBuilder + +import _root_.java.util.{ArrayList => JArrayList, Collections, List => JList} import _root_.scala.collection.JavaConversions._ abstract class LogicalWindowAggregateRule(ruleName: String) extends RelOptRule( RelOptRule.operand(classOf[LogicalAggregate], RelOptRule.operand(classOf[LogicalProject], RelOptRule.none())), + RelBuilder.proto( + Contexts.of( + RelFactories.DEFAULT_STRUCT, + RelBuilder.Config.DEFAULT + .withBloat(-1))), ruleName) { override def matches(call: RelOptRuleCall): Boolean = { @@ -50,7 +60,7 @@ abstract class LogicalWindowAggregateRule(ruleName: String) throw new TableException("Only a single window group function may be used in GROUP BY") } - !groupSets && !agg.indicator && windowExpressions.nonEmpty + !groupSets && windowExpressions.nonEmpty } /** @@ -61,8 +71,15 @@ abstract class LogicalWindowAggregateRule(ruleName: String) * that the types are equivalent. */ override def onMatch(call: RelOptRuleCall): Unit = { - val agg = call.rel[LogicalAggregate](0) - val project = agg.getInput.asInstanceOf[HepRelVertex].getCurrentRel.asInstanceOf[LogicalProject] + val agg0 = call.rel[LogicalAggregate](0) + val project0 = call.rel[LogicalProject](1) + val project = rewriteWindowCallWithFuncOperands(project0, call.builder()) + val agg = if (project != project0) { + agg0.copy(agg0.getTraitSet, Collections.singletonList(project)) + .asInstanceOf[LogicalAggregate] + } else { + agg0 + } val (windowExpr, windowExprIdx) = getWindowExpressions(agg).head val window = translateWindowExpression(windowExpr, project.getInput.getRowType) @@ -90,7 +107,6 @@ abstract class LogicalWindowAggregateRule(ruleName: String) // we don't use the builder here because it uses RelMetadataQuery which affects the plan val newAgg = LogicalAggregate.create( newProject, - agg.indicator, newGroupSet, ImmutableList.of(newGroupSet), finalCalls) @@ -125,6 +141,104 @@ abstract class LogicalWindowAggregateRule(ruleName: String) call.transformTo(result) } + /** Trim out the HepRelVertex wrapper and get current relational expression. */ + private def trimHep(node: RelNode): RelNode = { + node match { + case hepRelVertex: HepRelVertex => + hepRelVertex.getCurrentRel + case _ => node + } + } + + /** + * Rewrite plan with function call as window call operand: rewrite the window call to + * reference the input instead of invoking the function directly, in order to simplify the + * subsequent rewrite logic. + * + * For example, plan + *

    +   * LogicalAggregate(group=[{0}], a=[COUNT()])
    +   *   LogicalProject($f0=[$TUMBLE(TUMBLE_ROWTIME($0), 4:INTERVAL SECOND)], a=[$1])
    +   *     LogicalProject($f0=[1970-01-01 00:00:00:TIMESTAMP(3)], a=[$0])
    +   * 
    + * + * would be rewritten to + *
    +   * LogicalAggregate(group=[{0}], a=[COUNT()])
    +   *   LogicalProject($f0=[TUMBLE($1, 4:INTERVAL SECOND)], a=[$0])
    +   *     LogicalProject(a=[$1], zzzzz=[TUMBLE_ROWTIME($0)])
    +   *       LogicalProject($f0=[1970-01-01 00:00:00:TIMESTAMP(3)], a=[$0])
    +   * 
    + */ + private def rewriteWindowCallWithFuncOperands( + project: LogicalProject, + relBuilder: RelBuilder): LogicalProject = { + val projectInput = trimHep(project.getInput) + if (!projectInput.isInstanceOf[Project]) { + return project + } + val inputProjects = projectInput.asInstanceOf[Project].getChildExps + var hasWindowCallWithFuncOperands: Boolean = false + var lastIdx = projectInput.getRowType.getFieldCount - 1; + val pushDownCalls = new JArrayList[RexNode]() + 0 until projectInput.getRowType.getFieldCount foreach { + idx => pushDownCalls.add(RexInputRef.of(idx, projectInput.getRowType)) + } + val newProjectExprs = project.getChildExps.map { + case call: RexCall if isWindowCall(call) && + isTimeAttributeCall(call.getOperands.head, inputProjects) => + hasWindowCallWithFuncOperands = true + // Update the window call to reference a RexInputRef instead of a function call. + call.accept( + new RexShuttle { + override def visitCall(call: RexCall): RexNode = { + if (isTimeAttributeCall(call, inputProjects)) { + lastIdx += 1 + pushDownCalls.add(call) + relBuilder.getRexBuilder.makeInputRef( + call.getType, + // We would project plus an additional function call + // at the end of input projection. + lastIdx) + } else { + super.visitCall(call) + } + } + }) + case rex: RexNode => rex + } + + if (hasWindowCallWithFuncOperands) { + relBuilder + .push(projectInput) + // project plus the function call. + .project(pushDownCalls) + .project(newProjectExprs, project.getRowType.getFieldNames) + .build() + .asInstanceOf[LogicalProject] + } else { + project + } + } + + /** Decides if the [[RexNode]] is a call whose return type is + * a time indicator type. */ + def isTimeAttributeCall(rexNode: RexNode, projects: JList[RexNode]): Boolean = rexNode match { + case call: RexCall if FlinkTypeFactory.isTimeIndicatorType(call.getType) => + call.getOperands.forall { operand => + operand.isInstanceOf[RexInputRef] + } + case _ => false + } + + /** Decides whether the [[RexCall]] is a window call. */ + def isWindowCall(call: RexCall): Boolean = call.getOperator match { + case BasicOperatorTable.SESSION | + BasicOperatorTable.HOP | + BasicOperatorTable.TUMBLE => true + case _ => false + } + /** * Change the types of [[AggregateCall]] to the corresponding inferred types. */ @@ -182,7 +296,7 @@ abstract class LogicalWindowAggregateRule(ruleName: String) private[table] def getWindowExpressions(agg: LogicalAggregate): Seq[(RexCall, Int)] = { - val project = agg.getInput.asInstanceOf[HepRelVertex].getCurrentRel.asInstanceOf[LogicalProject] + val project = trimHep(agg.getInput).asInstanceOf[LogicalProject] val groupKeys = agg.getGroupSet // get grouping expressions diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/common/WindowAggregateReduceFunctionsRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/common/WindowAggregateReduceFunctionsRule.scala index 50c0758339..38bcaf5e84 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/common/WindowAggregateReduceFunctionsRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/common/WindowAggregateReduceFunctionsRule.scala @@ -19,8 +19,7 @@ package org.apache.flink.table.plan.rules.common import java.util - -import org.apache.calcite.plan.RelOptRule +import org.apache.calcite.plan.{Contexts, RelOptRule} import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.{Aggregate, AggregateCall, RelFactories} import org.apache.calcite.rel.logical.LogicalAggregate @@ -35,9 +34,14 @@ import scala.collection.JavaConversions._ * Rule to convert complex aggregation functions into simpler ones. * Have a look at [[AggregateReduceFunctionsRule]] for details. */ -class WindowAggregateReduceFunctionsRule extends AggregateReduceFunctionsRule( - RelOptRule.operand(classOf[LogicalWindowAggregate], RelOptRule.any()), - RelFactories.LOGICAL_BUILDER) { +class WindowAggregateReduceFunctionsRule + extends AggregateReduceFunctionsRule( + RelOptRule.operand(classOf[LogicalWindowAggregate], RelOptRule.any()), + RelBuilder.proto( + Contexts.of( + RelFactories.DEFAULT_STRUCT, + RelBuilder.Config.DEFAULT + .withPruneInputOfAggregate(false)))) { override def newAggregateRel( relBuilder: RelBuilder, -- Gitee From 5fef3c1272e5be6ca20ed83f87c147a2d18379b0 Mon Sep 17 00:00:00 2001 From: "yuzhao.cyz" Date: Tue, 17 Mar 2020 21:34:17 +0800 Subject: [PATCH 376/885] [FLINK-14338][table-planner-blink] Update files due to CALCITE-1824 * GROUP_ID translation was fixed --- .../logical/DecomposeGroupingSetsRuleTest.xml | 61 +++++++++----- .../stream/sql/agg/GroupingSetsTest.scala | 3 + .../batch/sql/agg/GroupingSetsITCase.scala | 11 ++- .../runtime/batch/sql/GroupingSetsITCase.java | 83 +++++++++++-------- .../api/batch/sql/DistinctAggregateTest.scala | 19 +---- .../api/batch/sql/GroupingSetsTest.scala | 47 +++-------- .../runtime/batch/sql/AggregateITCase.scala | 31 +++++-- 7 files changed, 132 insertions(+), 123 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/DecomposeGroupingSetsRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/DecomposeGroupingSetsRuleTest.xml index ca3284e908..d656c98260 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/DecomposeGroupingSetsRuleTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/DecomposeGroupingSetsRuleTest.xml @@ -24,16 +24,27 @@ SELECT a, GROUP_ID() AS g, COUNT(*) as c FROM MyTable GROUP BY GROUPING SETS (a, @@ -56,16 +67,17 @@ FROM MyTable @@ -84,9 +96,10 @@ FROM MyTable @@ -107,16 +120,17 @@ GROUP BY GROUPING SETS (b, c) @@ -139,7 +153,7 @@ LogicalProject(a=[$3], b=[$4], c=[$5]) @@ -161,16 +175,17 @@ FROM MyTable @@ -193,7 +208,7 @@ LogicalProject(b=[$2], c=[$3]) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/GroupingSetsTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/GroupingSetsTest.scala index b371e81873..652b8a0346 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/GroupingSetsTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/GroupingSetsTest.scala @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.stream.sql.agg import org.apache.flink.api.scala._ +import org.apache.flink.table.api.TableException import org.apache.flink.table.api.scala._ import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil import org.apache.flink.table.planner.utils.{TableTestBase, TableTestUtil} @@ -354,6 +355,8 @@ class GroupingSetsTest extends TableTestBase { @Test def testCALCITE1824(): Unit = { + expectedException.expect(classOf[TableException]) + expectedException.expectMessage("GROUPING SETS are currently not supported") val sqlQuery = """ |SELECT deptno, GROUP_ID() AS g, COUNT(*) AS c diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/GroupingSetsITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/GroupingSetsITCase.scala index 584d073270..a9c1be679c 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/GroupingSetsITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/GroupingSetsITCase.scala @@ -395,13 +395,16 @@ class GroupingSetsITCase extends BatchTestBase { @Test def testCALCITE1824(): Unit = { - // TODO: - // When "[CALCITE-1824] GROUP_ID returns wrong result" is fixed, - // there will be an extra row (null, 1, 14). checkResult( "select deptno, group_id() as g, count(*) as c " + "from scott_emp group by grouping sets (deptno, (), ())", - Seq(row(10, 0, 3), row(20, 0, 5), row(30, 0, 6), row(null, 0, 14)) + Seq(row(10, 0, 3), + row(10, 1, 3), + row(20, 0, 5), + row(20, 1, 5), + row(30, 0, 6), + row(30, 1, 6), + row(null, 0, 14)) ) } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/batch/sql/GroupingSetsITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/batch/sql/GroupingSetsITCase.java index a9fd29be9a..c73eaff646 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/batch/sql/GroupingSetsITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/batch/sql/GroupingSetsITCase.java @@ -89,33 +89,33 @@ public class GroupingSetsITCase extends TableProgramsClusterTestBase { " GROUP BY GROUPING SETS (f1, f2, ())"; String expected = - "1,null,1,1,1,0,1,0,2,1\n" + - "6,null,18,1,1,0,1,0,2,6\n" + - "2,null,2,1,1,0,1,0,2,2\n" + - "4,null,8,1,1,0,1,0,2,4\n" + - "5,null,13,1,1,0,1,0,2,5\n" + - "3,null,5,1,1,0,1,0,2,3\n" + - "null,Comment#11,17,2,0,1,0,1,1,1\n" + - "null,Comment#8,14,2,0,1,0,1,1,1\n" + - "null,Comment#2,8,2,0,1,0,1,1,1\n" + - "null,Comment#1,7,2,0,1,0,1,1,1\n" + - "null,Comment#14,20,2,0,1,0,1,1,1\n" + - "null,Comment#7,13,2,0,1,0,1,1,1\n" + - "null,Comment#6,12,2,0,1,0,1,1,1\n" + - "null,Comment#3,9,2,0,1,0,1,1,1\n" + - "null,Comment#12,18,2,0,1,0,1,1,1\n" + - "null,Comment#5,11,2,0,1,0,1,1,1\n" + - "null,Comment#15,21,2,0,1,0,1,1,1\n" + - "null,Comment#4,10,2,0,1,0,1,1,1\n" + - "null,Hi,1,2,0,1,0,1,1,1\n" + - "null,Comment#10,16,2,0,1,0,1,1,1\n" + - "null,Hello world,3,2,0,1,0,1,1,1\n" + - "null,I am fine.,5,2,0,1,0,1,1,1\n" + - "null,Hello world, how are you?,4,2,0,1,0,1,1,1\n" + - "null,Comment#9,15,2,0,1,0,1,1,1\n" + - "null,Comment#13,19,2,0,1,0,1,1,1\n" + - "null,Luke Skywalker,6,2,0,1,0,1,1,1\n" + - "null,Hello,2,2,0,1,0,1,1,1\n" + + "1,null,1,0,1,0,1,0,2,1\n" + + "2,null,2,0,1,0,1,0,2,2\n" + + "3,null,5,0,1,0,1,0,2,3\n" + + "4,null,8,0,1,0,1,0,2,4\n" + + "5,null,13,0,1,0,1,0,2,5\n" + + "6,null,18,0,1,0,1,0,2,6\n" + + "null,Comment#1,7,0,0,1,0,1,1,1\n" + + "null,Comment#10,16,0,0,1,0,1,1,1\n" + + "null,Comment#11,17,0,0,1,0,1,1,1\n" + + "null,Comment#12,18,0,0,1,0,1,1,1\n" + + "null,Comment#13,19,0,0,1,0,1,1,1\n" + + "null,Comment#14,20,0,0,1,0,1,1,1\n" + + "null,Comment#15,21,0,0,1,0,1,1,1\n" + + "null,Comment#2,8,0,0,1,0,1,1,1\n" + + "null,Comment#3,9,0,0,1,0,1,1,1\n" + + "null,Comment#4,10,0,0,1,0,1,1,1\n" + + "null,Comment#5,11,0,0,1,0,1,1,1\n" + + "null,Comment#6,12,0,0,1,0,1,1,1\n" + + "null,Comment#7,13,0,0,1,0,1,1,1\n" + + "null,Comment#8,14,0,0,1,0,1,1,1\n" + + "null,Comment#9,15,0,0,1,0,1,1,1\n" + + "null,Hello world, how are you?,4,0,0,1,0,1,1,1\n" + + "null,Hello world,3,0,0,1,0,1,1,1\n" + + "null,Hello,2,0,0,1,0,1,1,1\n" + + "null,Hi,1,0,0,1,0,1,1,1\n" + + "null,I am fine.,5,0,0,1,0,1,1,1\n" + + "null,Luke Skywalker,6,0,0,1,0,1,1,1\n" + "null,null,11,0,0,0,0,0,0,21"; checkSql(query, expected); @@ -128,14 +128,27 @@ public class GroupingSetsITCase extends TableProgramsClusterTestBase { " GROUP BY GROUPING SETS (f1, f2)"; String expected = - "6,null,18,1\n5,null,13,1\n4,null,8,1\n3,null,5,1\n2,null,2,1\n1,null,1,1\n" + - "null,Luke Skywalker,6,2\nnull,I am fine.,5,2\nnull,Hi,1,2\n" + - "null,null,3,2\nnull,Hello,2,2\nnull,Comment#9,15,2\nnull,Comment#8,14,2\n" + - "null,Comment#7,13,2\nnull,Comment#6,12,2\nnull,Comment#5,11,2\n" + - "null,Comment#4,10,2\nnull,Comment#3,9,2\nnull,Comment#2,8,2\n" + - "null,Comment#15,21,2\nnull,Comment#14,20,2\nnull,Comment#13,19,2\n" + - "null,Comment#12,18,2\nnull,Comment#11,17,2\nnull,Comment#10,16,2\n" + - "null,Comment#1,7,2"; + "1,Hi,1,0\n" + + "2,Hello,2,0\n" + + "2,null,3,0\n" + + "3,I am fine.,5,0\n" + + "3,Luke Skywalker,6,0\n" + + "3,null,4,0\n" + + "4,Comment#1,7,0\n" + + "4,Comment#2,8,0\n" + + "4,Comment#3,9,0\n" + + "4,Comment#4,10,0\n" + + "5,Comment#5,11,0\n" + + "5,Comment#6,12,0\n" + + "5,Comment#7,13,0\n" + + "5,Comment#8,14,0\n" + + "5,Comment#9,15,0\n" + + "6,Comment#10,16,0\n" + + "6,Comment#11,17,0\n" + + "6,Comment#12,18,0\n" + + "6,Comment#13,19,0\n" + + "6,Comment#14,20,0\n" + + "6,Comment#15,21,0"; checkSql(query, expected); } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/sql/DistinctAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/sql/DistinctAggregateTest.scala index d3c279562f..ff5e560b5a 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/sql/DistinctAggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/sql/DistinctAggregateTest.scala @@ -57,13 +57,7 @@ class DistinctAggregateTest extends TableTestBase { val sqlQuery = "SELECT COUNT(DISTINCT a), SUM(DISTINCT a), MAX(DISTINCT a) FROM MyTable" - val left = unaryNode("DataSetAggregate", - unaryNode("DataSetCalc", - batchTableNode(table), - term("select", "a")), - term("select", "MAX(a) AS EXPR$2")) - - val right = unaryNode( + val expected = unaryNode( "DataSetAggregate", unaryNode( "DataSetDistinct", @@ -74,18 +68,9 @@ class DistinctAggregateTest extends TableTestBase { ), term("distinct", "a") ), - term("select", "COUNT(a) AS EXPR$0", "SUM(a) AS EXPR$1") + term("select", "COUNT(a) AS EXPR$0", "SUM(a) AS EXPR$1", "MAX(a) AS EXPR$2") ) - val expected = unaryNode("DataSetCalc", - binaryNode("DataSetSingleRowJoin", - left, - right, - term("where", "true"), - term("join", "EXPR$2", "EXPR$0", "EXPR$1"), - term("joinType", "NestedLoopInnerJoin")), - term("select", "EXPR$0", "EXPR$1", "EXPR$2")) - util.verifySql(sqlQuery, expected) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/sql/GroupingSetsTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/sql/GroupingSetsTest.scala index 7b811aa66d..59faa2c585 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/sql/GroupingSetsTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/sql/GroupingSetsTest.scala @@ -34,38 +34,15 @@ class GroupingSetsTest extends TableTestBase { val sqlQuery = "SELECT b, c, avg(a) as a, GROUP_ID() as g FROM MyTable " + "GROUP BY GROUPING SETS (b, c)" - val aggregate = binaryNode( - "DataSetUnion", - unaryNode( - "DataSetCalc", - unaryNode( - "DataSetAggregate", - unaryNode( - "DataSetCalc", - batchTableNode(table), - term("select", "b", "a") - ), - term("groupBy", "b"), - term("select", "b", "AVG(a) AS a") - ), - term("select", "b", "null:INTEGER AS c", "a", "1:BIGINT AS g") - ), + val aggregate = unaryNode( + "DataSetCalc", unaryNode( - "DataSetCalc", - unaryNode( - "DataSetAggregate", - unaryNode( - "DataSetCalc", - batchTableNode(table), - term("select", "c", "a") - ), - term("groupBy", "c"), - term("select", "c", "AVG(a) AS a") - ), - term("select", "null:BIGINT AS b", "c", "a", "2:BIGINT AS g") + "DataSetAggregate", + batchTableNode(table), + term("groupBy", "b", "c"), + term("select", "b", "c", "AVG(a) AS a") ), - term("all", "true"), - term("union", "b", "c", "a", "g") + term("select", "b", "c", "a", "0:BIGINT AS g") ) util.verifySql(sqlQuery, aggregate) @@ -91,7 +68,7 @@ class GroupingSetsTest extends TableTestBase { term("groupBy", "b", "c"), term("select", "b", "c", "AVG(a) AS a") ), - term("select", "b", "c", "a", "3:BIGINT AS g", "1:BIGINT AS gb", "1:BIGINT AS gc", + term("select", "b", "c", "a", "0:BIGINT AS g", "1:BIGINT AS gb", "1:BIGINT AS gc", "1:BIGINT AS gib", "1:BIGINT AS gic", "3:BIGINT AS gid") ) @@ -107,7 +84,7 @@ class GroupingSetsTest extends TableTestBase { term("groupBy", "b"), term("select", "b", "AVG(a) AS a") ), - term("select", "b", "null:INTEGER AS c", "a", "1:BIGINT AS g", "1:BIGINT AS gb", + term("select", "b", "null:INTEGER AS c", "a", "0:BIGINT AS g", "1:BIGINT AS gb", "0:BIGINT AS gc", "1:BIGINT AS gib", "0:BIGINT AS gic", "2:BIGINT AS gid") ) @@ -123,7 +100,7 @@ class GroupingSetsTest extends TableTestBase { term("groupBy", "c"), term("select", "c", "AVG(a) AS a") ), - term("select", "null:BIGINT AS b", "c", "a", "2:BIGINT AS g", "0:BIGINT AS gb", + term("select", "null:BIGINT AS b", "c", "a", "0:BIGINT AS g", "0:BIGINT AS gb", "1:BIGINT AS gc", "0:BIGINT AS gib", "1:BIGINT AS gic", "1:BIGINT AS gid") ) @@ -185,7 +162,7 @@ class GroupingSetsTest extends TableTestBase { term("groupBy", "b", "c"), term("select", "b", "c", "AVG(a) AS a") ), - term("select", "b", "c", "a", "3:BIGINT AS g", "1:BIGINT AS gb", "1:BIGINT AS gc", + term("select", "b", "c", "a", "0:BIGINT AS g", "1:BIGINT AS gb", "1:BIGINT AS gc", "1:BIGINT AS gib", "1:BIGINT AS gic", "3:BIGINT AS gid") ) @@ -201,7 +178,7 @@ class GroupingSetsTest extends TableTestBase { term("groupBy", "b"), term("select", "b", "AVG(a) AS a") ), - term("select", "b", "null:INTEGER AS c", "a", "1:BIGINT AS g", "1:BIGINT AS gb", + term("select", "b", "null:INTEGER AS c", "a", "0:BIGINT AS g", "1:BIGINT AS gb", "0:BIGINT AS gc", "1:BIGINT AS gib", "0:BIGINT AS gic", "2:BIGINT AS gid") ) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/AggregateITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/AggregateITCase.scala index d2bc5aab8d..f7d0102c0f 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/AggregateITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/AggregateITCase.scala @@ -29,6 +29,7 @@ import org.apache.flink.table.runtime.utils.TableProgramsTestBase.TableConfigMod import org.apache.flink.table.utils.NonMergableCount import org.apache.flink.test.util.TestBaseUtils import org.apache.flink.types.Row + import org.junit._ import org.junit.runner.RunWith import org.junit.runners.Parameterized @@ -232,7 +233,6 @@ class AggregateITCase( @Test def testGroupingSetAggregate(): Unit = { - val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = BatchTableEnvironment.create(env, config) @@ -245,14 +245,27 @@ class AggregateITCase( val result = tEnv.sqlQuery(sqlQuery).toDataSet[Row].collect() val expected = - "6,null,18,1\n5,null,13,1\n4,null,8,1\n3,null,5,1\n2,null,2,1\n1,null,1,1\n" + - "null,Luke Skywalker,6,2\nnull,I am fine.,5,2\nnull,Hi,1,2\n" + - "null,Hello world, how are you?,4,2\nnull,Hello world,3,2\nnull,Hello,2,2\n" + - "null,Comment#9,15,2\nnull,Comment#8,14,2\nnull,Comment#7,13,2\n" + - "null,Comment#6,12,2\nnull,Comment#5,11,2\nnull,Comment#4,10,2\n" + - "null,Comment#3,9,2\nnull,Comment#2,8,2\nnull,Comment#15,21,2\n" + - "null,Comment#14,20,2\nnull,Comment#13,19,2\nnull,Comment#12,18,2\n" + - "null,Comment#11,17,2\nnull,Comment#10,16,2\nnull,Comment#1,7,2" + "1,Hi,1,0\n" + + "2,Hello world,3,0\n" + + "2,Hello,2,0\n" + + "3,Hello world, how are you?,4,0\n" + + "3,I am fine.,5,0\n" + + "3,Luke Skywalker,6,0\n" + + "4,Comment#1,7,0\n" + + "4,Comment#2,8,0\n" + + "4,Comment#3,9,0\n" + + "4,Comment#4,10,0\n" + + "5,Comment#5,11,0\n" + + "5,Comment#6,12,0\n" + + "5,Comment#7,13,0\n" + + "5,Comment#8,14,0\n" + + "5,Comment#9,15,0\n" + + "6,Comment#10,16,0\n" + + "6,Comment#11,17,0\n" + + "6,Comment#12,18,0\n" + + "6,Comment#13,19,0\n" + + "6,Comment#14,20,0\n" + + "6,Comment#15,21,0" TestBaseUtils.compareResultAsText(result.asJava, expected) } -- Gitee From fabff4f38a56b29400b644d370e99cb8a398e8bb Mon Sep 17 00:00:00 2001 From: "yuzhao.cyz" Date: Tue, 17 Mar 2020 21:35:48 +0800 Subject: [PATCH 377/885] [FLINK-14338][table-planner][table-planner-blink] Update all kinds of left plan changes * Some join order changes for blink-planner due to the rule fire sequence changes, see https://github.com/apache/calcite/commit/35caa059a762094c7df0b30e9b51358a19b48ac2, they are still correct * The Correlate row count estimation has been fixed from a always 1 to join like estimation, thus, if the inputs of Join is a Correlate, the join algorithm would very probably changes, i.e. batch.sql.SubplanReuseTest * Due to CALCITE-3729, the filter condition was pushed down for some Join cases: batch.sql.join.JoinReorderTest * Due to CALCITE-2450 RexNode normalization, the predicates sequence of some test changes: logical.subquery.FlinkRewriteSubQueryRuleTest * The Decimal modulus precision inference has been fixed: planner.expressions.DecimalTypeTest --- .../src/main/codegen/data/Parser.tdd | 1 + flink-table/flink-table-planner-blink/pom.xml | 5 + .../src/main/resources/META-INF/NOTICE | 3 + .../planner/calcite/FlinkRelFactories.scala | 16 ++- .../WindowAggregateReduceFunctionsRule.scala | 18 +-- .../batch/BatchExecWindowAggregateRule.scala | 20 +-- .../plan/batch/sql/SubplanReuseTest.xml | 41 +++--- .../plan/batch/sql/agg/HashAggregateTest.xml | 6 +- .../plan/batch/sql/agg/SortAggregateTest.xml | 6 +- .../join/BroadcastHashSemiAntiJoinTest.xml | 40 +++--- .../plan/batch/sql/join/JoinReorderTest.xml | 85 ++++++------ .../sql/join/NestedLoopSemiAntiJoinTest.xml | 82 ++++++----- .../plan/batch/sql/join/SemiAntiJoinTest.xml | 126 ++++++++--------- .../sql/join/ShuffledHashSemiAntiJoinTest.xml | 130 ++++++++++++++---- .../sql/join/SortMergeSemiAntiJoinTest.xml | 45 +++--- ...inDependentConditionDerivationRuleTest.xml | 14 +- .../subquery/FlinkRewriteSubQueryRuleTest.xml | 2 +- .../logical/subquery/SubQueryAntiJoinTest.xml | 24 ++-- .../plan/stream/sql/agg/AggregateTest.xml | 2 +- .../stream/sql/agg/TwoStageAggregateTest.xml | 2 +- .../plan/stream/sql/join/SemiAntiJoinTest.xml | 86 ++++++------ .../plan/stream/table/AggregateTest.xml | 4 +- .../stream/table/TwoStageAggregateTest.xml | 4 +- .../planner/expressions/DecimalTypeTest.scala | 6 +- .../join/ShuffledHashSemiAntiJoinTest.scala | 21 --- .../planner/plan/utils/FlinkRexUtilTest.scala | 6 +- .../QueryOperationCatalogViewTable.java | 30 +++- .../operations/PlannerQueryOperation.java | 20 ++- .../table/api/batch/sql/GroupWindowTest.scala | 6 +- .../api/batch/sql/SetOperatorsTest.scala | 12 +- .../table/api/batch/table/CalcTest.scala | 2 +- .../api/stream/sql/GroupWindowTest.scala | 10 +- .../flink/table/api/stream/sql/JoinTest.scala | 2 +- .../stream/sql/TemporalTableJoinTest.scala | 17 +-- .../stream/table/TemporalTableJoinTest.scala | 17 +-- .../table/plan/RexProgramExtractorTest.scala | 4 +- .../testSqlUpdateAndToDataStream.out | 2 +- 37 files changed, 496 insertions(+), 421 deletions(-) diff --git a/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd b/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd index 14158c73ce..f11f5d9a4e 100644 --- a/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd +++ b/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd @@ -68,6 +68,7 @@ # List of new keywords. Example: "DATABASES", "TABLES". If the keyword is not a reserved # keyword, please also add it to 'nonReservedKeywords' section. + # Please keep the keyword in alphabetical order if new keyword is added. keywords: [ "BYTES" "CATALOGS" diff --git a/flink-table/flink-table-planner-blink/pom.xml b/flink-table/flink-table-planner-blink/pom.xml index b61f7d1fde..0bb7afff19 100644 --- a/flink-table/flink-table-planner-blink/pom.xml +++ b/flink-table/flink-table-planner-blink/pom.xml @@ -320,6 +320,7 @@ under the License. com.fasterxml.jackson.core:jackson-databind com.fasterxml.jackson.core:jackson-annotations commons-codec:commons-codec + commons-io:commons-io org.apache.flink:flink-sql-parser @@ -352,6 +353,10 @@ under the License. org.apache.commons.codec org.apache.flink.calcite.shaded.org.apache.commons.codec + + org.apache.commons.io + org.apache.flink.calcite.shaded.org.apache.commons.io + diff --git a/flink-table/flink-table-planner-blink/src/main/resources/META-INF/NOTICE b/flink-table/flink-table-planner-blink/src/main/resources/META-INF/NOTICE index 381d6d5194..e1246a40fc 100644 --- a/flink-table/flink-table-planner-blink/src/main/resources/META-INF/NOTICE +++ b/flink-table/flink-table-planner-blink/src/main/resources/META-INF/NOTICE @@ -12,6 +12,9 @@ This project bundles the following dependencies under the Apache Software Licens - com.fasterxml.jackson.core:jackson-databind:2.10.1 - com.jayway.jsonpath:json-path:2.4.0 - joda-time:joda-time:2.5 +- net.minidev:json-smart:jar:2.3 +- net.minidev:accessors-smart:jar:1.2 +- org.ow2.asm:asm:jar:5.0.4 - org.apache.calcite:calcite-core:1.22.0 - org.apache.calcite:calcite-linq4j:1.22.0 - org.apache.calcite.avatica:avatica-core:1.16.0 diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelFactories.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelFactories.scala index 3af65df6cc..1f002b2758 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelFactories.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelFactories.scala @@ -24,9 +24,10 @@ import org.apache.flink.table.sinks.TableSink import org.apache.calcite.plan.Contexts import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeField} +import org.apache.calcite.rel.core.RelFactories import org.apache.calcite.rel.{RelCollation, RelNode} import org.apache.calcite.rex.RexNode -import org.apache.calcite.tools.RelBuilderFactory +import org.apache.calcite.tools.{RelBuilder, RelBuilderFactory} import org.apache.calcite.util.ImmutableBitSet import java.util @@ -38,6 +39,19 @@ object FlinkRelFactories { val FLINK_REL_BUILDER: RelBuilderFactory = FlinkRelBuilder.proto(Contexts.empty) + // Because of: + // [CALCITE-3763] RelBuilder.aggregate should prune unused fields from the input, + // if the input is a Project. + // + // the field can not be pruned if it is referenced by other expressions + // of the window aggregation(i.e. the TUMBLE_START/END). + // To solve this, we config the RelBuilder to forbidden this feature. + val LOGICAL_BUILDER_WITHOUT_AGG_INPUT_PRUNE: RelBuilderFactory = RelBuilder.proto( + Contexts.of( + RelFactories.DEFAULT_STRUCT, + RelBuilder.Config.DEFAULT + .withPruneInputOfAggregate(false))) + val DEFAULT_EXPAND_FACTORY = new ExpandFactoryImpl val DEFAULT_RANK_FACTORY = new RankFactoryImpl diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/WindowAggregateReduceFunctionsRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/WindowAggregateReduceFunctionsRule.scala index f6d7f9e4b4..0af6c632b6 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/WindowAggregateReduceFunctionsRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/WindowAggregateReduceFunctionsRule.scala @@ -18,12 +18,12 @@ package org.apache.flink.table.planner.plan.rules.logical +import org.apache.flink.table.planner.calcite.FlinkRelFactories import org.apache.flink.table.planner.plan.nodes.calcite.LogicalWindowAggregate -import org.apache.calcite.plan.Contexts import org.apache.calcite.plan.RelOptRule._ import org.apache.calcite.rel.`type`.RelDataType -import org.apache.calcite.rel.core.{Aggregate, AggregateCall, RelFactories} +import org.apache.calcite.rel.core.{Aggregate, AggregateCall} import org.apache.calcite.rel.logical.LogicalAggregate import org.apache.calcite.rel.rules.AggregateReduceFunctionsRule import org.apache.calcite.rex.RexNode @@ -40,11 +40,7 @@ import scala.collection.JavaConversions._ class WindowAggregateReduceFunctionsRule extends AggregateReduceFunctionsRule( operand(classOf[LogicalWindowAggregate], any()), - RelBuilder.proto( - Contexts.of( - RelFactories.DEFAULT_STRUCT, - RelBuilder.Config.DEFAULT - .withPruneInputOfAggregate(false)))) { + FlinkRelFactories.LOGICAL_BUILDER_WITHOUT_AGG_INPUT_PRUNE) { override def newAggregateRel( relBuilder: RelBuilder, @@ -52,14 +48,6 @@ class WindowAggregateReduceFunctionsRule newCalls: util.List[AggregateCall]): Unit = { // create a LogicalAggregate with simpler aggregation functions - - // Because of: - // [CALCITE-3763] RelBuilder.aggregate should prune unused fields from the input, - // if the input is a Project. - // - // the field can not be pruned if it is referenced by other expressions - // of the window aggregation(i.e. the TUMBLE_START/END). - // To solve this, we config the RelBuilder to forbidden this feature. super.newAggregateRel(relBuilder, oldAgg, newCalls) // pop LogicalAggregate from RelBuilder val newAgg = relBuilder.build().asInstanceOf[LogicalAggregate] diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecWindowAggregateRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecWindowAggregateRule.scala index bf430cb9f5..847f29b7b1 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecWindowAggregateRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecWindowAggregateRule.scala @@ -21,7 +21,7 @@ package org.apache.flink.table.planner.plan.rules.physical.batch import org.apache.flink.table.api.config.OptimizerConfigOptions import org.apache.flink.table.api.{TableConfig, TableException} import org.apache.flink.table.functions.{AggregateFunction, UserDefinedFunction} -import org.apache.flink.table.planner.calcite.{FlinkContext, FlinkTypeFactory} +import org.apache.flink.table.planner.calcite.{FlinkContext, FlinkRelFactories, FlinkTypeFactory} import org.apache.flink.table.planner.functions.aggfunctions.DeclarativeAggregateFunction import org.apache.flink.table.planner.plan.`trait`.FlinkRelDistribution import org.apache.flink.table.planner.plan.logical.{LogicalWindow, SlidingGroupWindow, TumblingGroupWindow} @@ -34,13 +34,12 @@ import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDat import org.apache.flink.table.types.logical.{BigIntType, IntType, LogicalType} import org.apache.calcite.plan.RelOptRule._ -import org.apache.calcite.plan.{Contexts, RelOptRule, RelOptRuleCall} +import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall} import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.Aggregate.Group -import org.apache.calcite.rel.core.{Aggregate, AggregateCall, RelFactories} +import org.apache.calcite.rel.core.{Aggregate, AggregateCall} import org.apache.calcite.rel.{RelCollations, RelNode} import org.apache.calcite.sql.`type`.SqlTypeName -import org.apache.calcite.tools.RelBuilder import org.apache.commons.math3.util.ArithmeticUtils import scala.collection.JavaConversions._ @@ -71,11 +70,7 @@ class BatchExecWindowAggregateRule extends RelOptRule( operand(classOf[FlinkLogicalWindowAggregate], operand(classOf[RelNode], any)), - RelBuilder.proto( - Contexts.of( - RelFactories.DEFAULT_STRUCT, - RelBuilder.Config.DEFAULT - .withPruneInputOfAggregate(false))), + FlinkRelFactories.LOGICAL_BUILDER_WITHOUT_AGG_INPUT_PRUNE, "BatchExecWindowAggregateRule") with BatchExecAggRuleBase { @@ -163,13 +158,6 @@ class BatchExecWindowAggregateRule // TODO aggregate include projection now, so do not provide new trait will be safe val aggProvidedTraitSet = input.getTraitSet.replace(FlinkConventions.BATCH_PHYSICAL) - // Because of: - // [CALCITE-3763] RelBuilder.aggregate should prune unused fields from the input, - // if the input is a Project. - // - // the field can not be pruned if it is referenced by other expressions - // of the window aggregation(i.e. the TUMBLE_START/END). - // To solve this, we config the RelBuilder to forbidden this feature. val inputTimeFieldIndex = AggregateUtil.timeFieldIndex( input.getRowType, call.builder(), window.timeAttribute) val inputTimeFieldType = agg.getInput.getRowType.getFieldList.get(inputTimeFieldIndex).getType diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.xml index d36ed2b476..157aadbba3 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.xml @@ -426,10 +426,11 @@ LogicalProject(a=[$0], b=[$1], c=[$2], v=[$3], a0=[$4], b0=[$5], c0=[$6], v0=[$7 @@ -459,12 +460,13 @@ LogicalProject(a=[$0], b=[$1], c=[$2], s=[$3], a0=[$4], b0=[$5], c0=[$6], s0=[$7 @@ -1093,17 +1095,16 @@ LogicalIntersect(all=[false]) NestedLoopJoin(joinType=[LeftSemiJoin], where=[IS NOT DISTINCT FROM(random, random0)], select=[random], build=[right]) :- SortAggregate(isMerge=[false], groupBy=[random], select=[random]) : +- Sort(orderBy=[random ASC]) -: +- Exchange(distribution=[hash[random]]) -: +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[IS NOT DISTINCT FROM(random, random0)], select=[random], build=[right]) -: :- Exchange(distribution=[any], shuffle_mode=[BATCH]) -: : +- Calc(select=[random], reuse_id=[1]) -: : +- SortLimit(orderBy=[EXPR$1 ASC], offset=[0], fetch=[1], global=[true]) -: : +- Exchange(distribution=[single]) -: : +- SortLimit(orderBy=[EXPR$1 ASC], offset=[0], fetch=[1], global=[false]) -: : +- Calc(select=[a AS random, RAND() AS EXPR$1]) -: : +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) -: +- Exchange(distribution=[broadcast], reuse_id=[2]) -: +- Reused(reference_id=[1]) +: +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[IS NOT DISTINCT FROM(random, random0)], select=[random], build=[right]) +: :- Exchange(distribution=[hash[random]], shuffle_mode=[BATCH]) +: : +- Calc(select=[random], reuse_id=[1]) +: : +- SortLimit(orderBy=[EXPR$1 ASC], offset=[0], fetch=[1], global=[true]) +: : +- Exchange(distribution=[single]) +: : +- SortLimit(orderBy=[EXPR$1 ASC], offset=[0], fetch=[1], global=[false]) +: : +- Calc(select=[a AS random, RAND() AS EXPR$1]) +: : +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- Exchange(distribution=[broadcast], reuse_id=[2]) +: +- Reused(reference_id=[1]) +- Reused(reference_id=[2]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.xml index 062a047558..73f01514d5 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.xml @@ -717,7 +717,7 @@ Calc(select=[a, EXPR$1, _UTF-16LE'test' AS c]) +- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_MAX(max$0) AS EXPR$1]) +- Exchange(distribution=[hash[a]]) +- LocalHashAggregate(groupBy=[a], select=[a, Partial_MAX(b) AS max$0]) - +- Calc(select=[a, _UTF-16LE'test' AS c, b]) + +- Calc(select=[a, b]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -741,7 +741,7 @@ LogicalProject(a=[$0], EXPR$1=[$2], c=[$1]) Calc(select=[a, EXPR$1, _UTF-16LE'test' AS c]) +- HashAggregate(isMerge=[false], groupBy=[a], select=[a, MAX(b) AS EXPR$1]) +- Exchange(distribution=[hash[a]]) - +- Calc(select=[a, _UTF-16LE'test' AS c, b]) + +- Calc(select=[a, b]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -766,7 +766,7 @@ Calc(select=[a, EXPR$1, _UTF-16LE'test' AS c]) +- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_MAX(max$0) AS EXPR$1]) +- Exchange(distribution=[hash[a]]) +- LocalHashAggregate(groupBy=[a], select=[a, Partial_MAX(b) AS max$0]) - +- Calc(select=[a, _UTF-16LE'test' AS c, b]) + +- Calc(select=[a, b]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.xml index b9b901df7a..3f3e03f3f2 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.xml @@ -737,7 +737,7 @@ Calc(select=[a, EXPR$1, _UTF-16LE'test' AS c]) +- Exchange(distribution=[hash[a]]) +- LocalSortAggregate(groupBy=[a], select=[a, Partial_MAX(b) AS max$0]) +- Sort(orderBy=[a ASC]) - +- Calc(select=[a, _UTF-16LE'test' AS c, b]) + +- Calc(select=[a, b]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -762,7 +762,7 @@ Calc(select=[a, EXPR$1, _UTF-16LE'test' AS c]) +- SortAggregate(isMerge=[false], groupBy=[a], select=[a, MAX(b) AS EXPR$1]) +- Sort(orderBy=[a ASC]) +- Exchange(distribution=[hash[a]]) - +- Calc(select=[a, _UTF-16LE'test' AS c, b]) + +- Calc(select=[a, b]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -789,7 +789,7 @@ Calc(select=[a, EXPR$1, _UTF-16LE'test' AS c]) +- Exchange(distribution=[hash[a]]) +- LocalSortAggregate(groupBy=[a], select=[a, Partial_MAX(b) AS max$0]) +- Sort(orderBy=[a ASC]) - +- Calc(select=[a, _UTF-16LE'test' AS c, b]) + +- Calc(select=[a, b]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.xml index cd2c9eb7ea..cbd5d3e4a3 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.xml @@ -1399,19 +1399,19 @@ LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 100))]) =(CAST(c), 1:BIGINT)]) -: : : +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) -: : +- Exchange(distribution=[broadcast]) -: : +- Calc(select=[j], where=[>(CAST(k), 50:BIGINT)]) -: : +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) -: +- Exchange(distribution=[broadcast]) -: +- Calc(select=[i], where=[<(j, 100)]) -: +- Reused(reference_id=[1]) -+- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +HashJoin(joinType=[LeftAntiJoin], where=[=(a, i)], select=[a, b, c, d, e, f], isBroadcast=[true], build=[right]) +:- HashJoin(joinType=[LeftAntiJoin], where=[=(b, j)], select=[a, b, c, d, e, f], isBroadcast=[true], build=[right]) +: :- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e, f], isBroadcast=[true], build=[left]) +: : :- Exchange(distribution=[broadcast]) +: : : +- Calc(select=[a, b, c], where=[>=(CAST(c), 1:BIGINT)]) +: : : +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- Exchange(distribution=[broadcast]) +: +- Calc(select=[j], where=[>(CAST(k), 50:BIGINT)]) +: +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) ++- Exchange(distribution=[broadcast]) + +- Calc(select=[i], where=[<(j, 100)]) + +- Reused(reference_id=[1]) ]]> @@ -1854,28 +1854,26 @@ Calc(select=[b]) : : : : +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) : : : : +- Exchange(distribution=[single]) : : : : +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(i) AS count$1]) - : : : : +- Calc(select=[i]) - : : : : +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) + : : : : +- Calc(select=[i], reuse_id=[1]) + : : : : +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[2]) : : : +- Exchange(distribution=[broadcast]) : : : +- Calc(select=[i, true AS i0]) : : : +- HashAggregate(isMerge=[true], groupBy=[i], select=[i]) : : : +- Exchange(distribution=[hash[i]]) : : : +- LocalHashAggregate(groupBy=[i], select=[i]) - : : : +- Calc(select=[i, true AS i0]) - : : : +- Reused(reference_id=[1]) + : : : +- Reused(reference_id=[1]) : : +- Exchange(distribution=[broadcast]) : : +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) : : +- Exchange(distribution=[single]) : : +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(EXPR$0) AS count$1]) - : : +- Calc(select=[CAST(j) AS EXPR$0]) - : : +- Reused(reference_id=[1]) + : : +- Calc(select=[CAST(j) AS EXPR$0], reuse_id=[3]) + : : +- Reused(reference_id=[2]) : +- Exchange(distribution=[broadcast]) : +- Calc(select=[EXPR$0, true AS i]) : +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0]) : +- Exchange(distribution=[hash[EXPR$0]]) : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0]) - : +- Calc(select=[CAST(j) AS EXPR$0, true AS i]) - : +- Reused(reference_id=[1]) + : +- Reused(reference_id=[3]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d, f]) +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/JoinReorderTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/JoinReorderTest.xml index 8fe210d2d2..b4fc401f77 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/JoinReorderTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/JoinReorderTest.xml @@ -174,17 +174,16 @@ LogicalProject(a1=[$0], b1=[$1], c1=[$2], a2=[$3], b2=[$4], c2=[$5], a3=[$6], b3 @@ -216,16 +215,15 @@ LogicalProject(a1=[$0], b1=[$1], c1=[$2], a2=[$3], b2=[$4], c2=[$5], a3=[$6], b3 @@ -348,7 +345,7 @@ LogicalProject(a1=[$0], b1=[$1], c1=[$2], a2=[$3], b2=[$4], c2=[$5], a3=[$6], b3 @@ -408,11 +404,10 @@ Calc(select=[a4, b4, c4, a1, b1, c1, a2, b2, c2, a3, b3, c3, a5, b5, c5]) : :- Calc(select=[a5, b5, c5], where=[<(b5, 15)]) : : +- TableSourceScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(a5, b5, c5)]]], fields=[a5, b5, c5]) : +- Exchange(distribution=[broadcast]) - : +- Calc(select=[a1, b1, c1, a3, b3, c3], where=[=(a3, a1)]) - : +- HashJoin(joinType=[InnerJoin], where=[AND(=(a3, a1), <(*(b1, b3), 2000))], select=[a1, b1, c1, a3, b3, c3], isBroadcast=[true], build=[right]) - : :- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1]) - : +- Exchange(distribution=[broadcast]) - : +- TableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3)]]], fields=[a3, b3, c3]) + : +- HashJoin(joinType=[InnerJoin], where=[AND(=(a3, a1), <(*(b1, b3), 2000))], select=[a1, b1, c1, a3, b3, c3], isBroadcast=[true], build=[right]) + : :- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1]) + : +- Exchange(distribution=[broadcast]) + : +- TableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3)]]], fields=[a3, b3, c3]) +- Exchange(distribution=[broadcast]) +- TableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4)]]], fields=[a4, b4, c4]) ]]> @@ -538,16 +533,15 @@ LogicalProject(a1=[$0], b1=[$1], c1=[$2], a2=[$3], b2=[$4], c2=[$5], a3=[$6], b3 diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.xml index fa78f5d709..02364c14f5 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.xml @@ -508,27 +508,27 @@ LogicalFilter(condition=[<>($cor0.b, $1)]) (b, e)], select=[a, b, c], build=[right]) -:- NestedLoopJoin(joinType=[LeftAntiJoin], where=[AND(OR(=(b, i), IS NULL(b), IS NULL(i)), =(c, k))], select=[a, b, c], build=[right]) -: :- NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[a, b, c], build=[right], singleRowJoin=[true]) +:- NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[a, b, c], build=[right], singleRowJoin=[true]) +: :- NestedLoopJoin(joinType=[LeftAntiJoin], where=[AND(OR(=(b, i), IS NULL(b), IS NULL(i)), =(c, k))], select=[a, b, c], build=[right]) : : :- NestedLoopJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], build=[right]) : : : :- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) : : : +- Exchange(distribution=[broadcast]) : : : +- Calc(select=[d]) : : : +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f], reuse_id=[1]) : : +- Exchange(distribution=[broadcast]) -: : +- Calc(select=[IS NOT NULL(m) AS $f0]) -: : +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) -: : +- Exchange(distribution=[single]) -: : +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) -: : +- Calc(select=[true AS i]) -: : +- HashAggregate(isMerge=[true], groupBy=[l], select=[l]) -: : +- Exchange(distribution=[hash[l]]) -: : +- LocalHashAggregate(groupBy=[l], select=[l]) -: : +- Calc(select=[l], where=[LIKE(n, _UTF-16LE'Test')]) -: : +- TableSourceScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(l, m, n)]]], fields=[l, m, n]) +: : +- Calc(select=[i, k], where=[>(i, 10)]) +: : +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) : +- Exchange(distribution=[broadcast]) -: +- Calc(select=[i, k], where=[>(i, 10)]) -: +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) +: +- Calc(select=[IS NOT NULL(m) AS $f0]) +: +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) +: +- Exchange(distribution=[single]) +: +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) +: +- Calc(select=[true AS i]) +: +- HashAggregate(isMerge=[true], groupBy=[l], select=[l]) +: +- Exchange(distribution=[hash[l]]) +: +- LocalHashAggregate(groupBy=[l], select=[l]) +: +- Calc(select=[l], where=[LIKE(n, _UTF-16LE'Test')]) +: +- TableSourceScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(l, m, n)]]], fields=[l, m, n]) +- Exchange(distribution=[broadcast]) +- Calc(select=[e]) +- Reused(reference_id=[1]) @@ -679,15 +679,14 @@ Calc(select=[b]) : : +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c]) : : +- Exchange(distribution=[single]) : : +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0]) - : : +- Calc(select=[1 AS EXPR$0]) - : : +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) + : : +- Calc(select=[1 AS EXPR$0], reuse_id=[1]) + : : +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) : +- Exchange(distribution=[broadcast]) : +- Calc(select=[true AS i]) : +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0]) : +- Exchange(distribution=[hash[EXPR$0]]) : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0]) - : +- Calc(select=[1 AS EXPR$0, true AS i]) - : +- Reused(reference_id=[1]) + : +- Reused(reference_id=[1]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d, f]) +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) @@ -1204,15 +1203,14 @@ Calc(select=[b]) : : +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c]) : : +- Exchange(distribution=[single]) : : +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0]) - : : +- Calc(select=[1 AS EXPR$0]) - : : +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) + : : +- Calc(select=[1 AS EXPR$0], reuse_id=[1]) + : : +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) : +- Exchange(distribution=[broadcast]) : +- Calc(select=[true AS i]) : +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0]) : +- Exchange(distribution=[hash[EXPR$0]]) : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0]) - : +- Calc(select=[1 AS EXPR$0, true AS i]) - : +- Reused(reference_id=[1]) + : +- Reused(reference_id=[1]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d]) +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) @@ -1872,19 +1870,19 @@ LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 100))]) =(CAST(c), 1:BIGINT)]) -: : : +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) -: : +- Exchange(distribution=[broadcast]) -: : +- Calc(select=[j], where=[>(CAST(k), 50:BIGINT)]) -: : +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) -: +- Exchange(distribution=[broadcast]) -: +- Calc(select=[i], where=[<(j, 100)]) -: +- Reused(reference_id=[1]) -+- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +NestedLoopJoin(joinType=[LeftAntiJoin], where=[=(a, i)], select=[a, b, c, d, e, f], build=[right]) +:- NestedLoopJoin(joinType=[LeftAntiJoin], where=[=(b, j)], select=[a, b, c, d, e, f], build=[right]) +: :- NestedLoopJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e, f], build=[left]) +: : :- Exchange(distribution=[broadcast]) +: : : +- Calc(select=[a, b, c], where=[>=(CAST(c), 1:BIGINT)]) +: : : +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- Exchange(distribution=[broadcast]) +: +- Calc(select=[j], where=[>(CAST(k), 50:BIGINT)]) +: +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) ++- Exchange(distribution=[broadcast]) + +- Calc(select=[i], where=[<(j, 100)]) + +- Reused(reference_id=[1]) ]]> @@ -2250,28 +2248,26 @@ Calc(select=[b]) : : : : +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) : : : : +- Exchange(distribution=[single]) : : : : +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(i) AS count$1]) - : : : : +- Calc(select=[i]) - : : : : +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) + : : : : +- Calc(select=[i], reuse_id=[1]) + : : : : +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[2]) : : : +- Exchange(distribution=[broadcast]) : : : +- Calc(select=[i, true AS i0]) : : : +- HashAggregate(isMerge=[true], groupBy=[i], select=[i]) : : : +- Exchange(distribution=[hash[i]]) : : : +- LocalHashAggregate(groupBy=[i], select=[i]) - : : : +- Calc(select=[i, true AS i0]) - : : : +- Reused(reference_id=[1]) + : : : +- Reused(reference_id=[1]) : : +- Exchange(distribution=[broadcast]) : : +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) : : +- Exchange(distribution=[single]) : : +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(EXPR$0) AS count$1]) - : : +- Calc(select=[CAST(j) AS EXPR$0]) - : : +- Reused(reference_id=[1]) + : : +- Calc(select=[CAST(j) AS EXPR$0], reuse_id=[3]) + : : +- Reused(reference_id=[2]) : +- Exchange(distribution=[broadcast]) : +- Calc(select=[EXPR$0, true AS i]) : +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0]) : +- Exchange(distribution=[hash[EXPR$0]]) : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0]) - : +- Calc(select=[CAST(j) AS EXPR$0, true AS i]) - : +- Reused(reference_id=[1]) + : +- Reused(reference_id=[3]) +- Exchange(distribution=[broadcast]) +- Calc(select=[d, f]) +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SemiAntiJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SemiAntiJoinTest.xml index a9dfe9d028..52e98cc3ee 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SemiAntiJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SemiAntiJoinTest.xml @@ -100,9 +100,10 @@ LogicalFilter(condition=[=($cor1.a, $0)]) ($cor0.b, $1)]) (b, e)], select=[a, b, c], build=[right]) -:- HashJoin(joinType=[LeftAntiJoin], where=[AND(OR(=(b, i), IS NULL(b), IS NULL(i)), =(c, k))], select=[a, b, c], build=[left]) -: :- Exchange(distribution=[hash[c]]) -: : +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[a, b, c], build=[right], singleRowJoin=[true]) -: : :- HashJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], build=[right]) -: : : :- Exchange(distribution=[hash[a]]) -: : : : +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) -: : : +- Exchange(distribution=[hash[d]]) -: : : +- Calc(select=[d]) -: : : +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f], reuse_id=[1]) -: : +- Exchange(distribution=[broadcast]) -: : +- Calc(select=[IS NOT NULL(m) AS $f0]) -: : +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) -: : +- Exchange(distribution=[single]) -: : +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) -: : +- Calc(select=[true AS i]) -: : +- HashAggregate(isMerge=[true], groupBy=[l], select=[l]) -: : +- Exchange(distribution=[hash[l]]) -: : +- LocalHashAggregate(groupBy=[l], select=[l]) -: : +- Calc(select=[l], where=[LIKE(n, _UTF-16LE'Test')]) -: : +- TableSourceScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(l, m, n)]]], fields=[l, m, n]) -: +- Exchange(distribution=[hash[k]]) -: +- Calc(select=[i, k], where=[>(i, 10)]) -: +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) +:- NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[a, b, c], build=[right], singleRowJoin=[true]) +: :- HashJoin(joinType=[LeftAntiJoin], where=[AND(OR(=(b, i), IS NULL(b), IS NULL(i)), =(c, k))], select=[a, b, c], build=[right]) +: : :- Exchange(distribution=[hash[c]]) +: : : +- HashJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], build=[right]) +: : : :- Exchange(distribution=[hash[a]]) +: : : : +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : : +- Exchange(distribution=[hash[d]]) +: : : +- Calc(select=[d]) +: : : +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f], reuse_id=[1]) +: : +- Exchange(distribution=[hash[k]]) +: : +- Calc(select=[i, k], where=[>(i, 10)]) +: : +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) +: +- Exchange(distribution=[broadcast]) +: +- Calc(select=[IS NOT NULL(m) AS $f0]) +: +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m]) +: +- Exchange(distribution=[single]) +: +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0]) +: +- Calc(select=[true AS i]) +: +- HashAggregate(isMerge=[true], groupBy=[l], select=[l]) +: +- Exchange(distribution=[hash[l]]) +: +- LocalHashAggregate(groupBy=[l], select=[l]) +: +- Calc(select=[l], where=[LIKE(n, _UTF-16LE'Test')]) +: +- TableSourceScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(l, m, n)]]], fields=[l, m, n]) +- Exchange(distribution=[broadcast]) +- Calc(select=[e]) +- Reused(reference_id=[1]) @@ -694,15 +695,14 @@ Calc(select=[b]) : : +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c]) : : +- Exchange(distribution=[single]) : : +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0]) - : : +- Calc(select=[1 AS EXPR$0]) - : : +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) + : : +- Calc(select=[1 AS EXPR$0], reuse_id=[1]) + : : +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) : +- Exchange(distribution=[broadcast]) : +- Calc(select=[true AS i]) : +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0]) : +- Exchange(distribution=[hash[EXPR$0]]) : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0]) - : +- Calc(select=[1 AS EXPR$0, true AS i]) - : +- Reused(reference_id=[1]) + : +- Reused(reference_id=[1]) +- Exchange(distribution=[hash[d, f]]) +- Calc(select=[d, f]) +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) @@ -839,9 +839,10 @@ LogicalProject(f1=[$3]) =(CAST(c), 1:BIGINT)]) -: : : +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) -: : +- Exchange(distribution=[hash[j]]) -: : +- Calc(select=[j], where=[>(CAST(k), 50:BIGINT)]) -: : +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) -: +- Exchange(distribution=[hash[i]]) -: +- Calc(select=[i], where=[<(j, 100)]) -: +- Reused(reference_id=[1]) -+- Exchange(distribution=[hash[d]]) - +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +HashJoin(joinType=[LeftAntiJoin], where=[=(a, i)], select=[a, b, c, d, e, f], build=[right]) +:- Exchange(distribution=[hash[a]]) +: +- HashJoin(joinType=[LeftAntiJoin], where=[=(b, j)], select=[a, b, c, d, e, f], build=[right]) +: :- Exchange(distribution=[hash[b]]) +: : +- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e, f], build=[left]) +: : :- Exchange(distribution=[hash[a]]) +: : : +- Calc(select=[a, b, c], where=[>=(CAST(c), 1:BIGINT)]) +: : : +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- Exchange(distribution=[hash[d]]) +: : +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- Exchange(distribution=[hash[j]]) +: +- Calc(select=[j], where=[>(CAST(k), 50:BIGINT)]) +: +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) ++- Exchange(distribution=[hash[i]]) + +- Calc(select=[i], where=[<(j, 100)]) + +- Reused(reference_id=[1]) ]]> @@ -2315,26 +2317,24 @@ Calc(select=[b]) : : : : +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) : : : : +- Exchange(distribution=[single]) : : : : +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(i) AS count$1]) - : : : : +- Calc(select=[i]) - : : : : +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) + : : : : +- Calc(select=[i], reuse_id=[1]) + : : : : +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[2]) : : : +- Calc(select=[i, true AS i0]) : : : +- HashAggregate(isMerge=[true], groupBy=[i], select=[i]) : : : +- Exchange(distribution=[hash[i]]) : : : +- LocalHashAggregate(groupBy=[i], select=[i]) - : : : +- Calc(select=[i, true AS i0]) - : : : +- Reused(reference_id=[1]) + : : : +- Reused(reference_id=[1]) : : +- Exchange(distribution=[broadcast]) : : +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) : : +- Exchange(distribution=[single]) : : +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(EXPR$0) AS count$1]) - : : +- Calc(select=[CAST(j) AS EXPR$0]) - : : +- Reused(reference_id=[1]) + : : +- Calc(select=[CAST(j) AS EXPR$0], reuse_id=[3]) + : : +- Reused(reference_id=[2]) : +- Calc(select=[EXPR$0, true AS i]) : +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0]) : +- Exchange(distribution=[hash[EXPR$0]]) : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0]) - : +- Calc(select=[CAST(j) AS EXPR$0, true AS i]) - : +- Reused(reference_id=[1]) + : +- Reused(reference_id=[3]) +- Exchange(distribution=[hash[f]]) +- Calc(select=[d, f]) +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.xml index 5100b1c62b..07ac0fb592 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.xml @@ -79,6 +79,34 @@ HashJoin(joinType=[LeftSemiJoin], where=[AND(=(b, e), =(c, f))], select=[a, b, c +- LocalHashAggregate(groupBy=[d, e, f], select=[d, e, f]) +- Calc(select=[d, e, f], where=[<(d, 100)]) +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +]]> + + + + + + + + + + + @@ -636,6 +664,63 @@ Calc(select=[c]) +- Exchange(distribution=[hash[i]]) +- Calc(select=[i, k], where=[>(i, 10)]) +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) +]]> + + + + + + + + + + + + + + + + + + + + + + @@ -1435,21 +1520,22 @@ LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 100))]) =(CAST(c), 1:BIGINT)]) -: : : +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) -: : +- Exchange(distribution=[hash[j]]) -: : +- Calc(select=[j], where=[>(CAST(k), 50:BIGINT)]) -: : +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) -: +- Exchange(distribution=[hash[i]]) -: +- Calc(select=[i], where=[<(j, 100)]) -: +- Reused(reference_id=[1]) -+- Exchange(distribution=[hash[d]]) - +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +HashJoin(joinType=[LeftAntiJoin], where=[=(a, i)], select=[a, b, c, d, e, f], build=[right]) +:- Exchange(distribution=[hash[a]]) +: +- HashJoin(joinType=[LeftAntiJoin], where=[=(b, j)], select=[a, b, c, d, e, f], build=[right]) +: :- Exchange(distribution=[hash[b]]) +: : +- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e, f], build=[left]) +: : :- Exchange(distribution=[hash[a]]) +: : : +- Calc(select=[a, b, c], where=[>=(CAST(c), 1:BIGINT)]) +: : : +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- Exchange(distribution=[hash[d]]) +: : +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- Exchange(distribution=[hash[j]]) +: +- Calc(select=[j], where=[>(CAST(k), 50:BIGINT)]) +: +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) ++- Exchange(distribution=[hash[i]]) + +- Calc(select=[i], where=[<(j, 100)]) + +- Reused(reference_id=[1]) ]]> @@ -1909,26 +1995,24 @@ Calc(select=[b]) : : : : +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) : : : : +- Exchange(distribution=[single]) : : : : +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(i) AS count$1]) - : : : : +- Calc(select=[i]) - : : : : +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) + : : : : +- Calc(select=[i], reuse_id=[1]) + : : : : +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[2]) : : : +- Calc(select=[i, true AS i0]) : : : +- HashAggregate(isMerge=[true], groupBy=[i], select=[i]) : : : +- Exchange(distribution=[hash[i]]) : : : +- LocalHashAggregate(groupBy=[i], select=[i]) - : : : +- Calc(select=[i, true AS i0]) - : : : +- Reused(reference_id=[1]) + : : : +- Reused(reference_id=[1]) : : +- Exchange(distribution=[broadcast]) : : +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) : : +- Exchange(distribution=[single]) : : +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(EXPR$0) AS count$1]) - : : +- Calc(select=[CAST(j) AS EXPR$0]) - : : +- Reused(reference_id=[1]) + : : +- Calc(select=[CAST(j) AS EXPR$0], reuse_id=[3]) + : : +- Reused(reference_id=[2]) : +- Calc(select=[EXPR$0, true AS i]) : +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0]) : +- Exchange(distribution=[hash[EXPR$0]]) : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0]) - : +- Calc(select=[CAST(j) AS EXPR$0, true AS i]) - : +- Reused(reference_id=[1]) + : +- Reused(reference_id=[3]) +- Exchange(distribution=[hash[f]]) +- Calc(select=[d, f]) +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.xml index 54bb7209d9..b9c22aadbf 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.xml @@ -1520,21 +1520,22 @@ LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 100))]) =(CAST(c), 1:BIGINT)]) -: : : +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) -: : +- Exchange(distribution=[hash[j]]) -: : +- Calc(select=[j], where=[>(CAST(k), 50:BIGINT)]) -: : +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) -: +- Exchange(distribution=[hash[i]]) -: +- Calc(select=[i], where=[<(j, 100)]) -: +- Reused(reference_id=[1]) -+- Exchange(distribution=[hash[d]]) - +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +SortMergeJoin(joinType=[LeftAntiJoin], where=[=(a, i)], select=[a, b, c, d, e, f]) +:- Exchange(distribution=[hash[a]]) +: +- SortMergeJoin(joinType=[LeftAntiJoin], where=[=(b, j)], select=[a, b, c, d, e, f]) +: :- Exchange(distribution=[hash[b]]) +: : +- SortMergeJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e, f]) +: : :- Exchange(distribution=[hash[a]]) +: : : +- Calc(select=[a, b, c], where=[>=(CAST(c), 1:BIGINT)]) +: : : +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- Exchange(distribution=[hash[d]]) +: : +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- Exchange(distribution=[hash[j]]) +: +- Calc(select=[j], where=[>(CAST(k), 50:BIGINT)]) +: +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) ++- Exchange(distribution=[hash[i]]) + +- Calc(select=[i], where=[<(j, 100)]) + +- Reused(reference_id=[1]) ]]> @@ -1994,26 +1995,24 @@ Calc(select=[b]) : : : : +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) : : : : +- Exchange(distribution=[single]) : : : : +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(i) AS count$1]) - : : : : +- Calc(select=[i]) - : : : : +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) + : : : : +- Calc(select=[i], reuse_id=[1]) + : : : : +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[2]) : : : +- Calc(select=[i, true AS i0]) : : : +- HashAggregate(isMerge=[true], groupBy=[i], select=[i]) : : : +- Exchange(distribution=[hash[i]]) : : : +- LocalHashAggregate(groupBy=[i], select=[i]) - : : : +- Calc(select=[i, true AS i0]) - : : : +- Reused(reference_id=[1]) + : : : +- Reused(reference_id=[1]) : : +- Exchange(distribution=[broadcast]) : : +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) : : +- Exchange(distribution=[single]) : : +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(EXPR$0) AS count$1]) - : : +- Calc(select=[CAST(j) AS EXPR$0]) - : : +- Reused(reference_id=[1]) + : : +- Calc(select=[CAST(j) AS EXPR$0], reuse_id=[3]) + : : +- Reused(reference_id=[2]) : +- Calc(select=[EXPR$0, true AS i]) : +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0]) : +- Exchange(distribution=[hash[EXPR$0]]) : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0]) - : +- Calc(select=[CAST(j) AS EXPR$0, true AS i]) - : +- Reused(reference_id=[1]) + : +- Reused(reference_id=[3]) +- Exchange(distribution=[hash[f]]) +- Calc(select=[d, f]) +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/JoinDependentConditionDerivationRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/JoinDependentConditionDerivationRuleTest.xml index ce66d1f253..919e964ecb 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/JoinDependentConditionDerivationRuleTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/JoinDependentConditionDerivationRuleTest.xml @@ -32,7 +32,7 @@ LogicalProject(a=[$0], d=[$3]) @@ -54,7 +54,7 @@ LogicalProject(a=[$0], d=[$3]) @@ -98,7 +98,7 @@ LogicalProject(a=[$0], d=[$3]) @@ -120,7 +120,7 @@ LogicalProject(a=[$0], d=[$3]) @@ -152,7 +152,7 @@ LogicalProject(a=[$0], d=[$6]) @@ -199,7 +199,7 @@ LogicalProject(a=[$0], d=[$3]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/FlinkRewriteSubQueryRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/FlinkRewriteSubQueryRuleTest.xml index 19dfb772d0..1ec0d2694e 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/FlinkRewriteSubQueryRuleTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/FlinkRewriteSubQueryRuleTest.xml @@ -156,7 +156,7 @@ LogicalSort(sort0=[$0], dir0=[ASC-nulls-first], fetch=[100]) +- LogicalJoin(condition=[=($7, $1)], joinType=[semi]) :- LogicalTableScan(table=[[default_catalog, default_database, item, source: [TestTableSource(i_manufact_id, i_manufact, i_product_name, i_category, i_color, i_units, i_size)]]]) +- LogicalProject(i_manufact=[$1]) - +- LogicalFilter(condition=[OR(AND(=($3, _UTF-16LE'Women'), OR(=($4, _UTF-16LE'powder'), =($4, _UTF-16LE'khaki')), OR(=($5, _UTF-16LE'Ounce'), =($5, _UTF-16LE'Oz')), OR(=($6, _UTF-16LE'medium'), =($6, _UTF-16LE'extra large'))), AND(=($3, _UTF-16LE'Women'), OR(=($4, _UTF-16LE'brown'), =($4, _UTF-16LE'honeydew')), OR(=($5, _UTF-16LE'Bunch'), =($5, _UTF-16LE'Ton')), OR(=($6, _UTF-16LE'N/A'), =($6, _UTF-16LE'small'))), AND(=($3, _UTF-16LE'Men'), OR(=($4, _UTF-16LE'floral'), =($4, _UTF-16LE'deep')), OR(=($5, _UTF-16LE'N/A'), =($5, _UTF-16LE'Dozen')), OR(=($6, _UTF-16LE'petite'), =($6, _UTF-16LE'large'))), AND(=($3, _UTF-16LE'Men'), OR(=($4, _UTF-16LE'light'), =($4, _UTF-16LE'cornflower')), OR(=($5, _UTF-16LE'Box'), =($5, _UTF-16LE'Pound')), OR(=($6, _UTF-16LE'medium'), =($6, _UTF-16LE'extra large'))), AND(=($3, _UTF-16LE'Women'), OR(=($4, _UTF-16LE'midnight'), =($4, _UTF-16LE'snow')), OR(=($5, _UTF-16LE'Pallet'), =($5, _UTF-16LE'Gross')), OR(=($6, _UTF-16LE'medium'), =($6, _UTF-16LE'extra large'))), AND(=($3, _UTF-16LE'Women'), OR(=($4, _UTF-16LE'cyan'), =($4, _UTF-16LE'papaya')), OR(=($5, _UTF-16LE'Cup'), =($5, _UTF-16LE'Dram')), OR(=($6, _UTF-16LE'N/A'), =($6, _UTF-16LE'small'))), AND(=($3, _UTF-16LE'Men'), OR(=($4, _UTF-16LE'orange'), =($4, _UTF-16LE'frosted')), OR(=($5, _UTF-16LE'Each'), =($5, _UTF-16LE'Tbl')), OR(=($6, _UTF-16LE'petite'), =($6, _UTF-16LE'large'))), AND(=($3, _UTF-16LE'Men'), OR(=($4, _UTF-16LE'forest'), =($4, _UTF-16LE'ghost')), OR(=($5, _UTF-16LE'Lb'), =($5, _UTF-16LE'Bundle')), OR(=($6, _UTF-16LE'medium'), =($6, _UTF-16LE'extra large'))))]) + +- LogicalFilter(condition=[OR(AND(=($3, _UTF-16LE'Women'), OR(=($4, _UTF-16LE'powder'), =($4, _UTF-16LE'khaki')), OR(=($5, _UTF-16LE'Ounce'), =($5, _UTF-16LE'Oz')), OR(=($6, _UTF-16LE'medium'), =($6, _UTF-16LE'extra large'))), AND(=(_UTF-16LE'Women', $3), OR(=($4, _UTF-16LE'brown'), =($4, _UTF-16LE'honeydew')), OR(=($5, _UTF-16LE'Bunch'), =($5, _UTF-16LE'Ton')), OR(=($6, _UTF-16LE'N/A'), =($6, _UTF-16LE'small'))), AND(=($3, _UTF-16LE'Men'), OR(=($4, _UTF-16LE'floral'), =($4, _UTF-16LE'deep')), OR(=($5, _UTF-16LE'N/A'), =($5, _UTF-16LE'Dozen')), OR(=($6, _UTF-16LE'petite'), =($6, _UTF-16LE'large'))), AND(=(_UTF-16LE'Men', $3), OR(=($4, _UTF-16LE'light'), =($4, _UTF-16LE'cornflower')), OR(=($5, _UTF-16LE'Box'), =($5, _UTF-16LE'Pound')), OR(=(_UTF-16LE'medium', $6), =(_UTF-16LE'extra large', $6))), AND(=(_UTF-16LE'Women', $3), OR(=($4, _UTF-16LE'midnight'), =($4, _UTF-16LE'snow')), OR(=($5, _UTF-16LE'Pallet'), =($5, _UTF-16LE'Gross')), OR(=(_UTF-16LE'medium', $6), =(_UTF-16LE'extra large', $6))), AND(=(_UTF-16LE'Women', $3), OR(=($4, _UTF-16LE'cyan'), =($4, _UTF-16LE'papaya')), OR(=($5, _UTF-16LE'Cup'), =($5, _UTF-16LE'Dram')), OR(=(_UTF-16LE'N/A', $6), =(_UTF-16LE'small', $6))), AND(=(_UTF-16LE'Men', $3), OR(=($4, _UTF-16LE'orange'), =($4, _UTF-16LE'frosted')), OR(=($5, _UTF-16LE'Each'), =($5, _UTF-16LE'Tbl')), OR(=(_UTF-16LE'petite', $6), =(_UTF-16LE'large', $6))), AND(=(_UTF-16LE'Men', $3), OR(=($4, _UTF-16LE'forest'), =($4, _UTF-16LE'ghost')), OR(=($5, _UTF-16LE'Lb'), =($5, _UTF-16LE'Bundle')), OR(=(_UTF-16LE'medium', $6), =(_UTF-16LE'extra large', $6))))]) +- LogicalTableScan(table=[[default_catalog, default_database, item, source: [TestTableSource(i_manufact_id, i_manufact, i_product_name, i_category, i_color, i_units, i_size)]]]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryAntiJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryAntiJoinTest.xml index 7736deae17..856ce5986b 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryAntiJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryAntiJoinTest.xml @@ -46,22 +46,22 @@ LogicalFilter(condition=[<>($cor0.b, $1)]) ($1, $3)], joinType=[anti]) - :- LogicalJoin(condition=[AND(OR(=($1, $3), IS NULL($1), IS NULL($3)), =($2, $4))], joinType=[anti]) - : :- LogicalJoin(condition=[$3], joinType=[semi]) + :- LogicalJoin(condition=[$3], joinType=[semi]) + : :- LogicalJoin(condition=[AND(OR(=($1, $3), IS NULL($1), IS NULL($3)), =($2, $4))], joinType=[anti]) : : :- LogicalJoin(condition=[=($0, $3)], joinType=[semi]) : : : :- LogicalTableScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]]) : : : +- LogicalProject(d=[$0]) : : : +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) - : : +- LogicalProject($f0=[IS NOT NULL($0)]) - : : +- LogicalAggregate(group=[{}], m=[MIN($0)]) - : : +- LogicalProject(i=[true]) - : : +- LogicalAggregate(group=[{0}], EXPR$0=[COUNT($0)]) - : : +- LogicalProject(l=[$0]) - : : +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'Test')]) - : : +- LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(l, m, n)]]]) - : +- LogicalProject(i=[$0], k=[$2]) - : +- LogicalFilter(condition=[>($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + : : +- LogicalProject(i=[$0], k=[$2]) + : : +- LogicalFilter(condition=[>($0, 10)]) + : : +- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]]) + : +- LogicalProject($f0=[IS NOT NULL($0)]) + : +- LogicalAggregate(group=[{}], m=[MIN($0)]) + : +- LogicalProject(i=[true]) + : +- LogicalAggregate(group=[{0}], EXPR$0=[COUNT($0)]) + : +- LogicalProject(l=[$0]) + : +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'Test')]) + : +- LogicalTableScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(l, m, n)]]]) +- LogicalProject(e=[$1]) +- LogicalFilter(condition=[true]) +- LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.xml index 390832adda..5c176e5a5c 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.xml @@ -234,7 +234,7 @@ LogicalProject(a=[$0], EXPR$1=[$2], c=[$1]) Calc(select=[a, EXPR$1, _UTF-16LE'test' AS c]) +- GroupAggregate(groupBy=[a], select=[a, MAX(b) AS EXPR$1]) +- Exchange(distribution=[hash[a]]) - +- Calc(select=[a, _UTF-16LE'test' AS c, b]) + +- Calc(select=[a, b]) +- TableSourceScan(table=[[default_catalog, default_database, T, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/TwoStageAggregateTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/TwoStageAggregateTest.xml index ddae15876c..55029e6928 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/TwoStageAggregateTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/TwoStageAggregateTest.xml @@ -175,7 +175,7 @@ Calc(select=[4 AS four, EXPR$1]) +- GlobalGroupAggregate(groupBy=[b], select=[b, SUM(sum$0) AS EXPR$1]) +- Exchange(distribution=[hash[b]]) +- LocalGroupAggregate(groupBy=[b], select=[b, SUM(a) AS sum$0]) - +- Calc(select=[b, 4 AS four, a]) + +- Calc(select=[b, a]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/SemiAntiJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/SemiAntiJoinTest.xml index 7ef21d99cc..c2dfa03041 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/SemiAntiJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/SemiAntiJoinTest.xml @@ -516,28 +516,28 @@ LogicalFilter(condition=[<>($cor0.b, $1)]) (b, e)], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[single]) -: +- Join(joinType=[LeftAntiJoin], where=[AND(OR(=(b, i), IS NULL(b), IS NULL(i)), =(c, k))], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) -: :- Exchange(distribution=[hash[c]]) -: : +- Join(joinType=[LeftSemiJoin], where=[$f0], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) -: : :- Exchange(distribution=[single]) +: +- Join(joinType=[LeftSemiJoin], where=[$f0], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) +: :- Exchange(distribution=[single]) +: : +- Join(joinType=[LeftAntiJoin], where=[AND(OR(=(b, i), IS NULL(b), IS NULL(i)), =(c, k))], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) +: : :- Exchange(distribution=[hash[c]]) : : : +- Join(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) : : : :- Exchange(distribution=[hash[a]]) : : : : +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) : : : +- Exchange(distribution=[hash[d]]) : : : +- Calc(select=[d]) : : : +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f], reuse_id=[1]) -: : +- Exchange(distribution=[single]) -: : +- Calc(select=[IS NOT NULL(m) AS $f0]) -: : +- GroupAggregate(select=[MIN(i) AS m]) -: : +- Exchange(distribution=[single]) -: : +- Calc(select=[true AS i]) -: : +- GroupAggregate(groupBy=[l], select=[l]) -: : +- Exchange(distribution=[hash[l]]) -: : +- Calc(select=[l], where=[LIKE(n, _UTF-16LE'Test')]) -: : +- TableSourceScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(l, m, n)]]], fields=[l, m, n]) -: +- Exchange(distribution=[hash[k]]) -: +- Calc(select=[i, k], where=[>(i, 10)]) -: +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) +: : +- Exchange(distribution=[hash[k]]) +: : +- Calc(select=[i, k], where=[>(i, 10)]) +: : +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) +: +- Exchange(distribution=[single]) +: +- Calc(select=[IS NOT NULL(m) AS $f0]) +: +- GroupAggregate(select=[MIN(i) AS m]) +: +- Exchange(distribution=[single]) +: +- Calc(select=[true AS i]) +: +- GroupAggregate(groupBy=[l], select=[l]) +: +- Exchange(distribution=[hash[l]]) +: +- Calc(select=[l], where=[LIKE(n, _UTF-16LE'Test')]) +: +- TableSourceScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(l, m, n)]]], fields=[l, m, n]) +- Exchange(distribution=[single]) +- Calc(select=[e]) +- Reused(reference_id=[1]) @@ -692,14 +692,13 @@ Calc(select=[b]) : : +- Exchange(distribution=[single]) : : +- GroupAggregate(select=[COUNT(*) AS c]) : : +- Exchange(distribution=[single]) - : : +- Calc(select=[1 AS EXPR$0]) - : : +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) + : : +- Calc(select=[1 AS EXPR$0], reuse_id=[1]) + : : +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) : +- Exchange(distribution=[single]) : +- Calc(select=[true AS i]) : +- GroupAggregate(groupBy=[EXPR$0], select=[EXPR$0]) : +- Exchange(distribution=[hash[EXPR$0]]) - : +- Calc(select=[1 AS EXPR$0, true AS i]) - : +- Reused(reference_id=[1]) + : +- Reused(reference_id=[1]) +- Exchange(distribution=[hash[d, f]]) +- Calc(select=[d, f]) +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) @@ -1235,14 +1234,13 @@ Calc(select=[b]) : : +- Exchange(distribution=[single]) : : +- GroupAggregate(select=[COUNT(*) AS c]) : : +- Exchange(distribution=[single]) - : : +- Calc(select=[1 AS EXPR$0]) - : : +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) + : : +- Calc(select=[1 AS EXPR$0], reuse_id=[1]) + : : +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) : +- Exchange(distribution=[single]) : +- Calc(select=[true AS i]) : +- GroupAggregate(groupBy=[EXPR$0], select=[EXPR$0]) : +- Exchange(distribution=[hash[EXPR$0]]) - : +- Calc(select=[1 AS EXPR$0, true AS i]) - : +- Reused(reference_id=[1]) + : +- Reused(reference_id=[1]) +- Exchange(distribution=[hash[d]]) +- Calc(select=[d]) +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) @@ -1923,22 +1921,22 @@ LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 100))]) =(CAST(c), 1:BIGINT)]) : : : +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) -: : +- Exchange(distribution=[hash[j]]) -: : +- Calc(select=[j], where=[>(CAST(k), 50:BIGINT)]) -: : +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) -: +- Exchange(distribution=[hash[i]]) -: +- Calc(select=[i], where=[<(j, 100)]) -: +- Reused(reference_id=[1]) -+- Exchange(distribution=[hash[d]]) - +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: : +- Exchange(distribution=[hash[d]]) +: : +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +: +- Exchange(distribution=[hash[j]]) +: +- Calc(select=[j], where=[>(CAST(k), 50:BIGINT)]) +: +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) ++- Exchange(distribution=[hash[i]]) + +- Calc(select=[i], where=[<(j, 100)]) + +- Reused(reference_id=[1]) ]]> @@ -2287,25 +2285,23 @@ Calc(select=[b]) : : : : +- Exchange(distribution=[single]) : : : : +- GroupAggregate(select=[COUNT(*) AS c, COUNT(i) AS ck]) : : : : +- Exchange(distribution=[single]) - : : : : +- Calc(select=[i]) - : : : : +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) + : : : : +- Calc(select=[i], reuse_id=[1]) + : : : : +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[2]) : : : +- Exchange(distribution=[hash[i]]) : : : +- Calc(select=[i, true AS i0]) : : : +- GroupAggregate(groupBy=[i], select=[i]) : : : +- Exchange(distribution=[hash[i]]) - : : : +- Calc(select=[i, true AS i0]) - : : : +- Reused(reference_id=[1]) + : : : +- Reused(reference_id=[1]) : : +- Exchange(distribution=[single]) : : +- GroupAggregate(select=[COUNT(*) AS c, COUNT(EXPR$0) AS ck]) : : +- Exchange(distribution=[single]) - : : +- Calc(select=[CAST(j) AS EXPR$0]) - : : +- Reused(reference_id=[1]) + : : +- Calc(select=[CAST(j) AS EXPR$0], reuse_id=[3]) + : : +- Reused(reference_id=[2]) : +- Exchange(distribution=[hash[EXPR$0]]) : +- Calc(select=[EXPR$0, true AS i]) : +- GroupAggregate(groupBy=[EXPR$0], select=[EXPR$0]) : +- Exchange(distribution=[hash[EXPR$0]]) - : +- Calc(select=[CAST(j) AS EXPR$0, true AS i]) - : +- Reused(reference_id=[1]) + : +- Reused(reference_id=[3]) +- Exchange(distribution=[hash[f]]) +- Calc(select=[d, f]) +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/AggregateTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/AggregateTest.xml index fd6047b216..e4218f05bf 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/AggregateTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/AggregateTest.xml @@ -130,7 +130,7 @@ LogicalProject(four=[$1], EXPR$0=[$2]) Calc(select=[4 AS four, EXPR$0]) +- GroupAggregate(groupBy=[a], select=[a, SUM(b) AS EXPR$0]) +- Exchange(distribution=[hash[a]]) - +- Calc(select=[a, 4 AS four, b]) + +- Calc(select=[a, b]) +- TableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -149,7 +149,7 @@ LogicalProject(four=[$1], EXPR$0=[$2]) Calc(select=[4 AS four, EXPR$0]) +- GroupAggregate(groupBy=[b], select=[b, SUM(a) AS EXPR$0]) +- Exchange(distribution=[hash[b]]) - +- Calc(select=[b, 4 AS four, a]) + +- Calc(select=[b, a]) +- TableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TwoStageAggregateTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TwoStageAggregateTest.xml index 8b595885f6..9bcdc6b35b 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TwoStageAggregateTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TwoStageAggregateTest.xml @@ -91,7 +91,7 @@ Calc(select=[4 AS four, EXPR$0]) +- GlobalGroupAggregate(groupBy=[a], select=[a, SUM(sum$0) AS EXPR$0]) +- Exchange(distribution=[hash[a]]) +- LocalGroupAggregate(groupBy=[a], select=[a, SUM(b) AS sum$0]) - +- Calc(select=[a, 4 AS four, b]) + +- Calc(select=[a, b]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) +- TableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -112,7 +112,7 @@ Calc(select=[4 AS four, EXPR$0]) +- GlobalGroupAggregate(groupBy=[b], select=[b, SUM(sum$0) AS EXPR$0]) +- Exchange(distribution=[hash[b]]) +- LocalGroupAggregate(groupBy=[b], select=[b, SUM(a) AS sum$0]) - +- Calc(select=[b, 4 AS four, a]) + +- Calc(select=[b, a]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime]) +- TableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/DecimalTypeTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/DecimalTypeTest.scala index 8848881617..d62a84dbc7 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/DecimalTypeTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/DecimalTypeTest.scala @@ -704,13 +704,13 @@ class DecimalTypeTest extends ExpressionTestBase { 'f42 % 'f41, "f42 % f41", "mod(f42, f41)", - "2.00") + "2.0000") testAllApis( 'f41 % 'f43, "f41 % f43", "mod(f41, f43)", - "3") + "3.00") testAllApis( 'f43 % 'f41, @@ -749,7 +749,7 @@ class DecimalTypeTest extends ExpressionTestBase { 'f46 % 'f47, "f46 % f47", "mod(f46, f47)", - "3.12") + "3.1234") } @Test // functions that treat Decimal as exact value diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.scala index 09d4916471..f4814c4cdf 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.scala @@ -162,25 +162,4 @@ class ShuffledHashSemiAntiJoinTest extends SemiAntiJoinTestBase { super.testNotInWithUncorrelated_SimpleCondition3() } - @Test - override def testExistsWithCorrelated_LateralTableInSubQuery(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testExistsWithCorrelated_LateralTableInSubQuery() - } - - @Test - override def testInWithUncorrelated_LateralTableInSubQuery(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testInWithUncorrelated_LateralTableInSubQuery() - } - - @Test - override def testInWithCorrelated_LateralTableInSubQuery(): Unit = { - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testInWithCorrelated_LateralTableInSubQuery() - } - } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/FlinkRexUtilTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/FlinkRexUtilTest.scala index 29ee98dc24..12e2ee8994 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/FlinkRexUtilTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/FlinkRexUtilTest.scala @@ -43,7 +43,7 @@ class FlinkRexUtilTest { val i_size = rexBuilder.makeInputRef(varcharType, 4) // this predicate contains 95 RexCalls. however, - // if this predicate is converted to CNF, the result contains 736450 RexCalls. + // if this predicate is converted to CNF, the result contains 557715 RexCalls. val predicate = rexBuilder.makeCall(OR, rexBuilder.makeCall(AND, rexBuilder.makeCall(EQUALS, i_manufact, rexBuilder.makeLiteral("able")), @@ -181,10 +181,10 @@ class FlinkRexUtilTest { val newPredicate1 = FlinkRexUtil.toCnf(rexBuilder, -1, predicate) assertEquals(predicate.toString, newPredicate1.toString) - val newPredicate2 = FlinkRexUtil.toCnf(rexBuilder, 736449, predicate) + val newPredicate2 = FlinkRexUtil.toCnf(rexBuilder, 557714, predicate) assertEquals(predicate.toString, newPredicate2.toString) - val newPredicate3 = FlinkRexUtil.toCnf(rexBuilder, 736450, predicate) + val newPredicate3 = FlinkRexUtil.toCnf(rexBuilder, 557715, predicate) assertEquals(RexUtil.toCnf(rexBuilder, predicate).toString, newPredicate3.toString) val newPredicate4 = FlinkRexUtil.toCnf(rexBuilder, Int.MaxValue, predicate) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/QueryOperationCatalogViewTable.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/QueryOperationCatalogViewTable.java index cbb3c5d897..dfcced1595 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/QueryOperationCatalogViewTable.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/QueryOperationCatalogViewTable.java @@ -28,10 +28,15 @@ import org.apache.calcite.plan.RelOptUtil; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.calcite.rel.type.RelDataTypeFieldImpl; import org.apache.calcite.rel.type.RelProtoDataType; import org.apache.calcite.schema.TranslatableTable; import org.apache.calcite.schema.impl.AbstractTable; +import java.util.List; +import java.util.stream.Collectors; + /** * A bridge between a Flink's specific {@link QueryOperationCatalogView} and a Calcite's * {@link org.apache.calcite.schema.Table}. It implements {@link TranslatableTable} interface. This enables @@ -49,7 +54,30 @@ public class QueryOperationCatalogViewTable extends AbstractTable implements Tra public static QueryOperationCatalogViewTable createCalciteTable(QueryOperationCatalogView catalogView) { return new QueryOperationCatalogViewTable(catalogView, typeFactory -> { TableSchema tableSchema = catalogView.getSchema(); - return ((FlinkTypeFactory) typeFactory).buildLogicalRowType(tableSchema); + final FlinkTypeFactory flinkTypeFactory = (FlinkTypeFactory) typeFactory; + final RelDataType relType = flinkTypeFactory.buildLogicalRowType(tableSchema); + Boolean[] nullables = tableSchema + .getTableColumns() + .stream() + .map(c -> c.getType().getLogicalType().isNullable()) + .toArray(Boolean[]::new); + final List fields = relType + .getFieldList() + .stream() + .map(f -> { + boolean nullable = nullables[f.getIndex()]; + if (nullable != f.getType().isNullable() + && !FlinkTypeFactory.isTimeIndicatorType(f.getType())) { + return new RelDataTypeFieldImpl( + f.getName(), + f.getIndex(), + flinkTypeFactory.createTypeWithNullability(f.getType(), nullable)); + } else { + return f; + } + }) + .collect(Collectors.toList()); + return flinkTypeFactory.createStructType(fields); }); } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/operations/PlannerQueryOperation.java b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/operations/PlannerQueryOperation.java index c30d741913..d5e581a2b9 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/operations/PlannerQueryOperation.java +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/operations/PlannerQueryOperation.java @@ -19,9 +19,10 @@ package org.apache.flink.table.operations; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.calcite.FlinkTypeFactory; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.utils.TypeConversions; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.type.RelDataType; @@ -43,11 +44,22 @@ public class PlannerQueryOperation implements QueryOperation { RelDataType rowType = calciteTree.getRowType(); String[] fieldNames = rowType.getFieldNames().toArray(new String[0]); - TypeInformation[] fieldTypes = rowType.getFieldList() + DataType[] fieldTypes = rowType.getFieldList() .stream() - .map(field -> FlinkTypeFactory.toTypeInfo(field.getType())).toArray(TypeInformation[]::new); + .map(field -> { + final DataType fieldType = TypeConversions + .fromLegacyInfoToDataType(FlinkTypeFactory.toTypeInfo(field.getType())); + final boolean nullable = field.getType().isNullable(); + if (nullable != fieldType.getLogicalType().isNullable() + && !FlinkTypeFactory.isTimeIndicatorType(field.getType())) { + return nullable ? fieldType.nullable() : fieldType.notNull(); + } else { + return fieldType; + } + }) + .toArray(DataType[]::new); - this.tableSchema = new TableSchema(fieldNames, fieldTypes); + this.tableSchema = TableSchema.builder().fields(fieldNames, fieldTypes).build(); } public RelNode getCalciteTree() { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/sql/GroupWindowTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/sql/GroupWindowTest.scala index 77a5a831f9..74f71c17ec 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/sql/GroupWindowTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/sql/GroupWindowTest.scala @@ -376,11 +376,11 @@ class GroupWindowTest extends TableTestBase { unaryNode( "DataSetCalc", batchTableNode(table), - term("select", "CASE(=(a, 1), 1, 99) AS correct, rowtime") + term("select", "rowtime, CASE(=(a, 1), 1, 99) AS $f1") ), term("window", "TumblingGroupWindow('w$, 'rowtime, 900000.millis)"), - term("select", "SUM(correct) AS s, AVG(correct) AS a, start('w$) AS w$start," + - " end('w$) AS w$end, rowtime('w$) AS w$rowtime") + term("select", "SUM($f1) AS s, AVG($f1) AS a, start('w$) AS w$start," + + " end('w$) AS w$end, rowtime('w$) AS w$rowtime") ), term("select", "CAST(s) AS s", "CAST(a) AS a", "CAST(w$start) AS wStart") ) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/sql/SetOperatorsTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/sql/SetOperatorsTest.scala index 1b3daaa529..056c882885 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/sql/SetOperatorsTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/sql/SetOperatorsTest.scala @@ -64,16 +64,16 @@ class SetOperatorsTest extends TableTestBase { unaryNode( "DataSetCalc", batchTableNode(table1), - term("select", "b_long AS b_long3", "true AS $f0"), + term("select", "b_long", "true AS $f0"), term("where", "IS NOT NULL(b_long)") ), - term("groupBy", "b_long3"), - term("select", "b_long3", "MIN($f0) AS $f1") + term("groupBy", "b_long"), + term("select", "b_long", "MIN($f0) AS $f1") ), - term("select", "b_long3") + term("select", "b_long") ), - term("where", "=(a_long, b_long3)"), - term("join", "a_long", "a_int", "a_string", "b_long3"), + term("where", "=(a_long, b_long)"), + term("join", "a_long", "a_int", "a_string", "b_long"), term("joinType", "InnerJoin") ), term("select", "a_int", "a_string") diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/table/CalcTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/table/CalcTest.scala index c501390ebe..ede7541561 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/table/CalcTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/table/CalcTest.scala @@ -351,7 +351,7 @@ class CalcTest extends TableTestBase { term("groupBy", "word"), term("select", "word", "SUM(frequency) AS EXPR$0") ), - term("select", "word, EXPR$0 AS frequency"), + term("select", "word, EXPR$0"), term("where", "=(EXPR$0, 2)") ) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/GroupWindowTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/GroupWindowTest.scala index c7c4aebba4..733470dce6 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/GroupWindowTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/GroupWindowTest.scala @@ -244,9 +244,9 @@ class GroupWindowTest extends TableTestBase { "rowtime('w$) AS w$rowtime", "proctime('w$) AS w$proctime") ), - term("select", "w$rowtime AS zzzzz") + term("select", "w$rowtime AS $f2") ), - term("window", "TumblingGroupWindow('w$, 'zzzzz, 4.millis)"), + term("window", "TumblingGroupWindow('w$, '$f2, 4.millis)"), term("select", "COUNT(*) AS a", "start('w$) AS w$start", @@ -329,12 +329,12 @@ class GroupWindowTest extends TableTestBase { unaryNode( "DataStreamCalc", streamTableNode(table), - term("select", "CASE(=(a, 1), 1, 99) AS correct", "rowtime") + term("select", "rowtime", "CASE(=(a, 1), 1, 99) AS $f1") ), term("window", "TumblingGroupWindow('w$, 'rowtime, 900000.millis)"), term("select", - "SUM(correct) AS s", - "AVG(correct) AS a", + "SUM($f1) AS s", + "AVG($f1) AS a", "start('w$) AS w$start", "end('w$) AS w$end", "rowtime('w$) AS w$rowtime", diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/JoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/JoinTest.scala index 89ad599233..dcc86ab8b4 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/JoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/JoinTest.scala @@ -290,7 +290,7 @@ class JoinTest extends TableTestBase { ), unaryNode("DataStreamCalc", streamTableNode(t2), - term("select", "a", "c", "proctime", "CAST(12:BIGINT) AS nullField") + term("select", "a", "c", "proctime", "12:BIGINT AS nullField") ), term("where", "AND(=(a, a0), =(nullField, nullField0), >=(PROCTIME(proctime), " + "-(PROCTIME(proctime0), 5000:INTERVAL SECOND)), <=(PROCTIME(proctime), " + diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/TemporalTableJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/TemporalTableJoinTest.scala index 05874bc3d6..cad41e9ed4 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/TemporalTableJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/TemporalTableJoinTest.scala @@ -139,15 +139,11 @@ class TemporalTableJoinTest extends TableTestBase { "DataStreamCalc", binaryNode( "DataStreamTemporalTableJoin", - unaryNode( - "DataStreamCalc", - streamTableNode(orders), - term("select", "o_rowtime, o_amount, o_currency, o_secondary_key") - ), + streamTableNode(orders), unaryNode( "DataStreamCalc", streamTableNode(ratesHistory), - term("select", "rowtime, currency, rate, secondary_key"), + term("select", "rowtime, comment, currency, rate, secondary_key"), term("where", ">(rate, 110:BIGINT)") ), term( @@ -158,10 +154,12 @@ class TemporalTableJoinTest extends TableTestBase { term( "join", "o_rowtime", + "o_comment", "o_amount", "o_currency", "o_secondary_key", "rowtime", + "comment", "currency", "rate", "secondary_key"), @@ -224,15 +222,12 @@ class TemporalTableJoinTest extends TableTestBase { binaryNode( "DataStreamTemporalTableJoin", streamTableNode(proctimeOrders), - unaryNode( - "DataStreamCalc", - streamTableNode(proctimeRatesHistory), - term("select", "currency, rate")), + streamTableNode(proctimeRatesHistory), term("where", "AND(" + s"${TEMPORAL_JOIN_CONDITION.getName}(o_proctime, currency), " + "=(currency, o_currency))"), - term("join", "o_amount", "o_currency", "o_proctime", "currency", "rate"), + term("join", "o_amount", "o_currency", "o_proctime", "currency", "rate", "proctime"), term("joinType", "InnerJoin") ), term("select", "*(o_amount, rate) AS rate") diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/table/TemporalTableJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/table/TemporalTableJoinTest.scala index 5e69382b86..0c0f096c37 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/table/TemporalTableJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/table/TemporalTableJoinTest.scala @@ -111,15 +111,11 @@ class TemporalTableJoinTest extends TableTestBase { "DataStreamCalc", binaryNode( "DataStreamTemporalTableJoin", - unaryNode( - "DataStreamCalc", - streamTableNode(orders), - term("select", "o_rowtime, o_amount, o_currency, o_secondary_key") - ), + streamTableNode(orders), unaryNode( "DataStreamCalc", streamTableNode(ratesHistory), - term("select", "rowtime, currency, rate, secondary_key"), + term("select", "rowtime, comment, currency, rate, secondary_key"), term("where", ">(rate, 110:BIGINT)") ), term( @@ -130,10 +126,12 @@ class TemporalTableJoinTest extends TableTestBase { term( "join", "o_rowtime", + "o_comment", "o_amount", "o_currency", "o_secondary_key", "rowtime", + "comment", "currency", "rate", "secondary_key"), @@ -240,15 +238,12 @@ class TemporalTableJoinTest extends TableTestBase { binaryNode( "DataStreamTemporalTableJoin", streamTableNode(proctimeOrders), - unaryNode( - "DataStreamCalc", - streamTableNode(proctimeRatesHistory), - term("select", "currency, rate")), + streamTableNode(proctimeRatesHistory), term("where", "AND(" + s"${TEMPORAL_JOIN_CONDITION.getName}(o_proctime, currency), " + "=(currency, o_currency))"), - term("join", "o_amount", "o_currency", "o_proctime", "currency", "rate"), + term("join", "o_amount", "o_currency", "o_proctime", "currency", "rate", "proctime"), term("joinType", "InnerJoin") ), term("select", "*(o_amount, rate) AS rate") diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/RexProgramExtractorTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/RexProgramExtractorTest.scala index 2e66dc8ecb..1a83546fd2 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/RexProgramExtractorTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/RexProgramExtractorTest.scala @@ -423,8 +423,8 @@ class RexProgramExtractorTest extends RexProgramTestBase { ) assertExpressionArrayEquals(expected, convertedExpressions) assertEquals(2, unconvertedRexNodes.length) - assertEquals(">(CAST($2):BIGINT NOT NULL, 100)", unconvertedRexNodes(0).toString) - assertEquals("OR(>(CAST($2):BIGINT NOT NULL, 100), <=($2, $1))", + assertEquals("<(100, CAST($2):BIGINT NOT NULL)", unconvertedRexNodes(0).toString) + assertEquals("OR(>=($1, $2), <(100, CAST($2):BIGINT NOT NULL))", unconvertedRexNodes(1).toString) } diff --git a/flink-table/flink-table-planner/src/test/scala/resources/testSqlUpdateAndToDataStream.out b/flink-table/flink-table-planner/src/test/scala/resources/testSqlUpdateAndToDataStream.out index 1ef95b0d2a..40fce87ba4 100644 --- a/flink-table/flink-table-planner/src/test/scala/resources/testSqlUpdateAndToDataStream.out +++ b/flink-table/flink-table-planner/src/test/scala/resources/testSqlUpdateAndToDataStream.out @@ -1,6 +1,6 @@ == Abstract Syntax Tree == LogicalProject(first=[$0]) - EnumerableTableScan(table=[[default_catalog, default_database, MyTable]]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) == Optimized Logical Plan == StreamTableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[first], source=[CsvTableSource(read fields: first)]) -- Gitee From 52315e79b439eda4f6cd7836051c1791382a4a7d Mon Sep 17 00:00:00 2001 From: HuangXingBo Date: Tue, 31 Mar 2020 10:26:03 +0800 Subject: [PATCH 378/885] [FLINK-16796][python] Fix the bug of Python UDTF in SQL query (#11521) --- flink-python/pyflink/table/tests/test_udtf.py | 19 +++++++++++++++++++ .../nodes/common/CommonPythonCorrelate.scala | 7 +++++-- .../plan/nodes/CommonPythonCorrelate.scala | 7 +++++-- 3 files changed, 29 insertions(+), 4 deletions(-) diff --git a/flink-python/pyflink/table/tests/test_udtf.py b/flink-python/pyflink/table/tests/test_udtf.py index 48f01b8ade..026ec5af1c 100644 --- a/flink-python/pyflink/table/tests/test_udtf.py +++ b/flink-python/pyflink/table/tests/test_udtf.py @@ -50,6 +50,25 @@ class UserDefinedTableFunctionTests(object): ["1,0,null", "1,1,null", "2,0,null", "2,1,null", "3,0,0", "3,0,1", "3,0,2", "3,1,1", "3,1,2", "3,2,2", "3,3,null"]) + def test_table_function_with_sql_query(self): + table_sink = source_sink_utils.TestAppendSink( + ['a', 'b', 'c'], + [DataTypes.BIGINT(), DataTypes.BIGINT(), DataTypes.BIGINT()]) + self.t_env.register_table_sink("Results", table_sink) + self.t_env.register_function( + "multi_emit", udtf(MultiEmit(), [DataTypes.BIGINT(), DataTypes.BIGINT()], + [DataTypes.BIGINT(), DataTypes.BIGINT()])) + + t = self.t_env.from_elements([(1, 1, 3), (2, 1, 6), (3, 2, 9)], ['a', 'b', 'c']) + self.t_env.register_table("MyTable", t) + self.t_env.sql_query( + "SELECT a, x, y FROM MyTable LEFT JOIN LATERAL TABLE(multi_emit(a, b)) as T(x, y)" + " ON TRUE") \ + .insert_into("Results") + self.t_env.execute("test") + actual = source_sink_utils.results() + self.assert_equals(actual, ["1,1,0", "2,2,0", "3,3,0", "3,3,1"]) + class PyFlinkStreamUserDefinedTableFunctionTests(UserDefinedTableFunctionTests, PyFlinkStreamTableTestCase): diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonCorrelate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonCorrelate.scala index b911b02e38..58c1ef8e31 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonCorrelate.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonCorrelate.scala @@ -20,7 +20,7 @@ package org.apache.flink.table.planner.plan.nodes.common import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.JoinRelType -import org.apache.calcite.rex.{RexCall, RexInputRef, RexNode} +import org.apache.calcite.rex.{RexCall, RexFieldAccess, RexInputRef, RexNode} import org.apache.flink.api.dag.Transformation import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.operators.OneInputStreamOperator @@ -67,7 +67,10 @@ trait CommonPythonCorrelate extends CommonPythonBase { val pythonTableFunctionInfo = createPythonFunctionInfo(pythonRexCall, inputNodes) val udtfInputOffsets = inputNodes.toArray .map(_._1) - .collect { case inputRef: RexInputRef => inputRef.getIndex } + .collect { + case inputRef: RexInputRef => inputRef.getIndex + case fac: RexFieldAccess => fac.getField.getIndex + } (udtfInputOffsets, pythonTableFunctionInfo) } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonPythonCorrelate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonPythonCorrelate.scala index f7f08021fb..223887d7dc 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonPythonCorrelate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonPythonCorrelate.scala @@ -18,7 +18,7 @@ package org.apache.flink.table.plan.nodes import org.apache.calcite.rel.core.JoinRelType -import org.apache.calcite.rex.{RexCall, RexInputRef, RexNode} +import org.apache.calcite.rex.{RexCall, RexFieldAccess, RexInputRef, RexNode} import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.operators.OneInputStreamOperator import org.apache.flink.table.functions.python.PythonFunctionInfo @@ -60,7 +60,10 @@ trait CommonPythonCorrelate extends CommonPythonBase { val pythonTableFunctionInfo = createPythonFunctionInfo(pythonRexCall, inputNodes) val udtfInputOffsets = inputNodes.toArray .map(_._1) - .collect { case inputRef: RexInputRef => inputRef.getIndex } + .collect { + case inputRef: RexInputRef => inputRef.getIndex + case fac: RexFieldAccess => fac.getField.getIndex + } (udtfInputOffsets, pythonTableFunctionInfo) } } -- Gitee From ee1e83ceb176cd5700b07f7a26e23b082d8f38fd Mon Sep 17 00:00:00 2001 From: Hequn Cheng Date: Tue, 31 Mar 2020 10:30:49 +0800 Subject: [PATCH 379/885] [FLINK-16673][python] Support metric for Python UDTF (#11566) --- flink-python/pyflink/table/tests/test_udtf.py | 12 +++++++++++- .../table/AbstractPythonTableFunctionRunner.java | 1 + 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/flink-python/pyflink/table/tests/test_udtf.py b/flink-python/pyflink/table/tests/test_udtf.py index 026ec5af1c..e4377c03e8 100644 --- a/flink-python/pyflink/table/tests/test_udtf.py +++ b/flink-python/pyflink/table/tests/test_udtf.py @@ -15,6 +15,7 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ +import unittest from pyflink.table import DataTypes from pyflink.table.udf import TableFunction, udtf, ScalarFunction, udf from pyflink.testing import source_sink_utils @@ -80,8 +81,17 @@ class PyFlinkBlinkStreamUserDefinedFunctionTests(UserDefinedTableFunctionTests, pass -class MultiEmit(TableFunction): +class MultiEmit(TableFunction, unittest.TestCase): + + def open(self, function_context): + mg = function_context.get_metric_group() + self.counter = mg.add_group("key", "value").counter("my_counter") + self.counter_sum = 0 + def eval(self, x, y): + self.counter.inc(y) + self.counter_sum += y + self.assertEqual(self.counter_sum, self.counter.get_count()) for i in range(y): yield x, i diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/table/AbstractPythonTableFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/table/AbstractPythonTableFunctionRunner.java index 50136c7d35..9e9f569be9 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/table/AbstractPythonTableFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/table/AbstractPythonTableFunctionRunner.java @@ -81,6 +81,7 @@ public abstract class AbstractPythonTableFunctionRunner extends AbstractPyth public FlinkFnApi.UserDefinedFunctions getUserDefinedFunctionsProto() { FlinkFnApi.UserDefinedFunctions.Builder builder = FlinkFnApi.UserDefinedFunctions.newBuilder(); builder.addUdfs(getUserDefinedFunctionProto(tableFunction)); + builder.setMetricEnabled(flinkMetricContainer != null); return builder.build(); } -- Gitee From e789bb7165d32b2501f87a071f0f17022d82ac95 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Tue, 31 Mar 2020 11:58:26 +0800 Subject: [PATCH 380/885] [FLINK-16857][table-planner-blink] Support partition prune by getPartitions of source This closes #11560 --- .../connectors/hive/HiveTableSource.java | 2 +- ...PushPartitionIntoTableSourceScanRule.scala | 72 +-- ...shPartitionIntoTableSourceScanRuleTest.xml | 442 +++++++++++++++--- ...PartitionIntoTableSourceScanRuleTest.scala | 17 +- .../planner/utils/testTableSources.scala | 42 +- 5 files changed, 447 insertions(+), 128 deletions(-) diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java index 3addbe33ba..7f00ad9f22 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java @@ -234,7 +234,7 @@ public class HiveTableSource implements @Override public List> getPartitions() { - throw new RuntimeException("This method is not expected to be called. " + + throw new UnsupportedOperationException( "Please use Catalog API to retrieve all partitions of a table"); } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoTableSourceScanRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoTableSourceScanRule.scala index fbc9616137..63117d7ada 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoTableSourceScanRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoTableSourceScanRule.scala @@ -107,12 +107,20 @@ class PushPartitionIntoTableSourceScanRule extends RelOptRule( inputFieldType.getFieldList.get(index).getType }.map(FlinkTypeFactory.toLogicalType) + val partitionsFromSource = try { + Some(tableSource.getPartitions) + } catch { + case _: UnsupportedOperationException => None + } + def getAllPartitions: util.List[util.Map[String, String]] = { - catalogOption match { - case Some(c) => - c.listPartitions(tableIdentifier.toObjectPath) - .map(_.getPartitionSpec).toList - case None => tableSource.getPartitions + partitionsFromSource match { + case Some(parts) => parts + case None => catalogOption match { + case Some(catalog) => + catalog.listPartitions(tableIdentifier.toObjectPath).map(_.getPartitionSpec).toList + case None => throw new TableException(s"The $tableSource must be a catalog.") + } } } @@ -132,35 +140,39 @@ class PushPartitionIntoTableSourceScanRule extends RelOptRule( ) } - val remainingPartitions: util.List[util.Map[String, String]] = catalogOption match { - case Some(catalog) => - val converter = new RexNodeToExpressionConverter( - inputFields, - context.getFunctionCatalog, - context.getCatalogManager, - TimeZone.getTimeZone(config.getLocalTimeZone)) - def toExpressions: Option[Seq[Expression]] = { - val expressions = new mutable.ArrayBuffer[Expression]() - for (predicate <- partitionPredicates) { - predicate.accept(converter) match { - case Some(expr) => expressions.add(expr) - case None => return None + val remainingPartitions: util.List[util.Map[String, String]] = partitionsFromSource match { + case Some(_) => internalPartitionPrune() + case None => + catalogOption match { + case Some(catalog) => + val converter = new RexNodeToExpressionConverter( + inputFields, + context.getFunctionCatalog, + context.getCatalogManager, + TimeZone.getTimeZone(config.getLocalTimeZone)) + def toExpressions: Option[Seq[Expression]] = { + val expressions = new mutable.ArrayBuffer[Expression]() + for (predicate <- partitionPredicates) { + predicate.accept(converter) match { + case Some(expr) => expressions.add(expr) + case None => return None + } + } + Some(expressions) } - } - Some(expressions) - } - toExpressions match { - case Some(expressions) => - try { - catalog - .listPartitionsByFilter(tableIdentifier.toObjectPath, expressions) - .map(_.getPartitionSpec) - } catch { - case _: UnsupportedOperationException => internalPartitionPrune() + toExpressions match { + case Some(expressions) => + try { + catalog + .listPartitionsByFilter(tableIdentifier.toObjectPath, expressions) + .map(_.getPartitionSpec) + } catch { + case _: UnsupportedOperationException => internalPartitionPrune() + } + case None => internalPartitionPrune() } case None => internalPartitionPrune() } - case None => internalPartitionPrune() } val newTableSource = tableSource.applyPartitionPruning(remainingPartitions) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoTableSourceScanRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoTableSourceScanRuleTest.xml index 4924daca28..32dc8a3190 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoTableSourceScanRuleTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoTableSourceScanRuleTest.xml @@ -16,10 +16,9 @@ See the License for the specific language governing permissions and limitations under the License. --> - + 2]]> - ($0, 2)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> - ($0, 2)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> - - + + + 2]]> + + + ($0, 2)]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +]]> + + + ($0, 2)]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +]]> + + + 2]]> - - ($0, 2)]) +- LogicalTableScan(table=[[default_catalog, default_database, VirtualTable]]) ]]> - - + + + 2]]> + + + ($0, 2)]) + +- LogicalProject(id=[$0], name=[$1], part1=[$2], part2=[$3], virtualField=[+($3, 1)]) + +- LogicalTableScan(table=[[default_catalog, default_database, VirtualTable]]) +]]> + + + ($0, 2)]) + +- LogicalTableScan(table=[[default_catalog, default_database, VirtualTable]]) +]]> + + + - - - - + + + + + + + + + + + + + + + + + + + + + + + 2 OR part1 = 'A') AND part2 > 1]]> - ($0, 2), =($2, _UTF-16LE'A')), >($3, 1))]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> - ($0, 2), =($2, _UTF-16LE'A'))]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [partitions={part1=A, part2=2}, {part1=B, part2=3}]]]) ]]> - - + - - - - + 1]]> - ($3, 1)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> - - - + + + 1]]> + + + ($3, 1)]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +]]> + + + + + + 1]]> - - - - + + + 1]]> + + + ($3, 1)]) + +- LogicalProject(id=[$0], name=[$1], part1=[$2], part2=[$3], virtualField=[+($3, 1)]) + +- LogicalTableScan(table=[[default_catalog, default_database, VirtualTable]]) +]]> + + + + + + 1]]> - ($3, 1))]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> - - - + + + 1]]> + + + ($3, 1))]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +]]> + + + + + + + + 1]]> + + + ($3, 1))]) + +- LogicalProject(id=[$0], name=[$1], part1=[$2], part2=[$3], virtualField=[+($3, 1)]) + +- LogicalTableScan(table=[[default_catalog, default_database, VirtualTable]]) +]]> + + + + + + 1]]> - - - - + + + 1]]> + + + ($3, 1))]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +]]> + + + + + + 1]]> - ($3, 1))]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> - - - + 1]]> - - - - + + + 1]]> + + + ($3, 1))]) + +- LogicalProject(id=[$0], name=[$1], part1=[$2], part2=[$3], virtualField=[+($3, 1)]) + +- LogicalTableScan(table=[[default_catalog, default_database, VirtualTable]]) +]]> + + + + + + + + 2 OR part1 = 'A') AND part2 > 1]]> + + + ($0, 2), =($2, _UTF-16LE'A')), >($3, 1))]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +]]> + + + ($0, 2), =($2, _UTF-16LE'A'))]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [partitions={part1=A, part2=2}, {part1=B, part2=3}]]]) +]]> + + + 2 OR part1 = 'A') AND part2 > 1]]> - - ($0, 2), =($2, _UTF-16LE'A'))]) +- LogicalTableScan(table=[[default_catalog, default_database, VirtualTable, source: [partitions={part1=A, part2=2}, {part1=B, part2=3}]]]) ]]> - - + + + 2 OR part1 = 'A') AND part2 > 1]]> + + + ($0, 2), =($2, _UTF-16LE'A')), >($3, 1))]) + +- LogicalProject(id=[$0], name=[$1], part1=[$2], part2=[$3], virtualField=[+($3, 1)]) + +- LogicalTableScan(table=[[default_catalog, default_database, VirtualTable]]) +]]> + + + ($0, 2), =($2, _UTF-16LE'A'))]) + +- LogicalTableScan(table=[[default_catalog, default_database, VirtualTable, source: [partitions={part1=A, part2=2}, {part1=B, part2=3}]]]) +]]> + + + + + 2 AND part1 = 'A']]> + + + ($0, 2), =($2, _UTF-16LE'A'))]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +]]> + + + ($0, 2)]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [partitions={part1=A, part2=1}, {part1=A, part2=2}]]]) +]]> + + + 2 AND part1 = 'A']]> - ($0, 2), =($2, _UTF-16LE'A'))]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> - ($0, 2)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [partitions={part1=A, part2=1}, {part1=A, part2=2}]]]) ]]> - - + 2 AND part1 = 'A']]> - - ($0, 2)]) +- LogicalTableScan(table=[[default_catalog, default_database, VirtualTable, source: [partitions={part1=A, part2=1}, {part1=A, part2=2}]]]) ]]> - - + + + 2 AND part1 = 'A']]> + + + ($0, 2), =($2, _UTF-16LE'A'))]) + +- LogicalProject(id=[$0], name=[$1], part1=[$2], part2=[$3], virtualField=[+($3, 1)]) + +- LogicalTableScan(table=[[default_catalog, default_database, VirtualTable]]) +]]> + + + ($0, 2)]) + +- LogicalTableScan(table=[[default_catalog, default_database, VirtualTable, source: [partitions={part1=A, part2=1}, {part1=A, part2=2}]]]) +]]> + + + 2 OR part1 = 'A']]> - ($0, 2), =($2, _UTF-16LE'A'))]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> - ($0, 2), =($2, _UTF-16LE'A'))]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> - - + + + 2 OR part1 = 'A']]> + + + ($0, 2), =($2, _UTF-16LE'A'))]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +]]> + + + ($0, 2), =($2, _UTF-16LE'A'))]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +]]> + + + + + 2 OR part1 = 'A']]> + + + ($0, 2), =($2, _UTF-16LE'A'))]) + +- LogicalProject(id=[$0], name=[$1], part1=[$2], part2=[$3], virtualField=[+($3, 1)]) + +- LogicalTableScan(table=[[default_catalog, default_database, VirtualTable]]) +]]> + + + ($0, 2), =($2, _UTF-16LE'A'))]) + +- LogicalTableScan(table=[[default_catalog, default_database, VirtualTable]]) +]]> + + + 2 AND MyUdf(part2) < 3]]> - - ($0, 2)]) +- LogicalTableScan(table=[[default_catalog, default_database, VirtualTable, source: [partitions={part1=A, part2=1}, {part1=C, part2=1}]]]) ]]> - - + 2 OR part1 = 'A']]> - - ($0, 2), =($2, _UTF-16LE'A'))]) +- LogicalTableScan(table=[[default_catalog, default_database, VirtualTable]]) ]]> - - + 2 AND MyUdf(part2) < 3]]> - ($0, 2), <(MyUdf($3), 3))]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) ]]> - ($0, 2)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [partitions={part1=A, part2=1}, {part1=C, part2=1}]]]) ]]> - + + + + + 2 AND MyUdf(part2) < 3]]> + + + ($0, 2), <(MyUdf($3), 3))]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +]]> + + + ($0, 2)]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [partitions={part1=A, part2=1}, {part1=C, part2=1}]]]) +]]> + + + + + 2 AND MyUdf(part2) < 3]]> + + + ($0, 2), <(MyUdf($3), 3))]) + +- LogicalProject(id=[$0], name=[$1], part1=[$2], part2=[$3], virtualField=[+($3, 1)]) + +- LogicalTableScan(table=[[default_catalog, default_database, VirtualTable]]) +]]> + + + ($0, 2)]) + +- LogicalTableScan(table=[[default_catalog, default_database, VirtualTable, source: [partitions={part1=A, part2=1}, {part1=C, part2=1}]]]) +]]> diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoTableSourceScanRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoTableSourceScanRuleTest.scala index edf4c31d37..5ffba2afe2 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoTableSourceScanRuleTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoTableSourceScanRuleTest.scala @@ -25,12 +25,16 @@ import org.apache.flink.table.planner.utils.{TableConfigUtils, TableTestBase, Te import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.rel.rules.FilterProjectTransposeRule import org.apache.calcite.tools.RuleSets +import org.junit.runner.RunWith +import org.junit.runners.Parameterized import org.junit.{Before, Test} /** * Test for [[PushPartitionIntoTableSourceScanRule]]. */ -class PushPartitionIntoTableSourceScanRuleTest extends TableTestBase { +@RunWith(classOf[Parameterized]) +class PushPartitionIntoTableSourceScanRuleTest( + sourceFetchPartitions: Boolean) extends TableTestBase { private val util = batchTestUtil() @Before @@ -63,9 +67,9 @@ class PushPartitionIntoTableSourceScanRuleTest extends TableTestBase { .build() TestPartitionableSourceFactory.registerTableSource(util.tableEnv, "MyTable", - tableSchema = tableSchema, isBounded = true) + tableSchema = tableSchema, isBounded = true, sourceFetchPartitions = sourceFetchPartitions) TestPartitionableSourceFactory.registerTableSource(util.tableEnv, "VirtualTable", - tableSchema = tableSchema2, isBounded = true) + tableSchema = tableSchema2, isBounded = true, sourceFetchPartitions = sourceFetchPartitions) } @Test @@ -161,3 +165,10 @@ class PushPartitionIntoTableSourceScanRuleTest extends TableTestBase { } } + +object PushPartitionIntoTableSourceScanRuleTest { + @Parameterized.Parameters(name = "{0}") + def parameters(): java.util.Collection[Boolean] = { + java.util.Arrays.asList(true, false) + } +} diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/testTableSources.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/testTableSources.scala index d7f855c012..20fc2b8a67 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/testTableSources.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/testTableSources.scala @@ -605,7 +605,7 @@ class TestFilterableTableSourceFactory extends StreamTableSourceFactory[Row] { class TestPartitionableTableSource( override val isBounded: Boolean, remainingPartitions: JList[JMap[String, String]], - isCatalogTable: Boolean) + sourceFetchPartitions: Boolean) extends StreamTableSource[Row] with PartitionableTableSource { @@ -626,8 +626,8 @@ class TestPartitionableTableSource( ) override def getPartitions: JList[JMap[String, String]] = { - if (isCatalogTable) { - throw new RuntimeException("Should not expected.") + if (!sourceFetchPartitions) { + throw new UnsupportedOperationException() } List( Map("part1" -> "A", "part2" -> "1").asJava, @@ -639,7 +639,7 @@ class TestPartitionableTableSource( override def applyPartitionPruning( remainingPartitions: JList[JMap[String, String]]): TableSource[_] = { - new TestPartitionableTableSource(isBounded, remainingPartitions, isCatalogTable) + new TestPartitionableTableSource(isBounded, remainingPartitions, sourceFetchPartitions) } override def getDataStream(execEnv: StreamExecutionEnvironment): DataStream[Row] = { @@ -787,7 +787,7 @@ class TestPartitionableSourceFactory extends TableSourceFactory[Row] { dp.putProperties(properties) val isBounded = dp.getBoolean("is-bounded") - val isCatalogTable = dp.getBoolean("is-catalog-table") + val sourceFetchPartitions = dp.getBoolean("source-fetch-partitions") val remainingPartitions = dp.getOptionalArray("remaining-partition", new function.Function[String, util.Map[String, String]] { override def apply(t: String): util.Map[String, String] = { @@ -800,7 +800,7 @@ class TestPartitionableSourceFactory extends TableSourceFactory[Row] { new TestPartitionableTableSource( isBounded, remainingPartitions, - isCatalogTable) + sourceFetchPartitions) } } @@ -827,11 +827,11 @@ object TestPartitionableSourceFactory { tableName: String, isBounded: Boolean, tableSchema: TableSchema = tableSchema, - remainingPartitions: JList[JMap[String, String]] = null): Unit = { + remainingPartitions: JList[JMap[String, String]] = null, + sourceFetchPartitions: Boolean = false): Unit = { val properties = new DescriptorProperties() properties.putString("is-bounded", isBounded.toString) - val isCatalogTable = true - properties.putBoolean("is-catalog-table", isCatalogTable) + properties.putBoolean("source-fetch-partitions", sourceFetchPartitions) properties.putString(CONNECTOR_TYPE, "TestPartitionableSource") if (remainingPartitions != null) { remainingPartitions.zipWithIndex.foreach { case (part, i) => @@ -854,19 +854,17 @@ object TestPartitionableSourceFactory { val path = new ObjectPath(tEnv.getCurrentDatabase, tableName) catalog.createTable(path, table, false) - if (isCatalogTable) { - val partitions = List( - Map("part1" -> "A", "part2" -> "1").asJava, - Map("part1" -> "A", "part2" -> "2").asJava, - Map("part1" -> "B", "part2" -> "3").asJava, - Map("part1" -> "C", "part2" -> "1").asJava - ) - partitions.foreach(spec => catalog.createPartition( - path, - new CatalogPartitionSpec(new java.util.LinkedHashMap(spec)), - new CatalogPartitionImpl(Map[String, String](), ""), - true)) - } + val partitions = List( + Map("part1" -> "A", "part2" -> "1").asJava, + Map("part1" -> "A", "part2" -> "2").asJava, + Map("part1" -> "B", "part2" -> "3").asJava, + Map("part1" -> "C", "part2" -> "1").asJava + ) + partitions.foreach(spec => catalog.createPartition( + path, + new CatalogPartitionSpec(new java.util.LinkedHashMap(spec)), + new CatalogPartitionImpl(Map[String, String](), ""), + true)) } } -- Gitee From 1444fd68115594b872201242886b4d789f4b26a5 Mon Sep 17 00:00:00 2001 From: Rui Li Date: Tue, 31 Mar 2020 17:04:03 +0800 Subject: [PATCH 381/885] [FLINK-16767][hive] Failed to read Hive table with RegexSerDe This closes #11504 --- .../connectors/hive/HiveTablePartition.java | 15 ++++++-- .../connectors/hive/HiveTableSource.java | 9 +++-- .../hive/read/HiveMapredSplitReader.java | 6 +--- .../catalog/hive/util/HiveTableUtil.java | 35 ------------------- .../hive/TableEnvHiveConnectorTest.java | 18 ++++++++++ 5 files changed, 38 insertions(+), 45 deletions(-) diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTablePartition.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTablePartition.java index f85ae014ba..4e1a90466c 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTablePartition.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTablePartition.java @@ -23,6 +23,7 @@ import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import java.io.Serializable; import java.util.LinkedHashMap; import java.util.Map; +import java.util.Properties; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -40,13 +41,17 @@ public class HiveTablePartition implements Serializable { /** The map of partition key names and their values. */ private final Map partitionSpec; - public HiveTablePartition(StorageDescriptor storageDescriptor) { - this(storageDescriptor, new LinkedHashMap<>()); + // Table properties that should be used to initialize SerDe + private final Properties tableProps; + + public HiveTablePartition(StorageDescriptor storageDescriptor, Properties tableProps) { + this(storageDescriptor, new LinkedHashMap<>(), tableProps); } - public HiveTablePartition(StorageDescriptor storageDescriptor, Map partitionSpec) { + public HiveTablePartition(StorageDescriptor storageDescriptor, Map partitionSpec, Properties tableProps) { this.storageDescriptor = checkNotNull(storageDescriptor, "storageDescriptor can not be null"); this.partitionSpec = checkNotNull(partitionSpec, "partitionSpec can not be null"); + this.tableProps = checkNotNull(tableProps, "tableProps can not be null"); } public StorageDescriptor getStorageDescriptor() { @@ -56,4 +61,8 @@ public class HiveTablePartition implements Serializable { public Map getPartitionSpec() { return partitionSpec; } + + public Properties getTableProps() { + return tableProps; + } } diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java index 7f00ad9f22..33c33dbba3 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java @@ -35,6 +35,7 @@ import org.apache.flink.table.catalog.hive.client.HiveMetastoreClientWrapper; import org.apache.flink.table.catalog.hive.client.HiveShim; import org.apache.flink.table.catalog.hive.client.HiveShimLoader; import org.apache.flink.table.catalog.hive.descriptors.HiveCatalogValidator; +import org.apache.flink.table.catalog.hive.util.HiveReflectionUtils; import org.apache.flink.table.dataformat.BaseRow; import org.apache.flink.table.functions.hive.conversion.HiveInspectors; import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter; @@ -51,6 +52,7 @@ import org.apache.flink.util.Preconditions; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.mapred.JobConf; import org.apache.thrift.TException; import org.slf4j.Logger; @@ -66,6 +68,7 @@ import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.stream.Collectors; /** @@ -281,6 +284,8 @@ public class HiveTableSource implements String dbName = tablePath.getDatabaseName(); String tableName = tablePath.getObjectName(); List partitionColNames = catalogTable.getPartitionKeys(); + Table hiveTable = client.getTable(dbName, tableName); + Properties tableProps = HiveReflectionUtils.getTableMetadata(hiveShim, hiveTable); if (partitionColNames != null && partitionColNames.size() > 0) { final String defaultPartitionName = jobConf.get(HiveConf.ConfVars.DEFAULTPARTITIONNAME.varname, HiveConf.ConfVars.DEFAULTPARTITIONNAME.defaultStrVal); @@ -309,11 +314,11 @@ public class HiveTableSource implements } partitionColValues.put(partitionColName, partitionObject); } - HiveTablePartition hiveTablePartition = new HiveTablePartition(sd, partitionColValues); + HiveTablePartition hiveTablePartition = new HiveTablePartition(sd, partitionColValues, tableProps); allHivePartitions.add(hiveTablePartition); } } else { - allHivePartitions.add(new HiveTablePartition(client.getTable(dbName, tableName).getSd())); + allHivePartitions.add(new HiveTablePartition(hiveTable.getSd(), tableProps)); } } catch (TException e) { throw new FlinkHiveException("Failed to collect all partitions from hive metaStore", e); diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveMapredSplitReader.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveMapredSplitReader.java index 877b52aec5..8164665d91 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveMapredSplitReader.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveMapredSplitReader.java @@ -22,7 +22,6 @@ import org.apache.flink.api.java.hadoop.mapred.wrapper.HadoopDummyReporter; import org.apache.flink.connectors.hive.FlinkHiveException; import org.apache.flink.connectors.hive.HiveTablePartition; import org.apache.flink.table.catalog.hive.client.HiveShim; -import org.apache.flink.table.catalog.hive.util.HiveTableUtil; import org.apache.flink.table.dataformat.BaseRow; import org.apache.flink.table.dataformat.DataFormatConverters; import org.apache.flink.table.dataformat.GenericRow; @@ -48,7 +47,6 @@ import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.Arrays; import java.util.List; -import java.util.Properties; import static org.apache.hadoop.mapreduce.lib.input.FileInputFormat.INPUT_DIR; @@ -122,9 +120,7 @@ public class HiveMapredSplitReader implements SplitReader { try { deserializer = (Deserializer) Class.forName(sd.getSerdeInfo().getSerializationLib()).newInstance(); Configuration conf = new Configuration(); - //properties are used to initialize hive Deserializer properly. - Properties properties = HiveTableUtil.createPropertiesFromStorageDescriptor(sd); - SerDeUtils.initializeSerDe(deserializer, conf, properties, null); + SerDeUtils.initializeSerDe(deserializer, conf, hiveTablePartition.getTableProps(), null); structObjectInspector = (StructObjectInspector) deserializer.getObjectInspector(); structFields = structObjectInspector.getAllStructFieldRefs(); } catch (Exception e) { diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/util/HiveTableUtil.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/util/HiveTableUtil.java index d69dd4c830..a6967080a8 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/util/HiveTableUtil.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/util/HiveTableUtil.java @@ -33,13 +33,9 @@ import org.apache.flink.table.functions.hive.conversion.HiveInspectors; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.LogicalTypeRoot; -import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.hadoop.hive.metastore.api.SerDeInfo; import org.apache.hadoop.hive.metastore.api.StorageDescriptor; -import org.apache.hadoop.hive.serde.serdeConstants; -import org.apache.hadoop.hive.serde2.SerDeUtils; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; import java.util.ArrayList; @@ -47,11 +43,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.Properties; import java.util.Set; -import static org.apache.flink.table.catalog.hive.HiveCatalogConfig.DEFAULT_LIST_COLUMN_TYPES_SEPARATOR; - /** * Utils to for Hive-backed table. */ @@ -113,34 +106,6 @@ public class HiveTableUtil { // Helper methods // -------------------------------------------------------------------------------------------- - /** - * Create properties info to initialize a SerDe. - * @param storageDescriptor - * @return - */ - public static Properties createPropertiesFromStorageDescriptor(StorageDescriptor storageDescriptor) { - SerDeInfo serDeInfo = storageDescriptor.getSerdeInfo(); - Map parameters = serDeInfo.getParameters(); - Properties properties = new Properties(); - properties.setProperty( - serdeConstants.SERIALIZATION_FORMAT, - parameters.get(serdeConstants.SERIALIZATION_FORMAT)); - List colTypes = new ArrayList<>(); - List colNames = new ArrayList<>(); - List cols = storageDescriptor.getCols(); - for (FieldSchema col: cols){ - colTypes.add(col.getType()); - colNames.add(col.getName()); - } - properties.setProperty(serdeConstants.LIST_COLUMNS, StringUtils.join(colNames, String.valueOf(SerDeUtils.COMMA))); - // Note: serdeConstants.COLUMN_NAME_DELIMITER is not defined in previous Hive. We use a literal to save on shim - properties.setProperty("column.name.delimite", String.valueOf(SerDeUtils.COMMA)); - properties.setProperty(serdeConstants.LIST_COLUMN_TYPES, StringUtils.join(colTypes, DEFAULT_LIST_COLUMN_TYPES_SEPARATOR)); - properties.setProperty(serdeConstants.SERIALIZATION_NULL_FORMAT, "NULL"); - properties.putAll(parameters); - return properties; - } - /** * Creates a Hive partition instance. */ diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java index 47483577cf..0242b383dd 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java @@ -565,6 +565,24 @@ public class TableEnvHiveConnectorTest { } } + @Test + public void testRegexSerDe() throws Exception { + hiveShell.execute("create database db1"); + try { + hiveShell.execute("create table db1.src (x int,y string) " + + "row format serde 'org.apache.hadoop.hive.serde2.RegexSerDe' " + + "with serdeproperties ('input.regex'='([\\\\d]+)\\u0001([\\\\S]+)')"); + HiveTestUtils.createTextTableInserter(hiveShell, "db1", "src") + .addRow(new Object[]{1, "a"}) + .addRow(new Object[]{2, "ab"}) + .commit(); + TableEnvironment tableEnv = getTableEnvWithHiveCatalog(); + assertEquals("[1,a, 2,ab]", TableUtils.collectToList(tableEnv.sqlQuery("select * from db1.src order by x")).toString()); + } finally { + hiveShell.execute("drop database db1 cascade"); + } + } + private TableEnvironment getTableEnvWithHiveCatalog() { TableEnvironment tableEnv = HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode(); tableEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog); -- Gitee From 979d005f14d04456bdb20bd7e605f389c0ee9e37 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Mon, 2 Mar 2020 12:18:43 +0100 Subject: [PATCH 382/885] [hotfix][runtime][tests] Rename/move ClassLoaderTest The test is actually testing FlinkUserCodeClassLoaders. --- .../FlinkUserCodeClassLoadersTest.java} | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) rename flink-runtime/src/test/java/org/apache/flink/runtime/{classloading/ClassLoaderTest.java => execution/librarycache/FlinkUserCodeClassLoadersTest.java} (94%) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/classloading/ClassLoaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoadersTest.java similarity index 94% rename from flink-runtime/src/test/java/org/apache/flink/runtime/classloading/ClassLoaderTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoadersTest.java index 7c664ce0b3..4333aa8b69 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/classloading/ClassLoaderTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoadersTest.java @@ -16,9 +16,8 @@ * limitations under the License. */ -package org.apache.flink.runtime.classloading; +package org.apache.flink.runtime.execution.librarycache; -import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders; import org.apache.flink.runtime.rpc.messages.RemoteRpcInvocation; import org.apache.flink.testutils.ClassLoaderUtils; import org.apache.flink.util.SerializedValue; @@ -44,7 +43,7 @@ import static org.junit.Assert.assertNotEquals; /** * Tests for classloading and class loader utilities. */ -public class ClassLoaderTest extends TestLogger { +public class FlinkUserCodeClassLoadersTest extends TestLogger { @ClassRule public static TemporaryFolder temporaryFolder = new TemporaryFolder(); @@ -104,7 +103,7 @@ public class ClassLoaderTest extends TestLogger { final URLClassLoader childClassLoader2 = FlinkUserCodeClassLoaders.parentFirst( new URL[] { childCodePath }, parentClassLoader); - final String className = ClassLoaderTest.class.getName(); + final String className = FlinkUserCodeClassLoadersTest.class.getName(); final Class clazz1 = Class.forName(className, false, parentClassLoader); final Class clazz2 = Class.forName(className, false, childClassLoader1); @@ -130,7 +129,7 @@ public class ClassLoaderTest extends TestLogger { final URLClassLoader childClassLoader2 = FlinkUserCodeClassLoaders.childFirst( new URL[] { childCodePath }, parentClassLoader, new String[0]); - final String className = ClassLoaderTest.class.getName(); + final String className = FlinkUserCodeClassLoadersTest.class.getName(); final Class clazz1 = Class.forName(className, false, parentClassLoader); final Class clazz2 = Class.forName(className, false, childClassLoader1); @@ -154,7 +153,7 @@ public class ClassLoaderTest extends TestLogger { final URLClassLoader childClassLoader = FlinkUserCodeClassLoaders.childFirst( new URL[] { childCodePath }, parentClassLoader, new String[0]); - final String className = ClassLoaderTest.class.getName(); + final String className = FlinkUserCodeClassLoadersTest.class.getName(); final Class clazz1 = Class.forName(className, false, parentClassLoader); final Class clazz2 = Class.forName(className, false, childClassLoader); @@ -171,7 +170,7 @@ public class ClassLoaderTest extends TestLogger { @Test public void testRepeatedParentFirstPatternClass() throws Exception { - final String className = ClassLoaderTest.class.getName(); + final String className = FlinkUserCodeClassLoadersTest.class.getName(); final String parentFirstPattern = className.substring(0, className.lastIndexOf('.')); final ClassLoader parentClassLoader = getClass().getClassLoader(); -- Gitee From b0dff77a85237b76c20c9db4138651f7fd7b8fce Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Mon, 2 Mar 2020 14:06:04 +0100 Subject: [PATCH 383/885] [FLINK-16245][runtime] Decouple user from context classloader Allows user classloader can be unloaded even if a reference on the context classloader outlives the user code. --- .../flink/util/ChildFirstClassLoader.java | 5 +- .../flink/util/FlinkUserCodeClassLoader.java | 38 ++++++++++ .../FlinkUserCodeClassLoaders.java | 73 ++++++++++++++++++- .../FlinkUserCodeClassLoadersTest.java | 29 ++++++++ 4 files changed, 139 insertions(+), 6 deletions(-) create mode 100644 flink-core/src/main/java/org/apache/flink/util/FlinkUserCodeClassLoader.java diff --git a/flink-core/src/main/java/org/apache/flink/util/ChildFirstClassLoader.java b/flink-core/src/main/java/org/apache/flink/util/ChildFirstClassLoader.java index 3942b1bd6a..0cac006db4 100644 --- a/flink-core/src/main/java/org/apache/flink/util/ChildFirstClassLoader.java +++ b/flink-core/src/main/java/org/apache/flink/util/ChildFirstClassLoader.java @@ -20,7 +20,6 @@ package org.apache.flink.util; import java.io.IOException; import java.net.URL; -import java.net.URLClassLoader; import java.util.ArrayList; import java.util.Enumeration; import java.util.Iterator; @@ -32,7 +31,7 @@ import java.util.List; *

    {@link #getResourceAsStream(String)} uses {@link #getResource(String)} internally so we * don't override that. */ -public final class ChildFirstClassLoader extends URLClassLoader { +public final class ChildFirstClassLoader extends FlinkUserCodeClassLoader { /** * The classes that should always go through the parent ClassLoader. This is relevant @@ -47,7 +46,7 @@ public final class ChildFirstClassLoader extends URLClassLoader { } @Override - protected synchronized Class loadClass( + public synchronized Class loadClass( String name, boolean resolve) throws ClassNotFoundException { // First, check if the class has already been loaded diff --git a/flink-core/src/main/java/org/apache/flink/util/FlinkUserCodeClassLoader.java b/flink-core/src/main/java/org/apache/flink/util/FlinkUserCodeClassLoader.java new file mode 100644 index 0000000000..7e49bf6846 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/util/FlinkUserCodeClassLoader.java @@ -0,0 +1,38 @@ +/* + * 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.util; + +import org.apache.flink.annotation.Internal; + +import java.net.URL; +import java.net.URLClassLoader; + +/** + * Base class for user code class loaders (child-first, parent-first). + */ +@Internal +public class FlinkUserCodeClassLoader extends URLClassLoader { + public FlinkUserCodeClassLoader(URL[] urls, ClassLoader parent) { + super(urls, parent); + } + + @Override + public Class loadClass(String name, boolean resolve) throws ClassNotFoundException { + return super.loadClass(name, resolve); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoaders.java index 0df7c7aaf2..aa9b123bc2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoaders.java @@ -19,9 +19,16 @@ package org.apache.flink.runtime.execution.librarycache; import org.apache.flink.util.ChildFirstClassLoader; +import org.apache.flink.util.FlinkUserCodeClassLoader; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.io.IOException; import java.net.URL; import java.net.URLClassLoader; +import java.util.Enumeration; /** * Gives the URLClassLoader a nicer name for debugging purposes. @@ -29,14 +36,14 @@ import java.net.URLClassLoader; public class FlinkUserCodeClassLoaders { public static URLClassLoader parentFirst(URL[] urls, ClassLoader parent) { - return new ParentFirstClassLoader(urls, parent); + return new SafetyNetWrapperClassLoader(new ParentFirstClassLoader(urls, parent)); } public static URLClassLoader childFirst( URL[] urls, ClassLoader parent, String[] alwaysParentFirstPatterns) { - return new ChildFirstClassLoader(urls, parent, alwaysParentFirstPatterns); + return new SafetyNetWrapperClassLoader(new ChildFirstClassLoader(urls, parent, alwaysParentFirstPatterns)); } public static URLClassLoader create( @@ -72,7 +79,7 @@ public class FlinkUserCodeClassLoaders { /** * Regular URLClassLoader that first loads from the parent and only after that from the URLs. */ - static class ParentFirstClassLoader extends URLClassLoader { + static class ParentFirstClassLoader extends FlinkUserCodeClassLoader { ParentFirstClassLoader(URL[] urls) { this(urls, FlinkUserCodeClassLoaders.class.getClassLoader()); @@ -82,4 +89,64 @@ public class FlinkUserCodeClassLoaders { super(urls, parent); } } + + /** + * Ensures that holding a reference on the context class loader outliving the scope of user code does not prevent + * the user classloader to be garbage collected (FLINK-16245). + * + *

    This classloader delegates to the actual user classloader. Upon {@link #close()}, the delegate is nulled + * and can be garbage collected. Additional class resolution will be resolved solely through the bootstrap + * classloader and most likely result in ClassNotFound exceptions. + */ + private static class SafetyNetWrapperClassLoader extends URLClassLoader implements Closeable { + private static final Logger LOG = LoggerFactory.getLogger(SafetyNetWrapperClassLoader.class); + + private FlinkUserCodeClassLoader inner; + + SafetyNetWrapperClassLoader(FlinkUserCodeClassLoader inner) { + super(new URL[0], null); + this.inner = inner; + } + + @Override + public void close() { + if (inner != null) { + try { + inner.close(); + } catch (IOException e) { + LOG.warn("Could not close user classloader", e); + } + } + inner = null; + } + + @Override + protected Class loadClass(String name, boolean resolve) throws ClassNotFoundException { + if (inner == null) { + try { + return super.loadClass(name, resolve); + } catch (ClassNotFoundException e) { + throw new ClassNotFoundException("Flink user code classloader was already closed.", e); + } + } + + return inner.loadClass(name, resolve); + } + + @Override + public URL findResource(String name) { + if (inner == null) { + return super.findResource(name); + } + return inner.getResource(name); + } + + @Override + public Enumeration findResources(String name) throws IOException { + if (inner == null) { + return super.findResources(name); + } + return inner.getResources(name); + } + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoadersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoadersTest.java index 4333aa8b69..240ba15082 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoadersTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoadersTest.java @@ -39,6 +39,7 @@ import static org.hamcrest.Matchers.hasItemInArray; import static org.hamcrest.Matchers.hasProperty; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNotSame; /** * Tests for classloading and class loader utilities. @@ -192,4 +193,32 @@ public class FlinkUserCodeClassLoadersTest extends TestLogger { childClassLoader.close(); } + + @Test + public void testClosingOfClassloader() throws Exception { + final String className = ClassToLoad.class.getName(); + + final ClassLoader parentClassLoader = ClassLoader.getSystemClassLoader().getParent(); + + final URL childCodePath = getClass().getProtectionDomain().getCodeSource().getLocation(); + + final URLClassLoader childClassLoader = FlinkUserCodeClassLoaders.create( + FlinkUserCodeClassLoaders.ResolveOrder.CHILD_FIRST, + new URL[] { childCodePath }, + parentClassLoader, + new String[0]); + + final Class loadedClass = childClassLoader.loadClass(className); + + assertNotSame(ClassToLoad.class, loadedClass); + + childClassLoader.close(); + + // after closing, no loaded class should be reachable anymore + expectedException.expect(isA(ClassNotFoundException.class)); + childClassLoader.loadClass(className); + } + + private static class ClassToLoad { + } } -- Gitee From 5f2f661e2c030b384634d90a13578c7c52c6a448 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Mon, 2 Mar 2020 14:12:32 +0100 Subject: [PATCH 384/885] [FLINK-16245][table] Close user classloader --- .../java/org/apache/flink/client/ClientUtils.java | 3 ++- .../table/client/gateway/local/ExecutionContext.java | 12 ++++++++++-- .../table/client/gateway/local/LocalExecutor.java | 7 ++++++- 3 files changed, 18 insertions(+), 4 deletions(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java b/flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java index f1b663ee74..ff03601b25 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java +++ b/flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java @@ -40,6 +40,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; import java.net.URL; +import java.net.URLClassLoader; import java.util.List; import java.util.concurrent.ExecutionException; @@ -53,7 +54,7 @@ public enum ClientUtils { private static final Logger LOG = LoggerFactory.getLogger(ClientUtils.class); - public static ClassLoader buildUserCodeClassLoader( + public static URLClassLoader buildUserCodeClassLoader( List jars, List classpaths, ClassLoader parent, diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java index 0529434326..8a8fe85edf 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java @@ -96,8 +96,11 @@ import org.slf4j.LoggerFactory; import javax.annotation.Nullable; +import java.io.Closeable; +import java.io.IOException; import java.lang.reflect.Method; import java.net.URL; +import java.net.URLClassLoader; import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashMap; @@ -118,13 +121,13 @@ import static org.apache.flink.util.Preconditions.checkState; * * @param cluster id */ -public class ExecutionContext { +public class ExecutionContext implements Closeable { private static final Logger LOG = LoggerFactory.getLogger(ExecutionContext.class); private final Environment environment; private final SessionContext originalSessionContext; - private final ClassLoader classLoader; + private final URLClassLoader classLoader; private final Configuration flinkConfig; private final ClusterClientFactory clusterClientFactory; @@ -703,6 +706,11 @@ public class ExecutionContext { } } + @Override + public void close() throws IOException { + this.classLoader.close(); + } + //~ Inner Class ------------------------------------------------------------------------------- /** Builder for {@link ExecutionContext}. */ diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java index 1c9ea2a276..a133be511b 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java @@ -239,7 +239,12 @@ public class LocalExecutor implements Executor { } }); // Remove the session's ExecutionContext from contextMap. - this.contextMap.remove(sessionId); + try { + this.contextMap.remove(sessionId).close(); + } catch (IOException e) { + LOG.debug("Error while closing execution context.", e); + // ignore any throwable to keep the clean up running + } } /** -- Gitee From a982488fd8a6b8155f4fcb45c893661e11bade61 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Thu, 12 Mar 2020 20:43:18 +0100 Subject: [PATCH 385/885] [FLINK-16245][tests] Adjust BatchFineGrainedRecoveryITCase Because the classloader is now closed when the task fails the UDF only has access to the bootstrap classloader, which doesn't contain our own test classes. --- .../BatchFineGrainedRecoveryITCase.java | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/BatchFineGrainedRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/BatchFineGrainedRecoveryITCase.java index 9a2a52c470..86e32edf3c 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/BatchFineGrainedRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/BatchFineGrainedRecoveryITCase.java @@ -60,6 +60,7 @@ import org.apache.flink.util.ConfigurationException; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.TemporaryClassLoaderContext; import org.apache.flink.util.TestLogger; import org.junit.After; @@ -396,14 +397,16 @@ public class BatchFineGrainedRecoveryITCase extends TestLogger { @Override void fail(int trackingIndex) throws Exception { //noinspection OverlyBroadCatchBlock - try { - restartTaskManager(); - } catch (InterruptedException e) { - // ignore the exception, task should have been failed while stopping TM - Thread.currentThread().interrupt(); - } catch (Throwable t) { - failureTracker.unrelatedFailure(t); - throw t; + try (TemporaryClassLoaderContext unused = TemporaryClassLoaderContext.of(ClassLoader.getSystemClassLoader())) { + try { + restartTaskManager(); + } catch (InterruptedException e) { + // ignore the exception, task should have been failed while stopping TM + Thread.currentThread().interrupt(); + } catch (Throwable t) { + failureTracker.unrelatedFailure(t); + throw t; + } } } } -- Gitee From 73958cc531a0f22e95c037912f52c09cbb9e0f17 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 30 Mar 2020 14:37:14 +0200 Subject: [PATCH 386/885] [FLINK-16018] Increase default value of web.timeout to 10 minutes In order to not fail the job submission with a TimeoutException if it takes longer than 10 s, this commit increases the web.timeout to 10 minutes. This closes #11565. --- docs/_includes/generated/web_configuration.html | 2 +- .../main/java/org/apache/flink/configuration/WebOptions.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/_includes/generated/web_configuration.html b/docs/_includes/generated/web_configuration.html index f377f51a05..4b5a98d45a 100644 --- a/docs/_includes/generated/web_configuration.html +++ b/docs/_includes/generated/web_configuration.html @@ -70,7 +70,7 @@

    - + diff --git a/flink-core/src/main/java/org/apache/flink/configuration/WebOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/WebOptions.java index 762f3760ae..cb4636e6b2 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/WebOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/WebOptions.java @@ -178,7 +178,7 @@ public class WebOptions { */ public static final ConfigOption TIMEOUT = key("web.timeout") - .defaultValue(10L * 1000L) + .defaultValue(10L * 60L * 1000L) .withDescription("Timeout for asynchronous operations by the web monitor in milliseconds."); // ------------------------------------------------------------------------ -- Gitee From a9d634b5388d57c43cfe64fd27c0bdc168513069 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 27 Mar 2020 14:30:35 +0100 Subject: [PATCH 387/885] [hotfix][rpc] Add proper error reporting to AkkaRpcActor#handleControlMessage --- .../flink/runtime/rpc/akka/AkkaRpcActor.java | 59 ++++++++++--------- 1 file changed, 32 insertions(+), 27 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java index 25fb38b24f..03971721a1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java @@ -119,7 +119,7 @@ class AkkaRpcActor extends AbstractActor { this.rpcEndpointTerminationResult = RpcEndpointTerminationResult.failure( new AkkaRpcException( String.format("RpcEndpoint %s has not been properly stopped.", rpcEndpoint.getEndpointId()))); - this.state = StoppedState.INSTANCE; + this.state = StoppedState.STOPPED; } @Override @@ -164,18 +164,23 @@ class AkkaRpcActor extends AbstractActor { } private void handleControlMessage(ControlMessages controlMessage) { - switch (controlMessage) { - case START: - state = state.start(this); - break; - case STOP: - state = state.stop(); - break; - case TERMINATE: - state.terminate(this); - break; - default: - handleUnknownControlMessage(controlMessage); + try { + switch (controlMessage) { + case START: + state = state.start(this); + break; + case STOP: + state = state.stop(); + break; + case TERMINATE: + state = state.terminate(this); + break; + default: + handleUnknownControlMessage(controlMessage); + } + } catch (Exception e) { + this.rpcEndpointTerminationResult = RpcEndpointTerminationResult.failure(e); + throw e; } } @@ -462,19 +467,19 @@ class AkkaRpcActor extends AbstractActor { interface State { default State start(AkkaRpcActor akkaRpcActor) { - throw new AkkaRpcInvalidStateException(invalidStateTransitionMessage(StartedState.INSTANCE)); + throw new AkkaRpcInvalidStateException(invalidStateTransitionMessage(StartedState.STARTED)); } default State stop() { - throw new AkkaRpcInvalidStateException(invalidStateTransitionMessage(StoppedState.INSTANCE)); + throw new AkkaRpcInvalidStateException(invalidStateTransitionMessage(StoppedState.STOPPED)); } default State terminate(AkkaRpcActor akkaRpcActor) { - throw new AkkaRpcInvalidStateException(invalidStateTransitionMessage(TerminatingState.INSTANCE)); + throw new AkkaRpcInvalidStateException(invalidStateTransitionMessage(TerminatingState.TERMINATING)); } default State finishTermination() { - return TerminatedState.INSTANCE; + return TerminatedState.TERMINATED; } default boolean isRunning() { @@ -488,16 +493,16 @@ class AkkaRpcActor extends AbstractActor { @SuppressWarnings("Singleton") enum StartedState implements State { - INSTANCE; + STARTED; @Override public State start(AkkaRpcActor akkaRpcActor) { - return INSTANCE; + return STARTED; } @Override public State stop() { - return StoppedState.INSTANCE; + return StoppedState.STOPPED; } @Override @@ -523,7 +528,7 @@ class AkkaRpcActor extends AbstractActor { terminationFuture.whenComplete((ignored, throwable) -> akkaRpcActor.stop(RpcEndpointTerminationResult.of(throwable))); - return TerminatingState.INSTANCE; + return TerminatingState.TERMINATING; } @Override @@ -534,7 +539,7 @@ class AkkaRpcActor extends AbstractActor { @SuppressWarnings("Singleton") enum StoppedState implements State { - INSTANCE; + STOPPED; @Override public State start(AkkaRpcActor akkaRpcActor) { @@ -552,25 +557,25 @@ class AkkaRpcActor extends AbstractActor { akkaRpcActor.mainThreadValidator.exitMainThread(); } - return StartedState.INSTANCE; + return StartedState.STARTED; } @Override public State stop() { - return INSTANCE; + return STOPPED; } @Override public State terminate(AkkaRpcActor akkaRpcActor) { akkaRpcActor.stop(RpcEndpointTerminationResult.success()); - return TerminatingState.INSTANCE; + return TerminatingState.TERMINATING; } } @SuppressWarnings("Singleton") enum TerminatingState implements State { - INSTANCE; + TERMINATING; @Override public boolean isRunning() { @@ -579,7 +584,7 @@ class AkkaRpcActor extends AbstractActor { } enum TerminatedState implements State { - INSTANCE + TERMINATED } private static final class RpcEndpointTerminationResult { -- Gitee From 1e0db6425f18b8d69b69176b9cb75e8b000bf768 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 27 Mar 2020 14:34:07 +0100 Subject: [PATCH 388/885] [FLINK-16703][rpc] Set AkkaRpcActor state to TERMINATING when terminating This commit fixes a bug where we did not update the state of the AkkaRpcActor in case of terminating it. Moreover, this commit fixes the problem that the onStop action could have been called multiple times. Last but not least, it changes the enum names of the state implementations for better diagnostics. This closes #11549. --- .../flink/runtime/rpc/akka/AkkaRpcActor.java | 5 ++ .../runtime/rpc/akka/AkkaRpcActorTest.java | 64 +++++++++++++++++++ 2 files changed, 69 insertions(+) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java index 03971721a1..7ad37762fc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java @@ -577,6 +577,11 @@ class AkkaRpcActor extends AbstractActor { enum TerminatingState implements State { TERMINATING; + @Override + public State terminate(AkkaRpcActor akkaRpcActor) { + return TERMINATING; + } + @Override public boolean isRunning() { return true; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java index 0597b01f46..64270c4367 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java @@ -36,6 +36,7 @@ import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.Preconditions; import org.apache.flink.util.TestLogger; +import akka.actor.ActorRef; import akka.actor.ActorSystem; import org.hamcrest.core.Is; import org.junit.AfterClass; @@ -395,6 +396,38 @@ public class AkkaRpcActorTest extends TestLogger { } } + /** + * Tests that multiple termination calls won't trigger the onStop action multiple times. + * Note that this test is a probabilistic test which only fails sometimes without the fix. + * See FLINK-16703. + */ + @Test + public void callsOnStopOnlyOnce() throws Exception { + final CompletableFuture onStopFuture = new CompletableFuture<>(); + final OnStopCountingRpcEndpoint endpoint = new OnStopCountingRpcEndpoint(akkaRpcService, onStopFuture); + + try { + endpoint.start(); + + final AkkaBasedEndpoint selfGateway = endpoint.getSelfGateway(AkkaBasedEndpoint.class); + + // try to terminate the actor twice + selfGateway.getActorRef().tell(ControlMessages.TERMINATE, ActorRef.noSender()); + selfGateway.getActorRef().tell(ControlMessages.TERMINATE, ActorRef.noSender()); + + endpoint.waitUntilOnStopHasBeenCalled(); + + onStopFuture.complete(null); + + endpoint.getTerminationFuture().get(); + + assertThat(endpoint.getNumOnStopCalls(), is(1)); + } finally { + onStopFuture.complete(null); + RpcUtils.terminateRpcEndpoint(endpoint, timeout); + } + } + // ------------------------------------------------------------------------ // Test Actors and Interfaces // ------------------------------------------------------------------------ @@ -609,4 +642,35 @@ public class AkkaRpcActorTest extends TestLogger { countDownLatch.await(); } } + + // ------------------------------------------------------------------------ + + private static final class OnStopCountingRpcEndpoint extends RpcEndpoint { + + private final AtomicInteger numOnStopCalls = new AtomicInteger(0); + + private final OneShotLatch onStopHasBeenCalled = new OneShotLatch(); + + private final CompletableFuture onStopFuture; + + private OnStopCountingRpcEndpoint(RpcService rpcService, CompletableFuture onStopFuture) { + super(rpcService); + this.onStopFuture = onStopFuture; + } + + @Override + protected CompletableFuture onStop() { + onStopHasBeenCalled.trigger(); + numOnStopCalls.incrementAndGet(); + return onStopFuture; + } + + private int getNumOnStopCalls() { + return numOnStopCalls.get(); + } + + private void waitUntilOnStopHasBeenCalled() throws InterruptedException { + onStopHasBeenCalled.await(); + } + } } -- Gitee From 7ff9ce16607fe345fa41d0c92eb956df4b31e067 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 27 Mar 2020 18:08:48 +0100 Subject: [PATCH 389/885] [FLINK-16836] Clear rpcConnection field in JobManagerLeaderListener when target loses leadership Clearing the rpcConnection field in the JobManagerLeaderListener when target loses leadership prevents that we try to reconnect to the target in case JobLeaderService.reconnect(JobID) is called. This closes #11552. --- .../taskexecutor/JobLeaderService.java | 2 + .../taskexecutor/JobLeaderServiceTest.java | 72 +++++++++++++++++-- 2 files changed, 68 insertions(+), 6 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java index fe9463a381..94832dac67 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java @@ -241,6 +241,7 @@ public class JobLeaderService { /** Rpc connection to the job leader. */ @GuardedBy("lock") + @Nullable private RegisteredRpcConnection rpcConnection; /** Leader id of the current job leader. */ @@ -311,6 +312,7 @@ public class JobLeaderService { // the leader lost leadership but there is no other leader yet. if (rpcConnection != null) { rpcConnection.close(); + rpcConnection = null; } jobManagerLostLeadership = Optional.ofNullable(currentJobMasterId); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/JobLeaderServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/JobLeaderServiceTest.java index a27e7d0632..d61d601a95 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/JobLeaderServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/JobLeaderServiceTest.java @@ -21,30 +21,34 @@ package org.apache.flink.runtime.taskexecutor; import org.apache.flink.api.common.JobID; import org.apache.flink.core.testutils.CheckedThread; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServicesBuilder; import org.apache.flink.runtime.jobmaster.JMTMRegistrationSuccess; import org.apache.flink.runtime.jobmaster.JobMasterGateway; import org.apache.flink.runtime.jobmaster.JobMasterId; +import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGateway; +import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGatewayBuilder; import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService; import org.apache.flink.runtime.registration.RetryingRegistrationConfiguration; import org.apache.flink.runtime.rpc.TestingRpcServiceResource; import org.apache.flink.runtime.taskmanager.LocalUnresolvedTaskManagerLocation; import org.apache.flink.util.TestLogger; -import org.junit.ClassRule; +import org.junit.Rule; import org.junit.Test; import java.util.UUID; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CountDownLatch; /** * Tests for the {@link JobLeaderService}. */ public class JobLeaderServiceTest extends TestLogger { - @ClassRule - public static final TestingRpcServiceResource RPC_SERVICE_RESOURCE = new TestingRpcServiceResource(); + @Rule + public final TestingRpcServiceResource rpcServiceResource = new TestingRpcServiceResource(); /** * Tests that we can concurrently modify the JobLeaderService and complete the leader retrieval operation. @@ -71,7 +75,7 @@ public class JobLeaderServiceTest extends TestLogger { jobLeaderService.start( "foobar", - RPC_SERVICE_RESOURCE.getTestingRpcService(), + rpcServiceResource.getTestingRpcService(), haServices, jobLeaderListener); @@ -96,21 +100,77 @@ public class JobLeaderServiceTest extends TestLogger { addJobAction.sync(); } + /** + * Tests that the JobLeaderService won't try to reconnect to JobMaster after it + * has lost the leadership. See FLINK-16836. + */ + @Test + public void doesNotReconnectAfterTargetLostLeadership() throws Exception { + final JobLeaderService jobLeaderService = new JobLeaderService( + new LocalUnresolvedTaskManagerLocation(), + RetryingRegistrationConfiguration.defaultConfiguration()); + + final JobID jobId = new JobID(); + + final SettableLeaderRetrievalService leaderRetrievalService = new SettableLeaderRetrievalService(); + final TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServicesBuilder() + .setJobMasterLeaderRetrieverFunction(ignored -> leaderRetrievalService) + .build(); + + final String jmAddress = "foobar"; + final TestingJobMasterGateway jobMasterGateway = new TestingJobMasterGatewayBuilder().build(); + rpcServiceResource.getTestingRpcService().registerGateway(jmAddress, jobMasterGateway); + + final TestingJobLeaderListener testingJobLeaderListener = new TestingJobLeaderListener(); + jobLeaderService.start( + "foobar", + rpcServiceResource.getTestingRpcService(), + haServices, + testingJobLeaderListener); + + try { + jobLeaderService.addJob(jobId, jmAddress); + + leaderRetrievalService.notifyListener(jmAddress, UUID.randomUUID()); + + testingJobLeaderListener.waitUntilJobManagerGainedLeadership(); + + // revoke the leadership + leaderRetrievalService.notifyListener(null, null); + testingJobLeaderListener.waitUntilJobManagerLostLeadership(); + + jobLeaderService.reconnect(jobId); + } finally { + jobLeaderService.stop(); + } + } + private static final class TestingJobLeaderListener implements JobLeaderListener { + private final CountDownLatch jobManagerGainedLeadership = new CountDownLatch(1); + private final CountDownLatch jobManagerLostLeadership = new CountDownLatch(1); + @Override public void jobManagerGainedLeadership(JobID jobId, JobMasterGateway jobManagerGateway, JMTMRegistrationSuccess registrationMessage) { - // ignored + jobManagerGainedLeadership.countDown(); } @Override public void jobManagerLostLeadership(JobID jobId, JobMasterId jobMasterId) { - // ignored + jobManagerLostLeadership.countDown(); } @Override public void handleError(Throwable throwable) { // ignored } + + private void waitUntilJobManagerGainedLeadership() throws InterruptedException { + jobManagerGainedLeadership.await(); + } + + private void waitUntilJobManagerLostLeadership() throws InterruptedException { + jobManagerLostLeadership.await(); + } } } -- Gitee From 451975853cb0a1999ed3dbb57f4190521f1e0d5c Mon Sep 17 00:00:00 2001 From: Danny Chan Date: Tue, 31 Mar 2020 21:06:14 +0800 Subject: [PATCH 390/885] [FLINK-16878][e2e][table] Fix dependency shading of table modules test failure after FLINK-14338 This closes #11575 --- .../test-scripts/test_table_shaded_dependencies.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/flink-end-to-end-tests/test-scripts/test_table_shaded_dependencies.sh b/flink-end-to-end-tests/test-scripts/test_table_shaded_dependencies.sh index 529a983eb6..479ce04b58 100755 --- a/flink-end-to-end-tests/test-scripts/test_table_shaded_dependencies.sh +++ b/flink-end-to-end-tests/test-scripts/test_table_shaded_dependencies.sh @@ -61,6 +61,8 @@ function checkCodeDependencies { grep -v "^\s*\-> org.json" |\ grep -v "^\s*\-> org.apache.tapestry5.json." |\ grep -v "^\s*\-> org.codehaus.jettison" |\ + grep -v "^\s*\-> org.apiguardian.api" |\ + grep -v "^\s*\-> org.apache.commons.io.input" |\ grep -v "^\s*\-> net.minidev.json" > $CONTENTS_FILE if [[ `cat $CONTENTS_FILE | wc -l` -eq '0' ]]; then echo "Success: There are no unwanted dependencies in the ${JAR} jar." -- Gitee From a857a274a7520ee9e29c304cd17e8b9e8dd49cc6 Mon Sep 17 00:00:00 2001 From: HuFeiHu <1773554122@qq.com> Date: Sun, 29 Mar 2020 16:52:20 +0800 Subject: [PATCH 391/885] [FLINK-16849] [docs] Comment is not reasonable --- .../src/main/java/org/apache/flink/annotation/Internal.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-annotations/src/main/java/org/apache/flink/annotation/Internal.java b/flink-annotations/src/main/java/org/apache/flink/annotation/Internal.java index 65231c29de..16208a0bc2 100644 --- a/flink-annotations/src/main/java/org/apache/flink/annotation/Internal.java +++ b/flink-annotations/src/main/java/org/apache/flink/annotation/Internal.java @@ -24,7 +24,7 @@ import java.lang.annotation.ElementType; import java.lang.annotation.Target; /** - * Interface to mark methods within stable, public APIs as an internal developer API. + * Annotation to mark methods within stable, public APIs as an internal developer API. * *

    Developer APIs are stable but internal to Flink and might change across releases. */ -- Gitee From f728eec8c71f57447702840f3e1f8f7aaf2bd16f Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Fri, 27 Mar 2020 11:07:37 +0100 Subject: [PATCH 392/885] [FLINK-16411][AZP] Cache maven artifacts for tests as well --- tools/azure-pipelines/jobs-template.yml | 2 -- 1 file changed, 2 deletions(-) diff --git a/tools/azure-pipelines/jobs-template.yml b/tools/azure-pipelines/jobs-template.yml index 911fa7f040..85217901a6 100644 --- a/tools/azure-pipelines/jobs-template.yml +++ b/tools/azure-pipelines/jobs-template.yml @@ -105,9 +105,7 @@ jobs: path: $(CACHE_FLINK_DIR) artifact: FlinkCompileCacheDir-${{parameters.stage_name}} - # only for the python stage (which runs a full mvn install), download the cache - task: Cache@2 - condition: eq(variables['module'], 'python') inputs: key: $(CACHE_KEY) restoreKeys: $(CACHE_FALLBACK_KEY) -- Gitee From 15d42c8756189aed9ee81e6fc08bc035311511de Mon Sep 17 00:00:00 2001 From: william <1342247033@qq.com> Date: Tue, 31 Mar 2020 21:07:35 +0800 Subject: [PATCH 393/885] [hotfix] Fix typo in org.apache.flink.sql.parser.ddl.SqlCreateTable This closes #11580 --- .../java/org/apache/flink/sql/parser/ddl/SqlCreateTable.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateTable.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateTable.java index 19839a0205..feaa373b3f 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateTable.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateTable.java @@ -86,7 +86,7 @@ public class SqlCreateTable extends SqlCreate implements ExtendedSqlNode { super(OPERATOR, pos, false, false); this.tableName = requireNonNull(tableName, "tableName should not be null"); this.columnList = requireNonNull(columnList, "columnList should not be null"); - this.primaryKeyList = requireNonNull(primaryKeyList, "primayKeyList should not be null"); + this.primaryKeyList = requireNonNull(primaryKeyList, "primaryKeyList should not be null"); this.uniqueKeysList = requireNonNull(uniqueKeysList, "uniqueKeysList should not be null"); this.propertyList = requireNonNull(propertyList, "propertyList should not be null"); this.partitionKeyList = requireNonNull(partitionKeyList, "partitionKeyList should not be null"); -- Gitee From 853fc3c580fa1166a25cec76596d843a9b49c948 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Wed, 1 Apr 2020 14:46:00 +0800 Subject: [PATCH 394/885] [FLINK-16858][table] Expose partitioned by grammar This closes #11559 --- .../table/catalog/hive/HiveTestUtils.java | 2 - .../src/main/codegen/includes/parserImpls.ftl | 7 +--- .../sql/parser/utils/ParserResource.java | 3 -- .../parser/validate/FlinkSqlConformance.java | 7 ---- .../ParserResource.properties | 1 - .../sql/parser/FlinkSqlParserImplTest.java | 40 +------------------ .../SqlToOperationConverterTest.java | 8 ++-- .../sqlexec/SqlToOperationConverterTest.java | 8 ++-- .../batch/sql/PartitionableSinkITCase.scala | 1 - 9 files changed, 10 insertions(+), 67 deletions(-) diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveTestUtils.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveTestUtils.java index 88184fc9d0..b3753351cd 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveTestUtils.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveTestUtils.java @@ -19,7 +19,6 @@ package org.apache.flink.table.catalog.hive; import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.SqlDialect; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.catalog.CatalogTest; import org.apache.flink.table.catalog.exceptions.CatalogException; @@ -114,7 +113,6 @@ public class HiveTestUtils { EnvironmentSettings settings = EnvironmentSettings.newInstance().useBlinkPlanner().inBatchMode().build(); TableEnvironment tableEnv = TableEnvironment.create(settings); tableEnv.getConfig().getConfiguration().setInteger(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM.key(), 1); - tableEnv.getConfig().setSqlDialect(SqlDialect.HIVE); return tableEnv; } diff --git a/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl b/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl index 96a25a42fe..3401d95db2 100644 --- a/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl +++ b/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl @@ -601,12 +601,7 @@ SqlCreate SqlCreateTable(Span s, boolean replace) : }] [ - partitionColumns = ParenthesizedSimpleIdentifierList() { - if (!((FlinkSqlConformance) this.conformance).allowCreatePartitionedTable()) { - throw SqlUtil.newContextException(getPos(), - ParserResource.RESOURCE.createPartitionedTableIsOnlyAllowedForHive()); - } - } + partitionColumns = ParenthesizedSimpleIdentifierList() ] [ diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/utils/ParserResource.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/utils/ParserResource.java index 5989a374fe..555ee5fa91 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/utils/ParserResource.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/utils/ParserResource.java @@ -35,7 +35,4 @@ public interface ParserResource { @Resources.BaseMessage("OVERWRITE expression is only used with INSERT statement.") Resources.ExInst overwriteIsOnlyUsedWithInsert(); - - @Resources.BaseMessage("Creating partitioned table is only allowed for HIVE dialect.") - Resources.ExInst createPartitionedTableIsOnlyAllowedForHive(); } diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/validate/FlinkSqlConformance.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/validate/FlinkSqlConformance.java index cf32026680..b6b2e83ea9 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/validate/FlinkSqlConformance.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/validate/FlinkSqlConformance.java @@ -155,11 +155,4 @@ public enum FlinkSqlConformance implements SqlConformance { public boolean allowQualifyingCommonColumn() { return true; } - - /** - * Whether to allow "create table T(i int, j int) partitioned by (i)" grammar. - */ - public boolean allowCreatePartitionedTable() { - return this == FlinkSqlConformance.HIVE; - } } diff --git a/flink-table/flink-sql-parser/src/main/resources/org.apache.flink.sql.parser.utils/ParserResource.properties b/flink-table/flink-sql-parser/src/main/resources/org.apache.flink.sql.parser.utils/ParserResource.properties index 9993e7c2e9..8a7d5a1c46 100644 --- a/flink-table/flink-sql-parser/src/main/resources/org.apache.flink.sql.parser.utils/ParserResource.properties +++ b/flink-table/flink-sql-parser/src/main/resources/org.apache.flink.sql.parser.utils/ParserResource.properties @@ -18,4 +18,3 @@ # MultipleWatermarksUnsupported=Multiple WATERMARK statements is not supported yet. OverwriteIsOnlyUsedWithInsert=OVERWRITE expression is only used with INSERT statement. -CreatePartitionedTableIsOnlyAllowedForHive=Creating partitioned table is only allowed for HIVE dialect. diff --git a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java index 0509e91533..337de11b17 100644 --- a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java +++ b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java @@ -21,19 +21,14 @@ package org.apache.flink.sql.parser; import org.apache.flink.sql.parser.ddl.SqlCreateTable; import org.apache.flink.sql.parser.error.SqlValidateException; import org.apache.flink.sql.parser.impl.FlinkSqlParserImpl; -import org.apache.flink.sql.parser.validate.FlinkSqlConformance; -import org.apache.calcite.avatica.util.Casing; -import org.apache.calcite.avatica.util.Quoting; import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.parser.SqlParseException; import org.apache.calcite.sql.parser.SqlParser; import org.apache.calcite.sql.parser.SqlParserImplFactory; import org.apache.calcite.sql.parser.SqlParserTest; -import org.apache.calcite.sql.validate.SqlConformance; import org.hamcrest.BaseMatcher; import org.hamcrest.Description; -import org.junit.Before; import org.junit.Test; import java.io.Reader; @@ -45,7 +40,6 @@ import static org.junit.Assert.fail; /** FlinkSqlParserImpl tests. **/ public class FlinkSqlParserImplTest extends SqlParserTest { - private SqlConformance conformance0; @Override protected SqlParserImplFactory parserImplFactory() { @@ -54,25 +48,7 @@ public class FlinkSqlParserImplTest extends SqlParserTest { protected SqlParser getSqlParser(Reader source, UnaryOperator transform) { - if (conformance0 == null) { - return super.getSqlParser(source, transform); - } else { - // overwrite the default sql conformance. - return SqlParser.create(source, - SqlParser.configBuilder() - .setParserFactory(parserImplFactory()) - .setQuoting(Quoting.DOUBLE_QUOTE) - .setUnquotedCasing(Casing.TO_UPPER) - .setQuotedCasing(Casing.UNCHANGED) - .setConformance(conformance0) - .build()); - } - } - - @Before - public void before() { - // clear the custom sql conformance. - conformance0 = null; + return super.getSqlParser(source, transform); } @Test @@ -224,7 +200,6 @@ public class FlinkSqlParserImplTest extends SqlParserTest { @Test public void testCreateTable() { - conformance0 = FlinkSqlConformance.HIVE; final String sql = "CREATE TABLE tbl1 (\n" + " a bigint,\n" + " h varchar, \n" + @@ -258,7 +233,6 @@ public class FlinkSqlParserImplTest extends SqlParserTest { @Test public void testCreateTableWithComment() { - conformance0 = FlinkSqlConformance.HIVE; final String sql = "CREATE TABLE tbl1 (\n" + " a bigint comment 'test column comment AAA.',\n" + " h varchar, \n" + @@ -294,7 +268,6 @@ public class FlinkSqlParserImplTest extends SqlParserTest { @Test public void testCreateTableWithPrimaryKeyAndUniqueKey() { - conformance0 = FlinkSqlConformance.HIVE; final String sql = "CREATE TABLE tbl1 (\n" + " a bigint comment 'test column comment AAA.',\n" + " h varchar, \n" + @@ -565,7 +538,6 @@ public class FlinkSqlParserImplTest extends SqlParserTest { @Test public void testCreateInvalidPartitionedTable() { - conformance0 = FlinkSqlConformance.HIVE; final String sql = "create table sls_stream1(\n" + " a bigint,\n" + " b VARCHAR,\n" + @@ -578,16 +550,6 @@ public class FlinkSqlParserImplTest extends SqlParserTest { .fails("Partition column [C] not defined in columns, at line 6, column 3")); } - @Test - public void testNotAllowedCreatePartition() { - conformance0 = FlinkSqlConformance.DEFAULT; - final String sql = "create table sls_stream1(\n" + - " a bigint,\n" + - " b VARCHAR\n" + - ") PARTITIONED BY (a^)^ with ( 'x' = 'y', 'asd' = 'dada')"; - sql(sql).fails("Creating partitioned table is only allowed for HIVE dialect."); - } - @Test public void testCreateTableWithMinusInOptionKey() { final String sql = "create table source_table(\n" + diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/operations/SqlToOperationConverterTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/operations/SqlToOperationConverterTest.java index 55e0023827..e4d0460a0a 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/operations/SqlToOperationConverterTest.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/operations/SqlToOperationConverterTest.java @@ -246,8 +246,8 @@ public class SqlToOperationConverterTest { " 'connector' = 'kafka', \n" + " 'kafka.topic' = 'log.test'\n" + ")\n"; - FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.HIVE); - final CalciteParser parser = getParserBySqlDialect(SqlDialect.HIVE); + FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); + final CalciteParser parser = getParserBySqlDialect(SqlDialect.DEFAULT); Operation operation = parse(sql, planner, parser); assert operation instanceof CreateTableOperation; CreateTableOperation op = (CreateTableOperation) operation; @@ -265,8 +265,8 @@ public class SqlToOperationConverterTest { @Test(expected = SqlConversionException.class) public void testCreateTableWithPkUniqueKeys() { - FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.HIVE); - final CalciteParser parser = getParserBySqlDialect(SqlDialect.HIVE); + FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); + final CalciteParser parser = getParserBySqlDialect(SqlDialect.DEFAULT); final String sql = "CREATE TABLE tbl1 (\n" + " a bigint,\n" + " b varchar, \n" + diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java index b95ea40b71..cd981cb499 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java @@ -245,8 +245,8 @@ public class SqlToOperationConverterTest { " 'connector' = 'kafka', \n" + " 'kafka.topic' = 'log.test'\n" + ")\n"; - final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.HIVE); - SqlNode node = getParserBySqlDialect(SqlDialect.HIVE).parse(sql); + final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); + SqlNode node = getParserBySqlDialect(SqlDialect.DEFAULT).parse(sql); assert node instanceof SqlCreateTable; Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node).get(); assert operation instanceof CreateTableOperation; @@ -306,8 +306,8 @@ public class SqlToOperationConverterTest { " 'connector' = 'kafka', \n" + " 'kafka.topic' = 'log.test'\n" + ")\n"; - final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.HIVE); - SqlNode node = getParserBySqlDialect(SqlDialect.HIVE).parse(sql); + final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); + SqlNode node = getParserBySqlDialect(SqlDialect.DEFAULT).parse(sql); assert node instanceof SqlCreateTable; SqlToOperationConverter.convert(planner, catalogManager, node); } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/PartitionableSinkITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/PartitionableSinkITCase.scala index 219b2db818..90a6f247f7 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/PartitionableSinkITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/PartitionableSinkITCase.scala @@ -62,7 +62,6 @@ class PartitionableSinkITCase extends AbstractTestBase { def before(): Unit = { batchExec.setParallelism(1) tEnv = BatchTableEnvironment.create(batchExec) - tEnv.getConfig.setSqlDialect(SqlDialect.HIVE) registerTableSource("nonSortTable", testData.toList) registerTableSource("sortTable", testData1.toList) PartitionableSinkITCase.init() -- Gitee From bb195633f3d1194f63a91d918581a51681440b28 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 31 Mar 2020 18:28:05 +0200 Subject: [PATCH 395/885] [FLINK-16888][legal] Add jquery license jquery is used for the documentation. --- licenses/LICENSE.jquery | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) create mode 100644 licenses/LICENSE.jquery diff --git a/licenses/LICENSE.jquery b/licenses/LICENSE.jquery new file mode 100644 index 0000000000..e3dbacb999 --- /dev/null +++ b/licenses/LICENSE.jquery @@ -0,0 +1,20 @@ +Copyright JS Foundation and other contributors, https://js.foundation/ + +Permission is hereby granted, free of charge, to any person obtaining +a copy of this software and associated documentation files (the +"Software"), to deal in the Software without restriction, including +without limitation the rights to use, copy, modify, merge, publish, +distribute, sublicense, and/or sell copies of the Software, and to +permit persons to whom the Software is furnished to do so, subject to +the following conditions: + +The above copyright notice and this permission notice shall be +included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND +NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE +LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION +OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION +WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. -- Gitee From e0333a41d72e59cf7a862f64d9dbee1b0626b4e7 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 26 Mar 2020 13:00:02 +0100 Subject: [PATCH 396/885] [FLINK-16807][e2e] Improve reporting for instantiation errors --- .../util/kafka/KafkaResourceFactory.java | 5 +- .../LocalStandaloneKafkaResourceFactory.java | 6 +-- .../util/kafka/SQLClientKafkaITCase.java | 3 +- .../util/cache/DownloadCacheFactory.java | 9 ++-- .../tests/util/cache/LolCacheFactory.java | 11 ++--- .../cache/PersistingDownloadCacheFactory.java | 6 +-- .../cache/TravisDownloadCacheFactory.java | 10 ++-- .../util/flink/FlinkResourceFactory.java | 9 ++-- .../LocalStandaloneFlinkResourceFactory.java | 6 +-- .../flink/tests/util/util/FactoryUtils.java | 48 ++++++++++++------- .../tests/MetricsAvailabilityITCase.java | 3 +- .../PrometheusReporterEndToEndITCase.java | 2 +- 12 files changed, 62 insertions(+), 56 deletions(-) diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/KafkaResourceFactory.java b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/KafkaResourceFactory.java index 3d08d64bb6..ae21a6a6c5 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/KafkaResourceFactory.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/KafkaResourceFactory.java @@ -30,7 +30,8 @@ public interface KafkaResourceFactory { * Returns a {@link KafkaResource} instance. If the instance could not be instantiated (for example, because a * mandatory parameter was missing), then an empty {@link Optional} should be returned. * - * @return KafkaResource instance, or an empty Optional if the instance could not be instantiated + * @return KafkaResource instance + * @throws Exception if the instance could not be instantiated */ - Optional create(String kafkaVersion); + KafkaResource create(String kafkaVersion) throws Exception; } diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/LocalStandaloneKafkaResourceFactory.java b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/LocalStandaloneKafkaResourceFactory.java index 0d61119155..2e274434f3 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/LocalStandaloneKafkaResourceFactory.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/LocalStandaloneKafkaResourceFactory.java @@ -21,8 +21,6 @@ package org.apache.flink.tests.util.kafka; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Optional; - /** * A {@link KafkaResourceFactory} for the {@link LocalStandaloneKafkaResourceFactory}. */ @@ -30,8 +28,8 @@ public final class LocalStandaloneKafkaResourceFactory implements KafkaResourceF private static final Logger LOG = LoggerFactory.getLogger(LocalStandaloneKafkaResourceFactory.class); @Override - public Optional create(final String kafkaVersion) { + public KafkaResource create(final String kafkaVersion) { LOG.info("Created {}.", LocalStandaloneKafkaResource.class.getSimpleName()); - return Optional.of(new LocalStandaloneKafkaResource(kafkaVersion)); + return new LocalStandaloneKafkaResource(kafkaVersion); } } diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientKafkaITCase.java b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientKafkaITCase.java index bf5e4f336b..923377eb01 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientKafkaITCase.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientKafkaITCase.java @@ -81,8 +81,7 @@ public class SQLClientKafkaITCase extends TestLogger { @Rule public final FlinkResource flink = new LocalStandaloneFlinkResourceFactory() - .create(FlinkResourceSetup.builder().build()) - .get(); + .create(FlinkResourceSetup.builder().build()); @Rule public final KafkaResource kafka; diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/DownloadCacheFactory.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/DownloadCacheFactory.java index 82ff48bdb1..767e96ac8f 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/DownloadCacheFactory.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/DownloadCacheFactory.java @@ -18,8 +18,6 @@ package org.apache.flink.tests.util.cache; -import java.util.Optional; - /** * A factory for {@link DownloadCache} implementations. */ @@ -28,9 +26,10 @@ public interface DownloadCacheFactory { /** * Returns a {@link DownloadCache} instance. If the instance could not be instantiated (for example, because a - * mandatory parameter was missing), then an empty {@link Optional} should be returned. + * mandatory parameter was missing), then an exception should be thrown. * - * @return DownloadCache instance, or an empty Optional if the instance could not be instantiated + * @return DownloadCache instance + * @throws Exception if the instance could not be instantiated */ - Optional create(); + DownloadCache create() throws Exception; } diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/LolCacheFactory.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/LolCacheFactory.java index 68e4973c53..b0be5ce8fd 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/LolCacheFactory.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/LolCacheFactory.java @@ -23,7 +23,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; -import java.util.Optional; /** * A {@link DownloadCacheFactory} for the {@link LolCache}. @@ -32,14 +31,10 @@ public final class LolCacheFactory implements DownloadCacheFactory { private static final Logger LOG = LoggerFactory.getLogger(LolCacheFactory.class); @Override - public Optional create() { + public DownloadCache create() throws IOException { final TemporaryFolder folder = new TemporaryFolder(); - try { - folder.create(); - } catch (IOException e) { - throw new RuntimeException("Could not initialize temporary directory.", e); - } + folder.create(); LOG.info("Created {}.", LolCache.class.getSimpleName()); - return Optional.of(new LolCache(folder)); + return new LolCache(folder); } } diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/PersistingDownloadCacheFactory.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/PersistingDownloadCacheFactory.java index 27013f2fab..bbc8ab3b8b 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/PersistingDownloadCacheFactory.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/PersistingDownloadCacheFactory.java @@ -40,14 +40,14 @@ public final class PersistingDownloadCacheFactory implements DownloadCacheFactor private static final Period TIME_TO_LIVE_DEFAULT = Period.ZERO; @Override - public Optional create() { + public DownloadCache create() { final Optional tmpDir = TMP_DIR.get(); final Period timeToLive = TIME_TO_LIVE.get(TIME_TO_LIVE_DEFAULT); if (!tmpDir.isPresent()) { LOG.debug("Not loading {} because {} was not set.", PersistingDownloadCache.class, TMP_DIR.getPropertyName()); - return Optional.empty(); + throw new IllegalArgumentException(String.format("Not loading %s because %s was not set.", PersistingDownloadCache.class, TMP_DIR.getPropertyName())); } LOG.info("Created {}.", PersistingDownloadCache.class.getSimpleName()); - return Optional.of(new PersistingDownloadCache(tmpDir.get(), timeToLive)); + return new PersistingDownloadCache(tmpDir.get(), timeToLive); } } diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/TravisDownloadCacheFactory.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/TravisDownloadCacheFactory.java index fa1057d10c..90520c536c 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/TravisDownloadCacheFactory.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/TravisDownloadCacheFactory.java @@ -38,23 +38,23 @@ public final class TravisDownloadCacheFactory implements DownloadCacheFactory { private static final ParameterProperty BUILD_NUMBER = new ParameterProperty<>("TRAVIS_BUILD_NUMBER", Integer::parseInt); @Override - public Optional create() { + public DownloadCache create() { final Optional tmpDir = TMP_DIR.get(); final Optional timeToLive = BUILDS_TO_LIVE.get(); final Optional buildNumber = BUILD_NUMBER.get(); if (!tmpDir.isPresent()) { LOG.debug("Not loading {} because {} was not set.", TravisDownloadCache.class, TMP_DIR.getPropertyName()); - return Optional.empty(); + throw new IllegalArgumentException(String.format("Not loading %s because %s was not set.", TravisDownloadCache.class, TMP_DIR.getPropertyName())); } if (!timeToLive.isPresent()) { LOG.debug("Not loading {} because {} was not set.", TravisDownloadCache.class, BUILDS_TO_LIVE.getPropertyName()); - return Optional.empty(); + throw new IllegalArgumentException(String.format("Not loading %s because %s was not set.", TravisDownloadCache.class, BUILDS_TO_LIVE.getPropertyName())); } if (!buildNumber.isPresent()) { LOG.debug("Not loading {} because {} was not set.", TravisDownloadCache.class, BUILD_NUMBER.getPropertyName()); - return Optional.empty(); + throw new IllegalArgumentException(String.format("Not loading %s because %s was not set.", TravisDownloadCache.class, BUILD_NUMBER.getPropertyName())); } LOG.info("Created {}.", TravisDownloadCache.class.getSimpleName()); - return Optional.of(new TravisDownloadCache(tmpDir.get(), timeToLive.get(), buildNumber.get())); + return new TravisDownloadCache(tmpDir.get(), timeToLive.get(), buildNumber.get()); } } diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkResourceFactory.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkResourceFactory.java index 385282be8c..20bc8dd136 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkResourceFactory.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkResourceFactory.java @@ -18,8 +18,6 @@ package org.apache.flink.tests.util.flink; -import java.util.Optional; - /** * A factory for {@link FlinkResource} implementations. */ @@ -28,10 +26,11 @@ public interface FlinkResourceFactory { /** * Returns a {@link FlinkResource} instance. If the instance could not be instantiated (for example, because a - * mandatory parameter was missing), then an empty {@link Optional} should be returned. + * mandatory parameter was missing), then an exception should be thrown. * * @param setup setup instructions for the FlinkResource - * @return FlinkResource instance, or an empty Optional if the instance could not be instantiated + * @return FlinkResource instance, + * @throws Exception if the instance could not be instantiated */ - Optional create(FlinkResourceSetup setup); + FlinkResource create(FlinkResourceSetup setup) throws Exception; } diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResourceFactory.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResourceFactory.java index e4c1e5c02f..2edf693e39 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResourceFactory.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResourceFactory.java @@ -37,17 +37,17 @@ public final class LocalStandaloneFlinkResourceFactory implements FlinkResourceF private static final ParameterProperty DISTRIBUTION_LOG_BACKUP_DIRECTORY = new ParameterProperty<>("logBackupDir", Paths::get); @Override - public Optional create(FlinkResourceSetup setup) { + public FlinkResource create(FlinkResourceSetup setup) { Optional distributionDirectory = DISTRIBUTION_DIRECTORY.get(); if (!distributionDirectory.isPresent()) { LOG.warn("The distDir property was not set. You can set it when running maven via -DdistDir= ."); - return Optional.empty(); + throw new IllegalArgumentException("The distDir property was not set. You can set it when running maven via -DdistDir= ."); } Optional logBackupDirectory = DISTRIBUTION_LOG_BACKUP_DIRECTORY.get(); if (!logBackupDirectory.isPresent()) { LOG.warn("Property {} not set, logs will not be backed up in case of test failures.", DISTRIBUTION_LOG_BACKUP_DIRECTORY.getPropertyName()); } LOG.info("Created {}.", LocalStandaloneFlinkResource.class.getSimpleName()); - return Optional.of(new LocalStandaloneFlinkResource(distributionDirectory.get(), logBackupDirectory.orElse(null), setup)); + return new LocalStandaloneFlinkResource(distributionDirectory.get(), logBackupDirectory.orElse(null), setup); } } diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/util/FactoryUtils.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/util/FactoryUtils.java index 079443c5bd..4100f259ca 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/util/FactoryUtils.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/util/FactoryUtils.java @@ -18,13 +18,10 @@ package org.apache.flink.tests.util.util; +import java.util.ArrayList; import java.util.List; -import java.util.Optional; import java.util.ServiceLoader; -import java.util.function.Function; import java.util.function.Supplier; -import java.util.stream.Collectors; -import java.util.stream.StreamSupport; /** * Utilities for factories. @@ -43,24 +40,43 @@ public enum FactoryUtils { * @throws RuntimeException if no or multiple resources could be instantiated * @return created instance */ - public static R loadAndInvokeFactory(final Class factoryInterface, final Function> factoryInvoker, final Supplier defaultProvider) { + public static R loadAndInvokeFactory(final Class factoryInterface, final FactoryInvoker factoryInvoker, final Supplier defaultProvider) { final ServiceLoader factories = ServiceLoader.load(factoryInterface); - final List resources = StreamSupport.stream(factories.spliterator(), false) - .map(factoryInvoker) - .filter(Optional::isPresent) - .map(Optional::get) - .collect(Collectors.toList()); + final List instantiatedResources = new ArrayList<>(); + final List errorsDuringInitialization = new ArrayList<>(); + for (F factory : factories) { + try { + R resource = factoryInvoker.invoke(factory); + instantiatedResources.add(resource); + } catch (Exception e) { + errorsDuringInitialization.add(e); + } + } - if (resources.size() == 1) { - return resources.get(0); + if (instantiatedResources.size() == 1) { + return instantiatedResources.get(0); } - if (resources.isEmpty()) { - return factoryInvoker.apply(defaultProvider.get()) - .orElseThrow(() -> new RuntimeException("Could not instantiate instance using default factory.")); + if (instantiatedResources.isEmpty()) { + try { + return factoryInvoker.invoke(defaultProvider.get()); + } catch (Exception e) { + final RuntimeException exception = new RuntimeException("Could not instantiate any instance."); + final RuntimeException defaultException = new RuntimeException("Could not instantiate default instance.", e); + exception.addSuppressed(defaultException); + errorsDuringInitialization.forEach(exception::addSuppressed); + throw exception; + } } - throw new RuntimeException("Multiple instances were created: " + resources); + throw new RuntimeException("Multiple instances were created: " + instantiatedResources); + } + + /** + * Interface for invoking the factory. + */ + public interface FactoryInvoker { + R invoke(F factory) throws Exception; } } diff --git a/flink-end-to-end-tests/flink-metrics-availability-test/src/test/java/org/pache/flink/metrics/tests/MetricsAvailabilityITCase.java b/flink-end-to-end-tests/flink-metrics-availability-test/src/test/java/org/pache/flink/metrics/tests/MetricsAvailabilityITCase.java index 68c94981f2..4d3179596e 100644 --- a/flink-end-to-end-tests/flink-metrics-availability-test/src/test/java/org/pache/flink/metrics/tests/MetricsAvailabilityITCase.java +++ b/flink-end-to-end-tests/flink-metrics-availability-test/src/test/java/org/pache/flink/metrics/tests/MetricsAvailabilityITCase.java @@ -76,8 +76,7 @@ public class MetricsAvailabilityITCase extends TestLogger { @Rule public final FlinkResource dist = new LocalStandaloneFlinkResourceFactory() - .create(FlinkResourceSetup.builder().build()) - .get(); + .create(FlinkResourceSetup.builder().build()); @Nullable private static ScheduledExecutorService scheduledExecutorService = null; diff --git a/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java index 5bde088b06..c4207e013b 100644 --- a/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java +++ b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java @@ -152,7 +152,7 @@ public class PrometheusReporterEndToEndITCase extends TestLogger { final FlinkResourceSetup.FlinkResourceSetupBuilder builder = FlinkResourceSetup.builder(); params.getBuilderSetup().accept(builder); builder.addConfiguration(getFlinkConfig(params.getInstantiationType())); - dist = new LocalStandaloneFlinkResourceFactory().create(builder.build()).get(); + dist = new LocalStandaloneFlinkResourceFactory().create(builder.build()); } @Rule -- Gitee From 3bef3a7f459f11fcd78658c0bd42aaf8bfe4386b Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 26 Mar 2020 13:02:13 +0100 Subject: [PATCH 397/885] [FLINK-16808][e2e] Consolidated logging --- .../util/kafka/LocalStandaloneKafkaResourceFactory.java | 5 ----- .../apache/flink/tests/util/cache/LolCacheFactory.java | 4 ---- .../tests/util/cache/PersistingDownloadCacheFactory.java | 6 ------ .../tests/util/cache/TravisDownloadCacheFactory.java | 8 -------- .../util/flink/LocalStandaloneFlinkResourceFactory.java | 2 -- .../org/apache/flink/tests/util/util/FactoryUtils.java | 7 +++++++ 6 files changed, 7 insertions(+), 25 deletions(-) diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/LocalStandaloneKafkaResourceFactory.java b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/LocalStandaloneKafkaResourceFactory.java index 2e274434f3..74ef5a5963 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/LocalStandaloneKafkaResourceFactory.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/LocalStandaloneKafkaResourceFactory.java @@ -18,18 +18,13 @@ package org.apache.flink.tests.util.kafka; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - /** * A {@link KafkaResourceFactory} for the {@link LocalStandaloneKafkaResourceFactory}. */ public final class LocalStandaloneKafkaResourceFactory implements KafkaResourceFactory { - private static final Logger LOG = LoggerFactory.getLogger(LocalStandaloneKafkaResourceFactory.class); @Override public KafkaResource create(final String kafkaVersion) { - LOG.info("Created {}.", LocalStandaloneKafkaResource.class.getSimpleName()); return new LocalStandaloneKafkaResource(kafkaVersion); } } diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/LolCacheFactory.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/LolCacheFactory.java index b0be5ce8fd..e8acefdfcc 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/LolCacheFactory.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/LolCacheFactory.java @@ -19,8 +19,6 @@ package org.apache.flink.tests.util.cache; import org.junit.rules.TemporaryFolder; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.IOException; @@ -28,13 +26,11 @@ import java.io.IOException; * A {@link DownloadCacheFactory} for the {@link LolCache}. */ public final class LolCacheFactory implements DownloadCacheFactory { - private static final Logger LOG = LoggerFactory.getLogger(LolCacheFactory.class); @Override public DownloadCache create() throws IOException { final TemporaryFolder folder = new TemporaryFolder(); folder.create(); - LOG.info("Created {}.", LolCache.class.getSimpleName()); return new LolCache(folder); } } diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/PersistingDownloadCacheFactory.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/PersistingDownloadCacheFactory.java index bbc8ab3b8b..824d52dec4 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/PersistingDownloadCacheFactory.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/PersistingDownloadCacheFactory.java @@ -20,9 +20,6 @@ package org.apache.flink.tests.util.cache; import org.apache.flink.tests.util.parameters.ParameterProperty; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.nio.file.Path; import java.nio.file.Paths; import java.time.Period; @@ -32,7 +29,6 @@ import java.util.Optional; * A {@link DownloadCacheFactory} for the {@link PersistingDownloadCache}. */ public final class PersistingDownloadCacheFactory implements DownloadCacheFactory { - private static final Logger LOG = LoggerFactory.getLogger(PersistingDownloadCacheFactory.class); private static final ParameterProperty TMP_DIR = new ParameterProperty<>("cache-dir", value -> Paths.get(value)); private static final ParameterProperty TIME_TO_LIVE = new ParameterProperty<>("cache-ttl", Period::parse); @@ -44,10 +40,8 @@ public final class PersistingDownloadCacheFactory implements DownloadCacheFactor final Optional tmpDir = TMP_DIR.get(); final Period timeToLive = TIME_TO_LIVE.get(TIME_TO_LIVE_DEFAULT); if (!tmpDir.isPresent()) { - LOG.debug("Not loading {} because {} was not set.", PersistingDownloadCache.class, TMP_DIR.getPropertyName()); throw new IllegalArgumentException(String.format("Not loading %s because %s was not set.", PersistingDownloadCache.class, TMP_DIR.getPropertyName())); } - LOG.info("Created {}.", PersistingDownloadCache.class.getSimpleName()); return new PersistingDownloadCache(tmpDir.get(), timeToLive); } } diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/TravisDownloadCacheFactory.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/TravisDownloadCacheFactory.java index 90520c536c..e7552435dd 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/TravisDownloadCacheFactory.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/TravisDownloadCacheFactory.java @@ -20,9 +20,6 @@ package org.apache.flink.tests.util.cache; import org.apache.flink.tests.util.parameters.ParameterProperty; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.nio.file.Path; import java.nio.file.Paths; import java.util.Optional; @@ -31,7 +28,6 @@ import java.util.Optional; * A {@link DownloadCacheFactory} for the {@link TravisDownloadCache}. */ public final class TravisDownloadCacheFactory implements DownloadCacheFactory { - private static final Logger LOG = LoggerFactory.getLogger(TravisDownloadCacheFactory.class); private static final ParameterProperty TMP_DIR = new ParameterProperty<>("cache-dir", value -> Paths.get(value)); private static final ParameterProperty BUILDS_TO_LIVE = new ParameterProperty<>("cache-btl", Integer::parseInt); @@ -43,18 +39,14 @@ public final class TravisDownloadCacheFactory implements DownloadCacheFactory { final Optional timeToLive = BUILDS_TO_LIVE.get(); final Optional buildNumber = BUILD_NUMBER.get(); if (!tmpDir.isPresent()) { - LOG.debug("Not loading {} because {} was not set.", TravisDownloadCache.class, TMP_DIR.getPropertyName()); throw new IllegalArgumentException(String.format("Not loading %s because %s was not set.", TravisDownloadCache.class, TMP_DIR.getPropertyName())); } if (!timeToLive.isPresent()) { - LOG.debug("Not loading {} because {} was not set.", TravisDownloadCache.class, BUILDS_TO_LIVE.getPropertyName()); throw new IllegalArgumentException(String.format("Not loading %s because %s was not set.", TravisDownloadCache.class, BUILDS_TO_LIVE.getPropertyName())); } if (!buildNumber.isPresent()) { - LOG.debug("Not loading {} because {} was not set.", TravisDownloadCache.class, BUILD_NUMBER.getPropertyName()); throw new IllegalArgumentException(String.format("Not loading %s because %s was not set.", TravisDownloadCache.class, BUILD_NUMBER.getPropertyName())); } - LOG.info("Created {}.", TravisDownloadCache.class.getSimpleName()); return new TravisDownloadCache(tmpDir.get(), timeToLive.get(), buildNumber.get()); } } diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResourceFactory.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResourceFactory.java index 2edf693e39..047a57de8f 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResourceFactory.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResourceFactory.java @@ -40,14 +40,12 @@ public final class LocalStandaloneFlinkResourceFactory implements FlinkResourceF public FlinkResource create(FlinkResourceSetup setup) { Optional distributionDirectory = DISTRIBUTION_DIRECTORY.get(); if (!distributionDirectory.isPresent()) { - LOG.warn("The distDir property was not set. You can set it when running maven via -DdistDir= ."); throw new IllegalArgumentException("The distDir property was not set. You can set it when running maven via -DdistDir= ."); } Optional logBackupDirectory = DISTRIBUTION_LOG_BACKUP_DIRECTORY.get(); if (!logBackupDirectory.isPresent()) { LOG.warn("Property {} not set, logs will not be backed up in case of test failures.", DISTRIBUTION_LOG_BACKUP_DIRECTORY.getPropertyName()); } - LOG.info("Created {}.", LocalStandaloneFlinkResource.class.getSimpleName()); return new LocalStandaloneFlinkResource(distributionDirectory.get(), logBackupDirectory.orElse(null), setup); } } diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/util/FactoryUtils.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/util/FactoryUtils.java index 4100f259ca..89ed049377 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/util/FactoryUtils.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/util/FactoryUtils.java @@ -18,6 +18,9 @@ package org.apache.flink.tests.util.util; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.util.ArrayList; import java.util.List; import java.util.ServiceLoader; @@ -29,6 +32,8 @@ import java.util.function.Supplier; public enum FactoryUtils { ; + private static final Logger LOG = LoggerFactory.getLogger(FactoryUtils.class); + /** * Loads all factories for the given class using the {@link ServiceLoader} and attempts to create an instance. * @@ -49,7 +54,9 @@ public enum FactoryUtils { try { R resource = factoryInvoker.invoke(factory); instantiatedResources.add(resource); + LOG.info("Instantiated {}.", resource.getClass().getSimpleName()); } catch (Exception e) { + LOG.debug("Factory {} could not instantiate instance.", factory.getClass().getSimpleName(), e); errorsDuringInitialization.add(e); } } -- Gitee From 7c3b065e4af44e3ab8ebc78e5c5e689c664e98ed Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Fri, 27 Mar 2020 22:29:12 +0100 Subject: [PATCH 398/885] [FLINK-16837][build] Disable trimStackTrace in surefire-plugin --- pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/pom.xml b/pom.xml index 5c0cfb1dc4..23ddcb7bfd 100644 --- a/pom.xml +++ b/pom.xml @@ -1475,6 +1475,7 @@ under the License. ${test.groups} ${flink.forkCount} ${flink.reuseForks} + false 0${surefire.forkNumber} ${log4j.configuration} -- Gitee From f97b1b5feb2d6736e1d2e0a76942c01bad6514d7 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Wed, 1 Apr 2020 16:00:52 +0800 Subject: [PATCH 399/885] [FLINK-16877][table-runtime] SingleDirectoryWriter should not produce file when no input record This closes #11572 --- .../filesystem/SingleDirectoryWriter.java | 23 ++++++++++++++--- .../table/filesystem/PartitionWriterTest.java | 25 ++++++++++++------- 2 files changed, 36 insertions(+), 12 deletions(-) diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/SingleDirectoryWriter.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/SingleDirectoryWriter.java index 7408b2d83e..af00a61e1e 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/SingleDirectoryWriter.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/SingleDirectoryWriter.java @@ -21,6 +21,7 @@ package org.apache.flink.table.filesystem; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.io.OutputFormat; +import java.io.IOException; import java.util.LinkedHashMap; import static org.apache.flink.table.filesystem.PartitionPathUtils.generatePartitionPath; @@ -33,15 +34,25 @@ import static org.apache.flink.table.filesystem.PartitionPathUtils.generateParti @Internal public class SingleDirectoryWriter implements PartitionWriter { + private final Context context; + private final PartitionTempFileManager manager; private final PartitionComputer computer; - private final OutputFormat format; + private final LinkedHashMap staticPartitions; + + private OutputFormat format; public SingleDirectoryWriter( Context context, PartitionTempFileManager manager, PartitionComputer computer, - LinkedHashMap staticPartitions) throws Exception { + LinkedHashMap staticPartitions) { + this.context = context; + this.manager = manager; this.computer = computer; + this.staticPartitions = staticPartitions; + } + + private void createFormat() throws IOException { this.format = context.createNewOutputFormat(staticPartitions.size() == 0 ? manager.createPartitionDir() : manager.createPartitionDir(generatePartitionPath(staticPartitions))); @@ -49,11 +60,17 @@ public class SingleDirectoryWriter implements PartitionWriter { @Override public void write(T in) throws Exception { + if (format == null) { + createFormat(); + } format.writeRecord(computer.projectColumnsToWrite(in)); } @Override public void close() throws Exception { - format.close(); + if (format != null) { + format.close(); + format = null; + } } } diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/filesystem/PartitionWriterTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/filesystem/PartitionWriterTest.java index 9ec213bc2a..853e0d145d 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/filesystem/PartitionWriterTest.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/filesystem/PartitionWriterTest.java @@ -54,20 +54,19 @@ public class PartitionWriterTest { @Override public void open(int taskNumber, int numTasks) { + records.put(getKey(), new ArrayList<>()); } - @Override - public void writeRecord(Row record) { + private String getKey() { Path parent = path.getParent(); - String key = parent.getName().startsWith("task-") ? + return parent.getName().startsWith("task-") ? parent.getParent().getName() : parent.getParent().getParent().getName() + Path.SEPARATOR + parent.getName(); + } - records.compute(key, (path1, rows) -> { - rows = rows == null ? new ArrayList<>() : rows; - rows.add(record); - return rows; - }); + @Override + public void writeRecord(Row record) { + records.get(getKey()).add(record); } @Override @@ -105,7 +104,15 @@ public class PartitionWriterTest { } @Test - public void testNonPartitionWriter() throws Exception { + public void testEmptySingleDirectoryWriter() throws Exception { + SingleDirectoryWriter writer = new SingleDirectoryWriter<>( + context, manager, computer, new LinkedHashMap<>()); + writer.close(); + Assert.assertTrue(records.isEmpty()); + } + + @Test + public void testSingleDirectoryWriter() throws Exception { SingleDirectoryWriter writer = new SingleDirectoryWriter<>( context, manager, computer, new LinkedHashMap<>()); -- Gitee From da734c371b2b3ca62b6c091e8bda5986e13dc33d Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 27 Mar 2020 17:31:33 +0100 Subject: [PATCH 400/885] [FLINK-16834] Add flink-clients dependency to all example modules This commit adds a flink-clients dependency to all example modules. That way they become executable from the IDE again. --- flink-examples/flink-examples-streaming/pom.xml | 6 ++++++ flink-examples/flink-examples-table/pom.xml | 6 ++++++ 2 files changed, 12 insertions(+) diff --git a/flink-examples/flink-examples-streaming/pom.xml b/flink-examples/flink-examples-streaming/pom.xml index bd6b97265e..63b633b36f 100644 --- a/flink-examples/flink-examples-streaming/pom.xml +++ b/flink-examples/flink-examples-streaming/pom.xml @@ -50,6 +50,12 @@ under the License. ${project.version} + + org.apache.flink + flink-clients_${scala.binary.version} + ${project.version} + + org.apache.flink flink-connector-twitter_${scala.binary.version} diff --git a/flink-examples/flink-examples-table/pom.xml b/flink-examples/flink-examples-table/pom.xml index a1ce03fe2e..582900e303 100644 --- a/flink-examples/flink-examples-table/pom.xml +++ b/flink-examples/flink-examples-table/pom.xml @@ -62,6 +62,12 @@ under the License. flink-streaming-scala_${scala.binary.version} ${project.version} + + + org.apache.flink + flink-clients_${scala.binary.version} + ${project.version} + -- Gitee From 390704e6760b99eb87d796ab536c122cbc5d0c3e Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 27 Mar 2020 17:34:48 +0100 Subject: [PATCH 401/885] [hotfix] Add log4j2 and logback properties file to flink-examples-table --- .../src/main/resources/log4j2.properties | 25 ++++++++++++++++ .../src/main/resources/logback.xml | 29 +++++++++++++++++++ 2 files changed, 54 insertions(+) create mode 100644 flink-examples/flink-examples-table/src/main/resources/log4j2.properties create mode 100644 flink-examples/flink-examples-table/src/main/resources/logback.xml diff --git a/flink-examples/flink-examples-table/src/main/resources/log4j2.properties b/flink-examples/flink-examples-table/src/main/resources/log4j2.properties new file mode 100644 index 0000000000..9206863eda --- /dev/null +++ b/flink-examples/flink-examples-table/src/main/resources/log4j2.properties @@ -0,0 +1,25 @@ +################################################################################ +# 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. +################################################################################ + +rootLogger.level = INFO +rootLogger.appenderRef.console.ref = ConsoleAppender + +appender.console.name = ConsoleAppender +appender.console.type = CONSOLE +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %-60c %x - %m%n diff --git a/flink-examples/flink-examples-table/src/main/resources/logback.xml b/flink-examples/flink-examples-table/src/main/resources/logback.xml new file mode 100644 index 0000000000..95f2d04193 --- /dev/null +++ b/flink-examples/flink-examples-table/src/main/resources/logback.xml @@ -0,0 +1,29 @@ + + + + + + %d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n + + + + + + + \ No newline at end of file -- Gitee From 3ce6ecd0e776f16e98beeda5aed0ea52de223645 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 31 Mar 2020 14:54:05 +0200 Subject: [PATCH 402/885] [FLINK-16834] Add flink-clients to flink-quickstarts poms Flink quickstarts' poms need to include flink-clients as a dependency because it is no longer transitively pulled into the project. --- .../src/main/resources/archetype-resources/pom.xml | 6 ++++++ .../src/main/resources/archetype-resources/pom.xml | 6 ++++++ 2 files changed, 12 insertions(+) diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml index 206d5bd7d4..c051ced4fd 100644 --- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml +++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml @@ -66,6 +66,12 @@ under the License. ${flink.version} provided + + org.apache.flink + flink-clients_${scala.binary.version} + ${flink.version} + provided + diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml index 478a41898d..f17f66313b 100644 --- a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml +++ b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml @@ -65,6 +65,12 @@ under the License. ${flink.version} provided + + org.apache.flink + flink-clients_${scala.binary.version} + ${flink.version} + provided + -- Gitee From da46ab619ecf663ce0d5e0e28d127cd53a41bc00 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 31 Mar 2020 15:11:26 +0200 Subject: [PATCH 403/885] [FLINK-16834] Add flink-clients dependency to flink-walkthroughs With inverting the dependency between flink-clients and flink-streaming-java, it is now necessary to explicitly add the flink-clients dependency in order to execute the walkthrough jobs from within the IDE. This closes #11551. --- .../src/main/resources/archetype-resources/pom.xml | 6 ++++++ .../src/main/resources/archetype-resources/pom.xml | 6 ++++++ .../src/main/resources/archetype-resources/pom.xml | 6 ++++++ .../src/main/resources/archetype-resources/pom.xml | 6 ++++++ 4 files changed, 24 insertions(+) diff --git a/flink-walkthroughs/flink-walkthrough-datastream-java/src/main/resources/archetype-resources/pom.xml b/flink-walkthroughs/flink-walkthrough-datastream-java/src/main/resources/archetype-resources/pom.xml index b73f8a39a6..f55cb1ca62 100644 --- a/flink-walkthroughs/flink-walkthrough-datastream-java/src/main/resources/archetype-resources/pom.xml +++ b/flink-walkthroughs/flink-walkthrough-datastream-java/src/main/resources/archetype-resources/pom.xml @@ -66,6 +66,12 @@ under the License. ${flink.version} provided + + org.apache.flink + flink-clients_${scala.binary.version} + ${flink.version} + provided + diff --git a/flink-walkthroughs/flink-walkthrough-datastream-scala/src/main/resources/archetype-resources/pom.xml b/flink-walkthroughs/flink-walkthrough-datastream-scala/src/main/resources/archetype-resources/pom.xml index 385b10c6aa..0af6b1b017 100644 --- a/flink-walkthroughs/flink-walkthrough-datastream-scala/src/main/resources/archetype-resources/pom.xml +++ b/flink-walkthroughs/flink-walkthrough-datastream-scala/src/main/resources/archetype-resources/pom.xml @@ -66,6 +66,12 @@ under the License. ${flink.version} provided + + org.apache.flink + flink-clients_${scala.binary.version} + ${flink.version} + provided + diff --git a/flink-walkthroughs/flink-walkthrough-table-java/src/main/resources/archetype-resources/pom.xml b/flink-walkthroughs/flink-walkthrough-table-java/src/main/resources/archetype-resources/pom.xml index b5a220ce94..13567ef64b 100644 --- a/flink-walkthroughs/flink-walkthrough-table-java/src/main/resources/archetype-resources/pom.xml +++ b/flink-walkthroughs/flink-walkthrough-table-java/src/main/resources/archetype-resources/pom.xml @@ -72,6 +72,12 @@ under the License. ${flink.version} provided + + org.apache.flink + flink-clients_${scala.binary.version} + ${flink.version} + provided + diff --git a/flink-walkthroughs/flink-walkthrough-table-scala/src/main/resources/archetype-resources/pom.xml b/flink-walkthroughs/flink-walkthrough-table-scala/src/main/resources/archetype-resources/pom.xml index 0d26957f45..c3b9982803 100644 --- a/flink-walkthroughs/flink-walkthrough-table-scala/src/main/resources/archetype-resources/pom.xml +++ b/flink-walkthroughs/flink-walkthrough-table-scala/src/main/resources/archetype-resources/pom.xml @@ -72,6 +72,12 @@ under the License. ${flink.version} provided + + org.apache.flink + flink-clients_${scala.binary.version} + ${flink.version} + provided + -- Gitee From 41724c72f3f7e6aa4b7b9e64d7626eca7d367c1a Mon Sep 17 00:00:00 2001 From: Hequn Cheng Date: Wed, 1 Apr 2020 18:15:24 +0800 Subject: [PATCH 404/885] [FLINK-16674][python][docs] Add documentation about how to use user-defined metrics for Python UDF (#11576) --- docs/dev/table/python/metrics.md | 207 ++++++++++++++++++ docs/dev/table/python/metrics.zh.md | 207 ++++++++++++++++++ flink-python/pyflink/metrics/metricbase.py | 13 +- .../pyflink/metrics/tests/test_metric.py | 2 +- 4 files changed, 420 insertions(+), 9 deletions(-) create mode 100644 docs/dev/table/python/metrics.md create mode 100644 docs/dev/table/python/metrics.zh.md diff --git a/docs/dev/table/python/metrics.md b/docs/dev/table/python/metrics.md new file mode 100644 index 0000000000..df98b79bff --- /dev/null +++ b/docs/dev/table/python/metrics.md @@ -0,0 +1,207 @@ +--- +title: "Metrics" +nav-parent_id: python_tableapi +nav-pos: 110 +--- + + +PyFlink exposes a metric system that allows gathering and exposing metrics to external systems. + +* This will be replaced by the TOC +{:toc} + +## Registering metrics + +You can access the metric system from a [User-defined Function]({{ site.baseurl }}/dev/table/python/python_udfs.html) by calling `function_context.get_metric_group()` in the `open` method. +The `get_metric_group()` method returns a `MetricGroup` object on which you can create and register new metrics. + +### Metric types + +PyFlink supports `Counters`, `Gauges`, `Distribution` and `Meters`. + +#### Counter + +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(n: int)` or `dec()/dec(n: int)`. +You can create and register a `Counter` by calling `counter(name: str)` on a `MetricGroup`. + +

    +
    +{% highlight python %} +from pyflink.table.udf import ScalarFunction + +class MyUDF(ScalarFunction): + + def __init__(self): + self.counter = None + + def open(self, function_context): + self.counter = function_context.get_metric_group().counter("my_counter") + + def eval(self, i): + self.counter.inc(i) + return i + +{% endhighlight %} +
    + +
    + +#### Gauge + +A `Gauge` provides a value on demand. You can register a gauge by calling `gauge(name: str, obj: Callable[[], int])` on a MetricGroup. The Callable object will be used to report the values. Gauge metrics are restricted to integer-only values. + +
    +
    +{% highlight python %} +from pyflink.table.udf import ScalarFunction + +class MyUDF(ScalarFunction): + + def __init__(self): + self.length = 0 + + def open(self, function_context): + function_context.get_metric_group().gauge("my_gauge", lambda : self.length) + + def eval(self, i): + self.length = i + return i - 1 +{% endhighlight %} +
    + +
    + +#### Distribution + +A metric that reports information(sum, count, min, max and mean) about the distribution of reported values. The value can be updated using `update(n: int)`. You can register a distribution by calling `distribution(name: str)` on a MetricGroup. Distribution metrics are restricted to integer-only distributions. + +
    +
    +{% highlight python %} +from pyflink.table.udf import ScalarFunction + +class MyUDF(ScalarFunction): + + def __init__(self): + self.distribution = None + + def open(self, function_context): + self.distribution = function_context.get_metric_group().distribution("my_distribution") + + def eval(self, i): + self.distribution.update(i) + return i - 1 +{% endhighlight %} +
    + +
    + +#### Meter + +A Meter measures an average throughput. An occurrence of an event can be registered with the `mark_event()` method. The occurrence of multiple events at the same time can be registered with mark_event(n: int) method. You can register a meter by calling `meter(self, name: str, time_span_in_seconds: int = 60)` on a MetricGroup. The default value of time_span_in_seconds is 60. + +
    +
    +{% highlight python %} +from pyflink.table.udf import ScalarFunction + +class MyUDF(ScalarFunction): + + def __init__(self): + self.meter = None + + def open(self, function_context): + super().open(function_context) + # an average rate of events per second over 120s, default is 60s. + self.meter = function_context.get_metric_group().meter("my_meter", time_span_in_seconds=120) + + def eval(self, i): + self.meter.mark_event(i) + return i - 1 +{% endhighlight %} +
    + +
    + +## Scope + +You can refer to the Java metric document for more details on [Scope definition]({{ site.baseurl }}/monitoring/metrics.html#Scope). + +### User Scope + +You can define a user scope by calling `MetricGroup.add_group(key: str, value: str = None)`. If extra is not None, creates a new key-value MetricGroup pair. The key group is added to this group's sub-groups, while the value group is added to the key group's sub-groups. In this case, the value group will be returned and a user variable will be defined. + +
    +
    +{% highlight python %} + +function_context + .get_metric_group() + .add_group("my_metrics") + .counter("my_counter") + +function_context + .get_metric_group() + .add_group("my_metrics_key", "my_metrics_value") + .counter("my_counter") + +{% endhighlight %} +
    + +
    + +### System Scope + +You can refer to the Java metric document for more details on [System Scope]({{ site.baseurl }}/monitoring/metrics.html#system-scope). + +### List of all Variables + +You can refer to the Java metric document for more details on [List of all Variables]({{ site.baseurl }}/monitoring/metrics.html#list-of-all-variables). + +### User Variables + +You can define a user variable by calling `MetricGroup.addGroup(key: str, value: str = None)` and specifying the value parameter. + +**Important:** User variables cannot be used in scope formats. + +
    +
    +{% highlight python %} +function_context + .get_metric_group() + .add_group("my_metrics_key", "my_metrics_value") + .counter("my_counter") +{% endhighlight %} +
    + +
    + +## Common part between PyFlink and Flink + +You can refer to the Java metric document for more details on the following sections: + +- [Reporter]({{ site.baseurl }}/monitoring/metrics.html#reporter). +- [System metrics]({{ site.baseurl }}/monitoring/metrics.html#system-metrics). +- [Latency tracking]({{ site.baseurl }}/monitoring/metrics.html#latency-tracking). +- [REST API integration]({{ site.baseurl }}/monitoring/metrics.html#rest-api-integration). +- [Dashboard integration]({{ site.baseurl }}/monitoring/metrics.html#dashboard-integration). + + +{% top %} diff --git a/docs/dev/table/python/metrics.zh.md b/docs/dev/table/python/metrics.zh.md new file mode 100644 index 0000000000..0f6fc13806 --- /dev/null +++ b/docs/dev/table/python/metrics.zh.md @@ -0,0 +1,207 @@ +--- +title: "指标" +nav-parent_id: python_tableapi +nav-pos: 110 +--- + + +PyFlink exposes a metric system that allows gathering and exposing metrics to external systems. + +* This will be replaced by the TOC +{:toc} + +## Registering metrics + +You can access the metric system from a [User-defined Function]({{ site.baseurl }}/zh/dev/table/python/python_udfs.html) by calling `function_context.get_metric_group()` in the `open` method. +The `get_metric_group()` method returns a `MetricGroup` object on which you can create and register new metrics. + +### Metric types + +PyFlink supports `Counters`, `Gauges`, `Distribution` and `Meters`. + +#### Counter + +A `Counter` is used to count something. The current value can be in- or decremented using `inc()/inc(n: int)` or `dec()/dec(n: int)`. +You can create and register a `Counter` by calling `counter(name: str)` on a `MetricGroup`. + +
    +
    +{% highlight python %} +from pyflink.table.udf import ScalarFunction + +class MyUDF(ScalarFunction): + + def __init__(self): + self.counter = None + + def open(self, function_context): + self.counter = function_context.get_metric_group().counter("my_counter") + + def eval(self, i): + self.counter.inc(i) + return i + +{% endhighlight %} +
    + +
    + +#### Gauge + +A `Gauge` provides a value on demand. You can register a gauge by calling `gauge(name: str, obj: Callable[[], int])` on a MetricGroup. The Callable object will be used to report the values. Gauge metrics are restricted to integer-only values. + +
    +
    +{% highlight python %} +from pyflink.table.udf import ScalarFunction + +class MyUDF(ScalarFunction): + + def __init__(self): + self.length = 0 + + def open(self, function_context): + function_context.get_metric_group().gauge("my_gauge", lambda : self.length) + + def eval(self, i): + self.length = i + return i - 1 +{% endhighlight %} +
    + +
    + +#### Distribution + +A metric that reports information(sum, count, min, max and mean) about the distribution of reported values. The value can be updated using `update(n: int)`. You can register a distribution by calling `distribution(name: str)` on a MetricGroup. Distribution metrics are restricted to integer-only distributions. + +
    +
    +{% highlight python %} +from pyflink.table.udf import ScalarFunction + +class MyUDF(ScalarFunction): + + def __init__(self): + self.distribution = None + + def open(self, function_context): + self.distribution = function_context.get_metric_group().distribution("my_distribution") + + def eval(self, i): + self.distribution.update(i) + return i - 1 +{% endhighlight %} +
    + +
    + +#### Meter + +A Meter measures an average throughput. An occurrence of an event can be registered with the `mark_event()` method. The occurrence of multiple events at the same time can be registered with mark_event(n: int) method. You can register a meter by calling `meter(self, name: str, time_span_in_seconds: int = 60)` on a MetricGroup. The default value of time_span_in_seconds is 60. + +
    +
    +{% highlight python %} +from pyflink.table.udf import ScalarFunction + +class MyUDF(ScalarFunction): + + def __init__(self): + self.meter = None + + def open(self, function_context): + super().open(function_context) + # an average rate of events per second over 120s, default is 60s. + self.meter = function_context.get_metric_group().meter("my_meter", time_span_in_seconds=120) + + def eval(self, i): + self.meter.mark_event(i) + return i - 1 +{% endhighlight %} +
    + +
    + +## Scope + +You can refer to the Java metric document for more details on [Scope definition]({{ site.baseurl }}/zh/monitoring/metrics.html#Scope). + +### User Scope + +You can define a user scope by calling `MetricGroup.add_group(key: str, value: str = None)`. If extra is not None, creates a new key-value MetricGroup pair. The key group is added to this group's sub-groups, while the value group is added to the key group's sub-groups. In this case, the value group will be returned and a user variable will be defined. + +
    +
    +{% highlight python %} + +function_context + .get_metric_group() + .add_group("my_metrics") + .counter("my_counter") + +function_context + .get_metric_group() + .add_group("my_metrics_key", "my_metrics_value") + .counter("my_counter") + +{% endhighlight %} +
    + +
    + +### System Scope + +You can refer to the Java metric document for more details on [System Scope]({{ site.baseurl }}/zh/monitoring/metrics.html#system-scope). + +### List of all Variables + +You can refer to the Java metric document for more details on [List of all Variables]({{ site.baseurl }}/zh/monitoring/metrics.html#list-of-all-variables). + +### User Variables + +You can define a user variable by calling `MetricGroup.addGroup(key: str, value: str = None)` and specifying the value parameter. + +**Important:** User variables cannot be used in scope formats. + +
    +
    +{% highlight python %} +function_context + .get_metric_group() + .add_group("my_metrics_key", "my_metrics_value") + .counter("my_counter") +{% endhighlight %} +
    + +
    + +## Common part between PyFlink and Flink + +You can refer to the Java metric document for more details on the following sections: + +- [Reporter]({{ site.baseurl }}/zh/monitoring/metrics.html#reporter). +- [System metrics]({{ site.baseurl }}/zh/monitoring/metrics.html#system-metrics). +- [Latency tracking]({{ site.baseurl }}/zh/monitoring/metrics.html#latency-tracking). +- [REST API integration]({{ site.baseurl }}/zh/monitoring/metrics.html#rest-api-integration). +- [Dashboard integration]({{ site.baseurl }}/zh/monitoring/metrics.html#dashboard-integration). + + +{% top %} diff --git a/flink-python/pyflink/metrics/metricbase.py b/flink-python/pyflink/metrics/metricbase.py index acbd6a993b..17966a392b 100644 --- a/flink-python/pyflink/metrics/metricbase.py +++ b/flink-python/pyflink/metrics/metricbase.py @@ -27,13 +27,10 @@ class MetricGroup(abc.ABC): """ Creates a new MetricGroup and adds it to this groups sub-groups. - If extra is not None, creates a new key-value MetricGroup pair. The key group - is added to this groups sub-groups, while the value group is added to the key - group's sub-groups. This method returns the value group. - - The only difference between calling this method and - `group.add_group(key).add_group(value)` is that get_all_variables() - of the value group return an additional `""="value"` pair. + If extra is not None, creates a new key-value MetricGroup pair. + The key group is added to this group's sub-groups, while the value + group is added to the key group's sub-groups. In this case, + the value group will be returned and a user variable will be defined. """ pass @@ -186,7 +183,7 @@ class Meter(Metric): def __init__(self, inner_counter): self._inner_counter = inner_counter - def make_event(self, value=1): + def mark_event(self, value=1): self._inner_counter.inc(value) def get_count(self): diff --git a/flink-python/pyflink/metrics/tests/test_metric.py b/flink-python/pyflink/metrics/tests/test_metric.py index f43ddbd029..ee31099747 100644 --- a/flink-python/pyflink/metrics/tests/test_metric.py +++ b/flink-python/pyflink/metrics/tests/test_metric.py @@ -83,7 +83,7 @@ class MetricTests(unittest.TestCase): MetricName( '[]', 'my_distribution')).get_cumulative()) counter.inc(-2) - meter.make_event(3) + meter.mark_event(3) distribution.update(10) distribution.update(2) self.assertEqual(-2, counter.get_count()) -- Gitee From eb12141c25498610a2326dac6409a16246a9d5e3 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 1 Apr 2020 13:01:47 +0200 Subject: [PATCH 405/885] [FLINK-14311] Relax restart checker to harden StreamingFileSink e2e test Before, we were checking for an exact number of restarts, this is too strict, because we could have more than the expected number of restarts. Now we check whether we have gte the number of expected restarts. --- flink-end-to-end-tests/test-scripts/common.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-end-to-end-tests/test-scripts/common.sh b/flink-end-to-end-tests/test-scripts/common.sh index dc4fc7bf18..44ce92f40f 100644 --- a/flink-end-to-end-tests/test-scripts/common.sh +++ b/flink-end-to-end-tests/test-scripts/common.sh @@ -731,7 +731,8 @@ function wait_for_restart_to_complete { local expected_num_restarts=$((current_num_restarts + 1)) echo "Waiting for restart to happen" - while ! [[ ${current_num_restarts} -eq ${expected_num_restarts} ]]; do + while [[ ${current_num_restarts} -lt ${expected_num_restarts} ]]; do + echo "Still waiting for restarts. Expected: $expected_num_restarts Current: $current_num_restarts" sleep 5 current_num_restarts=$(get_job_metric ${jobid} "fullRestarts") if [[ -z ${current_num_restarts} ]]; then -- Gitee From 4412c6ef4fbb208cba6e44e7a90fdfe1e07bf727 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 1 Apr 2020 13:36:24 +0200 Subject: [PATCH 406/885] [minor] Add debug logging if aws cli command fails The motivation is to help debugging FLINK-15772. --- flink-end-to-end-tests/test-scripts/common_s3_operations.sh | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/flink-end-to-end-tests/test-scripts/common_s3_operations.sh b/flink-end-to-end-tests/test-scripts/common_s3_operations.sh index ad4f996437..eae745b683 100644 --- a/flink-end-to-end-tests/test-scripts/common_s3_operations.sh +++ b/flink-end-to-end-tests/test-scripts/common_s3_operations.sh @@ -71,7 +71,9 @@ function aws_cli() { if [[ $S3_ENDPOINT ]]; then endpoint="--endpoint-url $S3_ENDPOINT" fi - docker exec "$AWSCLI_CONTAINER_ID" aws $endpoint "$@" + if ! docker exec "$AWSCLI_CONTAINER_ID" aws $endpoint "$@"; then + echo "Error executing aws command: $@"; + fi } ################################### -- Gitee From 612e1e9ff5ae022817a46dcb46243ec34b5b3882 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Thu, 26 Mar 2020 10:08:39 +0100 Subject: [PATCH 407/885] [FLINK-16035] Updated Stream/BatchTableEnvironment.java to use Java's Expression DSL This PR introduces alternatives for methods that are using the String based expressions in TableEnvironments. This PR also removes all usages of the String based expressions methods. It is a preparation for deprecating and in result removing those methods in the future. --- .../cassandra/CassandraConnectorITCase.java | 2 +- .../addons/hbase/HBaseConnectorITCase.java | 11 ++- .../io/jdbc/JDBCLookupFunctionITCase.java | 4 +- .../io/jdbc/JDBCUpsertTableSinkITCase.java | 14 +-- .../table/examples/java/StreamSQLExample.java | 6 +- .../examples/scala/StreamSQLExample.scala | 2 +- .../utils/DataStreamConversionUtil.java | 13 +-- .../PythonScalarFunctionOperatorTestBase.java | 4 +- .../table/api/java/BatchTableEnvironment.java | 47 +++++++++- .../api/java/StreamTableEnvironment.java | 47 +++++++++- .../internal/StreamTableEnvironmentImpl.java | 16 +++- .../JavaDataStreamQueryOperation.java | 3 +- .../ScalaDataStreamQueryOperation.java | 2 +- .../table/api/TableUtilsStreamingITCase.java | 3 +- .../table/api/TableEnvironmentTest.scala | 2 +- .../utils/ExpressionTestBase.scala | 2 +- .../match/PatternTranslatorTestBase.scala | 2 +- .../CorrelateStringExpressionTest.scala | 5 +- .../stream/sql/AggregateRemoveITCase.scala | 6 +- .../stream/sql/PruneAggregateCallITCase.scala | 4 +- .../runtime/utils/BatchTableEnvUtil.scala | 2 +- .../runtime/utils/StreamTableEnvUtil.scala | 4 +- .../table/planner/utils/TableTestBase.scala | 4 +- .../internal/BatchTableEnvironmentImpl.scala | 17 +++- .../runtime/batch/sql/GroupingSetsITCase.java | 4 +- .../runtime/batch/sql/JavaSqlITCase.java | 14 +-- .../table/JavaTableEnvironmentITCase.java | 92 +++++++++---------- .../runtime/stream/sql/JavaSqlITCase.java | 12 ++- .../api/batch/sql/SetOperatorsTest.scala | 4 +- .../api/batch/table/SetOperatorsTest.scala | 8 +- .../CorrelateStringExpressionTest.scala | 6 +- .../stream/StreamTableEnvironmentTest.scala | 24 +++-- .../table/api/stream/sql/UnionTest.scala | 6 +- .../api/stream/table/TableAggregateTest.scala | 4 +- .../CorrelateStringExpressionTest.scala | 6 +- .../TableEnvironmentValidationTest.scala | 6 +- .../utils/ExpressionTestBase.scala | 2 +- .../match/PatternTranslatorTestBase.scala | 2 +- .../runtime/batch/sql/AggregateITCase.scala | 22 ++--- .../table/runtime/batch/sql/CalcITCase.scala | 8 +- .../table/runtime/batch/sql/JoinITCase.scala | 8 +- .../batch/sql/SetOperatorsITCase.scala | 40 ++++---- .../table/runtime/batch/sql/SortITCase.scala | 8 +- .../batch/sql/TableEnvironmentITCase.scala | 2 +- .../runtime/batch/table/AggregateITCase.scala | 2 +- .../runtime/batch/table/CalcITCase.scala | 6 +- .../batch/table/TableEnvironmentITCase.scala | 6 +- .../runtime/stream/sql/InsertIntoITCase.scala | 16 ++-- .../table/runtime/stream/sql/SqlITCase.scala | 10 +- .../flink/table/utils/TableTestBase.scala | 17 ++-- 50 files changed, 356 insertions(+), 201 deletions(-) diff --git a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java index 5144945e74..28843f4d6d 100644 --- a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java +++ b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java @@ -461,7 +461,7 @@ public class CassandraConnectorITCase extends WriteAheadSinkTestBase source = env.fromCollection(rowCollection); - tEnv.registerDataStream("testFlinkTable", source); + tEnv.createTemporaryView("testFlinkTable", source); tEnv.registerTableSink( "cassandraTable", new CassandraAppendTableSink(builder, injectTableName(INSERT_DATA_QUERY)).configure( diff --git a/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/HBaseConnectorITCase.java b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/HBaseConnectorITCase.java index dfa40a0d9b..86621fa977 100644 --- a/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/HBaseConnectorITCase.java +++ b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/HBaseConnectorITCase.java @@ -66,6 +66,7 @@ import java.util.Map; import scala.Option; import static org.apache.flink.addons.hbase.util.PlannerType.OLD_PLANNER; +import static org.apache.flink.table.api.Expressions.$; import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_PROPERTY_VERSION; import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE; import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_VERSION; @@ -287,7 +288,7 @@ public class HBaseConnectorITCase extends HBaseTestBase { StreamTableEnvironment tEnv = StreamTableEnvironment.create(execEnv, streamSettings); DataStream ds = execEnv.fromCollection(testData1).returns(testTypeInfo1); - tEnv.registerDataStream("src", ds); + tEnv.createTemporaryView("src", ds); tEnv.registerTableSink("hbase", tableSink); String query = "INSERT INTO hbase SELECT ROW(f1c1), ROW(f2c1, f2c2), rowkey, ROW(f3c1, f3c2, f3c3) FROM src"; @@ -342,7 +343,7 @@ public class HBaseConnectorITCase extends HBaseTestBase { StreamTableEnvironment tEnv = StreamTableEnvironment.create(execEnv, streamSettings); DataStream ds = execEnv.fromCollection(testData1).returns(testTypeInfo1); - tEnv.registerDataStream("src", ds); + tEnv.createTemporaryView("src", ds); // register hbase table String quorum = getZookeeperQuorum(); @@ -428,7 +429,7 @@ public class HBaseConnectorITCase extends HBaseTestBase { // prepare a source table DataStream ds = streamEnv.fromCollection(testData2).returns(testTypeInfo2); - Table in = streamTableEnv.fromDataStream(ds, "a, b, c"); + Table in = streamTableEnv.fromDataStream(ds, $("a"), $("b"), $("c")); streamTableEnv.registerTable("src", in); Map tableProperties = hbaseTableProperties(); @@ -469,11 +470,11 @@ public class HBaseConnectorITCase extends HBaseTestBase { // prepare a source table String srcTableName = "src"; DataStream ds = streamEnv.fromCollection(testData2).returns(testTypeInfo2); - Table in = streamTableEnv.fromDataStream(ds, "a, b, c, proc.proctime"); + Table in = streamTableEnv.fromDataStream(ds, $("a"), $("b"), $("c"), $("proc").proctime()); streamTableEnv.registerTable(srcTableName, in); Map tableProperties = hbaseTableProperties(); - TableSource source = TableFactoryService + TableSource source = TableFactoryService .find(HBaseTableFactory.class, tableProperties) .createTableSource(tableProperties); streamTableEnv.registerTableSource("hbaseLookup", source); diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCLookupFunctionITCase.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCLookupFunctionITCase.java index dabd88cc43..7a2efaa879 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCLookupFunctionITCase.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCLookupFunctionITCase.java @@ -45,6 +45,8 @@ import java.util.Arrays; import java.util.Collection; import java.util.List; +import static org.apache.flink.table.api.Expressions.$; + /** * IT case for {@link JDBCLookupFunction}. */ @@ -142,7 +144,7 @@ public class JDBCLookupFunctionITCase extends AbstractTestBase { new Tuple2<>(2, 5), new Tuple2<>(3, 5), new Tuple2<>(3, 8) - )), "id1, id2"); + )), $("id1"), $("id2")); tEnv.registerTable("T", t); diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSinkITCase.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSinkITCase.java index ecba1977fc..f5ca948686 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSinkITCase.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSinkITCase.java @@ -44,6 +44,7 @@ import java.util.Collections; import java.util.List; import static org.apache.flink.api.java.io.jdbc.JdbcTableOutputFormatTest.check; +import static org.apache.flink.table.api.Expressions.$; /** * IT case for {@link JDBCUpsertTableSink}. @@ -153,11 +154,12 @@ public class JDBCUpsertTableSinkITCase extends AbstractTestBase { StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); Table t = tEnv.fromDataStream(get4TupleDataStream(env).assignTimestampsAndWatermarks( - new AscendingTimestampExtractor>() { - @Override - public long extractAscendingTimestamp(Tuple4 element) { - return element.f0; - }}), "id, num, text, ts"); + new AscendingTimestampExtractor>() { + @Override + public long extractAscendingTimestamp(Tuple4 element) { + return element.f0; + } + }), $("id"), $("num"), $("text"), $("ts")); tEnv.createTemporaryView("T", t); tEnv.sqlUpdate( @@ -195,7 +197,7 @@ public class JDBCUpsertTableSinkITCase extends AbstractTestBase { env.getConfig().setParallelism(1); StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); - Table t = tEnv.fromDataStream(get4TupleDataStream(env), "id, num, text, ts"); + Table t = tEnv.fromDataStream(get4TupleDataStream(env), $("id"), $("num"), $("text"), $("ts")); tEnv.registerTable("T", t); diff --git a/flink-examples/flink-examples-table/src/main/java/org/apache/flink/table/examples/java/StreamSQLExample.java b/flink-examples/flink-examples-table/src/main/java/org/apache/flink/table/examples/java/StreamSQLExample.java index d450f5b434..a4dbbf076e 100644 --- a/flink-examples/flink-examples-table/src/main/java/org/apache/flink/table/examples/java/StreamSQLExample.java +++ b/flink-examples/flink-examples-table/src/main/java/org/apache/flink/table/examples/java/StreamSQLExample.java @@ -28,6 +28,8 @@ import org.apache.flink.table.api.java.StreamTableEnvironment; import java.util.Arrays; import java.util.Objects; +import static org.apache.flink.table.api.Expressions.$; + /** * Simple example for demonstrating the use of SQL on a Stream Table in Java. * @@ -79,9 +81,9 @@ public class StreamSQLExample { new Order(4L, "beer", 1))); // convert DataStream to Table - Table tableA = tEnv.fromDataStream(orderA, "user, product, amount"); + Table tableA = tEnv.fromDataStream(orderA, $("user"), $("product"), $("amount")); // register DataStream as Table - tEnv.registerDataStream("OrderB", orderB, "user, product, amount"); + tEnv.createTemporaryView("OrderB", orderB, $("user"), $("product"), $("amount")); // union the two tables Table result = tEnv.sqlQuery("SELECT * FROM " + tableA + " WHERE amount > 2 UNION ALL " + diff --git a/flink-examples/flink-examples-table/src/main/scala/org/apache/flink/table/examples/scala/StreamSQLExample.scala b/flink-examples/flink-examples-table/src/main/scala/org/apache/flink/table/examples/scala/StreamSQLExample.scala index 79fa5e8179..5ddc95de79 100644 --- a/flink-examples/flink-examples-table/src/main/scala/org/apache/flink/table/examples/scala/StreamSQLExample.scala +++ b/flink-examples/flink-examples-table/src/main/scala/org/apache/flink/table/examples/scala/StreamSQLExample.scala @@ -75,7 +75,7 @@ object StreamSQLExample { // convert DataStream to Table val tableA = tEnv.fromDataStream(orderA, 'user, 'product, 'amount) // register DataStream as Table - tEnv.registerDataStream("OrderB", orderB, 'user, 'product, 'amount) + tEnv.createTemporaryView("OrderB", orderB, 'user, 'product, 'amount) // union the two tables val result = tEnv.sqlQuery( diff --git a/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/utils/DataStreamConversionUtil.java b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/utils/DataStreamConversionUtil.java index 226f4992da..4e2998d565 100644 --- a/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/utils/DataStreamConversionUtil.java +++ b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/utils/DataStreamConversionUtil.java @@ -26,11 +26,15 @@ import org.apache.flink.ml.common.MLEnvironment; import org.apache.flink.ml.common.MLEnvironmentFactory; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.table.api.Expressions; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.expressions.Expression; import org.apache.flink.types.Row; +import java.util.Arrays; + /** * Provide functions of conversions between DataStream and Table. */ @@ -101,12 +105,9 @@ public class DataStreamConversionUtil { if (null == colNames || colNames.length == 0) { return session.getStreamTableEnvironment().fromDataStream(data); } else { - StringBuilder sbd = new StringBuilder(); - sbd.append(colNames[0]); - for (int i = 1; i < colNames.length; i++) { - sbd.append(",").append(colNames[i]); - } - return session.getStreamTableEnvironment().fromDataStream(data, sbd.toString()); + return session.getStreamTableEnvironment().fromDataStream( + data, + Arrays.stream(colNames).map(Expressions::$).toArray(Expression[]::new)); } } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperatorTestBase.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperatorTestBase.java index 808bcec33a..b790004e65 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperatorTestBase.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperatorTestBase.java @@ -47,6 +47,8 @@ import java.util.Collection; import java.util.List; import java.util.concurrent.ConcurrentLinkedQueue; +import static org.apache.flink.table.api.Expressions.$; + /** * Base class for Python scalar function operator test. These test that: * @@ -201,7 +203,7 @@ public abstract class PythonScalarFunctionOperatorTestBase { StreamTableEnvironment tEnv = createTableEnvironment(env); tEnv.registerFunction("pyFunc", new PythonScalarFunction("pyFunc")); DataStream> ds = env.fromElements(new Tuple2<>(1, 2)); - Table t = tEnv.fromDataStream(ds, "a, b").select("pyFunc(a, b)"); + Table t = tEnv.fromDataStream(ds, $("a"), $("b")).select("pyFunc(a, b)"); // force generating the physical plan for the given table tEnv.toAppendStream(t, BasicTypeInfo.INT_TYPE_INFO); JobGraph jobGraph = env.getStreamGraph().getJobGraph(); diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/BatchTableEnvironment.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/BatchTableEnvironment.java index 42bcd2dd84..1442e33ebe 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/BatchTableEnvironment.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/BatchTableEnvironment.java @@ -31,6 +31,7 @@ import org.apache.flink.table.catalog.CatalogManager; import org.apache.flink.table.catalog.GenericInMemoryCatalog; import org.apache.flink.table.descriptors.BatchTableDescriptor; import org.apache.flink.table.descriptors.ConnectorDescriptor; +import org.apache.flink.table.expressions.Expression; import org.apache.flink.table.functions.AggregateFunction; import org.apache.flink.table.functions.TableFunction; import org.apache.flink.table.module.ModuleManager; @@ -92,7 +93,7 @@ public interface BatchTableEnvironment extends TableEnvironment { /** * Converts the given {@link DataSet} into a {@link Table} with specified field names. * - * Example: + *

    Example: * *

     	 * {@code
    @@ -108,6 +109,25 @@ public interface BatchTableEnvironment extends TableEnvironment {
     	 */
     	 Table fromDataSet(DataSet dataSet, String fields);
     
    +	/**
    +	 * Converts the given {@link DataSet} into a {@link Table} with specified field names.
    +	 *
    +	 * Example:
    +	 *
    +	 * 
    +	 * {@code
    +	 *   DataSet> set = ...
    +	 *   Table tab = tableEnv.fromDataSet(set, $("a"), $("b").as("name"), $("timestamp").rowtime());
    +	 * }
    +	 * 
    + * + * @param dataSet The {@link DataSet} to be converted. + * @param fields The field names of the resulting {@link Table}. + * @param The type of the {@link DataSet}. + * @return The converted {@link Table}. + */ + Table fromDataSet(DataSet dataSet, Expression... fields); + /** * Creates a view from the given {@link DataSet}. * Registered views can be referenced in SQL queries. @@ -202,6 +222,31 @@ public interface BatchTableEnvironment extends TableEnvironment { */ void createTemporaryView(String path, DataSet dataSet, String fields); + /** + * Creates a view from the given {@link DataSet} in a given path with specified field names. + * Registered views can be referenced in SQL queries. + * + *

    Example: + * + *

    +	 * {@code
    +	 *   DataSet> set = ...
    +	 *   tableEnv.createTemporaryView("cat.db.myTable", set, $("b").as("name"), $("timestamp").rowtime());
    +	 * }
    +	 * 
    + * + *

    Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * + * @param path The path under which the view is created. + * See also the {@link TableEnvironment} class description for the format of the path. + * @param dataSet The {@link DataSet} out of which to create the view. + * @param fields The field names of the registered view. + * @param The type of the {@link DataSet}. + */ + void createTemporaryView(String path, DataSet dataSet, Expression... fields); + /** * Converts the given {@link Table} into a {@link DataSet} of a specified type. * diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/StreamTableEnvironment.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/StreamTableEnvironment.java index fd0051780f..92cb111ac0 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/StreamTableEnvironment.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/StreamTableEnvironment.java @@ -33,6 +33,7 @@ import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.java.internal.StreamTableEnvironmentImpl; import org.apache.flink.table.descriptors.ConnectorDescriptor; import org.apache.flink.table.descriptors.StreamTableDescriptor; +import org.apache.flink.table.expressions.Expression; import org.apache.flink.table.functions.AggregateFunction; import org.apache.flink.table.functions.TableAggregateFunction; import org.apache.flink.table.functions.TableFunction; @@ -210,6 +211,25 @@ public interface StreamTableEnvironment extends TableEnvironment { */ Table fromDataStream(DataStream dataStream, String fields); + /** + * Converts the given {@link DataStream} into a {@link Table} with specified field names. + * + *

    Example: + * + *

    +	 * {@code
    +	 *   DataStream> stream = ...
    +	 *   Table tab = tableEnv.fromDataStream(stream, $("a"), $("b").as("name"), $("timestamp").rowtime());
    +	 * }
    +	 * 
    + * + * @param dataStream The {@link DataStream} to be converted. + * @param fields The field expressions of the resulting {@link Table}. + * @param The type of the {@link DataStream}. + * @return The converted {@link Table}. + */ + Table fromDataStream(DataStream dataStream, Expression... fields); + /** * Creates a view from the given {@link DataStream}. * Registered views can be referenced in SQL queries. @@ -274,7 +294,7 @@ public interface StreamTableEnvironment extends TableEnvironment { * @param dataStream The {@link DataStream} to register. * @param fields The field names of the registered view. * @param The type of the {@link DataStream} to register. - * @deprecated use {@link #createTemporaryView(String, DataStream, String)} + * @deprecated use {@link #createTemporaryView(String, DataStream, Expression...)} */ @Deprecated void registerDataStream(String name, DataStream dataStream, String fields); @@ -304,6 +324,31 @@ public interface StreamTableEnvironment extends TableEnvironment { */ void createTemporaryView(String path, DataStream dataStream, String fields); + /** + * Creates a view from the given {@link DataStream} in a given path with specified field names. + * Registered views can be referenced in SQL queries. + * + *

    Example: + * + *

    +	 * {@code
    +	 *   DataStream> stream = ...
    +	 *   tableEnv.createTemporaryView("cat.db.myTable", stream, $("a"), $("b").as("name"), $("timestamp").rowtime())
    +	 * }
    +	 * 
    + * + *

    Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * + * @param path The path under which the {@link DataStream} is created. + * See also the {@link TableEnvironment} class description for the format of the path. + * @param dataStream The {@link DataStream} out of which to create the view. + * @param fields The field expressions of the created view. + * @param The type of the {@link DataStream}. + */ + void createTemporaryView(String path, DataStream dataStream, Expression... fields); + /** * Converts the given {@link Table} into an append {@link DataStream} of a specified type. * diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImpl.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImpl.java index c88d5edec4..4227e2caea 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImpl.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImpl.java @@ -66,6 +66,7 @@ import org.apache.flink.table.types.utils.TypeConversions; import org.apache.flink.table.typeutils.FieldInfoUtils; import java.lang.reflect.Method; +import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; @@ -213,9 +214,14 @@ public final class StreamTableEnvironmentImpl extends TableEnvironmentImpl imple @Override public Table fromDataStream(DataStream dataStream, String fields) { List expressions = ExpressionParser.parseExpressionList(fields); + return fromDataStream(dataStream, expressions.toArray(new Expression[0])); + } + + @Override + public Table fromDataStream(DataStream dataStream, Expression... fields) { JavaDataStreamQueryOperation queryOperation = asQueryOperation( dataStream, - Optional.of(expressions)); + Optional.of(Arrays.asList(fields))); return createTable(queryOperation); } @@ -240,6 +246,14 @@ public final class StreamTableEnvironmentImpl extends TableEnvironmentImpl imple createTemporaryView(path, fromDataStream(dataStream, fields)); } + @Override + public void createTemporaryView( + String path, + DataStream dataStream, + Expression... fields) { + createTemporaryView(path, fromDataStream(dataStream, fields)); + } + @Override protected QueryOperation qualifyQueryOperation(ObjectIdentifier identifier, QueryOperation queryOperation) { if (queryOperation instanceof JavaDataStreamQueryOperation) { diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/operations/JavaDataStreamQueryOperation.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/operations/JavaDataStreamQueryOperation.java index 261bf616f1..9bd294a015 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/operations/JavaDataStreamQueryOperation.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/operations/JavaDataStreamQueryOperation.java @@ -23,6 +23,7 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.java.StreamTableEnvironment; import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.expressions.Expression; import javax.annotation.Nullable; @@ -45,7 +46,7 @@ public class JavaDataStreamQueryOperation implements QueryOperation { /** * The table identifier registered under the environment. The identifier might be null when - * the it is from {@link StreamTableEnvironment#fromDataStream(DataStream, String)}. But the + * the it is from {@link StreamTableEnvironment#fromDataStream(DataStream, Expression...)}. But the * identifier should be not null if is from {@link StreamTableEnvironment#createTemporaryView(String, DataStream)} * with a registered name. */ diff --git a/flink-table/flink-table-api-scala-bridge/src/main/java/org/apache/flink/table/operations/ScalaDataStreamQueryOperation.java b/flink-table/flink-table-api-scala-bridge/src/main/java/org/apache/flink/table/operations/ScalaDataStreamQueryOperation.java index ad975f16b9..63f03d8906 100644 --- a/flink-table/flink-table-api-scala-bridge/src/main/java/org/apache/flink/table/operations/ScalaDataStreamQueryOperation.java +++ b/flink-table/flink-table-api-scala-bridge/src/main/java/org/apache/flink/table/operations/ScalaDataStreamQueryOperation.java @@ -45,7 +45,7 @@ public class ScalaDataStreamQueryOperation implements QueryOperation { /** * The table identifier registered under the environment. The identifier might be null when * the it is from {@code StreamTableEnvironment#fromDataStream(DataStream)}. But the identifier - * should be not null if is from {@code StreamTableEnvironment#registerDataStream(String, DataStream)} + * should be not null if is from {@code StreamTableEnvironment#createTemporaryView(String, DataStream)} * with a registered name. */ @Nullable diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/api/TableUtilsStreamingITCase.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/api/TableUtilsStreamingITCase.java index 769ac2fb06..0c97472a6b 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/api/TableUtilsStreamingITCase.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/api/TableUtilsStreamingITCase.java @@ -30,6 +30,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import static org.apache.flink.table.api.Expressions.$; import static org.junit.Assert.assertEquals; /** @@ -58,7 +59,7 @@ public class TableUtilsStreamingITCase { Row.of(2, 21L), Row.of(2, 22L), Row.of(3, 31L)); - tEnv.registerTable("T", tEnv.fromDataStream(env.fromCollection(sourceData), "a, b")); + tEnv.registerTable("T", tEnv.fromDataStream(env.fromCollection(sourceData), $("a"), $("b"))); String sql = "SELECT b FROM T WHERE a NOT IN (1, 2, 4, 5)"; List expected = Collections.singletonList(Row.of(31L)); diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala index 787f2d673d..94628e8e74 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala @@ -64,7 +64,7 @@ class TableEnvironmentTest { thrown.expect(classOf[ValidationException]) thrown.expectMessage( "Temporary table `default_catalog`.`default_database`.`MyTable` already exists") - tableEnv.registerDataStream("MyTable", env.fromElements[(Int, Long)]()) + tableEnv.createTemporaryView("MyTable", env.fromElements[(Int, Long)]()) } @Test diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala index 15eb2904a8..8083f04c1a 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala @@ -87,7 +87,7 @@ abstract class ExpressionTestBase { @Before def prepare(): Unit = { val ds = env.fromCollection(Collections.emptyList[Row](), typeInfo) - tEnv.registerDataStream(tableName, ds) + tEnv.createTemporaryView(tableName, ds) functions.foreach(f => tEnv.registerFunction(f._1, f._2)) // prepare RelBuilder diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/match/PatternTranslatorTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/match/PatternTranslatorTestBase.scala index 2f56132b01..f02540ff2d 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/match/PatternTranslatorTestBase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/match/PatternTranslatorTestBase.scala @@ -68,7 +68,7 @@ abstract class PatternTranslatorTestBase extends TestLogger { val env = StreamExecutionEnvironment.getExecutionEnvironment val tEnv = StreamTableEnvironment.create(env, TableTestUtil.STREAM_SETTING) - TableTestUtil.registerDataStream( + TableTestUtil.createTemporaryView( tEnv, tableName, dataStreamMock.javaStream, Some(Array[Expression]('f0, 'proctime.proctime))) // prepare RelBuilder diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/stringexpr/CorrelateStringExpressionTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/stringexpr/CorrelateStringExpressionTest.scala index 7da7ead41c..6355ddf37b 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/stringexpr/CorrelateStringExpressionTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/stringexpr/CorrelateStringExpressionTest.scala @@ -22,6 +22,7 @@ import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.api.scala._ import org.apache.flink.streaming.api.datastream.{DataStream => JDataStream} import org.apache.flink.streaming.api.scala.DataStream +import org.apache.flink.table.api.Expressions.$ import org.apache.flink.table.api._ import org.apache.flink.table.api.scala._ import org.apache.flink.table.planner.utils._ @@ -44,7 +45,7 @@ class CorrelateStringExpressionTest extends TableTestBase { val sDs = mock(classOf[DataStream[Row]]) when(sDs.javaStream).thenReturn(jDs) - val jTab = javaUtil.tableEnv.fromDataStream(jDs, "a, b, c") + val jTab = javaUtil.tableEnv.fromDataStream(jDs, $("a"), $("b"), $("c")) val sTab = scalaUtil.tableEnv.fromDataStream(sDs, 'a, 'b, 'c) // test cross join @@ -112,7 +113,7 @@ class CorrelateStringExpressionTest extends TableTestBase { val sDs = mock(classOf[DataStream[Row]]) when(sDs.javaStream).thenReturn(jDs) - val jTab = javaUtil.tableEnv.fromDataStream(jDs, "a, b, c") + val jTab = javaUtil.tableEnv.fromDataStream(jDs, $("a"), $("b"), $("c")) val sTab = scalaUtil.tableEnv.fromDataStream(sDs, 'a, 'b, 'c) // test flatMap diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/AggregateRemoveITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/AggregateRemoveITCase.scala index 370cddacf3..abe4e7866c 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/AggregateRemoveITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/AggregateRemoveITCase.scala @@ -214,7 +214,7 @@ class AggregateRemoveITCase( (3, 2, "A", "Hi"), (5, 2, "B", "Hello"), (6, 3, "C", "Hello world"))) - StreamTableEnvUtil.registerDataStreamInternal[(Int, Int, String, String)]( + StreamTableEnvUtil.createTemporaryViewInternal[(Int, Int, String, String)]( tEnv, "T", ds1.javaStream, @@ -223,7 +223,7 @@ class AggregateRemoveITCase( Some(FlinkStatistic.builder().uniqueKeys(Set(Set("a").asJava).asJava).build()) ) - StreamTableEnvUtil.registerDataStreamInternal[(Int, Long, String)]( + StreamTableEnvUtil.createTemporaryViewInternal[(Int, Long, String)]( tEnv, "MyTable", env.fromCollection(TestData.smallTupleData3).javaStream, @@ -232,7 +232,7 @@ class AggregateRemoveITCase( Some(FlinkStatistic.builder().uniqueKeys(Set(Set("a").asJava).asJava).build()) ) - StreamTableEnvUtil.registerDataStreamInternal[(Int, Long, Int, String, Long)]( + StreamTableEnvUtil.createTemporaryViewInternal[(Int, Long, Int, String, Long)]( tEnv, "MyTable2", env.fromCollection(TestData.smallTupleData5).javaStream, diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/PruneAggregateCallITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/PruneAggregateCallITCase.scala index 01b16e9866..3c588c9e61 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/PruneAggregateCallITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/PruneAggregateCallITCase.scala @@ -99,7 +99,7 @@ class PruneAggregateCallITCase( private def checkResult(str: String, rows: Seq[Row]): Unit = { super.before() - StreamTableEnvUtil.registerDataStreamInternal[(Int, Long, String)]( + StreamTableEnvUtil.createTemporaryViewInternal[(Int, Long, String)]( tEnv, "MyTable", failingDataSource(TestData.smallTupleData3).javaStream, @@ -108,7 +108,7 @@ class PruneAggregateCallITCase( Some(FlinkStatistic.UNKNOWN) ) - StreamTableEnvUtil.registerDataStreamInternal[(Int, Long, Int, String, Long)]( + StreamTableEnvUtil.createTemporaryViewInternal[(Int, Long, Int, String, Long)]( tEnv, "MyTable2", failingDataSource(TestData.smallTupleData5).javaStream, diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTableEnvUtil.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTableEnvUtil.scala index ea678b9a4f..f65dd0481d 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTableEnvUtil.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTableEnvUtil.scala @@ -248,7 +248,7 @@ object BatchTableEnvUtil { fieldNullables: Option[Array[Boolean]], statistic: Option[FlinkStatistic]): Unit = { val fields = fieldNames.map((f: Array[String]) => f.map(ExpressionParser.parseExpression)) - TableTestUtil.registerDataStream( + TableTestUtil.createTemporaryView( tEnv, name, boundedStream, diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamTableEnvUtil.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamTableEnvUtil.scala index 6a922cdef0..f6fc09ac18 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamTableEnvUtil.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamTableEnvUtil.scala @@ -36,7 +36,7 @@ object StreamTableEnvUtil { * @param dataStream The [[DataStream]] to register as table in the catalog. * @tparam T the type of the [[DataStream]]. */ - def registerDataStreamInternal[T]( + def createTemporaryViewInternal[T]( tEnv: StreamTableEnvironment, name: String, dataStream: DataStream[T], @@ -47,7 +47,7 @@ object StreamTableEnvUtil { case Some(names) => Some(names.map(ExpressionParser.parseExpression)) case _ => None } - TableTestUtil.registerDataStream(tEnv, name, dataStream, fields, fieldNullables, statistic) + TableTestUtil.createTemporaryView(tEnv, name, dataStream, fields, fieldNullables, statistic) } } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala index c9d232cff6..169b7db674 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala @@ -173,7 +173,7 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) val env = new ScalaStreamExecEnv(new LocalStreamEnvironment()) val dataStream = env.fromElements[T]().javaStream val tableEnv = getTableEnv - TableTestUtil.registerDataStream(tableEnv, name, dataStream, Some(fields.toArray)) + TableTestUtil.createTemporaryView(tableEnv, name, dataStream, Some(fields.toArray)) tableEnv.scan(name) } @@ -1130,7 +1130,7 @@ object TableTestUtil { .getRelBuilder.queryOperation(table.getQueryOperation).build() } - def registerDataStream[T]( + def createTemporaryView[T]( tEnv: TableEnvironment, name: String, dataStream: DataStream[T], diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/java/internal/BatchTableEnvironmentImpl.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/java/internal/BatchTableEnvironmentImpl.scala index f79a3ba955..fceae68919 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/java/internal/BatchTableEnvironmentImpl.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/java/internal/BatchTableEnvironmentImpl.scala @@ -24,7 +24,7 @@ import org.apache.flink.table.api._ import org.apache.flink.table.api.internal.BatchTableEnvImpl import org.apache.flink.table.api.java.BatchTableEnvironment import org.apache.flink.table.catalog.CatalogManager -import org.apache.flink.table.expressions.ExpressionParser +import org.apache.flink.table.expressions.{Expression, ExpressionParser} import org.apache.flink.table.functions.{AggregateFunction, TableFunction} import org.apache.flink.table.module.ModuleManager @@ -58,7 +58,13 @@ class BatchTableEnvironmentImpl( .parseExpressionList(fields).asScala .toArray - createTable(asQueryOperation(dataSet, Some(exprs))) + fromDataSet(dataSet, exprs: _*) + } + + override def fromDataSet[T]( + dataSet: DataSet[T], + fields: Expression*): Table = { + createTable(asQueryOperation(dataSet, Some(fields.toArray))) } override def registerDataSet[T](name: String, dataSet: DataSet[T]): Unit = { @@ -82,6 +88,13 @@ class BatchTableEnvironmentImpl( createTemporaryView(path, fromDataSet(dataSet, fields)) } + override def createTemporaryView[T]( + path: String, + dataSet: DataSet[T], + fields: Expression*): Unit = { + createTemporaryView(path, fromDataSet(dataSet, fields: _*)) + } + override def toDataSet[T](table: Table, clazz: Class[T]): DataSet[T] = { // Use the default query config. translate[T](table)(TypeExtractor.createTypeInfo(clazz)) diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/batch/sql/GroupingSetsITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/batch/sql/GroupingSetsITCase.java index c73eaff646..93c1372af9 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/batch/sql/GroupingSetsITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/batch/sql/GroupingSetsITCase.java @@ -61,7 +61,7 @@ public class GroupingSetsITCase extends TableProgramsClusterTestBase { tableEnv = BatchTableEnvironment.create(env, new TableConfig()); DataSet> dataSet = CollectionDataSets.get3TupleDataSet(env); - tableEnv.registerDataSet(TABLE_NAME, dataSet); + tableEnv.createTemporaryView(TABLE_NAME, dataSet); MapOperator, Tuple3> dataSetWithNulls = dataSet.map(new MapFunction, Tuple3>() { @@ -74,7 +74,7 @@ public class GroupingSetsITCase extends TableProgramsClusterTestBase { return value; } }); - tableEnv.registerDataSet(TABLE_WITH_NULLS_NAME, dataSetWithNulls); + tableEnv.createTemporaryView(TABLE_WITH_NULLS_NAME, dataSetWithNulls); } @Test diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/batch/sql/JavaSqlITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/batch/sql/JavaSqlITCase.java index c1fdf19f6a..e1b4c35ba2 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/batch/sql/JavaSqlITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/batch/sql/JavaSqlITCase.java @@ -42,6 +42,8 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import static org.apache.flink.table.api.Expressions.$; + /** * Integration tests for batch SQL. */ @@ -78,7 +80,7 @@ public class JavaSqlITCase extends TableProgramsCollectionTestBase { BatchTableEnvironment tableEnv = BatchTableEnvironment.create(env, config()); DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - Table in = tableEnv.fromDataSet(ds, "a,b,c"); + Table in = tableEnv.fromDataSet(ds, $("a"), $("b"), $("c")); tableEnv.registerTable("T", in); String sqlQuery = "SELECT a, c FROM T"; @@ -102,7 +104,7 @@ public class JavaSqlITCase extends TableProgramsCollectionTestBase { BatchTableEnvironment tableEnv = BatchTableEnvironment.create(env, config()); DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - tableEnv.registerDataSet("DataSetTable", ds, "x, y, z"); + tableEnv.createTemporaryView("DataSetTable", ds, $("x"), $("y"), $("z")); String sqlQuery = "SELECT x FROM DataSetTable WHERE z LIKE '%Hello%'"; Table result = tableEnv.sqlQuery(sqlQuery); @@ -119,7 +121,7 @@ public class JavaSqlITCase extends TableProgramsCollectionTestBase { BatchTableEnvironment tableEnv = BatchTableEnvironment.create(env, config()); DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - tableEnv.registerDataSet("AggTable", ds, "x, y, z"); + tableEnv.createTemporaryView("AggTable", ds, $("x"), $("y"), $("z")); String sqlQuery = "SELECT sum(x), min(x), max(x), count(y), avg(x) FROM AggTable"; Table result = tableEnv.sqlQuery(sqlQuery); @@ -138,8 +140,8 @@ public class JavaSqlITCase extends TableProgramsCollectionTestBase { DataSet> ds1 = CollectionDataSets.getSmall3TupleDataSet(env); DataSet> ds2 = CollectionDataSets.get5TupleDataSet(env); - tableEnv.registerDataSet("t1", ds1, "a, b, c"); - tableEnv.registerDataSet("t2", ds2, "d, e, f, g, h"); + tableEnv.createTemporaryView("t1", ds1, $("a"), $("b"), $("c")); + tableEnv.createTemporaryView("t2", ds2, $("d"), $("e"), $("f"), $("g"), $("h")); String sqlQuery = "SELECT c, g FROM t1, t2 WHERE b = e"; Table result = tableEnv.sqlQuery(sqlQuery); @@ -164,7 +166,7 @@ public class JavaSqlITCase extends TableProgramsCollectionTestBase { new MapTypeInfo<>(BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO)); DataSet>> ds1 = env.fromCollection(rows, ty); - tableEnv.registerDataSet("t1", ds1, "a, b"); + tableEnv.createTemporaryView("t1", ds1, $("a"), $("b")); String sqlQuery = "SELECT b['foo'] FROM t1"; Table result = tableEnv.sqlQuery(sqlQuery); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/batch/table/JavaTableEnvironmentITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/batch/table/JavaTableEnvironmentITCase.java index e362027c48..8d72ba25ae 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/batch/table/JavaTableEnvironmentITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/batch/table/JavaTableEnvironmentITCase.java @@ -80,7 +80,7 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase DataSet> ds = CollectionDataSets.get3TupleDataSet(env); Table t = tableEnv.fromDataSet(ds); // Must fail. Table is empty - tableEnv.registerTable("", t); + tableEnv.createTemporaryView("", t); } @Test(expected = ValidationException.class) @@ -91,7 +91,7 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase DataSet> ds = CollectionDataSets.get3TupleDataSet(env); Table t = tableEnv.fromDataSet(ds); // Must fail. Table is empty - tableEnv.registerTable(" ", t); + tableEnv.createTemporaryView(" ", t); } @Test @@ -101,8 +101,8 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase BatchTableEnvironment tableEnv = BatchTableEnvironment.create(env, config()); DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - tableEnv.registerDataSet(tableName, ds); - Table t = tableEnv.scan(tableName); + tableEnv.createTemporaryView(tableName, ds); + Table t = tableEnv.from(tableName); Table result = t.select($("f0"), $("f1")); @@ -121,8 +121,8 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase BatchTableEnvironment tableEnv = BatchTableEnvironment.create(env, config()); DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - tableEnv.registerDataSet(tableName, ds, "a, b, c"); - Table t = tableEnv.scan(tableName); + tableEnv.createTemporaryView(tableName, ds, $("a"), $("b"), $("c")); + Table t = tableEnv.from(tableName); Table result = t.select($("a"), $("b"), $("c")); @@ -144,11 +144,11 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase BatchTableEnvironment tableEnv = BatchTableEnvironment.create(env, config()); DataSet> ds = CollectionDataSets.get3TupleDataSet(env); - tableEnv.registerDataSet("MyTable", ds); + tableEnv.createTemporaryView("MyTable", ds); DataSet> ds2 = CollectionDataSets.getSmall5TupleDataSet(env); // Must fail. Name is already used for different table. - tableEnv.registerDataSet("MyTable", ds2); + tableEnv.createTemporaryView("MyTable", ds2); } @Test(expected = TableException.class) @@ -157,7 +157,7 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase BatchTableEnvironment tableEnv = BatchTableEnvironment.create(env, config()); // Must fail. No table registered under that name. - tableEnv.scan("nonRegisteredTable"); + tableEnv.from("nonRegisteredTable"); } @Test @@ -168,7 +168,7 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase DataSet> ds = CollectionDataSets.get3TupleDataSet(env); Table t = tableEnv.fromDataSet(ds); - tableEnv.registerTable(tableName, t); + tableEnv.createTemporaryView(tableName, t); Table result = tableEnv.scan(tableName).select($("f0"), $("f1")).filter($("f0").isGreater(7)); DataSet resultSet = tableEnv.toDataSet(result, Row.class); @@ -187,7 +187,7 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase Table t = tableEnv1.fromDataSet(CollectionDataSets.get3TupleDataSet(env)); // Must fail. Table is bound to different TableEnvironment. - tableEnv2.registerTable("MyTable", t); + tableEnv2.createTemporaryView("MyTable", t); } @Test @@ -196,7 +196,7 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase BatchTableEnvironment tableEnv = BatchTableEnvironment.create(env, config()); Table table = tableEnv - .fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a, b, c") + .fromDataSet(CollectionDataSets.get3TupleDataSet(env), $("a"), $("b"), $("c")) .select($("a"), $("b"), $("c")); DataSet ds = tableEnv.toDataSet(table, Row.class); @@ -216,7 +216,7 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); BatchTableEnvironment tableEnv = BatchTableEnvironment.create(env, config()); - Table table = tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "f2"); + Table table = tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), $("f2")); DataSet ds = tableEnv.toDataSet(table, Row.class); List results = ds.collect(); @@ -236,7 +236,7 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase BatchTableEnvironment tableEnv = BatchTableEnvironment.create(env, config()); Table table = tableEnv - .fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a, b, c") + .fromDataSet(CollectionDataSets.get3TupleDataSet(env), $("a"), $("b"), $("c")) .select($("a"), $("b"), $("c")); TypeInformation ti = new TupleTypeInfo>( @@ -268,7 +268,7 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase data.add(new Tuple4<>("Test me", 4, 3.33, "Hello world")); Table table = tableEnv - .fromDataSet(env.fromCollection(data), "q, w, e, r") + .fromDataSet(env.fromCollection(data), $("q"), $("w"), $("e"), $("r")) .select($("q").as("a"), $("w").as("b"), $("e").as("c"), $("r").as("d")); DataSet ds = tableEnv.toDataSet(table, SmallPojo2.class); @@ -289,11 +289,11 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase Table table = tableEnv .fromDataSet(env.fromCollection(data), - "department AS a, " + - "age AS b, " + - "salary AS c, " + - "name AS d," + - "roles as e") + $("department").as("a"), + $("age").as("b"), + $("salary").as("c"), + $("name").as("d"), + $("roles").as("e")) .select($("a"), $("b"), $("c"), $("d"), $("e")); DataSet ds = tableEnv.toDataSet(table, Row.class); @@ -321,7 +321,7 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase data, TypeInformation.of(new TypeHint>() { }) ), - "either") + $("either")) .select("either"); DataSet ds = tableEnv.toDataSet(table, Row.class); @@ -344,7 +344,7 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase data.add(new SmallPojo("Lucy", 42, 6000.00, "HR", new Integer[] {1, 2, 3})); Table table = tableEnv - .fromDataSet(env.fromCollection(data), "name AS d") + .fromDataSet(env.fromCollection(data), $("name").as("d")) .select($("d")); DataSet ds = tableEnv.toDataSet(table, Row.class); @@ -368,10 +368,10 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase Table table = tableEnv .fromDataSet(env.fromCollection(data), - "department AS a, " + - "age AS b, " + - "salary AS c, " + - "name AS d") + $("department").as("a"), + $("age").as("b"), + $("salary").as("c"), + $("name").as("d")) .select($("a"), $("b"), $("c"), $("d")); DataSet ds = tableEnv.toDataSet(table, Row.class); @@ -395,11 +395,11 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase Table table = tableEnv .fromDataSet(env.fromCollection(data), - "department AS a, " + - "age AS b, " + - "salary AS c, " + - "name AS d," + - "roles AS e") + $("department").as("a"), + $("age").as("b"), + $("salary").as("c"), + $("name").as("d"), + $("roles").as("e")) .select($("a"), $("b"), $("c"), $("d"), $("e")); DataSet ds = tableEnv.toDataSet(table, SmallPojo2.class); @@ -423,10 +423,10 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase Table table = tableEnv .fromDataSet(env.fromCollection(data), - "department AS a, " + - "age AS b, " + - "salary AS c, " + - "name AS d") + $("department").as("a"), + $("age").as("b"), + $("salary").as("c"), + $("name").as("d")) .select($("a"), $("b"), $("c"), $("d")); DataSet ds = tableEnv.toDataSet(table, PrivateSmallPojo2.class); @@ -454,10 +454,10 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase Table table = tableEnv .fromDataSet(env.fromCollection(data), - "name AS a, " + - "age AS b, " + - "generic AS c, " + - "generic2 AS d") + $("name").as("a"), + $("age").as("b"), + $("generic").as("c"), + $("generic2").as("d")) .select($("a"), $("b"), $("c"), $("c").as("c2"), $("d")) .select($("a"), $("b"), $("c"), $("c").isEqual($("c2")), $("d")); @@ -495,7 +495,7 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase assertTrue(dataSet.getType().getTypeClass().equals(Row.class)); // Must fail. Cannot import DataSet with GenericTypeInfo. - tableEnv.fromDataSet(dataSet, "nullField"); + tableEnv.fromDataSet(dataSet, $("nullField")); } @Test(expected = ValidationException.class) @@ -504,7 +504,7 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase BatchTableEnvironment tableEnv = BatchTableEnvironment.create(env, config()); // Must fail. Too many field names specified. - tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a, b, c, d"); + tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), $("a"), $("b"), $("c"), $("d")); } @Test(expected = ValidationException.class) @@ -513,7 +513,7 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase BatchTableEnvironment tableEnv = BatchTableEnvironment.create(env, config()); // Must fail. Specified field names are not unique. - tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a, b, b"); + tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), $("a"), $("b"), $("b")); } @Test(expected = ValidationException.class) @@ -522,7 +522,7 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase BatchTableEnvironment tableEnv = BatchTableEnvironment.create(env, config()); // Must fail. as() does only allow field name expressions - tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a + 1, b, c"); + tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), $("a").plus(1), $("b"), $("c")); } @Test(expected = ValidationException.class) @@ -531,7 +531,7 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase BatchTableEnvironment tableEnv = BatchTableEnvironment.create(env, config()); // Must fail. as() does only allow field name expressions - tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a as foo, b, c"); + tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), $("a").as("foo"), $("b"), $("c")); } @Test(expected = ValidationException.class) @@ -540,7 +540,7 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase BatchTableEnvironment tableEnv = BatchTableEnvironment.create(env, config()); // Must fail since class is not static - tableEnv.fromDataSet(env.fromElements(new MyNonStatic()), "name"); + tableEnv.fromDataSet(env.fromElements(new MyNonStatic()), $("name")); } @Test(expected = ValidationException.class) @@ -549,7 +549,7 @@ public class JavaTableEnvironmentITCase extends TableProgramsCollectionTestBase BatchTableEnvironment tableEnv = BatchTableEnvironment.create(env, config()); // Must fail since class is not static - Table t = tableEnv.fromDataSet(env.fromElements(1, 2, 3), "number"); + Table t = tableEnv.fromDataSet(env.fromElements(1, 2, 3), $("number")); tableEnv.toDataSet(t, MyNonStatic.class); } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/stream/sql/JavaSqlITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/stream/sql/JavaSqlITCase.java index 72f309473a..df8854566f 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/stream/sql/JavaSqlITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/stream/sql/JavaSqlITCase.java @@ -37,6 +37,8 @@ import org.junit.Test; import java.util.ArrayList; import java.util.List; +import static org.apache.flink.table.api.Expressions.$; + /** * Integration tests for streaming SQL. */ @@ -63,7 +65,7 @@ public class JavaSqlITCase extends AbstractTestBase { DataStream ds = env.fromCollection(data).returns(typeInfo); - Table in = tableEnv.fromDataStream(ds, "a,b,c"); + Table in = tableEnv.fromDataStream(ds, $("a"), $("b"), $("c")); tableEnv.registerTable("MyTableRow", in); String sqlQuery = "SELECT a,c FROM MyTableRow"; @@ -88,7 +90,7 @@ public class JavaSqlITCase extends AbstractTestBase { StreamITCase.clear(); DataStream> ds = JavaStreamTestData.getSmall3TupleDataSet(env); - Table in = tableEnv.fromDataStream(ds, "a,b,c"); + Table in = tableEnv.fromDataStream(ds, $("a"), $("b"), $("c")); tableEnv.registerTable("MyTable", in); String sqlQuery = "SELECT * FROM MyTable"; @@ -113,7 +115,7 @@ public class JavaSqlITCase extends AbstractTestBase { StreamITCase.clear(); DataStream> ds = JavaStreamTestData.get5TupleDataStream(env); - tableEnv.registerDataStream("MyTable", ds, "a, b, c, d, e"); + tableEnv.createTemporaryView("MyTable", ds, $("a"), $("b"), $("c"), $("d"), $("e")); String sqlQuery = "SELECT a, b, e FROM MyTable WHERE c < 4"; Table result = tableEnv.sqlQuery(sqlQuery); @@ -138,11 +140,11 @@ public class JavaSqlITCase extends AbstractTestBase { StreamITCase.clear(); DataStream> ds1 = JavaStreamTestData.getSmall3TupleDataSet(env); - Table t1 = tableEnv.fromDataStream(ds1, "a,b,c"); + Table t1 = tableEnv.fromDataStream(ds1, $("a"), $("b"), $("c")); tableEnv.registerTable("T1", t1); DataStream> ds2 = JavaStreamTestData.get5TupleDataStream(env); - tableEnv.registerDataStream("T2", ds2, "a, b, d, c, e"); + tableEnv.createTemporaryView("T2", ds2, $("a"), $("b"), $("d"), $("c"), $("e")); String sqlQuery = "SELECT * FROM T1 " + "UNION ALL " + diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/sql/SetOperatorsTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/sql/SetOperatorsTest.scala index 056c882885..86e5ab8786 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/sql/SetOperatorsTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/sql/SetOperatorsTest.scala @@ -20,11 +20,13 @@ package org.apache.flink.table.api.batch.sql import org.apache.flink.api.java.typeutils.GenericTypeInfo import org.apache.flink.api.scala._ +import org.apache.flink.table.api.Expressions.$ import org.apache.flink.table.api.Types import org.apache.flink.table.api.scala._ import org.apache.flink.table.runtime.utils.CommonTestData.NonPojo import org.apache.flink.table.utils.TableTestBase import org.apache.flink.table.utils.TableTestUtil._ + import org.junit.Test class SetOperatorsTest extends TableTestBase { @@ -251,7 +253,7 @@ class SetOperatorsTest extends TableTestBase { val typeInfo = Types.ROW( new GenericTypeInfo(classOf[NonPojo]), new GenericTypeInfo(classOf[NonPojo])) - val table = util.addJavaTable(typeInfo, "A", "a, b") + val table = util.addJavaTable(typeInfo, "A", $("a"), $("b")) val expected = binaryNode( "DataSetUnion", diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/table/SetOperatorsTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/table/SetOperatorsTest.scala index fe1e84b2fe..677d11a136 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/table/SetOperatorsTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/table/SetOperatorsTest.scala @@ -18,17 +18,19 @@ package org.apache.flink.table.api.batch.table -import java.sql.Timestamp - import org.apache.flink.api.java.typeutils.GenericTypeInfo import org.apache.flink.api.scala._ +import org.apache.flink.table.api.Expressions.$ import org.apache.flink.table.api.Types import org.apache.flink.table.api.scala._ import org.apache.flink.table.runtime.utils.CommonTestData.NonPojo import org.apache.flink.table.utils.TableTestBase import org.apache.flink.table.utils.TableTestUtil._ + import org.junit.Test +import java.sql.Timestamp + class SetOperatorsTest extends TableTestBase { @Test @@ -115,7 +117,7 @@ class SetOperatorsTest extends TableTestBase { val typeInfo = Types.ROW( new GenericTypeInfo(classOf[NonPojo]), new GenericTypeInfo(classOf[NonPojo])) - val t = util.addJavaTable(typeInfo, "A", "a, b") + val t = util.addJavaTable(typeInfo, "A", $("a"), $("b")) val in = t.select('a).unionAll(t.select('b)) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/table/stringexpr/CorrelateStringExpressionTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/table/stringexpr/CorrelateStringExpressionTest.scala index c5bdf3c9da..c8683c1479 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/table/stringexpr/CorrelateStringExpressionTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/table/stringexpr/CorrelateStringExpressionTest.scala @@ -21,10 +21,12 @@ package org.apache.flink.table.api.batch.table.stringexpr import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.api.java.{DataSet => JDataSet} import org.apache.flink.api.scala._ -import org.apache.flink.table.api.Types +import org.apache.flink.table.api.Expressions.$ +import org.apache.flink.table.api.{Expressions, Types} import org.apache.flink.table.api.scala._ import org.apache.flink.table.utils.{PojoTableFunc, TableFunc2, _} import org.apache.flink.types.Row + import org.junit.Test import org.mockito.Mockito.{mock, when} @@ -42,7 +44,7 @@ class CorrelateStringExpressionTest extends TableTestBase { val sDs = mock(classOf[DataSet[Row]]) when(sDs.javaSet).thenReturn(jDs) - val jTab = util.javaTableEnv.fromDataSet(jDs, "a, b, c") + val jTab = util.javaTableEnv.fromDataSet(jDs, $("a"), $("b"), $("c")) val sTab = util.tableEnv.fromDataSet(sDs, 'a, 'b, 'c) // test cross join diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/StreamTableEnvironmentTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/StreamTableEnvironmentTest.scala index db039df9c0..7cae71f9cf 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/StreamTableEnvironmentTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/StreamTableEnvironmentTest.scala @@ -25,10 +25,11 @@ import org.apache.flink.streaming.api.TimeCharacteristic import org.apache.flink.streaming.api.datastream.DataStream import org.apache.flink.streaming.api.environment.{StreamExecutionEnvironment => JStreamExecEnv} import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment +import org.apache.flink.table.api.Expressions.$ import org.apache.flink.table.api.java.internal.{StreamTableEnvironmentImpl => JStreamTableEnvironmentImpl} import org.apache.flink.table.api.java.{StreamTableEnvironment => JStreamTableEnv} import org.apache.flink.table.api.scala._ -import org.apache.flink.table.api.{TableConfig, Types, ValidationException} +import org.apache.flink.table.api.{Expressions, TableConfig, Types, ValidationException} import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog, GenericInMemoryCatalog} import org.apache.flink.table.executor.StreamExecutor import org.apache.flink.table.planner.StreamPlanner @@ -36,10 +37,11 @@ import org.apache.flink.table.runtime.utils.StreamTestData import org.apache.flink.table.utils.{CatalogManagerMocks, TableTestBase} import org.apache.flink.table.utils.TableTestUtil.{binaryNode, streamTableNode, term, unaryNode} import org.apache.flink.types.Row + import org.junit.Test import org.mockito.Mockito.{mock, when} -import java.lang.{Integer => JInt, Long => JLong} +import java.lang.{Integer => JInt, Long => JLong} import org.apache.flink.table.module.ModuleManager class StreamTableEnvironmentTest extends TableTestBase { @@ -170,31 +172,39 @@ class StreamTableEnvironmentTest extends TableTestBase { @Test def testProctimeAttributeParsed(): Unit = { val (jTEnv, ds) = prepareSchemaExpressionParser - jTEnv.fromDataStream(ds, "a, b, c, d, e, pt.proctime") + jTEnv.fromDataStream(ds, $("a"), $("b"), $("c"), $("d"), $("e"), $("pt").proctime()) } @Test def testReplacingRowtimeAttributeParsed(): Unit = { val (jTEnv, ds) = prepareSchemaExpressionParser - jTEnv.fromDataStream(ds, "a.rowtime, b, c, d, e") + jTEnv.fromDataStream(ds, $("a").rowtime(), $("b"), $("c"), $("d"), $("e")) } @Test def testAppedingRowtimeAttributeParsed(): Unit = { val (jTEnv, ds) = prepareSchemaExpressionParser - jTEnv.fromDataStream(ds, "a, b, c, d, e, rt.rowtime") + jTEnv.fromDataStream(ds, $("a"), $("b"), $("c"), $("d"), $("e"), $("rt").rowtime()) } @Test def testRowtimeAndProctimeAttributeParsed1(): Unit = { val (jTEnv, ds) = prepareSchemaExpressionParser - jTEnv.fromDataStream(ds, "a, b, c, d, e, pt.proctime, rt.rowtime") + jTEnv.fromDataStream( + ds, + $("a"), + $("b"), + $("c"), + $("d"), + $("e"), + $("pt").proctime(), + $("rt").rowtime()) } @Test def testRowtimeAndProctimeAttributeParsed2(): Unit = { val (jTEnv, ds) = prepareSchemaExpressionParser - jTEnv.fromDataStream(ds, "rt.rowtime, b, c, d, e, pt.proctime") + jTEnv.fromDataStream(ds, $("rt").rowtime(), $("b"), $("c"), $("d"), $("e"), $("pt").proctime()) } private def prepareSchemaExpressionParser: diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/UnionTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/UnionTest.scala index 5bb520c226..9a039d98c8 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/UnionTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/UnionTest.scala @@ -20,11 +20,13 @@ package org.apache.flink.table.api.stream.sql import org.apache.flink.api.java.typeutils.GenericTypeInfo import org.apache.flink.api.scala._ -import org.apache.flink.table.api.Types +import org.apache.flink.table.api.Expressions.$ +import org.apache.flink.table.api.{Expressions, Types} import org.apache.flink.table.api.scala._ import org.apache.flink.table.runtime.utils.CommonTestData.NonPojo import org.apache.flink.table.utils.TableTestUtil._ import org.apache.flink.table.utils.TableTestBase + import org.junit.Test class UnionTest extends TableTestBase { @@ -63,7 +65,7 @@ class UnionTest extends TableTestBase { val typeInfo = Types.ROW( new GenericTypeInfo(classOf[NonPojo]), new GenericTypeInfo(classOf[NonPojo])) - val table = streamUtil.addJavaTable(typeInfo, "A", "a, b") + val table = streamUtil.addJavaTable(typeInfo, "A", $("a"), $("b")) val expected = binaryNode( "DataStreamUnion", diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/table/TableAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/table/TableAggregateTest.scala index 8282f572bc..ba2fc7fb9f 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/table/TableAggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/table/TableAggregateTest.scala @@ -20,12 +20,14 @@ package org.apache.flink.table.api.stream.table import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.api.scala._ +import org.apache.flink.table.api.Expressions.$ import org.apache.flink.table.api.Types import org.apache.flink.table.api.scala._ import org.apache.flink.table.expressions.utils.Func0 import org.apache.flink.table.utils.{EmptyTableAggFunc, EmptyTableAggFuncWithIntResultType, TableTestBase} import org.apache.flink.table.utils.TableTestUtil._ import org.apache.flink.types.Row + import org.junit.Test class TableAggregateTest extends TableTestBase { @@ -152,7 +154,7 @@ class TableAggregateTest extends TableTestBase { def testJavaRegisterFunction(): Unit = { val util = streamTestUtil() val typeInfo = new RowTypeInfo(Types.INT, Types.LONG, Types.STRING) - val table = util.addJavaTable[Row](typeInfo, "sourceTable", "a, b, c") + val table = util.addJavaTable[Row](typeInfo, "sourceTable", $("a"), $("b"), $("c")) val func = new EmptyTableAggFunc util.javaTableEnv.registerFunction("func", func) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/table/stringexpr/CorrelateStringExpressionTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/table/stringexpr/CorrelateStringExpressionTest.scala index e2068e7a31..7d65e70efb 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/table/stringexpr/CorrelateStringExpressionTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/table/stringexpr/CorrelateStringExpressionTest.scala @@ -24,10 +24,12 @@ import org.apache.flink.table.api._ import org.apache.flink.table.api.scala._ import org.apache.flink.table.utils._ import org.apache.flink.types.Row + import org.junit.Test import org.mockito.Mockito.{mock, when} import org.apache.flink.streaming.api.datastream.{DataStream => JDataStream} import org.apache.flink.streaming.api.scala.DataStream +import org.apache.flink.table.api.Expressions.$ class CorrelateStringExpressionTest extends TableTestBase { @@ -42,7 +44,7 @@ class CorrelateStringExpressionTest extends TableTestBase { val sDs = mock(classOf[DataStream[Row]]) when(sDs.javaStream).thenReturn(jDs) - val jTab = util.javaTableEnv.fromDataStream(jDs, "a, b, c") + val jTab = util.javaTableEnv.fromDataStream(jDs, $("a"), $("b"), $("c")) val sTab = util.tableEnv.fromDataStream(sDs, 'a, 'b, 'c) // test cross join @@ -111,7 +113,7 @@ class CorrelateStringExpressionTest extends TableTestBase { val sDs = mock(classOf[DataStream[Row]]) when(sDs.javaStream).thenReturn(jDs) - val jTab = util.javaTableEnv.fromDataStream(jDs, "a, b, c") + val jTab = util.javaTableEnv.fromDataStream(jDs, $("a"), $("b"), $("c")) val sTab = util.tableEnv.fromDataStream(sDs, 'a, 'b, 'c) // test flatMap diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/validation/TableEnvironmentValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/validation/TableEnvironmentValidationTest.scala index 4cef6fd22f..527c53a745 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/validation/TableEnvironmentValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/validation/TableEnvironmentValidationTest.scala @@ -117,10 +117,10 @@ class TableEnvironmentValidationTest extends TableTestBase { val tEnv = BatchTableEnvironment.create(env) val ds1 = CollectionDataSets.get3TupleDataSet(env) - tEnv.registerDataSet("MyTable", ds1) + tEnv.createTemporaryView("MyTable", ds1) val ds2 = CollectionDataSets.get5TupleDataSet(env) // Must fail. Name is already in use. - tEnv.registerDataSet("MyTable", ds2) + tEnv.createTemporaryView("MyTable", ds2) } @Test(expected = classOf[TableException]) @@ -140,7 +140,7 @@ class TableEnvironmentValidationTest extends TableTestBase { tEnv.registerTable("MyTable", t1) val t2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv) // Must fail. Name is already in use. - tEnv.registerDataSet("MyTable", t2) + tEnv.createTemporaryView("MyTable", t2) } @Test(expected = classOf[TableException]) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/expressions/utils/ExpressionTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/expressions/utils/ExpressionTestBase.scala index 180c83321f..d222e888a5 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/expressions/utils/ExpressionTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/expressions/utils/ExpressionTestBase.scala @@ -88,7 +88,7 @@ abstract class ExpressionTestBase { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = BatchTableEnvironment.create(env).asInstanceOf[BatchTableEnvironmentImpl] - tEnv.registerDataSet(tableName, dataSetMock) + tEnv.createTemporaryView(tableName, dataSetMock) functions.foreach(f => tEnv.registerFunction(f._1, f._2)) // prepare RelBuilder diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/match/PatternTranslatorTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/match/PatternTranslatorTestBase.scala index 6cb549f1b1..4d48b00282 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/match/PatternTranslatorTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/match/PatternTranslatorTestBase.scala @@ -60,7 +60,7 @@ abstract class PatternTranslatorTestBase extends TestLogger{ val env = StreamExecutionEnvironment.getExecutionEnvironment val tEnv = StreamTableEnvironment.create(env).asInstanceOf[StreamTableEnvironmentImpl] - tEnv.registerDataStream(tableName, dataStreamMock, 'f0, 'proctime.proctime) + tEnv.createTemporaryView(tableName, dataStreamMock, 'f0, 'proctime.proctime) val streamPlanner = tEnv.getPlanner.asInstanceOf[StreamPlanner] diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/AggregateITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/AggregateITCase.scala index f7d0102c0f..48548e1b14 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/AggregateITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/AggregateITCase.scala @@ -50,7 +50,7 @@ class AggregateITCase( val sqlQuery = "SELECT sum(_1), min(_1), max(_1), count(_1), avg(_1) FROM MyTable" val ds = CollectionDataSets.get3TupleDataSet(env) - tEnv.registerDataSet("MyTable", ds) + tEnv.createTemporaryView("MyTable", ds) val result = tEnv.sqlQuery(sqlQuery) @@ -86,7 +86,7 @@ class AggregateITCase( val sqlQuery = "SELECT sum(_1) FROM MyTable" val ds = CollectionDataSets.get3TupleDataSet(env) - tEnv.registerDataSet("MyTable", ds) + tEnv.createTemporaryView("MyTable", ds) val result = tEnv.sqlQuery(sqlQuery) @@ -240,7 +240,7 @@ class AggregateITCase( "SELECT _2, _3, avg(_1) as a, GROUP_ID() as g FROM MyTable GROUP BY GROUPING SETS (_2, _3)" val ds = CollectionDataSets.get3TupleDataSet(env) - tEnv.registerDataSet("MyTable", ds) + tEnv.createTemporaryView("MyTable", ds) val result = tEnv.sqlQuery(sqlQuery).toDataSet[Row].collect() @@ -334,7 +334,7 @@ class AggregateITCase( val ds = CollectionDataSets.get3TupleDataSet(env) // create timestamps .map(x => (x._1, x._2, x._3, toTimestamp(x._1 * 1000))) - tEnv.registerDataSet("T", ds, 'a, 'b, 'c, 'ts) + tEnv.createTemporaryView("T", ds, 'a, 'b, 'c, 'ts) val result = tEnv.sqlQuery(sqlQuery).toDataSet[Row].collect() val expected = Seq( @@ -363,7 +363,7 @@ class AggregateITCase( val ds = CollectionDataSets.get3TupleDataSet(env) // create timestamps .map(x => (x._1, x._2, x._3, toTimestamp(x._1 * 1000))) - tEnv.registerDataSet("T", ds, 'a, 'b, 'c, 'ts) + tEnv.createTemporaryView("T", ds, 'a, 'b, 'c, 'ts) val result = tEnv.sqlQuery(sqlQuery).toDataSet[Row].collect() val expected = Seq( @@ -386,7 +386,7 @@ class AggregateITCase( val ds = CollectionDataSets.get3TupleDataSet(env) // create timestamps .map(x => (x._1, x._2, x._3, toTimestamp(x._1 * 1000))) - tEnv.registerDataSet("t1", ds, 'a, 'b, 'c, 'ts) + tEnv.createTemporaryView("t1", ds, 'a, 'b, 'c, 'ts) val t2 = tEnv.sqlQuery("SELECT b, COLLECT(b) as `set`" + "FROM t1 " + @@ -422,7 +422,7 @@ class AggregateITCase( val ds = CollectionDataSets.get3TupleDataSet(env) // min time unit is seconds .map(x => (x._1, x._2, x._3, toTimestamp(x._1 * 1000))) - tEnv.registerDataSet("T", ds, 'a, 'b, 'c, 'ts) + tEnv.createTemporaryView("T", ds, 'a, 'b, 'c, 'ts) val result = tEnv.sqlQuery(sqlQuery).toDataSet[Row].collect() val expected = Seq( @@ -457,7 +457,7 @@ class AggregateITCase( val ds = CollectionDataSets.get3TupleDataSet(env) // create timestamps .map(x => (x._1, x._2, x._3, toTimestamp(x._1 * 1000))) - tEnv.registerDataSet("T", ds, 'a, 'b, 'c, 'ts) + tEnv.createTemporaryView("T", ds, 'a, 'b, 'c, 'ts) val result = tEnv.sqlQuery(sqlQuery).toDataSet[Row].collect() val expected = Seq( @@ -489,7 +489,7 @@ class AggregateITCase( val ds = CollectionDataSets.get3TupleDataSet(env) // create timestamps .map(x => (x._1, x._2, x._3, toTimestamp(x._1 * 1000))) - tEnv.registerDataSet("T", ds, 'a, 'b, 'c, 'ts) + tEnv.createTemporaryView("T", ds, 'a, 'b, 'c, 'ts) val result = tEnv.sqlQuery(sqlQuery).toDataSet[Row].collect() val expected = Seq( @@ -532,7 +532,7 @@ class AggregateITCase( // create timestamps .filter(x => (x._2 % 2) == 0) .map(x => (x._1, x._2, x._3, toTimestamp(x._1 * 1000))) - tEnv.registerDataSet("T", ds, 'a, 'b, 'c, 'ts) + tEnv.createTemporaryView("T", ds, 'a, 'b, 'c, 'ts) val result = tEnv.sqlQuery(sqlQuery).toDataSet[Row].collect() val expected = Seq( @@ -561,7 +561,7 @@ class AggregateITCase( // create timestamps .filter(x => (x._2 % 2) == 0) .map(x => (x._1, x._2, x._3, toTimestamp(x._1 * 1000))) - tEnv.registerDataSet("T", ds, 'a, 'b, 'c, 'ts) + tEnv.createTemporaryView("T", ds, 'a, 'b, 'c, 'ts) val result = tEnv.sqlQuery(sqlQuery).toDataSet[Row].collect() val expected = Seq( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/CalcITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/CalcITCase.scala index ce3a3719e1..ed9ba019bb 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/CalcITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/CalcITCase.scala @@ -96,7 +96,7 @@ class CalcITCase( val sqlQuery = "SELECT * FROM MyTable" val ds = CollectionDataSets.get3TupleDataSet(env) - tEnv.registerDataSet("MyTable", ds, 'a, 'b, 'c) + tEnv.createTemporaryView("MyTable", ds, 'a, 'b, 'c) val result = tEnv.sqlQuery(sqlQuery) @@ -300,7 +300,7 @@ class CalcITCase( Date.valueOf("1984-07-12"), Time.valueOf("14:34:24"), Timestamp.valueOf("1984-07-12 14:34:24"))) - tEnv.registerDataSet("MyTable", ds, 'a, 'b, 'c) + tEnv.createTemporaryView("MyTable", ds, 'a, 'b, 'c) val result = tEnv.sqlQuery(sqlQuery) @@ -320,7 +320,7 @@ class CalcITCase( val rowValue = ("foo", 12, Timestamp.valueOf("1984-07-12 14:34:24")) val ds = env.fromElements(rowValue) - tEnv.registerDataSet("MyTable", ds, 'a, 'b, 'c) + tEnv.createTemporaryView("MyTable", ds, 'a, 'b, 'c) val result = tEnv.sqlQuery(sqlQuery) @@ -345,7 +345,7 @@ class CalcITCase( tEnv.registerFunction("hashCode", MyHashCode) val ds = env.fromElements("a", "b", "c") - tEnv.registerDataSet("MyTable", ds, 'text) + tEnv.createTemporaryView("MyTable", ds, 'text) val result = tEnv.sqlQuery("SELECT hashCode(text) FROM MyTable") diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/JoinITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/JoinITCase.scala index 05235783db..ce06f25b48 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/JoinITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/JoinITCase.scala @@ -152,8 +152,8 @@ class JoinITCase( val ds1 = CollectionDataSets.getSmall3TupleDataSet(env) val ds2 = CollectionDataSets.get5TupleDataSet(env) - tEnv.registerDataSet("Table3", ds1, 'a, 'b, 'c) - tEnv.registerDataSet("Table5", ds2, 'd, 'e, 'f, 'g, 'h) + tEnv.createTemporaryView("Table3", ds1, 'a, 'b, 'c) + tEnv.createTemporaryView("Table5", ds2, 'd, 'e, 'f, 'g, 'h) val result = tEnv.sqlQuery(sqlQuery) @@ -482,7 +482,7 @@ class JoinITCase( (3, 2L, Array("Hello world", "x")) ) val stream = env.fromCollection(data) - tEnv.registerDataSet("T", stream, 'a, 'b, 'c) + tEnv.createTemporaryView("T", stream, 'a, 'b, 'c) val sqlQuery = "SELECT a, s FROM T, UNNEST(T.c) as A (s)" @@ -554,7 +554,7 @@ class JoinITCase( (3, Array((18, "42.6"))) ) val stream = env.fromCollection(data) - tEnv.registerDataSet("T", stream, 'a, 'b) + tEnv.createTemporaryView("T", stream, 'a, 'b) val sqlQuery = "" + "SELECT a, b, x, y " + diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/SetOperatorsITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/SetOperatorsITCase.scala index 014380f7d3..33c555357b 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/SetOperatorsITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/SetOperatorsITCase.scala @@ -48,8 +48,8 @@ class SetOperatorsITCase( val ds1 = CollectionDataSets.getSmall3TupleDataSet(env) val ds2 = CollectionDataSets.getSmall3TupleDataSet(env) - tEnv.registerDataSet("t1", ds1, 'a, 'b, 'c) - tEnv.registerDataSet("t2", ds2, 'd, 'e, 'f) + tEnv.createTemporaryView("t1", ds1, 'a, 'b, 'c) + tEnv.createTemporaryView("t2", ds2, 'd, 'e, 'f) val result = tEnv.sqlQuery(sqlQuery) @@ -68,8 +68,8 @@ class SetOperatorsITCase( val ds1 = CollectionDataSets.getSmall3TupleDataSet(env) val ds2 = CollectionDataSets.getSmall3TupleDataSet(env) - tEnv.registerDataSet("t1", ds1, 'a, 'b, 'c) - tEnv.registerDataSet("t2", ds2, 'd, 'e, 'f) + tEnv.createTemporaryView("t1", ds1, 'a, 'b, 'c) + tEnv.createTemporaryView("t2", ds2, 'd, 'e, 'f) val result = tEnv.sqlQuery(sqlQuery) @@ -90,8 +90,8 @@ class SetOperatorsITCase( val ds1 = CollectionDataSets.getSmall3TupleDataSet(env) val ds2 = CollectionDataSets.get5TupleDataSet(env) - tEnv.registerDataSet("t1", ds1, 'a, 'b, 'c) - tEnv.registerDataSet("t2", ds2, 'a, 'b, 'd, 'c, 'e) + tEnv.createTemporaryView("t1", ds1, 'a, 'b, 'c) + tEnv.createTemporaryView("t2", ds2, 'a, 'b, 'd, 'c, 'e) val result = tEnv.sqlQuery(sqlQuery) @@ -111,8 +111,8 @@ class SetOperatorsITCase( val ds1 = CollectionDataSets.getSmall3TupleDataSet(env) val ds2 = CollectionDataSets.get5TupleDataSet(env) - tEnv.registerDataSet("t1", ds1, 'a, 'b, 'c) - tEnv.registerDataSet("t2", ds2, 'a, 'b, 'd, 'c, 'e) + tEnv.createTemporaryView("t1", ds1, 'a, 'b, 'c) + tEnv.createTemporaryView("t2", ds2, 'a, 'b, 'd, 'c, 'e) val result = tEnv.sqlQuery(sqlQuery) @@ -147,8 +147,8 @@ class SetOperatorsITCase( val ds1 = CollectionDataSets.getSmall3TupleDataSet(env) val ds2 = env.fromElements((1, 1L, "Hi")) - tEnv.registerDataSet("t1", ds1, 'a, 'b, 'c) - tEnv.registerDataSet("t2", ds2, 'a, 'b, 'c) + tEnv.createTemporaryView("t1", ds1, 'a, 'b, 'c) + tEnv.createTemporaryView("t2", ds2, 'a, 'b, 'c) val result = tEnv.sqlQuery(sqlQuery) @@ -171,8 +171,8 @@ class SetOperatorsITCase( val ds1 = env.fromCollection(data1) val ds2 = env.fromCollection(data2) - tEnv.registerDataSet("t1", ds1, 'c) - tEnv.registerDataSet("t2", ds2, 'c) + tEnv.createTemporaryView("t1", ds1, 'c) + tEnv.createTemporaryView("t2", ds2, 'c) val result = tEnv.sqlQuery(sqlQuery) @@ -193,8 +193,8 @@ class SetOperatorsITCase( val ds1 = CollectionDataSets.getSmall3TupleDataSet(env) val ds2 = CollectionDataSets.get5TupleDataSet(env) - tEnv.registerDataSet("t1", ds1, 'a, 'b, 'c) - tEnv.registerDataSet("t2", ds2, 'a, 'b, 'd, 'c, 'e) + tEnv.createTemporaryView("t1", ds1, 'a, 'b, 'c) + tEnv.createTemporaryView("t2", ds2, 'a, 'b, 'd, 'c, 'e) val result = tEnv.sqlQuery(sqlQuery) @@ -218,8 +218,8 @@ class SetOperatorsITCase( data.+=((3, 2L, "Hello world!")) val ds2 = env.fromCollection(Random.shuffle(data)) - tEnv.registerDataSet("t1", ds1, 'a, 'b, 'c) - tEnv.registerDataSet("t2", ds2, 'a, 'b, 'c) + tEnv.createTemporaryView("t1", ds1, 'a, 'b, 'c) + tEnv.createTemporaryView("t2", ds2, 'a, 'b, 'c) val result = tEnv.sqlQuery(sqlQuery) @@ -244,8 +244,8 @@ class SetOperatorsITCase( val ds1 = env.fromCollection(data1) val ds2 = env.fromCollection(data2) - tEnv.registerDataSet("t1", ds1, 'c) - tEnv.registerDataSet("t2", ds2, 'c) + tEnv.createTemporaryView("t1", ds1, 'c) + tEnv.createTemporaryView("t2", ds2, 'c) val result = tEnv.sqlQuery(sqlQuery) @@ -264,8 +264,8 @@ class SetOperatorsITCase( val ds1 = CollectionDataSets.getSmall3TupleDataSet(env) val ds2 = CollectionDataSets.get3TupleDataSet(env) - tEnv.registerDataSet("t1", ds1, 'a, 'b, 'c) - tEnv.registerDataSet("t2", ds2, 'a, 'b, 'c) + tEnv.createTemporaryView("t1", ds1, 'a, 'b, 'c) + tEnv.createTemporaryView("t2", ds2, 'a, 'b, 'c) val result = tEnv.sqlQuery(sqlQuery) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/SortITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/SortITCase.scala index 0e35ac3009..e1c100be3f 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/SortITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/SortITCase.scala @@ -57,7 +57,7 @@ class SortITCase(mode: TestExecutionMode, configMode: TableConfigMode) (- x.productElement(0).asInstanceOf[Int], - x.productElement(1).asInstanceOf[Long])) val ds = CollectionDataSets.get3TupleDataSet(env) - tEnv.registerDataSet("MyTable", ds) + tEnv.createTemporaryView("MyTable", ds) val expected = sortExpectedly(tupleDataSetStrings) // squash all rows inside a partition into one element @@ -94,7 +94,7 @@ class SortITCase(mode: TestExecutionMode, configMode: TableConfigMode) - x.productElement(0).asInstanceOf[Int] ) val ds = CollectionDataSets.get3TupleDataSet(env) - tEnv.registerDataSet("MyTable", ds) + tEnv.createTemporaryView("MyTable", ds) val expected = sortExpectedly(tupleDataSetStrings, 2, 21) // squash all rows inside a partition into one element @@ -125,7 +125,7 @@ class SortITCase(mode: TestExecutionMode, configMode: TableConfigMode) x.productElement(0).asInstanceOf[Int] ) val ds = CollectionDataSets.get3TupleDataSet(env) - tEnv.registerDataSet("MyTable", ds) + tEnv.createTemporaryView("MyTable", ds) val expected = sortExpectedly(tupleDataSetStrings, 2, 7) // squash all rows inside a partition into one element @@ -156,7 +156,7 @@ class SortITCase(mode: TestExecutionMode, configMode: TableConfigMode) (x.productElement(1).asInstanceOf[Long], x.productElement(0).asInstanceOf[Int]) ) val ds = CollectionDataSets.get3TupleDataSet(env) - tEnv.registerDataSet("MyTable", ds) + tEnv.createTemporaryView("MyTable", ds) val expected = sortExpectedly(tupleDataSetStrings, 0, 5) // squash all rows inside a partition into one element diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/TableEnvironmentITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/TableEnvironmentITCase.scala index 93e37e492b..1b7f15b847 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/TableEnvironmentITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/TableEnvironmentITCase.scala @@ -45,7 +45,7 @@ class TableEnvironmentITCase( val tEnv = BatchTableEnvironment.create(env, config) val ds = CollectionDataSets.get3TupleDataSet(env) - tEnv.registerDataSet("MyTable", ds, 'a, 'b, 'c) + tEnv.createTemporaryView("MyTable", ds, 'a, 'b, 'c) val sqlQuery = "SELECT * FROM MyTable WHERE a > 9" diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/table/AggregateITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/table/AggregateITCase.scala index 41acc11c63..9f1d07a85f 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/table/AggregateITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/table/AggregateITCase.scala @@ -49,7 +49,7 @@ class AggregationsITCase( val tEnv = BatchTableEnvironment.create(env, config) val inputTable = CollectionDataSets.getSmallNestedTupleDataSet(env).toTable(tEnv, 'a, 'b) - tEnv.registerDataSet("MyTable", inputTable) + tEnv.createTemporaryView("MyTable", inputTable) val result = tEnv.scan("MyTable") .where('a.get("_1") > 0) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/table/CalcITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/table/CalcITCase.scala index 555f331b79..97524d87ad 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/table/CalcITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/table/CalcITCase.scala @@ -439,7 +439,7 @@ class CalcITCase( UserDefinedFunctionTestUtils.setJobParameters(env, Map("string.value" -> "ABC")) val ds = CollectionDataSets.getSmall3TupleDataSet(env) - tEnv.registerDataSet("t1", ds, 'a, 'b, 'c) + tEnv.createTemporaryView("t1", ds, 'a, 'b, 'c) val sqlQuery = "SELECT c FROM t1 where RichFunc2(c)='ABC#Hello'" @@ -460,7 +460,7 @@ class CalcITCase( tEnv.registerFunction("RichFunc3", new RichFunc3) val ds = CollectionDataSets.getSmall3TupleDataSet(env) - tEnv.registerDataSet("t1", ds, 'a, 'b, 'c) + tEnv.createTemporaryView("t1", ds, 'a, 'b, 'c) val sqlQuery = "SELECT c FROM t1 where RichFunc3(c)=true" @@ -508,7 +508,7 @@ class CalcITCase( UserDefinedFunctionTestUtils.setJobParameters(env, Map("string.value" -> "Abc")) val ds = CollectionDataSets.getSmall3TupleDataSet(env) - tEnv.registerDataSet("t1", ds, 'a, 'b, 'c) + tEnv.createTemporaryView("t1", ds, 'a, 'b, 'c) val sqlQuery = "SELECT c FROM t1 where " + "RichFunc2(c)='Abc#Hello' or RichFunc1(a)=3 and b=2" diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/table/TableEnvironmentITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/table/TableEnvironmentITCase.scala index 967cd6a95b..7090db102d 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/table/TableEnvironmentITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/table/TableEnvironmentITCase.scala @@ -48,7 +48,7 @@ class TableEnvironmentITCase( val tEnv = BatchTableEnvironment.create(env, config) val ds = CollectionDataSets.get3TupleDataSet(env) - tEnv.registerDataSet(tableName, ds) + tEnv.createTemporaryView(tableName, ds) val t = tEnv.scan(tableName).select('_1, '_2, '_3) val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + @@ -69,7 +69,7 @@ class TableEnvironmentITCase( val tEnv = BatchTableEnvironment.create(env, config) val ds = CollectionDataSets.get3TupleDataSet(env) - tEnv.registerDataSet(tableName, ds, 'a, 'b, 'c) // new alias + tEnv.createTemporaryView(tableName, ds, 'a, 'b, 'c) // new alias val t = tEnv.scan(tableName).select('a, 'b) val expected = "1,1\n" + "2,2\n" + "3,2\n" + "4,3\n" + "5,3\n" + "6,3\n" + @@ -87,7 +87,7 @@ class TableEnvironmentITCase( val tEnv = BatchTableEnvironment.create(env, config) val ds = CollectionDataSets.get3TupleDataSet(env) - tEnv.registerDataSet(tableName, ds, '_3, '_1, '_2) // new order + tEnv.createTemporaryView(tableName, ds, '_3, '_1, '_2) // new order val t = tEnv.scan(tableName).select('_1, '_2) val expected = "1,1\n" + "2,2\n" + "3,2\n" + "4,3\n" + "5,3\n" + "6,3\n" + diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/sql/InsertIntoITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/sql/InsertIntoITCase.scala index 87795447ba..1a464f7fb3 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/sql/InsertIntoITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/sql/InsertIntoITCase.scala @@ -46,7 +46,7 @@ class InsertIntoITCase extends StreamingWithStateTestBase { val input = StreamTestData.get3TupleDataStream(env) .assignAscendingTimestamps(r => r._2) - tEnv.registerDataStream("sourceTable", input, 'a, 'b, 'c, 't.rowtime) + tEnv.createTemporaryView("sourceTable", input, 'a, 'b, 'c, 't.rowtime) val fieldNames = Array("d", "e", "t") val fieldTypes: Array[TypeInformation[_]] = Array(Types.STRING, Types.SQL_TIMESTAMP, Types.LONG) @@ -82,7 +82,7 @@ class InsertIntoITCase extends StreamingWithStateTestBase { val t = StreamTestData.get3TupleDataStream(env) .assignAscendingTimestamps(_._1.toLong) - tEnv.registerDataStream("sourceTable", t, 'id, 'num, 'text) + tEnv.createTemporaryView("sourceTable", t, 'id, 'num, 'text) tEnv.registerTableSink( "targetTable", new TestRetractSink().configure( @@ -122,7 +122,7 @@ class InsertIntoITCase extends StreamingWithStateTestBase { val t = StreamTestData.get3TupleDataStream(env) .assignAscendingTimestamps(_._1.toLong) - tEnv.registerDataStream("sourceTable", t, 'id, 'num, 'text, 'rowtime.rowtime) + tEnv.createTemporaryView("sourceTable", t, 'id, 'num, 'text, 'rowtime.rowtime) tEnv.registerTableSink( "targetTable", new TestRetractSink().configure( @@ -170,7 +170,7 @@ class InsertIntoITCase extends StreamingWithStateTestBase { val t = StreamTestData.get3TupleDataStream(env) .assignAscendingTimestamps(_._1.toLong) - tEnv.registerDataStream("sourceTable", t, 'id, 'num, 'text) + tEnv.createTemporaryView("sourceTable", t, 'id, 'num, 'text) tEnv.registerTableSink( "targetTable", new TestUpsertSink(Array("cnt", "cTrue"), false).configure( @@ -217,7 +217,7 @@ class InsertIntoITCase extends StreamingWithStateTestBase { val t = StreamTestData.get3TupleDataStream(env) .assignAscendingTimestamps(_._1.toLong) - tEnv.registerDataStream("sourceTable", t, 'id, 'num, 'text, 'rowtime.rowtime) + tEnv.createTemporaryView("sourceTable", t, 'id, 'num, 'text, 'rowtime.rowtime) tEnv.registerTableSink( "targetTable", new TestUpsertSink(Array("wend", "num"), true).configure( @@ -268,7 +268,7 @@ class InsertIntoITCase extends StreamingWithStateTestBase { val t = StreamTestData.get3TupleDataStream(env) .assignAscendingTimestamps(_._1.toLong) - tEnv.registerDataStream("sourceTable", t, 'id, 'num, 'text, 'rowtime.rowtime) + tEnv.createTemporaryView("sourceTable", t, 'id, 'num, 'text, 'rowtime.rowtime) tEnv.registerTableSink( "targetTable", new TestUpsertSink(Array("wstart", "wend", "num"), true).configure( @@ -320,7 +320,7 @@ class InsertIntoITCase extends StreamingWithStateTestBase { val t = StreamTestData.get3TupleDataStream(env) .assignAscendingTimestamps(_._1.toLong) - tEnv.registerDataStream("sourceTable", t, 'id, 'num, 'text, 'rowtime.rowtime) + tEnv.createTemporaryView("sourceTable", t, 'id, 'num, 'text, 'rowtime.rowtime) tEnv.registerTableSink( "targetTable", new TestUpsertSink(null, true).configure( @@ -370,7 +370,7 @@ class InsertIntoITCase extends StreamingWithStateTestBase { val t = StreamTestData.get3TupleDataStream(env) .assignAscendingTimestamps(_._1.toLong) - tEnv.registerDataStream("sourceTable", t, 'id, 'num, 'text, 'rowtime.rowtime) + tEnv.createTemporaryView("sourceTable", t, 'id, 'num, 'text, 'rowtime.rowtime) tEnv.registerTableSink( "targetTable", new TestUpsertSink(null, true).configure( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala index c18fff18b2..a0f4118029 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala @@ -461,7 +461,7 @@ class SqlITCase extends StreamingWithStateTestBase { val sqlQuery = "SELECT * FROM MyTable WHERE _1 = 3" val t = StreamTestData.getSmall3TupleDataStream(env) - tEnv.registerDataStream("MyTable", t) + tEnv.createTemporaryView("MyTable", t) val result = tEnv.sqlQuery(sqlQuery).toAppendStream[Row] result.addSink(new StreamITCase.StringSink[Row]) @@ -538,7 +538,7 @@ class SqlITCase extends StreamingWithStateTestBase { val t1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv).as('a, 'b, 'c) tEnv.registerTable("T1", t1) val t2 = StreamTestData.get3TupleDataStream(env) - tEnv.registerDataStream("T2", t2, 'a, 'b, 'c) + tEnv.createTemporaryView("T2", t2, 'a, 'b, 'c) val result = tEnv.sqlQuery(sqlQuery).toAppendStream[Row] result.addSink(new StreamITCase.StringSink[Row]) @@ -560,7 +560,7 @@ class SqlITCase extends StreamingWithStateTestBase { (3, Array(18, 42), Array(Array(1), Array(45))) ) val stream = env.fromCollection(data) - tEnv.registerDataStream("T", stream, 'a, 'b, 'c) + tEnv.createTemporaryView("T", stream, 'a, 'b, 'c) val sqlQuery = "SELECT a, b, s FROM T, UNNEST(T.b) AS A (s)" @@ -591,7 +591,7 @@ class SqlITCase extends StreamingWithStateTestBase { (3, Array(18, 42), Array(Array(1), Array(45))) ) val stream = env.fromCollection(data) - tEnv.registerDataStream("T", stream, 'a, 'b, 'c) + tEnv.createTemporaryView("T", stream, 'a, 'b, 'c) val sqlQuery = "SELECT a, s FROM T, UNNEST(T.c) AS A (s)" @@ -620,7 +620,7 @@ class SqlITCase extends StreamingWithStateTestBase { (3, Array((18, "42.6"))) ) val stream = env.fromCollection(data) - tEnv.registerDataStream("T", stream, 'a, 'b) + tEnv.createTemporaryView("T", stream, 'a, 'b) val sqlQuery = "SELECT a, b, s, t FROM T, UNNEST(T.b) AS A (s, t) WHERE s > 13" diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala index a2dbb2bdcd..e2b916ae48 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala @@ -18,8 +18,6 @@ package org.apache.flink.table.utils -import org.apache.calcite.plan.RelOptUtil -import org.apache.calcite.rel.RelNode import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.{LocalEnvironment, DataSet => JDataSet} import org.apache.flink.api.scala.{DataSet, ExecutionEnvironment} @@ -31,7 +29,7 @@ import org.apache.flink.table.api.internal.{TableEnvImpl, TableEnvironmentImpl, import org.apache.flink.table.api.java.internal.{BatchTableEnvironmentImpl => JavaBatchTableEnvironmentImpl, StreamTableEnvironmentImpl => JavaStreamTableEnvironmentImpl} import org.apache.flink.table.api.scala._ import org.apache.flink.table.api.scala.internal.{BatchTableEnvironmentImpl => ScalaBatchTableEnvironmentImpl, StreamTableEnvironmentImpl => ScalaStreamTableEnvironmentImpl} -import org.apache.flink.table.api.{Table, TableConfig, TableSchema} +import org.apache.flink.table.api.{ApiExpression, Table, TableConfig, TableSchema} import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog} import org.apache.flink.table.executor.StreamExecutor import org.apache.flink.table.expressions.Expression @@ -39,6 +37,9 @@ import org.apache.flink.table.functions.{AggregateFunction, ScalarFunction, Tabl import org.apache.flink.table.module.ModuleManager import org.apache.flink.table.operations.{DataSetQueryOperation, JavaDataStreamQueryOperation, ScalaDataStreamQueryOperation} import org.apache.flink.table.planner.StreamPlanner + +import org.apache.calcite.plan.RelOptUtil +import org.apache.calcite.rel.RelNode import org.junit.Assert.assertEquals import org.junit.rules.ExpectedException import org.junit.{ComparisonFailure, Rule} @@ -248,13 +249,11 @@ case class BatchTableTestUtil( tableEnv.registerTable(name, t) t } - - def addJavaTable[T](typeInfo: TypeInformation[T], name: String, fields: String): Table = { - + def addJavaTable[T](typeInfo: TypeInformation[T], name: String, fields: ApiExpression*): Table = { val jDs = mock(classOf[JDataSet[T]]) when(jDs.getType).thenReturn(typeInfo) - val t = javaTableEnv.fromDataSet(jDs, fields) + val t = javaTableEnv.fromDataSet(jDs, fields: _*) javaTableEnv.registerTable(name, t) t } @@ -361,9 +360,9 @@ case class StreamTableTestUtil( table } - def addJavaTable[T](typeInfo: TypeInformation[T], name: String, fields: String): Table = { + def addJavaTable[T](typeInfo: TypeInformation[T], name: String, fields: ApiExpression*): Table = { val stream = javaEnv.addSource(new EmptySource[T], typeInfo) - val table = javaTableEnv.fromDataStream(stream, fields) + val table = javaTableEnv.fromDataStream(stream, fields: _*) javaTableEnv.registerTable(name, table) table } -- Gitee From fab6c2a1b50687b3d2953f28183da2cba3c54377 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Thu, 26 Mar 2020 10:14:21 +0100 Subject: [PATCH 408/885] [FLINK-16036][table-api] Deprecate String based Expression DSL in TableEnvironments This closes #11527 --- .../table/api/java/BatchTableEnvironment.java | 151 +++++++++++++- .../api/java/StreamTableEnvironment.java | 186 +++++++++++++++++- .../api/scala/BatchTableEnvironment.scala | 100 +++++++++- .../api/scala/StreamTableEnvironment.scala | 121 +++++++++++- 4 files changed, 527 insertions(+), 31 deletions(-) diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/BatchTableEnvironment.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/BatchTableEnvironment.java index 1442e33ebe..dac1a57456 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/BatchTableEnvironment.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/BatchTableEnvironment.java @@ -93,36 +93,90 @@ public interface BatchTableEnvironment extends TableEnvironment { /** * Converts the given {@link DataSet} into a {@link Table} with specified field names. * + *

    There are two modes for mapping original fields to the fields of the {@link Table}: + * + *

    1. Reference input fields by name: + * All fields in the schema definition are referenced by name + * (and possibly renamed using an alias (as). In this mode, fields can be reordered and + * projected out. This mode can be used for any input type, including POJOs. + * *

    Example: * *

     	 * {@code
     	 *   DataSet> set = ...
    -	 *   Table tab = tableEnv.fromDataSet(set, "a, b");
    +	 *   // use the original 'f0' field and give a better name to the 'f1' field
    +	 *   Table table = tableEnv.fromTable(set, "f0, f1 as name");
    +	 * }
    +	 * 
    + * + *

    2. Reference input fields by position: + * In this mode, fields are simply renamed. This mode can only be + * used if the input type has a defined field order (tuple, case class, Row) and none of + * the {@code fields} references a field of the input type. + * + *

    Example: + * + *

    +	 * {@code
    +	 *   DataSet> set = ...
    +	 *   // renames the original fields as 'a' and 'b'
    +	 *   Table table = tableEnv.fromDataSet(set, "a, b");
     	 * }
     	 * 
    * * @param dataSet The {@link DataSet} to be converted. - * @param fields The field names of the resulting {@link Table}. + * @param fields The fields expressions to map original fields of the DataSet to the fields of the {@link Table}. * @param The type of the {@link DataSet}. * @return The converted {@link Table}. + * @deprecated use {@link #fromDataSet(DataSet, Expression...)} */ + @Deprecated Table fromDataSet(DataSet dataSet, String fields); /** * Converts the given {@link DataSet} into a {@link Table} with specified field names. * - * Example: + *

    There are two modes for mapping original fields to the fields of the {@link Table}: + * + *

    1. Reference input fields by name: + * All fields in the schema definition are referenced by name + * (and possibly renamed using an alias (as). In this mode, fields can be reordered and + * projected out. This mode can be used for any input type, including POJOs. + * + *

    Example: * *

     	 * {@code
     	 *   DataSet> set = ...
    -	 *   Table tab = tableEnv.fromDataSet(set, $("a"), $("b").as("name"), $("timestamp").rowtime());
    +	 *   Table table = tableEnv.fromDataSet(
    +	 *      set,
    +	 *      $("f1"), // reorder and use the original field
    +	 *      $("f0").as("name") // reorder and give the original field a better name
    +	 *   );
    +	 * }
    +	 * 
    + * + *

    2. Reference input fields by position: + * In this mode, fields are simply renamed. This mode can only be + * used if the input type has a defined field order (tuple, case class, Row) and none of + * the {@code fields} references a field of the input type. + * + *

    Example: + * + *

    +	 * {@code
    +	 *   DataSet> set = ...
    +	 *   Table table = tableEnv.fromDataSet(
    +	 *      set,
    +	 *      $("a"), // renames the first field to 'a'
    +	 *      $("b") // renames the second field to 'b'
    +	 *   );
     	 * }
     	 * 
    * * @param dataSet The {@link DataSet} to be converted. - * @param fields The field names of the resulting {@link Table}. + * @param fields The fields expressions to map original fields of the DataSet to the fields of the {@link Table}. * @param The type of the {@link DataSet}. * @return The converted {@link Table}. */ @@ -172,11 +226,34 @@ public interface BatchTableEnvironment extends TableEnvironment { * Creates a view from the given {@link DataSet} in a given path with specified field names. * Registered views can be referenced in SQL queries. * + *

    There are two modes for mapping original fields to the fields of the View: + * + *

    1. Reference input fields by name: + * All fields in the schema definition are referenced by name + * (and possibly renamed using an alias (as). In this mode, fields can be reordered and + * projected out. This mode can be used for any input type, including POJOs. + * + *

    Example: + * + *

    +	 * {@code
    +	 *   DataSet> set = ...
    +	 *   // use the original 'f0' field and give a better name to the 'f1' field
    +	 *   tableEnv.registerDataSet("myTable", set, "f0, f1 as name");
    +	 * }
    +	 * 
    + * + *

    2. Reference input fields by position: + * In this mode, fields are simply renamed. This mode can only be + * used if the input type has a defined field order (tuple, case class, Row) and none of the {@code fields} + * references a field of the input type. + * *

    Example: * *

     	 * {@code
     	 *   DataSet> set = ...
    +	 *   // renames the original fields as 'a' and 'b'
     	 *   tableEnv.registerDataSet("myTable", set, "a, b");
     	 * }
     	 * 
    @@ -190,7 +267,7 @@ public interface BatchTableEnvironment extends TableEnvironment { * * @param name The name under which the {@link DataSet} is registered in the catalog. * @param dataSet The {@link DataSet} to register. - * @param fields The field names of the registered view. + * @param fields The fields expressions to map original fields of the DataSet to the fields of the View. * @param The type of the {@link DataSet} to register. * @deprecated use {@link #createTemporaryView(String, DataSet, String)} */ @@ -201,11 +278,34 @@ public interface BatchTableEnvironment extends TableEnvironment { * Creates a view from the given {@link DataSet} in a given path with specified field names. * Registered views can be referenced in SQL queries. * + *

    There are two modes for mapping original fields to the fields of the View: + * + *

    1. Reference input fields by name: + * All fields in the schema definition are referenced by name + * (and possibly renamed using an alias (as). In this mode, fields can be reordered and + * projected out. This mode can be used for any input type, including POJOs. + * *

    Example: * *

     	 * {@code
     	 *   DataSet> set = ...
    +	 *   // use the original 'f0' field and give a better name to the 'f1' field
    +	 *   tableEnv.createTemporaryView("cat.db.myTable", set, "f0, f1 as name");
    +	 * }
    +	 * 
    + * + *

    2. Reference input fields by position: + * In this mode, fields are simply renamed. This mode can only be + * used if the input type has a defined field order (tuple, case class, Row) and none of the {@code fields} + * references a field of the input type. + * + *

    Example: + * + *

    +	 * {@code
    +	 *   DataSet> set = ...
    +	 *   // renames the original fields as 'a' and 'b'
     	 *   tableEnv.createTemporaryView("cat.db.myTable", set, "a, b");
     	 * }
     	 * 
    @@ -217,21 +317,54 @@ public interface BatchTableEnvironment extends TableEnvironment { * @param path The path under which the view is created. * See also the {@link TableEnvironment} class description for the format of the path. * @param dataSet The {@link DataSet} out of which to create the view. - * @param fields The field names of the registered view. + * @param fields The fields expressions to map original fields of the DataSet to the fields of the View. * @param The type of the {@link DataSet}. + * @deprecated use {@link #createTemporaryView(String, DataSet, Expression...)} */ + @Deprecated void createTemporaryView(String path, DataSet dataSet, String fields); /** * Creates a view from the given {@link DataSet} in a given path with specified field names. * Registered views can be referenced in SQL queries. * + *

    There are two modes for mapping original fields to the fields of the View: + * + *

    1. Reference input fields by name: + * All fields in the schema definition are referenced by name + * (and possibly renamed using an alias (as). In this mode, fields can be reordered and + * projected out. This mode can be used for any input type, including POJOs. + * + *

    Example: + * + *

    +	 * {@code
    +	 *   DataSet> set = ...
    +	 *   tableEnv.createTemporaryView(
    +	 *      "cat.db.myTable",
    +	 *      set,
    +	 *      $("f1"), // reorder and use the original field
    +	 *      $("f0").as("name") // reorder and give the original field a better name
    +	 *   );
    +	 * }
    +	 * 
    + * + *

    2. Reference input fields by position: + * In this mode, fields are simply renamed. This mode can only be + * used if the input type has a defined field order (tuple, case class, Row) and none of + * the {@code fields} references a field of the input type. + * *

    Example: * *

     	 * {@code
     	 *   DataSet> set = ...
    -	 *   tableEnv.createTemporaryView("cat.db.myTable", set, $("b").as("name"), $("timestamp").rowtime());
    +	 *   tableEnv.createTemporaryView(
    +	 *      "cat.db.myTable",
    +	 *      set,
    +	 *      $("a"), // renames the first field to 'a'
    +	 *      $("b") // renames the second field to 'b'
    +	 *   );
     	 * }
     	 * 
    * @@ -242,7 +375,7 @@ public interface BatchTableEnvironment extends TableEnvironment { * @param path The path under which the view is created. * See also the {@link TableEnvironment} class description for the format of the path. * @param dataSet The {@link DataSet} out of which to create the view. - * @param fields The field names of the registered view. + * @param fields The fields expressions to map original fields of the DataSet to the fields of the View. * @param The type of the {@link DataSet}. */ void createTemporaryView(String path, DataSet dataSet, Expression... fields); diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/StreamTableEnvironment.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/StreamTableEnvironment.java index 92cb111ac0..364e619148 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/StreamTableEnvironment.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/StreamTableEnvironment.java @@ -195,36 +195,104 @@ public interface StreamTableEnvironment extends TableEnvironment { /** * Converts the given {@link DataStream} into a {@link Table} with specified field names. * + *

    There are two modes for mapping original fields to the fields of the {@link Table}: + * + *

    1. Reference input fields by name: + * All fields in the schema definition are referenced by name + * (and possibly renamed using an alias (as). Moreover, we can define proctime and rowtime + * attributes at arbitrary positions using arbitrary names (except those that exist in the + * result schema). In this mode, fields can be reordered and projected out. This mode can + * be used for any input type, including POJOs. + * *

    Example: * *

     	 * {@code
     	 *   DataStream> stream = ...
    -	 *   Table tab = tableEnv.fromDataStream(stream, "a, b");
    +	 *   // reorder the fields, rename the original 'f0' field to 'name' and add event-time
    +	 *   // attribute named 'rowtime'
    +	 *   Table table = tableEnv.fromDataStream(stream, "f1, rowtime.rowtime, f0 as 'name'");
    +	 * }
    +	 * 
    + * + *

    2. Reference input fields by position: + * In this mode, fields are simply renamed. Event-time attributes can + * replace the field on their position in the input data (if it is of correct type) or be + * appended at the end. Proctime attributes must be appended at the end. This mode can only be + * used if the input type has a defined field order (tuple, case class, Row) and none of + * the {@code fields} references a field of the input type. + * + *

    Example: + * + *

    +	 * {@code
    +	 *   DataStream> stream = ...
    +	 *   // rename the original fields to 'a' and 'b' and extract the internally attached timestamp into an event-time
    +	 *   // attribute named 'rowtime'
    +	 *   Table table = tableEnv.fromDataStream(stream, "a, b, rowtime.rowtime");
     	 * }
     	 * 
    * * @param dataStream The {@link DataStream} to be converted. - * @param fields The field names of the resulting {@link Table}. + * @param fields The fields expressions to map original fields of the DataStream to the fields of the {@link Table}. * @param The type of the {@link DataStream}. * @return The converted {@link Table}. + * @deprecated use {@link #fromDataStream(DataStream, Expression...)} */ + @Deprecated Table fromDataStream(DataStream dataStream, String fields); /** * Converts the given {@link DataStream} into a {@link Table} with specified field names. * + *

    There are two modes for mapping original fields to the fields of the {@link Table}: + * + *

    1. Reference input fields by name: + * All fields in the schema definition are referenced by name + * (and possibly renamed using an alias (as). Moreover, we can define proctime and rowtime + * attributes at arbitrary positions using arbitrary names (except those that exist in the + * result schema). In this mode, fields can be reordered and projected out. This mode can + * be used for any input type, including POJOs. + * *

    Example: * *

     	 * {@code
     	 *   DataStream> stream = ...
    -	 *   Table tab = tableEnv.fromDataStream(stream, $("a"), $("b").as("name"), $("timestamp").rowtime());
    +	 *   Table table = tableEnv.fromDataStream(
    +	 *      stream,
    +	 *      $("f1"), // reorder and use the original field
    +	 *      $("rowtime").rowtime(), // extract the internally attached timestamp into an event-time
    +	 *                              // attribute named 'rowtime'
    +	 *      $("f0").as("name") // reorder and give the original field a better name
    +	 *   );
    +	 * }
    +	 * 
    + * + *

    2. Reference input fields by position: + * In this mode, fields are simply renamed. Event-time attributes can + * replace the field on their position in the input data (if it is of correct type) or be + * appended at the end. Proctime attributes must be appended at the end. This mode can only be + * used if the input type has a defined field order (tuple, case class, Row) and none of + * the {@code fields} references a field of the input type. + * + *

    Example: + * + *

    +	 * {@code
    +	 *   DataStream> stream = ...
    +	 *   Table table = tableEnv.fromDataStream(
    +	 *      stream,
    +	 *      $("a"), // rename the first field to 'a'
    +	 *      $("b"), // rename the second field to 'b'
    +	 *      $("rowtime").rowtime() // extract the internally attached timestamp into an event-time
    +	 *                             // attribute named 'rowtime'
    +	 *   );
     	 * }
     	 * 
    * * @param dataStream The {@link DataStream} to be converted. - * @param fields The field expressions of the resulting {@link Table}. + * @param fields The fields expressions to map original fields of the DataStream to the fields of the {@code Table}. * @param The type of the {@link DataStream}. * @return The converted {@link Table}. */ @@ -274,12 +342,41 @@ public interface StreamTableEnvironment extends TableEnvironment { * Creates a view from the given {@link DataStream} in a given path with specified field names. * Registered views can be referenced in SQL queries. * + *

    There are two modes for mapping original fields to the fields of the View: + * + *

    1. Reference input fields by name: + * All fields in the schema definition are referenced by name + * (and possibly renamed using an alias (as). Moreover, we can define proctime and rowtime + * attributes at arbitrary positions using arbitrary names (except those that exist in the + * result schema). In this mode, fields can be reordered and projected out. This mode can + * be used for any input type, including POJOs. + * + *

    Example: + * + *

    +	 * {@code
    +	 *   DataStream> stream = ...
    +	 *   // reorder the fields, rename the original 'f0' field to 'name' and add event-time
    +	 *   // attribute named 'rowtime'
    +	 *   tableEnv.registerDataStream("myTable", stream, "f1, rowtime.rowtime, f0 as 'name'");
    +	 * }
    +	 * 
    + * + *

    2. Reference input fields by position: + * In this mode, fields are simply renamed. Event-time attributes can + * replace the field on their position in the input data (if it is of correct type) or be + * appended at the end. Proctime attributes must be appended at the end. This mode can only be + * used if the input type has a defined field order (tuple, case class, Row) and none of + * the {@code fields} references a field of the input type. + * *

    Example: * *

     	 * {@code
     	 *   DataStream> stream = ...
    -	 *   tableEnv.registerDataStream("myTable", stream, "a, b")
    +	 *   // rename the original fields to 'a' and 'b' and extract the internally attached timestamp into an event-time
    +	 *   // attribute named 'rowtime'
    +	 *   tableEnv.registerDataStream("myTable", stream, "a, b, rowtime.rowtime");
     	 * }
     	 * 
    * @@ -292,7 +389,7 @@ public interface StreamTableEnvironment extends TableEnvironment { * * @param name The name under which the {@link DataStream} is registered in the catalog. * @param dataStream The {@link DataStream} to register. - * @param fields The field names of the registered view. + * @param fields The fields expressions to map original fields of the DataStream to the fields of the View. * @param The type of the {@link DataStream} to register. * @deprecated use {@link #createTemporaryView(String, DataStream, Expression...)} */ @@ -303,12 +400,41 @@ public interface StreamTableEnvironment extends TableEnvironment { * Creates a view from the given {@link DataStream} in a given path with specified field names. * Registered views can be referenced in SQL queries. * + *

    There are two modes for mapping original fields to the fields of the View: + * + *

    1. Reference input fields by name: + * All fields in the schema definition are referenced by name + * (and possibly renamed using an alias (as). Moreover, we can define proctime and rowtime + * attributes at arbitrary positions using arbitrary names (except those that exist in the + * result schema). In this mode, fields can be reordered and projected out. This mode can + * be used for any input type, including POJOs. + * *

    Example: * *

     	 * {@code
     	 *   DataStream> stream = ...
    -	 *   tableEnv.createTemporaryView("cat.db.myTable", stream, "a, b")
    +	 *   // reorder the fields, rename the original 'f0' field to 'name' and add event-time
    +	 *   // attribute named 'rowtime'
    +	 *   tableEnv.createTemporaryView("cat.db.myTable", stream, "f1, rowtime.rowtime, f0 as 'name'");
    +	 * }
    +	 * 
    + * + *

    2. Reference input fields by position: + * In this mode, fields are simply renamed. Event-time attributes can + * replace the field on their position in the input data (if it is of correct type) or be + * appended at the end. Proctime attributes must be appended at the end. This mode can only be + * used if the input type has a defined field order (tuple, case class, Row) and none of + * the {@code fields} references a field of the input type. + * + *

    Example: + * + *

    +	 * {@code
    +	 *   DataStream> stream = ...
    +	 *   // rename the original fields to 'a' and 'b' and extract the internally attached timestamp into an event-time
    +	 *   // attribute named 'rowtime'
    +	 *   tableEnv.createTemporaryView("cat.db.myTable", stream, "a, b, rowtime.rowtime");
     	 * }
     	 * 
    * @@ -319,21 +445,61 @@ public interface StreamTableEnvironment extends TableEnvironment { * @param path The path under which the {@link DataStream} is created. * See also the {@link TableEnvironment} class description for the format of the path. * @param dataStream The {@link DataStream} out of which to create the view. - * @param fields The field names of the created view. + * @param fields The fields expressions to map original fields of the DataStream to the fields of the View. * @param The type of the {@link DataStream}. + * @deprecated use {@link #createTemporaryView(String, DataStream, Expression...)} */ + @Deprecated void createTemporaryView(String path, DataStream dataStream, String fields); /** * Creates a view from the given {@link DataStream} in a given path with specified field names. * Registered views can be referenced in SQL queries. * + *

    There are two modes for mapping original fields to the fields of the View: + * + *

    1. Reference input fields by name: + * All fields in the schema definition are referenced by name + * (and possibly renamed using an alias (as). Moreover, we can define proctime and rowtime + * attributes at arbitrary positions using arbitrary names (except those that exist in the + * result schema). In this mode, fields can be reordered and projected out. This mode can + * be used for any input type, including POJOs. + * + *

    Example: + * + *

    +	 * {@code
    +	 *   DataStream> stream = ...
    +	 *   tableEnv.createTemporaryView(
    +	 *      "cat.db.myTable",
    +	 *      stream,
    +	 *      $("f1"), // reorder and use the original field
    +	 *      $("rowtime").rowtime(), // extract the internally attached timestamp into an event-time
    +	 *                              // attribute named 'rowtime'
    +	 *      $("f0").as("name") // reorder and give the original field a better name
    +	 *   );
    +	 * }
    +	 * 
    + * + *

    2. Reference input fields by position: + * In this mode, fields are simply renamed. Event-time attributes can + * replace the field on their position in the input data (if it is of correct type) or be + * appended at the end. Proctime attributes must be appended at the end. This mode can only be + * used if the input type has a defined field order (tuple, case class, Row) and none of + * the {@code fields} references a field of the input type. + * *

    Example: * *

     	 * {@code
     	 *   DataStream> stream = ...
    -	 *   tableEnv.createTemporaryView("cat.db.myTable", stream, $("a"), $("b").as("name"), $("timestamp").rowtime())
    +	 *   tableEnv.createTemporaryView(
    +	 *      "cat.db.myTable",
    +	 *      stream,
    +	 *      $("a"), // rename the first field to 'a'
    +	 *      $("b"), // rename the second field to 'b'
    +	 *      $("rowtime").rowtime() // adds an event-time attribute named 'rowtime'
    +	 *   );
     	 * }
     	 * 
    * @@ -344,7 +510,7 @@ public interface StreamTableEnvironment extends TableEnvironment { * @param path The path under which the {@link DataStream} is created. * See also the {@link TableEnvironment} class description for the format of the path. * @param dataStream The {@link DataStream} out of which to create the view. - * @param fields The field expressions of the created view. + * @param fields The fields expressions to map original fields of the DataStream to the fields of the View. * @param The type of the {@link DataStream}. */ void createTemporaryView(String path, DataStream dataStream, Expression... fields); diff --git a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/BatchTableEnvironment.scala b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/BatchTableEnvironment.scala index 706ffc0212..9f1a28a100 100644 --- a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/BatchTableEnvironment.scala +++ b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/BatchTableEnvironment.scala @@ -80,15 +80,43 @@ trait BatchTableEnvironment extends TableEnvironment { /** * Converts the given [[DataSet]] into a [[Table]] with specified field names. * + * There are two modes for mapping original fields to the fields of the [[Table]]: + * + * 1. Reference input fields by name: + * All fields in the schema definition are referenced by name + * (and possibly renamed using an alias (as). In this mode, fields can be reordered and + * projected out. This mode can be used for any input type, including POJOs. + * + * Example: + * + * {{{ + * val set: DataSet[(String, Long)] = ... + * val table: Table = tableEnv.fromDataSet( + * set, + * $"_2", // reorder and use the original field + * $"_1" as "name" // reorder and give the original field a better name + * ) + * }}} + * + * 2. Reference input fields by position: + * In this mode, fields are simply renamed. This mode can only be + * used if the input type has a defined field order (tuple, case class, Row) and none of + * the `fields` references a field of the input type. + * * Example: * * {{{ * val set: DataSet[(String, Long)] = ... - * val tab: Table = tableEnv.fromDataSet(set, 'a, 'b) + * val table: Table = tableEnv.fromDataSet( + * set, + * $"a", // renames the first field to 'a' + * $"b" // renames the second field to 'b' + * ) * }}} * * @param dataSet The [[DataSet]] to be converted. - * @param fields The field names of the resulting [[Table]]. + * @param fields The fields expressions to map original fields of the DataSet to the fields of + * the [[Table]]. * @tparam T The type of the [[DataSet]]. * @return The converted [[Table]]. */ @@ -137,11 +165,40 @@ trait BatchTableEnvironment extends TableEnvironment { * Creates a view from the given [[DataSet]] in a given path with specified field names. * Registered views can be referenced in SQL queries. * + * There are two modes for mapping original fields to the fields of the View: + * + * 1. Reference input fields by name: + * All fields in the schema definition are referenced by name + * (and possibly renamed using an alias (as). In this mode, fields can be reordered and + * projected out. This mode can be used for any input type, including POJOs. + * * Example: * * {{{ * val set: DataSet[(String, Long)] = ... - * tableEnv.registerDataSet("myTable", set, 'a, 'b) + * tableEnv.registerDataSet( + * "myTable", + * set, + * $"_2", // reorder and use the original field + * $"_1" as "name" // reorder and give the original field a better name + * ); + * }}} + * + * 2. Reference input fields by position: + * In this mode, fields are simply renamed. This mode can only be + * used if the input type has a defined field order (tuple, case class, Row) and none of + * the `fields` references a field of the input type. + * + * Example: + * + * {{{ + * val set: DataSet[(String, Long)] = ... + * tableEnv.registerDataSet( + * "myTable", + * set, + * $"a", // renames the first field to 'a' + * $"b" // renames the second field to 'b' + * ) * }}} * * The view is registered in the namespace of the current catalog and database. To register the @@ -153,7 +210,8 @@ trait BatchTableEnvironment extends TableEnvironment { * * @param name The name under which the [[DataSet]] is registered in the catalog. * @param dataSet The [[DataSet]] to register. - * @param fields The field names of the registered table. + * @param fields The fields expressions to map original fields of the DataSet to the fields of + * the View. * @tparam T The type of the [[DataSet]] to register. * @deprecated use [[createTemporaryView]] */ @@ -164,11 +222,40 @@ trait BatchTableEnvironment extends TableEnvironment { * Creates a view from the given [[DataSet]] in a given path with specified field names. * Registered views can be referenced in SQL queries. * + * There are two modes for mapping original fields to the fields of the View: + * + * 1. Reference input fields by name: + * All fields in the schema definition are referenced by name + * (and possibly renamed using an alias (as). In this mode, fields can be reordered and + * projected out. This mode can be used for any input type, including POJOs. + * + * Example: + * + * {{{ + * val set: DataSet[(String, Long)] = ... + * tableEnv.createTemporaryView( + * "cat.db.myTable", + * set, + * $"_2", // reorder and use the original field + * $"_1" as "name" // reorder and give the original field a better name + * ) + * }}} + * + * 2. Reference input fields by position: + * In this mode, fields are simply renamed. This mode can only be + * used if the input type has a defined field order (tuple, case class, Row) and none of + * the `fields` references a field of the input type. + * * Example: * * {{{ * val set: DataSet[(String, Long)] = ... - * tableEnv.createTemporaryView("cat.db.myTable", set, 'a, 'b) + * tableEnv.createTemporaryView( + * "cat.db.myTable", + * set, + * $"a", // renames the first field to 'a' + * $"b" // renames the second field to 'b' + * ) * }}} * * Temporary objects can shadow permanent ones. If a permanent object in a given path exists, @@ -179,7 +266,8 @@ trait BatchTableEnvironment extends TableEnvironment { * See also the [[TableEnvironment]] class description for the format of the * path. * @param dataSet The [[DataSet]] out of which to create the view. - * @param fields The field names of the created view. + * @param fields The fields expressions to map original fields of the DataSet to the fields of + * the View. * @tparam T The type of the [[DataSet]]. */ def createTemporaryView[T](path: String, dataSet: DataSet[T], fields: Expression*): Unit diff --git a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala index d40b21a0de..319f5966f8 100644 --- a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala +++ b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala @@ -98,15 +98,50 @@ trait StreamTableEnvironment extends TableEnvironment { /** * Converts the given [[DataStream]] into a [[Table]] with specified field names. * + * There are two modes for mapping original fields to the fields of the [[Table]]: + * + * 1. Reference input fields by name: + * All fields in the schema definition are referenced by name + * (and possibly renamed using an alias (as). Moreover, we can define proctime and rowtime + * attributes at arbitrary positions using arbitrary names (except those that exist in the + * result schema). In this mode, fields can be reordered and projected out. This mode can + * be used for any input type, including POJOs. + * + * Example: + * + * {{{ + * val stream: DataStream[(String, Long)] = ... + * val table: Table = tableEnv.fromDataStream( + * stream, + * $"_2", // reorder and use the original field + * $"rowtime".rowtime, // extract the internally attached timestamp into an event-time + * // attribute named 'rowtime' + * $"_1" as "name" // reorder and give the original field a better name + * ) + * }}} + * + *

    2. Reference input fields by position: + * In this mode, fields are simply renamed. Event-time attributes can + * replace the field on their position in the input data (if it is of correct type) or be + * appended at the end. Proctime attributes must be appended at the end. This mode can only be + * used if the input type has a defined field order (tuple, case class, Row) and none of + * the `fields` references a field of the input type. + * * Example: * * {{{ * val stream: DataStream[(String, Long)] = ... - * val tab: Table = tableEnv.fromDataStream(stream, 'a, 'b) + * val table: Table = tableEnv.fromDataStream( + * stream, + * $"a", // rename the first field to 'a' + * $"b" // rename the second field to 'b' + * $"rowtime".rowtime // extract the internally attached timestamp into an event-time attribute named 'rowtime' + * ) * }}} * * @param dataStream The [[DataStream]] to be converted. - * @param fields The field names of the resulting [[Table]]. + * @param fields The fields expressions to map original fields of the DataStream to the fields of + * the [[Table]]. * @tparam T The type of the [[DataStream]]. * @return The converted [[Table]]. */ @@ -156,11 +191,47 @@ trait StreamTableEnvironment extends TableEnvironment { * Creates a view from the given [[DataStream]] in a given path with specified field names. * Registered views can be referenced in SQL queries. * + * There are two modes for mapping original fields to the fields of the View: + * + * 1. Reference input fields by name: + * All fields in the schema definition are referenced by name + * (and possibly renamed using an alias (as). Moreover, we can define proctime and rowtime + * attributes at arbitrary positions using arbitrary names (except those that exist in the + * result schema). In this mode, fields can be reordered and projected out. This mode can + * be used for any input type, including POJOs. + * * Example: * * {{{ * val stream: DataStream[(String, Long)] = ... - * tableEnv.registerDataStream("myTable", stream, 'a, 'b) + * tableEnv.registerDataStream( + * "myTable", + * stream, + * $"_2", // reorder and use the original field + * $"rowtime".rowtime, // extract the internally attached timestamp into an event-time + * // attribute named 'rowtime' + * $"_1" as "name" // reorder and give the original field a better name + * ) + * }}} + * + * 2. Reference input fields by position: + * In this mode, fields are simply renamed. Event-time attributes can + * replace the field on their position in the input data (if it is of correct type) or be + * appended at the end. Proctime attributes must be appended at the end. This mode can only be + * used if the input type has a defined field order (tuple, case class, Row) and none of + * the `fields` references a field of the input type. + * + * Example: + * + * {{{ + * val stream: DataStream[(String, Long)] = ... + * tableEnv.registerDataStream( + * "myTable", + * stream, + * $"a", // rename the first field to 'a' + * $"b" // rename the second field to 'b' + * $"rowtime".rowtime // adds an event-time attribute named 'rowtime' + * ) * }}} * * The view is registered in the namespace of the current catalog and database. To register the @@ -172,7 +243,8 @@ trait StreamTableEnvironment extends TableEnvironment { * * @param name The name under which the [[DataStream]] is registered in the catalog. * @param dataStream The [[DataStream]] to register. - * @param fields The field names of the registered view. + * @param fields The fields expressions to map original fields of the DataStream to the fields of + * the View. * @tparam T The type of the [[DataStream]] to register. * @deprecated use [[createTemporaryView]] */ @@ -183,11 +255,47 @@ trait StreamTableEnvironment extends TableEnvironment { * Creates a view from the given [[DataStream]] in a given path with specified field names. * Registered views can be referenced in SQL queries. * + * There are two modes for mapping original fields to the fields of the View: + * + * 1. Reference input fields by name: + * All fields in the schema definition are referenced by name + * (and possibly renamed using an alias (as). Moreover, we can define proctime and rowtime + * attributes at arbitrary positions using arbitrary names (except those that exist in the + * result schema). In this mode, fields can be reordered and projected out. This mode can + * be used for any input type, including POJOs. + * + * Example: + * + * {{{ + * val stream: DataStream[(String, Long)] = ... + * tableEnv.createTemporaryView( + * "cat.db.myTable", + * stream, + * $"_2", // reorder and use the original field + * $"rowtime".rowtime, // extract the internally attached timestamp into an event-time + * // attribute named 'rowtime' + * $"_1" as "name" // reorder and give the original field a better name + * ) + * }}} + * + * 2. Reference input fields by position: + * In this mode, fields are simply renamed. Event-time attributes can + * replace the field on their position in the input data (if it is of correct type) or be + * appended at the end. Proctime attributes must be appended at the end. This mode can only be + * used if the input type has a defined field order (tuple, case class, Row) and none of + * the `fields` references a field of the input type. + * * Example: * * {{{ * val stream: DataStream[(String, Long)] = ... - * tableEnv.createTemporaryView("cat.db.myTable", stream, 'a, 'b) + * tableEnv.createTemporaryView( + * "cat.db.myTable", + * stream, + * $"a", // rename the first field to 'a' + * $"b" // rename the second field to 'b' + * $"rowtime".rowtime // adds an event-time attribute named 'rowtime' + * ) * }}} * * Temporary objects can shadow permanent ones. If a permanent object in a given path exists, @@ -197,7 +305,8 @@ trait StreamTableEnvironment extends TableEnvironment { * @param path The path under which the [[DataStream]] is created. * See also the [[TableEnvironment]] class description for the format of the path. * @param dataStream The [[DataStream]] out of which to create the view. - * @param fields The field names of the created view. + * @param fields The fields expressions to map original fields of the DataStream to the fields of + * the View. * @tparam T The type of the [[DataStream]]. */ def createTemporaryView[T](path: String, dataStream: DataStream[T], fields: Expression*): Unit -- Gitee From d39f5b33221c813e6b9478001675cb56cf361236 Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Wed, 1 Apr 2020 19:50:16 +0200 Subject: [PATCH 409/885] [hotfix][table] Fix scalastyle issue --- .../apache/flink/table/api/scala/StreamTableEnvironment.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala index 319f5966f8..54d0c97e80 100644 --- a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala +++ b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala @@ -135,7 +135,8 @@ trait StreamTableEnvironment extends TableEnvironment { * stream, * $"a", // rename the first field to 'a' * $"b" // rename the second field to 'b' - * $"rowtime".rowtime // extract the internally attached timestamp into an event-time attribute named 'rowtime' + * $"rowtime".rowtime // extract the internally attached timestamp + * // into an event-time attribute named 'rowtime' * ) * }}} * -- Gitee From 0622ef7cac7d4e882e6a0f9df137e06400053e93 Mon Sep 17 00:00:00 2001 From: Canbin Zheng Date: Thu, 2 Apr 2020 10:59:48 +0800 Subject: [PATCH 410/885] [FLINK-16737][k8s] Remove KubernetesUtils#getContentFromFile This closes #11569 . --- .../kubeclient/KubeClientFactory.java | 5 ++-- .../kubernetes/utils/KubernetesUtils.java | 29 ------------------- 2 files changed, 3 insertions(+), 31 deletions(-) diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/KubeClientFactory.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/KubeClientFactory.java index 65c142d49b..5635654a38 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/KubeClientFactory.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/KubeClientFactory.java @@ -20,7 +20,7 @@ package org.apache.flink.kubernetes.kubeclient; import org.apache.flink.configuration.Configuration; import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; -import org.apache.flink.kubernetes.utils.KubernetesUtils; +import org.apache.flink.util.FileUtils; import io.fabric8.kubernetes.client.Config; import io.fabric8.kubernetes.client.DefaultKubernetesClient; @@ -29,6 +29,7 @@ import io.fabric8.kubernetes.client.KubernetesClientException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.File; import java.io.IOException; /** @@ -55,7 +56,7 @@ public class KubeClientFactory { // Note: the third parameter kubeconfigPath is optional and is set to null. It is only used to rewrite // relative tls asset paths inside kubeconfig when a file is passed, and in the case that the kubeconfig // references some assets via relative paths. - config = Config.fromKubeconfig(kubeContext, KubernetesUtils.getContentFromFile(kubeConfigFile), null); + config = Config.fromKubeconfig(kubeContext, FileUtils.readFileUtf8(new File(kubeConfigFile)), null); } catch (IOException e) { throw new KubernetesClientException("Load kubernetes config failed.", e); } diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java index f0c6f13879..d3ec4113e3 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java @@ -33,12 +33,6 @@ import org.slf4j.LoggerFactory; import javax.annotation.Nullable; -import java.io.BufferedReader; -import java.io.File; -import java.io.FileInputStream; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.io.InputStreamReader; import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -52,29 +46,6 @@ public class KubernetesUtils { private static final Logger LOG = LoggerFactory.getLogger(KubernetesUtils.class); - /** - * Read file content to string. - * - * @param filePath file path - * @return content - */ - public static String getContentFromFile(String filePath) throws FileNotFoundException { - File file = new File(filePath); - if (file.exists()) { - StringBuilder content = new StringBuilder(); - String line; - try (BufferedReader reader = new BufferedReader(new InputStreamReader(new FileInputStream(file)))){ - while ((line = reader.readLine()) != null) { - content.append(line).append(System.lineSeparator()); - } - } catch (IOException e) { - throw new RuntimeException("Error read file content.", e); - } - return content.toString(); - } - throw new FileNotFoundException("File " + filePath + " not exists."); - } - /** * Check whether the port config option is a fixed port. If not, the fallback port will be set to configuration. * @param flinkConfig flink configuration -- Gitee From 9ffa85aaa1b9025e3d2becdc084a02aa8440d4d9 Mon Sep 17 00:00:00 2001 From: godfrey he Date: Thu, 2 Apr 2020 11:52:37 +0800 Subject: [PATCH 411/885] [FLINK-16535][table] BatchTableSink emitDataSet to consumeDataSet This closes #11376 --- .../flink/api/java/io/jdbc/JDBCAppendTableSink.java | 5 +++-- .../client/gateway/local/CollectBatchTableSink.java | 5 +++-- .../org/apache/flink/table/sinks/BatchTableSink.java | 12 +++++++++--- .../org/apache/flink/table/sinks/CsvTableSink.java | 4 ++-- .../factories/utils/TestCollectionTableFactory.scala | 3 ++- .../flink/table/api/internal/BatchTableEnvImpl.scala | 2 +- .../factories/utils/TestCollectionTableFactory.scala | 3 ++- .../runtime/batch/sql/PartitionableSinkITCase.scala | 5 +++-- .../table/utils/MemoryTableSourceSinkUtil.scala | 3 ++- .../common/table/SpendReportTableSink.java | 5 +++-- 10 files changed, 30 insertions(+), 17 deletions(-) diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSink.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSink.java index fde2585ef6..412a0ce4a0 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSink.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSink.java @@ -21,6 +21,7 @@ package org.apache.flink.api.java.io.jdbc; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.operators.DataSink; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; @@ -69,8 +70,8 @@ public class JDBCAppendTableSink implements AppendStreamTableSink, BatchTab } @Override - public void emitDataSet(DataSet dataSet) { - dataSet.output(outputFormat); + public DataSink consumeDataSet(DataSet dataSet) { + return dataSet.output(outputFormat); } @Override diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/CollectBatchTableSink.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/CollectBatchTableSink.java index 801f1f7ca9..b2ef69525e 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/CollectBatchTableSink.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/CollectBatchTableSink.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.Utils; +import org.apache.flink.api.java.operators.DataSink; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.sinks.BatchTableSink; import org.apache.flink.table.sinks.OutputFormatTableSink; @@ -68,8 +69,8 @@ public class CollectBatchTableSink extends OutputFormatTableSink implements } @Override - public void emitDataSet(DataSet dataSet) { - dataSet + public DataSink consumeDataSet(DataSet dataSet) { + return dataSet .output(new Utils.CollectHelper<>(accumulatorName, serializer)) .name("SQL Client Batch Collect Sink"); } diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/BatchTableSink.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/BatchTableSink.java index f9413b7560..29221b4a4e 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/BatchTableSink.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/BatchTableSink.java @@ -18,10 +18,13 @@ package org.apache.flink.table.sinks; +import org.apache.flink.api.common.Plan; import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.operators.DataSink; import org.apache.flink.table.api.Table; -/** Defines an external {@link TableSink} to emit a batch {@link Table}. +/** + * Defines an external {@link TableSink} to emit a batch {@link Table}. * * @param Type of {@link DataSet} that this {@link TableSink} expects and supports. * @@ -30,6 +33,9 @@ import org.apache.flink.table.api.Table; @Deprecated public interface BatchTableSink extends TableSink { - /** Emits the DataSet. */ - void emitDataSet(DataSet dataSet); + /** + * Consumes the DataSet and return the {@link DataSink}. + * The returned {@link DataSink} will be used to generate {@link Plan}. + */ + DataSink consumeDataSet(DataSet dataSet); } diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSink.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSink.java index dfe5e8763f..4ea61ee30f 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSink.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSink.java @@ -112,7 +112,7 @@ public class CsvTableSink implements BatchTableSink, AppendStreamTableSink< } @Override - public void emitDataSet(DataSet dataSet) { + public DataSink consumeDataSet(DataSet dataSet) { MapOperator csvRows = dataSet.map(new CsvFormatter(fieldDelim == null ? "," : fieldDelim)); @@ -128,7 +128,7 @@ public class CsvTableSink implements BatchTableSink, AppendStreamTableSink< sink.setParallelism(numFiles); } - sink.name(TableConnectorUtils.generateRuntimeName(CsvTableSink.class, fieldNames)); + return sink.name(TableConnectorUtils.generateRuntimeName(CsvTableSink.class, fieldNames)); } @Override diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/factories/utils/TestCollectionTableFactory.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/factories/utils/TestCollectionTableFactory.scala index 228ea3c5f6..f1ae9428d0 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/factories/utils/TestCollectionTableFactory.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/factories/utils/TestCollectionTableFactory.scala @@ -22,6 +22,7 @@ import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.common.typeutils.TypeSerializer import org.apache.flink.api.java.io.{CollectionInputFormat, LocalCollectionOutputFormat} +import org.apache.flink.api.java.operators.DataSink import org.apache.flink.api.java.{DataSet, ExecutionEnvironment} import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.datastream.{DataStream, DataStreamSink, DataStreamSource} @@ -165,7 +166,7 @@ object TestCollectionTableFactory { class CollectionTableSink(val schema: TableSchema) extends BatchTableSink[Row] with AppendStreamTableSink[Row] { - override def emitDataSet(dataSet: DataSet[Row]): Unit = { + override def consumeDataSet(dataSet: DataSet[Row]): DataSink[_] = { dataSet.output(new LocalCollectionOutputFormat[Row](RESULT)).setParallelism(1) } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/BatchTableEnvImpl.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/BatchTableEnvImpl.scala index 9fe93cb340..19b26fcfe0 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/BatchTableEnvImpl.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/BatchTableEnvImpl.scala @@ -130,7 +130,7 @@ abstract class BatchTableEnvImpl( // translate the Table into a DataSet and provide the type that the TableSink expects. val result: DataSet[T] = translate(table)(outputType) // Give the DataSet to the TableSink to emit it. - batchSink.emitDataSet(result) + batchSink.consumeDataSet(result) case boundedSink: OutputFormatTableSink[T] => val outputType = fromDataTypeToLegacyInfo(sink.getConsumedDataType) .asInstanceOf[TypeInformation[T]] diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/factories/utils/TestCollectionTableFactory.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/factories/utils/TestCollectionTableFactory.scala index bd31d46419..ded1cbdbcf 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/factories/utils/TestCollectionTableFactory.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/factories/utils/TestCollectionTableFactory.scala @@ -22,6 +22,7 @@ import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.common.typeutils.TypeSerializer import org.apache.flink.api.java.io.{CollectionInputFormat, LocalCollectionOutputFormat} +import org.apache.flink.api.java.operators.DataSink import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.api.java.{DataSet, ExecutionEnvironment} import org.apache.flink.configuration.Configuration @@ -176,7 +177,7 @@ object TestCollectionTableFactory { class CollectionTableSink(val outputType: RowTypeInfo) extends BatchTableSink[Row] with AppendStreamTableSink[Row] { - override def emitDataSet(dataSet: DataSet[Row]): Unit = { + override def consumeDataSet(dataSet: DataSet[Row]): DataSink[_] = { dataSet.output(new LocalCollectionOutputFormat[Row](RESULT)).setParallelism(1) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/PartitionableSinkITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/PartitionableSinkITCase.scala index 90a6f247f7..ca3381b683 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/PartitionableSinkITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/PartitionableSinkITCase.scala @@ -19,7 +19,6 @@ package org.apache.flink.table.runtime.batch.sql import java.util.{LinkedList => JLinkedList, Map => JMap} - import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.functions.MapFunction import org.apache.flink.api.common.io.RichOutputFormat @@ -27,6 +26,7 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo.{INT_TYPE_INFO, LONG_T import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java import org.apache.flink.api.java.DataSet +import org.apache.flink.api.java.operators.DataSink import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.api.scala.ExecutionEnvironment import org.apache.flink.configuration.Configuration @@ -39,6 +39,7 @@ import org.apache.flink.table.sources.BatchTableSource import org.apache.flink.table.types.logical.{BigIntType, IntType, VarCharType} import org.apache.flink.test.util.AbstractTestBase import org.apache.flink.types.Row + import org.junit.Assert.assertEquals import org.junit.rules.ExpectedException import org.junit.{Before, Rule, Test} @@ -154,7 +155,7 @@ class PartitionableSinkITCase extends AbstractTestBase { staticPartitions } - override def emitDataSet(dataSet: DataSet[Row]): Unit = { + override def consumeDataSet(dataSet: DataSet[Row]): DataSink[_] = { dataSet.map(new MapFunction[Row, String] { override def map(value: Row): String = value.toString }).output(new CollectionOutputFormat) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MemoryTableSourceSinkUtil.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MemoryTableSourceSinkUtil.scala index a835bba752..034e9ca3ea 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MemoryTableSourceSinkUtil.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MemoryTableSourceSinkUtil.scala @@ -20,6 +20,7 @@ package org.apache.flink.table.utils import org.apache.flink.api.common.io.{OutputFormat, RichOutputFormat} import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.operators.DataSink import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.api.java.{DataSet, ExecutionEnvironment} import org.apache.flink.configuration.Configuration @@ -111,7 +112,7 @@ object MemoryTableSourceSinkUtil { new UnsafeMemoryAppendTableSink } - override def emitDataSet(dataSet: DataSet[Row]): Unit = { + override def consumeDataSet(dataSet: DataSet[Row]): DataSink[_] = { dataSet .output(new MemoryCollectionOutputFormat) .name(TableConnectorUtils.generateRuntimeName(this.getClass, getTableSchema.getFieldNames)) diff --git a/flink-walkthroughs/flink-walkthrough-common/src/main/java/org/apache/flink/walkthrough/common/table/SpendReportTableSink.java b/flink-walkthroughs/flink-walkthrough-common/src/main/java/org/apache/flink/walkthrough/common/table/SpendReportTableSink.java index a143306d07..5073d90e1f 100644 --- a/flink-walkthroughs/flink-walkthrough-common/src/main/java/org/apache/flink/walkthrough/common/table/SpendReportTableSink.java +++ b/flink-walkthroughs/flink-walkthrough-common/src/main/java/org/apache/flink/walkthrough/common/table/SpendReportTableSink.java @@ -22,6 +22,7 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.operators.DataSink; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.table.api.TableSchema; @@ -51,8 +52,8 @@ public class SpendReportTableSink implements AppendStreamTableSink, BatchTa } @Override - public void emitDataSet(DataSet dataSet) { - dataSet + public DataSink consumeDataSet(DataSet dataSet) { + return dataSet .map(SpendReportTableSink::format) .output(new LoggerOutputFormat()); } -- Gitee From c8a23c74e618b752bbdc58dca62d997ddd303d40 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Thu, 2 Apr 2020 15:09:10 +0800 Subject: [PATCH 412/885] [FLINK-16860][orc] Distinguish empty filters from no pushed down in expainSource This closes #11594 --- .../org/apache/flink/orc/OrcTableSource.java | 4 +- .../apache/flink/orc/OrcTableSourceTest.java | 44 +++++++++++++------ 2 files changed, 34 insertions(+), 14 deletions(-) diff --git a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/OrcTableSource.java b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/OrcTableSource.java index 8ba3f27b92..26094333dc 100644 --- a/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/OrcTableSource.java +++ b/flink-formats/flink-orc/src/main/java/org/apache/flink/orc/OrcTableSource.java @@ -218,7 +218,9 @@ public class OrcTableSource } private String predicateString() { - if (predicates == null || predicates.length == 0) { + if (predicates == null) { + return "NULL"; + } else if (predicates.length == 0) { return "TRUE"; } else { return "AND(" + Arrays.toString(predicates) + ")"; diff --git a/flink-formats/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceTest.java b/flink-formats/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceTest.java index 6d2b33d6ce..712bd93701 100644 --- a/flink-formats/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceTest.java +++ b/flink-formats/flink-orc/src/test/java/org/apache/flink/orc/OrcTableSourceTest.java @@ -44,12 +44,14 @@ import org.junit.Test; import org.mockito.ArgumentCaptor; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.stream.Collectors; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; @@ -184,18 +186,6 @@ public class OrcTableSourceTest { Expression pred2 = new EqualTo( new PlannerResolvedFieldReference("string1", Types.STRING), new Literal("hello", Types.STRING)); - // unsupported predicate - Expression unsupportedPred = new EqualTo( - new GetCompositeField( - new ItemAt( - new PlannerResolvedFieldReference( - "list", - ObjectArrayTypeInfo.getInfoFor( - Types.ROW_NAMED(new String[] {"int1", "string1"}, Types.INT, Types.STRING))), - new Literal(1, Types.INT)), - "int1"), - new Literal(1, Types.INT) - ); // invalid predicate Expression invalidPred = new EqualTo( new PlannerResolvedFieldReference("long1", Types.LONG), @@ -206,7 +196,7 @@ public class OrcTableSourceTest { ArrayList preds = new ArrayList<>(); preds.add(pred1); preds.add(pred2); - preds.add(unsupportedPred); + preds.add(unsupportedPred()); preds.add(invalidPred); // apply predicates on TableSource @@ -244,6 +234,34 @@ public class OrcTableSourceTest { assertFalse(orc.isFilterPushedDown()); } + private Expression unsupportedPred() { + return new EqualTo( + new GetCompositeField( + new ItemAt( + new PlannerResolvedFieldReference( + "list", + ObjectArrayTypeInfo.getInfoFor( + Types.ROW_NAMED(new String[] {"int1", "string1"}, Types.INT, Types.STRING))), + new Literal(1, Types.INT)), + "int1"), + new Literal(1, Types.INT) + ); + } + + @Test + public void testUnsupportedPredOnly() { + OrcTableSource orc = OrcTableSource.builder() + .path(getPath(TEST_FILE_NESTED)) + .forOrcSchema(TEST_SCHEMA_NESTED) + .build(); + + // apply predicates on TableSource + OrcTableSource projected = (OrcTableSource) orc.applyPredicate( + Collections.singletonList(unsupportedPred())); + + assertNotEquals(orc.explainSource(), projected.explainSource()); + } + @Test public void testBuilder() throws Exception { -- Gitee From a24734ea339872763306b44770678c4ace6a369f Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 1 Apr 2020 17:36:49 +0200 Subject: [PATCH 413/885] [FLINK-16560] Forward Configuration in PackagedProgramUtils#getPipelineFromProgram Before, when using PackagedProgramUtils (for example in the standalone cluster entrypoint or the web ui) the Flink Configuration would not be applied to the execution environment. This also adds a test that verifies that we forward configuration. --- .../apache/flink/client/cli/CliFrontend.java | 7 +- .../program/OptimizerPlanEnvironment.java | 4 +- .../client/program/PackagedProgramUtils.java | 7 +- .../client/program/StreamPlanEnvironment.java | 4 +- .../cli/CliFrontendPackageProgramTest.java | 2 +- .../flink/client/program/ClientTest.java | 2 +- .../program/ExecutionPlanCreationTest.java | 2 +- .../program/OptimizerPlanEnvironmentTest.java | 4 +- .../program/PackagedProgramUtilsTest.java | 123 ++++++++++++++++++ 9 files changed, 144 insertions(+), 11 deletions(-) create mode 100644 flink-clients/src/test/java/org/apache/flink/client/program/PackagedProgramUtilsTest.java diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java index c38c49c639..55244c3fe9 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java @@ -233,7 +233,7 @@ public class CliFrontend { * * @param args Command line arguments for the info action. */ - protected void info(String[] args) throws CliArgsException, FileNotFoundException, ProgramInvocationException { + protected void info(String[] args) throws Exception { LOG.info("Running 'info' command."); final Options commandOptions = CliFrontendParser.getInfoCommandOptions(); @@ -265,7 +265,10 @@ public class CliFrontend { LOG.info("Creating program plan dump"); - Pipeline pipeline = PackagedProgramUtils.getPipelineFromProgram(program, parallelism, true); + final Configuration effectiveConfiguration = + getEffectiveConfiguration(commandLine, programOptions, program.getJobJarAndDependencies()); + + Pipeline pipeline = PackagedProgramUtils.getPipelineFromProgram(program, effectiveConfiguration, parallelism, true); String jsonPlan = FlinkPipelineTranslationUtil.translateToJSONExecutionPlan(pipeline); if (jsonPlan != null) { diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/OptimizerPlanEnvironment.java b/flink-clients/src/main/java/org/apache/flink/client/program/OptimizerPlanEnvironment.java index 87ff1a4f51..4f9b40e5b9 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/OptimizerPlanEnvironment.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/OptimizerPlanEnvironment.java @@ -21,6 +21,7 @@ package org.apache.flink.client.program; import org.apache.flink.api.dag.Pipeline; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.ExecutionEnvironmentFactory; +import org.apache.flink.configuration.Configuration; import org.apache.flink.core.execution.JobClient; /** @@ -34,7 +35,8 @@ public class OptimizerPlanEnvironment extends ExecutionEnvironment { return pipeline; } - public OptimizerPlanEnvironment(int parallelism) { + public OptimizerPlanEnvironment(Configuration configuration, int parallelism) { + super(configuration); if (parallelism > 0) { setParallelism(parallelism); } diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgramUtils.java b/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgramUtils.java index 5761e950ed..6d2ff10f6d 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgramUtils.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgramUtils.java @@ -59,7 +59,7 @@ public enum PackagedProgramUtils { int defaultParallelism, @Nullable JobID jobID, boolean suppressOutput) throws ProgramInvocationException { - final Pipeline pipeline = getPipelineFromProgram(packagedProgram, defaultParallelism, suppressOutput); + final Pipeline pipeline = getPipelineFromProgram(packagedProgram, configuration, defaultParallelism, suppressOutput); final JobGraph jobGraph = FlinkPipelineTranslationUtil.getJobGraph(pipeline, configuration, defaultParallelism); if (jobID != null) { @@ -93,6 +93,7 @@ public enum PackagedProgramUtils { public static Pipeline getPipelineFromProgram( PackagedProgram program, + Configuration configuration, int parallelism, boolean suppressOutput) throws CompilerException, ProgramInvocationException { final ClassLoader contextClassLoader = Thread.currentThread().getContextClassLoader(); @@ -116,9 +117,9 @@ public enum PackagedProgramUtils { } // temporary hack to support the optimizer plan preview - OptimizerPlanEnvironment benv = new OptimizerPlanEnvironment(parallelism); + OptimizerPlanEnvironment benv = new OptimizerPlanEnvironment(configuration, parallelism); benv.setAsContext(); - StreamPlanEnvironment senv = new StreamPlanEnvironment(parallelism); + StreamPlanEnvironment senv = new StreamPlanEnvironment(configuration, parallelism); senv.setAsContext(); try { diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/StreamPlanEnvironment.java b/flink-clients/src/main/java/org/apache/flink/client/program/StreamPlanEnvironment.java index 86b05af906..b2ce783153 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/StreamPlanEnvironment.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/StreamPlanEnvironment.java @@ -19,6 +19,7 @@ package org.apache.flink.client.program; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.dag.Pipeline; +import org.apache.flink.configuration.Configuration; import org.apache.flink.core.execution.JobClient; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironmentFactory; @@ -37,7 +38,8 @@ public class StreamPlanEnvironment extends StreamExecutionEnvironment { return pipeline; } - public StreamPlanEnvironment(int parallelism) { + public StreamPlanEnvironment(Configuration configuration, int parallelism) { + super(configuration); if (parallelism > 0) { setParallelism(parallelism); } diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java index dd8f345b30..6750cca908 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java @@ -301,7 +301,7 @@ public class CliFrontendPackageProgramTest extends TestLogger { Optimizer compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), c); // we expect this to fail with a "ClassNotFoundException" - Pipeline pipeline = PackagedProgramUtils.getPipelineFromProgram(prog, 666, true); + Pipeline pipeline = PackagedProgramUtils.getPipelineFromProgram(prog, c, 666, true); FlinkPipelineTranslationUtil.translateToJSONExecutionPlan(pipeline); fail("Should have failed with a ClassNotFoundException"); } diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java index 6f291e722e..93248a1b86 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java @@ -234,7 +234,7 @@ public class ClientTest extends TestLogger { .build(); Optimizer optimizer = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), config); - Plan plan = (Plan) PackagedProgramUtils.getPipelineFromProgram(prg, 1, true); + Plan plan = (Plan) PackagedProgramUtils.getPipelineFromProgram(prg, new Configuration(), 1, true); OptimizedPlan op = optimizer.compile(plan); assertNotNull(op); diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanCreationTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanCreationTest.java index 0c258e41d7..b47928d137 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanCreationTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanCreationTest.java @@ -63,7 +63,7 @@ public class ExecutionPlanCreationTest { config.setInteger(JobManagerOptions.PORT, mockJmAddress.getPort()); Optimizer optimizer = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), config); - Plan plan = (Plan) PackagedProgramUtils.getPipelineFromProgram(prg, -1, true); + Plan plan = (Plan) PackagedProgramUtils.getPipelineFromProgram(prg, config, -1, true); OptimizedPlan op = optimizer.compile(plan); assertNotNull(op); diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/OptimizerPlanEnvironmentTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/OptimizerPlanEnvironmentTest.java index 0bd89e194c..9e952e1f55 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/OptimizerPlanEnvironmentTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/OptimizerPlanEnvironmentTest.java @@ -18,6 +18,8 @@ package org.apache.flink.client.program; +import org.apache.flink.configuration.Configuration; + import org.junit.Assert; import org.junit.Test; @@ -47,7 +49,7 @@ public class OptimizerPlanEnvironmentTest { try { // Flink will throw an error because no job graph will be generated by the main method. - PackagedProgramUtils.getPipelineFromProgram(packagedProgram, 1, suppressOutput); + PackagedProgramUtils.getPipelineFromProgram(packagedProgram, new Configuration(), 1, suppressOutput); Assert.fail("This should have failed to create the Flink Plan."); } catch (ProgramInvocationException e) { // Test that that Flink captured the expected stdout/stderr diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/PackagedProgramUtilsTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/PackagedProgramUtilsTest.java new file mode 100644 index 0000000000..59e75fc1f7 --- /dev/null +++ b/flink-clients/src/test/java/org/apache/flink/client/program/PackagedProgramUtilsTest.java @@ -0,0 +1,123 @@ +/* + * 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.client.program; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.Plan; +import org.apache.flink.api.dag.Pipeline; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.PipelineOptions; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.graph.StreamGraph; + +import org.junit.Test; + +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; + +/** + * Tests {@link PackagedProgramUtils}. + */ +public class PackagedProgramUtilsTest { + + /** + * This tests whether configuration forwarding from a {@link Configuration} to the environment + * works. + */ + @Test + public void testDataSetConfigurationForwarding() throws Exception { + assertPrecondition(ExecutionEnvironment.getExecutionEnvironment().getConfig()); + + PackagedProgram packagedProgram = PackagedProgram.newBuilder() + .setEntryPointClassName(DataSetTestProgram.class.getName()) + .build(); + + Configuration config = createConfigurationWithOption(); + + Pipeline pipeline = PackagedProgramUtils.getPipelineFromProgram( + packagedProgram, + config, + 1 /* parallelism */, + false /* suppress output */); + + ExecutionConfig executionConfig = ((Plan) pipeline).getExecutionConfig(); + + assertExpectedOption(executionConfig); + } + + /** + * This tests whether configuration forwarding from a {@link Configuration} to the environment + * works. + */ + @Test + public void testDataStreamConfigurationForwarding() throws Exception { + assertPrecondition(ExecutionEnvironment.getExecutionEnvironment().getConfig()); + + PackagedProgram packagedProgram = PackagedProgram.newBuilder() + .setEntryPointClassName(DataStreamTestProgram.class.getName()) + .build(); + + Configuration config = createConfigurationWithOption(); + + Pipeline pipeline = PackagedProgramUtils.getPipelineFromProgram( + packagedProgram, + config, + 1 /* parallelism */, + false /* suppress output */); + + ExecutionConfig executionConfig = ((StreamGraph) pipeline).getExecutionConfig(); + + assertExpectedOption(executionConfig); + } + + private static void assertPrecondition(ExecutionConfig executionConfig) { + // we want to test forwarding with this config, ensure that the default is what we expect. + assertThat(executionConfig.isAutoTypeRegistrationDisabled(), is(false)); + } + + private static void assertExpectedOption(ExecutionConfig executionConfig) { + // we want to test forwarding with this config, ensure that the default is what we expect. + assertThat(executionConfig.isAutoTypeRegistrationDisabled(), is(true)); + } + + private static Configuration createConfigurationWithOption() { + Configuration config = new Configuration(); + config.set(PipelineOptions.AUTO_TYPE_REGISTRATION, false); + return config; + } + + /** Test Program for the DataSet API. */ + public static class DataSetTestProgram { + public static void main(String[] args) throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.fromElements("hello").print(); + env.execute(); + } + } + + /** Test Program for the DataStream API. */ + public static class DataStreamTestProgram { + public static void main(String[] args) throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.fromElements("hello").print(); + env.execute(); + } + } +} -- Gitee From 356e2b61f2ddd5cdefbe74b31db41b97210ceec6 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 1 Apr 2020 16:30:42 +0200 Subject: [PATCH 414/885] [FLINK-14316] Properly manage rpcConnection in JobManagerLeaderListener under leader change This commit changes how the rpcConnection is managed in JobManagerLeaderListener under leader change. This component clears now the fields rpcConnection and currentJobMasterId if the leader loses leadership. Moreover, it only restarts a connection attempt if the leader session id is new. This closes #11603. --- .../taskexecutor/JobLeaderService.java | 63 +++++----- .../taskexecutor/JobLeaderServiceTest.java | 108 ++++++++++++++---- 2 files changed, 119 insertions(+), 52 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java index 94832dac67..aed66c38b2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java @@ -271,9 +271,7 @@ public class JobLeaderService { if (!stopped) { stopped = true; - if (rpcConnection != null) { - rpcConnection.close(); - } + closeRpcConnection(); } } } @@ -310,37 +308,16 @@ public class JobLeaderService { if (leaderAddress == null || leaderAddress.isEmpty()) { // the leader lost leadership but there is no other leader yet. - if (rpcConnection != null) { - rpcConnection.close(); - rpcConnection = null; - } - jobManagerLostLeadership = Optional.ofNullable(currentJobMasterId); - currentJobMasterId = jobMasterId; + closeRpcConnection(); } else { - currentJobMasterId = jobMasterId; - - if (rpcConnection != null) { - // check if we are already trying to connect to this leader - if (!Objects.equals(jobMasterId, rpcConnection.getTargetLeaderId())) { - rpcConnection.close(); - - rpcConnection = new JobManagerRegisteredRpcConnection( - LOG, - leaderAddress, - jobMasterId, - rpcService.getExecutor()); - } + // check whether we are already connecting to this leader + if (Objects.equals(jobMasterId, currentJobMasterId)) { + LOG.debug("Ongoing attempt to connect to leader of job {}. Ignoring duplicate leader information.", jobId); } else { - rpcConnection = new JobManagerRegisteredRpcConnection( - LOG, - leaderAddress, - jobMasterId, - rpcService.getExecutor()); + closeRpcConnection(); + openRpcConnectionTo(leaderAddress, jobMasterId); } - - LOG.info("Try to register at job manager {} with leader id {}.", leaderAddress, leaderId); - rpcConnection.start(); } } } @@ -349,6 +326,32 @@ public class JobLeaderService { jobManagerLostLeadership.ifPresent(oldJobMasterId -> jobLeaderListener.jobManagerLostLeadership(jobId, oldJobMasterId)); } + @GuardedBy("lock") + private void openRpcConnectionTo(String leaderAddress, JobMasterId jobMasterId) { + Preconditions.checkState( + currentJobMasterId == null && rpcConnection == null, + "Cannot open a new rpc connection if the previous connection has not been closed."); + + currentJobMasterId = jobMasterId; + rpcConnection = new JobManagerRegisteredRpcConnection( + LOG, + leaderAddress, + jobMasterId, + rpcService.getExecutor()); + + LOG.info("Try to register at job manager {} with leader id {}.", leaderAddress, jobMasterId.toUUID()); + rpcConnection.start(); + } + + @GuardedBy("lock") + private void closeRpcConnection() { + if (rpcConnection != null) { + rpcConnection.close(); + rpcConnection = null; + currentJobMasterId = null; + } + } + @Override public void handleError(Exception exception) { if (stopped) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/JobLeaderServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/JobLeaderServiceTest.java index d61d601a95..d1cb205c9b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/JobLeaderServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/JobLeaderServiceTest.java @@ -20,6 +20,7 @@ package org.apache.flink.runtime.taskexecutor; import org.apache.flink.api.common.JobID; import org.apache.flink.core.testutils.CheckedThread; +import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServicesBuilder; @@ -41,6 +42,10 @@ import java.util.UUID; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CountDownLatch; +import java.util.function.Consumer; + +import static org.hamcrest.CoreMatchers.is; +import static org.junit.Assert.assertThat; /** * Tests for the {@link JobLeaderService}. @@ -106,34 +111,25 @@ public class JobLeaderServiceTest extends TestLogger { */ @Test public void doesNotReconnectAfterTargetLostLeadership() throws Exception { - final JobLeaderService jobLeaderService = new JobLeaderService( - new LocalUnresolvedTaskManagerLocation(), - RetryingRegistrationConfiguration.defaultConfiguration()); - final JobID jobId = new JobID(); final SettableLeaderRetrievalService leaderRetrievalService = new SettableLeaderRetrievalService(); final TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServicesBuilder() .setJobMasterLeaderRetrieverFunction(ignored -> leaderRetrievalService) .build(); + final TestingJobMasterGateway jobMasterGateway = registerJobMaster(); - final String jmAddress = "foobar"; - final TestingJobMasterGateway jobMasterGateway = new TestingJobMasterGatewayBuilder().build(); - rpcServiceResource.getTestingRpcService().registerGateway(jmAddress, jobMasterGateway); + final OneShotLatch jobManagerGainedLeadership = new OneShotLatch(); + final TestingJobLeaderListener testingJobLeaderListener = new TestingJobLeaderListener(ignored -> jobManagerGainedLeadership.trigger()); - final TestingJobLeaderListener testingJobLeaderListener = new TestingJobLeaderListener(); - jobLeaderService.start( - "foobar", - rpcServiceResource.getTestingRpcService(), - haServices, - testingJobLeaderListener); + final JobLeaderService jobLeaderService = createAndStartJobLeaderService(haServices, testingJobLeaderListener); try { - jobLeaderService.addJob(jobId, jmAddress); + jobLeaderService.addJob(jobId, jobMasterGateway.getAddress()); - leaderRetrievalService.notifyListener(jmAddress, UUID.randomUUID()); + leaderRetrievalService.notifyListener(jobMasterGateway.getAddress(), UUID.randomUUID()); - testingJobLeaderListener.waitUntilJobManagerGainedLeadership(); + jobManagerGainedLeadership.await(); // revoke the leadership leaderRetrievalService.notifyListener(null, null); @@ -145,14 +141,86 @@ public class JobLeaderServiceTest extends TestLogger { } } + /** + * Tests that the JobLeaderService can reconnect to an old leader which seemed + * to have lost the leadership in between. See FLINK-14316. + */ + @Test + public void canReconnectToOldLeaderWithSameLeaderAddress() throws Exception { + final JobID jobId = new JobID(); + + final SettableLeaderRetrievalService leaderRetrievalService = new SettableLeaderRetrievalService(); + final TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServicesBuilder() + .setJobMasterLeaderRetrieverFunction(ignored -> leaderRetrievalService) + .build(); + + final TestingJobMasterGateway jobMasterGateway = registerJobMaster(); + + final BlockingQueue leadershipQueue = new ArrayBlockingQueue<>(1); + final TestingJobLeaderListener testingJobLeaderListener = new TestingJobLeaderListener(leadershipQueue::offer); + + final JobLeaderService jobLeaderService = createAndStartJobLeaderService(haServices, testingJobLeaderListener); + + try { + jobLeaderService.addJob(jobId, jobMasterGateway.getAddress()); + + final UUID leaderSessionId = UUID.randomUUID(); + leaderRetrievalService.notifyListener(jobMasterGateway.getAddress(), leaderSessionId); + + // wait for the first leadership + assertThat(leadershipQueue.take(), is(jobId)); + + // revoke the leadership + leaderRetrievalService.notifyListener(null, null); + + testingJobLeaderListener.waitUntilJobManagerLostLeadership(); + + leaderRetrievalService.notifyListener(jobMasterGateway.getAddress(), leaderSessionId); + + // check that we obtain the leadership a second time + assertThat(leadershipQueue.take(), is(jobId)); + } finally { + jobLeaderService.stop(); + } + } + + private JobLeaderService createAndStartJobLeaderService(HighAvailabilityServices haServices, JobLeaderListener testingJobLeaderListener) { + final JobLeaderService jobLeaderService = new JobLeaderService( + new LocalUnresolvedTaskManagerLocation(), + RetryingRegistrationConfiguration.defaultConfiguration()); + + jobLeaderService.start( + "foobar", + rpcServiceResource.getTestingRpcService(), + haServices, + testingJobLeaderListener); + return jobLeaderService; + } + + private TestingJobMasterGateway registerJobMaster() { + final TestingJobMasterGateway jobMasterGateway = new TestingJobMasterGatewayBuilder().build(); + rpcServiceResource.getTestingRpcService().registerGateway(jobMasterGateway.getAddress(), jobMasterGateway); + + return jobMasterGateway; + } + private static final class TestingJobLeaderListener implements JobLeaderListener { - private final CountDownLatch jobManagerGainedLeadership = new CountDownLatch(1); private final CountDownLatch jobManagerLostLeadership = new CountDownLatch(1); + private final Consumer jobManagerGainedLeadership; + + private TestingJobLeaderListener() { + this(ignored -> {}); + } + + private TestingJobLeaderListener(Consumer jobManagerGainedLeadership) { + this.jobManagerGainedLeadership = jobManagerGainedLeadership; + } + @Override public void jobManagerGainedLeadership(JobID jobId, JobMasterGateway jobManagerGateway, JMTMRegistrationSuccess registrationMessage) { - jobManagerGainedLeadership.countDown(); + jobManagerGainedLeadership.accept(jobId); } @Override @@ -165,10 +233,6 @@ public class JobLeaderServiceTest extends TestLogger { // ignored } - private void waitUntilJobManagerGainedLeadership() throws InterruptedException { - jobManagerGainedLeadership.await(); - } - private void waitUntilJobManagerLostLeadership() throws InterruptedException { jobManagerLostLeadership.await(); } -- Gitee From 89522f9fbf5974ba1acee0ec3cb7ee626d7a91c0 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 2 Apr 2020 12:14:03 +0200 Subject: [PATCH 415/885] [FLINK-16944][tests] Fix compile errors --- .../flink/test/optimizer/jsonplan/DumpCompiledPlanTest.java | 3 ++- .../flink/test/optimizer/jsonplan/PreviewPlanDumpTest.java | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/DumpCompiledPlanTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/DumpCompiledPlanTest.java index 04eb02461a..290c148b47 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/DumpCompiledPlanTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/DumpCompiledPlanTest.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.Plan; import org.apache.flink.api.dag.Pipeline; import org.apache.flink.client.program.PackagedProgram; import org.apache.flink.client.program.PackagedProgramUtils; +import org.apache.flink.configuration.Configuration; import org.apache.flink.examples.java.clustering.KMeans; import org.apache.flink.examples.java.graph.ConnectedComponents; import org.apache.flink.examples.java.graph.PageRank; @@ -104,7 +105,7 @@ public class DumpCompiledPlanTest extends CompilerTestBase { .setArguments(args) .build(); - final Pipeline pipeline = PackagedProgramUtils.getPipelineFromProgram(program, 1, true); + final Pipeline pipeline = PackagedProgramUtils.getPipelineFromProgram(program, new Configuration(), 1, true); assertTrue(pipeline instanceof Plan); diff --git a/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/PreviewPlanDumpTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/PreviewPlanDumpTest.java index 24d4ab052c..f0b0a2f9ea 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/PreviewPlanDumpTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/optimizer/jsonplan/PreviewPlanDumpTest.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.Plan; import org.apache.flink.api.dag.Pipeline; import org.apache.flink.client.program.PackagedProgram; import org.apache.flink.client.program.PackagedProgramUtils; +import org.apache.flink.configuration.Configuration; import org.apache.flink.examples.java.clustering.KMeans; import org.apache.flink.examples.java.graph.ConnectedComponents; import org.apache.flink.examples.java.graph.PageRank; @@ -107,7 +108,7 @@ public class PreviewPlanDumpTest extends CompilerTestBase { .setArguments(args) .build(); - final Pipeline pipeline = PackagedProgramUtils.getPipelineFromProgram(program, 1, true); + final Pipeline pipeline = PackagedProgramUtils.getPipelineFromProgram(program, new Configuration(), 1, true); assertTrue(pipeline instanceof Plan); -- Gitee From c79c26a1089187c50bbd39d53bcfa47b509f47e7 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Thu, 2 Apr 2020 09:34:37 +0200 Subject: [PATCH 416/885] [FLINK-16917][runtime] Revert FLINK-16245 --- .../org/apache/flink/client/ClientUtils.java | 3 +- .../flink/util/ChildFirstClassLoader.java | 5 +- .../flink/util/FlinkUserCodeClassLoader.java | 38 ---------- .../FlinkUserCodeClassLoaders.java | 73 +------------------ .../FlinkUserCodeClassLoadersTest.java | 29 -------- .../gateway/local/ExecutionContext.java | 12 +-- .../client/gateway/local/LocalExecutor.java | 7 +- .../BatchFineGrainedRecoveryITCase.java | 19 ++--- 8 files changed, 18 insertions(+), 168 deletions(-) delete mode 100644 flink-core/src/main/java/org/apache/flink/util/FlinkUserCodeClassLoader.java diff --git a/flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java b/flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java index ff03601b25..f1b663ee74 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java +++ b/flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java @@ -40,7 +40,6 @@ import org.slf4j.LoggerFactory; import java.io.IOException; import java.net.URL; -import java.net.URLClassLoader; import java.util.List; import java.util.concurrent.ExecutionException; @@ -54,7 +53,7 @@ public enum ClientUtils { private static final Logger LOG = LoggerFactory.getLogger(ClientUtils.class); - public static URLClassLoader buildUserCodeClassLoader( + public static ClassLoader buildUserCodeClassLoader( List jars, List classpaths, ClassLoader parent, diff --git a/flink-core/src/main/java/org/apache/flink/util/ChildFirstClassLoader.java b/flink-core/src/main/java/org/apache/flink/util/ChildFirstClassLoader.java index 0cac006db4..3942b1bd6a 100644 --- a/flink-core/src/main/java/org/apache/flink/util/ChildFirstClassLoader.java +++ b/flink-core/src/main/java/org/apache/flink/util/ChildFirstClassLoader.java @@ -20,6 +20,7 @@ package org.apache.flink.util; import java.io.IOException; import java.net.URL; +import java.net.URLClassLoader; import java.util.ArrayList; import java.util.Enumeration; import java.util.Iterator; @@ -31,7 +32,7 @@ import java.util.List; *

    {@link #getResourceAsStream(String)} uses {@link #getResource(String)} internally so we * don't override that. */ -public final class ChildFirstClassLoader extends FlinkUserCodeClassLoader { +public final class ChildFirstClassLoader extends URLClassLoader { /** * The classes that should always go through the parent ClassLoader. This is relevant @@ -46,7 +47,7 @@ public final class ChildFirstClassLoader extends FlinkUserCodeClassLoader { } @Override - public synchronized Class loadClass( + protected synchronized Class loadClass( String name, boolean resolve) throws ClassNotFoundException { // First, check if the class has already been loaded diff --git a/flink-core/src/main/java/org/apache/flink/util/FlinkUserCodeClassLoader.java b/flink-core/src/main/java/org/apache/flink/util/FlinkUserCodeClassLoader.java deleted file mode 100644 index 7e49bf6846..0000000000 --- a/flink-core/src/main/java/org/apache/flink/util/FlinkUserCodeClassLoader.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * 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.util; - -import org.apache.flink.annotation.Internal; - -import java.net.URL; -import java.net.URLClassLoader; - -/** - * Base class for user code class loaders (child-first, parent-first). - */ -@Internal -public class FlinkUserCodeClassLoader extends URLClassLoader { - public FlinkUserCodeClassLoader(URL[] urls, ClassLoader parent) { - super(urls, parent); - } - - @Override - public Class loadClass(String name, boolean resolve) throws ClassNotFoundException { - return super.loadClass(name, resolve); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoaders.java index aa9b123bc2..0df7c7aaf2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoaders.java @@ -19,16 +19,9 @@ package org.apache.flink.runtime.execution.librarycache; import org.apache.flink.util.ChildFirstClassLoader; -import org.apache.flink.util.FlinkUserCodeClassLoader; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.Closeable; -import java.io.IOException; import java.net.URL; import java.net.URLClassLoader; -import java.util.Enumeration; /** * Gives the URLClassLoader a nicer name for debugging purposes. @@ -36,14 +29,14 @@ import java.util.Enumeration; public class FlinkUserCodeClassLoaders { public static URLClassLoader parentFirst(URL[] urls, ClassLoader parent) { - return new SafetyNetWrapperClassLoader(new ParentFirstClassLoader(urls, parent)); + return new ParentFirstClassLoader(urls, parent); } public static URLClassLoader childFirst( URL[] urls, ClassLoader parent, String[] alwaysParentFirstPatterns) { - return new SafetyNetWrapperClassLoader(new ChildFirstClassLoader(urls, parent, alwaysParentFirstPatterns)); + return new ChildFirstClassLoader(urls, parent, alwaysParentFirstPatterns); } public static URLClassLoader create( @@ -79,7 +72,7 @@ public class FlinkUserCodeClassLoaders { /** * Regular URLClassLoader that first loads from the parent and only after that from the URLs. */ - static class ParentFirstClassLoader extends FlinkUserCodeClassLoader { + static class ParentFirstClassLoader extends URLClassLoader { ParentFirstClassLoader(URL[] urls) { this(urls, FlinkUserCodeClassLoaders.class.getClassLoader()); @@ -89,64 +82,4 @@ public class FlinkUserCodeClassLoaders { super(urls, parent); } } - - /** - * Ensures that holding a reference on the context class loader outliving the scope of user code does not prevent - * the user classloader to be garbage collected (FLINK-16245). - * - *

    This classloader delegates to the actual user classloader. Upon {@link #close()}, the delegate is nulled - * and can be garbage collected. Additional class resolution will be resolved solely through the bootstrap - * classloader and most likely result in ClassNotFound exceptions. - */ - private static class SafetyNetWrapperClassLoader extends URLClassLoader implements Closeable { - private static final Logger LOG = LoggerFactory.getLogger(SafetyNetWrapperClassLoader.class); - - private FlinkUserCodeClassLoader inner; - - SafetyNetWrapperClassLoader(FlinkUserCodeClassLoader inner) { - super(new URL[0], null); - this.inner = inner; - } - - @Override - public void close() { - if (inner != null) { - try { - inner.close(); - } catch (IOException e) { - LOG.warn("Could not close user classloader", e); - } - } - inner = null; - } - - @Override - protected Class loadClass(String name, boolean resolve) throws ClassNotFoundException { - if (inner == null) { - try { - return super.loadClass(name, resolve); - } catch (ClassNotFoundException e) { - throw new ClassNotFoundException("Flink user code classloader was already closed.", e); - } - } - - return inner.loadClass(name, resolve); - } - - @Override - public URL findResource(String name) { - if (inner == null) { - return super.findResource(name); - } - return inner.getResource(name); - } - - @Override - public Enumeration findResources(String name) throws IOException { - if (inner == null) { - return super.findResources(name); - } - return inner.getResources(name); - } - } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoadersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoadersTest.java index 240ba15082..4333aa8b69 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoadersTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/FlinkUserCodeClassLoadersTest.java @@ -39,7 +39,6 @@ import static org.hamcrest.Matchers.hasItemInArray; import static org.hamcrest.Matchers.hasProperty; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; -import static org.junit.Assert.assertNotSame; /** * Tests for classloading and class loader utilities. @@ -193,32 +192,4 @@ public class FlinkUserCodeClassLoadersTest extends TestLogger { childClassLoader.close(); } - - @Test - public void testClosingOfClassloader() throws Exception { - final String className = ClassToLoad.class.getName(); - - final ClassLoader parentClassLoader = ClassLoader.getSystemClassLoader().getParent(); - - final URL childCodePath = getClass().getProtectionDomain().getCodeSource().getLocation(); - - final URLClassLoader childClassLoader = FlinkUserCodeClassLoaders.create( - FlinkUserCodeClassLoaders.ResolveOrder.CHILD_FIRST, - new URL[] { childCodePath }, - parentClassLoader, - new String[0]); - - final Class loadedClass = childClassLoader.loadClass(className); - - assertNotSame(ClassToLoad.class, loadedClass); - - childClassLoader.close(); - - // after closing, no loaded class should be reachable anymore - expectedException.expect(isA(ClassNotFoundException.class)); - childClassLoader.loadClass(className); - } - - private static class ClassToLoad { - } } diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java index 8a8fe85edf..0529434326 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java @@ -96,11 +96,8 @@ import org.slf4j.LoggerFactory; import javax.annotation.Nullable; -import java.io.Closeable; -import java.io.IOException; import java.lang.reflect.Method; import java.net.URL; -import java.net.URLClassLoader; import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashMap; @@ -121,13 +118,13 @@ import static org.apache.flink.util.Preconditions.checkState; * * @param cluster id */ -public class ExecutionContext implements Closeable { +public class ExecutionContext { private static final Logger LOG = LoggerFactory.getLogger(ExecutionContext.class); private final Environment environment; private final SessionContext originalSessionContext; - private final URLClassLoader classLoader; + private final ClassLoader classLoader; private final Configuration flinkConfig; private final ClusterClientFactory clusterClientFactory; @@ -706,11 +703,6 @@ public class ExecutionContext implements Closeable { } } - @Override - public void close() throws IOException { - this.classLoader.close(); - } - //~ Inner Class ------------------------------------------------------------------------------- /** Builder for {@link ExecutionContext}. */ diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java index a133be511b..1c9ea2a276 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java @@ -239,12 +239,7 @@ public class LocalExecutor implements Executor { } }); // Remove the session's ExecutionContext from contextMap. - try { - this.contextMap.remove(sessionId).close(); - } catch (IOException e) { - LOG.debug("Error while closing execution context.", e); - // ignore any throwable to keep the clean up running - } + this.contextMap.remove(sessionId); } /** diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/BatchFineGrainedRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/BatchFineGrainedRecoveryITCase.java index 86e32edf3c..9a2a52c470 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/BatchFineGrainedRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/BatchFineGrainedRecoveryITCase.java @@ -60,7 +60,6 @@ import org.apache.flink.util.ConfigurationException; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.FlinkRuntimeException; -import org.apache.flink.util.TemporaryClassLoaderContext; import org.apache.flink.util.TestLogger; import org.junit.After; @@ -397,16 +396,14 @@ public class BatchFineGrainedRecoveryITCase extends TestLogger { @Override void fail(int trackingIndex) throws Exception { //noinspection OverlyBroadCatchBlock - try (TemporaryClassLoaderContext unused = TemporaryClassLoaderContext.of(ClassLoader.getSystemClassLoader())) { - try { - restartTaskManager(); - } catch (InterruptedException e) { - // ignore the exception, task should have been failed while stopping TM - Thread.currentThread().interrupt(); - } catch (Throwable t) { - failureTracker.unrelatedFailure(t); - throw t; - } + try { + restartTaskManager(); + } catch (InterruptedException e) { + // ignore the exception, task should have been failed while stopping TM + Thread.currentThread().interrupt(); + } catch (Throwable t) { + failureTracker.unrelatedFailure(t); + throw t; } } } -- Gitee From 3fa4a7e5ae9befc5a01df4202942090b63b60a69 Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Thu, 2 Apr 2020 10:20:01 +0200 Subject: [PATCH 417/885] [FLINK-16939][rest] Declare field taskManagerIdParameter final This closes #11612. --- .../rest/messages/taskmanager/TaskManagerMessageParameters.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerMessageParameters.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerMessageParameters.java index 4a761215bc..aafe4eee19 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerMessageParameters.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerMessageParameters.java @@ -31,7 +31,7 @@ import java.util.Collections; */ public class TaskManagerMessageParameters extends MessageParameters { - public TaskManagerIdPathParameter taskManagerIdParameter = new TaskManagerIdPathParameter(); + public final TaskManagerIdPathParameter taskManagerIdParameter = new TaskManagerIdPathParameter(); @Override public Collection> getPathParameters() { -- Gitee From f1c91cca3f66bd5f91446a58ce1b2d4c6015af4a Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Thu, 2 Apr 2020 10:33:41 +0200 Subject: [PATCH 418/885] [FLINK-16940][runtime] Create currentRegion HashSet with default capacity This closes #11613. --- .../failover/flip1/PipelinedRegionComputeUtil.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/PipelinedRegionComputeUtil.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/PipelinedRegionComputeUtil.java index 856116b21b..73f77a59f3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/PipelinedRegionComputeUtil.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/PipelinedRegionComputeUtil.java @@ -72,7 +72,7 @@ public final class PipelinedRegionComputeUtil { // iterate all the vertices which are topologically sorted for (V vertex : topology.getVertices()) { - Set currentRegion = new HashSet<>(1); + Set currentRegion = new HashSet<>(); currentRegion.add(vertex); vertexToRegion.put(vertex, currentRegion); -- Gitee From 88cd7007b4b192f0273c5f7c311d56be52612147 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Wed, 18 Mar 2020 12:36:53 +0100 Subject: [PATCH 419/885] [hotfix][coordination] Add sanity checks to ClusterPartitionReportEntry --- .../taskexecutor/partition/ClusterPartitionReport.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/partition/ClusterPartitionReport.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/partition/ClusterPartitionReport.java index a75dfa8b82..1e93b1d4ba 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/partition/ClusterPartitionReport.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/partition/ClusterPartitionReport.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.taskexecutor.partition; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.util.Preconditions; import java.io.Serializable; import java.util.Collection; @@ -63,6 +64,12 @@ public class ClusterPartitionReport implements Serializable { private final int numTotalPartitions; public ClusterPartitionReportEntry(IntermediateDataSetID dataSetId, Set hostedPartitions, int numTotalPartitions) { + Preconditions.checkNotNull(dataSetId); + Preconditions.checkNotNull(hostedPartitions); + Preconditions.checkArgument(!hostedPartitions.isEmpty()); + Preconditions.checkArgument(numTotalPartitions > 0); + Preconditions.checkState(hostedPartitions.size() <= numTotalPartitions); + this.dataSetId = dataSetId; this.hostedPartitions = hostedPartitions; this.numTotalPartitions = numTotalPartitions; -- Gitee From e63c31ba8a28a609e385eb98a874c2abdf72a28e Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Wed, 18 Mar 2020 11:02:58 +0100 Subject: [PATCH 420/885] [FLINK-14791][coordination] ResourceManager tracks ClusterPartitions --- .../kubernetes/KubernetesResourceManager.java | 3 + .../KubernetesResourceManagerFactory.java | 2 + .../KubernetesResourceManagerTest.java | 2 + .../MesosResourceManager.java | 7 +- .../MesosResourceManagerFactory.java | 2 + .../MesosResourceManagerTest.java | 2 + .../io/network/partition/DataSetMetaInfo.java | 34 +++ .../ResourceManagerPartitionTracker.java | 67 +++++ ...esourceManagerPartitionTrackerFactory.java | 26 ++ .../ResourceManagerPartitionTrackerImpl.java | 247 ++++++++++++++++++ .../TaskExecutorClusterPartitionReleaser.java | 31 +++ .../ActiveResourceManager.java | 7 +- .../resourcemanager/ResourceManager.java | 20 +- .../StandaloneResourceManager.java | 9 +- .../StandaloneResourceManagerFactory.java | 6 +- .../runtime/taskexecutor/TaskExecutor.java | 3 +- .../taskexecutor/TaskExecutorGateway.java | 3 +- .../NoOpResourceManagerPartitionTracker.java | 57 ++++ ...sourceManagerPartitionTrackerImplTest.java | 234 +++++++++++++++++ ...ndaloneResourceManagerWithUUIDFactory.java | 6 +- .../ResourceManagerHATest.java | 6 +- .../ResourceManagerJobMasterTest.java | 5 +- ...ResourceManagerPartitionLifecycleTest.java | 247 ++++++++++++++++++ .../ResourceManagerTaskExecutorTest.java | 5 +- .../resourcemanager/ResourceManagerTest.java | 2 + .../StandaloneResourceManagerTest.java | 6 +- .../TestingResourceManager.java | 7 +- .../TestingTaskExecutorGateway.java | 8 +- .../TestingTaskExecutorGatewayBuilder.java | 9 + .../flink/yarn/YarnResourceManager.java | 3 + .../YarnResourceManagerFactory.java | 2 + .../flink/yarn/YarnResourceManagerTest.java | 2 + 32 files changed, 1055 insertions(+), 15 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/DataSetMetaInfo.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTracker.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTrackerFactory.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTrackerImpl.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorClusterPartitionReleaser.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/NoOpResourceManagerPartitionTracker.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTrackerImplTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerPartitionLifecycleTest.java diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManager.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManager.java index d3c65e1e83..c8a70b55ef 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManager.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/KubernetesResourceManager.java @@ -37,6 +37,7 @@ import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.entrypoint.ClusterInformation; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.io.network.partition.ResourceManagerPartitionTrackerFactory; import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup; import org.apache.flink.runtime.resourcemanager.ActiveResourceManager; import org.apache.flink.runtime.resourcemanager.JobLeaderIdService; @@ -96,6 +97,7 @@ public class KubernetesResourceManager extends ActiveResourceManager releaseClusterPartitions(IntermediateDataSetID dataSetId); + + /** + * Returns all data sets for which all partitions are being tracked. + * + * @return completely tracked datasets + */ + Map listDataSets(); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTrackerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTrackerFactory.java new file mode 100644 index 0000000000..215e3db455 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTrackerFactory.java @@ -0,0 +1,26 @@ +/* + * 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.runtime.io.network.partition; + +/** + * Factory for {@link ResourceManagerPartitionTracker}. + */ +@FunctionalInterface +public interface ResourceManagerPartitionTrackerFactory { + ResourceManagerPartitionTracker get(TaskExecutorClusterPartitionReleaser taskExecutorClusterPartitionReleaser); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTrackerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTrackerImpl.java new file mode 100644 index 0000000000..47940ed6a3 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTrackerImpl.java @@ -0,0 +1,247 @@ +/* + * 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.runtime.io.network.partition; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.taskexecutor.partition.ClusterPartitionReport; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; + +/** + * Default {@link ResourceManagerPartitionTracker} implementation. + * + *

    Internal tracking info must only be updated upon reception of a {@link ClusterPartitionReport}, as the task + * executor state is the source of truth. + */ +public class ResourceManagerPartitionTrackerImpl implements ResourceManagerPartitionTracker { + + private static final Logger LOG = LoggerFactory.getLogger(ResourceManagerPartitionTrackerImpl.class); + + private final Map> taskExecutorToDataSets = new HashMap<>(); + private final Map>> dataSetToTaskExecutors = new HashMap<>(); + private final Map dataSetMetaInfo = new HashMap<>(); + private final Map> partitionReleaseCompletionFutures = new HashMap<>(); + + private final TaskExecutorClusterPartitionReleaser taskExecutorClusterPartitionReleaser; + + public ResourceManagerPartitionTrackerImpl(TaskExecutorClusterPartitionReleaser taskExecutorClusterPartitionReleaser) { + this.taskExecutorClusterPartitionReleaser = taskExecutorClusterPartitionReleaser; + } + + @Override + public void processTaskExecutorClusterPartitionReport(ResourceID taskExecutorId, ClusterPartitionReport clusterPartitionReport) { + Preconditions.checkNotNull(taskExecutorId); + Preconditions.checkNotNull(clusterPartitionReport); + LOG.debug("Processing cluster partition report from task executor {}: {}.", taskExecutorId, clusterPartitionReport); + + internalProcessClusterPartitionReport(taskExecutorId, clusterPartitionReport); + } + + @Override + public void processTaskExecutorShutdown(ResourceID taskExecutorId) { + Preconditions.checkNotNull(taskExecutorId); + LOG.debug("Processing shutdown of task executor {}.", taskExecutorId); + + internalProcessClusterPartitionReport(taskExecutorId, new ClusterPartitionReport(Collections.emptyList())); + } + + @Override + public CompletableFuture releaseClusterPartitions(IntermediateDataSetID dataSetId) { + Preconditions.checkNotNull(dataSetId); + if (!dataSetMetaInfo.containsKey(dataSetId)) { + LOG.debug("Attempted released of unknown data set {}.", dataSetId); + return CompletableFuture.completedFuture(null); + } + LOG.debug("Releasing cluster partitions for data set {}.", dataSetId); + + CompletableFuture partitionReleaseCompletionFuture = partitionReleaseCompletionFutures.computeIfAbsent(dataSetId, ignored -> new CompletableFuture<>()); + internalReleasePartitions(Collections.singleton(dataSetId)); + return partitionReleaseCompletionFuture; + } + + private void internalProcessClusterPartitionReport(ResourceID taskExecutorId, ClusterPartitionReport clusterPartitionReport) { + final Set dataSetsWithLostPartitions = clusterPartitionReport.getEntries().isEmpty() + ? processEmptyReport(taskExecutorId) + : setHostedDataSetsAndCheckCorruption(taskExecutorId, clusterPartitionReport.getEntries()); + + updateDataSetMetaData(clusterPartitionReport); + + checkForFullyLostDatasets(dataSetsWithLostPartitions); + + internalReleasePartitions(dataSetsWithLostPartitions); + } + + private void internalReleasePartitions(Set dataSetsToRelease) { + Map> releaseCalls = prepareReleaseCalls(dataSetsToRelease); + releaseCalls.forEach(taskExecutorClusterPartitionReleaser::releaseClusterPartitions); + } + + private Set processEmptyReport(ResourceID taskExecutorId) { + Set previouslyHostedDatasets = taskExecutorToDataSets.remove(taskExecutorId); + if (previouslyHostedDatasets == null) { + // default path for task executors that never have any cluster partitions + previouslyHostedDatasets = Collections.emptySet(); + } else { + previouslyHostedDatasets.forEach(dataSetId -> removeInnerKey(dataSetId, taskExecutorId, dataSetToTaskExecutors)); + } + return previouslyHostedDatasets; + } + + /** + * Updates the data sets for which the given task executor is hosting partitions and returns data sets that were + * corrupted due to a loss of partitions. + * + * @param taskExecutorId ID of the hosting TaskExecutor + * @param reportEntries IDs of data sets for which partitions are hosted + * @return corrupted data sets + */ + private Set setHostedDataSetsAndCheckCorruption(ResourceID taskExecutorId, Collection reportEntries) { + final Set currentlyHostedDatasets = reportEntries + .stream() + .map(ClusterPartitionReport.ClusterPartitionReportEntry::getDataSetId) + .collect(Collectors.toSet()); + + final Set previouslyHostedDataSets = taskExecutorToDataSets.put( + taskExecutorId, + currentlyHostedDatasets); + + // previously tracked data sets may be corrupted since we may be tracking less partitions than before + final Set potentiallyCorruptedDataSets = Optional + .ofNullable(previouslyHostedDataSets) + .orElse(new HashSet<>(0)); + + // update data set -> task executor mapping and find datasets for which lost a partition + reportEntries.forEach(hostedPartition -> { + final Map> taskExecutorHosts = dataSetToTaskExecutors.computeIfAbsent(hostedPartition.getDataSetId(), ignored -> new HashMap<>()); + final Set previouslyHostedPartitions = taskExecutorHosts.put(taskExecutorId, hostedPartition.getHostedPartitions()); + + final boolean noPartitionLost = previouslyHostedPartitions == null || hostedPartition.getHostedPartitions().containsAll(previouslyHostedPartitions); + if (noPartitionLost) { + potentiallyCorruptedDataSets.remove(hostedPartition.getDataSetId()); + } + }); + + // now only contains data sets for which a partition is no longer tracked + return potentiallyCorruptedDataSets; + } + + private void updateDataSetMetaData(ClusterPartitionReport clusterPartitionReport) { + // add meta info for new data sets + clusterPartitionReport.getEntries().forEach(entry -> + dataSetMetaInfo.compute(entry.getDataSetId(), (dataSetID, dataSetMetaInfo) -> { + if (dataSetMetaInfo == null) { + return new DataSetMetaInfo(entry.getNumTotalPartitions()); + } else { + // double check that the meta data is consistent + Preconditions.checkState(dataSetMetaInfo.getNumTotalPartitions() == entry.getNumTotalPartitions()); + return dataSetMetaInfo; + } + })); + } + + private void checkForFullyLostDatasets(Set dataSetsWithLostPartitions) { + dataSetsWithLostPartitions.forEach(dataSetId -> { + if (getHostingTaskExecutors(dataSetId).isEmpty()) { + LOG.debug("There are no longer partitions being tracked for dataset {}.", dataSetId); + dataSetMetaInfo.remove(dataSetId); + Optional.ofNullable(partitionReleaseCompletionFutures.remove(dataSetId)).map(future -> future.complete(null)); + } + }); + } + + private Map> prepareReleaseCalls(Set dataSetsToRelease) { + final Map> releaseCalls = new HashMap<>(); + dataSetsToRelease.forEach(dataSetToRelease -> { + final Set hostingTaskExecutors = getHostingTaskExecutors(dataSetToRelease); + hostingTaskExecutors.forEach(hostingTaskExecutor -> insert(hostingTaskExecutor, dataSetToRelease, releaseCalls)); + }); + return releaseCalls; + } + + private Set getHostingTaskExecutors(IntermediateDataSetID dataSetId) { + Preconditions.checkNotNull(dataSetId); + + Map> trackedPartitions = dataSetToTaskExecutors.get(dataSetId); + if (trackedPartitions == null) { + return Collections.emptySet(); + } else { + return trackedPartitions.keySet(); + } + } + + @Override + public Map listDataSets() { + return dataSetMetaInfo.entrySet().stream() + .filter(entry -> { + final Map> taskExecutorToPartitions = dataSetToTaskExecutors.get(entry.getKey()); + Preconditions.checkState(taskExecutorToPartitions != null, "Have metadata entry for dataset %s, but no partition is tracked.", entry.getKey()); + + int numTrackedPartitions = 0; + for (Set hostedPartitions : taskExecutorToPartitions.values()) { + numTrackedPartitions += hostedPartitions.size(); + } + + return numTrackedPartitions == entry.getValue().getNumTotalPartitions(); + }) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + } + + /** + * Returns whether all maps are empty; used for checking for resource leaks in case entries aren't properly removed. + * + * @return whether all contained maps are empty + */ + @VisibleForTesting + boolean areAllMapsEmpty() { + return taskExecutorToDataSets.isEmpty() && dataSetToTaskExecutors.isEmpty() && dataSetMetaInfo.isEmpty() && partitionReleaseCompletionFutures.isEmpty(); + } + + private static void insert(K key1, V value, Map> collection) { + collection.compute(key1, (key, values) -> { + if (values == null) { + values = new HashSet<>(); + } + values.add(value); + return values; + }); + } + + private static void removeInnerKey(K1 key1, K2 value, Map> collection) { + collection.computeIfPresent(key1, (key, values) -> { + values.remove(value); + if (values.isEmpty()) { + return null; + } + return values; + }); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorClusterPartitionReleaser.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorClusterPartitionReleaser.java new file mode 100644 index 0000000000..2f9dd0f44a --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorClusterPartitionReleaser.java @@ -0,0 +1,31 @@ +/* + * 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.runtime.io.network.partition; + +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; + +import java.util.Set; + +/** + * Interface for releasing cluster partitions on a task executor. + */ +@FunctionalInterface +public interface TaskExecutorClusterPartitionReleaser { + void releaseClusterPartitions(ResourceID taskExecutorId, Set dataSetsToRelease); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ActiveResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ActiveResourceManager.java index fc81765165..166d967b9e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ActiveResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ActiveResourceManager.java @@ -20,6 +20,7 @@ package org.apache.flink.runtime.resourcemanager; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec; import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils; import org.apache.flink.runtime.clusterframework.types.ResourceID; @@ -29,6 +30,7 @@ import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.entrypoint.ClusterInformation; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.io.network.partition.ResourceManagerPartitionTrackerFactory; import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; import org.apache.flink.runtime.rpc.FatalErrorHandler; @@ -76,6 +78,7 @@ public abstract class ActiveResourceManager /** The slot manager maintains the available slots. */ private final SlotManager slotManager; + private final ResourceManagerPartitionTracker clusterPartitionTracker; + private final ClusterInformation clusterInformation; private final ResourceManagerMetricGroup resourceManagerMetricGroup; @@ -155,10 +160,12 @@ public abstract class ResourceManager HighAvailabilityServices highAvailabilityServices, HeartbeatServices heartbeatServices, SlotManager slotManager, + ResourceManagerPartitionTrackerFactory clusterPartitionTrackerFactory, JobLeaderIdService jobLeaderIdService, ClusterInformation clusterInformation, FatalErrorHandler fatalErrorHandler, - ResourceManagerMetricGroup resourceManagerMetricGroup) { + ResourceManagerMetricGroup resourceManagerMetricGroup, + Time rpcTimeout) { super(rpcService, resourceManagerEndpointId, null); @@ -178,6 +185,15 @@ public abstract class ResourceManager this.jobManagerHeartbeatManager = NoOpHeartbeatManager.getInstance(); this.taskManagerHeartbeatManager = NoOpHeartbeatManager.getInstance(); + + this.clusterPartitionTracker = checkNotNull(clusterPartitionTrackerFactory).get( + (taskExecutorResourceId, dataSetIds) -> taskExecutors.get(taskExecutorResourceId).getTaskExecutorGateway() + .releaseClusterPartitions(dataSetIds, rpcTimeout) + .exceptionally(throwable -> { + log.debug("Request for release of cluster partitions belonging to data sets {} was not successful.", dataSetIds, throwable); + throw new CompletionException(throwable); + }) + ); } @@ -818,6 +834,7 @@ public abstract class ResourceManager // TODO :: suggest failed task executor to stop itself slotManager.unregisterTaskManager(workerRegistration.getInstanceID(), cause); + clusterPartitionTracker.processTaskExecutorShutdown(resourceID); workerRegistration.getTaskExecutorGateway().disconnectResourceManager(cause); } else { @@ -1164,6 +1181,7 @@ public abstract class ResourceManager InstanceID instanceId = workerRegistration.getInstanceID(); slotManager.reportSlotStatus(instanceId, payload.getSlotReport()); + clusterPartitionTracker.processTaskExecutorClusterPartitionReport(resourceID, payload.getClusterPartitionReport()); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java index d8fb0a6e62..d1eceacec9 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java @@ -25,6 +25,7 @@ import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.entrypoint.ClusterInformation; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.io.network.partition.ResourceManagerPartitionTrackerFactory; import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup; import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; @@ -56,11 +57,13 @@ public class StandaloneResourceManager extends ResourceManager { HighAvailabilityServices highAvailabilityServices, HeartbeatServices heartbeatServices, SlotManager slotManager, + ResourceManagerPartitionTrackerFactory clusterPartitionTrackerFactory, JobLeaderIdService jobLeaderIdService, ClusterInformation clusterInformation, FatalErrorHandler fatalErrorHandler, ResourceManagerMetricGroup resourceManagerMetricGroup, - Time startupPeriodTime) { + Time startupPeriodTime, + Time rpcTimeout) { super( rpcService, resourceManagerEndpointId, @@ -68,10 +71,12 @@ public class StandaloneResourceManager extends ResourceManager { highAvailabilityServices, heartbeatServices, slotManager, + clusterPartitionTrackerFactory, jobLeaderIdService, clusterInformation, fatalErrorHandler, - resourceManagerMetricGroup); + resourceManagerMetricGroup, + rpcTimeout); this.startupPeriodTime = Preconditions.checkNotNull(startupPeriodTime); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManagerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManagerFactory.java index e5055a6c3e..26cc5b235d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManagerFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManagerFactory.java @@ -21,10 +21,12 @@ package org.apache.flink.runtime.resourcemanager; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ConfigurationUtils; +import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.entrypoint.ClusterInformation; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.io.network.partition.ResourceManagerPartitionTrackerImpl; import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; @@ -63,10 +65,12 @@ public enum StandaloneResourceManagerFactory implements ResourceManagerFactory dataSetsToRelease, Time timeout) { + public CompletableFuture releaseClusterPartitions(Collection dataSetsToRelease, Time timeout) { partitionTracker.stopTrackingAndReleaseClusterPartitions(dataSetsToRelease); + return CompletableFuture.completedFuture(Acknowledge.get()); } // ---------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java index 69257be8b8..01e5d540bb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java @@ -120,8 +120,9 @@ public interface TaskExecutorGateway extends RpcGateway, TaskExecutorOperatorEve * * @param dataSetsToRelease data sets for which all cluster partitions should be released * @param timeout for the partitions release operation + * @return Future acknowledge that the request was received */ - void releaseClusterPartitions(Collection dataSetsToRelease, @RpcTimeout Time timeout); + CompletableFuture releaseClusterPartitions(Collection dataSetsToRelease, @RpcTimeout Time timeout); /** * Trigger the checkpoint for the given task. The checkpoint is identified by the checkpoint ID diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/NoOpResourceManagerPartitionTracker.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/NoOpResourceManagerPartitionTracker.java new file mode 100644 index 0000000000..f92a092588 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/NoOpResourceManagerPartitionTracker.java @@ -0,0 +1,57 @@ +/* + * 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.runtime.io.network.partition; + +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.taskexecutor.partition.ClusterPartitionReport; + +import java.util.Collections; +import java.util.Map; +import java.util.concurrent.CompletableFuture; + +/** + * No-op {@link ResourceManagerPartitionTracker} implementation; does not track any partition and never regards a data + * set corrupted. + */ +public enum NoOpResourceManagerPartitionTracker implements ResourceManagerPartitionTracker { + INSTANCE; + + @Override + public void processTaskExecutorClusterPartitionReport(ResourceID taskExecutorId, ClusterPartitionReport clusterPartitionReport) { + } + + @Override + public void processTaskExecutorShutdown(ResourceID taskExecutorId) { + } + + @Override + public CompletableFuture releaseClusterPartitions(IntermediateDataSetID dataSetId) { + return CompletableFuture.completedFuture(null); + } + + @Override + public Map listDataSets() { + return Collections.emptyMap(); + } + + @SuppressWarnings("unused") // unused parameter allows usage as a ResourceManagerPartitionTrackerFactory + public static ResourceManagerPartitionTracker get(TaskExecutorClusterPartitionReleaser ignored) { + return INSTANCE; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTrackerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTrackerImplTest.java new file mode 100644 index 0000000000..71e1c76883 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTrackerImplTest.java @@ -0,0 +1,234 @@ +/* + * 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.runtime.io.network.partition; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.taskexecutor.partition.ClusterPartitionReport; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.hasKey; +import static org.hamcrest.collection.IsEmptyCollection.empty; +import static org.hamcrest.core.Is.is; + +/** + * Test for the {@link ResourceManagerPartitionTrackerImpl}. + */ +public class ResourceManagerPartitionTrackerImplTest extends TestLogger { + + private static final ClusterPartitionReport EMPTY_PARTITION_REPORT = new ClusterPartitionReport(Collections.emptySet()); + + private static final ResourceID TASK_EXECUTOR_ID_1 = ResourceID.generate(); + private static final ResourceID TASK_EXECUTOR_ID_2 = ResourceID.generate(); + private static final IntermediateDataSetID DATA_SET_ID = new IntermediateDataSetID(); + private static final ResultPartitionID PARTITION_ID_1 = new ResultPartitionID(); + private static final ResultPartitionID PARTITION_ID_2 = new ResultPartitionID(); + + @Test + public void testProcessEmptyClusterPartitionReport() { + TestClusterPartitionReleaser partitionReleaser = new TestClusterPartitionReleaser(); + final ResourceManagerPartitionTrackerImpl tracker = new ResourceManagerPartitionTrackerImpl(partitionReleaser); + + reportEmpty(tracker, TASK_EXECUTOR_ID_1); + assertThat(partitionReleaser.releaseCalls, empty()); + assertThat(tracker.areAllMapsEmpty(), is(true)); + } + + /** + * Verifies that a task executor hosting multiple partitions of a data set receives a release call if a subset of + * its partitions is lost. + */ + @Test + public void testReportProcessingWithPartitionLossOnSameTaskExecutor() { + TestClusterPartitionReleaser partitionReleaser = new TestClusterPartitionReleaser(); + final ResourceManagerPartitionTracker tracker = new ResourceManagerPartitionTrackerImpl(partitionReleaser); + + report(tracker, TASK_EXECUTOR_ID_1, DATA_SET_ID, 2, PARTITION_ID_1, PARTITION_ID_2); + report(tracker, TASK_EXECUTOR_ID_1, DATA_SET_ID, 2, PARTITION_ID_2); + + assertThat(partitionReleaser.releaseCalls, contains(Tuple2.of(TASK_EXECUTOR_ID_1, Collections.singleton(DATA_SET_ID)))); + } + + /** + * Verifies that a task executor hosting partitions of a data set receives a release call if a partition of the + * data set is lost on another task executor. + */ + @Test + public void testReportProcessingWithPartitionLossOnOtherTaskExecutor() { + TestClusterPartitionReleaser partitionReleaser = new TestClusterPartitionReleaser(); + final ResourceManagerPartitionTracker tracker = new ResourceManagerPartitionTrackerImpl(partitionReleaser); + + report(tracker, TASK_EXECUTOR_ID_1, DATA_SET_ID, 2, PARTITION_ID_1); + report(tracker, TASK_EXECUTOR_ID_2, DATA_SET_ID, 2, PARTITION_ID_2); + + reportEmpty(tracker, TASK_EXECUTOR_ID_1); + + assertThat(partitionReleaser.releaseCalls, contains(Tuple2.of(TASK_EXECUTOR_ID_2, Collections.singleton(DATA_SET_ID)))); + } + + @Test + public void testListDataSetsBasics() { + final ResourceManagerPartitionTrackerImpl tracker = new ResourceManagerPartitionTrackerImpl(new TestClusterPartitionReleaser()); + + assertThat(tracker.listDataSets().size(), is(0)); + + report(tracker, TASK_EXECUTOR_ID_1, DATA_SET_ID, 1, PARTITION_ID_1); + + final Map listing = tracker.listDataSets(); + assertThat(listing, hasKey(DATA_SET_ID)); + DataSetMetaInfo metaInfo = listing.get(DATA_SET_ID); + assertThat(metaInfo.getNumTotalPartitions(), is(1)); + + reportEmpty(tracker, TASK_EXECUTOR_ID_1); + assertThat(tracker.listDataSets().size(), is(0)); + + assertThat(tracker.areAllMapsEmpty(), is(true)); + } + + @Test + public void testListDataSetsMultiplePartitionsOnSingleTaskExecutor() { + final ResourceManagerPartitionTrackerImpl tracker = new ResourceManagerPartitionTrackerImpl(new TestClusterPartitionReleaser()); + + // data set consists of 2 partitions but only 1 is being tracked -> incomplete and should not be listed (yet) + report(tracker, TASK_EXECUTOR_ID_1, DATA_SET_ID, 2, PARTITION_ID_1); + assertThat(tracker.listDataSets().size(), is(0)); + + // start tracking another partitions, but we lost partition 1 so the data set is still incomplete + report(tracker, TASK_EXECUTOR_ID_1, DATA_SET_ID, 2, PARTITION_ID_2); + assertThat(tracker.listDataSets().size(), is(0)); + + // dataset is considered complete since all partitions are being tracked + report(tracker, TASK_EXECUTOR_ID_1, DATA_SET_ID, 2, PARTITION_ID_1, PARTITION_ID_2); + final Map listing = tracker.listDataSets(); + assertThat(listing, hasKey(DATA_SET_ID)); + + // dataset is no longer considered complete since partition 2 was lost + report(tracker, TASK_EXECUTOR_ID_1, DATA_SET_ID, 2, PARTITION_ID_1); + assertThat(tracker.listDataSets().size(), is(0)); + + assertThat(tracker.areAllMapsEmpty(), is(false)); + } + + @Test + public void testListDataSetsMultiplePartitionsAcrossTaskExecutors() { + final ResourceManagerPartitionTrackerImpl tracker = new ResourceManagerPartitionTrackerImpl(new TestClusterPartitionReleaser()); + + report(tracker, TASK_EXECUTOR_ID_1, DATA_SET_ID, 2, PARTITION_ID_1); + report(tracker, TASK_EXECUTOR_ID_2, DATA_SET_ID, 2, PARTITION_ID_2); + final Map listing = tracker.listDataSets(); + assertThat(listing, hasKey(DATA_SET_ID)); + + reportEmpty(tracker, TASK_EXECUTOR_ID_1); + assertThat(tracker.listDataSets().size(), is(0)); + + assertThat(tracker.areAllMapsEmpty(), is(false)); + } + + @Test + public void testReleasePartition() { + TestClusterPartitionReleaser partitionReleaser = new TestClusterPartitionReleaser(); + final ResourceManagerPartitionTrackerImpl tracker = new ResourceManagerPartitionTrackerImpl(partitionReleaser); + + report(tracker, TASK_EXECUTOR_ID_1, DATA_SET_ID, 2, PARTITION_ID_1); + report(tracker, TASK_EXECUTOR_ID_2, DATA_SET_ID, 2, PARTITION_ID_2); + + final CompletableFuture partitionReleaseFuture = tracker.releaseClusterPartitions(DATA_SET_ID); + + assertThat(partitionReleaser.releaseCalls, containsInAnyOrder( + Tuple2.of(TASK_EXECUTOR_ID_1, Collections.singleton(DATA_SET_ID)), + Tuple2.of(TASK_EXECUTOR_ID_2, Collections.singleton(DATA_SET_ID)))); + + // the data set should still be tracked, since the partition release was not confirmed yet by the task executors + assertThat(tracker.listDataSets().keySet(), contains(DATA_SET_ID)); + + // ack the partition release + reportEmpty(tracker, TASK_EXECUTOR_ID_1, TASK_EXECUTOR_ID_2); + + assertThat(partitionReleaseFuture.isDone(), is(true)); + assertThat(tracker.areAllMapsEmpty(), is(true)); + } + + @Test + public void testShutdownProcessing() { + TestClusterPartitionReleaser partitionReleaser = new TestClusterPartitionReleaser(); + final ResourceManagerPartitionTrackerImpl tracker = new ResourceManagerPartitionTrackerImpl(partitionReleaser); + + tracker.processTaskExecutorShutdown(TASK_EXECUTOR_ID_1); + assertThat(partitionReleaser.releaseCalls, empty()); + + report(tracker, TASK_EXECUTOR_ID_1, DATA_SET_ID, 3, PARTITION_ID_1, PARTITION_ID_2); + report(tracker, TASK_EXECUTOR_ID_2, DATA_SET_ID, 3, new ResultPartitionID()); + + tracker.processTaskExecutorShutdown(TASK_EXECUTOR_ID_1); + + assertThat(partitionReleaser.releaseCalls, contains(Tuple2.of(TASK_EXECUTOR_ID_2, Collections.singleton(DATA_SET_ID)))); + + assertThat(tracker.areAllMapsEmpty(), is(false)); + + tracker.processTaskExecutorShutdown(TASK_EXECUTOR_ID_2); + + assertThat(tracker.areAllMapsEmpty(), is(true)); + } + + private static void reportEmpty(ResourceManagerPartitionTracker tracker, ResourceID... taskExecutorIds) { + for (ResourceID taskExecutorId : taskExecutorIds) { + tracker.processTaskExecutorClusterPartitionReport( + taskExecutorId, + EMPTY_PARTITION_REPORT); + } + } + + private static void report(ResourceManagerPartitionTracker tracker, ResourceID taskExecutorId, IntermediateDataSetID dataSetId, int numTotalPartitions, ResultPartitionID... partitionIds) { + tracker.processTaskExecutorClusterPartitionReport( + taskExecutorId, + createClusterPartitionReport(dataSetId, numTotalPartitions, partitionIds)); + } + + private static ClusterPartitionReport createClusterPartitionReport(IntermediateDataSetID dataSetId, int numTotalPartitions, ResultPartitionID... partitionId) { + return new ClusterPartitionReport(Collections.singletonList( + new ClusterPartitionReport.ClusterPartitionReportEntry( + dataSetId, + new HashSet<>(Arrays.asList(partitionId)), + numTotalPartitions))); + } + + private static class TestClusterPartitionReleaser implements TaskExecutorClusterPartitionReleaser { + + final List>> releaseCalls = new ArrayList<>(); + + @Override + public void releaseClusterPartitions(ResourceID taskExecutorId, Set dataSetsToRelease) { + releaseCalls.add(Tuple2.of(taskExecutorId, dataSetsToRelease)); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/StandaloneResourceManagerWithUUIDFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/StandaloneResourceManagerWithUUIDFactory.java index 40a306f2a5..20c42a7364 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/StandaloneResourceManagerWithUUIDFactory.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/StandaloneResourceManagerWithUUIDFactory.java @@ -25,6 +25,7 @@ import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.entrypoint.ClusterInformation; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.io.network.partition.NoOpResourceManagerPartitionTracker; import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup; import org.apache.flink.runtime.resourcemanager.ResourceManager; import org.apache.flink.runtime.resourcemanager.ResourceManagerFactory; @@ -33,6 +34,7 @@ import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServicesCo import org.apache.flink.runtime.resourcemanager.StandaloneResourceManager; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.rpc.RpcUtils; import javax.annotation.Nullable; @@ -69,10 +71,12 @@ public enum StandaloneResourceManagerWithUUIDFactory implements ResourceManagerF highAvailabilityServices, heartbeatServices, resourceManagerRuntimeServices.getSlotManager(), + NoOpResourceManagerPartitionTracker::get, resourceManagerRuntimeServices.getJobLeaderIdService(), clusterInformation, fatalErrorHandler, resourceManagerMetricGroup, - standaloneClusterStartupPeriodTime); + standaloneClusterStartupPeriodTime, + RpcUtils.INF_TIMEOUT); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java index 44c02e2ea4..314843f933 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java @@ -23,10 +23,12 @@ import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.entrypoint.ClusterInformation; import org.apache.flink.runtime.heartbeat.TestingHeartbeatServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; +import org.apache.flink.runtime.io.network.partition.NoOpResourceManagerPartitionTracker; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerConfiguration; import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.util.TestingFatalErrorHandler; @@ -87,11 +89,13 @@ public class ResourceManagerHATest extends TestLogger { highAvailabilityServices, heartbeatServices, resourceManagerRuntimeServices.getSlotManager(), + NoOpResourceManagerPartitionTracker::get, resourceManagerRuntimeServices.getJobLeaderIdService(), new ClusterInformation("localhost", 1234), testingFatalErrorHandler, UnregisteredMetricGroups.createUnregisteredResourceManagerMetricGroup(), - Time.minutes(5L)) { + Time.minutes(5L), + RpcUtils.INF_TIMEOUT) { @Override public void revokeLeadership() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java index f527887530..09ce92ddc5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java @@ -26,6 +26,7 @@ import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServicesBuilder; +import org.apache.flink.runtime.io.network.partition.NoOpResourceManagerPartitionTracker; import org.apache.flink.runtime.jobmaster.JobMaster; import org.apache.flink.runtime.jobmaster.JobMasterId; import org.apache.flink.runtime.jobmaster.JobMasterRegistrationSuccess; @@ -147,11 +148,13 @@ public class ResourceManagerJobMasterTest extends TestLogger { haServices, heartbeatServices, slotManager, + NoOpResourceManagerPartitionTracker::get, jobLeaderIdService, new ClusterInformation("localhost", 1234), testingFatalErrorHandler, UnregisteredMetricGroups.createUnregisteredResourceManagerMetricGroup(), - Time.minutes(5L)); + Time.minutes(5L), + RpcUtils.INF_TIMEOUT); resourceManager.start(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerPartitionLifecycleTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerPartitionLifecycleTest.java new file mode 100644 index 0000000000..aa20ff4638 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerPartitionLifecycleTest.java @@ -0,0 +1,247 @@ +/* + * 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.runtime.resourcemanager; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.heartbeat.HeartbeatServices; +import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; +import org.apache.flink.runtime.instance.HardwareDescription; +import org.apache.flink.runtime.io.network.partition.ResourceManagerPartitionTrackerImpl; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; +import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; +import org.apache.flink.runtime.registration.RegistrationResponse; +import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; +import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerBuilder; +import org.apache.flink.runtime.rpc.RpcUtils; +import org.apache.flink.runtime.rpc.TestingRpcService; +import org.apache.flink.runtime.taskexecutor.SlotReport; +import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; +import org.apache.flink.runtime.taskexecutor.TaskExecutorHeartbeatPayload; +import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder; +import org.apache.flink.runtime.taskexecutor.partition.ClusterPartitionReport; +import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.runtime.util.TestingFatalErrorHandler; +import org.apache.flink.util.TestLogger; + +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; + +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.instanceOf; +import static org.junit.Assert.assertThat; + +/** + * Tests for the partition-lifecycle logic in the {@link ResourceManager}. + */ +public class ResourceManagerPartitionLifecycleTest extends TestLogger { + + private static final Time TIMEOUT = Time.minutes(2L); + + private static TestingRpcService rpcService; + + private TestingHighAvailabilityServices highAvailabilityServices; + + private TestingLeaderElectionService resourceManagerLeaderElectionService; + + private TestingFatalErrorHandler testingFatalErrorHandler; + + private TestingResourceManager resourceManager; + + @BeforeClass + public static void setupClass() { + rpcService = new TestingRpcService(); + } + + @Before + public void setup() throws Exception { + highAvailabilityServices = new TestingHighAvailabilityServices(); + resourceManagerLeaderElectionService = new TestingLeaderElectionService(); + highAvailabilityServices.setResourceManagerLeaderElectionService(resourceManagerLeaderElectionService); + testingFatalErrorHandler = new TestingFatalErrorHandler(); + } + + @After + public void after() throws Exception { + if (resourceManager != null) { + RpcUtils.terminateRpcEndpoint(resourceManager, TIMEOUT); + } + + if (highAvailabilityServices != null) { + highAvailabilityServices.closeAndCleanupAllData(); + } + + if (testingFatalErrorHandler.hasExceptionOccurred()) { + testingFatalErrorHandler.rethrowError(); + } + } + + @AfterClass + public static void tearDownClass() throws Exception { + if (rpcService != null) { + RpcUtils.terminateRpcServices(TIMEOUT, rpcService); + } + } + + @Test + public void testClusterPartitionReportHandling() throws Exception { + final CompletableFuture> clusterPartitionReleaseFuture = new CompletableFuture<>(); + runTest( + builder -> builder.setReleaseClusterPartitionsConsumer(clusterPartitionReleaseFuture::complete), + (resourceManagerGateway, taskManagerId1, ignored) -> { + IntermediateDataSetID dataSetID = new IntermediateDataSetID(); + ResultPartitionID resultPartitionID = new ResultPartitionID(); + + resourceManagerGateway.heartbeatFromTaskManager( + taskManagerId1, + createTaskExecutorHeartbeatPayload(dataSetID, 2, resultPartitionID, new ResultPartitionID())); + + // send a heartbeat containing 1 partition less -> partition loss -> should result in partition release + resourceManagerGateway.heartbeatFromTaskManager( + taskManagerId1, + createTaskExecutorHeartbeatPayload(dataSetID, 2, resultPartitionID)); + + Collection intermediateDataSetIDS = clusterPartitionReleaseFuture.get(TIMEOUT.toMilliseconds(), TimeUnit.MILLISECONDS); + assertThat(intermediateDataSetIDS, contains(dataSetID)); + }); + } + + @Test + public void testTaskExecutorShutdownHandling() throws Exception { + final CompletableFuture> clusterPartitionReleaseFuture = new CompletableFuture<>(); + runTest( + builder -> builder.setReleaseClusterPartitionsConsumer(clusterPartitionReleaseFuture::complete), + (resourceManagerGateway, taskManagerId1, taskManagerId2) -> { + IntermediateDataSetID dataSetID = new IntermediateDataSetID(); + + resourceManagerGateway.heartbeatFromTaskManager( + taskManagerId1, + createTaskExecutorHeartbeatPayload(dataSetID, 2, new ResultPartitionID())); + + // we need a partition on another task executor so that there's something to release when one task executor goes down + resourceManagerGateway.heartbeatFromTaskManager( + taskManagerId2, + createTaskExecutorHeartbeatPayload(dataSetID, 2, new ResultPartitionID())); + + resourceManagerGateway.disconnectTaskManager(taskManagerId2, new RuntimeException("test exception")); + Collection intermediateDataSetIDS = clusterPartitionReleaseFuture.get(TIMEOUT.toMilliseconds(), TimeUnit.MILLISECONDS); + assertThat(intermediateDataSetIDS, contains(dataSetID)); + }); + } + + private void runTest(TaskExecutorSetup taskExecutorBuilderSetup, TestAction testAction) throws Exception { + final ResourceManagerGateway resourceManagerGateway = createAndStartResourceManager(); + + TestingTaskExecutorGatewayBuilder testingTaskExecutorGateway1Builder = new TestingTaskExecutorGatewayBuilder(); + taskExecutorBuilderSetup.accept(testingTaskExecutorGateway1Builder); + final TaskExecutorGateway taskExecutorGateway1 = testingTaskExecutorGateway1Builder + .setAddress(UUID.randomUUID().toString()) + .createTestingTaskExecutorGateway(); + rpcService.registerGateway(taskExecutorGateway1.getAddress(), taskExecutorGateway1); + + final TaskExecutorGateway taskExecutorGateway2 = new TestingTaskExecutorGatewayBuilder() + .setAddress(UUID.randomUUID().toString()) + .createTestingTaskExecutorGateway(); + rpcService.registerGateway(taskExecutorGateway2.getAddress(), taskExecutorGateway2); + + final ResourceID taskManagerId1 = ResourceID.generate(); + final ResourceID taskManagerId2 = ResourceID.generate(); + registerTaskExecutor(resourceManagerGateway, taskManagerId1, taskExecutorGateway1.getAddress()); + registerTaskExecutor(resourceManagerGateway, taskManagerId2, taskExecutorGateway2.getAddress()); + + testAction.accept(resourceManagerGateway, taskManagerId1, taskManagerId2); + } + + private void registerTaskExecutor(ResourceManagerGateway resourceManagerGateway, ResourceID taskExecutorId, String taskExecutorAddress) throws Exception { + final TaskExecutorRegistration taskExecutorRegistration = new TaskExecutorRegistration( + taskExecutorAddress, + taskExecutorId, + 1234, + new HardwareDescription(42, 1337L, 1337L, 0L), + ResourceProfile.ZERO, + ResourceProfile.ZERO); + final CompletableFuture registrationFuture = resourceManagerGateway.registerTaskExecutor( + taskExecutorRegistration, + TestingUtils.TIMEOUT()); + + assertThat(registrationFuture.get(), instanceOf(RegistrationResponse.Success.class)); + } + + private ResourceManagerGateway createAndStartResourceManager() throws Exception { + final SlotManager slotManager = SlotManagerBuilder.newBuilder() + .setScheduledExecutor(rpcService.getScheduledExecutor()) + .build(); + final JobLeaderIdService jobLeaderIdService = new JobLeaderIdService( + highAvailabilityServices, + rpcService.getScheduledExecutor(), + TestingUtils.infiniteTime()); + + final TestingResourceManager resourceManager = new TestingResourceManager( + rpcService, + ResourceManager.RESOURCE_MANAGER_NAME + UUID.randomUUID(), + ResourceID.generate(), + highAvailabilityServices, + new HeartbeatServices(100000L, 1000000L), + slotManager, + ResourceManagerPartitionTrackerImpl::new, + jobLeaderIdService, + testingFatalErrorHandler, + UnregisteredMetricGroups.createUnregisteredResourceManagerMetricGroup()); + + resourceManager.start(); + + // first make the ResourceManager the leader + resourceManagerLeaderElectionService.isLeader(ResourceManagerId.generate().toUUID()).get(); + + this.resourceManager = resourceManager; + + return resourceManager.getSelfGateway(ResourceManagerGateway.class); + } + + private static TaskExecutorHeartbeatPayload createTaskExecutorHeartbeatPayload(IntermediateDataSetID dataSetId, int numTotalPartitions, ResultPartitionID... partitionIds) { + return new TaskExecutorHeartbeatPayload( + new SlotReport(), + new ClusterPartitionReport(Collections.singletonList( + new ClusterPartitionReport.ClusterPartitionReportEntry(dataSetId, new HashSet<>(Arrays.asList(partitionIds)), numTotalPartitions) + ))); + } + + @FunctionalInterface + private interface TaskExecutorSetup { + void accept(TestingTaskExecutorGatewayBuilder taskExecutorGatewayBuilder) throws Exception; + } + + @FunctionalInterface + private interface TestAction { + void accept(ResourceManagerGateway resourceManagerGateway, ResourceID taskExecutorId1, ResourceID taskExecutorId2) throws Exception; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java index 3a1632cac2..b80f8c675d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java @@ -28,6 +28,7 @@ import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.instance.HardwareDescription; import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.runtime.io.network.partition.NoOpResourceManagerPartitionTracker; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; @@ -158,11 +159,13 @@ public class ResourceManagerTaskExecutorTest extends TestLogger { highAvailabilityServices, heartbeatServices, slotManager, + NoOpResourceManagerPartitionTracker::get, jobLeaderIdService, new ClusterInformation("localhost", 1234), fatalErrorHandler, UnregisteredMetricGroups.createUnregisteredResourceManagerMetricGroup(), - Time.minutes(5L)); + Time.minutes(5L), + RpcUtils.INF_TIMEOUT); resourceManager.start(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java index 0b76c23abd..29c22b5cae 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java @@ -25,6 +25,7 @@ import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.instance.HardwareDescription; +import org.apache.flink.runtime.io.network.partition.NoOpResourceManagerPartitionTracker; import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGateway; import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGatewayBuilder; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; @@ -261,6 +262,7 @@ public class ResourceManagerTest extends TestLogger { highAvailabilityServices, heartbeatServices, slotManager, + NoOpResourceManagerPartitionTracker::get, jobLeaderIdService, testingFatalErrorHandler, UnregisteredMetricGroups.createUnregisteredResourceManagerMetricGroup()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManagerTest.java index b433fa4bda..1974ede947 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManagerTest.java @@ -23,6 +23,7 @@ import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.entrypoint.ClusterInformation; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.io.network.partition.NoOpResourceManagerPartitionTracker; import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; @@ -30,6 +31,7 @@ import org.apache.flink.runtime.resourcemanager.slotmanager.TestingSlotManagerBu import org.apache.flink.runtime.resourcemanager.utils.MockResourceManagerRuntimeServices; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.runtime.rpc.TestingRpcServiceResource; import org.apache.flink.runtime.util.TestingFatalErrorHandler; import org.apache.flink.util.TestLogger; @@ -153,11 +155,13 @@ public class StandaloneResourceManagerTest extends TestLogger { highAvailabilityServices, heartbeatServices, slotManager, + NoOpResourceManagerPartitionTracker::get, jobLeaderIdService, clusterInformation, fatalErrorHandler, resourceManagerMetricGroup, - startupPeriodTime); + startupPeriodTime, + RpcUtils.INF_TIMEOUT); this.rmServices = rmServices; } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/TestingResourceManager.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/TestingResourceManager.java index 42831e4bf3..4f80e35f5b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/TestingResourceManager.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/TestingResourceManager.java @@ -24,11 +24,13 @@ import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.entrypoint.ClusterInformation; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.io.network.partition.ResourceManagerPartitionTrackerFactory; import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup; import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.rpc.RpcUtils; import javax.annotation.Nullable; @@ -47,6 +49,7 @@ public class TestingResourceManager extends ResourceManager { HighAvailabilityServices highAvailabilityServices, HeartbeatServices heartbeatServices, SlotManager slotManager, + ResourceManagerPartitionTrackerFactory clusterPartitionTrackerFactory, JobLeaderIdService jobLeaderIdService, FatalErrorHandler fatalErrorHandler, ResourceManagerMetricGroup resourceManagerMetricGroup) { @@ -57,10 +60,12 @@ public class TestingResourceManager extends ResourceManager { highAvailabilityServices, heartbeatServices, slotManager, + clusterPartitionTrackerFactory, jobLeaderIdService, new ClusterInformation("localhost", 1234), fatalErrorHandler, - resourceManagerMetricGroup); + resourceManagerMetricGroup, + RpcUtils.INF_TIMEOUT); } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGateway.java index a15994754e..b4d7781bdb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGateway.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGateway.java @@ -86,6 +86,8 @@ public class TestingTaskExecutorGateway implements TaskExecutorGateway { private final TriConsumer, Set> releaseOrPromotePartitionsConsumer; + private final Consumer> releaseClusterPartitionsConsumer; + private final TriFunction, CompletableFuture> operatorEventHandler; TestingTaskExecutorGateway( @@ -101,6 +103,7 @@ public class TestingTaskExecutorGateway implements TaskExecutorGateway { Function> cancelTaskFunction, Supplier> canBeReleasedSupplier, TriConsumer, Set> releaseOrPromotePartitionsConsumer, + Consumer> releaseClusterPartitionsConsumer, TriFunction, CompletableFuture> operatorEventHandler) { this.address = Preconditions.checkNotNull(address); @@ -115,6 +118,7 @@ public class TestingTaskExecutorGateway implements TaskExecutorGateway { this.cancelTaskFunction = cancelTaskFunction; this.canBeReleasedSupplier = canBeReleasedSupplier; this.releaseOrPromotePartitionsConsumer = releaseOrPromotePartitionsConsumer; + this.releaseClusterPartitionsConsumer = releaseClusterPartitionsConsumer; this.operatorEventHandler = operatorEventHandler; } @@ -144,7 +148,9 @@ public class TestingTaskExecutorGateway implements TaskExecutorGateway { } @Override - public void releaseClusterPartitions(Collection dataSetsToRelease, Time timeout) { + public CompletableFuture releaseClusterPartitions(Collection dataSetsToRelease, Time timeout) { + releaseClusterPartitionsConsumer.accept(dataSetsToRelease); + return CompletableFuture.completedFuture(Acknowledge.get()); } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGatewayBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGatewayBuilder.java index 7e1b0f1a8b..0fbbd89598 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGatewayBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGatewayBuilder.java @@ -27,6 +27,7 @@ import org.apache.flink.runtime.clusterframework.types.SlotID; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobmaster.AllocatedSlotReport; import org.apache.flink.runtime.jobmaster.JobMasterId; @@ -37,6 +38,7 @@ import org.apache.flink.util.SerializedValue; import org.apache.flink.util.function.TriConsumer; import org.apache.flink.util.function.TriFunction; +import java.util.Collection; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.function.BiConsumer; @@ -73,6 +75,7 @@ public class TestingTaskExecutorGatewayBuilder { private Function> cancelTaskFunction = NOOP_CANCEL_TASK_FUNCTION; private Supplier> canBeReleasedSupplier = () -> CompletableFuture.completedFuture(true); private TriConsumer, Set> releaseOrPromotePartitionsConsumer = NOOP_RELEASE_PARTITIONS_CONSUMER; + private Consumer> releaseClusterPartitionsConsumer = ignored -> {}; private TriFunction, CompletableFuture> operatorEventHandler = DEFAULT_OPERATOR_EVENT_HANDLER; public TestingTaskExecutorGatewayBuilder setAddress(String address) { @@ -135,6 +138,11 @@ public class TestingTaskExecutorGatewayBuilder { return this; } + public TestingTaskExecutorGatewayBuilder setReleaseClusterPartitionsConsumer(Consumer> releaseClusterPartitionsConsumer) { + this.releaseClusterPartitionsConsumer = releaseClusterPartitionsConsumer; + return this; + } + public TestingTaskExecutorGatewayBuilder setOperatorEventHandler(TriFunction, CompletableFuture> operatorEventHandler) { this.operatorEventHandler = operatorEventHandler; return this; @@ -154,6 +162,7 @@ public class TestingTaskExecutorGatewayBuilder { cancelTaskFunction, canBeReleasedSupplier, releaseOrPromotePartitionsConsumer, + releaseClusterPartitionsConsumer, operatorEventHandler); } } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java index 05f9a43b23..569eaa4ff5 100755 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java @@ -32,6 +32,7 @@ import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.entrypoint.ClusterInformation; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.io.network.partition.ResourceManagerPartitionTrackerFactory; import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup; import org.apache.flink.runtime.resourcemanager.ActiveResourceManager; import org.apache.flink.runtime.resourcemanager.JobLeaderIdService; @@ -128,6 +129,7 @@ public class YarnResourceManager extends ActiveResourceManager HighAvailabilityServices highAvailabilityServices, HeartbeatServices heartbeatServices, SlotManager slotManager, + ResourceManagerPartitionTrackerFactory clusterPartitionTrackerFactory, JobLeaderIdService jobLeaderIdService, ClusterInformation clusterInformation, FatalErrorHandler fatalErrorHandler, @@ -142,6 +144,7 @@ public class YarnResourceManager extends ActiveResourceManager highAvailabilityServices, heartbeatServices, slotManager, + clusterPartitionTrackerFactory, jobLeaderIdService, clusterInformation, fatalErrorHandler, diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnResourceManagerFactory.java b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnResourceManagerFactory.java index 318ca09793..81c6df3b20 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnResourceManagerFactory.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnResourceManagerFactory.java @@ -23,6 +23,7 @@ import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.entrypoint.ClusterInformation; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.io.network.partition.ResourceManagerPartitionTrackerImpl; import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup; import org.apache.flink.runtime.resourcemanager.ActiveResourceManagerFactory; import org.apache.flink.runtime.resourcemanager.ResourceManager; @@ -75,6 +76,7 @@ public class YarnResourceManagerFactory extends ActiveResourceManagerFactory Date: Thu, 2 Apr 2020 16:36:15 +0800 Subject: [PATCH 421/885] [FLINK-16916][serialization] Fix the logic of NullableSerializer#copy --- .../flink/api/java/typeutils/runtime/NullableSerializer.java | 2 +- .../api/java/typeutils/runtime/NullableSerializerTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NullableSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NullableSerializer.java index 0883daa052..8d26fe188b 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NullableSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NullableSerializer.java @@ -218,7 +218,7 @@ public class NullableSerializer extends TypeSerializer { @Override public void copy(DataInputView source, DataOutputView target) throws IOException { - boolean isNull = source.readBoolean(); + boolean isNull = deserializeNull(source); target.writeBoolean(isNull); if (isNull) { target.write(padding); diff --git a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/NullableSerializerTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/NullableSerializerTest.java index 3bd176a8fb..d7e14e2f4f 100644 --- a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/NullableSerializerTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/NullableSerializerTest.java @@ -71,7 +71,7 @@ public class NullableSerializerTest extends SerializerTestBase { @Override protected Integer[] getTestData() { - return new Integer[] { 5, -1, 0, null }; + return new Integer[] { 5, -1, null, 5 }; } @Test -- Gitee From f93346c93c6a841cdce576d48a0b5ca8076cc195 Mon Sep 17 00:00:00 2001 From: Qingsheng Ren Date: Tue, 17 Mar 2020 12:26:50 +0800 Subject: [PATCH 422/885] [FLINK-16125][connecotr/kafka] Remove Kafka connector property zookeeper.connect and clear documentation because Kafka 0.8 connector has been removed. --- docs/dev/table/connect.md | 9 --------- docs/dev/table/connect.zh.md | 9 --------- docs/dev/table/hive/hive_catalog.md | 2 -- docs/dev/table/hive/hive_catalog.zh.md | 2 -- docs/dev/table/sqlClient.md | 2 -- docs/dev/table/sqlClient.zh.md | 2 -- .../kafka/KafkaTestEnvironmentImpl.java | 1 - .../kafka/KafkaTestEnvironmentImpl.java | 3 +-- .../flink/table/descriptors/KafkaValidator.java | 8 ++------ .../connectors/kafka/KafkaConsumerTestBase.java | 1 - .../KafkaTableSourceSinkFactoryTestBase.java | 15 ++++----------- .../connectors/kafka/KafkaTableTestBase.java | 3 --- .../kafka/KafkaTestEnvironmentImpl.java | 1 - .../registry/test/TestAvroConsumerConfluent.java | 4 +--- .../tests/util/kafka/StreamingKafkaITCase.java | 1 - .../test/resources/kafka_json_source_schema.yaml | 3 --- .../kafka/test/base/KafkaExampleUtil.java | 4 ++-- .../flink/streaming/kafka/test/KafkaExample.java | 2 +- .../streaming/kafka/test/Kafka010Example.java | 2 +- .../streaming/kafka/test/Kafka011Example.java | 2 +- .../test-scripts/kafka_sql_common.sh | 1 - .../test_confluent_schema_registry.sh | 2 +- flink-python/pyflink/table/table_environment.py | 1 - .../pyflink/table/tests/test_descriptor.py | 4 +--- .../table/api/java/StreamTableEnvironment.java | 1 - .../apache/flink/table/api/TableEnvironment.java | 1 - .../table/api/scala/StreamTableEnvironment.scala | 1 - 27 files changed, 15 insertions(+), 72 deletions(-) diff --git a/docs/dev/table/connect.md b/docs/dev/table/connect.md index 204c4cb882..2bb80174d3 100644 --- a/docs/dev/table/connect.md +++ b/docs/dev/table/connect.md @@ -159,7 +159,6 @@ CREATE TABLE MyUserTable ( 'connector.version' = '0.10', 'connector.topic' = 'topic_name', 'connector.startup-mode' = 'earliest-offset', - 'connector.properties.zookeeper.connect' = 'localhost:2181', 'connector.properties.bootstrap.servers' = 'localhost:9092', -- declare a format for this system @@ -177,7 +176,6 @@ tableEnvironment .version("0.10") .topic("test-input") .startFromEarliest() - .property("zookeeper.connect", "localhost:2181") .property("bootstrap.servers", "localhost:9092") ) @@ -211,7 +209,6 @@ table_environment \ .version("0.10") .topic("test-input") .start_from_earliest() - .property("zookeeper.connect", "localhost:2181") .property("bootstrap.servers", "localhost:9092") ) \ .with_format( # declare a format for this system @@ -246,7 +243,6 @@ tables: topic: test-input startup-mode: earliest-offset properties: - zookeeper.connect: localhost:2181 bootstrap.servers: localhost:9092 # declare a format for this system @@ -773,8 +769,6 @@ CREATE TABLE MyUserTable ( 'connector.topic' = 'topic_name', -- required: topic name from which the table is read - -- required: specify the ZooKeeper connection string - 'connector.properties.zookeeper.connect' = 'localhost:2181', -- required: specify the Kafka server connection string 'connector.properties.bootstrap.servers' = 'localhost:9092', -- required for Kafka source, optional for Kafka sink, specify consumer group @@ -814,7 +808,6 @@ CREATE TABLE MyUserTable ( .topic("...") // required: topic name from which the table is read // optional: connector specific properties - .property("zookeeper.connect", "localhost:2181") .property("bootstrap.servers", "localhost:9092") .property("group.id", "testGroup") @@ -844,7 +837,6 @@ CREATE TABLE MyUserTable ( .topic("...") # required: topic name from which the table is read # optional: connector specific properties - .property("zookeeper.connect", "localhost:2181") .property("bootstrap.servers", "localhost:9092") .property("group.id", "testGroup") @@ -874,7 +866,6 @@ connector: topic: ... # required: topic name from which the table is read properties: - zookeeper.connect: localhost:2181 # required: specify the ZooKeeper connection string bootstrap.servers: localhost:9092 # required: specify the Kafka server connection string group.id: testGroup # optional: required in Kafka consumer, specify consumer group diff --git a/docs/dev/table/connect.zh.md b/docs/dev/table/connect.zh.md index 66b8d9a74f..720ab54d24 100644 --- a/docs/dev/table/connect.zh.md +++ b/docs/dev/table/connect.zh.md @@ -159,7 +159,6 @@ CREATE TABLE MyUserTable ( 'connector.version' = '0.10', 'connector.topic' = 'topic_name', 'connector.startup-mode' = 'earliest-offset', - 'connector.properties.zookeeper.connect' = 'localhost:2181', 'connector.properties.bootstrap.servers' = 'localhost:9092', -- declare a format for this system @@ -177,7 +176,6 @@ tableEnvironment .version("0.10") .topic("test-input") .startFromEarliest() - .property("zookeeper.connect", "localhost:2181") .property("bootstrap.servers", "localhost:9092") ) @@ -211,7 +209,6 @@ table_environment \ .version("0.10") .topic("test-input") .start_from_earliest() - .property("zookeeper.connect", "localhost:2181") .property("bootstrap.servers", "localhost:9092") ) \ .with_format( # declare a format for this system @@ -246,7 +243,6 @@ tables: topic: test-input startup-mode: earliest-offset properties: - zookeeper.connect: localhost:2181 bootstrap.servers: localhost:9092 # declare a format for this system @@ -773,8 +769,6 @@ CREATE TABLE MyUserTable ( 'connector.topic' = 'topic_name', -- required: topic name from which the table is read - -- required: specify the ZooKeeper connection string - 'connector.properties.zookeeper.connect' = 'localhost:2181', -- required: specify the Kafka server connection string 'connector.properties.bootstrap.servers' = 'localhost:9092', -- required for Kafka source, optional for Kafka sink, specify consumer group @@ -814,7 +808,6 @@ CREATE TABLE MyUserTable ( .topic("...") // required: topic name from which the table is read // optional: connector specific properties - .property("zookeeper.connect", "localhost:2181") .property("bootstrap.servers", "localhost:9092") .property("group.id", "testGroup") @@ -844,7 +837,6 @@ CREATE TABLE MyUserTable ( .topic("...") # required: topic name from which the table is read # optional: connector specific properties - .property("zookeeper.connect", "localhost:2181") .property("bootstrap.servers", "localhost:9092") .property("group.id", "testGroup") @@ -874,7 +866,6 @@ connector: topic: ... # required: topic name from which the table is read properties: - zookeeper.connect: localhost:2181 # required: specify the ZooKeeper connection string bootstrap.servers: localhost:9092 # required: specify the Kafka server connection string group.id: testGroup # optional: required in Kafka consumer, specify consumer group diff --git a/docs/dev/table/hive/hive_catalog.md b/docs/dev/table/hive/hive_catalog.md index d2e7d5a652..d907703fe2 100644 --- a/docs/dev/table/hive/hive_catalog.md +++ b/docs/dev/table/hive/hive_catalog.md @@ -190,7 +190,6 @@ Flink SQL> CREATE TABLE mykafka (name String, age Int) WITH ( 'connector.type' = 'kafka', 'connector.version' = 'universal', 'connector.topic' = 'test', - 'connector.properties.zookeeper.connect' = 'localhost:2181', 'connector.properties.bootstrap.servers' = 'localhost:9092', 'format.type' = 'csv', 'update-mode' = 'append' @@ -227,7 +226,6 @@ Location: ...... Table Type: MANAGED_TABLE Table Parameters: flink.connector.properties.bootstrap.servers localhost:9092 - flink.connector.properties.zookeeper.connect localhost:2181 flink.connector.topic test flink.connector.type kafka flink.connector.version universal diff --git a/docs/dev/table/hive/hive_catalog.zh.md b/docs/dev/table/hive/hive_catalog.zh.md index d2e7d5a652..d907703fe2 100644 --- a/docs/dev/table/hive/hive_catalog.zh.md +++ b/docs/dev/table/hive/hive_catalog.zh.md @@ -190,7 +190,6 @@ Flink SQL> CREATE TABLE mykafka (name String, age Int) WITH ( 'connector.type' = 'kafka', 'connector.version' = 'universal', 'connector.topic' = 'test', - 'connector.properties.zookeeper.connect' = 'localhost:2181', 'connector.properties.bootstrap.servers' = 'localhost:9092', 'format.type' = 'csv', 'update-mode' = 'append' @@ -227,7 +226,6 @@ Location: ...... Table Type: MANAGED_TABLE Table Parameters: flink.connector.properties.bootstrap.servers localhost:9092 - flink.connector.properties.zookeeper.connect localhost:2181 flink.connector.topic test flink.connector.type kafka flink.connector.version universal diff --git a/docs/dev/table/sqlClient.md b/docs/dev/table/sqlClient.md index ed4ff80f9f..9ea1bb897a 100644 --- a/docs/dev/table/sqlClient.md +++ b/docs/dev/table/sqlClient.md @@ -317,7 +317,6 @@ tables: topic: TaxiRides startup-mode: earliest-offset properties: - zookeeper.connect: localhost:2181 bootstrap.servers: localhost:9092 group.id: testGroup format: @@ -483,7 +482,6 @@ tables: version: "0.11" topic: OutputTopic properties: - zookeeper.connect: localhost:2181 bootstrap.servers: localhost:9092 group.id: testGroup format: diff --git a/docs/dev/table/sqlClient.zh.md b/docs/dev/table/sqlClient.zh.md index 93d506b8ac..642f44810e 100644 --- a/docs/dev/table/sqlClient.zh.md +++ b/docs/dev/table/sqlClient.zh.md @@ -317,7 +317,6 @@ tables: topic: TaxiRides startup-mode: earliest-offset properties: - zookeeper.connect: localhost:2181 bootstrap.servers: localhost:9092 group.id: testGroup format: @@ -483,7 +482,6 @@ tables: version: "0.11" topic: OutputTopic properties: - zookeeper.connect: localhost:2181 bootstrap.servers: localhost:9092 group.id: testGroup format: diff --git a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java index 64649eee6a..322c3aa284 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java @@ -253,7 +253,6 @@ public class KafkaTestEnvironmentImpl extends KafkaTestEnvironment { LOG.info("ZK and KafkaServer started."); standardProps = new Properties(); - standardProps.setProperty("zookeeper.connect", zookeeperConnectionString); standardProps.setProperty("bootstrap.servers", brokerConnectionString); standardProps.setProperty("group.id", "flink-tests"); standardProps.setProperty("enable.auto.commit", "false"); diff --git a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java index a3982badae..478ce38886 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java @@ -134,7 +134,6 @@ public class KafkaTestEnvironmentImpl extends KafkaTestEnvironment { LOG.info("ZK and KafkaServer started."); standardProps = new Properties(); - standardProps.setProperty("zookeeper.connect", zookeeperConnectionString); standardProps.setProperty("bootstrap.servers", brokerConnectionString); standardProps.setProperty("group.id", "flink-tests"); standardProps.setProperty("enable.auto.commit", "false"); @@ -393,8 +392,8 @@ public class KafkaTestEnvironmentImpl extends KafkaTestEnvironment { kafkaProperties.put("advertised.host.name", KAFKA_HOST); kafkaProperties.put("broker.id", Integer.toString(brokerId)); kafkaProperties.put("log.dir", tmpFolder.toString()); - kafkaProperties.put("zookeeper.connect", zookeeperConnectionString); kafkaProperties.put("message.max.bytes", String.valueOf(50 * 1024 * 1024)); + kafkaProperties.put("zookeeper.connect", zookeeperConnectionString); kafkaProperties.put("replica.fetch.max.bytes", String.valueOf(50 * 1024 * 1024)); kafkaProperties.put("transaction.max.timeout.ms", Integer.toString(1000 * 60 * 60 * 2)); // 2hours diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/table/descriptors/KafkaValidator.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/table/descriptors/KafkaValidator.java index f55bc1e923..158417a300 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/table/descriptors/KafkaValidator.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/table/descriptors/KafkaValidator.java @@ -54,7 +54,6 @@ public class KafkaValidator extends ConnectorDescriptorValidator { public static final String CONNECTOR_SPECIFIC_OFFSETS_OFFSET = "offset"; public static final String CONNECTOR_STARTUP_TIMESTAMP_MILLIS = "connector.startup-timestamp-millis"; public static final String CONNECTOR_PROPERTIES = "connector.properties"; - public static final String CONNECTOR_PROPERTIES_ZOOKEEPER_CONNECT = "connector.properties.zookeeper.connect"; public static final String CONNECTOR_PROPERTIES_BOOTSTRAP_SERVER = "connector.properties.bootstrap.servers"; public static final String CONNECTOR_PROPERTIES_GROUP_ID = "connector.properties.group.id"; public static final String CONNECTOR_PROPERTIES_KEY = "key"; @@ -136,11 +135,9 @@ public class KafkaValidator extends ConnectorDescriptorValidator { } private void validateKafkaProperties(DescriptorProperties properties) { - if (properties.containsKey(CONNECTOR_PROPERTIES_ZOOKEEPER_CONNECT) - || properties.containsKey(CONNECTOR_PROPERTIES_BOOTSTRAP_SERVER) + if (properties.containsKey(CONNECTOR_PROPERTIES_BOOTSTRAP_SERVER) || properties.containsKey(CONNECTOR_PROPERTIES_GROUP_ID)) { - properties.validateString(CONNECTOR_PROPERTIES_ZOOKEEPER_CONNECT, false); properties.validateString(CONNECTOR_PROPERTIES_BOOTSTRAP_SERVER, false); properties.validateString(CONNECTOR_PROPERTIES_GROUP_ID, true); @@ -235,8 +232,7 @@ public class KafkaValidator extends ConnectorDescriptorValidator { } public static boolean hasConciseKafkaProperties(DescriptorProperties descriptorProperties) { - return descriptorProperties.containsKey(CONNECTOR_PROPERTIES_ZOOKEEPER_CONNECT) || - descriptorProperties.containsKey(CONNECTOR_PROPERTIES_BOOTSTRAP_SERVER) || + return descriptorProperties.containsKey(CONNECTOR_PROPERTIES_BOOTSTRAP_SERVER) || descriptorProperties.containsKey(CONNECTOR_PROPERTIES_GROUP_ID); } } diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java index 2cf999ba86..b788fb88eb 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java @@ -169,7 +169,6 @@ public abstract class KafkaConsumerTestBase extends KafkaTestBaseWithFlink { // use wrong ports for the consumers properties.setProperty("bootstrap.servers", "localhost:80"); - properties.setProperty("zookeeper.connect", "localhost:80"); properties.setProperty("group.id", "test"); properties.setProperty("request.timeout.ms", "3000"); // let the test fail fast properties.setProperty("socket.timeout.ms", "3000"); diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceSinkFactoryTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceSinkFactoryTestBase.java index 0875c28d4e..d8eb011225 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceSinkFactoryTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceSinkFactoryTestBase.java @@ -93,7 +93,6 @@ public abstract class KafkaTableSourceSinkFactoryTestBase extends TestLogger { private static final Properties KAFKA_PROPERTIES = new Properties(); static { - KAFKA_PROPERTIES.setProperty("zookeeper.connect", "dummy"); KAFKA_PROPERTIES.setProperty("group.id", "dummy"); KAFKA_PROPERTIES.setProperty("bootstrap.servers", "dummy"); } @@ -224,7 +223,6 @@ public abstract class KafkaTableSourceSinkFactoryTestBase extends TestLogger { // use legacy properties legacyPropertiesMap.remove("connector.specific-offsets"); - legacyPropertiesMap.remove("connector.properties.zookeeper.connect"); legacyPropertiesMap.remove("connector.properties.bootstrap.servers"); legacyPropertiesMap.remove("connector.properties.group.id"); @@ -236,12 +234,10 @@ public abstract class KafkaTableSourceSinkFactoryTestBase extends TestLogger { legacyPropertiesMap.put("connector.specific-offsets.0.offset", "100"); legacyPropertiesMap.put("connector.specific-offsets.1.partition", "1"); legacyPropertiesMap.put("connector.specific-offsets.1.offset", "123"); - legacyPropertiesMap.put("connector.properties.0.key", "zookeeper.connect"); + legacyPropertiesMap.put("connector.properties.0.key", "bootstrap.servers"); legacyPropertiesMap.put("connector.properties.0.value", "dummy"); - legacyPropertiesMap.put("connector.properties.1.key", "bootstrap.servers"); + legacyPropertiesMap.put("connector.properties.1.key", "group.id"); legacyPropertiesMap.put("connector.properties.1.value", "dummy"); - legacyPropertiesMap.put("connector.properties.2.key", "group.id"); - legacyPropertiesMap.put("connector.properties.2.value", "dummy"); final TableSource actualSource = TableFactoryService.find(StreamTableSourceFactory.class, legacyPropertiesMap) .createStreamTableSource(legacyPropertiesMap); @@ -330,7 +326,6 @@ public abstract class KafkaTableSourceSinkFactoryTestBase extends TestLogger { // use legacy properties legacyPropertiesMap.remove("connector.specific-offsets"); - legacyPropertiesMap.remove("connector.properties.zookeeper.connect"); legacyPropertiesMap.remove("connector.properties.bootstrap.servers"); legacyPropertiesMap.remove("connector.properties.group.id"); @@ -342,12 +337,10 @@ public abstract class KafkaTableSourceSinkFactoryTestBase extends TestLogger { legacyPropertiesMap.put("connector.specific-offsets.0.offset", "100"); legacyPropertiesMap.put("connector.specific-offsets.1.partition", "1"); legacyPropertiesMap.put("connector.specific-offsets.1.offset", "123"); - legacyPropertiesMap.put("connector.properties.0.key", "zookeeper.connect"); + legacyPropertiesMap.put("connector.properties.0.key", "bootstrap.servers"); legacyPropertiesMap.put("connector.properties.0.value", "dummy"); - legacyPropertiesMap.put("connector.properties.1.key", "bootstrap.servers"); + legacyPropertiesMap.put("connector.properties.1.key", "group.id"); legacyPropertiesMap.put("connector.properties.1.value", "dummy"); - legacyPropertiesMap.put("connector.properties.2.key", "group.id"); - legacyPropertiesMap.put("connector.properties.2.value", "dummy"); final TableSink actualSink = TableFactoryService.find(StreamTableSinkFactory.class, legacyPropertiesMap) .createStreamTableSink(legacyPropertiesMap); diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableTestBase.java index bd526e9f80..a3d0acd912 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableTestBase.java @@ -66,7 +66,6 @@ public abstract class KafkaTableTestBase extends KafkaTestBase { // ---------- Produce an event time stream into Kafka ------------------- String groupId = standardProps.getProperty("group.id"); - String zk = standardProps.getProperty("zookeeper.connect"); String bootstraps = standardProps.getProperty("bootstrap.servers"); // TODO: use DDL to register Kafka once FLINK-15282 is fixed. @@ -83,7 +82,6 @@ public abstract class KafkaTableTestBase extends KafkaTestBase { properties.put("connector.type", "kafka"); properties.put("connector.topic", topic); properties.put("connector.version", kafkaVersion()); - properties.put("connector.properties.zookeeper.connect", zk); properties.put("connector.properties.bootstrap.servers", bootstraps); properties.put("connector.properties.group.id", groupId); properties.put("connector.startup-mode", "earliest-offset"); @@ -112,7 +110,6 @@ public abstract class KafkaTableTestBase extends KafkaTestBase { // " 'connector.type' = 'kafka',\n" + // " 'connector.topic' = '" + topic + "',\n" + // " 'connector.version' = 'universal',\n" + -// " 'connector.properties.zookeeper.connect' = '" + zk + "',\n" + // " 'connector.properties.bootstrap.servers' = '" + bootstraps + "',\n" + // " 'connector.properties.group.id' = '" + groupId + "', \n" + // " 'connector.startup-mode' = 'earliest-offset', \n" + diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java index 15b159457b..16cb724011 100644 --- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java +++ b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java @@ -137,7 +137,6 @@ public class KafkaTestEnvironmentImpl extends KafkaTestEnvironment { LOG.info("ZK and KafkaServer started."); standardProps = new Properties(); - standardProps.setProperty("zookeeper.connect", zookeeperConnectionString); standardProps.setProperty("bootstrap.servers", brokerConnectionString); standardProps.setProperty("group.id", "flink-tests"); standardProps.setProperty("enable.auto.commit", "false"); diff --git a/flink-end-to-end-tests/flink-confluent-schema-registry/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java b/flink-end-to-end-tests/flink-confluent-schema-registry/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java index 55549dee97..dda4617973 100644 --- a/flink-end-to-end-tests/flink-confluent-schema-registry/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java +++ b/flink-end-to-end-tests/flink-confluent-schema-registry/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java @@ -37,7 +37,7 @@ import java.util.Properties; * A simple example that shows how to read from and write to Kafka with Confluent Schema Registry. * This will read AVRO messages from the input topic, parse them into a POJO type via checking the Schema by calling Schema registry. * Then this example publish the POJO type to kafka by converting the POJO to AVRO and verifying the schema. - * --input-topic test-input --output-string-topic test-output --output-avro-topic test-avro-output --output-subject --bootstrap.servers localhost:9092 --zookeeper.connect localhost:2181 --schema-registry-url http://localhost:8081 --group.id myconsumer + * --input-topic test-input --output-string-topic test-output --output-avro-topic test-avro-output --output-subject --bootstrap.servers localhost:9092 --schema-registry-url http://localhost:8081 --group.id myconsumer */ public class TestAvroConsumerConfluent { @@ -49,14 +49,12 @@ public class TestAvroConsumerConfluent { System.out.println("Missing parameters!\n" + "Usage: Kafka --input-topic --output-string-topic --output-avro-topic " + "--bootstrap.servers " + - "--zookeeper.connect " + "--schema-registry-url --group.id "); return; } Properties config = new Properties(); config.setProperty("bootstrap.servers", parameterTool.getRequired("bootstrap.servers")); config.setProperty("group.id", parameterTool.getRequired("group.id")); - config.setProperty("zookeeper.connect", parameterTool.getRequired("zookeeper.connect")); String schemaRegistryUrl = parameterTool.getRequired("schema-registry-url"); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/StreamingKafkaITCase.java b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/StreamingKafkaITCase.java index a7ffb14f09..22d8506f58 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/StreamingKafkaITCase.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/StreamingKafkaITCase.java @@ -101,7 +101,6 @@ public class StreamingKafkaITCase extends TestLogger { .addArgument("--output-topic", outputTopic) .addArgument("--prefix", "PREFIX") .addArgument("--bootstrap.servers", kafka.getBootstrapServerAddresses().stream().map(address -> address.getHostString() + ':' + address.getPort()).collect(Collectors.joining(","))) - .addArgument("--zookeeper.connect ", kafka.getZookeeperAddress().getHostString() + ':' + kafka.getZookeeperAddress().getPort()) .addArgument("--group.id", "myconsumer") .addArgument("--auto.offset.reset", "earliest") .addArgument("--transaction.timeout.ms", "900000") diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/resources/kafka_json_source_schema.yaml b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/resources/kafka_json_source_schema.yaml index 6de0c71319..600e3f1c4a 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/resources/kafka_json_source_schema.yaml +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/resources/kafka_json_source_schema.yaml @@ -40,7 +40,6 @@ tables: topic: $TOPIC_NAME startup-mode: earliest-offset properties: - zookeeper.connect: $KAFKA_ZOOKEEPER_ADDRESS bootstrap.servers: $KAFKA_BOOTSTRAP_SERVERS format: type: json @@ -86,8 +85,6 @@ tables: topic: test-avro startup-mode: earliest-offset properties: - - key: zookeeper.connect - value: $KAFKA_ZOOKEEPER_ADDRESS - key: bootstrap.servers value: $KAFKA_BOOTSTRAP_SERVERS format: diff --git a/flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/KafkaExampleUtil.java b/flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/KafkaExampleUtil.java index 0e3c4eafca..5505a7ad49 100644 --- a/flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/KafkaExampleUtil.java +++ b/flink-end-to-end-tests/flink-streaming-kafka-test-base/src/main/java/org/apache/flink/streaming/kafka/test/base/KafkaExampleUtil.java @@ -34,11 +34,11 @@ public class KafkaExampleUtil { System.out.println("Missing parameters!\n" + "Usage: Kafka --input-topic --output-topic " + "--bootstrap.servers " + - "--zookeeper.connect --group.id "); + "--group.id "); throw new Exception("Missing parameters!\n" + "Usage: Kafka --input-topic --output-topic " + "--bootstrap.servers " + - "--zookeeper.connect --group.id "); + "--group.id "); } StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); diff --git a/flink-end-to-end-tests/flink-streaming-kafka-test/src/main/java/org/apache/flink/streaming/kafka/test/KafkaExample.java b/flink-end-to-end-tests/flink-streaming-kafka-test/src/main/java/org/apache/flink/streaming/kafka/test/KafkaExample.java index 3a3be93208..f3c844c6e3 100644 --- a/flink-end-to-end-tests/flink-streaming-kafka-test/src/main/java/org/apache/flink/streaming/kafka/test/KafkaExample.java +++ b/flink-end-to-end-tests/flink-streaming-kafka-test/src/main/java/org/apache/flink/streaming/kafka/test/KafkaExample.java @@ -40,7 +40,7 @@ import org.apache.flink.streaming.kafka.test.base.RollingAdditionMapper; * *

    Example usage: * --input-topic test-input --output-topic test-output --bootstrap.servers localhost:9092 - * --zookeeper.connect localhost:2181 --group.id myconsumer + * --group.id myconsumer */ public class KafkaExample extends KafkaExampleUtil { diff --git a/flink-end-to-end-tests/flink-streaming-kafka010-test/src/main/java/org/apache/flink/streaming/kafka/test/Kafka010Example.java b/flink-end-to-end-tests/flink-streaming-kafka010-test/src/main/java/org/apache/flink/streaming/kafka/test/Kafka010Example.java index 0b97179425..14c9493a21 100644 --- a/flink-end-to-end-tests/flink-streaming-kafka010-test/src/main/java/org/apache/flink/streaming/kafka/test/Kafka010Example.java +++ b/flink-end-to-end-tests/flink-streaming-kafka010-test/src/main/java/org/apache/flink/streaming/kafka/test/Kafka010Example.java @@ -38,7 +38,7 @@ import org.apache.flink.streaming.kafka.test.base.RollingAdditionMapper; * the String messages are of formatted as a (word,frequency,timestamp) tuple. * *

    Example usage: - * --input-topic test-input --output-topic test-output --bootstrap.servers localhost:9092 --zookeeper.connect localhost:2181 --group.id myconsumer + * --input-topic test-input --output-topic test-output --bootstrap.servers localhost:9092 localhost:2181 --group.id myconsumer */ public class Kafka010Example { diff --git a/flink-end-to-end-tests/flink-streaming-kafka011-test/src/main/java/org/apache/flink/streaming/kafka/test/Kafka011Example.java b/flink-end-to-end-tests/flink-streaming-kafka011-test/src/main/java/org/apache/flink/streaming/kafka/test/Kafka011Example.java index 1f877c5a5b..fafd3076c6 100644 --- a/flink-end-to-end-tests/flink-streaming-kafka011-test/src/main/java/org/apache/flink/streaming/kafka/test/Kafka011Example.java +++ b/flink-end-to-end-tests/flink-streaming-kafka011-test/src/main/java/org/apache/flink/streaming/kafka/test/Kafka011Example.java @@ -38,7 +38,7 @@ import org.apache.flink.streaming.kafka.test.base.RollingAdditionMapper; * the String messages are of formatted as a (word,frequency,timestamp) tuple. * *

    Example usage: - * --input-topic test-input --output-topic test-output --bootstrap.servers localhost:9092 --zookeeper.connect localhost:2181 --group.id myconsumer + * --input-topic test-input --output-topic test-output --bootstrap.servers localhost:9092 --group.id myconsumer */ public class Kafka011Example { diff --git a/flink-end-to-end-tests/test-scripts/kafka_sql_common.sh b/flink-end-to-end-tests/test-scripts/kafka_sql_common.sh index 560e43e078..c7ed12ae4e 100644 --- a/flink-end-to-end-tests/test-scripts/kafka_sql_common.sh +++ b/flink-end-to-end-tests/test-scripts/kafka_sql_common.sh @@ -68,7 +68,6 @@ function get_kafka_json_source_schema { topic: $topicName startup-mode: earliest-offset properties: - zookeeper.connect: localhost:2181 bootstrap.servers: localhost:9092 format: type: json diff --git a/flink-end-to-end-tests/test-scripts/test_confluent_schema_registry.sh b/flink-end-to-end-tests/test-scripts/test_confluent_schema_registry.sh index 5d3e9e4970..a023f3989d 100755 --- a/flink-end-to-end-tests/test-scripts/test_confluent_schema_registry.sh +++ b/flink-end-to-end-tests/test-scripts/test_confluent_schema_registry.sh @@ -78,7 +78,7 @@ create_kafka_topic 1 1 test-avro-out # Read Avro message from [test-avro-input], check the schema and send message to [test-string-ou] $FLINK_DIR/bin/flink run -d $TEST_PROGRAM_JAR \ --input-topic test-avro-input --output-string-topic test-string-out --output-avro-topic test-avro-out --output-subject test-output-subject \ - --bootstrap.servers localhost:9092 --zookeeper.connect localhost:2181 --group.id myconsumer --auto.offset.reset earliest \ + --bootstrap.servers localhost:9092 --group.id myconsumer --auto.offset.reset earliest \ --schema-registry-url ${SCHEMA_REGISTRY_URL} #echo "Reading messages from Kafka topic [test-string-ou] ..." diff --git a/flink-python/pyflink/table/table_environment.py b/flink-python/pyflink/table/table_environment.py index 46aa70f4ea..7acdaa03f9 100644 --- a/flink-python/pyflink/table/table_environment.py +++ b/flink-python/pyflink/table/table_environment.py @@ -522,7 +522,6 @@ class TableEnvironment(object): ... 'connector.type' = 'kafka', ... 'update-mode' = 'append', ... 'connector.topic' = 'xxx', - ... 'connector.properties.zookeeper.connect' = 'localhost:2181', ... 'connector.properties.bootstrap.servers' = 'localhost:9092' ... ) ... ''' diff --git a/flink-python/pyflink/table/tests/test_descriptor.py b/flink-python/pyflink/table/tests/test_descriptor.py index 002f499866..6894e10c62 100644 --- a/flink-python/pyflink/table/tests/test_descriptor.py +++ b/flink-python/pyflink/table/tests/test_descriptor.py @@ -61,12 +61,10 @@ class KafkaDescriptorTests(PyFlinkTestCase): self.assertEqual(expected, properties) def test_properties(self): - kafka = Kafka().properties({"zookeeper.connect": "localhost:2181", - "bootstrap.servers": "localhost:9092"}) + kafka = Kafka().properties({"bootstrap.servers": "localhost:9092"}) properties = kafka.to_properties() expected = {'connector.type': 'kafka', - 'connector.properties.zookeeper.connect': 'localhost:2181', 'connector.properties.bootstrap.servers': 'localhost:9092', 'connector.property-version': '1'} self.assertEqual(expected, properties) diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/StreamTableEnvironment.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/StreamTableEnvironment.java index 364e619148..74f0b654f2 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/StreamTableEnvironment.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/StreamTableEnvironment.java @@ -701,7 +701,6 @@ public interface StreamTableEnvironment extends TableEnvironment { * new Kafka() * .version("0.11") * .topic("clicks") - * .property("zookeeper.connect", "localhost") * .property("group.id", "click-group") * .startFromEarliest()) * .withFormat( diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java index f086dd8f88..c89d76aa6c 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java @@ -689,7 +689,6 @@ public interface TableEnvironment { * 'connector.type' = 'kafka', * 'update-mode' = 'append', * 'connector.topic' = 'xxx', - * 'connector.properties.zookeeper.connect' = 'localhost:2181', * 'connector.properties.bootstrap.servers' = 'localhost:9092', * ... * )"; diff --git a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala index 54d0c97e80..d569c8308a 100644 --- a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala +++ b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala @@ -458,7 +458,6 @@ trait StreamTableEnvironment extends TableEnvironment { * new Kafka() * .version("0.11") * .topic("clicks") - * .property("zookeeper.connect", "localhost") * .property("group.id", "click-group") * .startFromEarliest()) * .withFormat( -- Gitee From 2b94ca60ca437df98d0efeb7b9a43ecdf216825b Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Fri, 3 Apr 2020 11:33:30 +0800 Subject: [PATCH 423/885] [FLINK-16885][hive] Remove wilcard excludes that don't work on maven 3.1.X This closes #11620 --- .../flink-sql-connector-hive-1.2.2/pom.xml | 46 +++++++++---------- .../flink-sql-connector-hive-2.2.0/pom.xml | 32 ++++++------- .../flink-sql-connector-hive-2.3.6/pom.xml | 18 ++++---- .../flink-sql-connector-hive-3.1.2/pom.xml | 20 +++----- 4 files changed, 54 insertions(+), 62 deletions(-) diff --git a/flink-connectors/flink-sql-connector-hive-1.2.2/pom.xml b/flink-connectors/flink-sql-connector-hive-1.2.2/pom.xml index 66dec680b8..d8a2c4230a 100644 --- a/flink-connectors/flink-sql-connector-hive-1.2.2/pom.xml +++ b/flink-connectors/flink-sql-connector-hive-1.2.2/pom.xml @@ -40,12 +40,6 @@ under the License. org.apache.flink flink-connector-hive_${scala.binary.version} ${project.version} - - - * - * - - @@ -54,8 +48,16 @@ under the License. 1.2.2 - * - * + log4j + log4j + + + org.slf4j + slf4j-log4j12 + + + org.pentaho + pentaho-aggdesigner-algorithm @@ -66,8 +68,12 @@ under the License. 1.2.2 - * - * + log4j + log4j + + + org.slf4j + slf4j-log4j12 @@ -76,12 +82,6 @@ under the License. org.apache.thrift libfb303 0.9.2 - - - * - * - - @@ -91,8 +91,12 @@ under the License. nohive - * - * + log4j + log4j + + + org.slf4j + slf4j-log4j12 @@ -101,12 +105,6 @@ under the License. io.airlift aircompressor 0.8 - - - * - * - - diff --git a/flink-connectors/flink-sql-connector-hive-2.2.0/pom.xml b/flink-connectors/flink-sql-connector-hive-2.2.0/pom.xml index d3ef7c57b6..856b38825d 100644 --- a/flink-connectors/flink-sql-connector-hive-2.2.0/pom.xml +++ b/flink-connectors/flink-sql-connector-hive-2.2.0/pom.xml @@ -40,12 +40,6 @@ under the License. org.apache.flink flink-connector-hive_${scala.binary.version} ${project.version} - - - * - * - - @@ -54,8 +48,16 @@ under the License. 2.2.0 - * - * + log4j + log4j + + + org.slf4j + slf4j-log4j12 + + + org.pentaho + pentaho-aggdesigner-algorithm @@ -66,8 +68,12 @@ under the License. 1.4.3 - * - * + log4j + log4j + + + org.slf4j + slf4j-log4j12 @@ -76,12 +82,6 @@ under the License. io.airlift aircompressor 0.8 - - - * - * - - diff --git a/flink-connectors/flink-sql-connector-hive-2.3.6/pom.xml b/flink-connectors/flink-sql-connector-hive-2.3.6/pom.xml index 81c562ebb8..74586f2e97 100644 --- a/flink-connectors/flink-sql-connector-hive-2.3.6/pom.xml +++ b/flink-connectors/flink-sql-connector-hive-2.3.6/pom.xml @@ -40,12 +40,6 @@ under the License. org.apache.flink flink-connector-hive_${scala.binary.version} ${project.version} - - - * - * - - @@ -54,8 +48,16 @@ under the License. 2.3.6 - * - * + log4j + log4j + + + org.slf4j + slf4j-log4j12 + + + org.pentaho + pentaho-aggdesigner-algorithm diff --git a/flink-connectors/flink-sql-connector-hive-3.1.2/pom.xml b/flink-connectors/flink-sql-connector-hive-3.1.2/pom.xml index 8225a4a5cb..857775e17d 100644 --- a/flink-connectors/flink-sql-connector-hive-3.1.2/pom.xml +++ b/flink-connectors/flink-sql-connector-hive-3.1.2/pom.xml @@ -40,12 +40,6 @@ under the License. org.apache.flink flink-connector-hive_${scala.binary.version} ${project.version} - - - * - * - - @@ -54,8 +48,12 @@ under the License. 3.1.2 - * - * + log4j + log4j + + + org.slf4j + slf4j-log4j12 @@ -64,12 +62,6 @@ under the License. org.apache.thrift libfb303 0.9.3 - - - * - * - - -- Gitee From b93c75b31e8ce17f57c8311f591a70316180df04 Mon Sep 17 00:00:00 2001 From: godfreyhe Date: Tue, 10 Mar 2020 10:24:27 +0800 Subject: [PATCH 424/885] [FLINK-16476] [table-planner-blink] Remove PowerMockito to avoid LinkageError in SelectivityEstimatorTest --- .../metadata/SelectivityEstimatorTest.scala | 62 ++++++++----------- 1 file changed, 26 insertions(+), 36 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimatorTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimatorTest.scala index 031a668793..6b682b1886 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimatorTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimatorTest.scala @@ -22,14 +22,17 @@ import org.apache.flink.table.api.TableConfig import org.apache.flink.table.catalog.FunctionCatalog import org.apache.flink.table.module.ModuleManager import org.apache.flink.table.plan.stats.{ColumnStats, TableStats} -import org.apache.flink.table.planner.calcite.{FlinkContext, FlinkContextImpl, FlinkTypeFactory, FlinkTypeSystem} -import org.apache.flink.table.planner.plan.schema._ +import org.apache.flink.table.planner.calcite.{FlinkTypeFactory, FlinkTypeSystem} +import org.apache.flink.table.planner.delegation.PlannerContext +import org.apache.flink.table.planner.plan.`trait`.FlinkRelDistributionTraitDef import org.apache.flink.table.planner.plan.stats.FlinkStatistic import org.apache.flink.table.planner.{JDouble, JLong} import org.apache.flink.table.utils.CatalogManagerMocks import org.apache.flink.util.Preconditions -import org.apache.calcite.plan.{AbstractRelOptPlanner, RelOptCluster} +import org.apache.calcite.jdbc.CalciteSchema +import org.apache.calcite.plan.ConventionTraitDef +import org.apache.calcite.rel.RelCollationTraitDef import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.TableScan import org.apache.calcite.rel.metadata.{JaninoRelMetadataProvider, RelMetadataQueryBase} @@ -40,24 +43,17 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable import org.apache.calcite.sql.fun.SqlStdOperatorTable._ import org.apache.calcite.util.{DateString, TimeString, TimestampString} import org.junit.Assert._ -import org.junit.runner.RunWith import org.junit.{Before, BeforeClass, Test} -import org.powermock.api.mockito.PowerMockito._ -import org.powermock.core.classloader.annotations.PrepareForTest -import org.powermock.modules.junit4.PowerMockRunner import java.math.BigDecimal import java.sql.{Date, Time, Timestamp} +import java.util import scala.collection.JavaConverters._ /** * Tests for [[SelectivityEstimator]]. - * - * We use PowerMockito instead of Mockito here, because [[TableScan#getRowType]] is a final method. */ -@RunWith(classOf[PowerMockRunner]) -@PrepareForTest(Array(classOf[TableScan])) class SelectivityEstimatorTest { private val allFieldNames = Seq("name", "amount", "price", "flag", "partition", "date_col", "time_col", "timestamp_col") @@ -82,33 +78,27 @@ class SelectivityEstimatorTest { private def mockScan( statistic: FlinkStatistic = FlinkStatistic.UNKNOWN, - isFilterPushedDown: Boolean = false, tableConfig: TableConfig = TableConfig.getDefault): TableScan = { - val tableScan = mock(classOf[TableScan]) - val cluster = mock(classOf[RelOptCluster]) - val planner = mock(classOf[AbstractRelOptPlanner]) val catalogManager = CatalogManagerMocks.createEmptyCatalogManager() - val moduleManager = mock(classOf[ModuleManager]) - val functionCatalog = new FunctionCatalog(tableConfig, catalogManager, moduleManager) - val context: FlinkContext = new FlinkContextImpl( - tableConfig, functionCatalog, catalogManager, null) - when(tableScan, "getCluster").thenReturn(cluster) - when(cluster, "getRexBuilder").thenReturn(rexBuilder) - when(cluster, "getPlanner").thenReturn(planner) - when(planner, "getContext").thenReturn(context) - when(tableScan, "getRowType").thenReturn(relDataType) - val sourceTable = mock(classOf[TableSourceTable[_]]) - when(sourceTable, "unwrap", classOf[TableSourceTable[_]]).thenReturn(sourceTable) - when(sourceTable, "getStatistic").thenReturn(statistic) - when(sourceTable, "getRowType").thenReturn(relDataType) - when(tableScan, "getTable").thenReturn(sourceTable) - val rowCount: JDouble = if (statistic != null && statistic.getRowCount != null) { - statistic.getRowCount - } else { - 100D - } - when(tableScan, "estimateRowCount", mq).thenReturn(rowCount) - tableScan + val rootSchema = CalciteSchema.createRootSchema(true, false).plus() + val table = new MockMetaTable(relDataType, statistic) + rootSchema.add("test", table) + val plannerContext: PlannerContext = + new PlannerContext( + tableConfig, + new FunctionCatalog(tableConfig, catalogManager, new ModuleManager), + catalogManager, + CalciteSchema.from(rootSchema), + util.Arrays.asList( + ConventionTraitDef.INSTANCE, + FlinkRelDistributionTraitDef.INSTANCE, + RelCollationTraitDef.INSTANCE + ) + ) + + val relBuilder = plannerContext.createRelBuilder("default_catalog", "default_database") + relBuilder.clear() + relBuilder.scan(util.Arrays.asList("test")).build().asInstanceOf[TableScan] } private def createNumericLiteral(num: Long): RexLiteral = { -- Gitee From 00526eba0f37f8869e62f41f43a40906e4169790 Mon Sep 17 00:00:00 2001 From: godfreyhe Date: Tue, 10 Mar 2020 11:22:28 +0800 Subject: [PATCH 425/885] [FLINK-16476] [table-planner-blink] Remove PowerMockito to avoid LinkageError in AggCallSelectivityEstimatorTest --- .../AggCallSelectivityEstimatorTest.scala | 63 ++++++++----------- 1 file changed, 27 insertions(+), 36 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/AggCallSelectivityEstimatorTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/AggCallSelectivityEstimatorTest.scala index e4ddf880c3..6f7776475d 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/AggCallSelectivityEstimatorTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/AggCallSelectivityEstimatorTest.scala @@ -22,15 +22,18 @@ import org.apache.flink.table.api.TableConfig import org.apache.flink.table.catalog.FunctionCatalog import org.apache.flink.table.module.ModuleManager import org.apache.flink.table.plan.stats.{ColumnStats, TableStats} -import org.apache.flink.table.planner.calcite.{FlinkContextImpl, FlinkTypeFactory, FlinkTypeSystem} -import org.apache.flink.table.planner.plan.schema._ +import org.apache.flink.table.planner.calcite.{FlinkTypeFactory, FlinkTypeSystem} +import org.apache.flink.table.planner.delegation.PlannerContext +import org.apache.flink.table.planner.plan.`trait`.FlinkRelDistributionTraitDef import org.apache.flink.table.planner.plan.stats.FlinkStatistic import org.apache.flink.table.planner.{JDouble, JLong} import org.apache.flink.table.utils.CatalogManagerMocks import org.apache.flink.util.Preconditions import com.google.common.collect.ImmutableList -import org.apache.calcite.plan.{AbstractRelOptPlanner, RelOptCluster} +import org.apache.calcite.jdbc.CalciteSchema +import org.apache.calcite.plan.ConventionTraitDef +import org.apache.calcite.rel.RelCollationTraitDef import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.{Aggregate, AggregateCall, TableScan} import org.apache.calcite.rel.logical.LogicalAggregate @@ -43,23 +46,16 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable._ import org.apache.calcite.sql.{SqlAggFunction, SqlOperator} import org.apache.calcite.util.ImmutableBitSet import org.junit.Assert._ -import org.junit.runner.RunWith import org.junit.{Before, BeforeClass, Test} -import org.powermock.api.mockito.PowerMockito._ -import org.powermock.core.classloader.annotations.PrepareForTest -import org.powermock.modules.junit4.PowerMockRunner import java.math.BigDecimal +import java.util import scala.collection.JavaConversions._ /** * Tests for [[AggCallSelectivityEstimator]]. - * - * We use PowerMockito instead of Mockito here, because [[TableScan#getRowType]] is a final method. */ -@RunWith(classOf[PowerMockRunner]) -@PrepareForTest(Array(classOf[TableScan])) class AggCallSelectivityEstimatorTest { private val allFieldNames = Seq("name", "amount", "price") private val allFieldTypes = Seq(VARCHAR, INTEGER, DOUBLE) @@ -81,32 +77,27 @@ class AggCallSelectivityEstimatorTest { private def mockScan( statistic: FlinkStatistic = FlinkStatistic.UNKNOWN): TableScan = { - val tableScan = mock(classOf[TableScan]) - val cluster = mock(classOf[RelOptCluster]) - val planner = mock(classOf[AbstractRelOptPlanner]) + val tableConfig = new TableConfig val catalogManager = CatalogManagerMocks.createEmptyCatalogManager() - val moduleManager = mock(classOf[ModuleManager]) - val config = new TableConfig - val functionCatalog = new FunctionCatalog(config, catalogManager, moduleManager) - val context = new FlinkContextImpl(new TableConfig, functionCatalog, catalogManager, null) - when(tableScan, "getCluster").thenReturn(cluster) - when(cluster, "getRexBuilder").thenReturn(rexBuilder) - when(cluster, "getTypeFactory").thenReturn(typeFactory) - when(cluster, "getPlanner").thenReturn(planner) - when(planner, "getContext").thenReturn(context) - when(tableScan, "getRowType").thenReturn(relDataType) - val sourceTable = mock(classOf[TableSourceTable[_]]) - when(sourceTable, "unwrap", classOf[TableSourceTable[_]]).thenReturn(sourceTable) - when(sourceTable, "getStatistic").thenReturn(statistic) - when(sourceTable, "getRowType").thenReturn(relDataType) - when(tableScan, "getTable").thenReturn(sourceTable) - val rowCount: JDouble = if (statistic != null && statistic.getRowCount != null) { - statistic.getRowCount - } else { - 100D - } - when(tableScan, "estimateRowCount", mq).thenReturn(rowCount) - tableScan + val rootSchema = CalciteSchema.createRootSchema(true, false).plus() + val table = new MockMetaTable(relDataType, statistic) + rootSchema.add("test", table) + val plannerContext: PlannerContext = + new PlannerContext( + tableConfig, + new FunctionCatalog(tableConfig, catalogManager, new ModuleManager), + catalogManager, + CalciteSchema.from(rootSchema), + util.Arrays.asList( + ConventionTraitDef.INSTANCE, + FlinkRelDistributionTraitDef.INSTANCE, + RelCollationTraitDef.INSTANCE + ) + ) + + val relBuilder = plannerContext.createRelBuilder("default_catalog", "default_database") + relBuilder.clear() + relBuilder.scan(util.Arrays.asList("test")).build().asInstanceOf[TableScan] } private def createAggregate( -- Gitee From 70a320b010910bf01ec342afe8371dbbf04c7ec0 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Fri, 3 Apr 2020 08:25:17 +0200 Subject: [PATCH 426/885] [FLINK-16389][kafka] Bump kafka version to 0.10.2.2 --- flink-connectors/flink-connector-kafka-0.10/pom.xml | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-0.10/pom.xml b/flink-connectors/flink-connector-kafka-0.10/pom.xml index 6ad4dfa293..693df52af3 100644 --- a/flink-connectors/flink-connector-kafka-0.10/pom.xml +++ b/flink-connectors/flink-connector-kafka-0.10/pom.xml @@ -37,7 +37,7 @@ under the License. - 0.10.2.1 + 0.10.2.2 diff --git a/flink-connectors/flink-sql-connector-kafka-0.10/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-kafka-0.10/src/main/resources/META-INF/NOTICE index fefdb3e980..1a0d87599c 100644 --- a/flink-connectors/flink-sql-connector-kafka-0.10/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-kafka-0.10/src/main/resources/META-INF/NOTICE @@ -6,4 +6,4 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) -- org.apache.kafka:kafka-clients:0.10.2.1 +- org.apache.kafka:kafka-clients:0.10.2.2 -- Gitee From 0629d169358e1b8e89545a0977f8995d298db398 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 2 Apr 2020 11:03:49 +0200 Subject: [PATCH 427/885] [FLINK-16942][es] Allow users to override http/transport type --- .../connectors/elasticsearch5/Elasticsearch5ApiCallBridge.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-elasticsearch5/src/main/java/org/apache/flink/streaming/connectors/elasticsearch5/Elasticsearch5ApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch5/src/main/java/org/apache/flink/streaming/connectors/elasticsearch5/Elasticsearch5ApiCallBridge.java index ae9b0bcc45..73928195fc 100644 --- a/flink-connectors/flink-connector-elasticsearch5/src/main/java/org/apache/flink/streaming/connectors/elasticsearch5/Elasticsearch5ApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch5/src/main/java/org/apache/flink/streaming/connectors/elasticsearch5/Elasticsearch5ApiCallBridge.java @@ -67,9 +67,10 @@ public class Elasticsearch5ApiCallBridge implements ElasticsearchApiCallBridge clientConfig) { - Settings settings = Settings.builder().put(clientConfig) + Settings settings = Settings.builder() .put(NetworkModule.HTTP_TYPE_KEY, Netty3Plugin.NETTY_HTTP_TRANSPORT_NAME) .put(NetworkModule.TRANSPORT_TYPE_KEY, Netty3Plugin.NETTY_TRANSPORT_NAME) + .put(clientConfig) .build(); TransportClient transportClient = new PreBuiltTransportClient(settings); -- Gitee From 6a42bf6e72ddd86b88554c2342f0696b7c9af12b Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Fri, 3 Apr 2020 09:29:26 +0200 Subject: [PATCH 428/885] Revert "[FLINK-16535][table] BatchTableSink emitDataSet to consumeDataSet" This reverts commit 9ffa85aaa1b9025e3d2becdc084a02aa8440d4d9. --- .../flink/api/java/io/jdbc/JDBCAppendTableSink.java | 5 ++--- .../client/gateway/local/CollectBatchTableSink.java | 5 ++--- .../org/apache/flink/table/sinks/BatchTableSink.java | 12 +++--------- .../org/apache/flink/table/sinks/CsvTableSink.java | 4 ++-- .../factories/utils/TestCollectionTableFactory.scala | 3 +-- .../flink/table/api/internal/BatchTableEnvImpl.scala | 2 +- .../factories/utils/TestCollectionTableFactory.scala | 3 +-- .../runtime/batch/sql/PartitionableSinkITCase.scala | 5 ++--- .../table/utils/MemoryTableSourceSinkUtil.scala | 3 +-- .../common/table/SpendReportTableSink.java | 5 ++--- 10 files changed, 17 insertions(+), 30 deletions(-) diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSink.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSink.java index 412a0ce4a0..fde2585ef6 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSink.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSink.java @@ -21,7 +21,6 @@ package org.apache.flink.api.java.io.jdbc; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.operators.DataSink; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; @@ -70,8 +69,8 @@ public class JDBCAppendTableSink implements AppendStreamTableSink, BatchTab } @Override - public DataSink consumeDataSet(DataSet dataSet) { - return dataSet.output(outputFormat); + public void emitDataSet(DataSet dataSet) { + dataSet.output(outputFormat); } @Override diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/CollectBatchTableSink.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/CollectBatchTableSink.java index b2ef69525e..801f1f7ca9 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/CollectBatchTableSink.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/CollectBatchTableSink.java @@ -23,7 +23,6 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.Utils; -import org.apache.flink.api.java.operators.DataSink; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.sinks.BatchTableSink; import org.apache.flink.table.sinks.OutputFormatTableSink; @@ -69,8 +68,8 @@ public class CollectBatchTableSink extends OutputFormatTableSink implements } @Override - public DataSink consumeDataSet(DataSet dataSet) { - return dataSet + public void emitDataSet(DataSet dataSet) { + dataSet .output(new Utils.CollectHelper<>(accumulatorName, serializer)) .name("SQL Client Batch Collect Sink"); } diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/BatchTableSink.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/BatchTableSink.java index 29221b4a4e..f9413b7560 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/BatchTableSink.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/BatchTableSink.java @@ -18,13 +18,10 @@ package org.apache.flink.table.sinks; -import org.apache.flink.api.common.Plan; import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.operators.DataSink; import org.apache.flink.table.api.Table; -/** - * Defines an external {@link TableSink} to emit a batch {@link Table}. +/** Defines an external {@link TableSink} to emit a batch {@link Table}. * * @param Type of {@link DataSet} that this {@link TableSink} expects and supports. * @@ -33,9 +30,6 @@ import org.apache.flink.table.api.Table; @Deprecated public interface BatchTableSink extends TableSink { - /** - * Consumes the DataSet and return the {@link DataSink}. - * The returned {@link DataSink} will be used to generate {@link Plan}. - */ - DataSink consumeDataSet(DataSet dataSet); + /** Emits the DataSet. */ + void emitDataSet(DataSet dataSet); } diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSink.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSink.java index 4ea61ee30f..dfe5e8763f 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSink.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSink.java @@ -112,7 +112,7 @@ public class CsvTableSink implements BatchTableSink, AppendStreamTableSink< } @Override - public DataSink consumeDataSet(DataSet dataSet) { + public void emitDataSet(DataSet dataSet) { MapOperator csvRows = dataSet.map(new CsvFormatter(fieldDelim == null ? "," : fieldDelim)); @@ -128,7 +128,7 @@ public class CsvTableSink implements BatchTableSink, AppendStreamTableSink< sink.setParallelism(numFiles); } - return sink.name(TableConnectorUtils.generateRuntimeName(CsvTableSink.class, fieldNames)); + sink.name(TableConnectorUtils.generateRuntimeName(CsvTableSink.class, fieldNames)); } @Override diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/factories/utils/TestCollectionTableFactory.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/factories/utils/TestCollectionTableFactory.scala index f1ae9428d0..228ea3c5f6 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/factories/utils/TestCollectionTableFactory.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/factories/utils/TestCollectionTableFactory.scala @@ -22,7 +22,6 @@ import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.common.typeutils.TypeSerializer import org.apache.flink.api.java.io.{CollectionInputFormat, LocalCollectionOutputFormat} -import org.apache.flink.api.java.operators.DataSink import org.apache.flink.api.java.{DataSet, ExecutionEnvironment} import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.datastream.{DataStream, DataStreamSink, DataStreamSource} @@ -166,7 +165,7 @@ object TestCollectionTableFactory { class CollectionTableSink(val schema: TableSchema) extends BatchTableSink[Row] with AppendStreamTableSink[Row] { - override def consumeDataSet(dataSet: DataSet[Row]): DataSink[_] = { + override def emitDataSet(dataSet: DataSet[Row]): Unit = { dataSet.output(new LocalCollectionOutputFormat[Row](RESULT)).setParallelism(1) } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/BatchTableEnvImpl.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/BatchTableEnvImpl.scala index 19b26fcfe0..9fe93cb340 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/BatchTableEnvImpl.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/BatchTableEnvImpl.scala @@ -130,7 +130,7 @@ abstract class BatchTableEnvImpl( // translate the Table into a DataSet and provide the type that the TableSink expects. val result: DataSet[T] = translate(table)(outputType) // Give the DataSet to the TableSink to emit it. - batchSink.consumeDataSet(result) + batchSink.emitDataSet(result) case boundedSink: OutputFormatTableSink[T] => val outputType = fromDataTypeToLegacyInfo(sink.getConsumedDataType) .asInstanceOf[TypeInformation[T]] diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/factories/utils/TestCollectionTableFactory.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/factories/utils/TestCollectionTableFactory.scala index ded1cbdbcf..bd31d46419 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/factories/utils/TestCollectionTableFactory.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/factories/utils/TestCollectionTableFactory.scala @@ -22,7 +22,6 @@ import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.common.typeutils.TypeSerializer import org.apache.flink.api.java.io.{CollectionInputFormat, LocalCollectionOutputFormat} -import org.apache.flink.api.java.operators.DataSink import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.api.java.{DataSet, ExecutionEnvironment} import org.apache.flink.configuration.Configuration @@ -177,7 +176,7 @@ object TestCollectionTableFactory { class CollectionTableSink(val outputType: RowTypeInfo) extends BatchTableSink[Row] with AppendStreamTableSink[Row] { - override def consumeDataSet(dataSet: DataSet[Row]): DataSink[_] = { + override def emitDataSet(dataSet: DataSet[Row]): Unit = { dataSet.output(new LocalCollectionOutputFormat[Row](RESULT)).setParallelism(1) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/PartitionableSinkITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/PartitionableSinkITCase.scala index ca3381b683..90a6f247f7 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/PartitionableSinkITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/PartitionableSinkITCase.scala @@ -19,6 +19,7 @@ package org.apache.flink.table.runtime.batch.sql import java.util.{LinkedList => JLinkedList, Map => JMap} + import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.functions.MapFunction import org.apache.flink.api.common.io.RichOutputFormat @@ -26,7 +27,6 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo.{INT_TYPE_INFO, LONG_T import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java import org.apache.flink.api.java.DataSet -import org.apache.flink.api.java.operators.DataSink import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.api.scala.ExecutionEnvironment import org.apache.flink.configuration.Configuration @@ -39,7 +39,6 @@ import org.apache.flink.table.sources.BatchTableSource import org.apache.flink.table.types.logical.{BigIntType, IntType, VarCharType} import org.apache.flink.test.util.AbstractTestBase import org.apache.flink.types.Row - import org.junit.Assert.assertEquals import org.junit.rules.ExpectedException import org.junit.{Before, Rule, Test} @@ -155,7 +154,7 @@ class PartitionableSinkITCase extends AbstractTestBase { staticPartitions } - override def consumeDataSet(dataSet: DataSet[Row]): DataSink[_] = { + override def emitDataSet(dataSet: DataSet[Row]): Unit = { dataSet.map(new MapFunction[Row, String] { override def map(value: Row): String = value.toString }).output(new CollectionOutputFormat) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MemoryTableSourceSinkUtil.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MemoryTableSourceSinkUtil.scala index 034e9ca3ea..a835bba752 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MemoryTableSourceSinkUtil.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MemoryTableSourceSinkUtil.scala @@ -20,7 +20,6 @@ package org.apache.flink.table.utils import org.apache.flink.api.common.io.{OutputFormat, RichOutputFormat} import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.java.operators.DataSink import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.api.java.{DataSet, ExecutionEnvironment} import org.apache.flink.configuration.Configuration @@ -112,7 +111,7 @@ object MemoryTableSourceSinkUtil { new UnsafeMemoryAppendTableSink } - override def consumeDataSet(dataSet: DataSet[Row]): DataSink[_] = { + override def emitDataSet(dataSet: DataSet[Row]): Unit = { dataSet .output(new MemoryCollectionOutputFormat) .name(TableConnectorUtils.generateRuntimeName(this.getClass, getTableSchema.getFieldNames)) diff --git a/flink-walkthroughs/flink-walkthrough-common/src/main/java/org/apache/flink/walkthrough/common/table/SpendReportTableSink.java b/flink-walkthroughs/flink-walkthrough-common/src/main/java/org/apache/flink/walkthrough/common/table/SpendReportTableSink.java index 5073d90e1f..a143306d07 100644 --- a/flink-walkthroughs/flink-walkthrough-common/src/main/java/org/apache/flink/walkthrough/common/table/SpendReportTableSink.java +++ b/flink-walkthroughs/flink-walkthrough-common/src/main/java/org/apache/flink/walkthrough/common/table/SpendReportTableSink.java @@ -22,7 +22,6 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.operators.DataSink; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.table.api.TableSchema; @@ -52,8 +51,8 @@ public class SpendReportTableSink implements AppendStreamTableSink, BatchTa } @Override - public DataSink consumeDataSet(DataSet dataSet) { - return dataSet + public void emitDataSet(DataSet dataSet) { + dataSet .map(SpendReportTableSink::format) .output(new LoggerOutputFormat()); } -- Gitee From 0c94de1b968f4f9da8dd101d97292ccf8bd66d04 Mon Sep 17 00:00:00 2001 From: Rui Li Date: Wed, 1 Apr 2020 21:13:29 +0800 Subject: [PATCH 429/885] [FLINK-16772][hive] Move derby dependency to test scope --- flink-connectors/flink-connector-hive/pom.xml | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/flink-connectors/flink-connector-hive/pom.xml b/flink-connectors/flink-connector-hive/pom.xml index 2b43b969b6..2b31c7adf6 100644 --- a/flink-connectors/flink-connector-hive/pom.xml +++ b/flink-connectors/flink-connector-hive/pom.xml @@ -38,6 +38,7 @@ under the License. 4.0.0 0.9.8 + 10.10.2.0 @@ -285,6 +286,10 @@ under the License. org.slf4j slf4j-log4j12 + + org.apache.derby + derby + @@ -740,6 +745,14 @@ under the License. test + + + org.apache.derby + derby + ${derby.version} + test + + @@ -824,6 +837,7 @@ under the License. 1.0.1 2.6.5 3.1.1 + 10.10.1.1 @@ -851,6 +865,7 @@ under the License. 1.1.0 2.6.5 3.1.1 + 10.11.1.1 @@ -932,6 +947,7 @@ under the License. hive-3.1.1 3.1.1 + 10.14.1.0 -- Gitee From 8d66baa1385079089462385072fc6a67831fb1f5 Mon Sep 17 00:00:00 2001 From: wangyang0918 Date: Thu, 2 Apr 2020 16:50:15 +0800 Subject: [PATCH 430/885] [FLINK-16921][e2e] Do not start/stop minikube in non-linux environment for k8s e2e tests --- .../test-scripts/common_kubernetes.sh | 80 ++++++++++--------- 1 file changed, 44 insertions(+), 36 deletions(-) diff --git a/flink-end-to-end-tests/test-scripts/common_kubernetes.sh b/flink-end-to-end-tests/test-scripts/common_kubernetes.sh index 1e0c063bc4..7c0b9cd035 100755 --- a/flink-end-to-end-tests/test-scripts/common_kubernetes.sh +++ b/flink-end-to-end-tests/test-scripts/common_kubernetes.sh @@ -27,6 +27,8 @@ MINIKUBE_START_RETRIES=3 MINIKUBE_START_BACKOFF=5 RESULT_HASH="e682ec6622b5e83f2eb614617d5ab2cf" +NON_LINUX_ENV_NOTE="****** Please start/stop minikube manually in non-linux environment. ******" + # If running tests on non-linux os, the kubectl and minikube should be installed manually function setup_kubernetes_for_linux { # Download kubectl, which is a requirement for using minikube. @@ -53,32 +55,28 @@ function start_kubernetes_if_not_running { if ! check_kubernetes_status; then echo "Starting minikube ..." # We need sudo permission to set vm-driver to none in linux os. - if [[ "${OS_TYPE}" = "linux" ]] ; then - # tl;dr: Configure minikube for a low disk space environment - # - # The VMs provided by azure have ~100GB of disk space, out of which - # 85% are allocated, only 15GB are free. That's enough space - # for our purposes. However, the kubernetes nodes running during - # the k8s tests believe that 85% are not enough free disk space, - # so they start garbage collecting their host. During GCing, they - # are deleting all docker images currently not in use. - # However, the k8s test is first building a flink image, then launching - # stuff on k8s. Sometimes, k8s deletes the new Flink images, - # thus it can not find them anymore, letting the test fail / - # timeout. That's why we have set the GC threshold to 98% and 99% - # here. - # Similarly, the kubelets are marking themself as "low disk space", - # causing Flink to avoid this node (again, failing the test) - sudo CHANGE_MINIKUBE_NONE_USER=true minikube start --vm-driver=none \ - --extra-config=kubelet.image-gc-high-threshold=99 \ - --extra-config=kubelet.image-gc-low-threshold=98 \ - --extra-config=kubelet.minimum-container-ttl-duration=120m \ - --extra-config=kubelet.eviction-hard="memory.available<5Mi,nodefs.available<1Mi,imagefs.available<1Mi" \ - --extra-config=kubelet.eviction-soft="memory.available<5Mi,nodefs.available<2Mi,imagefs.available<2Mi" \ - --extra-config=kubelet.eviction-soft-grace-period="memory.available=2h,nodefs.available=2h,imagefs.available=2h" - else - sudo minikube start - fi + # tl;dr: Configure minikube for a low disk space environment + # + # The VMs provided by azure have ~100GB of disk space, out of which + # 85% are allocated, only 15GB are free. That's enough space + # for our purposes. However, the kubernetes nodes running during + # the k8s tests believe that 85% are not enough free disk space, + # so they start garbage collecting their host. During GCing, they + # are deleting all docker images currently not in use. + # However, the k8s test is first building a flink image, then launching + # stuff on k8s. Sometimes, k8s deletes the new Flink images, + # thus it can not find them anymore, letting the test fail / + # timeout. That's why we have set the GC threshold to 98% and 99% + # here. + # Similarly, the kubelets are marking themself as "low disk space", + # causing Flink to avoid this node (again, failing the test) + sudo CHANGE_MINIKUBE_NONE_USER=true minikube start --vm-driver=none \ + --extra-config=kubelet.image-gc-high-threshold=99 \ + --extra-config=kubelet.image-gc-low-threshold=98 \ + --extra-config=kubelet.minimum-container-ttl-duration=120m \ + --extra-config=kubelet.eviction-hard="memory.available<5Mi,nodefs.available<1Mi,imagefs.available<1Mi" \ + --extra-config=kubelet.eviction-soft="memory.available<5Mi,nodefs.available<2Mi,imagefs.available<2Mi" \ + --extra-config=kubelet.eviction-soft-grace-period="memory.available=2h,nodefs.available=2h,imagefs.available=2h" # Fix the kubectl context, as it's often stale. minikube update-context fi @@ -88,21 +86,31 @@ function start_kubernetes_if_not_running { } function start_kubernetes { - [[ "${OS_TYPE}" = "linux" ]] && setup_kubernetes_for_linux - if ! retry_times ${MINIKUBE_START_RETRIES} ${MINIKUBE_START_BACKOFF} start_kubernetes_if_not_running; then - echo "Could not start minikube. Aborting..." - exit 1 + if [[ "${OS_TYPE}" != "linux" ]]; then + if ! check_kubernetes_status; then + echo "$NON_LINUX_ENV_NOTE" + exit 1 + fi + else + setup_kubernetes_for_linux + if ! retry_times ${MINIKUBE_START_RETRIES} ${MINIKUBE_START_BACKOFF} start_kubernetes_if_not_running; then + echo "Could not start minikube. Aborting..." + exit 1 + fi fi eval $(minikube docker-env) } function stop_kubernetes { - echo "Stopping minikube ..." - stop_command="minikube stop" - [[ "${OS_TYPE}" = "linux" ]] && stop_command="sudo ${stop_command}" - if ! retry_times ${MINIKUBE_START_RETRIES} ${MINIKUBE_START_BACKOFF} "${stop_command}"; then - echo "Could not stop minikube. Aborting..." - exit 1 + if [[ "${OS_TYPE}" != "linux" ]]; then + echo "$NON_LINUX_ENV_NOTE" + else + echo "Stopping minikube ..." + stop_command="sudo minikube stop" + if ! retry_times ${MINIKUBE_START_RETRIES} ${MINIKUBE_START_BACKOFF} "${stop_command}"; then + echo "Could not stop minikube. Aborting..." + exit 1 + fi fi } -- Gitee From c3ec5b5fa10c2c71d6f4ced3ec1a223baa2f5c4b Mon Sep 17 00:00:00 2001 From: wangyang0918 Date: Thu, 2 Apr 2020 17:41:32 +0800 Subject: [PATCH 431/885] [FLINK-16921][e2e] Print more information for debugging when Kubernetes e2e tests failed --- .../test-scripts/common_kubernetes.sh | 11 ++++++++++ .../test_kubernetes_embedded_job.sh | 4 ++++ .../test-scripts/test_kubernetes_session.sh | 21 ++++++++++++++++++- 3 files changed, 35 insertions(+), 1 deletion(-) diff --git a/flink-end-to-end-tests/test-scripts/common_kubernetes.sh b/flink-end-to-end-tests/test-scripts/common_kubernetes.sh index 7c0b9cd035..487695d5a2 100755 --- a/flink-end-to-end-tests/test-scripts/common_kubernetes.sh +++ b/flink-end-to-end-tests/test-scripts/common_kubernetes.sh @@ -114,4 +114,15 @@ function stop_kubernetes { fi } +function debug_copy_and_show_logs { + echo "Debugging failed Kubernetes test:" + echo "Currently existing Kubernetes resources" + kubectl get all + + echo "Flink logs:" + kubectl get pods -o jsonpath='{range .items[*]}{.metadata.name}{"\n"}{end}' | while read pod;do + kubectl logs $pod; + done +} + on_exit cleanup diff --git a/flink-end-to-end-tests/test-scripts/test_kubernetes_embedded_job.sh b/flink-end-to-end-tests/test-scripts/test_kubernetes_embedded_job.sh index eb051840c6..fe112b1969 100755 --- a/flink-end-to-end-tests/test-scripts/test_kubernetes_embedded_job.sh +++ b/flink-end-to-end-tests/test-scripts/test_kubernetes_embedded_job.sh @@ -49,3 +49,7 @@ kubectl wait --for=condition=complete job/flink-job-cluster --timeout=1h kubectl cp `kubectl get pods | awk '/task-manager/ {print $1}'`:/cache/${OUTPUT_FILE} ${OUTPUT_VOLUME}/${OUTPUT_FILE} check_result_hash "WordCount" ${OUTPUT_VOLUME}/${OUTPUT_FILE} "${RESULT_HASH}" + +if [ $? != 0 ];then + debug_copy_and_show_logs +fi diff --git a/flink-end-to-end-tests/test-scripts/test_kubernetes_session.sh b/flink-end-to-end-tests/test-scripts/test_kubernetes_session.sh index 3dc3bae84d..5140a87e1a 100755 --- a/flink-end-to-end-tests/test-scripts/test_kubernetes_session.sh +++ b/flink-end-to-end-tests/test-scripts/test_kubernetes_session.sh @@ -32,6 +32,20 @@ function cleanup { stop_kubernetes } +function setConsoleLogging { + cat >> $FLINK_DIR/conf/log4j.properties < Date: Thu, 2 Apr 2020 18:00:58 +0800 Subject: [PATCH 432/885] [FLINK-16921][e2e] Wait for rest endpoint up and then submit Flink job to existing Kubernetes session --- .../test-scripts/test_kubernetes_session.sh | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/flink-end-to-end-tests/test-scripts/test_kubernetes_session.sh b/flink-end-to-end-tests/test-scripts/test_kubernetes_session.sh index 5140a87e1a..0cfc633f27 100755 --- a/flink-end-to-end-tests/test-scripts/test_kubernetes_session.sh +++ b/flink-end-to-end-tests/test-scripts/test_kubernetes_session.sh @@ -57,13 +57,18 @@ kubectl create clusterrolebinding ${CLUSTER_ROLE_BINDING} --clusterrole=edit --s mkdir -p "$(dirname $LOCAL_OUTPUT_PATH)" # Set the memory and cpu smaller than default, so that the jobmanager and taskmanager pods could be allocated in minikube. -"$FLINK_DIR"/bin/kubernetes-session.sh -Dkubernetes.cluster-id=${CLUSTER_ID} \ +OUTPUT=`"$FLINK_DIR"/bin/kubernetes-session.sh -Dkubernetes.cluster-id=${CLUSTER_ID} \ -Dkubernetes.container.image=${FLINK_IMAGE_NAME} \ -Djobmanager.heap.size=512m \ -Dcontainerized.heap-cutoff-min=100 \ -Dkubernetes.jobmanager.cpu=0.5 \ -Dkubernetes.taskmanager.cpu=0.5 \ - -Dkubernetes.container-start-command-template="%java% %classpath% %jvmmem% %jvmopts% %logging% %class% %args%" + -Dkubernetes.container-start-command-template="%java% %classpath% %jvmmem% %jvmopts% %logging% %class% %args%"` + +echo "$OUTPUT" + +JOBMANAGER_URL=$(echo "$OUTPUT" | grep 'JobManager Web Interface: ' | awk -F'JobManager Web Interface: ' '{print $2}') +wait_rest_endpoint_up "${JOBMANAGER_URL}/taskmanagers" "Dispatcher" "\{\"taskmanagers\":\[.*\]\}" "$FLINK_DIR"/bin/flink run -e kubernetes-session \ -Dkubernetes.cluster-id=${CLUSTER_ID} \ -- Gitee From 13ef8238c488f9389414e03d046bb5e2143d955e Mon Sep 17 00:00:00 2001 From: "kevin.cyj" Date: Wed, 1 Apr 2020 15:35:04 +0800 Subject: [PATCH 433/885] [FLINK-15305][tests] Make BoundedDataTestBase#testGetSizeMultipleRegions respect system page size to avoid potential failure --- .../io/network/partition/BoundedDataTestBase.java | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedDataTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedDataTestBase.java index 2503c00e53..86cecea3bb 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedDataTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedDataTestBase.java @@ -144,7 +144,7 @@ public abstract class BoundedDataTestBase { @Test public void testGetSizeSingleRegion() throws Exception { try (BoundedData bd = createBoundedData()) { - testGetSize(bd); + testGetSize(bd, 60_787, 76_687); } } @@ -154,14 +154,13 @@ public abstract class BoundedDataTestBase { return; } - try (BoundedData bd = createBoundedDataWithRegion(100_000)) { - testGetSize(bd); + int pageSize = PageSizeUtil.getSystemPageSizeOrConservativeMultiple(); + try (BoundedData bd = createBoundedDataWithRegion(pageSize)) { + testGetSize(bd, pageSize / 3, pageSize - BufferReaderWriterUtil.HEADER_LENGTH); } } - private static void testGetSize(BoundedData bd) throws Exception { - final int bufferSize1 = 60_787; - final int bufferSize2 = 76_687; + private static void testGetSize(BoundedData bd, int bufferSize1, int bufferSize2) throws Exception { final int expectedSize1 = bufferSize1 + BufferReaderWriterUtil.HEADER_LENGTH; final int expectedSizeFinal = bufferSize1 + bufferSize2 + 2 * BufferReaderWriterUtil.HEADER_LENGTH; -- Gitee From f87734df01f63511e82a36c046afb0842ac7a63a Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Fri, 3 Apr 2020 11:14:53 +0200 Subject: [PATCH 434/885] [hotfix][metrics] Extend MetricReporterFactory javadocs --- .../flink/metrics/reporter/MetricReporterFactory.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/reporter/MetricReporterFactory.java b/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/reporter/MetricReporterFactory.java index 125d01f413..95afd4a931 100644 --- a/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/reporter/MetricReporterFactory.java +++ b/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/reporter/MetricReporterFactory.java @@ -22,6 +22,14 @@ import java.util.Properties; /** * {@link MetricReporter} factory. + * + *

    Reporters that can be instantiated with a factory automatically qualify for being loaded as a plugin, so long as + * the reporter jar is self-contained (excluding Flink dependencies) and contains a + * {@code META-INF/services/org.apache.flink.metrics.reporter.MetricReporterFactory} file containing the + * qualified class name of the factory. + * + *

    Reporters that previously relied on reflection for instantiation can use the {@link InstantiateViaFactory} + * annotation to redirect reflection-base instantiation attempts to the factory instead. */ public interface MetricReporterFactory { -- Gitee From db81417b8616850aa36b34fecafcc275db2cb10a Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Sat, 21 Mar 2020 14:17:34 +0100 Subject: [PATCH 435/885] [FLINK-16705] Ensure MiniCluster shutdown does not interfere with JobResult retrieval There is a race condition in `LocalExecutor` between (a) shutting down the cluster when the job has finished and (b) the client which retrieves the result of the job execution. This was observed in Beam, running a large test suite with the Flink Runner. We should make sure the job result retrieval and the cluster shutdown do not interfere. This adds a PerJobMiniClusterClient which guarantees that. Improve message for running flag state checks in MiniCluster Additionally check for the JobID in PerJobMiniClusterClient Introduce PerJobMiniCluster and a corresponding JobClient Add TestLogger to test Convert shutdown methods to be async This closes #11473. --- .../deployment/executors/LocalExecutor.java | 85 +++---- .../executors/LocalExecutorFactory.java | 2 +- .../program/PerJobMiniClusterFactory.java | 182 +++++++++++++++ .../program/PerJobMiniClusterFactoryTest.java | 211 ++++++++++++++++++ .../core/execution/PipelineExecutor.java | 2 +- .../flink/api/java/ExecutionEnvironment.java | 2 +- .../runtime/minicluster/MiniCluster.java | 6 +- .../StreamExecutionEnvironment.java | 2 +- .../client/gateway/local/ProgramDeployer.java | 4 +- .../flink/core/testutils/CommonTestUtils.java | 20 ++ .../example/client/LocalExecutorITCase.java | 61 ++++- 11 files changed, 504 insertions(+), 73 deletions(-) create mode 100644 flink-clients/src/main/java/org/apache/flink/client/program/PerJobMiniClusterFactory.java create mode 100644 flink-clients/src/test/java/org/apache/flink/client/program/PerJobMiniClusterFactoryTest.java diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/LocalExecutor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/LocalExecutor.java index 49ed5c97df..ebe71d7bd7 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/LocalExecutor.java +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/LocalExecutor.java @@ -19,26 +19,22 @@ package org.apache.flink.client.deployment.executors; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.Plan; import org.apache.flink.api.dag.Pipeline; import org.apache.flink.client.FlinkPipelineTranslationUtil; -import org.apache.flink.client.deployment.ClusterClientJobClientAdapter; -import org.apache.flink.client.program.MiniClusterClient; +import org.apache.flink.client.program.PerJobMiniClusterFactory; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.DeploymentOptions; -import org.apache.flink.configuration.RestOptions; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.core.execution.JobClient; import org.apache.flink.core.execution.PipelineExecutor; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.minicluster.MiniCluster; import org.apache.flink.runtime.minicluster.MiniClusterConfiguration; -import org.apache.flink.runtime.minicluster.RpcServiceSharing; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CompletionException; +import java.util.function.Function; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.flink.util.Preconditions.checkState; @@ -51,26 +47,38 @@ public class LocalExecutor implements PipelineExecutor { public static final String NAME = "local"; + private final Configuration configuration; + private final Function miniClusterFactory; + + public static LocalExecutor create(Configuration configuration) { + return new LocalExecutor(configuration, MiniCluster::new); + } + + public static LocalExecutor createWithFactory( + Configuration configuration, Function miniClusterFactory) { + return new LocalExecutor(configuration, miniClusterFactory); + } + + private LocalExecutor(Configuration configuration, Function miniClusterFactory) { + this.configuration = configuration; + this.miniClusterFactory = miniClusterFactory; + } + @Override - public CompletableFuture execute(Pipeline pipeline, Configuration configuration) throws Exception { + public CompletableFuture execute(Pipeline pipeline, Configuration configuration) throws Exception { checkNotNull(pipeline); checkNotNull(configuration); + Configuration effectiveConfig = new Configuration(); + effectiveConfig.addAll(this.configuration); + effectiveConfig.addAll(configuration); + // we only support attached execution with the local executor. checkState(configuration.getBoolean(DeploymentOptions.ATTACHED)); - final JobGraph jobGraph = getJobGraph(pipeline, configuration); - final MiniCluster miniCluster = startMiniCluster(jobGraph, configuration); - final MiniClusterClient clusterClient = new MiniClusterClient(configuration, miniCluster); - - CompletableFuture jobIdFuture = clusterClient.submitJob(jobGraph); + final JobGraph jobGraph = getJobGraph(pipeline, effectiveConfig); - jobIdFuture - .thenCompose(clusterClient::requestJobResult) - .thenAccept((jobResult) -> clusterClient.shutDownCluster()); - - return jobIdFuture.thenApply(jobID -> - new ClusterClientJobClientAdapter<>(() -> clusterClient, jobID)); + return PerJobMiniClusterFactory.createWithFactory(effectiveConfig, miniClusterFactory).submitJob(jobGraph); } private JobGraph getJobGraph(Pipeline pipeline, Configuration configuration) { @@ -89,45 +97,4 @@ public class LocalExecutor implements PipelineExecutor { return FlinkPipelineTranslationUtil.getJobGraph(pipeline, configuration, 1); } - - private MiniCluster startMiniCluster(final JobGraph jobGraph, final Configuration configuration) throws Exception { - if (!configuration.contains(RestOptions.BIND_PORT)) { - configuration.setString(RestOptions.BIND_PORT, "0"); - } - - int numTaskManagers = configuration.getInteger( - ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, - ConfigConstants.DEFAULT_LOCAL_NUMBER_TASK_MANAGER); - - // we have to use the maximum parallelism as a default here, otherwise streaming - // pipelines would not run - int numSlotsPerTaskManager = configuration.getInteger( - TaskManagerOptions.NUM_TASK_SLOTS, - jobGraph.getMaximumParallelism()); - - final MiniClusterConfiguration miniClusterConfiguration = - new MiniClusterConfiguration.Builder() - .setConfiguration(configuration) - .setNumTaskManagers(numTaskManagers) - .setRpcServiceSharing(RpcServiceSharing.SHARED) - .setNumSlotsPerTaskManager(numSlotsPerTaskManager) - .build(); - - final MiniCluster miniCluster = new MiniCluster(miniClusterConfiguration); - miniCluster.start(); - - configuration.setInteger(RestOptions.PORT, miniCluster.getRestAddress().get().getPort()); - - return miniCluster; - } - - private void shutdownMiniCluster(final MiniCluster miniCluster) { - try { - if (miniCluster != null) { - miniCluster.close(); - } - } catch (Exception e) { - throw new CompletionException(e); - } - } } diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/LocalExecutorFactory.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/LocalExecutorFactory.java index 8bc605644a..dd9bfb54eb 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/LocalExecutorFactory.java +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/LocalExecutorFactory.java @@ -42,6 +42,6 @@ public class LocalExecutorFactory implements PipelineExecutorFactory { @Override public PipelineExecutor getExecutor(final Configuration configuration) { - return new LocalExecutor(); + return LocalExecutor.create(configuration); } } diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/PerJobMiniClusterFactory.java b/flink-clients/src/main/java/org/apache/flink/client/program/PerJobMiniClusterFactory.java new file mode 100644 index 0000000000..ec57cde9c3 --- /dev/null +++ b/flink-clients/src/main/java/org/apache/flink/client/program/PerJobMiniClusterFactory.java @@ -0,0 +1,182 @@ +/* + * 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.client.program; + +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.RestOptions; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmaster.JobResult; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.minicluster.MiniClusterConfiguration; +import org.apache.flink.runtime.minicluster.RpcServiceSharing; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.function.Function; + +/** + * Starts a {@link MiniCluster} for every submitted job. + * This class guarantees to tear down the MiniCluster in case of normal or exceptional job completion. + * */ +public final class PerJobMiniClusterFactory { + + private static final Logger LOG = LoggerFactory.getLogger(PerJobMiniClusterFactory.class); + + private final Configuration configuration; + private final Function miniClusterFactory; + + public static PerJobMiniClusterFactory create() { + return new PerJobMiniClusterFactory(new Configuration(), MiniCluster::new); + } + + public static PerJobMiniClusterFactory createWithFactory( + Configuration configuration, + Function miniClusterFactory) { + return new PerJobMiniClusterFactory(configuration, miniClusterFactory); + } + + private PerJobMiniClusterFactory( + Configuration configuration, + Function miniClusterFactory) { + this.configuration = configuration; + this.miniClusterFactory = miniClusterFactory; + } + + /** + * Starts a {@link MiniCluster} and submits a job. + */ + public CompletableFuture submitJob(JobGraph jobGraph) throws Exception { + MiniClusterConfiguration miniClusterConfig = getMiniClusterConfig(jobGraph.getMaximumParallelism()); + MiniCluster miniCluster = miniClusterFactory.apply(miniClusterConfig); + miniCluster.start(); + + return miniCluster + .submitJob(jobGraph) + .thenApply(result -> new PerJobMiniClusterJobClient(result.getJobID(), miniCluster)) + .whenComplete((ignored, throwable) -> { + if (throwable != null) { + // We failed to create the JobClient and must shutdown to ensure cleanup. + shutDownCluster(miniCluster); + } + }); + } + + private MiniClusterConfiguration getMiniClusterConfig(int maximumParallelism) { + Configuration configuration = new Configuration(this.configuration); + + if (!configuration.contains(RestOptions.BIND_PORT)) { + configuration.setString(RestOptions.BIND_PORT, "0"); + } + + int numTaskManagers = configuration.getInteger( + ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, + ConfigConstants.DEFAULT_LOCAL_NUMBER_TASK_MANAGER); + + // we have to use the maximum parallelism as a default here, otherwise streaming pipelines would not run + int numSlotsPerTaskManager = configuration.getInteger(TaskManagerOptions.NUM_TASK_SLOTS, maximumParallelism); + + return new MiniClusterConfiguration.Builder() + .setConfiguration(configuration) + .setNumTaskManagers(numTaskManagers) + .setRpcServiceSharing(RpcServiceSharing.SHARED) + .setNumSlotsPerTaskManager(numSlotsPerTaskManager) + .build(); + } + + private static void shutDownCluster(MiniCluster miniCluster) { + miniCluster.closeAsync() + .whenComplete((ignored, throwable) -> { + if (throwable != null) { + LOG.warn("Shutdown of MiniCluster failed.", throwable); + } + }); + } + + /** + * A {@link JobClient} for a {@link PerJobMiniClusterFactory}. + */ + private static final class PerJobMiniClusterJobClient implements JobClient { + + private final JobID jobID; + private final MiniCluster miniCluster; + private final CompletableFuture jobResultFuture; + + private PerJobMiniClusterJobClient(JobID jobID, MiniCluster miniCluster) { + this.jobID = jobID; + this.miniCluster = miniCluster; + this.jobResultFuture = miniCluster + .requestJobResult(jobID) + // Make sure to shutdown the cluster when the job completes. + .whenComplete((result, throwable) -> shutDownCluster(miniCluster)); + } + + @Override + public JobID getJobID() { + return jobID; + } + + @Override + public CompletableFuture getJobStatus() { + return miniCluster.getJobStatus(jobID); + } + + @Override + public CompletableFuture cancel() { + return miniCluster.cancelJob(jobID).thenAccept(result -> {}); + } + + @Override + public CompletableFuture stopWithSavepoint(boolean advanceToEndOfEventTime, @Nullable String savepointDirectory) { + return miniCluster.stopWithSavepoint(jobID, savepointDirectory, advanceToEndOfEventTime); + } + + @Override + public CompletableFuture triggerSavepoint(@Nullable String savepointDirectory) { + return miniCluster.triggerSavepoint(jobID, savepointDirectory, false); + } + + @Override + public CompletableFuture> getAccumulators(ClassLoader classLoader) { + return getJobExecutionResult(classLoader).thenApply(JobExecutionResult::getAllAccumulatorResults); + } + + @Override + public CompletableFuture getJobExecutionResult(ClassLoader classLoader) { + return jobResultFuture.thenApply(result -> { + try { + return result.toJobExecutionResult(classLoader); + } catch (Exception e) { + throw new CompletionException("Failed to convert JobResult to JobExecutionResult.", e); + } + }); + } + } +} diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/PerJobMiniClusterFactoryTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/PerJobMiniClusterFactoryTest.java new file mode 100644 index 0000000000..82a4750e20 --- /dev/null +++ b/flink-clients/src/test/java/org/apache/flink/client/program/PerJobMiniClusterFactoryTest.java @@ -0,0 +1,211 @@ +/* + * 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.client.program; + +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.runtime.testutils.CancelableInvokable; +import org.apache.flink.util.TestLogger; + +import org.junit.After; +import org.junit.Test; + +import java.util.Map; +import java.util.concurrent.ExecutionException; + +import static org.apache.flink.core.testutils.CommonTestUtils.assertThrows; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.MatcherAssert.assertThat; + +/** + * Tests for {@link PerJobMiniClusterFactory}. + */ +public class PerJobMiniClusterFactoryTest extends TestLogger { + + private MiniCluster miniCluster; + + @After + public void teardown() throws Exception { + if (miniCluster != null) { + miniCluster.close(); + } + } + + @Test + public void testJobExecution() throws Exception { + PerJobMiniClusterFactory perJobMiniClusterFactory = initializeMiniCluster(); + + JobClient jobClient = perJobMiniClusterFactory.submitJob(getNoopJobGraph()).get(); + + JobExecutionResult jobExecutionResult = jobClient.getJobExecutionResult(getClass().getClassLoader()).get(); + assertThat(jobExecutionResult, is(notNullValue())); + + Map actual = jobClient.getAccumulators(getClass().getClassLoader()).get(); + assertThat(actual, is(notNullValue())); + + assertThatMiniClusterIsShutdown(); + } + + @Test + public void testJobClient() throws Exception { + PerJobMiniClusterFactory perJobMiniClusterFactory = initializeMiniCluster(); + + JobGraph cancellableJobGraph = getCancellableJobGraph(); + JobClient jobClient = perJobMiniClusterFactory + .submitJob(cancellableJobGraph) + .get(); + + assertThat(jobClient.getJobID(), is(cancellableJobGraph.getJobID())); + assertThat(jobClient.getJobStatus().get(), is(JobStatus.RUNNING)); + + jobClient.cancel().get(); + + assertThrows( + "Job was cancelled.", + ExecutionException.class, + () -> jobClient.getJobExecutionResult(getClass().getClassLoader()).get() + ); + + assertThatMiniClusterIsShutdown(); + } + + @Test + public void testJobClientSavepoint() throws Exception { + PerJobMiniClusterFactory perJobMiniClusterFactory = initializeMiniCluster(); + JobClient jobClient = perJobMiniClusterFactory.submitJob(getCancellableJobGraph()).get(); + + assertThrows( + "is not a streaming job.", + ExecutionException.class, + () -> jobClient.triggerSavepoint(null).get()); + + assertThrows( + "is not a streaming job.", + ExecutionException.class, + () -> jobClient.stopWithSavepoint(true, null).get()); + } + + @Test + public void testSubmissionError() throws Exception { + PerJobMiniClusterFactory perJobMiniClusterFactory = initializeMiniCluster(); + + // JobGraph is not a valid job + JobGraph jobGraph = new JobGraph(); + + assertThrows( + "Failed to submit job.", + ExecutionException.class, + () -> perJobMiniClusterFactory.submitJob(jobGraph).get()); + + assertThatMiniClusterIsShutdown(); + } + + @Test + public void testMultipleExecutions() throws Exception { + PerJobMiniClusterFactory perJobMiniClusterFactory = initializeMiniCluster(); + { + JobClient jobClient = perJobMiniClusterFactory.submitJob(getNoopJobGraph()).get(); + jobClient.getJobExecutionResult(getClass().getClassLoader()).get(); + assertThatMiniClusterIsShutdown(); + } + { + JobClient jobClient = perJobMiniClusterFactory.submitJob(getNoopJobGraph()).get(); + jobClient.getJobExecutionResult(getClass().getClassLoader()).get(); + assertThatMiniClusterIsShutdown(); + } + } + + @Test + public void testJobClientInteractionAfterShutdown() throws Exception { + PerJobMiniClusterFactory perJobMiniClusterFactory = initializeMiniCluster(); + JobClient jobClient = perJobMiniClusterFactory.submitJob(getNoopJobGraph()).get(); + jobClient.getJobExecutionResult(getClass().getClassLoader()).get(); + assertThatMiniClusterIsShutdown(); + + assertThrows( + "MiniCluster is not yet running or has already been shut down.", + IllegalStateException.class, + jobClient::cancel); + } + + private PerJobMiniClusterFactory initializeMiniCluster() { + return PerJobMiniClusterFactory.createWithFactory(new Configuration(), config -> { + miniCluster = new MiniCluster(config); + return miniCluster; + }); + } + + private void assertThatMiniClusterIsShutdown() { + assertThat(miniCluster.isRunning(), is(false)); + } + + private static JobGraph getNoopJobGraph() { + JobGraph jobGraph = new JobGraph(); + JobVertex jobVertex = new JobVertex("jobVertex"); + jobVertex.setInvokableClass(NoOpInvokable.class); + jobGraph.addVertex(jobVertex); + return jobGraph; + } + + private static JobGraph getCancellableJobGraph() { + JobGraph jobGraph = new JobGraph(); + JobVertex jobVertex = new JobVertex("jobVertex"); + jobVertex.setInvokableClass(MyCancellableInvokable.class); + jobGraph.addVertex(jobVertex); + return jobGraph; + } + + /** + * Invokable which waits until it is cancelled. + */ + public static class MyCancellableInvokable extends CancelableInvokable { + + private final Object lock = new Object(); + private boolean running = true; + + public MyCancellableInvokable(Environment environment) { + super(environment); + } + + @Override + public void invoke() throws Exception { + synchronized (lock) { + while (running) { + lock.wait(); + } + } + } + + @Override + public void cancel() { + synchronized (lock) { + running = false; + lock.notifyAll(); + } + } + } +} diff --git a/flink-core/src/main/java/org/apache/flink/core/execution/PipelineExecutor.java b/flink-core/src/main/java/org/apache/flink/core/execution/PipelineExecutor.java index afe7cad870..733dbd6844 100644 --- a/flink-core/src/main/java/org/apache/flink/core/execution/PipelineExecutor.java +++ b/flink-core/src/main/java/org/apache/flink/core/execution/PipelineExecutor.java @@ -41,5 +41,5 @@ public interface PipelineExecutor { * @param configuration the {@link Configuration} with the required execution parameters * @return a {@link CompletableFuture} with the {@link JobClient} corresponding to the pipeline. */ - CompletableFuture execute(final Pipeline pipeline, final Configuration configuration) throws Exception; + CompletableFuture execute(final Pipeline pipeline, final Configuration configuration) throws Exception; } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java index d439a2f646..8674280a3f 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java @@ -956,7 +956,7 @@ public class ExecutionEnvironment { "Cannot find compatible factory for specified execution.target (=%s)", configuration.get(DeploymentOptions.TARGET)); - CompletableFuture jobClientFuture = executorFactory + CompletableFuture jobClientFuture = executorFactory .getExecutor(configuration) .execute(plan, configuration); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java index 4f4316f5cb..832e7d77c7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java @@ -206,14 +206,14 @@ public class MiniCluster implements JobExecutorService, AutoCloseableAsync { public CompletableFuture getRestAddress() { synchronized (lock) { - checkState(running, "MiniCluster is not yet running."); + checkState(running, "MiniCluster is not yet running or has already been shut down."); return webMonitorLeaderRetriever.getLeaderFuture().thenApply(FunctionUtils.uncheckedFunction(addressLeaderIdTuple -> new URI(addressLeaderIdTuple.f0))); } } public ClusterInformation getClusterInformation() { synchronized (lock) { - checkState(running, "MiniCluster is not yet running."); + checkState(running, "MiniCluster is not yet running or has already been shut down."); return new ClusterInformation("localhost", blobServer.getPort()); } } @@ -686,7 +686,7 @@ public class MiniCluster implements JobExecutorService, AutoCloseableAsync { @VisibleForTesting protected CompletableFuture getDispatcherGatewayFuture() { synchronized (lock) { - checkState(running, "MiniCluster is not yet running."); + checkState(running, "MiniCluster is not yet running or has already been shut down."); return dispatcherGatewayRetriever.getFuture(); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java index cd684fcc3b..77cef24304 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java @@ -1752,7 +1752,7 @@ public class StreamExecutionEnvironment { "Cannot find compatible factory for specified execution.target (=%s)", configuration.get(DeploymentOptions.TARGET)); - CompletableFuture jobClientFuture = executorFactory + CompletableFuture jobClientFuture = executorFactory .getExecutor(configuration) .execute(streamGraph, configuration); diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ProgramDeployer.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ProgramDeployer.java index 0070b1f852..be86a7a358 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ProgramDeployer.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ProgramDeployer.java @@ -58,7 +58,7 @@ public class ProgramDeployer { this.jobName = jobName; } - public CompletableFuture deploy() { + public CompletableFuture deploy() { LOG.info("Submitting job {} for query {}`", pipeline, jobName); if (LOG.isDebugEnabled()) { LOG.debug("Submitting job {} with configuration: \n{}", pipeline, configuration); @@ -77,7 +77,7 @@ public class ProgramDeployer { } final PipelineExecutor executor = executorFactory.getExecutor(configuration); - CompletableFuture jobClient; + CompletableFuture jobClient; try { jobClient = executor.execute(pipeline, configuration); } catch (Exception e) { diff --git a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/core/testutils/CommonTestUtils.java b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/core/testutils/CommonTestUtils.java index aaed2b118a..2eb45065c7 100644 --- a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/core/testutils/CommonTestUtils.java +++ b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/core/testutils/CommonTestUtils.java @@ -18,6 +18,8 @@ package org.apache.flink.core.testutils; +import org.junit.Assert; + import java.io.BufferedWriter; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -28,6 +30,11 @@ import java.io.ObjectInputStream; import java.io.ObjectOutputStream; import java.lang.reflect.Field; import java.util.Map; +import java.util.concurrent.Callable; + +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.MatcherAssert.assertThat; /** * This class contains reusable utility methods for unit tests. @@ -161,4 +168,17 @@ public class CommonTestUtils { return false; } + + /** + * Checks whether an exception with a message occurs when running a piece of code. + */ + public static void assertThrows(String msg, Class expected, Callable code) { + try { + Object result = code.call(); + Assert.fail("Previous method call should have failed but it returned: " + result); + } catch (Exception e) { + assertThat(e, instanceOf(expected)); + assertThat(e.getMessage(), containsString(msg)); + } + } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/client/LocalExecutorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/client/LocalExecutorITCase.java index b0ee3ff959..f13a2433fa 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/example/client/LocalExecutorITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/example/client/LocalExecutorITCase.java @@ -22,20 +22,28 @@ package org.apache.flink.test.example.client; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.Plan; import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.io.DiscardingOutputFormat; import org.apache.flink.client.deployment.executors.LocalExecutor; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.configuration.DeploymentOptions; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.minicluster.MiniCluster; import org.apache.flink.test.testdata.WordCountData; import org.apache.flink.test.testfunctions.Tokenizer; import org.apache.flink.util.TestLogger; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; import java.io.File; import java.io.FileWriter; +import static org.apache.flink.core.testutils.CommonTestUtils.assertThrows; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; + /** * Integration tests for {@link LocalExecutor}. */ @@ -43,8 +51,19 @@ public class LocalExecutorITCase extends TestLogger { private static final int parallelism = 4; - @Test - public void testLocalExecutorWithWordCount() { + private MiniCluster miniCluster; + private LocalExecutor executor; + + @Before + public void before() { + executor = LocalExecutor.createWithFactory(new Configuration(), config -> { + miniCluster = new MiniCluster(config); + return miniCluster; + }); + } + + @Test(timeout = 60_000) + public void testLocalExecutorWithWordCount() throws InterruptedException { try { // set up the files File inFile = File.createTempFile("wctext", ".in"); @@ -60,15 +79,34 @@ public class LocalExecutorITCase extends TestLogger { config.setBoolean(CoreOptions.FILESYTEM_DEFAULT_OVERRIDE, true); config.setBoolean(DeploymentOptions.ATTACHED, true); - final LocalExecutor executor = new LocalExecutor(); - Plan wcPlan = getWordCountPlan(inFile, outFile, parallelism); wcPlan.setExecutionConfig(new ExecutionConfig()); - executor.execute(wcPlan, config); + JobClient jobClient = executor.execute(wcPlan, config).get(); + jobClient.getJobExecutionResult(getClass().getClassLoader()).get(); } catch (Exception e) { e.printStackTrace(); Assert.fail(e.getMessage()); } + + assertThat(miniCluster.isRunning(), is(false)); + } + + @Test(timeout = 60_000) + public void testMiniClusterShutdownOnErrors() throws Exception { + Plan runtimeExceptionPlan = getRuntimeExceptionPlan(); + runtimeExceptionPlan.setExecutionConfig(new ExecutionConfig()); + + Configuration config = new Configuration(); + config.setBoolean(DeploymentOptions.ATTACHED, true); + + JobClient jobClient = executor.execute(runtimeExceptionPlan, config).get(); + + assertThrows( + "Job execution failed.", + Exception.class, + () -> jobClient.getJobExecutionResult(getClass().getClassLoader()).get()); + + assertThat(miniCluster.isRunning(), is(false)); } private Plan getWordCountPlan(File inFile, File outFile, int parallelism) { @@ -81,4 +119,17 @@ public class LocalExecutorITCase extends TestLogger { .writeAsCsv(outFile.getAbsolutePath()); return env.createProgramPlan(); } + + private Plan getRuntimeExceptionPlan() { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.fromElements(1) + .map(element -> { + if (element == 1) { + throw new RuntimeException("oups"); + } + return element; + }) + .output(new DiscardingOutputFormat<>()); + return env.createProgramPlan(); + } } -- Gitee From 7ea3f6f9022d6da40ebfb5b33b605dbd43d65739 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 2 Apr 2020 18:27:00 +0200 Subject: [PATCH 436/885] [hotfix] Add resourceManagerGateway null check to SlotPoolImpl SlotPoolImpl's resourceManagerGateway field can be null if JM has not connected to the ResourceManager yet. SlotPoolImpl's methods should respect this contract. --- .../jobmaster/slotpool/SlotPoolImpl.java | 27 ++++++++++--------- 1 file changed, 15 insertions(+), 12 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolImpl.java index 03ae4a800f..d9be79681b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolImpl.java @@ -127,6 +127,7 @@ public class SlotPoolImpl implements SlotPool { private JobMasterId jobMasterId; /** The gateway to communicate with resource manager. */ + @Nullable private ResourceManagerGateway resourceManagerGateway; private String jobManagerAddress; @@ -227,13 +228,7 @@ public class SlotPoolImpl implements SlotPool { log.info("Suspending SlotPool."); - // cancel all pending allocations --> we can request these slots - // again after we regained the leadership - Set allocationIds = pendingRequests.keySetB(); - - for (AllocationID allocationId : allocationIds) { - resourceManagerGateway.cancelSlotRequest(allocationId); - } + cancelPendingSlotRequests(); // do not accept any requests jobMasterId = null; @@ -244,15 +239,23 @@ public class SlotPoolImpl implements SlotPool { clear(); } + private void cancelPendingSlotRequests() { + if (resourceManagerGateway != null) { + // cancel all pending allocations --> we can request these slots + // again after we regained the leadership + Set allocationIds = pendingRequests.keySetB(); + + for (AllocationID allocationId : allocationIds) { + resourceManagerGateway.cancelSlotRequest(allocationId); + } + } + } + @Override public void close() { log.info("Stopping SlotPool."); - // cancel all pending allocations - Set allocationIds = pendingRequests.keySetB(); - for (AllocationID allocationId : allocationIds) { - resourceManagerGateway.cancelSlotRequest(allocationId); - } + cancelPendingSlotRequests(); // release all registered slots by releasing the corresponding TaskExecutors for (ResourceID taskManagerResourceId : registeredTaskManagers) { -- Gitee From 10aadfc6906a1629f7e60eacf087e351ba40d517 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 1 Apr 2020 14:45:23 +0200 Subject: [PATCH 437/885] [FLINK-6258] Deprecate ListCheckpointed interface This also updates the documentation and changes usage in our code to CheckpointedFunction. The Google PubSub source unfortunately also uses ListCheckpointed but here we cannot migrate to CheckpointedFunction because of savepoint compatibility. --- docs/dev/stream/state/state.md | 63 ++++++++++--------- .../tests/BucketingSinkTestProgram.java | 29 ++++++--- .../streaming/tests/PeriodicStreamingJob.java | 30 +++++---- .../flink/sql/tests/StreamSQLTestProgram.java | 47 +++++++++----- .../main/java/StreamingFileSinkProgram.java | 29 ++++++--- .../examples/async/AsyncIOExample.java | 31 ++++++--- .../api/checkpoint/ListCheckpointed.java | 4 ++ 7 files changed, 147 insertions(+), 86 deletions(-) diff --git a/docs/dev/stream/state/state.md b/docs/dev/stream/state/state.md index e9869e738a..93d0ec0d13 100644 --- a/docs/dev/stream/state/state.md +++ b/docs/dev/stream/state/state.md @@ -533,8 +533,8 @@ in that: ## Using Operator State -To use operator state, a stateful function can implement either the more general `CheckpointedFunction` -interface, or the `ListCheckpointed` interface. +To use operator state, a stateful function can implement the `CheckpointedFunction` +interface. #### CheckpointedFunction @@ -729,22 +729,6 @@ of all objects included by the previous checkpoint, and is then filled with the As a side note, the keyed state can also be initialized in the `initializeState()` method. This can be done using the provided `FunctionInitializationContext`. -#### ListCheckpointed - -The `ListCheckpointed` interface is a more limited variant of `CheckpointedFunction`, -which only supports list-style state with even-split redistribution scheme on restore. -It also requires the implementation of two methods: - -{% highlight java %} -List snapshotState(long checkpointId, long timestamp) throws Exception; - -void restoreState(List state) throws Exception; -{% endhighlight %} - -On `snapshotState()` the operator should return a list of objects to checkpoint and -`restoreState` has to handle such a list upon recovery. If the state is not re-partitionable, you can always -return a `Collections.singletonList(MY_STATE)` in the `snapshotState()`. - ### Stateful Source Functions Stateful sources require a bit more care as opposed to other operators. @@ -756,13 +740,16 @@ on failure/recovery), the user is required to get a lock from the source's conte {% highlight java %} public static class CounterSource extends RichParallelSourceFunction - implements ListCheckpointed { + implements CheckpointedFunction { /** current offset for exactly once semantics */ private Long offset = 0L; /** flag for job cancellation */ private volatile boolean isRunning = true; + + /** Our state object. */ + private ListState state; @Override public void run(SourceContext ctx) { @@ -783,14 +770,22 @@ public static class CounterSource } @Override - public List snapshotState(long checkpointId, long checkpointTimestamp) { - return Collections.singletonList(offset); + public void initializeState(FunctionInitializationContext context) throws Exception { + state = context.getOperatorStateStore().getListState(new ListStateDescriptor<>( + "state", + LongSerializer.INSTANCE)); + + // restore any state that we might already have to our fields, initialize state + // is also called in case of restore. + for (Long l : state.get()) { + offset = l; + } } @Override - public void restoreState(List state) { - for (Long s : state) - offset = s; + public void snapshotState(FunctionSnapshotContext context) throws Exception { + state.clear(); + state.add(offset); } } {% endhighlight %} @@ -800,12 +795,13 @@ public static class CounterSource {% highlight scala %} class CounterSource extends RichParallelSourceFunction[Long] - with ListCheckpointed[Long] { + with CheckpointedFunction { @volatile private var isRunning = true private var offset = 0L + private var state: ListState[Long] = _ override def run(ctx: SourceFunction.SourceContext[Long]): Unit = { val lock = ctx.getCheckpointLock @@ -821,15 +817,20 @@ class CounterSource } override def cancel(): Unit = isRunning = false + + override def initializeState(context: FunctionInitializationContext): Unit = { + state = context.getOperatorStateStore.getListState( + new ListStateDescriptor[Long]("state", classOf[Long])) - override def restoreState(state: util.List[Long]): Unit = - for (s <- state) { - offset = s + for (l <- state.get().asScala) { + offset = l } + } - override def snapshotState(checkpointId: Long, timestamp: Long): util.List[Long] = - Collections.singletonList(offset) - + override def snapshotState(context: FunctionSnapshotContext): Unit = { + state.clear() + state.add(offset) + } } {% endhighlight %} diff --git a/flink-end-to-end-tests/flink-bucketing-sink-test/src/main/java/org/apache/flink/streaming/tests/BucketingSinkTestProgram.java b/flink-end-to-end-tests/flink-bucketing-sink-test/src/main/java/org/apache/flink/streaming/tests/BucketingSinkTestProgram.java index 2409b5e554..d5d348b09c 100644 --- a/flink-end-to-end-tests/flink-bucketing-sink-test/src/main/java/org/apache/flink/streaming/tests/BucketingSinkTestProgram.java +++ b/flink-end-to-end-tests/flink-bucketing-sink-test/src/main/java/org/apache/flink/streaming/tests/BucketingSinkTestProgram.java @@ -20,15 +20,20 @@ package org.apache.flink.streaming.tests; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.time.Time; import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.common.typeutils.base.LongSerializer; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple4; import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.checkpoint.ListCheckpointed; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.connectors.fs.Clock; @@ -38,8 +43,6 @@ import org.apache.flink.streaming.connectors.fs.bucketing.BucketingSink; import org.apache.hadoop.fs.Path; import java.io.IOException; -import java.util.Collections; -import java.util.List; import java.util.concurrent.TimeUnit; /** @@ -140,7 +143,7 @@ public class BucketingSinkTestProgram { /** * Data-generating source function. */ - public static class Generator implements SourceFunction>, ListCheckpointed { + public static class Generator implements SourceFunction>, CheckpointedFunction { private final int numKeys; private final int idlenessMs; @@ -149,6 +152,8 @@ public class BucketingSinkTestProgram { private long ms = 0; private volatile boolean canceled = false; + private ListState state = null; + public Generator(int numKeys, int idlenessMs, int durationSeconds) { this.numKeys = numKeys; this.idlenessMs = idlenessMs; @@ -178,15 +183,19 @@ public class BucketingSinkTestProgram { } @Override - public List snapshotState(long checkpointId, long timestamp) { - return Collections.singletonList(ms); - } + public void initializeState(FunctionInitializationContext context) throws Exception { + state = context.getOperatorStateStore().getListState( + new ListStateDescriptor("state", LongSerializer.INSTANCE)); - @Override - public void restoreState(List state) { - for (Long l : state) { + for (Long l : state.get()) { ms += l; } } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + state.clear(); + state.add(ms); + } } } diff --git a/flink-end-to-end-tests/flink-cli-test/src/main/java/org/apache/flink/streaming/tests/PeriodicStreamingJob.java b/flink-end-to-end-tests/flink-cli-test/src/main/java/org/apache/flink/streaming/tests/PeriodicStreamingJob.java index 7cabf849aa..7784b4384b 100644 --- a/flink-end-to-end-tests/flink-cli-test/src/main/java/org/apache/flink/streaming/tests/PeriodicStreamingJob.java +++ b/flink-end-to-end-tests/flink-cli-test/src/main/java/org/apache/flink/streaming/tests/PeriodicStreamingJob.java @@ -18,23 +18,25 @@ package org.apache.flink.streaming.tests; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.common.typeutils.base.LongSerializer; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; import org.apache.flink.streaming.api.TimeCharacteristic; -import org.apache.flink.streaming.api.checkpoint.ListCheckpointed; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.api.windowing.time.Time; -import java.util.Collections; -import java.util.List; - /** * This is a periodic streaming job that runs for CLI testing purposes. * @@ -79,12 +81,14 @@ public class PeriodicStreamingJob { /** * Data-generating source function. */ - public static class PeriodicSourceGenerator implements SourceFunction, ResultTypeQueryable, ListCheckpointed { + public static class PeriodicSourceGenerator implements SourceFunction, ResultTypeQueryable, CheckpointedFunction { private final int sleepMs; private final int durationMs; private final int offsetSeconds; private long ms = 0; + private ListState state = null; + public PeriodicSourceGenerator(float rowsPerSecond, int durationSeconds, int offsetSeconds) { this.durationMs = durationSeconds * 1000; this.sleepMs = (int) (1000 / rowsPerSecond); @@ -115,15 +119,19 @@ public class PeriodicStreamingJob { } @Override - public List snapshotState(long checkpointId, long timestamp) { - return Collections.singletonList(ms); - } + public void initializeState(FunctionInitializationContext context) throws Exception { + state = context.getOperatorStateStore().getListState( + new ListStateDescriptor("state", LongSerializer.INSTANCE)); - @Override - public void restoreState(List state) { - for (Long l : state) { + for (Long l : state.get()) { ms += l; } } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + state.clear(); + state.add(ms); + } } } diff --git a/flink-end-to-end-tests/flink-stream-sql-test/src/main/java/org/apache/flink/sql/tests/StreamSQLTestProgram.java b/flink-end-to-end-tests/flink-stream-sql-test/src/main/java/org/apache/flink/sql/tests/StreamSQLTestProgram.java index d7bea9bd3c..40a4005c53 100644 --- a/flink-end-to-end-tests/flink-stream-sql-test/src/main/java/org/apache/flink/sql/tests/StreamSQLTestProgram.java +++ b/flink-end-to-end-tests/flink-stream-sql-test/src/main/java/org/apache/flink/sql/tests/StreamSQLTestProgram.java @@ -21,15 +21,21 @@ package org.apache.flink.sql.tests; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.serialization.Encoder; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.time.Time; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.core.fs.Path; import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; import org.apache.flink.streaming.api.TimeCharacteristic; -import org.apache.flink.streaming.api.checkpoint.ListCheckpointed; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.filesystem.BucketAssigner; @@ -255,7 +261,7 @@ public class StreamSQLTestProgram { /** * Data-generating source function. */ - public static class Generator implements SourceFunction, ResultTypeQueryable, ListCheckpointed { + public static class Generator implements SourceFunction, ResultTypeQueryable, CheckpointedFunction { private final int numKeys; private final int offsetSeconds; @@ -264,6 +270,7 @@ public class StreamSQLTestProgram { private final int durationMs; private long ms = 0; + private ListState state = null; public Generator(int numKeys, float rowsPerKeyAndSecond, int durationSeconds, int offsetSeconds) { this.numKeys = numKeys; @@ -297,28 +304,34 @@ public class StreamSQLTestProgram { } @Override - public List snapshotState(long checkpointId, long timestamp) { - return Collections.singletonList(ms); - } + public void initializeState(FunctionInitializationContext context) throws Exception { + state = context.getOperatorStateStore().getListState( + new ListStateDescriptor("state", LongSerializer.INSTANCE)); - @Override - public void restoreState(List state) { - for (Long l : state) { + for (Long l : state.get()) { ms += l; } } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + state.clear(); + state.add(ms); + } } /** * Kills the first execution attempt of an application when it receives the second record. */ - public static class KillMapper implements MapFunction, ListCheckpointed, ResultTypeQueryable { + public static class KillMapper implements MapFunction, CheckpointedFunction, ResultTypeQueryable { // counts all processed records of all previous execution attempts private int saveRecordCnt = 0; // counts all processed records of this execution attempt private int lostRecordCnt = 0; + private ListState state = null; + @Override public Row map(Row value) { @@ -342,15 +355,19 @@ public class StreamSQLTestProgram { } @Override - public List snapshotState(long checkpointId, long timestamp) { - return Collections.singletonList(saveRecordCnt); - } + public void initializeState(FunctionInitializationContext context) throws Exception { + state = context.getOperatorStateStore().getListState( + new ListStateDescriptor("state", IntSerializer.INSTANCE)); - @Override - public void restoreState(List state) { - for (Integer i : state) { + for (Integer i : state.get()) { saveRecordCnt += i; } } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + state.clear(); + state.add(saveRecordCnt); + } } } diff --git a/flink-end-to-end-tests/flink-streaming-file-sink-test/src/main/java/StreamingFileSinkProgram.java b/flink-end-to-end-tests/flink-streaming-file-sink-test/src/main/java/StreamingFileSinkProgram.java index ac623acc40..c9cf6d78d7 100644 --- a/flink-end-to-end-tests/flink-streaming-file-sink-test/src/main/java/StreamingFileSinkProgram.java +++ b/flink-end-to-end-tests/flink-streaming-file-sink-test/src/main/java/StreamingFileSinkProgram.java @@ -18,12 +18,17 @@ import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.serialization.Encoder; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.common.typeutils.base.IntSerializer; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.core.fs.Path; import org.apache.flink.core.io.SimpleVersionedSerializer; -import org.apache.flink.streaming.api.checkpoint.ListCheckpointed; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.filesystem.BucketAssigner; import org.apache.flink.streaming.api.functions.sink.filesystem.StreamingFileSink; @@ -32,8 +37,6 @@ import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies. import org.apache.flink.streaming.api.functions.source.SourceFunction; import java.io.PrintStream; -import java.util.Collections; -import java.util.List; import java.util.concurrent.TimeUnit; /** @@ -99,7 +102,7 @@ public enum StreamingFileSinkProgram { /** * Data-generating source function. */ - public static final class Generator implements SourceFunction>, ListCheckpointed { + public static final class Generator implements SourceFunction>, CheckpointedFunction { private static final long serialVersionUID = -2819385275681175792L; @@ -110,6 +113,8 @@ public enum StreamingFileSinkProgram { private volatile int numRecordsEmitted = 0; private volatile boolean canceled = false; + private ListState state = null; + Generator(final int numKeys, final int idlenessMs, final int durationSeconds) { this.numKeys = numKeys; this.idlenessMs = idlenessMs; @@ -141,15 +146,19 @@ public enum StreamingFileSinkProgram { } @Override - public List snapshotState(final long checkpointId, final long timestamp) { - return Collections.singletonList(numRecordsEmitted); + public void initializeState(FunctionInitializationContext context) throws Exception { + state = context.getOperatorStateStore().getListState( + new ListStateDescriptor("state", IntSerializer.INSTANCE)); + + for (Integer i : state.get()) { + numRecordsEmitted += i; + } } @Override - public void restoreState(final List states) { - for (final Integer state : states) { - numRecordsEmitted += state; - } + public void snapshotState(FunctionSnapshotContext context) throws Exception { + state.clear(); + state.add(numRecordsEmitted); } } } diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java index 97de5fc390..1e22606118 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/async/AsyncIOExample.java @@ -18,13 +18,18 @@ package org.apache.flink.streaming.examples.async; import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeutils.base.IntSerializer; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; import org.apache.flink.runtime.state.filesystem.FsStateBackend; import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.TimeCharacteristic; -import org.apache.flink.streaming.api.checkpoint.ListCheckpointed; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; import org.apache.flink.streaming.api.datastream.AsyncDataStream; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -40,7 +45,6 @@ import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.Collections; -import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ThreadLocalRandom; @@ -61,23 +65,32 @@ public class AsyncIOExample { /** * A checkpointed source. */ - private static class SimpleSource implements SourceFunction, ListCheckpointed { + private static class SimpleSource implements SourceFunction, CheckpointedFunction { private static final long serialVersionUID = 1L; private volatile boolean isRunning = true; private int counter = 0; private int start = 0; + private ListState state; + @Override - public List snapshotState(long checkpointId, long timestamp) throws Exception { - return Collections.singletonList(start); + public void initializeState(FunctionInitializationContext context) throws Exception { + state = context.getOperatorStateStore().getListState(new ListStateDescriptor<>( + "state", + IntSerializer.INSTANCE)); + + // restore any state that we might already have to our fields, initialize state + // is also called in case of restore. + for (Integer i : state.get()) { + start = i; + } } @Override - public void restoreState(List state) throws Exception { - for (Integer i : state) { - this.start = i; - } + public void snapshotState(FunctionSnapshotContext context) throws Exception { + state.clear(); + state.add(start); } public SimpleSource(int maxNum) { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/ListCheckpointed.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/ListCheckpointed.java index f4cc794f31..da43224521 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/ListCheckpointed.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/ListCheckpointed.java @@ -105,9 +105,13 @@ import java.util.List; * } * }

    * + * @deprecated If you need to do non-keyed state snapshots of your operator, use {@link + * CheckpointedFunction}. This should only be needed in rare cases, though. + * * @param The type of the operator state. */ @PublicEvolving +@Deprecated public interface ListCheckpointed { /** -- Gitee From e25f0d3c322982291005369da8a5a7cf62d4e59a Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 13 Feb 2020 14:26:54 +0100 Subject: [PATCH 438/885] [FLINK-16044][doc] Extract libraries documentation to a top-level section --- docs/dev/libraries.md | 5 +++-- docs/dev/libraries.zh.md | 5 +++-- docs/flinkDev/index.md | 2 +- docs/flinkDev/index.zh.md | 2 +- docs/internals/index.md | 2 +- docs/internals/index.zh.md | 2 +- docs/monitoring/index.md | 2 +- docs/monitoring/index.zh.md | 2 +- docs/ops/index.md | 2 +- docs/ops/index.zh.md | 2 +- 10 files changed, 14 insertions(+), 12 deletions(-) diff --git a/docs/dev/libraries.md b/docs/dev/libraries.md index 586637b119..673181a4d2 100644 --- a/docs/dev/libraries.md +++ b/docs/dev/libraries.md @@ -1,8 +1,9 @@ --- title: "Libraries" nav-id: libs -nav-parent_id: dev -nav-pos: 80 +nav-title: ' Libraries' +nav-parent_id: root +nav-pos: 6 --- + +* toc +{:toc} diff --git a/docs/connectors/index.zh.md b/docs/connectors/index.zh.md new file mode 100644 index 0000000000..5a744166b3 --- /dev/null +++ b/docs/connectors/index.zh.md @@ -0,0 +1,28 @@ +--- +title: "Connectors" +nav-id: connectors-root +nav-title: ' Connectors' +nav-parent_id: root +nav-pos: 7 +--- + + +* toc +{:toc} diff --git a/docs/dev/batch/connectors.md b/docs/dev/batch/connectors.md index 6a6fd79f04..a66ba9d495 100644 --- a/docs/dev/batch/connectors.md +++ b/docs/dev/batch/connectors.md @@ -1,7 +1,7 @@ --- -title: "Connectors" -nav-parent_id: batch -nav-pos: 4 +title: "DataSet Connectors" +nav-parent_id: connectors-root +nav-pos: 10 --- -[Apache Hive](https://hive.apache.org/) has established itself as a focal point of the data warehousing ecosystem. -It serves as not only a SQL engine for big data analytics and ETL, but also a data management platform, where data is discovered, defined, and evolved. +[Apache Hive](https://hive.apache.org/) 已经成为了数据仓库生态系统中的核心。 +它不仅仅是一个用于大数据分析和ETL场景的SQL引擎,同样它也是一个数据管理平台,可用于发现,定义,和演化数据。 -Flink offers a two-fold integration with Hive. +Flink 与 Hive 的集成包含两个层面。 -The first is to leverage Hive's Metastore as a persistent catalog with Flink's `HiveCatalog` for storing Flink specific metadata across sessions. -For example, users can store their Kafka or ElasticSearch tables in Hive Metastore by using `HiveCatalog`, and reuse them later on in SQL queries. +一是利用了 Hive 的 MetaStore 作为持久化的 Catalog,用户可通过`HiveCatalog`将不同会话中的 Flink 元数据存储到 Hive Metastore 中。 +例如,用户可以使用`HiveCatalog`将其 Kafka 表或 Elasticsearch 表存储在 Hive Metastore 中,并后续在 SQL 查询中重新使用它们。 -The second is to offer Flink as an alternative engine for reading and writing Hive tables. +二是利用 Flink 来读写 Hive 的表。 -The `HiveCatalog` is designed to be “out of the box” compatible with existing Hive installations. -You do not need to modify your existing Hive Metastore or change the data placement or partitioning of your tables. +`HiveCatalog`的设计提供了与 Hive 良好的兼容性,用户可以"开箱即用"的访问其已有的 Hive 数仓。 +您不需要修改现有的 Hive Metastore,也不需要更改表的数据位置或分区。 -* Note that we highly recommend users using the [blink planner]({{ site.baseurl }}/dev/table/#dependency-structure) with Hive integration. +* 我们强烈建议用户使用 [Blink planner]({{ site.baseurl }}/zh/dev/table/#dependency-structure) 与 Hive 集成。 * This will be replaced by the TOC {:toc} -## Supported Hive Versions +## 支持的Hive版本 -Flink supports the following Hive versions. +Flink 支持一下的 Hive 版本。 - 1.0 - 1.0.0 @@ -77,33 +77,33 @@ Flink supports the following Hive versions. - 3.1.1 - 3.1.2 -Please note Hive itself have different features available for different versions, and these issues are not caused by Flink: +请注意,某些功能是否可用取决于您使用的 Hive 版本,这些限制不是由 Flink 所引起的: -- Hive built-in functions are supported in 1.2.0 and later. -- Column constraints, i.e. PRIMARY KEY and NOT NULL, are supported in 3.1.0 and later. -- Altering table statistics is supported in 1.2.0 and later. -- `DATE` column statistics are supported in 1.2.0 and later. -- Writing to ORC tables is not supported in 2.0.x. +- Hive 内置函数在使用 Hive-1.2.0 及更高版本时支持。 +- 列约束,也就是 PRIMARY KEY 和 NOT NULL,在使用 Hive-3.1.0 及更高版本时支持。 +- 更改表的统计信息,在使用 Hive-1.2.0 及更高版本时支持。 +- `DATE`列统计信息,在使用 Hive-1.2.0 及更高版时支持。 +- 使用 Hive-2.0.x 版本时不支持写入 ORC 表。 -### Dependencies +### 依赖项 -To integrate with Hive, you need to add some extra dependencies to the `/lib/` directory in Flink distribution -to make the integration work in Table API program or SQL in SQL Client. -Alternatively, you can put these dependencies in a dedicated folder, and add them to classpath with the `-C` -or `-l` option for Table API program or SQL Client respectively. +要与 Hive 集成,您需要在 Flink 下的`/lib/`目录中添加一些额外的依赖包, +以便通过 Table API 或 SQL Client 与 Hive 进行交互。 +或者,您可以将这些依赖项放在专用文件夹中,并分别使用 Table API 程序或 SQL Client 的`-C`或`-l`选项将它们添加到 classpath 中。 -Apache Hive is built on Hadoop, so you need Hadoop dependency first, please refer to -[Providing Hadoop classes]({{ site.baseurl }}/ops/deployment/hadoop.html#providing-hadoop-classes). +Apache Hive 是基于 Hadoop 之上构建的, 首先您需要 Hadoop 的依赖,请参考 +[Providing Hadoop classes]({{ site.baseurl }}/zh/ops/deployment/hadoop.html#providing-hadoop-classes). -There are two ways to add Hive dependencies. First is to use Flink's bundled Hive jars. You can choose a bundled Hive jar according to the version of the metastore you use. Second is to add each of the required jars separately. The second way can be useful if the Hive version you're using is not listed here. +有两种添加 Hive 依赖项的方法。第一种是使用 Flink 提供的 Hive Jar包。您可以根据使用的 Metastore 的版本来选择对应的 Hive jar。第二个方式是分别添加每个所需的 jar 包。如果您使用的 Hive 版本尚未在此处列出,则第二种方法会更适合。 -#### Using bundled hive jar +#### 使用 Flink 提供的 Hive jar + +下表列出了所有可用的 Hive jar。您可以选择一个并放在 Flink 发行版的`/lib/` 目录中。 -The following tables list all available bundled hive jars. You can pick one to the `/lib/` directory in Flink distribution. {% if site.is_stable %} -| Metastore version | Maven dependency | SQL Client JAR | +| Metastore 版本 | Maven 依赖 | SQL Client JAR | | :---------------- | :--------------------------- | :----------------------| | 1.0.0 - 1.2.2 | `flink-connector-hive-1.2.2` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-sql-connector-hive-1.2.2{{site.scala_version_suffix}}/{{site.version}}/flink-sql-connector-hive-1.2.2{{site.scala_version_suffix}}-{{site.version}}.jar) | | 2.0.0 - 2.2.0 | `flink-connector-hive-2.2.0` | [Download](http://central.maven.org/maven2/org/apache/flink/flink-sql-connector-hive-2.2.0{{site.scala_version_suffix}}/{{site.version}}/flink-sql-connector-hive-2.2.0{{site.scala_version_suffix}}-{{site.version}}.jar) | @@ -112,13 +112,13 @@ The following tables list all available bundled hive jars. You can pick one to t {% else %} -These tables are only available for stable releases. +下表仅适用与稳定版本。 {% endif %} -#### User defined dependencies +#### 用户定义的依赖项 -Please find the required dependencies for different Hive major versions below. +您可以在下方找到不同Hive主版本所需要的依赖项。
    @@ -259,15 +259,14 @@ Please find the required dependencies for different Hive major versions below.
    -If you use the hive version of HDP or CDH, you need to refer to the dependency in the previous section and select a similar version. +如果使用 Hive 的 HDP 或 CDH 版本,则需要参考上一节中的依赖项并选择一个类似的版本。 -And you need to specify selected and supported "hive-version" in yaml, HiveCatalog and HiveModule. +并且您需要在定义 yaml 文件,或者创建 HiveCatalog 和 HiveModule 时,指定一个支持的 “hive-version”。 -#### Program maven +### Maven 依赖 -If you are building your own program, you need the following dependencies in your mvn file. -It's recommended not to include these dependencies in the resulting jar file. -You're supposed to add dependencies as stated above at runtime. +如果您在构建自己的应用程序,则需要在 mvn 文件中添加以下依赖项。 +您应该在运行时添加以上的这些依赖项,而不要在已生成的 jar 文件中去包含它们。 {% highlight xml %} @@ -294,18 +293,15 @@ You're supposed to add dependencies as stated above at runtime. {% endhighlight %} -## Connecting To Hive +## 连接到Hive -Connect to an existing Hive installation using the [catalog interface]({{ site.baseurl }}/dev/table/catalogs.html) -and [HiveCatalog]({{ site.baseurl }}/dev/table/hive/hive_catalog.html) through the table environment or YAML configuration. +通过 TableEnvironment 或者 YAML 配置,使用 [Catalog 接口]({{ site.baseurl }}/zh/dev/table/catalogs.html) 和 [HiveCatalog]({{ site.baseurl }}/zh/dev/table/hive/hive_catalog.html)连接到现有的 Hive 集群。 -If the `hive-conf/hive-site.xml` file is stored in remote storage system, users should download -the hive configuration file to their local environment first. +如果`hive-conf/hive-site.xml`文件存储在远端存储系统,则用户首先应该将hive配置文件下载至其本地环境中。 -Please note while HiveCatalog doesn't require a particular planner, reading/writing Hive tables only works with blink planner. -Therefore it's highly recommended that you use blink planner when connecting to your Hive warehouse. +请注意,虽然 HiveCatalog 不需要特定的 planner,但读写Hive表仅适用于 Blink planner。因此,强烈建议您在连接到 Hive 仓库时使用 Blink planner。 -Take Hive version 2.3.4 for example: +以Hive 2.3.4版本为例:
    @@ -364,8 +360,8 @@ catalogs: ## DDL -DDL to create Hive tables, views, partitions, functions within Flink will be supported soon. +即将支持在 Flink 中创建 Hive 表,视图,分区和函数的DDL。 ## DML -Flink supports DML writing to Hive tables. Please refer to details in [Reading & Writing Hive Tables]({{ site.baseurl }}/dev/table/hive/read_write_hive.html) +Flink 支持 DML 写入 Hive 表,请参考[读写 Hive 表]({{ site.baseurl }}/zh/dev/table/hive/read_write_hive.html) -- Gitee From bb46756b84940a6134910e74406bfaff4f2f37e9 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Thu, 2 Apr 2020 17:01:19 +0200 Subject: [PATCH 444/885] [FLINK-16913] Migrate StateBackends to use ReadableConfig instead of Configuration StateBackendFactories do not need a full read and write access to the Configuration object. It's sufficient to have read only access. Moreover the ReadableConfig is a lightweight interface that can be implemented in other ways, not just through the Configuration. Lastly we exposed this lightweight interface as a configuration entry point for ExecutionEnvironments. This change will make it possible to pass the ReadableConfig directly to the StateBackendFactories without fragile adapters. --- .../generated/rocks_db_configuration.html | 4 +- .../state_backend_rocksdb_section.html | 4 +- .../ReadableConfigToConfigurationAdapter.java | 328 ------------------ .../state/ConfigurableStateBackend.java | 4 +- .../runtime/state/StateBackendFactory.java | 4 +- .../runtime/state/StateBackendLoader.java | 11 +- .../filesystem/AbstractFileStateBackend.java | 8 +- .../state/filesystem/FsStateBackend.java | 14 +- .../filesystem/FsStateBackendFactory.java | 6 +- .../state/memory/MemoryStateBackend.java | 10 +- .../memory/MemoryStateBackendFactory.java | 4 +- .../state/StateBackendLoadingTest.java | 3 +- .../state/testutils/BackendForTestStream.java | 4 +- .../state/ConfigurableOptionsFactory.java | 4 +- .../ConfigurableRocksDBOptionsFactory.java | 4 +- .../DefaultConfigurableOptionsFactory.java | 6 +- .../state/RocksDBMemoryConfiguration.java | 10 +- .../state/RocksDBNativeMetricOptions.java | 56 +-- .../streaming/state/RocksDBOptions.java | 6 +- .../state/RocksDBOptionsFactoryAdapter.java | 4 +- .../streaming/state/RocksDBStateBackend.java | 25 +- .../state/RocksDBStateBackendFactory.java | 6 +- .../state/RocksDBAsyncSnapshotTest.java | 2 +- ...ocksDBOptionsFactoryCompatibilityTest.java | 3 +- .../state/RocksDBStateBackendConfigTest.java | 24 +- .../RocksDBStateBackendMigrationTest.java | 4 +- .../state/RocksDBStateBackendTest.java | 4 +- .../StreamExecutionEnvironment.java | 3 +- .../runtime/tasks/StreamTaskTest.java | 5 +- .../tasks/TaskCheckpointingBehaviourTest.java | 5 +- .../EventTimeWindowCheckpointingITCase.java | 4 +- 31 files changed, 126 insertions(+), 453 deletions(-) delete mode 100644 flink-core/src/main/java/org/apache/flink/configuration/ReadableConfigToConfigurationAdapter.java diff --git a/docs/_includes/generated/rocks_db_configuration.html b/docs/_includes/generated/rocks_db_configuration.html index 3ce04c75d2..1648ab3725 100644 --- a/docs/_includes/generated/rocks_db_configuration.html +++ b/docs/_includes/generated/rocks_db_configuration.html @@ -58,8 +58,8 @@
    - - + + diff --git a/docs/_includes/generated/state_backend_rocksdb_section.html b/docs/_includes/generated/state_backend_rocksdb_section.html index 974c5c12ab..a5c9e78e95 100644 --- a/docs/_includes/generated/state_backend_rocksdb_section.html +++ b/docs/_includes/generated/state_backend_rocksdb_section.html @@ -34,8 +34,8 @@ - - + + diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ReadableConfigToConfigurationAdapter.java b/flink-core/src/main/java/org/apache/flink/configuration/ReadableConfigToConfigurationAdapter.java deleted file mode 100644 index c2e9292f68..0000000000 --- a/flink-core/src/main/java/org/apache/flink/configuration/ReadableConfigToConfigurationAdapter.java +++ /dev/null @@ -1,328 +0,0 @@ -/* - * 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.configuration; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; - -import java.io.IOException; -import java.util.Map; -import java.util.Optional; -import java.util.Properties; -import java.util.Set; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * A simple adapter between {@link ReadableConfig} and {@link Configuration}. - * It is used to bridge some of the old public interfaces that work with {@link Configuration} even though they - * should actually work with {@link ReadableConfig}. - */ -@Internal -public class ReadableConfigToConfigurationAdapter extends Configuration { - private final ReadableConfig backingConfig; - - public ReadableConfigToConfigurationAdapter(ReadableConfig backingConfig) { - this.backingConfig = checkNotNull(backingConfig); - } - - @Override - public String getString(ConfigOption configOption) { - return backingConfig.get(configOption); - } - - @Override - public String getString(ConfigOption configOption, String overrideDefault) { - return backingConfig.getOptional(configOption).orElse(overrideDefault); - } - - @Override - public int getInteger(ConfigOption configOption) { - return backingConfig.get(configOption); - } - - @Override - public int getInteger(ConfigOption configOption, int overrideDefault) { - return backingConfig.getOptional(configOption).orElse(overrideDefault); - } - - @Override - public long getLong(ConfigOption configOption) { - return backingConfig.get(configOption); - } - - @Override - public long getLong(ConfigOption configOption, long overrideDefault) { - return backingConfig.getOptional(configOption).orElse(overrideDefault); - } - - @Override - public boolean getBoolean(ConfigOption configOption) { - return backingConfig.get(configOption); - } - - @Override - public boolean getBoolean(ConfigOption configOption, boolean overrideDefault) { - return backingConfig.getOptional(configOption).orElse(overrideDefault); - } - - @Override - public float getFloat(ConfigOption configOption) { - return backingConfig.get(configOption); - } - - @Override - public float getFloat(ConfigOption configOption, float overrideDefault) { - return backingConfig.getOptional(configOption).orElse(overrideDefault); - } - - @Override - public double getDouble(ConfigOption configOption) { - return backingConfig.get(configOption); - } - - @Override - public double getDouble(ConfigOption configOption, double overrideDefault) { - return backingConfig.getOptional(configOption).orElse(overrideDefault); - } - - @Override - public T get(ConfigOption option) { - return backingConfig.get(option); - } - - @Override - public Optional getOptional(ConfigOption option) { - return backingConfig.getOptional(option); - } - - @Override - public boolean contains(ConfigOption configOption) { - return this.backingConfig.getOptional(configOption).isPresent(); - } - - @Override - public int hashCode() { - return backingConfig.hashCode(); - } - - @Override - public boolean equals(Object obj) { - return backingConfig.equals(obj); - } - - @Override - public String toString() { - return backingConfig.toString(); - } - - /* - Modifying methods - */ - - @Override - public void setClass(String key, Class klazz) { - throw new UnsupportedOperationException("The configuration is read only"); - } - - @Override - public void setString(String key, String value) { - throw new UnsupportedOperationException("The configuration is read only"); - } - - @Override - public void setString(ConfigOption key, String value) { - throw new UnsupportedOperationException("The configuration is read only"); - } - - @Override - public void setInteger(String key, int value) { - throw new UnsupportedOperationException("The configuration is read only"); - } - - @Override - public void setInteger(ConfigOption key, int value) { - throw new UnsupportedOperationException("The configuration is read only"); - } - - @Override - public void setLong(String key, long value) { - throw new UnsupportedOperationException("The configuration is read only"); - } - - @Override - public void setLong(ConfigOption key, long value) { - throw new UnsupportedOperationException("The configuration is read only"); - } - - @Override - public void setBoolean(String key, boolean value) { - throw new UnsupportedOperationException("The configuration is read only"); - } - - @Override - public void setBoolean(ConfigOption key, boolean value) { - throw new UnsupportedOperationException("The configuration is read only"); - } - - @Override - public void setFloat(String key, float value) { - throw new UnsupportedOperationException("The configuration is read only"); - } - - @Override - public void setFloat(ConfigOption key, float value) { - throw new UnsupportedOperationException("The configuration is read only"); - } - - @Override - public void setDouble(String key, double value) { - throw new UnsupportedOperationException("The configuration is read only"); - } - - @Override - public void setDouble(ConfigOption key, double value) { - throw new UnsupportedOperationException("The configuration is read only"); - } - - @Override - public void setBytes(String key, byte[] bytes) { - throw new UnsupportedOperationException("The configuration is read only"); - } - - @Override - public void addAllToProperties(Properties props) { - throw new UnsupportedOperationException("The configuration is read only"); - } - - @Override - public void addAll(Configuration other) { - throw new UnsupportedOperationException("The configuration is read only"); - } - - @Override - public void addAll(Configuration other, String prefix) { - throw new UnsupportedOperationException("The configuration is read only"); - } - - @Override - public Configuration set(ConfigOption option, T value) { - throw new UnsupportedOperationException("The configuration is read only"); - } - - @Override - void setValueInternal(String key, T value) { - throw new UnsupportedOperationException("The configuration is read only"); - } - - @Override - public boolean removeConfig(ConfigOption configOption) { - throw new UnsupportedOperationException("The configuration is read only"); - } - - /* - * Other unsupported options. - */ - - @Override - public byte[] getBytes(String key, byte[] defaultValue) { - throw new UnsupportedOperationException("The adapter does not support this method"); - } - - @Override - public String getValue(ConfigOption configOption) { - throw new UnsupportedOperationException("The adapter does not support this method"); - } - - @Override - public > T getEnum( - Class enumClass, - ConfigOption configOption) { - throw new UnsupportedOperationException("The adapter does not support this method"); - } - - @Override - public Configuration clone() { - throw new UnsupportedOperationException("The adapter does not support this method"); - } - - @Override - public boolean containsKey(String key) { - throw new UnsupportedOperationException("The adapter does not support this method"); - } - - @Override - public double getDouble(String key, double defaultValue) { - throw new UnsupportedOperationException("The adapter does not support this method"); - } - - @Override - public Class getClass( - String key, - Class defaultValue, - ClassLoader classLoader) throws ClassNotFoundException { - throw new UnsupportedOperationException("The adapter does not support this method"); - } - - @Override - public String getString(String key, String defaultValue) { - throw new UnsupportedOperationException("The adapter does not support this method"); - } - - @Override - public int getInteger(String key, int defaultValue) { - throw new UnsupportedOperationException("The adapter does not support this method"); - } - - @Override - public long getLong(String key, long defaultValue) { - throw new UnsupportedOperationException("The adapter does not support this method"); - } - - @Override - public boolean getBoolean(String key, boolean defaultValue) { - throw new UnsupportedOperationException("The adapter does not support this method"); - } - - @Override - public float getFloat(String key, float defaultValue) { - throw new UnsupportedOperationException("The adapter does not support this method"); - } - - @Override - public Set keySet() { - throw new UnsupportedOperationException("The adapter does not support this method"); - } - - @Override - public Map toMap() { - throw new UnsupportedOperationException("The adapter does not support this method"); - } - - @Override - public void read(DataInputView in) throws IOException { - throw new UnsupportedOperationException("The adapter does not support this method"); - } - - @Override - public void write(DataOutputView out) throws IOException { - throw new UnsupportedOperationException("The adapter does not support this method"); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/ConfigurableStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ConfigurableStateBackend.java index dd48467872..a5d003ce58 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/ConfigurableStateBackend.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ConfigurableStateBackend.java @@ -19,8 +19,8 @@ package org.apache.flink.runtime.state; import org.apache.flink.annotation.Internal; -import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.configuration.ReadableConfig; /** * An interface for state backends that pick up additional parameters from a configuration. @@ -44,5 +44,5 @@ public interface ConfigurableStateBackend { * * @throws IllegalConfigurationException Thrown if the configuration contained invalid entries. */ - StateBackend configure(Configuration config, ClassLoader classLoader) throws IllegalConfigurationException; + StateBackend configure(ReadableConfig config, ClassLoader classLoader) throws IllegalConfigurationException; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackendFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackendFactory.java index c9d7ef4b7f..5b0bd339a9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackendFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackendFactory.java @@ -19,8 +19,8 @@ package org.apache.flink.runtime.state; import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.configuration.ReadableConfig; import java.io.IOException; @@ -48,5 +48,5 @@ public interface StateBackendFactory { * @throws IOException * If the state backend initialization failed due to an I/O exception */ - T createFromConfig(Configuration config, ClassLoader classLoader) throws IllegalConfigurationException, IOException; + T createFromConfig(ReadableConfig config, ClassLoader classLoader) throws IllegalConfigurationException, IOException; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackendLoader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackendLoader.java index 7ab572e6c1..4f551d7686 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackendLoader.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackendLoader.java @@ -21,6 +21,7 @@ package org.apache.flink.runtime.state; import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.state.filesystem.FsStateBackend; import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory; @@ -65,7 +66,7 @@ public class StateBackendLoader { *

    The state backends can be specified either via their shortcut name, or via the class name * of a {@link StateBackendFactory}. If a StateBackendFactory class name is specified, the factory * is instantiated (via its zero-argument constructor) and its - * {@link StateBackendFactory#createFromConfig(Configuration, ClassLoader)} method is called. + * {@link StateBackendFactory#createFromConfig(ReadableConfig, ClassLoader)} method is called. * *

    Recognized shortcut names are '{@value StateBackendLoader#MEMORY_STATE_BACKEND_NAME}', * '{@value StateBackendLoader#FS_STATE_BACKEND_NAME}', and @@ -87,14 +88,14 @@ public class StateBackendLoader { * May be thrown by the StateBackendFactory when instantiating the state backend */ public static StateBackend loadStateBackendFromConfig( - Configuration config, + ReadableConfig config, ClassLoader classLoader, @Nullable Logger logger) throws IllegalConfigurationException, DynamicCodeLoadingException, IOException { checkNotNull(config, "config"); checkNotNull(classLoader, "classLoader"); - final String backendName = config.getString(CheckpointingOptions.STATE_BACKEND); + final String backendName = config.get(CheckpointingOptions.STATE_BACKEND); if (backendName == null) { return null; } @@ -162,10 +163,10 @@ public class StateBackendLoader { * default state backend (the {@link MemoryStateBackend}). * *

    If an application-defined state backend is found, and the state backend is a - * {@link ConfigurableStateBackend}, this methods calls {@link ConfigurableStateBackend#configure(Configuration, ClassLoader)} + * {@link ConfigurableStateBackend}, this methods calls {@link ConfigurableStateBackend#configure(ReadableConfig, ClassLoader)} * on the state backend. * - *

    Refer to {@link #loadStateBackendFromConfig(Configuration, ClassLoader, Logger)} for details on + *

    Refer to {@link #loadStateBackendFromConfig(ReadableConfig, ClassLoader, Logger)} for details on * how the state backend is loaded from the configuration. * * @param config The configuration to load the state backend from diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/AbstractFileStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/AbstractFileStateBackend.java index 0929504b2b..2bbeedebcc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/AbstractFileStateBackend.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/AbstractFileStateBackend.java @@ -21,8 +21,8 @@ package org.apache.flink.runtime.state.filesystem; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.state.AbstractStateBackend; import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation; @@ -126,7 +126,7 @@ public abstract class AbstractFileStateBackend extends AbstractStateBackend { protected AbstractFileStateBackend( @Nullable Path baseCheckpointPath, @Nullable Path baseSavepointPath, - Configuration configuration) { + ReadableConfig configuration) { this(parameterOrConfigured(baseCheckpointPath, configuration, CheckpointingOptions.CHECKPOINTS_DIRECTORY), parameterOrConfigured(baseSavepointPath, configuration, CheckpointingOptions.SAVEPOINT_DIRECTORY)); @@ -199,12 +199,12 @@ public abstract class AbstractFileStateBackend extends AbstractStateBackend { } @Nullable - private static Path parameterOrConfigured(@Nullable Path path, Configuration config, ConfigOption option) { + private static Path parameterOrConfigured(@Nullable Path path, ReadableConfig config, ConfigOption option) { if (path != null) { return path; } else { - String configValue = config.getString(option); + String configValue = config.get(option); try { return configValue == null ? null : new Path(configValue); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackend.java index 472b23a901..e45fb20c44 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackend.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackend.java @@ -22,7 +22,7 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.configuration.CheckpointingOptions; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.fs.CloseableRegistry; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.Path; @@ -93,7 +93,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull; * parameters from the Flink configuration. For example, if the backend if configured in the application * without a default savepoint directory, it will pick up a default savepoint directory specified in the * Flink configuration of the running job/cluster. That behavior is implemented via the - * {@link #configure(Configuration, ClassLoader)} method. + * {@link #configure(ReadableConfig, ClassLoader)} method. */ @PublicEvolving public class FsStateBackend extends AbstractFileStateBackend implements ConfigurableStateBackend { @@ -355,17 +355,17 @@ public class FsStateBackend extends AbstractFileStateBackend implements Configur * @param original The state backend to re-configure * @param configuration The configuration */ - private FsStateBackend(FsStateBackend original, Configuration configuration, ClassLoader classLoader) { + private FsStateBackend(FsStateBackend original, ReadableConfig configuration, ClassLoader classLoader) { super(original.getCheckpointPath(), original.getSavepointPath(), configuration); // if asynchronous snapshots were configured, use that setting, // else check the configuration this.asynchronousSnapshots = original.asynchronousSnapshots.resolveUndefined( - configuration.getBoolean(CheckpointingOptions.ASYNC_SNAPSHOTS)); + configuration.get(CheckpointingOptions.ASYNC_SNAPSHOTS)); final int sizeThreshold = original.fileStateThreshold >= 0 ? original.fileStateThreshold : - configuration.getInteger(CheckpointingOptions.FS_SMALL_FILE_THRESHOLD); + configuration.get(CheckpointingOptions.FS_SMALL_FILE_THRESHOLD); if (sizeThreshold >= 0 && sizeThreshold <= MAX_FILE_STATE_THRESHOLD) { this.fileStateThreshold = sizeThreshold; @@ -383,7 +383,7 @@ public class FsStateBackend extends AbstractFileStateBackend implements Configur final int bufferSize = original.writeBufferSize >= 0 ? original.writeBufferSize : - configuration.getInteger(CheckpointingOptions.FS_WRITE_BUFFER_SIZE); + configuration.get(CheckpointingOptions.FS_WRITE_BUFFER_SIZE); this.writeBufferSize = Math.max(bufferSize, this.fileStateThreshold); } @@ -471,7 +471,7 @@ public class FsStateBackend extends AbstractFileStateBackend implements Configur * @return The re-configured variant of the state backend */ @Override - public FsStateBackend configure(Configuration config, ClassLoader classLoader) { + public FsStateBackend configure(ReadableConfig config, ClassLoader classLoader) { return new FsStateBackend(this, config, classLoader); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackendFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackendFactory.java index e0db5de8eb..828070a9fb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackendFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackendFactory.java @@ -20,8 +20,8 @@ package org.apache.flink.runtime.state.filesystem; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.configuration.CheckpointingOptions; -import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.runtime.state.StateBackendFactory; /** @@ -31,10 +31,10 @@ import org.apache.flink.runtime.state.StateBackendFactory; public class FsStateBackendFactory implements StateBackendFactory { @Override - public FsStateBackend createFromConfig(Configuration config, ClassLoader classLoader) throws IllegalConfigurationException { + public FsStateBackend createFromConfig(ReadableConfig config, ClassLoader classLoader) throws IllegalConfigurationException { // we need to explicitly read the checkpoint directory here, because that // is a required constructor parameter - final String checkpointDir = config.getString(CheckpointingOptions.CHECKPOINTS_DIRECTORY); + final String checkpointDir = config.get(CheckpointingOptions.CHECKPOINTS_DIRECTORY); if (checkpointDir == null) { throw new IllegalConfigurationException( "Cannot create the file system state backend: The configuration does not specify the " + diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackend.java index c0fbb9ae75..3a6b875a2b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackend.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackend.java @@ -22,7 +22,7 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.configuration.CheckpointingOptions; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.fs.CloseableRegistry; import org.apache.flink.core.fs.Path; import org.apache.flink.metrics.MetricGroup; @@ -100,7 +100,7 @@ import static org.apache.flink.util.Preconditions.checkArgument; * parameters from the Flink configuration. For example, if the backend if configured in the application * without a default savepoint directory, it will pick up a default savepoint directory specified in the * Flink configuration of the running job/cluster. That behavior is implemented via the - * {@link #configure(Configuration, ClassLoader)} method. + * {@link #configure(ReadableConfig, ClassLoader)} method. */ @PublicEvolving public class MemoryStateBackend extends AbstractFileStateBackend implements ConfigurableStateBackend { @@ -234,7 +234,7 @@ public class MemoryStateBackend extends AbstractFileStateBackend implements Conf * @param configuration The configuration * @param classLoader The class loader */ - private MemoryStateBackend(MemoryStateBackend original, Configuration configuration, ClassLoader classLoader) { + private MemoryStateBackend(MemoryStateBackend original, ReadableConfig configuration, ClassLoader classLoader) { super(original.getCheckpointPath(), original.getSavepointPath(), configuration); this.maxStateSize = original.maxStateSize; @@ -242,7 +242,7 @@ public class MemoryStateBackend extends AbstractFileStateBackend implements Conf // if asynchronous snapshots were configured, use that setting, // else check the configuration this.asynchronousSnapshots = original.asynchronousSnapshots.resolveUndefined( - configuration.getBoolean(CheckpointingOptions.ASYNC_SNAPSHOTS)); + configuration.get(CheckpointingOptions.ASYNC_SNAPSHOTS)); } // ------------------------------------------------------------------------ @@ -282,7 +282,7 @@ public class MemoryStateBackend extends AbstractFileStateBackend implements Conf * @return The re-configured variant of the state backend */ @Override - public MemoryStateBackend configure(Configuration config, ClassLoader classLoader) { + public MemoryStateBackend configure(ReadableConfig config, ClassLoader classLoader) { return new MemoryStateBackend(this, config, classLoader); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackendFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackendFactory.java index 149063fba0..39625c506c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackendFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackendFactory.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.state.memory; import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.runtime.state.StateBackendFactory; /** @@ -29,7 +29,7 @@ import org.apache.flink.runtime.state.StateBackendFactory; public class MemoryStateBackendFactory implements StateBackendFactory { @Override - public MemoryStateBackend createFromConfig(Configuration config, ClassLoader classLoader) { + public MemoryStateBackend createFromConfig(ReadableConfig config, ClassLoader classLoader) { return new MemoryStateBackend().configure(config, classLoader); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendLoadingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendLoadingTest.java index 3567049f99..877a51da21 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendLoadingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendLoadingTest.java @@ -21,6 +21,7 @@ package org.apache.flink.runtime.state; import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.HighAvailabilityOptions; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.state.filesystem.FsStateBackend; @@ -427,7 +428,7 @@ public class StateBackendLoadingTest { static final class FailingFactory implements StateBackendFactory { @Override - public StateBackend createFromConfig(Configuration config, ClassLoader classLoader) throws IOException { + public StateBackend createFromConfig(ReadableConfig config, ClassLoader classLoader) throws IOException { throw new IOException("fail!"); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/testutils/BackendForTestStream.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/testutils/BackendForTestStream.java index c8697f8c01..4a6330d6f3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/testutils/BackendForTestStream.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/testutils/BackendForTestStream.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.state.testutils; import org.apache.flink.api.common.JobID; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.runtime.state.CheckpointStorage; import org.apache.flink.runtime.state.CheckpointStorageLocation; import org.apache.flink.runtime.state.CheckpointStorageLocationReference; @@ -56,7 +56,7 @@ public class BackendForTestStream extends MemoryStateBackend { // make no reconfiguration! @Override - public MemoryStateBackend configure(Configuration config, ClassLoader classLoader) { + public MemoryStateBackend configure(ReadableConfig config, ClassLoader classLoader) { return this; } diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/ConfigurableOptionsFactory.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/ConfigurableOptionsFactory.java index 343eda8ace..9256e5911e 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/ConfigurableOptionsFactory.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/ConfigurableOptionsFactory.java @@ -18,7 +18,7 @@ package org.apache.flink.contrib.streaming.state; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; /** * @deprecated Replaced by {@link ConfigurableRocksDBOptionsFactory}. @@ -35,5 +35,5 @@ public interface ConfigurableOptionsFactory extends OptionsFactory { * @param configuration The configuration to pick the values from. * @return A reconfigured options factory. */ - OptionsFactory configure(Configuration configuration); + OptionsFactory configure(ReadableConfig configuration); } diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/ConfigurableRocksDBOptionsFactory.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/ConfigurableRocksDBOptionsFactory.java index ac7b882a86..3a7e549732 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/ConfigurableRocksDBOptionsFactory.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/ConfigurableRocksDBOptionsFactory.java @@ -18,7 +18,7 @@ package org.apache.flink.contrib.streaming.state; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; /** * An interface for options factory that pick up additional parameters from a configuration. @@ -35,5 +35,5 @@ public interface ConfigurableRocksDBOptionsFactory extends RocksDBOptionsFactory * @param configuration The configuration to pick the values from. * @return A reconfigured options factory. */ - RocksDBOptionsFactory configure(Configuration configuration); + RocksDBOptionsFactory configure(ReadableConfig configuration); } diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/DefaultConfigurableOptionsFactory.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/DefaultConfigurableOptionsFactory.java index aaa9e83217..88c8fada5c 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/DefaultConfigurableOptionsFactory.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/DefaultConfigurableOptionsFactory.java @@ -19,8 +19,8 @@ package org.apache.flink.contrib.streaming.state; import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.util.Preconditions; import org.rocksdb.BlockBasedTableConfig; @@ -333,7 +333,7 @@ public class DefaultConfigurableOptionsFactory implements ConfigurableRocksDBOpt )); /** - * Creates a {@link DefaultConfigurableOptionsFactory} instance from a {@link Configuration}. + * Creates a {@link DefaultConfigurableOptionsFactory} instance from a {@link ReadableConfig}. * *

    If no options within {@link RocksDBConfigurableOptions} has ever been configured, * the created OptionsFactory would not override anything defined in {@link PredefinedOptions}. @@ -342,7 +342,7 @@ public class DefaultConfigurableOptionsFactory implements ConfigurableRocksDBOpt * @return A ConfigurableOptionsFactory created from the given configuration */ @Override - public DefaultConfigurableOptionsFactory configure(Configuration configuration) { + public DefaultConfigurableOptionsFactory configure(ReadableConfig configuration) { for (ConfigOption option : CANDIDATE_CONFIGS) { Optional newValue = configuration.getOptional(option); diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBMemoryConfiguration.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBMemoryConfiguration.java index 40f20faa67..96bc247c49 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBMemoryConfiguration.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBMemoryConfiguration.java @@ -18,8 +18,8 @@ package org.apache.flink.contrib.streaming.state; -import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.util.Preconditions; import javax.annotation.Nullable; @@ -183,13 +183,13 @@ public final class RocksDBMemoryConfiguration implements Serializable { */ public static RocksDBMemoryConfiguration fromOtherAndConfiguration( RocksDBMemoryConfiguration other, - Configuration config) { + ReadableConfig config) { final RocksDBMemoryConfiguration newConfig = new RocksDBMemoryConfiguration(); newConfig.useManagedMemory = other.useManagedMemory != null ? other.useManagedMemory - : config.getBoolean(RocksDBOptions.USE_MANAGED_MEMORY); + : config.get(RocksDBOptions.USE_MANAGED_MEMORY); newConfig.fixedMemoryPerSlot = other.fixedMemoryPerSlot != null ? other.fixedMemoryPerSlot @@ -197,11 +197,11 @@ public final class RocksDBMemoryConfiguration implements Serializable { newConfig.writeBufferRatio = other.writeBufferRatio != null ? other.writeBufferRatio - : config.getDouble(RocksDBOptions.WRITE_BUFFER_RATIO); + : config.get(RocksDBOptions.WRITE_BUFFER_RATIO); newConfig.highPriorityPoolRatio = other.highPriorityPoolRatio != null ? other.highPriorityPoolRatio - : config.getDouble(RocksDBOptions.HIGH_PRIORITY_POOL_RATIO); + : config.get(RocksDBOptions.HIGH_PRIORITY_POOL_RATIO); return newConfig; } diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricOptions.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricOptions.java index bbdb145f1a..0a56c7e38c 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricOptions.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricOptions.java @@ -20,7 +20,7 @@ package org.apache.flink.contrib.streaming.state; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import java.io.Serializable; import java.util.Collection; @@ -186,109 +186,109 @@ public class RocksDBNativeMetricOptions implements Serializable { * Creates a {@link RocksDBNativeMetricOptions} based on an * external configuration. */ - public static RocksDBNativeMetricOptions fromConfig(Configuration config) { + public static RocksDBNativeMetricOptions fromConfig(ReadableConfig config) { RocksDBNativeMetricOptions options = new RocksDBNativeMetricOptions(); - if (config.getBoolean(MONITOR_NUM_IMMUTABLE_MEM_TABLES)) { + if (config.get(MONITOR_NUM_IMMUTABLE_MEM_TABLES)) { options.enableNumImmutableMemTable(); } - if (config.getBoolean(MONITOR_MEM_TABLE_FLUSH_PENDING)) { + if (config.get(MONITOR_MEM_TABLE_FLUSH_PENDING)) { options.enableMemTableFlushPending(); } - if (config.getBoolean(TRACK_COMPACTION_PENDING)) { + if (config.get(TRACK_COMPACTION_PENDING)) { options.enableCompactionPending(); } - if (config.getBoolean(MONITOR_BACKGROUND_ERRORS)) { + if (config.get(MONITOR_BACKGROUND_ERRORS)) { options.enableBackgroundErrors(); } - if (config.getBoolean(MONITOR_CUR_SIZE_ACTIVE_MEM_TABLE)) { + if (config.get(MONITOR_CUR_SIZE_ACTIVE_MEM_TABLE)) { options.enableCurSizeActiveMemTable(); } - if (config.getBoolean(MONITOR_CUR_SIZE_ALL_MEM_TABLE)) { + if (config.get(MONITOR_CUR_SIZE_ALL_MEM_TABLE)) { options.enableCurSizeAllMemTables(); } - if (config.getBoolean(MONITOR_SIZE_ALL_MEM_TABLES)) { + if (config.get(MONITOR_SIZE_ALL_MEM_TABLES)) { options.enableSizeAllMemTables(); } - if (config.getBoolean(MONITOR_NUM_ENTRIES_ACTIVE_MEM_TABLE)) { + if (config.get(MONITOR_NUM_ENTRIES_ACTIVE_MEM_TABLE)) { options.enableNumEntriesActiveMemTable(); } - if (config.getBoolean(MONITOR_NUM_ENTRIES_IMM_MEM_TABLES)) { + if (config.get(MONITOR_NUM_ENTRIES_IMM_MEM_TABLES)) { options.enableNumEntriesImmMemTables(); } - if (config.getBoolean(MONITOR_NUM_DELETES_ACTIVE_MEM_TABLE)) { + if (config.get(MONITOR_NUM_DELETES_ACTIVE_MEM_TABLE)) { options.enableNumDeletesActiveMemTable(); } - if (config.getBoolean(MONITOR_NUM_DELETES_IMM_MEM_TABLE)) { + if (config.get(MONITOR_NUM_DELETES_IMM_MEM_TABLE)) { options.enableNumDeletesImmMemTables(); } - if (config.getBoolean(ESTIMATE_NUM_KEYS)) { + if (config.get(ESTIMATE_NUM_KEYS)) { options.enableEstimateNumKeys(); } - if (config.getBoolean(ESTIMATE_TABLE_READERS_MEM)) { + if (config.get(ESTIMATE_TABLE_READERS_MEM)) { options.enableEstimateTableReadersMem(); } - if (config.getBoolean(MONITOR_NUM_SNAPSHOTS)) { + if (config.get(MONITOR_NUM_SNAPSHOTS)) { options.enableNumSnapshots(); } - if (config.getBoolean(MONITOR_NUM_LIVE_VERSIONS)) { + if (config.get(MONITOR_NUM_LIVE_VERSIONS)) { options.enableNumLiveVersions(); } - if (config.getBoolean(ESTIMATE_LIVE_DATA_SIZE)) { + if (config.get(ESTIMATE_LIVE_DATA_SIZE)) { options.enableEstimateLiveDataSize(); } - if (config.getBoolean(MONITOR_TOTAL_SST_FILES_SIZE)) { + if (config.get(MONITOR_TOTAL_SST_FILES_SIZE)) { options.enableTotalSstFilesSize(); } - if (config.getBoolean(ESTIMATE_PENDING_COMPACTION_BYTES)) { + if (config.get(ESTIMATE_PENDING_COMPACTION_BYTES)) { options.enableEstimatePendingCompactionBytes(); } - if (config.getBoolean(MONITOR_NUM_RUNNING_COMPACTIONS)) { + if (config.get(MONITOR_NUM_RUNNING_COMPACTIONS)) { options.enableNumRunningCompactions(); } - if (config.getBoolean(MONITOR_NUM_RUNNING_FLUSHES)) { + if (config.get(MONITOR_NUM_RUNNING_FLUSHES)) { options.enableNumRunningFlushes(); } - if (config.getBoolean(MONITOR_ACTUAL_DELAYED_WRITE_RATE)) { + if (config.get(MONITOR_ACTUAL_DELAYED_WRITE_RATE)) { options.enableActualDelayedWriteRate(); } - if (config.getBoolean(IS_WRITE_STOPPED)) { + if (config.get(IS_WRITE_STOPPED)) { options.enableIsWriteStopped(); } - if (config.getBoolean(BLOCK_CACHE_CAPACITY)) { + if (config.get(BLOCK_CACHE_CAPACITY)) { options.enableBlockCacheCapacity(); } - if (config.getBoolean(BLOCK_CACHE_USAGE)) { + if (config.get(BLOCK_CACHE_USAGE)) { options.enableBlockCacheUsage(); } - if (config.getBoolean(BLOCK_CACHE_PINNED_USAGE)) { + if (config.get(BLOCK_CACHE_PINNED_USAGE)) { options.enableBlockCachePinnedUsage(); } - options.setColumnFamilyAsVariable(config.getBoolean(COLUMN_FAMILY_AS_VARIABLE)); + options.setColumnFamilyAsVariable(config.get(COLUMN_FAMILY_AS_VARIABLE)); return options; } diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBOptions.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBOptions.java index 932d036606..5a5470efb5 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBOptions.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBOptions.java @@ -22,6 +22,7 @@ import org.apache.flink.annotation.docs.Documentation; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.MemorySize; +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend.PriorityQueueStateType; import static org.apache.flink.contrib.streaming.state.PredefinedOptions.DEFAULT; import static org.apache.flink.contrib.streaming.state.PredefinedOptions.FLASH_SSD_OPTIMIZED; @@ -47,9 +48,10 @@ public class RocksDBOptions { * Choice of timer service implementation. */ @Documentation.Section(Documentation.Sections.STATE_BACKEND_ROCKSDB) - public static final ConfigOption TIMER_SERVICE_FACTORY = ConfigOptions + public static final ConfigOption TIMER_SERVICE_FACTORY = ConfigOptions .key("state.backend.rocksdb.timer-service.factory") - .defaultValue(ROCKSDB.name()) + .enumType(PriorityQueueStateType.class) + .defaultValue(ROCKSDB) .withDescription(String.format("This determines the factory for timer service state implementation. Options " + "are either %s (heap-based, default) or %s for an implementation based on RocksDB .", HEAP.name(), ROCKSDB.name())); diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBOptionsFactoryAdapter.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBOptionsFactoryAdapter.java index 36e8c3f597..862832f763 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBOptionsFactoryAdapter.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBOptionsFactoryAdapter.java @@ -18,7 +18,7 @@ package org.apache.flink.contrib.streaming.state; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.rocksdb.ColumnFamilyOptions; import org.rocksdb.DBOptions; @@ -57,7 +57,7 @@ final class RocksDBOptionsFactoryAdapter implements ConfigurableRocksDBOptionsFa } @Override - public RocksDBOptionsFactory configure(Configuration configuration) { + public RocksDBOptionsFactory configure(ReadableConfig configuration) { if (optionsFactory instanceof ConfigurableOptionsFactory) { final OptionsFactory reconfigured = ((ConfigurableOptionsFactory) optionsFactory).configure(configuration); return reconfigured == optionsFactory ? this : new RocksDBOptionsFactoryAdapter(reconfigured); diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java index 873f57d697..add3cf1c14 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java @@ -22,8 +22,8 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.configuration.CheckpointingOptions; -import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.fs.CloseableRegistry; import org.apache.flink.core.fs.Path; import org.apache.flink.metrics.MetricGroup; @@ -304,7 +304,7 @@ public class RocksDBStateBackend extends AbstractStateBackend implements Configu * @param config The configuration. * @param classLoader The class loader. */ - private RocksDBStateBackend(RocksDBStateBackend original, Configuration config, ClassLoader classLoader) { + private RocksDBStateBackend(RocksDBStateBackend original, ReadableConfig config, ClassLoader classLoader) { // reconfigure the state backend backing the streams final StateBackend originalStreamBackend = original.checkpointStreamBackend; this.checkpointStreamBackend = originalStreamBackend instanceof ConfigurableStateBackend ? @@ -313,10 +313,10 @@ public class RocksDBStateBackend extends AbstractStateBackend implements Configu // configure incremental checkpoints this.enableIncrementalCheckpointing = original.enableIncrementalCheckpointing.resolveUndefined( - config.getBoolean(CheckpointingOptions.INCREMENTAL_CHECKPOINTS)); + config.get(CheckpointingOptions.INCREMENTAL_CHECKPOINTS)); if (original.numberOfTransferThreads == UNDEFINED_NUMBER_OF_TRANSFER_THREADS) { - this.numberOfTransferThreads = config.getInteger(CHECKPOINT_TRANSFER_THREAD_NUM); + this.numberOfTransferThreads = config.get(CHECKPOINT_TRANSFER_THREAD_NUM); } else { this.numberOfTransferThreads = original.numberOfTransferThreads; } @@ -327,13 +327,13 @@ public class RocksDBStateBackend extends AbstractStateBackend implements Configu this.writeBatchSize = original.writeBatchSize; } this.enableTtlCompactionFilter = original.enableTtlCompactionFilter - .resolveUndefined(config.getBoolean(TTL_COMPACT_FILTER_ENABLED)); + .resolveUndefined(config.get(TTL_COMPACT_FILTER_ENABLED)); this.memoryConfiguration = RocksDBMemoryConfiguration.fromOtherAndConfiguration(original.memoryConfiguration, config); this.memoryConfiguration.validate(); if (null == original.priorityQueueStateType) { - this.priorityQueueStateType = config.getEnum(PriorityQueueStateType.class, TIMER_SERVICE_FACTORY); + this.priorityQueueStateType = config.get(TIMER_SERVICE_FACTORY); } else { this.priorityQueueStateType = original.priorityQueueStateType; } @@ -343,7 +343,7 @@ public class RocksDBStateBackend extends AbstractStateBackend implements Configu this.localRocksDbDirectories = original.localRocksDbDirectories; } else { - final String rocksdbLocalPaths = config.getString(RocksDBOptions.LOCAL_DIRECTORIES); + final String rocksdbLocalPaths = config.get(RocksDBOptions.LOCAL_DIRECTORIES); if (rocksdbLocalPaths != null) { String[] directories = rocksdbLocalPaths.split(",|" + File.pathSeparator); @@ -362,14 +362,14 @@ public class RocksDBStateBackend extends AbstractStateBackend implements Configu // configure RocksDB predefined options this.predefinedOptions = original.predefinedOptions == null ? - PredefinedOptions.valueOf(config.getString(RocksDBOptions.PREDEFINED_OPTIONS)) : original.predefinedOptions; + PredefinedOptions.valueOf(config.get(RocksDBOptions.PREDEFINED_OPTIONS)) : original.predefinedOptions; LOG.info("Using predefined options: {}.", predefinedOptions.name()); // configure RocksDB options factory try { rocksDbOptionsFactory = configureOptionsFactory( original.rocksDbOptionsFactory, - config.getString(RocksDBOptions.OPTIONS_FACTORY), + config.get(RocksDBOptions.OPTIONS_FACTORY), config, classLoader); } catch (DynamicCodeLoadingException e) { @@ -390,7 +390,7 @@ public class RocksDBStateBackend extends AbstractStateBackend implements Configu * @return The re-configured variant of the state backend */ @Override - public RocksDBStateBackend configure(Configuration config, ClassLoader classLoader) { + public RocksDBStateBackend configure(ReadableConfig config, ClassLoader classLoader) { return new RocksDBStateBackend(this, config, classLoader); } @@ -568,7 +568,7 @@ public class RocksDBStateBackend extends AbstractStateBackend implements Configu private RocksDBOptionsFactory configureOptionsFactory( @Nullable RocksDBOptionsFactory originalOptionsFactory, String factoryClassName, - Configuration config, + ReadableConfig config, ClassLoader classLoader) throws DynamicCodeLoadingException { if (originalOptionsFactory != null) { @@ -775,8 +775,7 @@ public class RocksDBStateBackend extends AbstractStateBackend implements Configu * @return The type of the priority queue state. */ public PriorityQueueStateType getPriorityQueueStateType() { - return priorityQueueStateType == null ? - PriorityQueueStateType.valueOf(TIMER_SERVICE_FACTORY.defaultValue()) : priorityQueueStateType; + return priorityQueueStateType == null ? TIMER_SERVICE_FACTORY.defaultValue() : priorityQueueStateType; } /** diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendFactory.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendFactory.java index 7f3872fe14..8d12ef0ceb 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendFactory.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendFactory.java @@ -19,8 +19,8 @@ package org.apache.flink.contrib.streaming.state; import org.apache.flink.configuration.CheckpointingOptions; -import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.runtime.state.StateBackendFactory; import java.io.IOException; @@ -32,12 +32,12 @@ import java.io.IOException; public class RocksDBStateBackendFactory implements StateBackendFactory { @Override - public RocksDBStateBackend createFromConfig(Configuration config, ClassLoader classLoader) + public RocksDBStateBackend createFromConfig(ReadableConfig config, ClassLoader classLoader) throws IllegalConfigurationException, IOException { // we need to explicitly read the checkpoint directory here, because that // is a required constructor parameter - final String checkpointDirURI = config.getString(CheckpointingOptions.CHECKPOINTS_DIRECTORY); + final String checkpointDirURI = config.get(CheckpointingOptions.CHECKPOINTS_DIRECTORY); if (checkpointDirURI == null) { throw new IllegalConfigurationException( "Cannot create the RocksDB state backend: The configuration does not specify the " + diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java index 25a7bce508..6ebc9d2fca 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java @@ -257,7 +257,7 @@ public class RocksDBAsyncSnapshotTest extends TestLogger { File dbDir = temporaryFolder.newFolder(); - final RocksDBStateBackend.PriorityQueueStateType timerServicePriorityQueueType = RocksDBStateBackend.PriorityQueueStateType.valueOf(RocksDBOptions.TIMER_SERVICE_FACTORY.defaultValue()); + final RocksDBStateBackend.PriorityQueueStateType timerServicePriorityQueueType = RocksDBOptions.TIMER_SERVICE_FACTORY.defaultValue(); final int skipStreams; diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBOptionsFactoryCompatibilityTest.java b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBOptionsFactoryCompatibilityTest.java index f96bed3419..8c77ac249f 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBOptionsFactoryCompatibilityTest.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBOptionsFactoryCompatibilityTest.java @@ -19,6 +19,7 @@ package org.apache.flink.contrib.streaming.state; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.junit.Test; import org.rocksdb.ColumnFamilyOptions; @@ -70,7 +71,7 @@ public class RocksDBOptionsFactoryCompatibilityTest { boolean wasConfigured; @Override - public OptionsFactory configure(Configuration configuration) { + public OptionsFactory configure(ReadableConfig configuration) { wasConfigured = true; return this; } diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java index 2167a000d1..d3eb6dbc28 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java @@ -21,8 +21,10 @@ package org.apache.flink.contrib.streaming.state; import org.apache.flink.api.common.typeutils.base.IntSerializer; import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.fs.CloseableRegistry; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.Path; @@ -153,7 +155,7 @@ public class RocksDBStateBackendConfigTest { // Fix the default Assert.assertEquals( - RocksDBStateBackend.PriorityQueueStateType.ROCKSDB.toString(), + RocksDBStateBackend.PriorityQueueStateType.ROCKSDB, RocksDBOptions.TIMER_SERVICE_FACTORY.defaultValue()); RocksDBStateBackend rocksDbBackend = new RocksDBStateBackend(tempFolder.newFolder().toURI().toString()); @@ -163,9 +165,7 @@ public class RocksDBStateBackendConfigTest { keyedBackend.dispose(); Configuration conf = new Configuration(); - conf.setString( - RocksDBOptions.TIMER_SERVICE_FACTORY, - RocksDBStateBackend.PriorityQueueStateType.HEAP.toString()); + conf.set(RocksDBOptions.TIMER_SERVICE_FACTORY, RocksDBStateBackend.PriorityQueueStateType.HEAP); rocksDbBackend = rocksDbBackend.configure(conf, Thread.currentThread().getContextClassLoader()); keyedBackend = createKeyedStateBackend(rocksDbBackend, env); @@ -190,7 +190,7 @@ public class RocksDBStateBackendConfigTest { // priorityQueueStateType in the cluster config final Configuration configFromConfFile = new Configuration(); configFromConfFile.setString( - RocksDBOptions.TIMER_SERVICE_FACTORY, + RocksDBOptions.TIMER_SERVICE_FACTORY.key(), RocksDBStateBackend.PriorityQueueStateType.ROCKSDB.toString()); // configure final backend from job and cluster config @@ -547,7 +547,7 @@ public class RocksDBStateBackendConfigTest { // verify that user-defined options factory could be configured via flink-conf.yaml Configuration config = new Configuration(); config.setString(RocksDBOptions.OPTIONS_FACTORY.key(), TestOptionsFactory.class.getName()); - config.setInteger(TestOptionsFactory.BACKGROUND_JOBS_OPTION, 4); + config.setString(TestOptionsFactory.BACKGROUND_JOBS_OPTION.key(), "4"); rocksDbBackend = rocksDbBackend.configure(config, getClass().getClassLoader()); @@ -776,10 +776,12 @@ public class RocksDBStateBackendConfigTest { * An implementation of options factory for testing. */ public static class TestOptionsFactory implements ConfigurableRocksDBOptionsFactory { - public static final String BACKGROUND_JOBS_OPTION = "my.custom.rocksdb.backgroundJobs"; + public static final ConfigOption BACKGROUND_JOBS_OPTION = + ConfigOptions.key("my.custom.rocksdb.backgroundJobs") + .intType() + .defaultValue(2); - private static final int DEFAULT_BACKGROUND_JOBS = 2; - private int backgroundJobs = DEFAULT_BACKGROUND_JOBS; + private int backgroundJobs = BACKGROUND_JOBS_OPTION.defaultValue(); @Override public DBOptions createDBOptions(DBOptions currentOptions, Collection handlesToClose) { @@ -792,8 +794,8 @@ public class RocksDBStateBackendConfigTest { } @Override - public RocksDBOptionsFactory configure(Configuration configuration) { - this.backgroundJobs = configuration.getInteger(BACKGROUND_JOBS_OPTION, DEFAULT_BACKGROUND_JOBS); + public RocksDBOptionsFactory configure(ReadableConfig configuration) { + this.backgroundJobs = configuration.get(BACKGROUND_JOBS_OPTION); return this; } } diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendMigrationTest.java b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendMigrationTest.java index 4d58a64444..056e46ff1a 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendMigrationTest.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendMigrationTest.java @@ -53,9 +53,7 @@ public class RocksDBStateBackendMigrationTest extends StateBackendMigrationTestB RocksDBStateBackend backend = new RocksDBStateBackend(new FsStateBackend(checkpointPath), enableIncrementalCheckpointing); Configuration configuration = new Configuration(); - configuration.setString( - RocksDBOptions.TIMER_SERVICE_FACTORY, - RocksDBStateBackend.PriorityQueueStateType.ROCKSDB.toString()); + configuration.set(RocksDBOptions.TIMER_SERVICE_FACTORY, RocksDBStateBackend.PriorityQueueStateType.ROCKSDB); backend = backend.configure(configuration, Thread.currentThread().getContextClassLoader()); backend.setDbStoragePath(dbPath); return backend; diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java index b31e190e9e..10cb6e71e6 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java @@ -133,9 +133,7 @@ public class RocksDBStateBackendTest extends StateBackendTestBase Date: Sun, 5 Apr 2020 11:56:14 +0200 Subject: [PATCH 445/885] [FLINK-16979][hive][build] Exclude jdk.tools --- flink-connectors/flink-sql-connector-hive-1.2.2/pom.xml | 5 +++++ flink-connectors/flink-sql-connector-hive-2.2.0/pom.xml | 5 +++++ 2 files changed, 10 insertions(+) diff --git a/flink-connectors/flink-sql-connector-hive-1.2.2/pom.xml b/flink-connectors/flink-sql-connector-hive-1.2.2/pom.xml index d8a2c4230a..8824b3d4c4 100644 --- a/flink-connectors/flink-sql-connector-hive-1.2.2/pom.xml +++ b/flink-connectors/flink-sql-connector-hive-1.2.2/pom.xml @@ -98,6 +98,11 @@ under the License. org.slf4j slf4j-log4j12 + + + jdk.tools + jdk.tools + diff --git a/flink-connectors/flink-sql-connector-hive-2.2.0/pom.xml b/flink-connectors/flink-sql-connector-hive-2.2.0/pom.xml index 856b38825d..46cd1353a0 100644 --- a/flink-connectors/flink-sql-connector-hive-2.2.0/pom.xml +++ b/flink-connectors/flink-sql-connector-hive-2.2.0/pom.xml @@ -75,6 +75,11 @@ under the License. org.slf4j slf4j-log4j12 + + + jdk.tools + jdk.tools + -- Gitee From dd9f9bf040cb82ed7e18c9fdf7c7e1ca6f43f896 Mon Sep 17 00:00:00 2001 From: ifndef-SleePy Date: Fri, 3 Apr 2020 17:27:00 +0800 Subject: [PATCH 446/885] [hotfix] Make the checkpoint resuming e2e case pass by increasing the retained checkpoints number --- .../test-scripts/test_resume_externalized_checkpoints.sh | 2 ++ flink-end-to-end-tests/test-scripts/test_resume_savepoint.sh | 2 ++ 2 files changed, 4 insertions(+) diff --git a/flink-end-to-end-tests/test-scripts/test_resume_externalized_checkpoints.sh b/flink-end-to-end-tests/test-scripts/test_resume_externalized_checkpoints.sh index a24f848411..e0c822a97f 100755 --- a/flink-end-to-end-tests/test-scripts/test_resume_externalized_checkpoints.sh +++ b/flink-end-to-end-tests/test-scripts/test_resume_externalized_checkpoints.sh @@ -39,6 +39,8 @@ fi set_config_key "taskmanager.numberOfTaskSlots" "${NUM_SLOTS}" set_config_key "metrics.fetcher.update-interval" "2000" +# hotfix for FLINK-16770 +set_config_key "state.checkpoints.num-retained" "2" setup_flink_slf4j_metric_reporter start_cluster diff --git a/flink-end-to-end-tests/test-scripts/test_resume_savepoint.sh b/flink-end-to-end-tests/test-scripts/test_resume_savepoint.sh index 0852b5dd4c..e4d8f17dd1 100755 --- a/flink-end-to-end-tests/test-scripts/test_resume_savepoint.sh +++ b/flink-end-to-end-tests/test-scripts/test_resume_savepoint.sh @@ -61,6 +61,8 @@ if [ $STATE_BACKEND_ROCKS_TIMER_SERVICE_TYPE == 'heap' ]; then set_config_key "state.backend.rocksdb.timer-service.factory" "heap" fi set_config_key "metrics.fetcher.update-interval" "2000" +# hotfix for FLINK-16770 +set_config_key "state.checkpoints.num-retained" "2" setup_flink_slf4j_metric_reporter -- Gitee From d6efa7580438ba72d6792d36b8a08de979f44cc8 Mon Sep 17 00:00:00 2001 From: Jaskaran Bindra Date: Mon, 6 Apr 2020 13:57:38 +0530 Subject: [PATCH 447/885] [hotfix][metrics][javadocs] Fix typo --- .../java/org/apache/flink/metrics/reporter/MetricReporter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/reporter/MetricReporter.java b/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/reporter/MetricReporter.java index 3ca6e6ac14..55ff97636f 100644 --- a/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/reporter/MetricReporter.java +++ b/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/reporter/MetricReporter.java @@ -69,7 +69,7 @@ public interface MetricReporter { void notifyOfAddedMetric(Metric metric, String metricName, MetricGroup group); /** - * Called when a {@link Metric} was should be removed. + * Called when a {@link Metric} was removed. * * @param metric the metric that should be removed * @param metricName the name of the metric -- Gitee From c9825980f14f10a0acb26a531beb643e9dd0c400 Mon Sep 17 00:00:00 2001 From: wangxiyuan Date: Mon, 6 Apr 2020 03:40:50 -0500 Subject: [PATCH 448/885] [FLINK-14126][e2e] Enforce elasticsearch version check --- .../test-scripts/elasticsearch-common.sh | 7 ++++++- flink-end-to-end-tests/test-scripts/test_quickstarts.sh | 2 +- flink-end-to-end-tests/test-scripts/test_sql_client.sh | 2 +- 3 files changed, 8 insertions(+), 3 deletions(-) diff --git a/flink-end-to-end-tests/test-scripts/elasticsearch-common.sh b/flink-end-to-end-tests/test-scripts/elasticsearch-common.sh index 8a2afcb4ff..0c23526237 100644 --- a/flink-end-to-end-tests/test-scripts/elasticsearch-common.sh +++ b/flink-end-to-end-tests/test-scripts/elasticsearch-common.sh @@ -26,7 +26,12 @@ function setup_elasticsearch { mkdir -p $TEST_DATA_DIR local downloadUrl=$1 - local elasticsearch_version=${2-0} + local elasticsearch_version=$2 + + if [ -z $elasticsearch_version ]; then + echo "Elasticsearch version not declared." + exit 1 + fi # start downloading Elasticsearch echo "Downloading Elasticsearch from $downloadUrl ..." diff --git a/flink-end-to-end-tests/test-scripts/test_quickstarts.sh b/flink-end-to-end-tests/test-scripts/test_quickstarts.sh index 3b06e19fba..97ba9f74d8 100755 --- a/flink-end-to-end-tests/test-scripts/test_quickstarts.sh +++ b/flink-end-to-end-tests/test-scripts/test_quickstarts.sh @@ -95,7 +95,7 @@ else echo "Success: Elasticsearch5SinkExample.class and other user classes are included in the jar." fi -setup_elasticsearch "https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-5.3.3.tar.gz" +setup_elasticsearch "https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-5.3.3.tar.gz" 5 wait_elasticsearch_working function shutdownAndCleanup { diff --git a/flink-end-to-end-tests/test-scripts/test_sql_client.sh b/flink-end-to-end-tests/test-scripts/test_sql_client.sh index 54209249b0..ab02378385 100755 --- a/flink-end-to-end-tests/test-scripts/test_sql_client.sh +++ b/flink-end-to-end-tests/test-scripts/test_sql_client.sh @@ -110,7 +110,7 @@ function prepare_elasticsearch { exit 1 fi - setup_elasticsearch $DOWNLOAD_URL + setup_elasticsearch $DOWNLOAD_URL $ELASTICSEARCH_VERSION wait_elasticsearch_working } -- Gitee From bf671fb3d2a05f3f684d0acbea049d04cd014b17 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Sat, 4 Apr 2020 17:52:01 +0200 Subject: [PATCH 449/885] [FLINK-16981][tests] Add global PowerMock exclusions --- .../state/RocksDBAsyncSnapshotTest.java | 2 -- .../extensions/configuration.properties | 19 +++++++++++++++++++ 2 files changed, 19 insertions(+), 2 deletions(-) create mode 100644 flink-test-utils-parent/flink-test-utils-junit/src/main/resources/org/powermock/extensions/configuration.properties diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java index 6ebc9d2fca..9913986794 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java @@ -76,7 +76,6 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; -import org.powermock.core.classloader.annotations.PowerMockIgnore; import org.powermock.modules.junit4.PowerMockRunner; import javax.annotation.Nullable; @@ -107,7 +106,6 @@ import static org.mockito.Mockito.verify; * Tests for asynchronous RocksDB Key/Value state checkpoints. */ @RunWith(PowerMockRunner.class) -@PowerMockIgnore({"javax.management.*", "com.sun.jndi.*", "org.apache.log4j.*"}) @SuppressWarnings("serial") public class RocksDBAsyncSnapshotTest extends TestLogger { diff --git a/flink-test-utils-parent/flink-test-utils-junit/src/main/resources/org/powermock/extensions/configuration.properties b/flink-test-utils-parent/flink-test-utils-junit/src/main/resources/org/powermock/extensions/configuration.properties new file mode 100644 index 0000000000..d2835ca5b8 --- /dev/null +++ b/flink-test-utils-parent/flink-test-utils-junit/src/main/resources/org/powermock/extensions/configuration.properties @@ -0,0 +1,19 @@ +################################################################################ +# 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. +################################################################################ + +powermock.global-ignore=javax.management.*,com.sun.jndi.*,org.apache.log4j.*,org.apache.logging.log4j.* -- Gitee From 5f2f7d637d8073b1034f8c1124f52604745936eb Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 6 Apr 2020 12:44:01 +0200 Subject: [PATCH 450/885] [minor] Remove --quiet from aws file fetching to help debugging --- flink-end-to-end-tests/test-scripts/common_s3_operations.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-end-to-end-tests/test-scripts/common_s3_operations.sh b/flink-end-to-end-tests/test-scripts/common_s3_operations.sh index eae745b683..1bc6132de4 100644 --- a/flink-end-to-end-tests/test-scripts/common_s3_operations.sh +++ b/flink-end-to-end-tests/test-scripts/common_s3_operations.sh @@ -99,7 +99,7 @@ function s3_get_by_full_path_and_filename_prefix() { args="$args --recursive" fi local relative_dir=${1#$TEST_INFRA_DIR} - aws_cli s3 cp --quiet "s3://$IT_CASE_S3_BUCKET/$2" "/hostdir/${relative_dir}" $args + aws_cli s3 cp "s3://$IT_CASE_S3_BUCKET/$2" "/hostdir/${relative_dir}" $args } ################################### -- Gitee From 1a06dd82a06d2fb80b4a878e1bc90eda72d1c307 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 6 Apr 2020 12:44:45 +0200 Subject: [PATCH 451/885] [FLINK-15772] Add retry to test_batch_wordcount file fetching The suspicion is that test flakyness was caused by S3 flakyness/eventual consistency. --- .../test-scripts/common_s3_operations.sh | 1 + .../test-scripts/test_batch_wordcount.sh | 7 ++++++- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/flink-end-to-end-tests/test-scripts/common_s3_operations.sh b/flink-end-to-end-tests/test-scripts/common_s3_operations.sh index 1bc6132de4..7c8745e113 100644 --- a/flink-end-to-end-tests/test-scripts/common_s3_operations.sh +++ b/flink-end-to-end-tests/test-scripts/common_s3_operations.sh @@ -73,6 +73,7 @@ function aws_cli() { fi if ! docker exec "$AWSCLI_CONTAINER_ID" aws $endpoint "$@"; then echo "Error executing aws command: $@"; + return 1 fi } diff --git a/flink-end-to-end-tests/test-scripts/test_batch_wordcount.sh b/flink-end-to-end-tests/test-scripts/test_batch_wordcount.sh index e0cedf47a7..59870f7ee6 100755 --- a/flink-end-to-end-tests/test-scripts/test_batch_wordcount.sh +++ b/flink-end-to-end-tests/test-scripts/test_batch_wordcount.sh @@ -85,5 +85,10 @@ start_cluster # But the sources should provide the same test data, so the checksum stays the same for all tests. ${FLINK_DIR}/bin/flink run -p 1 ${FLINK_DIR}/examples/batch/WordCount.jar ${ARGS} # Fetches result from AWS s3 to the OUTPUT_PATH, no-op for other filesystems and minio-based tests -${fetch_complete_result[@]} + +# it seems we need a function for retry_times +function fetch_it() { + ${fetch_complete_result[@]} +} +retry_times 10 5 fetch_it check_result_hash "WordCount (${INPUT_TYPE})" "${OUTPUT_PATH}" "${RESULT_HASH}" -- Gitee From c9121f06e44f56fc10f827540773c4f68e019550 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 14 Feb 2020 11:03:59 +0100 Subject: [PATCH 452/885] [FLINK-15998] In Glossary, clarify Application/Job cluster description --- docs/concepts/glossary.md | 39 +++++++++++++++++++++++++++++---------- 1 file changed, 29 insertions(+), 10 deletions(-) diff --git a/docs/concepts/glossary.md b/docs/concepts/glossary.md index 0645786f8b..697b1b044f 100644 --- a/docs/concepts/glossary.md +++ b/docs/concepts/glossary.md @@ -25,11 +25,16 @@ under the License. #### Flink Application Cluster -A Flink Application Cluster is a dedicated [Flink Cluster](#flink-cluster) that only +A Flink Application Cluster is a dedicated [Flink Cluster](#flink-cluster) that +only executes [Flink Jobs](#flink-job) from one [Flink +Application](#flink-application). The lifetime of the [Flink +Cluster](#flink-cluster) is bound to the lifetime of the Flink Application. + +#### Flink Job Cluster + +A Flink Job Cluster is a dedicated [Flink Cluster](#flink-cluster) that only executes a single [Flink Job](#flink-job). The lifetime of the -[Flink Cluster](#flink-cluster) is bound to the lifetime of the Flink Job. Formerly -Flink Application Clusters were also known as Flink Clusters in *job mode*. Compare to -[Flink Session Cluster](#flink-session-cluster). +[Flink Cluster](#flink-cluster) is bound to the lifetime of the Flink Job. #### Flink Cluster @@ -60,11 +65,22 @@ Java, this corresponds to the definition of *Instance* or *Object* in Java. In t Flink, the term *parallel instance* is also frequently used to emphasize that multiple instances of the same [Operator](#operator) or [Function](#function) type are running in parallel. +#### Flink Application + +A Flink application is a Java Application that submits one or multiple [Flink +Jobs](#flink-job) from the `main()` method (or by some other means). Submitting +jobs is usually done by calling `execute()` on an execution environment. + +The jobs of an application can either be submitted to a long running [Flink +Session Cluster](#flink-session-cluster), to a dedicated [Flink Application +Cluster](#flink-application-cluster), or to a [Flink Job +Cluster](#flink-job-cluster). + #### Flink Job -A Flink Job is the runtime representation of a Flink program. A Flink Job can either be submitted -to a long running [Flink Session Cluster](#flink-session-cluster) or it can be started as a -self-contained [Flink Application Cluster](#flink-application-cluster). +A Flink Job is the runtime representation of a [logical graph](#logical-graph) +(also often called dataflow graph) that is created and submitted by calling +`execute()` in a [Flink Application](#flink-application). #### JobGraph @@ -78,9 +94,12 @@ whole [Flink Master](#flink-master) was called JobManager. #### Logical Graph -A logical graph is a directed graph describing the high-level logic of a stream processing program. -The nodes are [Operators](#operator) and the edges indicate input/output-relationships of the -operators and correspond to data streams or data sets. +A logical graph is a directed graph where the nodes are [Operators](#operator) +and the edges define input/output-relationships of the operators and correspond +to data streams or data sets. A logical graph is created by submitting jobs +from a [Flink Application](#flink-application). + +Logical graphs are also often referred to as *dataflow graphs*. #### Managed State -- Gitee From d2be6aef18b3e1afbbf1d3f4ed0694460ed870b6 Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Sun, 5 Apr 2020 19:44:05 +0200 Subject: [PATCH 453/885] [FLINK-16921][e2e] Disable K8s tests until instability is resolved --- flink-end-to-end-tests/run-nightly-tests.sh | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/flink-end-to-end-tests/run-nightly-tests.sh b/flink-end-to-end-tests/run-nightly-tests.sh index a16fc8946a..484e17ad74 100755 --- a/flink-end-to-end-tests/run-nightly-tests.sh +++ b/flink-end-to-end-tests/run-nightly-tests.sh @@ -127,8 +127,9 @@ if [[ ${PROFILE} != *"jdk11"* ]]; then run_test "Running Kerberized YARN on Docker test (default input)" "$END_TO_END_DIR/test-scripts/test_yarn_kerberos_docker.sh" run_test "Running Kerberized YARN on Docker test (custom fs plugin)" "$END_TO_END_DIR/test-scripts/test_yarn_kerberos_docker.sh dummy-fs" - run_test "Run Kubernetes test" "$END_TO_END_DIR/test-scripts/test_kubernetes_embedded_job.sh" - run_test "Run kubernetes session test" "$END_TO_END_DIR/test-scripts/test_kubernetes_session.sh" + # Disable K8s tests until FLINK-16921 is resolved + # run_test "Run Kubernetes test" "$END_TO_END_DIR/test-scripts/test_kubernetes_embedded_job.sh" + # run_test "Run kubernetes session test" "$END_TO_END_DIR/test-scripts/test_kubernetes_session.sh" run_test "Running Flink over NAT end-to-end test" "$END_TO_END_DIR/test-scripts/test_nat.sh" "skip_check_exceptions" -- Gitee From 7381304930a964098df52d9fa79a55241538b301 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Mon, 16 Mar 2020 10:19:18 +0100 Subject: [PATCH 454/885] [FLINK-16655][FLINK-16657] Introduce embedded executor and use it in Web Submission This closes #11460. --- .../application/ApplicationRunner.java | 49 +++++++ .../DetachedApplicationRunner.java | 80 +++++++++++ .../application/EmbeddedJobClient.java | 118 ++++++++++++++++ .../executors/EmbeddedExecutor.java | 131 ++++++++++++++++++ .../executors/EmbeddedExecutorFactory.java | 73 ++++++++++ .../EmbeddedExecutorServiceLoader.java | 68 +++++++++ .../deployment/executors/ExecutorUtils.java | 6 + .../PipelineOptionsInternal.java | 37 +++++ .../webmonitor/WebSubmissionExtension.java | 4 +- .../webmonitor/handlers/JarRunHandler.java | 67 ++++----- .../handlers/utils/JarHandlerUtils.java | 48 +++++-- .../webmonitor/handlers/JarHandlerTest.java | 22 +-- .../webmonitor/handlers/JarHandlers.java | 4 +- .../handlers/JarRunHandlerParameterTest.java | 25 +++- .../handlers/JarSubmissionITCase.java | 2 +- .../handlers/utils/TestProgram.java | 4 +- .../testutils/ParameterProgram.java | 4 +- 17 files changed, 667 insertions(+), 75 deletions(-) create mode 100644 flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationRunner.java create mode 100644 flink-clients/src/main/java/org/apache/flink/client/deployment/application/DetachedApplicationRunner.java create mode 100644 flink-clients/src/main/java/org/apache/flink/client/deployment/application/EmbeddedJobClient.java create mode 100644 flink-clients/src/main/java/org/apache/flink/client/deployment/application/executors/EmbeddedExecutor.java create mode 100644 flink-clients/src/main/java/org/apache/flink/client/deployment/application/executors/EmbeddedExecutorFactory.java create mode 100644 flink-clients/src/main/java/org/apache/flink/client/deployment/application/executors/EmbeddedExecutorServiceLoader.java create mode 100644 flink-core/src/main/java/org/apache/flink/configuration/PipelineOptionsInternal.java diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationRunner.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationRunner.java new file mode 100644 index 0000000000..1e03ccbc2c --- /dev/null +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationRunner.java @@ -0,0 +1,49 @@ +/* + * 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.client.deployment.application; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.JobID; +import org.apache.flink.client.program.PackagedProgram; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.dispatcher.DispatcherGateway; + +import java.util.List; + +/** + * An interface to be implemented by the entities responsible for application + * submission for the different deployment environments. + * + *

    This interface assumes access to the cluster's {@link DispatcherGateway}, + * and it does not go through the publicly exposed REST API. + */ +@Internal +public interface ApplicationRunner { + + /** + * Runs the application using the provided {@code dispatcherGateway}. + * + * @param dispatcherGateway the dispatcher of the cluster to run the application. + * @param program the {@link PackagedProgram} containing the user's main method. + * @param configuration the configuration used to run the application. + * + * @return a list of the submitted jobs that belong to the provided application. + */ + List run(final DispatcherGateway dispatcherGateway, final PackagedProgram program, final Configuration configuration); +} diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/application/DetachedApplicationRunner.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/DetachedApplicationRunner.java new file mode 100644 index 0000000000..8bc1a72e00 --- /dev/null +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/DetachedApplicationRunner.java @@ -0,0 +1,80 @@ +/* + * 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.client.deployment.application; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.JobID; +import org.apache.flink.client.ClientUtils; +import org.apache.flink.client.deployment.application.executors.EmbeddedExecutor; +import org.apache.flink.client.deployment.application.executors.EmbeddedExecutorServiceLoader; +import org.apache.flink.client.program.PackagedProgram; +import org.apache.flink.client.program.ProgramInvocationException; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.DeploymentOptions; +import org.apache.flink.core.execution.PipelineExecutorServiceLoader; +import org.apache.flink.runtime.dispatcher.DispatcherGateway; +import org.apache.flink.util.FlinkRuntimeException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * An {@link ApplicationRunner} which runs the user specified application using the {@link EmbeddedExecutor}. + * This runner invokes methods of the provided {@link DispatcherGateway} directly, and it does not go through + * the REST API. + * + *

    In addition, this runner does not wait for the application to finish, but it submits the + * application in a {@code DETACHED} mode. As a consequence, applications with jobs that rely on + * operations like {@code [collect, print, printToErr, count]} will fail. + */ +@Internal +public class DetachedApplicationRunner implements ApplicationRunner { + + private static final Logger LOG = LoggerFactory.getLogger(DetachedApplicationRunner.class); + + @Override + public List run(final DispatcherGateway dispatcherGateway, final PackagedProgram program, final Configuration configuration) { + checkNotNull(dispatcherGateway); + checkNotNull(program); + checkNotNull(configuration); + return tryExecuteJobs(dispatcherGateway, program, configuration); + } + + private List tryExecuteJobs(final DispatcherGateway dispatcherGateway, final PackagedProgram program, final Configuration configuration) { + configuration.set(DeploymentOptions.ATTACHED, false); + + final List applicationJobIds = new ArrayList<>(); + final PipelineExecutorServiceLoader executorServiceLoader = + new EmbeddedExecutorServiceLoader(applicationJobIds, dispatcherGateway); + + try { + ClientUtils.executeProgram(executorServiceLoader, configuration, program); + } catch (ProgramInvocationException e) { + LOG.warn("Could not execute application: ", e); + throw new FlinkRuntimeException("Could not execute application.", e); + } + + return applicationJobIds; + } +} diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/application/EmbeddedJobClient.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/EmbeddedJobClient.java new file mode 100644 index 0000000000..8518494a07 --- /dev/null +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/EmbeddedJobClient.java @@ -0,0 +1,118 @@ +/* + * 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.client.deployment.application; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.common.accumulators.AccumulatorHelper; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.dispatcher.DispatcherGateway; +import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; + +import javax.annotation.Nullable; + +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A {@link JobClient} with the ability to also submit jobs which + * uses directly the {@link DispatcherGateway}. + */ +@Internal +public class EmbeddedJobClient implements JobClient { + + private final JobID jobId; + + private final DispatcherGateway dispatcherGateway; + + private final Time timeout; + + public EmbeddedJobClient( + final JobID jobId, + final DispatcherGateway dispatcherGateway, + final Time rpcTimeout) { + this.jobId = checkNotNull(jobId); + this.dispatcherGateway = checkNotNull(dispatcherGateway); + this.timeout = checkNotNull(rpcTimeout); + } + + @Override + public JobID getJobID() { + return jobId; + } + + @Override + public CompletableFuture getJobStatus() { + return dispatcherGateway.requestJobStatus(jobId, timeout); + } + + @Override + public CompletableFuture cancel() { + return dispatcherGateway + .cancelJob(jobId, timeout) + .thenApply(ignores -> null); + } + + @Override + public CompletableFuture stopWithSavepoint(final boolean advanceToEndOfEventTime, @Nullable final String savepointDirectory) { + return dispatcherGateway.stopWithSavepoint(jobId, savepointDirectory, advanceToEndOfEventTime, timeout); + } + + @Override + public CompletableFuture triggerSavepoint(@Nullable final String savepointDirectory) { + return dispatcherGateway.triggerSavepoint(jobId, savepointDirectory, false, timeout); + } + + @Override + public CompletableFuture> getAccumulators(final ClassLoader classLoader) { + checkNotNull(classLoader); + + return dispatcherGateway.requestJob(jobId, timeout) + .thenApply(ArchivedExecutionGraph::getAccumulatorsSerialized) + .thenApply(accumulators -> { + try { + return AccumulatorHelper.deserializeAndUnwrapAccumulators(accumulators, classLoader); + } catch (Exception e) { + throw new CompletionException("Cannot deserialize and unwrap accumulators properly.", e); + } + }); + } + + @Override + public CompletableFuture getJobExecutionResult(final ClassLoader userClassloader) { + checkNotNull(userClassloader); + + return dispatcherGateway + .requestJobResult(jobId, timeout) + .thenApply((jobResult) -> { + try { + return jobResult.toJobExecutionResult(userClassloader); + } catch (Throwable t) { + throw new CompletionException( + new Exception("Job " + jobId + " failed", t)); + } + }); + } +} diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/application/executors/EmbeddedExecutor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/executors/EmbeddedExecutor.java new file mode 100644 index 0000000000..50b494d432 --- /dev/null +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/executors/EmbeddedExecutor.java @@ -0,0 +1,131 @@ +/* + * 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.client.deployment.application.executors; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.dag.Pipeline; +import org.apache.flink.client.deployment.application.EmbeddedJobClient; +import org.apache.flink.client.deployment.executors.ExecutorUtils; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.WebOptions; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.core.execution.PipelineExecutor; +import org.apache.flink.runtime.blob.BlobClient; +import org.apache.flink.runtime.client.ClientUtils; +import org.apache.flink.runtime.dispatcher.DispatcherGateway; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.function.FunctionWithException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.Collection; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A {@link PipelineExecutor} that invokes directly methods of the + * {@link org.apache.flink.runtime.dispatcher.DispatcherGateway Dispatcher} and does + * not go through the REST API. + */ +@Internal +public class EmbeddedExecutor implements PipelineExecutor { + + private static final Logger LOG = LoggerFactory.getLogger(EmbeddedExecutor.class); + + public static final String NAME = "embedded"; + + private final Collection submittedJobIds; + + private final DispatcherGateway dispatcherGateway; + + /** + * Creates an {@link EmbeddedExecutor}. + * @param submittedJobIds a list that is going to be filled with the job ids of the + * new jobs that will be submitted. This is essentially used to return the submitted job ids + * to the caller. + * @param dispatcherGateway the dispatcher of the cluster which is going to be used to submit jobs. + */ + public EmbeddedExecutor( + final Collection submittedJobIds, + final DispatcherGateway dispatcherGateway) { + this.submittedJobIds = checkNotNull(submittedJobIds); + this.dispatcherGateway = checkNotNull(dispatcherGateway); + } + + @Override + public CompletableFuture execute(final Pipeline pipeline, final Configuration configuration) { + checkNotNull(pipeline); + checkNotNull(configuration); + + final JobGraph jobGraph = ExecutorUtils.getJobGraph(pipeline, configuration); + final JobID actualJobId = jobGraph.getJobID(); + + this.submittedJobIds.add(actualJobId); + LOG.info("Job {} is submitted.", actualJobId); + + final Time timeout = Time.milliseconds(configuration.getLong(WebOptions.TIMEOUT)); + + final CompletableFuture jobSubmissionFuture = submitJob( + dispatcherGateway, + blobServerAddress -> new BlobClient(blobServerAddress, configuration), + jobGraph, + timeout); + + final EmbeddedJobClient embeddedClient = new EmbeddedJobClient( + actualJobId, + dispatcherGateway, + timeout); + + return jobSubmissionFuture + .thenApplyAsync(jobID -> embeddedClient); + } + + private static CompletableFuture submitJob( + final DispatcherGateway dispatcherGateway, + final FunctionWithException blobClientCreator, + final JobGraph jobGraph, + final Time rpcTimeout) { + checkNotNull(blobClientCreator); + checkNotNull(jobGraph); + + LOG.info("Submitting Job with JobId={}.", jobGraph.getJobID()); + + return dispatcherGateway + .getBlobServerPort(rpcTimeout) + .thenApply(blobServerPort -> new InetSocketAddress(dispatcherGateway.getHostname(), blobServerPort)) + .thenCompose(blobServerAddress -> { + + try { + ClientUtils.extractAndUploadJobGraphFiles(jobGraph, () -> blobClientCreator.apply(blobServerAddress)); + } catch (FlinkException e) { + throw new CompletionException(e); + } + + return dispatcherGateway.submitJob(jobGraph, rpcTimeout); + }).thenApply(ack -> jobGraph.getJobID()); + } +} diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/application/executors/EmbeddedExecutorFactory.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/executors/EmbeddedExecutorFactory.java new file mode 100644 index 0000000000..1275e442e7 --- /dev/null +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/executors/EmbeddedExecutorFactory.java @@ -0,0 +1,73 @@ +/* + * 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.client.deployment.application.executors; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.PipelineExecutor; +import org.apache.flink.core.execution.PipelineExecutorFactory; +import org.apache.flink.runtime.dispatcher.DispatcherGateway; + +import java.util.Collection; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * An {@link PipelineExecutorFactory} for the {@link EmbeddedExecutor}. + */ +@Internal +public class EmbeddedExecutorFactory implements PipelineExecutorFactory { + + private final Collection submittedJobIds; + + private final DispatcherGateway dispatcherGateway; + + /** + * Creates an {@link EmbeddedExecutorFactory}. + * @param submittedJobIds a list that is going to be filled with the job ids of the + * new jobs that will be submitted. This is essentially used to return the submitted job ids + * to the caller. + * @param dispatcherGateway the dispatcher of the cluster which is going to be used to submit jobs. + */ + public EmbeddedExecutorFactory( + final Collection submittedJobIds, + final DispatcherGateway dispatcherGateway) { + this.submittedJobIds = checkNotNull(submittedJobIds); + this.dispatcherGateway = checkNotNull(dispatcherGateway); + } + + @Override + public String getName() { + return EmbeddedExecutor.NAME; + } + + @Override + public boolean isCompatibleWith(final Configuration configuration) { + // this is always false because we simply have a special executor loader + // for this one that does not check for compatibility. + return false; + } + + @Override + public PipelineExecutor getExecutor(final Configuration configuration) { + checkNotNull(configuration); + return new EmbeddedExecutor(submittedJobIds, dispatcherGateway); + } +} diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/application/executors/EmbeddedExecutorServiceLoader.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/executors/EmbeddedExecutorServiceLoader.java new file mode 100644 index 0000000000..f2e65d54cb --- /dev/null +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/executors/EmbeddedExecutorServiceLoader.java @@ -0,0 +1,68 @@ +/* + * 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.client.deployment.application.executors; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.PipelineExecutorFactory; +import org.apache.flink.core.execution.PipelineExecutorServiceLoader; +import org.apache.flink.runtime.dispatcher.DispatcherGateway; + +import java.util.Collection; +import java.util.stream.Stream; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A {@link PipelineExecutorServiceLoader} that always returns an {@link EmbeddedExecutorFactory}. + * This is useful when running the user's main on the cluster. + */ +@Internal +public class EmbeddedExecutorServiceLoader implements PipelineExecutorServiceLoader { + + private final Collection submittedJobIds; + + private final DispatcherGateway dispatcherGateway; + + + /** + * Creates an {@link EmbeddedExecutorServiceLoader}. + * @param submittedJobIds a list that is going to be filled by the {@link EmbeddedExecutor} with the job ids of the + * new jobs that will be submitted. This is essentially used to return the submitted job ids + * to the caller. + * @param dispatcherGateway the dispatcher of the cluster which is going to be used to submit jobs. + */ + public EmbeddedExecutorServiceLoader( + final Collection submittedJobIds, + final DispatcherGateway dispatcherGateway) { + this.submittedJobIds = checkNotNull(submittedJobIds); + this.dispatcherGateway = checkNotNull(dispatcherGateway); + } + + @Override + public PipelineExecutorFactory getExecutorFactory(final Configuration configuration) { + return new EmbeddedExecutorFactory(submittedJobIds, dispatcherGateway); + } + + @Override + public Stream getExecutorNames() { + return Stream.builder().add(EmbeddedExecutor.NAME).build(); + } +} diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/ExecutorUtils.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/ExecutorUtils.java index 4943d460a2..76fc9546c7 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/ExecutorUtils.java +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/ExecutorUtils.java @@ -18,10 +18,12 @@ package org.apache.flink.client.deployment.executors; +import org.apache.flink.api.common.JobID; import org.apache.flink.api.dag.Pipeline; import org.apache.flink.client.FlinkPipelineTranslationUtil; import org.apache.flink.client.cli.ExecutionConfigAccessor; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.PipelineOptionsInternal; import org.apache.flink.runtime.jobgraph.JobGraph; import javax.annotation.Nonnull; @@ -50,6 +52,10 @@ public class ExecutorUtils { final JobGraph jobGraph = FlinkPipelineTranslationUtil .getJobGraph(pipeline, configuration, executionConfigAccessor.getParallelism()); + configuration + .getOptional(PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID) + .ifPresent(strJobID -> jobGraph.setJobID(JobID.fromHexString(strJobID))); + jobGraph.addJars(executionConfigAccessor.getJars()); jobGraph.setClasspaths(executionConfigAccessor.getClasspaths()); jobGraph.setSavepointRestoreSettings(executionConfigAccessor.getSavepointRestoreSettings()); diff --git a/flink-core/src/main/java/org/apache/flink/configuration/PipelineOptionsInternal.java b/flink-core/src/main/java/org/apache/flink/configuration/PipelineOptionsInternal.java new file mode 100644 index 0000000000..617fafd22b --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/configuration/PipelineOptionsInternal.java @@ -0,0 +1,37 @@ +/* + * 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.configuration; + +import org.apache.flink.annotation.Internal; + +import static org.apache.flink.configuration.ConfigOptions.key; + +/** + * Pipeline options that are not meant to be used by the user. + */ +@Internal +public class PipelineOptionsInternal { + + public static final ConfigOption PIPELINE_FIXED_JOB_ID = + key("$internal.pipeline.job-id") + .stringType() + .noDefaultValue() + .withDescription("**DO NOT USE** The static JobId to be used for the specific pipeline. " + + "For fault-tolerance, this value needs to stay the same across runs."); +} diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebSubmissionExtension.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebSubmissionExtension.java index b676b4a150..6e4b96a305 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebSubmissionExtension.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebSubmissionExtension.java @@ -20,6 +20,7 @@ package org.apache.flink.runtime.webmonitor; import org.apache.flink.api.common.time.Time; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.client.deployment.application.DetachedApplicationRunner; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.dispatcher.DispatcherGateway; import org.apache.flink.runtime.rest.handler.RestHandlerSpecification; @@ -88,7 +89,8 @@ public class WebSubmissionExtension implements WebMonitorExtension { JarRunHeaders.getInstance(), jarDir, configuration, - executor); + executor, + DetachedApplicationRunner::new); final JarDeleteHandler jarDeleteHandler = new JarDeleteHandler( leaderRetriever, diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandler.java index d5b7248c4e..84a74bd536 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandler.java @@ -19,29 +19,29 @@ package org.apache.flink.runtime.webmonitor.handlers; import org.apache.flink.api.common.time.Time; +import org.apache.flink.client.deployment.application.ApplicationRunner; +import org.apache.flink.client.deployment.application.executors.EmbeddedExecutor; +import org.apache.flink.client.program.PackagedProgram; +import org.apache.flink.client.program.ProgramInvocationException; import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.blob.BlobClient; -import org.apache.flink.runtime.client.ClientUtils; +import org.apache.flink.configuration.DeploymentOptions; import org.apache.flink.runtime.dispatcher.DispatcherGateway; -import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; -import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.rest.handler.AbstractRestHandler; import org.apache.flink.runtime.rest.handler.HandlerRequest; import org.apache.flink.runtime.rest.handler.RestHandlerException; import org.apache.flink.runtime.rest.messages.MessageHeaders; import org.apache.flink.runtime.webmonitor.handlers.utils.JarHandlerUtils.JarHandlerContext; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; -import org.apache.flink.util.FlinkException; import javax.annotation.Nonnull; -import java.net.InetSocketAddress; import java.nio.file.Path; import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.Executor; +import java.util.function.Supplier; import static java.util.Objects.requireNonNull; import static org.apache.flink.runtime.rest.handler.util.HandlerRequestUtils.fromRequestBodyOrQueryParameter; @@ -58,6 +58,8 @@ public class JarRunHandler extends private final Configuration configuration; + private final ApplicationRunner applicationRunner; + private final Executor executor; public JarRunHandler( @@ -67,41 +69,40 @@ public class JarRunHandler extends final MessageHeaders messageHeaders, final Path jarDir, final Configuration configuration, - final Executor executor) { + final Executor executor, + final Supplier applicationRunnerSupplier) { super(leaderRetriever, timeout, responseHeaders, messageHeaders); this.jarDir = requireNonNull(jarDir); this.configuration = requireNonNull(configuration); this.executor = requireNonNull(executor); + + this.applicationRunner = applicationRunnerSupplier.get(); } @Override protected CompletableFuture handleRequest( @Nonnull final HandlerRequest request, @Nonnull final DispatcherGateway gateway) throws RestHandlerException { - final JarHandlerContext context = JarHandlerContext.fromRequest(request, jarDir, log); - - final SavepointRestoreSettings savepointRestoreSettings = getSavepointRestoreSettings(request); - - final CompletableFuture jobGraphFuture = getJobGraphAsync(context, savepointRestoreSettings); - - CompletableFuture blobServerPortFuture = gateway.getBlobServerPort(timeout); - CompletableFuture jarUploadFuture = jobGraphFuture.thenCombine(blobServerPortFuture, (jobGraph, blobServerPort) -> { - final InetSocketAddress address = new InetSocketAddress(gateway.getHostname(), blobServerPort); - try { - ClientUtils.extractAndUploadJobGraphFiles(jobGraph, () -> new BlobClient(address, configuration)); - } catch (FlinkException e) { - throw new CompletionException(e); - } + final Configuration effectiveConfiguration = new Configuration(configuration); + effectiveConfiguration.set(DeploymentOptions.ATTACHED, false); + effectiveConfiguration.set(DeploymentOptions.TARGET, EmbeddedExecutor.NAME); - return jobGraph; - }); - - CompletableFuture jobSubmissionFuture = jarUploadFuture.thenCompose(jobGraph -> gateway.submitJob(jobGraph, timeout)); - - return jobSubmissionFuture - .thenCombine(jarUploadFuture, (ack, jobGraph) -> new JarRunResponseBody(jobGraph.getJobID())); + final JarHandlerContext context = JarHandlerContext.fromRequest(request, jarDir, log); + context.applyToConfiguration(effectiveConfiguration); + SavepointRestoreSettings.toConfiguration(getSavepointRestoreSettings(request), effectiveConfiguration); + + final PackagedProgram program = context.toPackagedProgram(effectiveConfiguration); + + return CompletableFuture + .supplyAsync(() -> applicationRunner.run(gateway, program, effectiveConfiguration), executor) + .thenApply(jobIds -> { + if (jobIds.isEmpty()) { + throw new CompletionException(new ProgramInvocationException("No jobs submitted.")); + } + return new JarRunResponseBody(jobIds.get(0)); + }); } private SavepointRestoreSettings getSavepointRestoreSettings( @@ -130,14 +131,4 @@ public class JarRunHandler extends } return savepointRestoreSettings; } - - private CompletableFuture getJobGraphAsync( - JarHandlerContext context, - final SavepointRestoreSettings savepointRestoreSettings) { - return CompletableFuture.supplyAsync(() -> { - final JobGraph jobGraph = context.toJobGraph(configuration, false); - jobGraph.setSavepointRestoreSettings(savepointRestoreSettings); - return jobGraph; - }, executor); - } } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/utils/JarHandlerUtils.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/utils/JarHandlerUtils.java index 4b0da5cc58..a6576f0af3 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/utils/JarHandlerUtils.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/utils/JarHandlerUtils.java @@ -19,12 +19,15 @@ package org.apache.flink.runtime.webmonitor.handlers.utils; import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.client.program.PackagedProgram; import org.apache.flink.client.program.PackagedProgramUtils; import org.apache.flink.client.program.ProgramInvocationException; +import org.apache.flink.configuration.ConfigUtils; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.configuration.PipelineOptions; +import org.apache.flink.configuration.PipelineOptionsInternal; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.rest.handler.HandlerRequest; import org.apache.flink.runtime.rest.handler.RestHandlerException; @@ -43,6 +46,7 @@ import org.slf4j.Logger; import javax.annotation.Nonnull; import javax.annotation.Nullable; +import java.net.URL; import java.nio.file.Files; import java.nio.file.Path; import java.util.ArrayList; @@ -55,6 +59,7 @@ import java.util.regex.Pattern; import static org.apache.flink.runtime.rest.handler.util.HandlerRequestUtils.fromRequestBodyOrQueryParameter; import static org.apache.flink.runtime.rest.handler.util.HandlerRequestUtils.getQueryParameter; import static org.apache.flink.shaded.guava18.com.google.common.base.Strings.emptyToNull; +import static org.apache.flink.util.Preconditions.checkNotNull; /** * Utils for jar handlers. @@ -100,7 +105,7 @@ public class JarHandlerUtils { int parallelism = fromRequestBodyOrQueryParameter( requestBody.getParallelism(), () -> getQueryParameter(request, ParallelismQueryParameter.class), - ExecutionConfig.PARALLELISM_DEFAULT, + CoreOptions.DEFAULT_PARALLELISM.defaultValue(), log); JobID jobId = fromRequestBodyOrQueryParameter( @@ -112,20 +117,43 @@ public class JarHandlerUtils { return new JarHandlerContext(jarFile, entryClass, programArgs, parallelism, jobId); } + public void applyToConfiguration(final Configuration configuration) { + checkNotNull(configuration); + + if (jobId != null) { + configuration.set(PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID, jobId.toHexString()); + } + configuration.set(CoreOptions.DEFAULT_PARALLELISM, parallelism); + + final PackagedProgram program = toPackagedProgram(configuration); + ConfigUtils.encodeCollectionToConfig(configuration, PipelineOptions.JARS, program.getJobJarAndDependencies(), URL::toString); + ConfigUtils.encodeCollectionToConfig(configuration, PipelineOptions.CLASSPATHS, program.getClasspaths(), URL::toString); + } + public JobGraph toJobGraph(Configuration configuration, boolean suppressOutput) { + try { + final PackagedProgram packagedProgram = toPackagedProgram(configuration); + return PackagedProgramUtils.createJobGraph(packagedProgram, configuration, parallelism, jobId, suppressOutput); + } catch (final ProgramInvocationException e) { + throw new CompletionException(e); + } + } + + public PackagedProgram toPackagedProgram(Configuration configuration) { + checkNotNull(configuration); + if (!Files.exists(jarFile)) { throw new CompletionException(new RestHandlerException( - String.format("Jar file %s does not exist", jarFile), HttpResponseStatus.BAD_REQUEST)); + String.format("Jar file %s does not exist", jarFile), HttpResponseStatus.BAD_REQUEST)); } try { - final PackagedProgram packagedProgram = PackagedProgram.newBuilder() - .setJarFile(jarFile.toFile()) - .setEntryPointClassName(entryClass) - .setConfiguration(configuration) - .setArguments(programArgs.toArray(new String[0])) - .build(); - return PackagedProgramUtils.createJobGraph(packagedProgram, configuration, parallelism, jobId, suppressOutput); + return PackagedProgram.newBuilder() + .setJarFile(jarFile.toFile()) + .setEntryPointClassName(entryClass) + .setConfiguration(configuration) + .setArguments(programArgs.toArray(new String[0])) + .build(); } catch (final ProgramInvocationException e) { throw new CompletionException(e); } diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarHandlerTest.java index 33ada39f0c..a83e4cc7a7 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarHandlerTest.java @@ -49,22 +49,12 @@ public class JarHandlerTest extends TestLogger { @ClassRule public static final TemporaryFolder TMP = new TemporaryFolder(); - enum Type { - PLAN, - RUN - } - @Test public void testPlanJar() throws Exception { - runTest(Type.PLAN, "hello out!", "hello err!"); - } - - @Test - public void testRunJar() throws Exception { - runTest(Type.RUN, "(none)", "(none)"); + runTest("hello out!", "hello err!"); } - private static void runTest(Type type, String expectedCapturedStdOut, String expectedCapturedStdErr) throws Exception { + private static void runTest(String expectedCapturedStdOut, String expectedCapturedStdErr) throws Exception { final TestingDispatcherGateway restfulGateway = new TestingDispatcherGateway.Builder().build(); final JarHandlers handlers = new JarHandlers(TMP.newFolder().toPath(), restfulGateway); @@ -76,13 +66,7 @@ public class JarHandlerTest extends TestLogger { final String storedJarName = Paths.get(storedJarPath).getFileName().toString(); try { - switch (type) { - case RUN: - JarHandlers.runJar(handlers.runHandler, storedJarName, restfulGateway); - break; - case PLAN: - JarHandlers.showPlan(handlers.planHandler, storedJarName, restfulGateway); - } + JarHandlers.showPlan(handlers.planHandler, storedJarName, restfulGateway); Assert.fail("Should have failed with an exception."); } catch (Exception e) { Optional expected = ExceptionUtils.findThrowable(e, ProgramInvocationException.class); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarHandlers.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarHandlers.java index 8308a9c6ed..5788d80872 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarHandlers.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarHandlers.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.webmonitor.handlers; import org.apache.flink.api.common.time.Time; +import org.apache.flink.client.deployment.application.DetachedApplicationRunner; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.dispatcher.DispatcherGateway; import org.apache.flink.runtime.rest.handler.HandlerRequest; @@ -86,7 +87,8 @@ public class JarHandlers { JarRunHeaders.getInstance(), jarDir, new Configuration(), - executor); + executor, + DetachedApplicationRunner::new); deleteHandler = new JarDeleteHandler( gatewayRetriever, diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerParameterTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerParameterTest.java index 796c08b0b0..a7e29a4af6 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerParameterTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerParameterTest.java @@ -20,7 +20,12 @@ package org.apache.flink.runtime.webmonitor.handlers; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; +import org.apache.flink.client.deployment.application.DetachedApplicationRunner; +import org.apache.flink.client.deployment.application.executors.EmbeddedExecutor; +import org.apache.flink.client.program.PackagedProgram; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.DeploymentOptions; +import org.apache.flink.runtime.dispatcher.DispatcherGateway; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.runtime.rest.handler.HandlerRequest; @@ -40,6 +45,9 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; import java.util.stream.Collectors; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; + /** * Tests for the parameter handling of the {@link JarRunHandler}. */ @@ -64,7 +72,18 @@ public class JarRunHandlerParameterTest extends JarHandlerParameterTest run(DispatcherGateway dispatcherGateway, PackagedProgram program, Configuration configuration) { + assertFalse(configuration.get(DeploymentOptions.ATTACHED)); + assertEquals(EmbeddedExecutor.NAME, configuration.get(DeploymentOptions.TARGET)); + return super.run(dispatcherGateway, program, configuration); + } } @Override @@ -143,7 +162,7 @@ public class JarRunHandlerParameterTest extends JarHandlerParameterTest()); + env.execute(); } } diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/testutils/ParameterProgram.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/testutils/ParameterProgram.java index c96cce8a83..a8ac5a66e6 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/testutils/ParameterProgram.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/testutils/ParameterProgram.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.webmonitor.testutils; import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.io.DiscardingOutputFormat; /** * Simple test program that exposes passed arguments. @@ -31,6 +32,7 @@ public class ParameterProgram { actualArguments = args; ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.fromElements("hello", "world").print(); + env.fromElements("hello", "world").output(new DiscardingOutputFormat<>()); + env.execute(); } } -- Gitee From ab123f2438999384127b2cb401b50fcd441f1c1a Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Thu, 2 Apr 2020 15:47:10 +0200 Subject: [PATCH 455/885] [FLINK-16657] Allow the (Stream)ContenxtEnv to enforce single job execution --- .../org/apache/flink/client/ClientUtils.java | 9 ++- .../apache/flink/client/cli/CliFrontend.java | 2 +- .../DetachedApplicationRunner.java | 8 ++- .../client/program/ContextEnvironment.java | 26 ++++++++- .../program/StreamContextEnvironment.java | 26 ++++++++- .../flink/client/program/ClientTest.java | 57 +++++++++++++++++-- .../webmonitor/WebSubmissionExtension.java | 2 +- .../webmonitor/handlers/JarHandlers.java | 2 +- .../handlers/JarRunHandlerParameterTest.java | 4 ++ 9 files changed, 118 insertions(+), 18 deletions(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java b/flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java index f1b663ee74..3732c664b9 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java +++ b/flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java @@ -119,7 +119,8 @@ public enum ClientUtils { public static void executeProgram( PipelineExecutorServiceLoader executorServiceLoader, Configuration configuration, - PackagedProgram program) throws ProgramInvocationException { + PackagedProgram program, + boolean enforceSingleJobExecution) throws ProgramInvocationException { checkNotNull(executorServiceLoader); final ClassLoader userCodeClassLoader = program.getUserCodeClassLoader(); final ClassLoader contextClassLoader = Thread.currentThread().getContextClassLoader(); @@ -131,12 +132,14 @@ public enum ClientUtils { ContextEnvironment.setAsContext( executorServiceLoader, configuration, - userCodeClassLoader); + userCodeClassLoader, + enforceSingleJobExecution); StreamContextEnvironment.setAsContext( executorServiceLoader, configuration, - userCodeClassLoader); + userCodeClassLoader, + enforceSingleJobExecution); try { program.invokeInteractiveModeForExecution(); diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java index 55244c3fe9..9111a52527 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java @@ -659,7 +659,7 @@ public class CliFrontend { // -------------------------------------------------------------------------------------------- protected void executeProgram(final Configuration configuration, final PackagedProgram program) throws ProgramInvocationException { - ClientUtils.executeProgram(DefaultExecutorServiceLoader.INSTANCE, configuration, program); + ClientUtils.executeProgram(DefaultExecutorServiceLoader.INSTANCE, configuration, program, false); } /** diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/application/DetachedApplicationRunner.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/DetachedApplicationRunner.java index 8bc1a72e00..889052ae44 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/deployment/application/DetachedApplicationRunner.java +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/DetachedApplicationRunner.java @@ -53,6 +53,12 @@ public class DetachedApplicationRunner implements ApplicationRunner { private static final Logger LOG = LoggerFactory.getLogger(DetachedApplicationRunner.class); + private final boolean enforceSingleJobExecution; + + public DetachedApplicationRunner(final boolean enforceSingleJobExecution) { + this.enforceSingleJobExecution = enforceSingleJobExecution; + } + @Override public List run(final DispatcherGateway dispatcherGateway, final PackagedProgram program, final Configuration configuration) { checkNotNull(dispatcherGateway); @@ -69,7 +75,7 @@ public class DetachedApplicationRunner implements ApplicationRunner { new EmbeddedExecutorServiceLoader(applicationJobIds, dispatcherGateway); try { - ClientUtils.executeProgram(executorServiceLoader, configuration, program); + ClientUtils.executeProgram(executorServiceLoader, configuration, program, enforceSingleJobExecution); } catch (ProgramInvocationException e) { LOG.warn("Could not execute application: ", e); throw new FlinkRuntimeException("Could not execute application.", e); diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java b/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java index 5b7afb464d..cb51d3a5dc 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java @@ -27,6 +27,7 @@ import org.apache.flink.configuration.DeploymentOptions; import org.apache.flink.core.execution.DetachedJobExecutionResult; import org.apache.flink.core.execution.JobClient; import org.apache.flink.core.execution.PipelineExecutorServiceLoader; +import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.ShutdownHookUtil; import org.slf4j.Logger; @@ -42,11 +43,19 @@ public class ContextEnvironment extends ExecutionEnvironment { private static final Logger LOG = LoggerFactory.getLogger(ExecutionEnvironment.class); + private final boolean enforceSingleJobExecution; + + private int jobCounter; + public ContextEnvironment( final PipelineExecutorServiceLoader executorServiceLoader, final Configuration configuration, - final ClassLoader userCodeClassLoader) { + final ClassLoader userCodeClassLoader, + final boolean enforceSingleJobExecution) { super(executorServiceLoader, configuration, userCodeClassLoader); + this.enforceSingleJobExecution = enforceSingleJobExecution; + + this.jobCounter = 0; } @Override @@ -82,6 +91,8 @@ public class ContextEnvironment extends ExecutionEnvironment { @Override public JobClient executeAsync(String jobName) throws Exception { + validateAllowedExecution(); + final JobClient jobClient = super.executeAsync(jobName); System.out.println("Job has been submitted with JobID " + jobClient.getJobID()); @@ -89,6 +100,13 @@ public class ContextEnvironment extends ExecutionEnvironment { return jobClient; } + private void validateAllowedExecution() { + if (enforceSingleJobExecution && jobCounter > 0) { + throw new FlinkRuntimeException("Cannot have more than one execute() or executeAsync() call in a single environment."); + } + jobCounter++; + } + @Override public String toString() { return "Context Environment (parallelism = " + (getParallelism() == ExecutionConfig.PARALLELISM_DEFAULT ? "default" : getParallelism()) + ")"; @@ -99,11 +117,13 @@ public class ContextEnvironment extends ExecutionEnvironment { public static void setAsContext( final PipelineExecutorServiceLoader executorServiceLoader, final Configuration configuration, - final ClassLoader userCodeClassLoader) { + final ClassLoader userCodeClassLoader, + final boolean enforceSingleJobExecution) { ExecutionEnvironmentFactory factory = () -> new ContextEnvironment( executorServiceLoader, configuration, - userCodeClassLoader); + userCodeClassLoader, + enforceSingleJobExecution); initializeContextEnvironment(factory); } diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/StreamContextEnvironment.java b/flink-clients/src/main/java/org/apache/flink/client/program/StreamContextEnvironment.java index 1778202cd9..5942dc4e9b 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/StreamContextEnvironment.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/StreamContextEnvironment.java @@ -28,6 +28,7 @@ import org.apache.flink.core.execution.PipelineExecutorServiceLoader; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironmentFactory; import org.apache.flink.streaming.api.graph.StreamGraph; +import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.ShutdownHookUtil; import org.slf4j.Logger; @@ -46,11 +47,19 @@ public class StreamContextEnvironment extends StreamExecutionEnvironment { private static final Logger LOG = LoggerFactory.getLogger(ExecutionEnvironment.class); + private final boolean enforceSingleJobExecution; + + private int jobCounter; + public StreamContextEnvironment( final PipelineExecutorServiceLoader executorServiceLoader, final Configuration configuration, - final ClassLoader userCodeClassLoader) { + final ClassLoader userCodeClassLoader, + final boolean enforceSingleJobExecution) { super(executorServiceLoader, configuration, userCodeClassLoader); + this.enforceSingleJobExecution = enforceSingleJobExecution; + + this.jobCounter = 0; } @Override @@ -87,6 +96,8 @@ public class StreamContextEnvironment extends StreamExecutionEnvironment { @Override public JobClient executeAsync(StreamGraph streamGraph) throws Exception { + validateAllowedExecution(); + final JobClient jobClient = super.executeAsync(streamGraph); System.out.println("Job has been submitted with JobID " + jobClient.getJobID()); @@ -94,16 +105,25 @@ public class StreamContextEnvironment extends StreamExecutionEnvironment { return jobClient; } + private void validateAllowedExecution() { + if (enforceSingleJobExecution && jobCounter > 0) { + throw new FlinkRuntimeException("Cannot have more than one execute() or executeAsync() call in a single environment."); + } + jobCounter++; + } + // -------------------------------------------------------------------------------------------- public static void setAsContext( final PipelineExecutorServiceLoader executorServiceLoader, final Configuration configuration, - final ClassLoader userCodeClassLoader) { + final ClassLoader userCodeClassLoader, + final boolean enforceSingleJobExecution) { StreamExecutionEnvironmentFactory factory = () -> new StreamContextEnvironment( executorServiceLoader, configuration, - userCodeClassLoader); + userCodeClassLoader, + enforceSingleJobExecution); initializeContextEnvironment(factory); } diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java index 93248a1b86..7e1fabab7f 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java @@ -53,6 +53,7 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.testutils.MiniClusterResource; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.testutils.junit.category.AlsoRunWithLegacyScheduler; +import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.NetUtils; import org.apache.flink.util.TestLogger; @@ -133,7 +134,7 @@ public class ClientTest extends TestLogger { PackagedProgram prg = PackagedProgram.newBuilder().setEntryPointClassName(TestEager.class.getName()).build(); final Configuration configuration = fromPackagedProgram(prg, 1, true); - ClientUtils.executeProgram(new TestExecutorServiceLoader(clusterClient, plan), configuration, prg); + ClientUtils.executeProgram(new TestExecutorServiceLoader(clusterClient, plan), configuration, prg, false); fail(FAIL_MESSAGE); } catch (ProgramInvocationException e) { assertEquals( @@ -145,7 +146,7 @@ public class ClientTest extends TestLogger { PackagedProgram prg = PackagedProgram.newBuilder().setEntryPointClassName(TestGetRuntime.class.getName()).build(); final Configuration configuration = fromPackagedProgram(prg, 1, true); - ClientUtils.executeProgram(new TestExecutorServiceLoader(clusterClient, plan), configuration, prg); + ClientUtils.executeProgram(new TestExecutorServiceLoader(clusterClient, plan), configuration, prg, false); fail(FAIL_MESSAGE); } catch (ProgramInvocationException e) { assertEquals( @@ -157,7 +158,7 @@ public class ClientTest extends TestLogger { PackagedProgram prg = PackagedProgram.newBuilder().setEntryPointClassName(TestGetAccumulator.class.getName()).build(); final Configuration configuration = fromPackagedProgram(prg, 1, true); - ClientUtils.executeProgram(new TestExecutorServiceLoader(clusterClient, plan), configuration, prg); + ClientUtils.executeProgram(new TestExecutorServiceLoader(clusterClient, plan), configuration, prg, false); fail(FAIL_MESSAGE); } catch (ProgramInvocationException e) { assertEquals( @@ -169,7 +170,7 @@ public class ClientTest extends TestLogger { PackagedProgram prg = PackagedProgram.newBuilder().setEntryPointClassName(TestGetAllAccumulator.class.getName()).build(); final Configuration configuration = fromPackagedProgram(prg, 1, true); - ClientUtils.executeProgram(new TestExecutorServiceLoader(clusterClient, plan), configuration, prg); + ClientUtils.executeProgram(new TestExecutorServiceLoader(clusterClient, plan), configuration, prg, false); fail(FAIL_MESSAGE); } catch (ProgramInvocationException e) { assertEquals( @@ -178,6 +179,37 @@ public class ClientTest extends TestLogger { } } + @Test(expected = FlinkRuntimeException.class) + public void testMultiExecuteWithEnforcingSingleJobExecution() throws Throwable { + try { + launchMultiExecuteJob(true); + } catch (Exception e) { + if (e instanceof ProgramInvocationException) { + throw e.getCause(); + } + } + fail("Test should have failed due to multiple execute() calls."); + } + + @Test + public void testMultiExecuteWithoutEnforcingSingleJobExecution() throws ProgramInvocationException { + launchMultiExecuteJob(false); + } + + private void launchMultiExecuteJob(final boolean enforceSingleJobExecution) throws ProgramInvocationException { + try (final ClusterClient clusterClient = + new MiniClusterClient(new Configuration(), MINI_CLUSTER_RESOURCE.getMiniCluster())) { + + final PackagedProgram prg = PackagedProgram.newBuilder() + .setEntryPointClassName(TestMultiExecute.class.getName()) + .build(); + + final Configuration configuration = fromPackagedProgram(prg, 1, false); + + ClientUtils.executeProgram(new TestExecutorServiceLoader(clusterClient, plan), configuration, prg, enforceSingleJobExecution); + } + } + /** * This test verifies correct job submission messaging logic and plan translation calls. */ @@ -218,7 +250,7 @@ public class ClientTest extends TestLogger { try { final ClusterClient client = new MiniClusterClient(new Configuration(), MINI_CLUSTER_RESOURCE.getMiniCluster()); final Configuration configuration = fromPackagedProgram(packagedProgramMock, 1, true); - ClientUtils.executeProgram(new TestExecutorServiceLoader(client, plan), configuration, packagedProgramMock); + ClientUtils.executeProgram(new TestExecutorServiceLoader(client, plan), configuration, packagedProgramMock, false); fail("Creating the local execution environment should not be possible"); } catch (InvalidProgramException e) { @@ -295,6 +327,21 @@ public class ClientTest extends TestLogger { } } + /** + * Test job with multiple execute() calls. + */ + public static final class TestMultiExecute { + + public static void main(String[] args) throws Exception { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + for (int i = 0; i < 2; i++) { + env.fromElements(1, 2).output(new DiscardingOutputFormat<>()); + env.execute(); + } + } + } + /** * Test job that retrieves the net runtime from the {@link JobExecutionResult}. */ diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebSubmissionExtension.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebSubmissionExtension.java index 6e4b96a305..a9a2503fdc 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebSubmissionExtension.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebSubmissionExtension.java @@ -90,7 +90,7 @@ public class WebSubmissionExtension implements WebMonitorExtension { jarDir, configuration, executor, - DetachedApplicationRunner::new); + () -> new DetachedApplicationRunner(true)); final JarDeleteHandler jarDeleteHandler = new JarDeleteHandler( leaderRetriever, diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarHandlers.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarHandlers.java index 5788d80872..16488ccbf2 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarHandlers.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarHandlers.java @@ -88,7 +88,7 @@ public class JarHandlers { jarDir, new Configuration(), executor, - DetachedApplicationRunner::new); + () -> new DetachedApplicationRunner(true)); deleteHandler = new JarDeleteHandler( gatewayRetriever, diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerParameterTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerParameterTest.java index a7e29a4af6..13ba43c896 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerParameterTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerParameterTest.java @@ -78,6 +78,10 @@ public class JarRunHandlerParameterTest extends JarHandlerParameterTest run(DispatcherGateway dispatcherGateway, PackagedProgram program, Configuration configuration) { assertFalse(configuration.get(DeploymentOptions.ATTACHED)); -- Gitee From e6cea4f2af9ee487152cd4145af054bc60164fb5 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Fri, 3 Apr 2020 16:30:53 +0200 Subject: [PATCH 456/885] [FLINK-16657] Forbid blocking calls in JobClient when in Web Submission --- .../org/apache/flink/client/ClientUtils.java | 9 +- .../apache/flink/client/cli/CliFrontend.java | 2 +- .../DetachedOnlyJobClientAdapter.java | 85 +++++++++++++++++++ .../DetachedApplicationRunner.java | 9 +- .../client/program/ContextEnvironment.java | 17 +++- .../program/StreamContextEnvironment.java | 17 +++- .../flink/client/program/ClientTest.java | 44 +++++++--- .../webmonitor/WebSubmissionExtension.java | 2 +- .../webmonitor/handlers/JarHandlers.java | 2 +- .../handlers/JarRunHandlerParameterTest.java | 2 +- 10 files changed, 160 insertions(+), 29 deletions(-) create mode 100644 flink-clients/src/main/java/org/apache/flink/client/deployment/DetachedOnlyJobClientAdapter.java diff --git a/flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java b/flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java index 3732c664b9..55fdc25524 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java +++ b/flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java @@ -120,7 +120,8 @@ public enum ClientUtils { PipelineExecutorServiceLoader executorServiceLoader, Configuration configuration, PackagedProgram program, - boolean enforceSingleJobExecution) throws ProgramInvocationException { + boolean enforceSingleJobExecution, + boolean forbidBlockingJobClient) throws ProgramInvocationException { checkNotNull(executorServiceLoader); final ClassLoader userCodeClassLoader = program.getUserCodeClassLoader(); final ClassLoader contextClassLoader = Thread.currentThread().getContextClassLoader(); @@ -133,13 +134,15 @@ public enum ClientUtils { executorServiceLoader, configuration, userCodeClassLoader, - enforceSingleJobExecution); + enforceSingleJobExecution, + forbidBlockingJobClient); StreamContextEnvironment.setAsContext( executorServiceLoader, configuration, userCodeClassLoader, - enforceSingleJobExecution); + enforceSingleJobExecution, + forbidBlockingJobClient); try { program.invokeInteractiveModeForExecution(); diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java index 9111a52527..91e0f001c9 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java @@ -659,7 +659,7 @@ public class CliFrontend { // -------------------------------------------------------------------------------------------- protected void executeProgram(final Configuration configuration, final PackagedProgram program) throws ProgramInvocationException { - ClientUtils.executeProgram(DefaultExecutorServiceLoader.INSTANCE, configuration, program, false); + ClientUtils.executeProgram(DefaultExecutorServiceLoader.INSTANCE, configuration, program, false, false); } /** diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/DetachedOnlyJobClientAdapter.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/DetachedOnlyJobClientAdapter.java new file mode 100644 index 0000000000..977b00447d --- /dev/null +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/DetachedOnlyJobClientAdapter.java @@ -0,0 +1,85 @@ +/* + * 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.client.deployment; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.util.FlinkRuntimeException; + +import javax.annotation.Nullable; + +import java.util.Map; +import java.util.concurrent.CompletableFuture; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A {@link JobClient} that wraps any other job client and transforms it into one that is not allowed + * to wait for the job result. + * + *

    This is used in web submission, where we do not want the Web UI to have jobs blocking threads while + * waiting for their completion. + */ +@Internal +public class DetachedOnlyJobClientAdapter implements JobClient { + + private final JobClient jobClient; + + public DetachedOnlyJobClientAdapter(final JobClient jobClient) { + this.jobClient = checkNotNull(jobClient); + } + + @Override + public JobID getJobID() { + return jobClient.getJobID(); + } + + @Override + public CompletableFuture getJobStatus() { + throw new FlinkRuntimeException("The Job Status cannot be requested when in Web Submission."); + } + + @Override + public CompletableFuture cancel() { + throw new FlinkRuntimeException("Cancelling the job is not supported by the Job Client when in Web Submission."); + } + + @Override + public CompletableFuture stopWithSavepoint(boolean advanceToEndOfEventTime, @Nullable String savepointDirectory) { + throw new FlinkRuntimeException("Stop with Savepoint is not supported by the Job Client when in Web Submission."); + } + + @Override + public CompletableFuture triggerSavepoint(@Nullable String savepointDirectory) { + throw new FlinkRuntimeException("A savepoint cannot be taken through the Job Client when in Web Submission."); + } + + @Override + public CompletableFuture> getAccumulators(ClassLoader classLoader) { + throw new FlinkRuntimeException("The Accumulators cannot be fetched through the Job Client when in Web Submission."); + } + + @Override + public CompletableFuture getJobExecutionResult(ClassLoader userClassloader) { + throw new FlinkRuntimeException("The Job Result cannot be fetched through the Job Client when in Web Submission."); + } +} diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/application/DetachedApplicationRunner.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/DetachedApplicationRunner.java index 889052ae44..c40e009cd6 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/deployment/application/DetachedApplicationRunner.java +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/DetachedApplicationRunner.java @@ -53,10 +53,15 @@ public class DetachedApplicationRunner implements ApplicationRunner { private static final Logger LOG = LoggerFactory.getLogger(DetachedApplicationRunner.class); + private final boolean forbidBlockingJobClient; + private final boolean enforceSingleJobExecution; - public DetachedApplicationRunner(final boolean enforceSingleJobExecution) { + public DetachedApplicationRunner( + final boolean enforceSingleJobExecution, + final boolean forbidBlockingJobClient) { this.enforceSingleJobExecution = enforceSingleJobExecution; + this.forbidBlockingJobClient = forbidBlockingJobClient; } @Override @@ -75,7 +80,7 @@ public class DetachedApplicationRunner implements ApplicationRunner { new EmbeddedExecutorServiceLoader(applicationJobIds, dispatcherGateway); try { - ClientUtils.executeProgram(executorServiceLoader, configuration, program, enforceSingleJobExecution); + ClientUtils.executeProgram(executorServiceLoader, configuration, program, enforceSingleJobExecution, forbidBlockingJobClient); } catch (ProgramInvocationException e) { LOG.warn("Could not execute application: ", e); throw new FlinkRuntimeException("Could not execute application.", e); diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java b/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java index cb51d3a5dc..5c9097d945 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.ExecutionEnvironmentFactory; +import org.apache.flink.client.deployment.DetachedOnlyJobClientAdapter; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.DeploymentOptions; import org.apache.flink.core.execution.DetachedJobExecutionResult; @@ -43,6 +44,8 @@ public class ContextEnvironment extends ExecutionEnvironment { private static final Logger LOG = LoggerFactory.getLogger(ExecutionEnvironment.class); + private final boolean forbidBlockingJobClient; + private final boolean enforceSingleJobExecution; private int jobCounter; @@ -51,8 +54,10 @@ public class ContextEnvironment extends ExecutionEnvironment { final PipelineExecutorServiceLoader executorServiceLoader, final Configuration configuration, final ClassLoader userCodeClassLoader, - final boolean enforceSingleJobExecution) { + final boolean enforceSingleJobExecution, + final boolean forbidBlockingJobClient) { super(executorServiceLoader, configuration, userCodeClassLoader); + this.forbidBlockingJobClient = forbidBlockingJobClient; this.enforceSingleJobExecution = enforceSingleJobExecution; this.jobCounter = 0; @@ -97,7 +102,9 @@ public class ContextEnvironment extends ExecutionEnvironment { System.out.println("Job has been submitted with JobID " + jobClient.getJobID()); - return jobClient; + return forbidBlockingJobClient + ? new DetachedOnlyJobClientAdapter(jobClient) + : jobClient; } private void validateAllowedExecution() { @@ -118,12 +125,14 @@ public class ContextEnvironment extends ExecutionEnvironment { final PipelineExecutorServiceLoader executorServiceLoader, final Configuration configuration, final ClassLoader userCodeClassLoader, - final boolean enforceSingleJobExecution) { + final boolean enforceSingleJobExecution, + final boolean disallowBlockingJobClient) { ExecutionEnvironmentFactory factory = () -> new ContextEnvironment( executorServiceLoader, configuration, userCodeClassLoader, - enforceSingleJobExecution); + enforceSingleJobExecution, + disallowBlockingJobClient); initializeContextEnvironment(factory); } diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/StreamContextEnvironment.java b/flink-clients/src/main/java/org/apache/flink/client/program/StreamContextEnvironment.java index 5942dc4e9b..87048c28ce 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/StreamContextEnvironment.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/StreamContextEnvironment.java @@ -20,6 +20,7 @@ package org.apache.flink.client.program; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.client.deployment.DetachedOnlyJobClientAdapter; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.DeploymentOptions; import org.apache.flink.core.execution.DetachedJobExecutionResult; @@ -47,6 +48,8 @@ public class StreamContextEnvironment extends StreamExecutionEnvironment { private static final Logger LOG = LoggerFactory.getLogger(ExecutionEnvironment.class); + private final boolean forbidBlockingJobClient; + private final boolean enforceSingleJobExecution; private int jobCounter; @@ -55,8 +58,10 @@ public class StreamContextEnvironment extends StreamExecutionEnvironment { final PipelineExecutorServiceLoader executorServiceLoader, final Configuration configuration, final ClassLoader userCodeClassLoader, - final boolean enforceSingleJobExecution) { + final boolean enforceSingleJobExecution, + final boolean forbidBlockingJobClient) { super(executorServiceLoader, configuration, userCodeClassLoader); + this.forbidBlockingJobClient = forbidBlockingJobClient; this.enforceSingleJobExecution = enforceSingleJobExecution; this.jobCounter = 0; @@ -102,7 +107,9 @@ public class StreamContextEnvironment extends StreamExecutionEnvironment { System.out.println("Job has been submitted with JobID " + jobClient.getJobID()); - return jobClient; + return forbidBlockingJobClient + ? new DetachedOnlyJobClientAdapter(jobClient) + : jobClient; } private void validateAllowedExecution() { @@ -118,12 +125,14 @@ public class StreamContextEnvironment extends StreamExecutionEnvironment { final PipelineExecutorServiceLoader executorServiceLoader, final Configuration configuration, final ClassLoader userCodeClassLoader, - final boolean enforceSingleJobExecution) { + final boolean enforceSingleJobExecution, + final boolean disallowBlockingJobClient) { StreamExecutionEnvironmentFactory factory = () -> new StreamContextEnvironment( executorServiceLoader, configuration, userCodeClassLoader, - enforceSingleJobExecution); + enforceSingleJobExecution, + disallowBlockingJobClient); initializeContextEnvironment(factory); } diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java index 7e1fabab7f..ada7dd122e 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java @@ -41,6 +41,7 @@ import org.apache.flink.configuration.DeploymentOptions; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.PipelineOptions; import org.apache.flink.core.execution.DetachedJobExecutionResult; +import org.apache.flink.core.execution.JobClient; import org.apache.flink.core.execution.PipelineExecutor; import org.apache.flink.core.execution.PipelineExecutorFactory; import org.apache.flink.core.execution.PipelineExecutorServiceLoader; @@ -134,7 +135,7 @@ public class ClientTest extends TestLogger { PackagedProgram prg = PackagedProgram.newBuilder().setEntryPointClassName(TestEager.class.getName()).build(); final Configuration configuration = fromPackagedProgram(prg, 1, true); - ClientUtils.executeProgram(new TestExecutorServiceLoader(clusterClient, plan), configuration, prg, false); + ClientUtils.executeProgram(new TestExecutorServiceLoader(clusterClient, plan), configuration, prg, false, false); fail(FAIL_MESSAGE); } catch (ProgramInvocationException e) { assertEquals( @@ -146,7 +147,7 @@ public class ClientTest extends TestLogger { PackagedProgram prg = PackagedProgram.newBuilder().setEntryPointClassName(TestGetRuntime.class.getName()).build(); final Configuration configuration = fromPackagedProgram(prg, 1, true); - ClientUtils.executeProgram(new TestExecutorServiceLoader(clusterClient, plan), configuration, prg, false); + ClientUtils.executeProgram(new TestExecutorServiceLoader(clusterClient, plan), configuration, prg, false, false); fail(FAIL_MESSAGE); } catch (ProgramInvocationException e) { assertEquals( @@ -158,7 +159,7 @@ public class ClientTest extends TestLogger { PackagedProgram prg = PackagedProgram.newBuilder().setEntryPointClassName(TestGetAccumulator.class.getName()).build(); final Configuration configuration = fromPackagedProgram(prg, 1, true); - ClientUtils.executeProgram(new TestExecutorServiceLoader(clusterClient, plan), configuration, prg, false); + ClientUtils.executeProgram(new TestExecutorServiceLoader(clusterClient, plan), configuration, prg, false, false); fail(FAIL_MESSAGE); } catch (ProgramInvocationException e) { assertEquals( @@ -170,7 +171,7 @@ public class ClientTest extends TestLogger { PackagedProgram prg = PackagedProgram.newBuilder().setEntryPointClassName(TestGetAllAccumulator.class.getName()).build(); final Configuration configuration = fromPackagedProgram(prg, 1, true); - ClientUtils.executeProgram(new TestExecutorServiceLoader(clusterClient, plan), configuration, prg, false); + ClientUtils.executeProgram(new TestExecutorServiceLoader(clusterClient, plan), configuration, prg, false, false); fail(FAIL_MESSAGE); } catch (ProgramInvocationException e) { assertEquals( @@ -182,7 +183,7 @@ public class ClientTest extends TestLogger { @Test(expected = FlinkRuntimeException.class) public void testMultiExecuteWithEnforcingSingleJobExecution() throws Throwable { try { - launchMultiExecuteJob(true); + launchMultiExecuteJob(true, false); } catch (Exception e) { if (e instanceof ProgramInvocationException) { throw e.getCause(); @@ -193,20 +194,37 @@ public class ClientTest extends TestLogger { @Test public void testMultiExecuteWithoutEnforcingSingleJobExecution() throws ProgramInvocationException { - launchMultiExecuteJob(false); + launchMultiExecuteJob(false, false); } - private void launchMultiExecuteJob(final boolean enforceSingleJobExecution) throws ProgramInvocationException { + @Test(expected = FlinkRuntimeException.class) + public void testMultiExecuteWithDisallowingToWaitForResult() throws Throwable { + try { + launchMultiExecuteJob(false, true); + } catch (Exception e) { + if (e instanceof ProgramInvocationException) { + throw e.getCause(); + } + } + fail("Test should have failed due to trying to fetch the job result via the JobClient."); + } + + private void launchMultiExecuteJob(final boolean enforceSingleJobExecution, final boolean forbidBlockingJobClient) throws ProgramInvocationException { try (final ClusterClient clusterClient = new MiniClusterClient(new Configuration(), MINI_CLUSTER_RESOURCE.getMiniCluster())) { - final PackagedProgram prg = PackagedProgram.newBuilder() + final PackagedProgram program = PackagedProgram.newBuilder() .setEntryPointClassName(TestMultiExecute.class.getName()) .build(); - final Configuration configuration = fromPackagedProgram(prg, 1, false); + final Configuration configuration = fromPackagedProgram(program, 1, false); - ClientUtils.executeProgram(new TestExecutorServiceLoader(clusterClient, plan), configuration, prg, enforceSingleJobExecution); + ClientUtils.executeProgram( + new TestExecutorServiceLoader(clusterClient, plan), + configuration, + program, + enforceSingleJobExecution, + forbidBlockingJobClient); } } @@ -250,7 +268,7 @@ public class ClientTest extends TestLogger { try { final ClusterClient client = new MiniClusterClient(new Configuration(), MINI_CLUSTER_RESOURCE.getMiniCluster()); final Configuration configuration = fromPackagedProgram(packagedProgramMock, 1, true); - ClientUtils.executeProgram(new TestExecutorServiceLoader(client, plan), configuration, packagedProgramMock, false); + ClientUtils.executeProgram(new TestExecutorServiceLoader(client, plan), configuration, packagedProgramMock, false, false); fail("Creating the local execution environment should not be possible"); } catch (InvalidProgramException e) { @@ -337,7 +355,9 @@ public class ClientTest extends TestLogger { for (int i = 0; i < 2; i++) { env.fromElements(1, 2).output(new DiscardingOutputFormat<>()); - env.execute(); + JobClient jc = env.executeAsync(); + + jc.getJobExecutionResult(TestMultiExecute.class.getClassLoader()); } } } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebSubmissionExtension.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebSubmissionExtension.java index a9a2503fdc..97114a63dc 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebSubmissionExtension.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebSubmissionExtension.java @@ -90,7 +90,7 @@ public class WebSubmissionExtension implements WebMonitorExtension { jarDir, configuration, executor, - () -> new DetachedApplicationRunner(true)); + () -> new DetachedApplicationRunner(true, true)); final JarDeleteHandler jarDeleteHandler = new JarDeleteHandler( leaderRetriever, diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarHandlers.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarHandlers.java index 16488ccbf2..02ab498c67 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarHandlers.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarHandlers.java @@ -88,7 +88,7 @@ public class JarHandlers { jarDir, new Configuration(), executor, - () -> new DetachedApplicationRunner(true)); + () -> new DetachedApplicationRunner(true, true)); deleteHandler = new JarDeleteHandler( gatewayRetriever, diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerParameterTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerParameterTest.java index 13ba43c896..1e6d942ce2 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerParameterTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerParameterTest.java @@ -79,7 +79,7 @@ public class JarRunHandlerParameterTest extends JarHandlerParameterTest Date: Fri, 3 Apr 2020 21:24:28 +0200 Subject: [PATCH 457/885] [minor] Rename ExecutorUtils to PipelineExecutorUtils --- .../deployment/application/executors/EmbeddedExecutor.java | 4 ++-- .../deployment/executors/AbstractJobClusterExecutor.java | 2 +- .../deployment/executors/AbstractSessionClusterExecutor.java | 2 +- .../{ExecutorUtils.java => PipelineExecutorUtils.java} | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) rename flink-clients/src/main/java/org/apache/flink/client/deployment/executors/{ExecutorUtils.java => PipelineExecutorUtils.java} (98%) diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/application/executors/EmbeddedExecutor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/executors/EmbeddedExecutor.java index 50b494d432..3552b4c5ac 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/deployment/application/executors/EmbeddedExecutor.java +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/executors/EmbeddedExecutor.java @@ -23,7 +23,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; import org.apache.flink.api.dag.Pipeline; import org.apache.flink.client.deployment.application.EmbeddedJobClient; -import org.apache.flink.client.deployment.executors.ExecutorUtils; +import org.apache.flink.client.deployment.executors.PipelineExecutorUtils; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.WebOptions; import org.apache.flink.core.execution.JobClient; @@ -81,7 +81,7 @@ public class EmbeddedExecutor implements PipelineExecutor { checkNotNull(pipeline); checkNotNull(configuration); - final JobGraph jobGraph = ExecutorUtils.getJobGraph(pipeline, configuration); + final JobGraph jobGraph = PipelineExecutorUtils.getJobGraph(pipeline, configuration); final JobID actualJobId = jobGraph.getJobID(); this.submittedJobIds.add(actualJobId); diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/AbstractJobClusterExecutor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/AbstractJobClusterExecutor.java index d8ccfc47a9..60254f9637 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/AbstractJobClusterExecutor.java +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/AbstractJobClusterExecutor.java @@ -59,7 +59,7 @@ public class AbstractJobClusterExecutor execute(@Nonnull final Pipeline pipeline, @Nonnull final Configuration configuration) throws Exception { - final JobGraph jobGraph = ExecutorUtils.getJobGraph(pipeline, configuration); + final JobGraph jobGraph = PipelineExecutorUtils.getJobGraph(pipeline, configuration); try (final ClusterDescriptor clusterDescriptor = clusterClientFactory.createClusterDescriptor(configuration)) { final ExecutionConfigAccessor configAccessor = ExecutionConfigAccessor.fromConfiguration(configuration); diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/AbstractSessionClusterExecutor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/AbstractSessionClusterExecutor.java index cd07f9299a..6445fb6e2a 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/AbstractSessionClusterExecutor.java +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/AbstractSessionClusterExecutor.java @@ -54,7 +54,7 @@ public class AbstractSessionClusterExecutor execute(@Nonnull final Pipeline pipeline, @Nonnull final Configuration configuration) throws Exception { - final JobGraph jobGraph = ExecutorUtils.getJobGraph(pipeline, configuration); + final JobGraph jobGraph = PipelineExecutorUtils.getJobGraph(pipeline, configuration); try (final ClusterDescriptor clusterDescriptor = clusterClientFactory.createClusterDescriptor(configuration)) { final ClusterID clusterID = clusterClientFactory.getClusterId(configuration); diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/ExecutorUtils.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/PipelineExecutorUtils.java similarity index 98% rename from flink-clients/src/main/java/org/apache/flink/client/deployment/executors/ExecutorUtils.java rename to flink-clients/src/main/java/org/apache/flink/client/deployment/executors/PipelineExecutorUtils.java index 76fc9546c7..7fec53da50 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/ExecutorUtils.java +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/executors/PipelineExecutorUtils.java @@ -33,7 +33,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull; /** * Utility class with method related to job execution. */ -public class ExecutorUtils { +public class PipelineExecutorUtils { /** * Creates the {@link JobGraph} corresponding to the provided {@link Pipeline}. -- Gitee From d3c149d9f6b4891b4b33a90576d616a8eac35c0b Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Tue, 7 Apr 2020 10:58:49 +0800 Subject: [PATCH 458/885] [FLINK-16912][parquet] Introduce table row write support for parquet writer This closes #11602 --- .../parquet/row/ParquetRowDataBuilder.java | 155 +++++++++ .../parquet/row/ParquetRowDataWriter.java | 325 ++++++++++++++++++ .../parquet/utils/ParquetSchemaConverter.java | 86 +++++ .../utils/SerializableConfiguration.java | 55 +++ .../parquet/row/ParquetRowDataWriterTest.java | 166 +++++++++ 5 files changed, 787 insertions(+) create mode 100644 flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/row/ParquetRowDataBuilder.java create mode 100644 flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/row/ParquetRowDataWriter.java create mode 100644 flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/utils/SerializableConfiguration.java create mode 100644 flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/row/ParquetRowDataWriterTest.java diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/row/ParquetRowDataBuilder.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/row/ParquetRowDataBuilder.java new file mode 100644 index 0000000000..e8399f09d7 --- /dev/null +++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/row/ParquetRowDataBuilder.java @@ -0,0 +1,155 @@ +/* + * 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.formats.parquet.row; + +import org.apache.flink.api.common.serialization.BulkWriter; +import org.apache.flink.formats.parquet.ParquetBuilder; +import org.apache.flink.formats.parquet.ParquetWriterFactory; +import org.apache.flink.formats.parquet.utils.SerializableConfiguration; +import org.apache.flink.table.dataformat.BaseRow; +import org.apache.flink.table.types.logical.RowType; + +import org.apache.hadoop.conf.Configuration; +import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.hadoop.api.WriteSupport; +import org.apache.parquet.io.OutputFile; +import org.apache.parquet.io.api.RecordConsumer; +import org.apache.parquet.schema.MessageType; + +import java.io.IOException; +import java.util.HashMap; + +import static org.apache.flink.formats.parquet.utils.ParquetSchemaConverter.convertToParquetMessageType; +import static org.apache.parquet.hadoop.ParquetOutputFormat.MAX_PADDING_BYTES; +import static org.apache.parquet.hadoop.ParquetOutputFormat.getBlockSize; +import static org.apache.parquet.hadoop.ParquetOutputFormat.getDictionaryPageSize; +import static org.apache.parquet.hadoop.ParquetOutputFormat.getEnableDictionary; +import static org.apache.parquet.hadoop.ParquetOutputFormat.getPageSize; +import static org.apache.parquet.hadoop.ParquetOutputFormat.getValidation; +import static org.apache.parquet.hadoop.ParquetOutputFormat.getWriterVersion; +import static org.apache.parquet.hadoop.codec.CodecConfig.getParquetCompressionCodec; + +/** + * {@link BaseRow} of {@link ParquetWriter.Builder}. + */ +public class ParquetRowDataBuilder extends ParquetWriter.Builder { + + private final RowType rowType; + private final boolean utcTimestamp; + + public ParquetRowDataBuilder( + OutputFile path, + RowType rowType, + boolean utcTimestamp) { + super(path); + this.rowType = rowType; + this.utcTimestamp = utcTimestamp; + } + + @Override + protected ParquetRowDataBuilder self() { + return this; + } + + @Override + protected WriteSupport getWriteSupport(Configuration conf) { + return new ParquetWriteSupport(); + } + + private class ParquetWriteSupport extends WriteSupport { + + private MessageType schema = convertToParquetMessageType("flink_schema", rowType); + private ParquetRowDataWriter writer; + + @Override + public WriteContext init(Configuration configuration) { + return new WriteContext(schema, new HashMap<>()); + } + + @Override + public void prepareForWrite(RecordConsumer recordConsumer) { + this.writer = new ParquetRowDataWriter( + recordConsumer, + rowType, + schema, + utcTimestamp); + } + + @Override + public void write(BaseRow record) { + try { + this.writer.write(record); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + } + + /** + * Create a parquet {@link BulkWriter.Factory}. + * + * @param rowType row type of parquet table. + * @param conf hadoop configuration. + * @param utcTimestamp Use UTC timezone or local timezone to the conversion between epoch time + * and LocalDateTime. Hive 0.x/1.x/2.x use local timezone. But Hive 3.x + * use UTC timezone. + */ + public static ParquetWriterFactory createWriterFactory( + RowType rowType, + Configuration conf, + boolean utcTimestamp) { + return new ParquetWriterFactory<>( + new FlinkParquetBuilder(rowType, conf, utcTimestamp)); + } + + /** + * Flink Row {@link ParquetBuilder}. + */ + public static class FlinkParquetBuilder implements ParquetBuilder { + + private final RowType rowType; + private final SerializableConfiguration configuration; + private final boolean utcTimestamp; + + public FlinkParquetBuilder( + RowType rowType, + Configuration conf, + boolean utcTimestamp) { + this.rowType = rowType; + this.configuration = new SerializableConfiguration(conf); + this.utcTimestamp = utcTimestamp; + } + + @Override + public ParquetWriter createWriter(OutputFile out) throws IOException { + Configuration conf = configuration.conf(); + return new ParquetRowDataBuilder(out, rowType, utcTimestamp) + .withCompressionCodec(getParquetCompressionCodec(conf)) + .withRowGroupSize(getBlockSize(conf)) + .withPageSize(getPageSize(conf)) + .withDictionaryPageSize(getDictionaryPageSize(conf)) + .withMaxPaddingSize(conf.getInt( + MAX_PADDING_BYTES, ParquetWriter.MAX_PADDING_SIZE_DEFAULT)) + .withDictionaryEncoding(getEnableDictionary(conf)) + .withValidation(getValidation(conf)) + .withWriterVersion(getWriterVersion(conf)) + .withConf(conf).build(); + } + } +} diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/row/ParquetRowDataWriter.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/row/ParquetRowDataWriter.java new file mode 100644 index 0000000000..13d578398e --- /dev/null +++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/row/ParquetRowDataWriter.java @@ -0,0 +1,325 @@ +/* + * 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.formats.parquet.row; + +import org.apache.flink.table.dataformat.BaseRow; +import org.apache.flink.table.dataformat.Decimal; +import org.apache.flink.table.dataformat.SqlTimestamp; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.LocalZonedTimestampType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.TimestampType; +import org.apache.flink.util.Preconditions; + +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.io.api.RecordConsumer; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.Type; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.sql.Timestamp; +import java.util.Arrays; + +import static org.apache.flink.formats.parquet.utils.ParquetSchemaConverter.computeMinBytesForDecimalPrecision; +import static org.apache.flink.formats.parquet.vector.reader.TimestampColumnReader.JULIAN_EPOCH_OFFSET_DAYS; +import static org.apache.flink.formats.parquet.vector.reader.TimestampColumnReader.MILLIS_IN_DAY; +import static org.apache.flink.formats.parquet.vector.reader.TimestampColumnReader.NANOS_PER_MILLISECOND; +import static org.apache.flink.formats.parquet.vector.reader.TimestampColumnReader.NANOS_PER_SECOND; + +/** + * Writes a record to the Parquet API with the expected schema in order to be written to a file. + */ +public class ParquetRowDataWriter { + + private final RecordConsumer recordConsumer; + private final boolean utcTimestamp; + + private final FieldWriter[] filedWriters; + private final String[] fieldNames; + + public ParquetRowDataWriter( + RecordConsumer recordConsumer, + RowType rowType, + GroupType schema, + boolean utcTimestamp) { + this.recordConsumer = recordConsumer; + this.utcTimestamp = utcTimestamp; + + this.filedWriters = new FieldWriter[rowType.getFieldCount()]; + this.fieldNames = rowType.getFieldNames().toArray(new String[0]); + for (int i = 0; i < rowType.getFieldCount(); i++) { + this.filedWriters[i] = createWriter(rowType.getTypeAt(i), schema.getType(i)); + } + } + + /** + * It writes a record to Parquet. + * + * @param record Contains the record that is going to be written. + */ + public void write(final BaseRow record) { + recordConsumer.startMessage(); + for (int i = 0; i < filedWriters.length; i++) { + if (!record.isNullAt(i)) { + String fieldName = fieldNames[i]; + FieldWriter writer = filedWriters[i]; + + recordConsumer.startField(fieldName, i); + writer.write(record, i); + recordConsumer.endField(fieldName, i); + } + } + recordConsumer.endMessage(); + } + + private FieldWriter createWriter(LogicalType t, Type type) { + if (type.isPrimitive()) { + switch (t.getTypeRoot()) { + case CHAR: + case VARCHAR: + return new StringWriter(); + case BOOLEAN: + return new BooleanWriter(); + case BINARY: + case VARBINARY: + return new BinaryWriter(); + case DECIMAL: + DecimalType decimalType = (DecimalType) t; + return createDecimalWriter(decimalType.getPrecision(), decimalType.getScale()); + case TINYINT: + return new ByteWriter(); + case SMALLINT: + return new ShortWriter(); + case DATE: + case TIME_WITHOUT_TIME_ZONE: + case INTEGER: + return new IntWriter(); + case BIGINT: + return new LongWriter(); + case FLOAT: + return new FloatWriter(); + case DOUBLE: + return new DoubleWriter(); + case TIMESTAMP_WITHOUT_TIME_ZONE: + TimestampType timestampType = (TimestampType) t; + return new TimestampWriter(timestampType.getPrecision()); + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + LocalZonedTimestampType localZonedTimestampType = (LocalZonedTimestampType) t; + return new TimestampWriter(localZonedTimestampType.getPrecision()); + default: + throw new UnsupportedOperationException("Unsupported type: " + type); + } + } else { + throw new IllegalArgumentException("Unsupported data type: " + t); + } + } + + private interface FieldWriter { + + void write(BaseRow row, int ordinal); + } + + private class BooleanWriter implements FieldWriter { + + @Override + public void write(BaseRow row, int ordinal) { + recordConsumer.addBoolean(row.getBoolean(ordinal)); + } + } + + private class ByteWriter implements FieldWriter { + + @Override + public void write(BaseRow row, int ordinal) { + recordConsumer.addInteger(row.getByte(ordinal)); + } + } + + private class ShortWriter implements FieldWriter { + + @Override + public void write(BaseRow row, int ordinal) { + recordConsumer.addInteger(row.getShort(ordinal)); + } + } + + private class LongWriter implements FieldWriter { + + @Override + public void write(BaseRow row, int ordinal) { + recordConsumer.addLong(row.getLong(ordinal)); + } + } + + private class FloatWriter implements FieldWriter { + + @Override + public void write(BaseRow row, int ordinal) { + recordConsumer.addFloat(row.getFloat(ordinal)); + } + } + + private class DoubleWriter implements FieldWriter { + + @Override + public void write(BaseRow row, int ordinal) { + recordConsumer.addDouble(row.getDouble(ordinal)); + } + } + + private class StringWriter implements FieldWriter { + + @Override + public void write(BaseRow row, int ordinal) { + recordConsumer.addBinary( + Binary.fromReusedByteArray(row.getString(ordinal).getBytes())); + } + } + + private class BinaryWriter implements FieldWriter { + + @Override + public void write(BaseRow row, int ordinal) { + recordConsumer.addBinary( + Binary.fromReusedByteArray(row.getBinary(ordinal))); + } + } + + private class IntWriter implements FieldWriter { + + @Override + public void write(BaseRow row, int ordinal) { + recordConsumer.addInteger(row.getInt(ordinal)); + } + } + + /** + * We only support INT96 bytes now, julianDay(4) + nanosOfDay(8). + * See https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#timestamp + * TIMESTAMP_MILLIS and TIMESTAMP_MICROS are the deprecated ConvertedType. + */ + private class TimestampWriter implements FieldWriter { + + private final int precision; + + private TimestampWriter(int precision) { + this.precision = precision; + } + + @Override + public void write(BaseRow row, int ordinal) { + recordConsumer.addBinary(timestampToInt96(row.getTimestamp(ordinal, precision))); + } + } + + private Binary timestampToInt96(SqlTimestamp sqlTimestamp) { + int julianDay; + long nanosOfDay; + if (utcTimestamp) { + long mills = sqlTimestamp.getMillisecond(); + julianDay = (int) ((mills / MILLIS_IN_DAY) + JULIAN_EPOCH_OFFSET_DAYS); + nanosOfDay = (mills % MILLIS_IN_DAY) * NANOS_PER_MILLISECOND + sqlTimestamp.getNanoOfMillisecond(); + } else { + Timestamp timestamp = sqlTimestamp.toTimestamp(); + long mills = timestamp.getTime(); + julianDay = (int) ((mills / MILLIS_IN_DAY) + JULIAN_EPOCH_OFFSET_DAYS); + nanosOfDay = ((mills % MILLIS_IN_DAY) / 1000) * NANOS_PER_SECOND + timestamp.getNanos(); + } + + ByteBuffer buf = ByteBuffer.allocate(12); + buf.order(ByteOrder.LITTLE_ENDIAN); + buf.putLong(nanosOfDay); + buf.putInt(julianDay); + buf.flip(); + return Binary.fromConstantByteBuffer(buf); + } + + private FieldWriter createDecimalWriter(int precision, int scale) { + Preconditions.checkArgument( + precision <= DecimalType.MAX_PRECISION, + "Decimal precision %s exceeds max precision %s", + precision, DecimalType.MAX_PRECISION); + + /* + * This is optimizer for UnscaledBytesWriter. + */ + class LongUnscaledBytesWriter implements FieldWriter { + private final int numBytes; + private final int initShift; + private final byte[] decimalBuffer; + + private LongUnscaledBytesWriter() { + this.numBytes = computeMinBytesForDecimalPrecision(precision); + this.initShift = 8 * (numBytes - 1); + this.decimalBuffer = new byte[numBytes]; + } + + @Override + public void write(BaseRow row, int ordinal) { + long unscaledLong = row.getDecimal(ordinal, precision, scale).toUnscaledLong(); + int i = 0; + int shift = initShift; + while (i < numBytes) { + decimalBuffer[i] = (byte) (unscaledLong >> shift); + i += 1; + shift -= 8; + } + + recordConsumer.addBinary(Binary.fromReusedByteArray(decimalBuffer, 0, numBytes)); + } + } + + class UnscaledBytesWriter implements FieldWriter { + private final int numBytes; + private final byte[] decimalBuffer; + + private UnscaledBytesWriter() { + this.numBytes = computeMinBytesForDecimalPrecision(precision); + this.decimalBuffer = new byte[numBytes]; + } + + @Override + public void write(BaseRow row, int ordinal) { + byte[] bytes = row.getDecimal(ordinal, precision, scale).toUnscaledBytes(); + byte[] writtenBytes; + if (bytes.length == numBytes) { + // Avoid copy. + writtenBytes = bytes; + } else { + byte signByte = bytes[0] < 0 ? (byte) -1 : (byte) 0; + Arrays.fill(decimalBuffer, 0, numBytes - bytes.length, signByte); + System.arraycopy(bytes, 0, decimalBuffer, numBytes - bytes.length, bytes.length); + writtenBytes = decimalBuffer; + } + recordConsumer.addBinary(Binary.fromReusedByteArray(writtenBytes, 0, numBytes)); + } + } + + // 1 <= precision <= 18, writes as FIXED_LEN_BYTE_ARRAY + // optimizer for UnscaledBytesWriter + if (Decimal.is32BitDecimal(precision) || Decimal.is64BitDecimal(precision)) { + return new LongUnscaledBytesWriter(); + } + + // 19 <= precision <= 38, writes as FIXED_LEN_BYTE_ARRAY + return new UnscaledBytesWriter(); + } +} diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/utils/ParquetSchemaConverter.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/utils/ParquetSchemaConverter.java index 326931db2f..fe79c6fa9f 100644 --- a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/utils/ParquetSchemaConverter.java +++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/utils/ParquetSchemaConverter.java @@ -25,6 +25,9 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.MapTypeInfo; import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; import org.apache.parquet.schema.GroupType; import org.apache.parquet.schema.MessageType; @@ -432,4 +435,87 @@ public class ParquetSchemaConverter { return fieldType; } + + public static MessageType convertToParquetMessageType(String name, RowType rowType) { + Type[] types = new Type[rowType.getFieldCount()]; + for (int i = 0; i < rowType.getFieldCount(); i++) { + types[i] = convertToParquetType( + rowType.getFieldNames().get(i), rowType.getTypeAt(i)); + } + return new MessageType(name, types); + } + + private static Type convertToParquetType(String name, LogicalType type) { + return convertToParquetType(name, type, Type.Repetition.OPTIONAL); + } + + private static Type convertToParquetType( + String name, LogicalType type, Type.Repetition repetition) { + switch (type.getTypeRoot()) { + case CHAR: + case VARCHAR: + return Types + .primitive(PrimitiveType.PrimitiveTypeName.BINARY, repetition) + .as(OriginalType.UTF8) + .named(name); + case BOOLEAN: + return Types.primitive(PrimitiveType.PrimitiveTypeName.BOOLEAN, repetition).named(name); + case BINARY: + case VARBINARY: + return Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, repetition).named(name); + case DECIMAL: + int precision = ((DecimalType) type).getPrecision(); + int scale = ((DecimalType) type).getScale(); + int numBytes = computeMinBytesForDecimalPrecision(precision); + return Types.primitive( + PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, repetition) + .precision(precision) + .scale(scale) + .length(numBytes) + .as(OriginalType.DECIMAL) + .named(name); + case TINYINT: + return Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, repetition) + .as(OriginalType.INT_8) + .named(name); + case SMALLINT: + return Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, repetition) + .as(OriginalType.INT_16) + .named(name); + case INTEGER: + return Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, repetition) + .named(name); + case BIGINT: + return Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, repetition) + .named(name); + case FLOAT: + return Types.primitive(PrimitiveType.PrimitiveTypeName.FLOAT, repetition).named(name); + case DOUBLE: + return Types.primitive(PrimitiveType.PrimitiveTypeName.DOUBLE, repetition).named(name); + case DATE: + return Types + .primitive(PrimitiveType.PrimitiveTypeName.INT32, repetition) + .as(OriginalType.DATE) + .named(name); + case TIME_WITHOUT_TIME_ZONE: + return Types + .primitive(PrimitiveType.PrimitiveTypeName.INT32, repetition) + .as(OriginalType.TIME_MILLIS) + .named(name); + case TIMESTAMP_WITHOUT_TIME_ZONE: + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return Types.primitive(PrimitiveType.PrimitiveTypeName.INT96, repetition) + .named(name); + default: + throw new UnsupportedOperationException("Unsupported type: " + type); + } + } + + public static int computeMinBytesForDecimalPrecision(int precision) { + int numBytes = 1; + while (Math.pow(2.0, 8 * numBytes - 1) < Math.pow(10.0, precision)) { + numBytes += 1; + } + return numBytes; + } } diff --git a/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/utils/SerializableConfiguration.java b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/utils/SerializableConfiguration.java new file mode 100644 index 0000000000..6a48fc8893 --- /dev/null +++ b/flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/utils/SerializableConfiguration.java @@ -0,0 +1,55 @@ +/* + * 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.formats.parquet.utils; + +import org.apache.hadoop.conf.Configuration; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; + +/** + * Wrap {@link Configuration} to a serializable class. + */ +public class SerializableConfiguration implements Serializable { + + private static final long serialVersionUID = 1L; + + private Configuration conf; + + public SerializableConfiguration(Configuration conf) { + this.conf = conf; + } + + public Configuration conf() { + return conf; + } + + private void writeObject(ObjectOutputStream out) throws IOException { + conf.write(out); + } + + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + if (conf == null) { + conf = new Configuration(); + } + conf.readFields(in); + } +} diff --git a/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/row/ParquetRowDataWriterTest.java b/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/row/ParquetRowDataWriterTest.java new file mode 100644 index 0000000000..508a36c756 --- /dev/null +++ b/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/row/ParquetRowDataWriterTest.java @@ -0,0 +1,166 @@ +/* + * 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.formats.parquet.row; + +import org.apache.flink.api.common.serialization.BulkWriter; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.formats.parquet.ParquetWriterFactory; +import org.apache.flink.formats.parquet.vector.ParquetColumnarRowSplitReader; +import org.apache.flink.formats.parquet.vector.ParquetSplitReaderUtil; +import org.apache.flink.table.dataformat.BaseRow; +import org.apache.flink.table.dataformat.DataFormatConverters; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.DoubleType; +import org.apache.flink.table.types.logical.FloatType; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.SmallIntType; +import org.apache.flink.table.types.logical.TimestampType; +import org.apache.flink.table.types.logical.TinyIntType; +import org.apache.flink.table.types.logical.VarBinaryType; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.flink.types.Row; + +import org.apache.hadoop.conf.Configuration; +import org.apache.parquet.hadoop.ParquetOutputFormat; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.IOException; +import java.math.BigDecimal; +import java.nio.charset.StandardCharsets; +import java.time.LocalDateTime; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.UUID; +import java.util.stream.IntStream; + +/** + * Test for {@link ParquetRowDataBuilder} and {@link ParquetRowDataWriter}. + */ +public class ParquetRowDataWriterTest { + + @ClassRule + public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + private static final RowType ROW_TYPE = RowType.of( + new VarCharType(VarCharType.MAX_LENGTH), + new VarBinaryType(VarBinaryType.MAX_LENGTH), + new BooleanType(), + new TinyIntType(), + new SmallIntType(), + new IntType(), + new BigIntType(), + new FloatType(), + new DoubleType(), + new TimestampType(9), + new DecimalType(5, 0), + new DecimalType(15, 0), + new DecimalType(20, 0)); + + @SuppressWarnings("unchecked") + private static final DataFormatConverters.DataFormatConverter CONVERTER = + DataFormatConverters.getConverterForDataType( + TypeConversions.fromLogicalToDataType(ROW_TYPE)); + + @Test + public void testTypes() throws IOException { + Configuration conf = new Configuration(); + innerTest(conf, true); + innerTest(conf, false); + } + + @Test + public void testCompression() throws IOException { + Configuration conf = new Configuration(); + conf.set(ParquetOutputFormat.COMPRESSION, "GZIP"); + innerTest(conf, true); + innerTest(conf, false); + } + + private void innerTest( + Configuration conf, + boolean utcTimestamp) throws IOException { + Path path = new Path(TEMPORARY_FOLDER.newFolder().getPath(), UUID.randomUUID().toString()); + int number = 1000; + List rows = new ArrayList<>(number); + for (int i = 0; i < number; i++) { + Integer v = i; + rows.add(Row.of( + String.valueOf(v), + String.valueOf(v).getBytes(StandardCharsets.UTF_8), + v % 2 == 0, + v.byteValue(), + v.shortValue(), + v, + v.longValue(), + v.floatValue(), + v.doubleValue(), + toDateTime(v), + BigDecimal.valueOf(v), + BigDecimal.valueOf(v), + BigDecimal.valueOf(v))); + } + + ParquetWriterFactory factory = ParquetRowDataBuilder.createWriterFactory( + ROW_TYPE, conf, utcTimestamp); + BulkWriter writer = factory.create(path.getFileSystem().create( + path, FileSystem.WriteMode.OVERWRITE)); + for (int i = 0; i < number; i++) { + writer.addElement(CONVERTER.toInternal(rows.get(i))); + } + writer.flush(); + writer.finish(); + + // verify + ParquetColumnarRowSplitReader reader = ParquetSplitReaderUtil.genPartColumnarRowReader( + utcTimestamp, + conf, + ROW_TYPE.getFieldNames().toArray(new String[0]), + ROW_TYPE.getChildren().stream() + .map(TypeConversions::fromLogicalToDataType) + .toArray(DataType[]::new), + new HashMap<>(), + IntStream.range(0, ROW_TYPE.getFieldCount()).toArray(), + 50, + path, + 0, + Long.MAX_VALUE); + int cnt = 0; + while (!reader.reachedEnd()) { + Row row = CONVERTER.toExternal(reader.nextRecord()); + Assert.assertEquals(rows.get(cnt), row); + cnt++; + } + Assert.assertEquals(number, cnt); + } + + private LocalDateTime toDateTime(Integer v) { + v = (v > 0 ? v : -v) % 1000; + return LocalDateTime.now().plusNanos(v).plusSeconds(v); + } +} -- Gitee From d156d09a6cb3f21245da4ae73b263ee4c8dfe4b7 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Mon, 30 Mar 2020 07:48:42 +0200 Subject: [PATCH 459/885] [FLINK-16830][table-api] Let users use Row/List/Map/Seq directly in Expression DSL This PR implements conversion logic from Row/List/Map/Seq to a corresponding Expression. From now on users can use those types in the Expression DSL without the need to converting those types manually. It is also a prerequisite to use those type directly in TableEnvironment#fromValues. --- .../table/expressions/ApiExpressionUtils.java | 150 +++++++++++++++++- .../expressions/ObjectToExpressionTest.java | 85 ++++++++++ .../flink/table/api/expressionDsl.scala | 96 ++++++----- .../api/ExpressionsConsistencyCheckTest.scala | 7 +- .../table/api/ImplicitConversionsTest.scala | 91 +++++++++++ 5 files changed, 376 insertions(+), 53 deletions(-) create mode 100644 flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/expressions/ObjectToExpressionTest.java create mode 100644 flink-table/flink-table-api-scala/src/test/scala/org/apache/flink/table/api/ImplicitConversionsTest.scala diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/ApiExpressionUtils.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/ApiExpressionUtils.java index 92f8d83f4e..1d8998af5b 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/ApiExpressionUtils.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/ApiExpressionUtils.java @@ -24,15 +24,26 @@ import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.functions.BuiltInFunctionDefinition; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; import org.apache.flink.table.functions.FunctionDefinition; import org.apache.flink.table.functions.FunctionIdentifier; import org.apache.flink.table.functions.FunctionKind; import org.apache.flink.table.operations.QueryOperation; import org.apache.flink.table.types.DataType; +import org.apache.flink.types.Row; +import java.lang.reflect.Array; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.math.BigDecimal; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Map; +import java.util.Optional; import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; /** * Utilities for API-specific {@link Expression}s. @@ -52,14 +63,151 @@ public final class ApiExpressionUtils { // private } + /** + * Converts a given object to an expression. + * + *

    It converts: + *

      + *
    • {@link Row} to a call to a row constructor expression
    • + *
    • {@link Map} to a call to a map constructor expression
    • + *
    • {@link List} to a call to an array constructor expression
    • + *
    • arrays to a call to an array constructor expression
    • + *
    • Scala's {@code Seq} to an array constructor via reflection
    • + *
    • Scala's {@code Map} to a map constructor via reflection
    • + *
    • Scala's {@code BigDecimal} to a DECIMAL literal
    • + *
    • if none of the above applies, the function tries to convert the object + * to a value literal with {@link #valueLiteral(Object)}
    • + *
    + * + * @param expression An object to convert to an expression + */ public static Expression objectToExpression(Object expression) { if (expression instanceof ApiExpression) { return ((ApiExpression) expression).toExpr(); } else if (expression instanceof Expression) { return (Expression) expression; + } else if (expression instanceof Row) { + return convertRow((Row) expression); + } else if (expression instanceof Map) { + return convertJavaMap((Map) expression); + } else if (expression instanceof List) { + return convertJavaList((List) expression); + } else if (expression.getClass().isArray()) { + return convertArray(expression); } else { - return valueLiteral(expression); + return convertScala(expression).orElseGet(() -> valueLiteral(expression)); + } + } + + private static Expression convertRow(Row expression) { + List fields = IntStream.range(0, expression.getArity()) + .mapToObj(expression::getField) + .map(ApiExpressionUtils::objectToExpression) + .collect(Collectors.toList()); + + return unresolvedCall(BuiltInFunctionDefinitions.ROW, fields); + } + + private static Expression convertJavaMap(Map expression) { + List entries = expression.entrySet() + .stream() + .flatMap(e -> Stream.of( + objectToExpression(e.getKey()), + objectToExpression(e.getValue()) + )).collect(Collectors.toList()); + + return unresolvedCall(BuiltInFunctionDefinitions.MAP, entries); + } + + private static Expression convertJavaList(List expression) { + List entries = expression + .stream() + .map(ApiExpressionUtils::objectToExpression) + .collect(Collectors.toList()); + + return unresolvedCall(BuiltInFunctionDefinitions.ARRAY, entries); + } + + private static Expression convertArray(Object expression) { + int length = Array.getLength(expression); + List entries = IntStream.range(0, length) + .mapToObj(idx -> Array.get(expression, idx)) + .map(ApiExpressionUtils::objectToExpression) + .collect(Collectors.toList()); + return unresolvedCall(BuiltInFunctionDefinitions.ARRAY, entries); + } + + private static Optional convertScala(Object obj) { + try { + Optional array = convertScalaSeq(obj); + if (array.isPresent()) { + return array; + } + + Optional bigDecimal = convertScalaBigDecimal(obj); + if (bigDecimal.isPresent()) { + return bigDecimal; + } + + return convertScalaMap(obj); + } catch (Exception e) { + return Optional.empty(); + } + } + + private static Optional convertScalaMap(Object obj) + throws ClassNotFoundException, NoSuchMethodException, IllegalAccessException, InvocationTargetException { + Class mapClass = Class.forName("scala.collection.Map"); + if (mapClass.isAssignableFrom(obj.getClass())) { + Class seqClass = Class.forName("scala.collection.Seq"); + Class productClass = Class.forName("scala.Product"); + Method getElement = productClass.getMethod("productElement", int.class); + Method toSeq = mapClass.getMethod("toSeq"); + Method getMethod = seqClass.getMethod("apply", Object.class); + Method lengthMethod = seqClass.getMethod("length"); + + Object mapAsSeq = toSeq.invoke(obj); + List entries = new ArrayList<>(); + for (int i = 0; i < (Integer) lengthMethod.invoke(mapAsSeq); i++) { + Object mapEntry = getMethod.invoke(mapAsSeq, i); + + Object key = getElement.invoke(mapEntry, 0); + Object value = getElement.invoke(mapEntry, 1); + entries.add(objectToExpression(key)); + entries.add(objectToExpression(value)); + } + + return Optional.of(unresolvedCall(BuiltInFunctionDefinitions.MAP, entries)); + } + return Optional.empty(); + } + + private static Optional convertScalaSeq(Object obj) + throws ClassNotFoundException, NoSuchMethodException, IllegalAccessException, InvocationTargetException { + Class seqClass = Class.forName("scala.collection.Seq"); + if (seqClass.isAssignableFrom(obj.getClass())) { + Method getMethod = seqClass.getMethod("apply", Object.class); + Method lengthMethod = seqClass.getMethod("length"); + + List entries = new ArrayList<>(); + for (int i = 0; i < (Integer) lengthMethod.invoke(obj); i++) { + entries.add(objectToExpression(getMethod.invoke(obj, i))); + } + + return Optional.of(unresolvedCall(BuiltInFunctionDefinitions.ARRAY, entries)); + } + return Optional.empty(); + } + + private static Optional convertScalaBigDecimal(Object obj) + throws ClassNotFoundException, NoSuchMethodException, IllegalAccessException, InvocationTargetException { + Class decimalClass = Class.forName("scala.math.BigDecimal"); + if (decimalClass.equals(obj.getClass())) { + Method toJava = decimalClass.getMethod("bigDecimal"); + BigDecimal bigDecimal = (BigDecimal) toJava.invoke(obj); + return Optional.of(valueLiteral(bigDecimal)); } + return Optional.empty(); } public static Expression unwrapFromApi(Expression expression) { diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/expressions/ObjectToExpressionTest.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/expressions/ObjectToExpressionTest.java new file mode 100644 index 0000000000..cb3fc25a0d --- /dev/null +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/expressions/ObjectToExpressionTest.java @@ -0,0 +1,85 @@ +/* + * 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.table.expressions; + +import org.apache.flink.types.Row; + +import org.junit.Test; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static java.util.Arrays.asList; +import static java.util.Collections.singletonList; +import static org.apache.flink.table.api.Expressions.array; +import static org.apache.flink.table.api.Expressions.map; +import static org.apache.flink.table.api.Expressions.row; +import static org.apache.flink.table.expressions.ApiExpressionUtils.objectToExpression; +import static org.apache.flink.table.expressions.ApiExpressionUtils.unwrapFromApi; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.junit.Assert.assertThat; + +/** + * Tests for converting an object to a {@link Expression} via {@link ApiExpressionUtils#objectToExpression(Object)}. + */ +public class ObjectToExpressionTest { + + @Test + public void testListConversion() { + Expression expr = objectToExpression(asList(1, 2)); + + assertThatEquals(expr, array(1, 2)); + } + + @Test + public void testNestedListConversion() { + Expression expr = objectToExpression(asList(singletonList(1), singletonList(2))); + + assertThatEquals(expr, array(array(1), array(2))); + } + + @Test + public void testMapConversion() { + Map> map = new HashMap<>(); + map.put("key1", singletonList(2)); + map.put("key2", asList(1, 2)); + + Expression expr = objectToExpression(map); + assertThatEquals( + expr, + map( + "key1", array(2), + "key2", array(1, 2) + ) + ); + } + + @Test + public void testRowConversion() { + Expression expr = objectToExpression(Row.of(1, "ABC", new int[]{1, 2, 3})); + + assertThatEquals(expr, row(1, "ABC", array(1, 2, 3))); + } + + private static void assertThatEquals(Expression actual, Expression expected) { + assertThat(unwrapFromApi(actual), equalTo(unwrapFromApi(expected))); + } + +} diff --git a/flink-table/flink-table-api-scala/src/main/scala/org/apache/flink/table/api/expressionDsl.scala b/flink-table/flink-table-api-scala/src/main/scala/org/apache/flink/table/api/expressionDsl.scala index d42403109a..2a7577a2d4 100644 --- a/flink-table/flink-table-api-scala/src/main/scala/org/apache/flink/table/api/expressionDsl.scala +++ b/flink-table/flink-table-api-scala/src/main/scala/org/apache/flink/table/api/expressionDsl.scala @@ -25,11 +25,13 @@ import org.apache.flink.table.expressions._ import org.apache.flink.table.functions.BuiltInFunctionDefinitions._ import org.apache.flink.table.functions.{ScalarFunction, TableFunction, UserDefinedAggregateFunction, UserDefinedFunctionHelper, _} import org.apache.flink.table.types.DataType +import org.apache.flink.types.Row import java.lang.{Boolean => JBoolean, Byte => JByte, Double => JDouble, Float => JFloat, Integer => JInteger, Long => JLong, Short => JShort} import java.math.{BigDecimal => JBigDecimal} import java.sql.{Date, Time, Timestamp} import java.time.{LocalDate, LocalDateTime, LocalTime} +import java.util.{List => JList, Map => JMap} import _root_.scala.language.implicitConversions @@ -239,6 +241,11 @@ trait ImplicitExpressionConversions { def expr: Expression = unresolvedRef(s.name) } + implicit class AnyWithOperations[T](e: T)(implicit toExpr: T => Expression) + extends ImplicitExpressionOperations { + def expr: Expression = toExpr(e) + } + implicit class LiteralLongExpression(l: Long) extends ImplicitExpressionOperations { def expr: Expression = valueLiteral(l) } @@ -272,17 +279,17 @@ trait ImplicitExpressionConversions { } implicit class LiteralJavaDecimalExpression(javaDecimal: JBigDecimal) - extends ImplicitExpressionOperations { + extends ImplicitExpressionOperations { def expr: Expression = valueLiteral(javaDecimal) } implicit class LiteralScalaDecimalExpression(scalaDecimal: BigDecimal) - extends ImplicitExpressionOperations { + extends ImplicitExpressionOperations { def expr: Expression = valueLiteral(scalaDecimal.bigDecimal) } implicit class LiteralSqlDateExpression(sqlDate: Date) - extends ImplicitExpressionOperations { + extends ImplicitExpressionOperations { def expr: Expression = valueLiteral(sqlDate) } @@ -292,7 +299,7 @@ trait ImplicitExpressionConversions { } implicit class LiteralSqlTimestampExpression(sqlTimestamp: Timestamp) - extends ImplicitExpressionOperations { + extends ImplicitExpressionOperations { def expr: Expression = valueLiteral(sqlTimestamp) } @@ -370,10 +377,9 @@ trait ImplicitExpressionConversions { * Creates an unresolved reference to a table's field. * * Example: - * ``` + * {{{ * tab.select($"key", $"value") - * ``` - * + * }}} */ def $(args: Any*): Expression = unresolvedRef(sc.s(args: _*)) } @@ -392,20 +398,34 @@ trait ImplicitExpressionConversions { implicit def byte2Literal(b: Byte): Expression = valueLiteral(b) + implicit def byte2Literal(b: JByte): Expression = valueLiteral(b) + implicit def short2Literal(s: Short): Expression = valueLiteral(s) + implicit def short2Literal(s: JShort): Expression = valueLiteral(s) + implicit def int2Literal(i: Int): Expression = valueLiteral(i) + implicit def int2Literal(i: JInteger): Expression = valueLiteral(i) + implicit def long2Literal(l: Long): Expression = valueLiteral(l) + implicit def long2Literal(l: JLong): Expression = valueLiteral(l) + implicit def double2Literal(d: Double): Expression = valueLiteral(d) + implicit def double2Literal(d: JDouble): Expression = valueLiteral(d) + implicit def float2Literal(d: Float): Expression = valueLiteral(d) + implicit def float2Literal(d: JFloat): Expression = valueLiteral(d) + implicit def string2Literal(str: String): Expression = valueLiteral(str) implicit def boolean2Literal(bool: Boolean): Expression = valueLiteral(bool) + implicit def boolean2Literal(bool: JBoolean): Expression = valueLiteral(bool) + implicit def javaDec2Literal(javaDec: JBigDecimal): Expression = valueLiteral(javaDec) implicit def scalaDec2Literal(scalaDec: BigDecimal): Expression = @@ -425,54 +445,28 @@ trait ImplicitExpressionConversions { implicit def localDateTime2Literal(localDateTime: LocalDateTime): Expression = valueLiteral(localDateTime) + implicit def javaList2ArrayConstructor(jList: JList[_]): Expression = { + ApiExpressionUtils.objectToExpression(jList) + } + + implicit def seq2ArrayConstructor(seq: Seq[_]): Expression = { + ApiExpressionUtils.objectToExpression(seq) + } + implicit def array2ArrayConstructor(array: Array[_]): Expression = { + ApiExpressionUtils.objectToExpression(array) + } - def createArray(elements: Array[_]): Expression = { - unresolvedCall(BuiltInFunctionDefinitions.ARRAY, elements.map(valueLiteral): _*) - } + implicit def javaMap2MapConstructor(map: JMap[_, _]): Expression = { + ApiExpressionUtils.objectToExpression(map) + } - def convertArray(array: Array[_]): Expression = array match { - // primitives - case _: Array[Boolean] => createArray(array) - case _: Array[Byte] => createArray(array) - case _: Array[Short] => createArray(array) - case _: Array[Int] => createArray(array) - case _: Array[Long] => createArray(array) - case _: Array[Float] => createArray(array) - case _: Array[Double] => createArray(array) - - // boxed types - case _: Array[JBoolean] => createArray(array) - case _: Array[JByte] => createArray(array) - case _: Array[JShort] => createArray(array) - case _: Array[JInteger] => createArray(array) - case _: Array[JLong] => createArray(array) - case _: Array[JFloat] => createArray(array) - case _: Array[JDouble] => createArray(array) - - // others - case _: Array[String] => createArray(array) - case _: Array[JBigDecimal] => createArray(array) - case _: Array[Date] => createArray(array) - case _: Array[Time] => createArray(array) - case _: Array[Timestamp] => createArray(array) - case _: Array[LocalDate] => createArray(array) - case _: Array[LocalTime] => createArray(array) - case _: Array[LocalDateTime] => createArray(array) - case bda: Array[BigDecimal] => createArray(bda.map(_.bigDecimal)) - - case _ => - // nested - if (array.length > 0 && array.head.isInstanceOf[Array[_]]) { - unresolvedCall( - BuiltInFunctionDefinitions.ARRAY, - array.map { na => convertArray(na.asInstanceOf[Array[_]]) } :_*) - } else { - throw new ValidationException("Unsupported array type.") - } - } + implicit def map2MapConstructor(map: Map[_, _]): Expression = { + ApiExpressionUtils.objectToExpression(map) + } - convertArray(array) + implicit def row2RowConstructor(rowObject: Row): Expression = { + ApiExpressionUtils.objectToExpression(rowObject) } // ---------------------------------------------------------------------------------------------- diff --git a/flink-table/flink-table-api-scala/src/test/scala/org/apache/flink/table/api/ExpressionsConsistencyCheckTest.scala b/flink-table/flink-table-api-scala/src/test/scala/org/apache/flink/table/api/ExpressionsConsistencyCheckTest.scala index f76009e954..d864bdcb27 100644 --- a/flink-table/flink-table-api-scala/src/test/scala/org/apache/flink/table/api/ExpressionsConsistencyCheckTest.scala +++ b/flink-table/flink-table-api-scala/src/test/scala/org/apache/flink/table/api/ExpressionsConsistencyCheckTest.scala @@ -90,7 +90,7 @@ class ExpressionsConsistencyCheckTest { // Scala implicit conversions to ImplicitExpressionOperations //----------------------------------------------------------------------------------- "WithOperations", - "apiExpressionToExpression", + "AnyWithOperations", "LiteralScalaDecimalExpression", "LiteralJavaDecimalExpression", "LiteralShortExpression", @@ -127,6 +127,11 @@ class ExpressionsConsistencyCheckTest { "localDate2Literal", "float2Literal", "array2ArrayConstructor", + "seq2ArrayConstructor", + "javaList2ArrayConstructor", + "map2MapConstructor", + "javaMap2MapConstructor", + "row2RowConstructor", "tableSymbolToExpression", //----------------------------------------------------------------------------------- diff --git a/flink-table/flink-table-api-scala/src/test/scala/org/apache/flink/table/api/ImplicitConversionsTest.scala b/flink-table/flink-table-api-scala/src/test/scala/org/apache/flink/table/api/ImplicitConversionsTest.scala new file mode 100644 index 0000000000..8fec57af57 --- /dev/null +++ b/flink-table/flink-table-api-scala/src/test/scala/org/apache/flink/table/api/ImplicitConversionsTest.scala @@ -0,0 +1,91 @@ +/* + * 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.table.api + +import org.apache.flink.table.expressions.ApiExpressionUtils.unwrapFromApi +import org.apache.flink.table.expressions.Expression +import org.apache.flink.types.Row + +import org.hamcrest.CoreMatchers.equalTo +import org.junit.Assert.assertThat +import org.junit.Test + +/** + * Tests for conversion between objects and [[Expression]]s used in Expression DSL. + */ +class ImplicitConversionsTest extends ImplicitExpressionConversions { + @Test + def testSeqConversion(): Unit = { + val expr = Seq(1, 2).toExpr + + assertThatEquals(expr, array(1, 2)) + } + + @Test + def testSeqOfExpressionsConversion(): Unit = { + val expr = Seq(row(1, "ABC"), row(3, "DEF")).toExpr + + assertThatEquals(expr, array(row(1, "ABC"), row(3, "DEF"))) + } + + @Test + def testListConversion(): Unit = { + val expr = List(1, 2).toExpr + + assertThatEquals(expr, array(1, 2)) + } + + @Test + def testMapConversion(): Unit = { + val expr = Map("key1" -> List(2), "key2" -> List(1, 2)).toExpr + + assertThatEquals( + expr, + map( + "key1", array(2), + "key2", array(1, 2) + ) + ) + } + + @Test + def testNestedListConversion(): Unit = { + val expr = List(List(1), List(2)).toExpr + + assertThatEquals(expr, array(array(1), array(2))) + } + + @Test + def testRowConversion(): Unit = { + val expr = Row.of(Int.box(1), "ABC").toExpr + + assertThatEquals(expr, row(1, "ABC")) + } + + @Test + def testRowConversionWithScalaTypes(): Unit = { + val expr = Row.of(Int.box(1), Seq("ABC", "DEF"), BigDecimal(1234)).toExpr + + assertThatEquals(expr, row(1, array("ABC", "DEF"), BigDecimal(1234))) + } + + private def assertThatEquals(actual: Expression, expected: Expression): Unit = { + assertThat(unwrapFromApi(actual), equalTo(unwrapFromApi(expected))) + } +} -- Gitee From b0e334afc7eb5531996b21d28e212febc7d38bf8 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Tue, 7 Apr 2020 14:48:46 +0800 Subject: [PATCH 460/885] [FLINK-16859][filesystem][table-runtime] Introduce file system table factory This closes #11574 --- .../utils/TestCsvFileSystemFormatFactory.java | 157 +++++++++ .../utils/TestRowDataCsvInputFormat.java | 179 ++++++++++ ....apache.flink.table.factories.TableFactory | 1 + .../runtime/FileSystemITCaseBase.scala | 284 ++++++++++++++++ .../batch/sql/BatchFileSystemITCaseBase.scala | 48 +++ .../batch/sql/FileSystemTestCsvITCase.scala | 44 +++ .../sql/StreamFileSystemITCaseBase.scala | 58 ++++ .../planner/runtime/utils/TestSinkUtil.scala | 2 +- flink-table/flink-table-runtime-blink/pom.xml | 6 + .../filesystem/FileSystemFormatFactory.java | 147 +++++++++ .../filesystem/FileSystemTableFactory.java | 143 ++++++++ .../table/filesystem/FileSystemTableSink.java | 289 +++++++++++++++++ .../filesystem/FileSystemTableSource.java | 307 ++++++++++++++++++ .../filesystem/RowDataPartitionComputer.java | 112 +++++++ ....apache.flink.table.factories.TableFactory | 16 + 15 files changed, 1792 insertions(+), 1 deletion(-) create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/utils/TestCsvFileSystemFormatFactory.java create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/utils/TestRowDataCsvInputFormat.java create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/FileSystemITCaseBase.scala create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/BatchFileSystemITCaseBase.scala create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/FileSystemTestCsvITCase.scala create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/StreamFileSystemITCaseBase.scala create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/FileSystemFormatFactory.java create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/FileSystemTableFactory.java create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/FileSystemTableSink.java create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/FileSystemTableSource.java create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/RowDataPartitionComputer.java create mode 100644 flink-table/flink-table-runtime-blink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/utils/TestCsvFileSystemFormatFactory.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/utils/TestCsvFileSystemFormatFactory.java new file mode 100644 index 0000000000..1f7b349574 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/utils/TestCsvFileSystemFormatFactory.java @@ -0,0 +1,157 @@ +/* + * 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.table.planner.utils; + +import org.apache.flink.api.common.io.InputFormat; +import org.apache.flink.api.common.serialization.BulkWriter; +import org.apache.flink.api.common.serialization.Encoder; +import org.apache.flink.table.dataformat.BaseRow; +import org.apache.flink.table.dataformat.DataFormatConverters; +import org.apache.flink.table.filesystem.FileSystemFormatFactory; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.flink.types.Row; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static org.apache.flink.api.java.io.CsvOutputFormat.DEFAULT_FIELD_DELIMITER; +import static org.apache.flink.api.java.io.CsvOutputFormat.DEFAULT_LINE_DELIMITER; +import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT; + +/** + * Test csv {@link FileSystemFormatFactory}. + */ +public class TestCsvFileSystemFormatFactory implements FileSystemFormatFactory { + + public static final String USE_BULK_WRITER = "format.use-bulk-writer"; + + @Override + public boolean supportsSchemaDerivation() { + return true; + } + + @Override + public Map requiredContext() { + Map context = new HashMap<>(); + context.put(FORMAT, "testcsv"); + return context; + } + + @Override + public List supportedProperties() { + return Collections.singletonList(USE_BULK_WRITER); + } + + @Override + public InputFormat createReader(ReaderContext context) { + return new TestRowDataCsvInputFormat( + context.getPaths(), + context.getSchema(), + context.getPartitionKeys(), + context.getDefaultPartName(), + context.getProjectFields(), + context.getPushedDownLimit()); + } + + private boolean useBulkWriter(WriterContext context) { + return Boolean.parseBoolean(context.getFormatProperties().get(USE_BULK_WRITER)); + } + + @Override + public Optional> createEncoder(WriterContext context) { + if (useBulkWriter(context)) { + return Optional.empty(); + } + + DataType[] types = context.getFieldTypesWithoutPartKeys(); + return Optional.of((baseRow, stream) -> { + writeCsvToStream(types, baseRow, stream); + }); + } + + private static void writeCsvToStream( + DataType[] types, + BaseRow baseRow, + OutputStream stream) throws IOException { + LogicalType[] fieldTypes = Arrays.stream(types) + .map(DataType::getLogicalType) + .toArray(LogicalType[]::new); + DataFormatConverters.DataFormatConverter converter = DataFormatConverters.getConverterForDataType( + TypeConversions.fromLogicalToDataType(RowType.of(fieldTypes))); + + Row row = (Row) converter.toExternal(baseRow); + StringBuilder builder = new StringBuilder(); + Object o; + for (int i = 0; i < row.getArity(); i++) { + if (i > 0) { + builder.append(DEFAULT_FIELD_DELIMITER); + } + if ((o = row.getField(i)) != null) { + builder.append(o); + } + } + String str = builder.toString(); + stream.write(str.getBytes(StandardCharsets.UTF_8)); + stream.write(DEFAULT_LINE_DELIMITER.getBytes(StandardCharsets.UTF_8)); + } + + @Override + public Optional> createBulkWriterFactory(WriterContext context) { + if (!useBulkWriter(context)) { + return Optional.empty(); + } + + DataType[] types = context.getFieldTypesWithoutPartKeys(); + return Optional.of(out -> new CsvBulkWriter(types, out)); + } + + private static class CsvBulkWriter implements BulkWriter { + + private final DataType[] types; + private final OutputStream stream; + + private CsvBulkWriter(DataType[] types, OutputStream stream) { + this.types = types; + this.stream = stream; + } + + @Override + public void addElement(BaseRow element) throws IOException { + writeCsvToStream(types, element, stream); + } + + @Override + public void flush() { + } + + @Override + public void finish() { + } + } +} diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/utils/TestRowDataCsvInputFormat.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/utils/TestRowDataCsvInputFormat.java new file mode 100644 index 0000000000..72b21568af --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/utils/TestRowDataCsvInputFormat.java @@ -0,0 +1,179 @@ +/* + * 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.table.planner.utils; + +import org.apache.flink.api.common.io.InputFormat; +import org.apache.flink.api.common.io.RichInputFormat; +import org.apache.flink.api.common.io.statistics.BaseStatistics; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.io.RowCsvInputFormat; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.FileInputSplit; +import org.apache.flink.core.fs.Path; +import org.apache.flink.core.io.InputSplitAssigner; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.dataformat.BaseRow; +import org.apache.flink.table.dataformat.BinaryString; +import org.apache.flink.table.dataformat.DataFormatConverters; +import org.apache.flink.table.dataformat.GenericRow; +import org.apache.flink.table.filesystem.PartitionPathUtils; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.flink.types.Row; + +import java.io.IOException; +import java.util.Arrays; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.stream.Collectors; + +/** + * The {@link InputFormat} that output {@link BaseRow}. + */ +public class TestRowDataCsvInputFormat extends RichInputFormat { + + private final List partitionKeys; + private final String defaultPartValue; + private final int[] selectFields; + private final long limit; + private final RowCsvInputFormat inputFormat; + private final List fieldTypes; + private final List fieldNames; + private final List csvSelectConverters; + private final int[] csvFieldMapping; + + private transient Row csvRow; + private transient GenericRow row; + private transient long emitted; + + public TestRowDataCsvInputFormat( + Path[] paths, + TableSchema schema, + List partitionKeys, + String defaultPartValue, + int[] selectFields, + long limit) { + this.partitionKeys = partitionKeys; + this.defaultPartValue = defaultPartValue; + this.selectFields = selectFields; + this.limit = limit; + RowTypeInfo rowType = (RowTypeInfo) schema.toRowType(); + this.fieldTypes = Arrays.asList(rowType.getFieldTypes()); + this.fieldNames = Arrays.asList(rowType.getFieldNames()); + + List csvFieldNames = fieldNames.stream() + .filter(name -> !partitionKeys.contains(name)).collect(Collectors.toList()); + + List selectFieldNames = Arrays.stream(selectFields) + .mapToObj(fieldNames::get) + .collect(Collectors.toList()); + List csvSelectFieldNames = selectFieldNames.stream() + .filter(name -> !partitionKeys.contains(name)).collect(Collectors.toList()); + List csvSelectTypes = csvSelectFieldNames.stream() + .map(name -> fieldTypes.get(fieldNames.indexOf(name))).collect(Collectors.toList()); + this.csvSelectConverters = csvSelectTypes.stream() + .map(TypeConversions::fromLegacyInfoToDataType) + .map(DataFormatConverters::getConverterForDataType) + .collect(Collectors.toList()); + int[] csvSelectFields = csvSelectFieldNames.stream().mapToInt(csvFieldNames::indexOf).toArray(); + this.inputFormat = new RowCsvInputFormat( + null, csvSelectTypes.toArray(new TypeInformation[0]), csvSelectFields); + this.inputFormat.setFilePaths(paths); + + this.csvFieldMapping = csvSelectFieldNames.stream().mapToInt(selectFieldNames::indexOf).toArray(); + this.emitted = 0; + } + + @Override + public void configure(Configuration parameters) { + inputFormat.configure(parameters); + } + + @Override + public BaseStatistics getStatistics(BaseStatistics cachedStatistics) throws IOException { + return inputFormat.getStatistics(cachedStatistics); + } + + @Override + public FileInputSplit[] createInputSplits(int minNumSplits) throws IOException { + return inputFormat.createInputSplits(minNumSplits); + } + + @Override + public InputSplitAssigner getInputSplitAssigner(FileInputSplit[] inputSplits) { + return inputFormat.getInputSplitAssigner(inputSplits); + } + + @Override + public void open(FileInputSplit split) throws IOException { + inputFormat.open(split); + Path path = split.getPath(); + LinkedHashMap partSpec = PartitionPathUtils.extractPartitionSpecFromPath(path); + this.row = new GenericRow(selectFields.length); + for (int i = 0; i < selectFields.length; i++) { + int selectField = selectFields[i]; + String name = fieldNames.get(selectField); + if (partitionKeys.contains(name)) { + String value = partSpec.get(name); + value = defaultPartValue.equals(value) ? null : value; + this.row.setField( + i, convertStringToInternal(value, fieldTypes.get(selectField))); + } + } + this.csvRow = new Row(csvSelectConverters.size()); + } + + private Object convertStringToInternal(String value, TypeInformation type) { + if (type.equals(Types.INT)) { + return Integer.parseInt(value); + } else if (type.equals(Types.LONG)) { + return Long.parseLong(value); + } else if (type.equals(Types.STRING)) { + return BinaryString.fromString(value); + } else { + throw new UnsupportedOperationException("Unsupported partition type: " + type); + } + } + + @Override + public boolean reachedEnd() { + return emitted >= limit || inputFormat.reachedEnd(); + } + + @Override + public BaseRow nextRecord(BaseRow reuse) throws IOException { + Row csvRow = inputFormat.nextRecord(this.csvRow); + if (csvRow == null) { + return null; + } + for (int i = 0; i < csvSelectConverters.size(); i++) { + row.setField( + csvFieldMapping[i], + csvSelectConverters.get(i).toInternal(csvRow.getField(i))); + } + emitted++; + return row; + } + + @Override + public void close() throws IOException { + inputFormat.close(); + } +} diff --git a/flink-table/flink-table-planner-blink/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink-table/flink-table-planner-blink/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFactory index 9b16bc6b24..7a843a7310 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFactory +++ b/flink-table/flink-table-planner-blink/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFactory @@ -18,3 +18,4 @@ org.apache.flink.table.planner.runtime.batch.sql.TestPartitionableSinkFactory org.apache.flink.table.planner.utils.TestPartitionableSourceFactory org.apache.flink.table.planner.utils.TestFilterableTableSourceFactory org.apache.flink.table.planner.utils.TestProjectableTableSourceFactory +org.apache.flink.table.planner.utils.TestCsvFileSystemFormatFactory diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/FileSystemITCaseBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/FileSystemITCaseBase.scala new file mode 100644 index 0000000000..cb0f2f7064 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/FileSystemITCaseBase.scala @@ -0,0 +1,284 @@ +/* + * 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.table.planner.runtime + +import org.apache.flink.api.common.typeinfo.{TypeInformation, Types} +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.table.api.TableEnvironment +import org.apache.flink.table.api.config.ExecutionConfigOptions +import org.apache.flink.table.planner.runtime.FileSystemITCaseBase._ +import org.apache.flink.table.planner.runtime.utils.BatchTableEnvUtil +import org.apache.flink.table.planner.runtime.utils.BatchTestBase.row +import org.apache.flink.types.Row + +import org.junit.rules.TemporaryFolder +import org.junit.{Rule, Test} + +import scala.collection.Seq + +/** + * Test File system table factory. + */ +trait FileSystemITCaseBase { + + val fileTmpFolder = new TemporaryFolder + protected var resultPath: String = _ + + @Rule + def fileTempFolder: TemporaryFolder = fileTmpFolder + + def formatProperties(): Array[String] = Array() + + def tableEnv: TableEnvironment + + def check(sqlQuery: String, expectedResult: Seq[Row]): Unit + + def open(): Unit = { + resultPath = fileTmpFolder.newFolder().toURI.toString + BatchTableEnvUtil.registerCollection( + tableEnv, + "originalT", + data_with_partitions, + dataType, + "x, y, a, b") + tableEnv.sqlUpdate( + s""" + |create table partitionedTable ( + | x string, + | y int, + | a int, + | b bigint + |) partitioned by (a, b) with ( + | 'connector' = 'filesystem', + | 'path' = '$resultPath', + | ${formatProperties().mkString(",\n")} + |) + """.stripMargin + ) + tableEnv.sqlUpdate( + s""" + |create table nonPartitionedTable ( + | x string, + | y int, + | a int, + | b bigint + |) with ( + | 'connector' = 'filesystem', + | 'path' = '$resultPath', + | ${formatProperties().mkString(",\n")} + |) + """.stripMargin + ) + } + + @Test + def testAllStaticPartitions1(): Unit = { + tableEnv.sqlUpdate("insert into partitionedTable " + + "partition(a='1', b='1') select x, y from originalT where a=1 and b=1") + tableEnv.execute("test") + + check( + "select x, y from partitionedTable where a=1 and b=1", + data_partition_1_1 + ) + + check( + "select x, y from partitionedTable", + data_partition_1_1 + ) + } + + @Test + def testAllStaticPartitions2(): Unit = { + tableEnv.sqlUpdate("insert into partitionedTable " + + "partition(a='2', b='1') select x, y from originalT where a=2 and b=1") + tableEnv.execute("test") + + check( + "select x, y from partitionedTable where a=2 and b=1", + data_partition_2_1 + ) + + check( + "select x, y from partitionedTable", + data_partition_2_1 + ) + } + + @Test + def testPartialDynamicPartition(): Unit = { + tableEnv.sqlUpdate("insert into partitionedTable " + + "partition(a=3) select x, y, b from originalT where a=3") + tableEnv.execute("test") + + check( + "select x, y from partitionedTable where a=2 and b=1", + Seq() + ) + + check( + "select x, y from partitionedTable where a=3 and b=1", + Seq( + row("x17", 17) + ) + ) + + check( + "select x, y from partitionedTable where a=3 and b=2", + Seq( + row("x18", 18) + ) + ) + + check( + "select x, y from partitionedTable where a=3 and b=3", + Seq( + row("x19", 19) + ) + ) + + check( + "select x, y from partitionedTable where a=3", + Seq( + row("x17", 17), + row("x18", 18), + row("x19", 19) + ) + ) + } + + @Test + def testDynamicPartition(): Unit = { + tableEnv.sqlUpdate("insert into partitionedTable " + + "select x, y, a, b from originalT") + tableEnv.execute("test") + + check( + "select x, y from partitionedTable where a=1 and b=1", + data_partition_1_1 + ) + + check( + "select x, y from partitionedTable where a=2 and b=1", + data_partition_2_1 + ) + + check( + "select x, y from partitionedTable", + data + ) + } + + @Test + def testNonPartition(): Unit = { + tableEnv.sqlUpdate("insert into nonPartitionedTable " + + "select x, y, a, b from originalT where a=1 and b=1") + tableEnv.execute("test") + + check( + "select x, y from nonPartitionedTable where a=1 and b=1", + data_partition_1_1 + ) + } + + @Test + def testLimitPushDown(): Unit = { + tableEnv.getConfig.getConfiguration.setInteger( + ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1) + tableEnv.sqlUpdate("insert into nonPartitionedTable select x, y, a, b from originalT") + tableEnv.execute("test") + + check( + "select x, y from nonPartitionedTable limit 3", + Seq( + row("x1", 1), + row("x2", 2), + row("x3", 3))) + } + + @Test + def testFilterPushDown(): Unit = { + tableEnv.sqlUpdate("insert into nonPartitionedTable select x, y, a, b from originalT") + tableEnv.execute("test") + + check( + "select x, y from nonPartitionedTable where a=10086", + Seq()) + } +} + +object FileSystemITCaseBase { + + val fieldNames = Array("x", "y", "a", "b") + + val fieldTypes: Array[TypeInformation[_]] = Array( + Types.STRING, + Types.INT, + Types.INT, + Types.LONG) + val dataType = new RowTypeInfo(fieldTypes :_*) + + val data_with_partitions: Seq[Row] = Seq( + row("x1", 1, 1, 1L), + row("x2", 2, 1, 1L), + row("x3", 3, 1, 1L), + row("x4", 4, 1, 1L), + row("x5", 5, 1, 1L), + row("x6", 6, 1, 2L), + row("x7", 7, 1, 2L), + row("x8", 8, 1, 2L), + row("x9", 9, 1, 2L), + row("x10", 10, 1, 2L), + row("x11", 11, 2, 1L), + row("x12", 12, 2, 1L), + row("x13", 13, 2, 1L), + row("x14", 14, 2, 1L), + row("x15", 15, 2, 1L), + row("x16", 16, 2, 2L), + row("x17", 17, 3, 1L), + row("x18", 18, 3, 2L), + row("x19", 19, 3, 3L), + row("x20", 20, 4, 1L), + row("x21", 21, 4, 2L), + row("x22", 22, 4, 3L), + row("x23", 23, 4, 4L), + row("x24", 24, 5, 1L), + row("x25", 25, 5, 2L), + row("x26", 26, 5, 3L), + row("x27", 27, 5, 4L) + ) + + val data: Seq[Row] = data_with_partitions.map(row => Row.of(row.getField(0), row.getField(1))) + + val data_partition_1_1: Seq[Row] = Seq( + row("x1", 1), + row("x2", 2), + row("x3", 3), + row("x4", 4), + row("x5", 5) + ) + + val data_partition_2_1: Seq[Row] = Seq( + row("x11", 11), + row("x12", 12), + row("x13", 13), + row("x14", 14), + row("x15", 15) + ) +} diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/BatchFileSystemITCaseBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/BatchFileSystemITCaseBase.scala new file mode 100644 index 0000000000..b7d923cc0a --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/BatchFileSystemITCaseBase.scala @@ -0,0 +1,48 @@ +/* + * 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.table.planner.runtime.batch.sql + +import org.apache.flink.table.api.TableEnvironment +import org.apache.flink.table.planner.runtime.FileSystemITCaseBase +import org.apache.flink.table.planner.runtime.utils.BatchTestBase +import org.apache.flink.types.Row + +import org.junit.Before + +import scala.collection.Seq + +/** + * Batch [[FileSystemITCaseBase]]. + */ +abstract class BatchFileSystemITCaseBase extends BatchTestBase with FileSystemITCaseBase { + + @Before + override def before(): Unit = { + super.before() + super.open() + } + + override def tableEnv: TableEnvironment = { + tEnv + } + + override def check(sqlQuery: String, expectedResult: Seq[Row]): Unit = { + checkResult(sqlQuery, expectedResult) + } +} diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/FileSystemTestCsvITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/FileSystemTestCsvITCase.scala new file mode 100644 index 0000000000..cdf0a8c951 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/FileSystemTestCsvITCase.scala @@ -0,0 +1,44 @@ +/* + * 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.table.planner.runtime.batch.sql + +import org.apache.flink.table.planner.utils.TestCsvFileSystemFormatFactory.USE_BULK_WRITER + +import org.junit.runner.RunWith +import org.junit.runners.Parameterized + +/** + * Test for file system table factory with testcsv format. + */ +@RunWith(classOf[Parameterized]) +class FileSystemTestCsvITCase(useBulkWriter: Boolean) extends BatchFileSystemITCaseBase { + + override def formatProperties(): Array[String] = { + super.formatProperties() ++ Seq( + "'format' = 'testcsv'", + s"'$USE_BULK_WRITER' = '$useBulkWriter'") + } +} + +object FileSystemTestCsvITCase { + @Parameterized.Parameters(name = "useBulkWriter-{0}") + def parameters(): java.util.Collection[Boolean] = { + java.util.Arrays.asList(true, false) + } +} diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/StreamFileSystemITCaseBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/StreamFileSystemITCaseBase.scala new file mode 100644 index 0000000000..c32379f6b3 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/StreamFileSystemITCaseBase.scala @@ -0,0 +1,58 @@ +/* + * 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.table.planner.runtime.stream.sql + +import org.apache.flink.api.scala._ +import org.apache.flink.table.api.TableEnvironment +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.planner.runtime.FileSystemITCaseBase +import org.apache.flink.table.planner.runtime.utils.{StreamingTestBase, TestSinkUtil, TestingAppendSink} +import org.apache.flink.types.Row + +import org.junit.Assert.assertEquals +import org.junit.Before + +import scala.collection.Seq + +/** + * Streaming [[FileSystemITCaseBase]]. + */ +abstract class StreamFileSystemITCaseBase extends StreamingTestBase with FileSystemITCaseBase { + + @Before + override def before(): Unit = { + super.before() + super.open() + } + + override def tableEnv: TableEnvironment = { + tEnv + } + + override def check(sqlQuery: String, expectedResult: Seq[Row]): Unit = { + val result = tEnv.sqlQuery(sqlQuery).toAppendStream[Row] + val sink = new TestingAppendSink() + result.addSink(sink) + env.execute() + + assertEquals( + expectedResult.map(TestSinkUtil.rowToString(_)).sorted, + sink.getAppendResults.sorted) + } +} diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/TestSinkUtil.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/TestSinkUtil.scala index b9cfa8d6e2..ed452996fd 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/TestSinkUtil.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/TestSinkUtil.scala @@ -62,7 +62,7 @@ object TestSinkUtil { } } - def rowToString(row: Row, tz: TimeZone): String = { + def rowToString(row: Row, tz: TimeZone = TimeZone.getTimeZone("UTC")): String = { val sb = StringBuilder.newBuilder for (i <- 0 until row.getArity ) { if (i > 0) { diff --git a/flink-table/flink-table-runtime-blink/pom.xml b/flink-table/flink-table-runtime-blink/pom.xml index e8ed8afdf6..62115b9b26 100644 --- a/flink-table/flink-table-runtime-blink/pom.xml +++ b/flink-table/flink-table-runtime-blink/pom.xml @@ -54,6 +54,12 @@ under the License. ${project.version} + + org.apache.flink + flink-table-api-java-bridge_${scala.binary.version} + ${project.version} + + org.apache.flink flink-streaming-java_${scala.binary.version} diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/FileSystemFormatFactory.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/FileSystemFormatFactory.java new file mode 100644 index 0000000000..48a4792ba1 --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/FileSystemFormatFactory.java @@ -0,0 +1,147 @@ +/* + * 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.table.filesystem; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.io.InputFormat; +import org.apache.flink.api.common.serialization.BulkWriter; +import org.apache.flink.api.common.serialization.Encoder; +import org.apache.flink.core.fs.Path; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.dataformat.BaseRow; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.factories.TableFormatFactory; +import org.apache.flink.table.types.DataType; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +/** + * File system format factory for creating configured instances of reader and writer. + */ +@Internal +public interface FileSystemFormatFactory extends TableFormatFactory { + + /** + * Create {@link InputFormat} reader. + */ + InputFormat createReader(ReaderContext context); + + /** + * Create {@link Encoder} writer. + */ + Optional> createEncoder(WriterContext context); + + /** + * Create {@link BulkWriter.Factory} writer. + */ + Optional> createBulkWriterFactory(WriterContext context); + + /** + * Context of {@link #createReader}. + */ + interface ReaderContext { + + /** + * Full schema of the table. + */ + TableSchema getSchema(); + + /** + * Properties of this format. + */ + Map getFormatProperties(); + + /** + * Partition keys of the table. + */ + List getPartitionKeys(); + + /** + * The default partition name in case the dynamic partition column value is + * null/empty string. + */ + String getDefaultPartName(); + + /** + * Read paths. + */ + Path[] getPaths(); + + /** + * Project the fields of the reader returned. + */ + int[] getProjectFields(); + + /** + * Limiting push-down to reader. Reader only needs to try its best to limit the number + * of output records, but does not need to guarantee that the number must be less than or + * equal to the limit. + */ + long getPushedDownLimit(); + + /** + * Pushed down filters, reader can try its best to filter records. + * The follow up operator will filter the records again. + */ + List getPushedDownFilters(); + } + + /** + * Context of {@link #createEncoder} and {@link #createBulkWriterFactory}. + */ + interface WriterContext { + + /** + * Full schema of the table. + */ + TableSchema getSchema(); + + /** + * Properties of this format. + */ + Map getFormatProperties(); + + /** + * Partition keys of the table. + */ + List getPartitionKeys(); + + /** + * Get field names without partition keys. + */ + default String[] getFieldNamesWithoutPartKeys() { + return Arrays.stream(getSchema().getFieldNames()) + .filter(name -> !getPartitionKeys().contains(name)) + .toArray(String[]::new); + } + + /** + * Get field types without partition keys. + */ + default DataType[] getFieldTypesWithoutPartKeys() { + return Arrays.stream(getSchema().getFieldNames()) + .filter(name -> !getPartitionKeys().contains(name)) + .map(name -> getSchema().getFieldDataType(name).get()) + .toArray(DataType[]::new); + } + } +} diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/FileSystemTableFactory.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/FileSystemTableFactory.java new file mode 100644 index 0000000000..6d658fcef0 --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/FileSystemTableFactory.java @@ -0,0 +1,143 @@ +/* + * 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.table.filesystem; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.core.fs.Path; +import org.apache.flink.table.dataformat.BaseRow; +import org.apache.flink.table.descriptors.DescriptorProperties; +import org.apache.flink.table.factories.TableFactory; +import org.apache.flink.table.factories.TableFactoryService; +import org.apache.flink.table.factories.TableSinkFactory; +import org.apache.flink.table.factories.TableSourceFactory; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.table.sources.TableSource; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.configuration.ConfigOptions.key; +import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR; +import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT; +import static org.apache.flink.table.descriptors.Schema.SCHEMA; + +/** + * File system {@link TableFactory}. + * + *

    1.The partition information should be in the file system path, whether it's a temporary + * table or a catalog table. + * 2.Support insert into (append) and insert overwrite. + * 3.Support static and dynamic partition inserting. + * + *

    Migrate to new source/sink interface after FLIP-95 is ready. + */ +public class FileSystemTableFactory implements + TableSourceFactory, + TableSinkFactory { + + public static final String CONNECTOR_VALUE = "filesystem"; + + /** + * Not use "connector.path" because: + * 1.Using "connector.path" will conflict with current batch csv source and batch csv sink. + * 2.This is compatible with FLIP-122. + */ + public static final String PATH = "path"; + + /** + * Move these properties to validator after FLINK-16904. + */ + public static final ConfigOption PARTITION_DEFAULT_NAME = key("partition.default-name") + .stringType() + .defaultValue("__DEFAULT_PARTITION__") + .withDescription("The default partition name in case the dynamic partition" + + " column value is null/empty string"); + + @Override + public Map requiredContext() { + Map context = new HashMap<>(); + context.put(CONNECTOR, CONNECTOR_VALUE); + return context; + } + + @Override + public List supportedProperties() { + List properties = new ArrayList<>(); + + // path + properties.add(PATH); + + // schema + properties.add(SCHEMA + ".#." + DescriptorProperties.TABLE_SCHEMA_DATA_TYPE); + properties.add(SCHEMA + ".#." + DescriptorProperties.TABLE_SCHEMA_NAME); + + properties.add(PARTITION_DEFAULT_NAME.key()); + + // format + properties.add(FORMAT); + properties.add(FORMAT + ".*"); + + return properties; + } + + @Override + public TableSource createTableSource(TableSourceFactory.Context context) { + DescriptorProperties properties = new DescriptorProperties(); + properties.putProperties(context.getTable().getProperties()); + + return new FileSystemTableSource( + context.getTable().getSchema(), + new Path(properties.getString(PATH)), + context.getTable().getPartitionKeys(), + getPartitionDefaultName(properties), + getFormatProperties(context.getTable().getProperties())); + } + + @Override + public TableSink createTableSink(TableSinkFactory.Context context) { + DescriptorProperties properties = new DescriptorProperties(); + properties.putProperties(context.getTable().getProperties()); + + return new FileSystemTableSink( + context.getTable().getSchema(), + new Path(properties.getString(PATH)), + context.getTable().getPartitionKeys(), + getPartitionDefaultName(properties), + getFormatProperties(context.getTable().getProperties())); + } + + private static Map getFormatProperties(Map tableProperties) { + return tableProperties; + } + + private static String getPartitionDefaultName(DescriptorProperties properties) { + return properties + .getOptionalString(PARTITION_DEFAULT_NAME.key()) + .orElse(PARTITION_DEFAULT_NAME.defaultValue()); + } + + public static FileSystemFormatFactory createFormatFactory(Map properties) { + return TableFactoryService.find( + FileSystemFormatFactory.class, + properties, + FileSystemTableFactory.class.getClassLoader()); + } +} diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/FileSystemTableSink.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/FileSystemTableSink.java new file mode 100644 index 0000000000..06d453c2ca --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/FileSystemTableSink.java @@ -0,0 +1,289 @@ +/* + * 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.table.filesystem; + +import org.apache.flink.api.common.io.OutputFormat; +import org.apache.flink.api.common.serialization.BulkWriter; +import org.apache.flink.api.common.serialization.Encoder; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.FSDataOutputStream; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.dataformat.BaseRow; +import org.apache.flink.table.sinks.AppendStreamTableSink; +import org.apache.flink.table.sinks.OverwritableTableSink; +import org.apache.flink.table.sinks.PartitionableTableSink; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.table.types.DataType; +import org.apache.flink.util.Preconditions; + +import java.io.IOException; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static org.apache.flink.table.filesystem.FileSystemTableFactory.createFormatFactory; + +/** + * File system {@link TableSink}. + */ +public class FileSystemTableSink implements + AppendStreamTableSink, + PartitionableTableSink, + OverwritableTableSink { + + private final TableSchema schema; + private final List partitionKeys; + private final Path path; + private final String defaultPartName; + private final Map formatProperties; + + private boolean overwrite = false; + private boolean dynamicGrouping = false; + private LinkedHashMap staticPartitions = new LinkedHashMap<>(); + + /** + * Construct a file system table sink. + * + * @param schema schema of the table. + * @param path directory path of the file system table. + * @param partitionKeys partition keys of the table. + * @param defaultPartName The default partition name in case the dynamic partition column value + * is null/empty string. + * @param formatProperties format properties. + */ + public FileSystemTableSink( + TableSchema schema, + Path path, + List partitionKeys, + String defaultPartName, + Map formatProperties) { + this.schema = schema; + this.path = path; + this.defaultPartName = defaultPartName; + this.formatProperties = formatProperties; + this.partitionKeys = partitionKeys; + } + + @Override + public final DataStreamSink consumeDataStream(DataStream dataStream) { + RowDataPartitionComputer computer = new RowDataPartitionComputer( + defaultPartName, + schema.getFieldNames(), + schema.getFieldDataTypes(), + partitionKeys.toArray(new String[0])); + + FileSystemOutputFormat.Builder builder = new FileSystemOutputFormat.Builder<>(); + builder.setPartitionComputer(computer); + builder.setDynamicGrouped(dynamicGrouping); + builder.setPartitionColumns(partitionKeys.toArray(new String[0])); + builder.setFormatFactory(createOutputFormatFactory()); + builder.setMetaStoreFactory(createTableMetaStoreFactory(path)); + builder.setOverwrite(overwrite); + builder.setStaticPartitions(staticPartitions); + builder.setTempPath(toStagingPath()); + return dataStream.writeUsingOutputFormat(builder.build()) + .setParallelism(dataStream.getParallelism()); + } + + private Path toStagingPath() { + Path stagingDir = new Path(path, ".staging_" + System.currentTimeMillis()); + try { + FileSystem fs = stagingDir.getFileSystem(); + Preconditions.checkState( + fs.exists(stagingDir) || fs.mkdirs(stagingDir), + "Failed to create staging dir " + stagingDir); + return stagingDir; + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private OutputFormatFactory createOutputFormatFactory() { + FileSystemFormatFactory formatFactory = createFormatFactory(formatProperties); + FileSystemFormatFactory.WriterContext context = new FileSystemFormatFactory.WriterContext() { + + @Override + public TableSchema getSchema() { + return schema; + } + + @Override + public Map getFormatProperties() { + return formatProperties; + } + + @Override + public List getPartitionKeys() { + return partitionKeys; + } + }; + + Optional> encoder = formatFactory.createEncoder(context); + Optional> bulk = formatFactory.createBulkWriterFactory(context); + + if (!encoder.isPresent() && !bulk.isPresent()) { + throw new TableException( + formatFactory + " format should implement at least one Encoder or BulkWriter"); + } + return encoder + .>map(en -> path -> createEncoderOutputFormat(en, path)) + .orElseGet(() -> { + // Optional is not serializable. + BulkWriter.Factory bulkWriterFactory = bulk.get(); + return path -> createBulkWriterOutputFormat(bulkWriterFactory, path); + }); + } + + private static OutputFormat createBulkWriterOutputFormat( + BulkWriter.Factory factory, + Path path) { + return new OutputFormat() { + + private static final long serialVersionUID = 1L; + + private transient BulkWriter writer; + + @Override + public void configure(Configuration parameters) { + } + + @Override + public void open(int taskNumber, int numTasks) throws IOException { + this.writer = factory.create(path.getFileSystem() + .create(path, FileSystem.WriteMode.OVERWRITE)); + } + + @Override + public void writeRecord(BaseRow record) throws IOException { + writer.addElement(record); + } + + @Override + public void close() throws IOException { + writer.flush(); + writer.finish(); + } + }; + } + + private static OutputFormat createEncoderOutputFormat( + Encoder encoder, + Path path) { + return new OutputFormat() { + + private static final long serialVersionUID = 1L; + + private transient FSDataOutputStream output; + + @Override + public void configure(Configuration parameters) { + } + + @Override + public void open(int taskNumber, int numTasks) throws IOException { + this.output = path.getFileSystem() + .create(path, FileSystem.WriteMode.OVERWRITE); + } + + @Override + public void writeRecord(BaseRow record) throws IOException { + encoder.encode(record, output); + } + + @Override + public void close() throws IOException { + this.output.flush(); + this.output.close(); + } + }; + } + + private static TableMetaStoreFactory createTableMetaStoreFactory(Path path) { + return (TableMetaStoreFactory) () -> new TableMetaStoreFactory.TableMetaStore() { + + @Override + public Path getLocationPath() { + return path; + } + + @Override + public Optional getPartition(LinkedHashMap partitionSpec) { + return Optional.empty(); + } + + @Override + public void createPartition( + LinkedHashMap partitionSpec, + Path partitionPath) { + } + + @Override + public void close() { + } + }; + } + + @Override + public FileSystemTableSink configure(String[] fieldNames, TypeInformation[] fieldTypes) { + return this; + } + + @Override + public void setOverwrite(boolean overwrite) { + this.overwrite = overwrite; + } + + @Override + public void setStaticPartition(Map partitions) { + this.staticPartitions = toPartialLinkedPartSpec(partitions); + } + + private LinkedHashMap toPartialLinkedPartSpec(Map part) { + LinkedHashMap partSpec = new LinkedHashMap<>(); + for (String partitionKey : partitionKeys) { + if (part.containsKey(partitionKey)) { + partSpec.put(partitionKey, part.get(partitionKey)); + } + } + return partSpec; + } + + @Override + public TableSchema getTableSchema() { + return schema; + } + + @Override + public DataType getConsumedDataType() { + return schema.toRowDataType().bridgedTo(BaseRow.class); + } + + @Override + public boolean configurePartitionGrouping(boolean supportsGrouping) { + this.dynamicGrouping = supportsGrouping; + return dynamicGrouping; + } +} diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/FileSystemTableSource.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/FileSystemTableSource.java new file mode 100644 index 0000000000..8c0befdfb6 --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/FileSystemTableSource.java @@ -0,0 +1,307 @@ +/* + * 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.table.filesystem; + +import org.apache.flink.api.common.io.InputFormat; +import org.apache.flink.api.java.io.CollectionInputFormat; +import org.apache.flink.core.fs.Path; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.dataformat.BaseRow; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.sources.FilterableTableSource; +import org.apache.flink.table.sources.InputFormatTableSource; +import org.apache.flink.table.sources.LimitableTableSource; +import org.apache.flink.table.sources.PartitionableTableSource; +import org.apache.flink.table.sources.ProjectableTableSource; +import org.apache.flink.table.types.DataType; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.apache.flink.table.filesystem.FileSystemTableFactory.createFormatFactory; + +/** + * File system table source. + */ +public class FileSystemTableSource extends InputFormatTableSource implements + PartitionableTableSource, + ProjectableTableSource, + LimitableTableSource, + FilterableTableSource { + + private final TableSchema schema; + private final Path path; + private final List partitionKeys; + private final String defaultPartName; + private final Map formatProperties; + + private final int[] selectFields; + private final Long limit; + private final List filters; + + private List> readPartitions; + + /** + * Construct a file system table source. + * + * @param schema schema of the table. + * @param path directory path of the file system table. + * @param partitionKeys partition keys of the table. + * @param defaultPartName The default partition name in case the dynamic partition column value + * is null/empty string. + * @param formatProperties format properties. + */ + public FileSystemTableSource( + TableSchema schema, + Path path, + List partitionKeys, + String defaultPartName, + Map formatProperties) { + this(schema, path, partitionKeys, defaultPartName, formatProperties, null, null, null, null); + } + + private FileSystemTableSource( + TableSchema schema, + Path path, + List partitionKeys, + String defaultPartName, + Map formatProperties, + List> readPartitions, + int[] selectFields, + Long limit, + List filters) { + this.schema = schema; + this.path = path; + this.partitionKeys = partitionKeys; + this.defaultPartName = defaultPartName; + this.formatProperties = formatProperties; + this.readPartitions = readPartitions; + this.selectFields = selectFields; + this.limit = limit; + this.filters = filters; + } + + @Override + public InputFormat getInputFormat() { + // When this table has no partition, just return a empty source. + if (!partitionKeys.isEmpty() && getOrFetchPartitions().isEmpty()) { + return new CollectionInputFormat<>(new ArrayList<>(), null); + } + + return createFormatFactory(formatProperties).createReader( + new FileSystemFormatFactory.ReaderContext() { + + @Override + public TableSchema getSchema() { + return schema; + } + + @Override + public Map getFormatProperties() { + return formatProperties; + } + + @Override + public List getPartitionKeys() { + return partitionKeys; + } + + @Override + public String getDefaultPartName() { + return defaultPartName; + } + + @Override + public Path[] getPaths() { + if (partitionKeys.isEmpty()) { + return new Path[] {path}; + } else { + return getOrFetchPartitions().stream() + .map(FileSystemTableSource.this::toFullLinkedPartSpec) + .map(PartitionPathUtils::generatePartitionPath) + .map(n -> new Path(path, n)) + .toArray(Path[]::new); + } + } + + @Override + public int[] getProjectFields() { + return readFields(); + } + + @Override + public long getPushedDownLimit() { + return limit == null ? Long.MAX_VALUE : limit; + } + + @Override + public List getPushedDownFilters() { + return filters == null ? Collections.emptyList() : filters; + } + }); + } + + private List> getOrFetchPartitions() { + if (readPartitions == null) { + readPartitions = getPartitions(); + } + return readPartitions; + } + + private LinkedHashMap toFullLinkedPartSpec(Map part) { + LinkedHashMap map = new LinkedHashMap<>(); + for (String k : partitionKeys) { + if (!part.containsKey(k)) { + throw new TableException("Partition keys are: " + partitionKeys + + ", incomplete partition spec: " + part); + } + map.put(k, part.get(k)); + } + return map; + } + + @Override + public List> getPartitions() { + try { + return PartitionPathUtils + .searchPartSpecAndPaths(path.getFileSystem(), path, partitionKeys.size()) + .stream() + .map(tuple2 -> tuple2.f0) + .map(spec -> { + LinkedHashMap ret = new LinkedHashMap<>(); + spec.forEach((k, v) -> ret.put(k, defaultPartName.equals(v) ? null : v)); + return ret; + }) + .collect(Collectors.toList()); + } catch (Exception e) { + throw new TableException("Fetch partitions fail.", e); + } + } + + @Override + public FileSystemTableSource applyPartitionPruning( + List> remainingPartitions) { + return new FileSystemTableSource( + schema, + path, + partitionKeys, + defaultPartName, + formatProperties, + remainingPartitions, + selectFields, + limit, + filters); + } + + @Override + public FileSystemTableSource projectFields(int[] fields) { + return new FileSystemTableSource( + schema, + path, + partitionKeys, + defaultPartName, + formatProperties, + readPartitions, + fields, + limit, + filters); + } + + @Override + public FileSystemTableSource applyLimit(long limit) { + return new FileSystemTableSource( + schema, + path, + partitionKeys, + defaultPartName, + formatProperties, + readPartitions, + selectFields, + limit, + filters); + } + + @Override + public boolean isLimitPushedDown() { + return limit != null; + } + + @Override + public FileSystemTableSource applyPredicate(List predicates) { + return new FileSystemTableSource( + schema, + path, + partitionKeys, + defaultPartName, + formatProperties, + readPartitions, + selectFields, + limit, + new ArrayList<>(predicates)); + } + + @Override + public boolean isFilterPushedDown() { + return this.filters != null; + } + + private int[] readFields() { + return selectFields == null ? + IntStream.range(0, schema.getFieldCount()).toArray() : + selectFields; + } + + @Override + public DataType getProducedDataType() { + int[] fields = readFields(); + String[] schemaFieldNames = schema.getFieldNames(); + DataType[] schemaTypes = schema.getFieldDataTypes(); + + return DataTypes.ROW(Arrays.stream(fields) + .mapToObj(i -> DataTypes.FIELD(schemaFieldNames[i], schemaTypes[i])) + .toArray(DataTypes.Field[]::new)) + .bridgedTo(BaseRow.class); + } + + @Override + public TableSchema getTableSchema() { + return schema; + } + + @Override + public String explainSource() { + return super.explainSource() + + (readPartitions == null ? "" : ", readPartitions=" + readPartitions) + + (selectFields == null ? "" : ", selectFields=" + Arrays.toString(selectFields)) + + (limit == null ? "" : ", limit=" + limit) + + (filters == null ? "" : ", filters=" + filtersString()); + } + + private String filtersString() { + return filters.stream().map(Expression::asSummaryString).collect(Collectors.joining(",")); + } +} diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/RowDataPartitionComputer.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/RowDataPartitionComputer.java new file mode 100644 index 0000000000..a55c3221cb --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/filesystem/RowDataPartitionComputer.java @@ -0,0 +1,112 @@ +/* + * 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.table.filesystem; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.dataformat.BaseRow; +import org.apache.flink.table.dataformat.GenericRow; +import org.apache.flink.table.dataformat.TypeGetterSetters; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; + +import java.util.Arrays; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +/** + * {@link PartitionComputer} for {@link BaseRow}. + */ +@Internal +public class RowDataPartitionComputer implements PartitionComputer { + + private static final long serialVersionUID = 1L; + + protected final String defaultPartValue; + protected final String[] partitionColumns; + protected final int[] partitionIndexes; + protected final LogicalType[] partitionTypes; + + private final int[] nonPartitionIndexes; + private final LogicalType[] nonPartitionTypes; + + private transient GenericRow reuseRow; + + public RowDataPartitionComputer( + String defaultPartValue, + String[] columnNames, + DataType[] columnTypes, + String[] partitionColumns) { + this.defaultPartValue = defaultPartValue; + this.partitionColumns = partitionColumns; + + List columnList = Arrays.asList(columnNames); + List columnTypeList = Arrays.stream(columnTypes) + .map(DataType::getLogicalType) + .collect(Collectors.toList()); + + this.partitionIndexes = Arrays.stream(partitionColumns) + .mapToInt(columnList::indexOf) + .toArray(); + this.partitionTypes = Arrays.stream(partitionIndexes) + .mapToObj(columnTypeList::get) + .toArray(LogicalType[]::new); + + List partitionIndexList = Arrays.stream(partitionIndexes).boxed().collect(Collectors.toList()); + this.nonPartitionIndexes = IntStream.range(0, columnNames.length) + .filter(c -> !partitionIndexList.contains(c)) + .toArray(); + this.nonPartitionTypes = Arrays.stream(nonPartitionIndexes) + .mapToObj(columnTypeList::get) + .toArray(LogicalType[]::new); + } + + @Override + public LinkedHashMap generatePartValues(BaseRow in) { + LinkedHashMap partSpec = new LinkedHashMap<>(); + + for (int i = 0; i < partitionIndexes.length; i++) { + Object field = TypeGetterSetters.get(in, partitionIndexes[i], partitionTypes[i]); + String partitionValue = field != null ? field.toString() : null; + if (partitionValue == null || "".equals(partitionValue)) { + partitionValue = defaultPartValue; + } + partSpec.put(partitionColumns[i], partitionValue); + } + return partSpec; + } + + @Override + public BaseRow projectColumnsToWrite(BaseRow in) { + if (partitionIndexes.length == 0) { + return in; + } + + if (reuseRow == null) { + this.reuseRow = new GenericRow(nonPartitionIndexes.length); + } + + for (int i = 0; i < nonPartitionIndexes.length; i++) { + reuseRow.setField(i, TypeGetterSetters.get( + in, nonPartitionIndexes[i], nonPartitionTypes[i])); + } + return reuseRow; + } +} diff --git a/flink-table/flink-table-runtime-blink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink-table/flink-table-runtime-blink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory new file mode 100644 index 0000000000..544c566d80 --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory @@ -0,0 +1,16 @@ +# 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. + +org.apache.flink.table.filesystem.FileSystemTableFactory -- Gitee From 04096fcb3d71ec29a4c695761496651c1f0fb3b2 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Tue, 7 Apr 2020 14:54:37 +0800 Subject: [PATCH 461/885] [FLINK-14266][csv] Introduce RowCsvInputFormat to CSV module This closes #9884 --- .../formats/csv/AbstractCsvInputFormat.java | 130 +++ .../csv/CsvRowDeserializationSchema.java | 6 +- .../flink/formats/csv/RowCsvInputFormat.java | 232 ++++++ .../csv/RowCsvInputFormatSplitTest.java | 173 ++++ .../formats/csv/RowCsvInputFormatTest.java | 756 ++++++++++++++++++ 5 files changed, 1294 insertions(+), 3 deletions(-) create mode 100644 flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/AbstractCsvInputFormat.java create mode 100644 flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/RowCsvInputFormat.java create mode 100644 flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/RowCsvInputFormatSplitTest.java create mode 100644 flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/RowCsvInputFormatTest.java diff --git a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/AbstractCsvInputFormat.java b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/AbstractCsvInputFormat.java new file mode 100644 index 0000000000..c870efe9e7 --- /dev/null +++ b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/AbstractCsvInputFormat.java @@ -0,0 +1,130 @@ +/* + * 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.formats.csv; + +import org.apache.flink.api.common.io.FileInputFormat; +import org.apache.flink.core.fs.FileInputSplit; +import org.apache.flink.core.fs.Path; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvSchema; + +import org.apache.commons.compress.utils.BoundedInputStream; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Input format that reads csv. This abstract class is responsible for cutting the boundary of + * InputSplit. + */ +public abstract class AbstractCsvInputFormat extends FileInputFormat { + + protected final CsvSchema csvSchema; + protected transient InputStream csvInputStream; + + public AbstractCsvInputFormat(Path[] filePaths, CsvSchema csvSchema) { + setFilePaths(filePaths); + this.csvSchema = checkNotNull(csvSchema); + } + + @Override + public boolean supportsMultiPaths() { + return true; + } + + @Override + public void open(FileInputSplit split) throws IOException { + super.open(split); + + csvInputStream = stream; + + long csvStart = splitStart; + if (splitStart != 0) { + csvStart = findNextLineStartOffset(); + } + + if (splitLength != READ_WHOLE_SPLIT_FLAG) { + stream.seek(splitStart + splitLength); + long nextLineStartOffset = findNextLineStartOffset(); + stream.seek(csvStart); + csvInputStream = new BoundedInputStream(stream, nextLineStartOffset - csvStart); + } else { + stream.seek(csvStart); + } + } + + /** + * Find next legal line separator to return next offset (first byte offset of next line). + * + *

    NOTE: Because of the particularity of UTF-8 encoding, we can determine the number of bytes + * of this character only by comparing the first byte, so we do not need to traverse M*N in comparison. + */ + private long findNextLineStartOffset() throws IOException { + boolean usesEscapeChar = csvSchema.usesEscapeChar(); + byte[] escapeBytes = Character.toString((char) csvSchema.getEscapeChar()) + .getBytes(StandardCharsets.UTF_8); + long startPos = stream.getPos(); + + byte b; + while ((b = (byte) stream.read()) != -1) { + if (b == '\r' || b == '\n') { + // If there may be escape tags ahead + if (usesEscapeChar && stream.getPos() - startPos <= escapeBytes.length) { + long front = stream.getPos() - escapeBytes.length - 1; + if (front > 0) { + stream.seek(front); + byte[] readBytes = new byte[escapeBytes.length]; + stream.read(readBytes); // we have judge front must bigger than zero + stream.read(); // back to current next one + if (Arrays.equals(escapeBytes, readBytes)) { + // equal, we should skip this one line separator + continue; + } + } + } + + long pos = stream.getPos(); + + // deal with "\r\n", next one maybe '\n', so we need skip it. + if (b == '\r' && (byte) stream.read() == '\n') { + return stream.getPos(); + } else { + return pos; + } + } else if (usesEscapeChar && b == escapeBytes[0]) { + boolean equal = true; + for (int i = 1; i < escapeBytes.length; i++) { + if ((byte) stream.read() != escapeBytes[i]) { + equal = false; + break; + } + } + if (equal) { + // equal, we should skip next one + stream.skip(1); + } + } + } + return stream.getPos(); + } +} diff --git a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvRowDeserializationSchema.java b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvRowDeserializationSchema.java index 825300137b..684bea7c1c 100644 --- a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvRowDeserializationSchema.java +++ b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvRowDeserializationSchema.java @@ -213,7 +213,7 @@ public final class CsvRowDeserializationSchema implements DeserializationSchema< // -------------------------------------------------------------------------------------------- - private interface RuntimeConverter extends Serializable { + interface RuntimeConverter extends Serializable { Object convert(JsonNode node); } @@ -230,7 +230,7 @@ public final class CsvRowDeserializationSchema implements DeserializationSchema< return assembleRowRuntimeConverter(ignoreParseErrors, isTopLevel, fieldNames, fieldConverters); } - private static RuntimeConverter[] createFieldRuntimeConverters(boolean ignoreParseErrors, TypeInformation[] fieldTypes) { + static RuntimeConverter[] createFieldRuntimeConverters(boolean ignoreParseErrors, TypeInformation[] fieldTypes) { final RuntimeConverter[] fieldConverters = new RuntimeConverter[fieldTypes.length]; for (int i = 0; i < fieldTypes.length; i++) { fieldConverters[i] = createNullableRuntimeConverter(fieldTypes[i], ignoreParseErrors); @@ -365,7 +365,7 @@ public final class CsvRowDeserializationSchema implements DeserializationSchema< }; } - private static void validateArity(int expected, int actual, boolean ignoreParseErrors) { + static void validateArity(int expected, int actual, boolean ignoreParseErrors) { if (expected != actual && !ignoreParseErrors) { throw new RuntimeException("Row length mismatch. " + expected + " fields expected but was " + actual + "."); diff --git a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/RowCsvInputFormat.java b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/RowCsvInputFormat.java new file mode 100644 index 0000000000..df4a222d22 --- /dev/null +++ b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/RowCsvInputFormat.java @@ -0,0 +1,232 @@ +/* + * 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.formats.csv; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.core.fs.FileInputSplit; +import org.apache.flink.core.fs.Path; +import org.apache.flink.formats.csv.CsvRowDeserializationSchema.RuntimeConverter; +import org.apache.flink.types.Row; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.MappingIterator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvSchema; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Arrays; +import java.util.NoSuchElementException; + +import static org.apache.flink.formats.csv.CsvRowDeserializationSchema.createFieldRuntimeConverters; +import static org.apache.flink.formats.csv.CsvRowDeserializationSchema.validateArity; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Input format that reads csv into {@link Row}. + * + *

    Different from old csv {@code org.apache.flink.api.java.io.RowCsvInputFormat}: + * 1.New csv will emit this row (Fill null the remaining fields) when row is too short. + * But Old csv will skip this too short row. + * 2.New csv, escape char will be removed. But old csv will keep the escape char. + * + *

    These can be continuously improved in new csv input format: + * 1.New csv not support configure comment char. The comment char is "#". + * 2.New csv not support configure multi chars field delimiter. + * 3.New csv not support read first N, it will throw exception. + * 4.Only support configure line delimiter: "\r" or "\n" or "\r\n". + */ +public class RowCsvInputFormat extends AbstractCsvInputFormat { + + private static final long serialVersionUID = 1L; + + private final TypeInformation[] fieldTypes; + + private final String[] selectedFieldNames; + private final boolean ignoreParseErrors; + + /** Runtime instance that performs the actual work. */ + private transient RuntimeConverter runtimeConverter; + + private transient MappingIterator iterator; + + private transient boolean end; + + private RowCsvInputFormat( + Path[] filePaths, + TypeInformation[] fieldTypes, + CsvSchema csvSchema, + int[] selectedFields, + boolean ignoreParseErrors) { + super(filePaths, csvSchema); + + this.fieldTypes = checkNotNull(fieldTypes); + checkArgument(fieldTypes.length == csvSchema.size()); + this.ignoreParseErrors = ignoreParseErrors; + this.selectedFieldNames = Arrays.stream(checkNotNull(selectedFields)) + .mapToObj(csvSchema::columnName).toArray(String[]::new); + } + + @Override + public void open(FileInputSplit split) throws IOException { + super.open(split); + prepareConverter(); + this.iterator = new CsvMapper() + .readerFor(JsonNode.class) + .with(csvSchema) + .readValues(csvInputStream); + } + + private void prepareConverter() { + RuntimeConverter[] fieldConverters = createFieldRuntimeConverters(ignoreParseErrors, fieldTypes); + + this.runtimeConverter = (node) -> { + final int nodeSize = node.size(); + + validateArity(csvSchema.size(), nodeSize, ignoreParseErrors); + + Row row = new Row(selectedFieldNames.length); + for (int i = 0; i < Math.min(selectedFieldNames.length, nodeSize); i++) { + // Jackson only supports mapping by name in the first level + row.setField(i, fieldConverters[i].convert(node.get(selectedFieldNames[i]))); + } + return row; + }; + } + + @Override + public boolean reachedEnd() { + return end; + } + + @Override + public Row nextRecord(Row record) throws IOException { + Row returnRecord = null; + do { + try { + JsonNode root = iterator.nextValue(); + returnRecord = (Row) runtimeConverter.convert(root); + } catch (NoSuchElementException e) { + end = true; + } catch (Throwable t) { + if (!ignoreParseErrors) { + throw new IOException("Failed to deserialize CSV row.", t); + } + } + } while (returnRecord == null && !reachedEnd()); + + return returnRecord; + } + + /** + * Create a builder. + */ + public static Builder builder(TypeInformation typeInfo, Path... filePaths) { + return new Builder(typeInfo, filePaths); + } + + /** + * A builder for creating a {@link RowCsvInputFormat}. + */ + public static class Builder implements Serializable { + + private final Path[] filePaths; + private final TypeInformation[] fieldTypes; + private CsvSchema csvSchema; + private boolean ignoreParseErrors; + private int[] selectedFields; + + /** + * Creates a row CSV input format for the given {@link TypeInformation} and file paths + * with optional parameters. + */ + private Builder(TypeInformation typeInfo, Path... filePaths) { + checkNotNull(filePaths, "File paths must not be null."); + checkNotNull(typeInfo, "Type information must not be null."); + + if (!(typeInfo instanceof RowTypeInfo)) { + throw new IllegalArgumentException("Row type information expected."); + } + + this.filePaths = filePaths; + this.fieldTypes = ((RowTypeInfo) typeInfo).getFieldTypes(); + this.csvSchema = CsvRowSchemaConverter.convert((RowTypeInfo) typeInfo); + } + + public Builder setFieldDelimiter(char delimiter) { + this.csvSchema = this.csvSchema.rebuild().setColumnSeparator(delimiter).build(); + return this; + } + + public Builder setAllowComments(boolean allowComments) { + this.csvSchema = this.csvSchema.rebuild().setAllowComments(allowComments).build(); + return this; + } + + public Builder setArrayElementDelimiter(String delimiter) { + checkNotNull(delimiter, "Array element delimiter must not be null."); + this.csvSchema = this.csvSchema.rebuild().setArrayElementSeparator(delimiter).build(); + return this; + } + + public Builder setQuoteCharacter(char c) { + this.csvSchema = this.csvSchema.rebuild().setQuoteChar(c).build(); + return this; + } + + public Builder setEscapeCharacter(char c) { + this.csvSchema = this.csvSchema.rebuild().setEscapeChar(c).build(); + return this; + } + + public Builder setNullLiteral(String nullLiteral) { + checkNotNull(nullLiteral, "Null literal must not be null."); + this.csvSchema = this.csvSchema.rebuild().setNullValue(nullLiteral).build(); + return this; + } + + public Builder setIgnoreParseErrors(boolean ignoreParseErrors) { + this.ignoreParseErrors = ignoreParseErrors; + return this; + } + + public Builder setSelectedFields(int[] selectedFields) { + this.selectedFields = selectedFields; + return this; + } + + public RowCsvInputFormat build() { + if (selectedFields == null) { + selectedFields = new int[fieldTypes.length]; + for (int i = 0; i < fieldTypes.length; i++) { + selectedFields[i] = i; + } + } + + return new RowCsvInputFormat( + filePaths, + fieldTypes, + csvSchema, + selectedFields, + ignoreParseErrors); + } + } +} diff --git a/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/RowCsvInputFormatSplitTest.java b/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/RowCsvInputFormatSplitTest.java new file mode 100644 index 0000000000..5a7e5ff7f1 --- /dev/null +++ b/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/RowCsvInputFormatSplitTest.java @@ -0,0 +1,173 @@ +/* + * 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.formats.csv; + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.FileInputSplit; +import org.apache.flink.types.Row; + +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +import static java.util.Arrays.asList; +import static java.util.Collections.singletonList; +import static org.apache.flink.formats.csv.RowCsvInputFormatTest.PATH; +import static org.apache.flink.formats.csv.RowCsvInputFormatTest.createTempFile; +import static org.junit.Assert.assertEquals; + +/** + * Test split logic for {@link RowCsvInputFormat}. + */ +public class RowCsvInputFormatSplitTest { + + @Test + public void readAll() throws Exception { + test("11$\n1,222\n" + "22$2,333\n", 0, -1, '$', asList(Row.of("11\n1", "222"), Row.of("222", "333"))); + } + + @Test + public void readStartOffset() throws Exception { + test("11$\n1,222\n" + "22$2,333\n", 1, -1, '$', singletonList(Row.of("222", "333"))); + } + + @Test + public void readStartOffsetWithSeparator() throws Exception { + test("11$\n1,222\n" + "22$2,333\n", 3, -1, '$', singletonList(Row.of("222", "333"))); + } + + @Test + public void readLengthWithSeparator() throws Exception { + test("11$\n1,222\n" + "22$\n2,333\n", 0, 13, '$', asList(Row.of("11\n1", "222"), Row.of("22\n2", "333"))); + } + + @Test + public void readLengthWithMultiBytesEscapeChar() throws Exception { + test("11好\n1,222\n" + "22好\n2,333\n", 0, 13, '好', asList(Row.of("11\n1", "222"), Row.of("22\n2", "333"))); + } + + @Test + public void readLengthWithMultiBytesEscapeChar2() throws Exception { + test("11好\n1,222\n" + "22好\n2,333\n", 0, 16, '好', asList(Row.of("11\n1", "222"), Row.of("22\n2", "333"))); + } + + @Test + public void readLengthWithMultiBytesEscapeChar3() throws Exception { + test("11好\n1,222\n" + "22好\n2,333\n", 0, 18, '好', asList(Row.of("11\n1", "222"), Row.of("22\n2", "333"))); + } + + @Test + public void readStartOffsetAndLength() throws Exception { + test("11好\n1,222\n" + "22好\n2,333\n", 3, 18, '好', singletonList(Row.of("22\n2", "333"))); + } + + @Test + public void readMultiLineSeparator() throws Exception { + test("111,222\r\n" + "222,333\r\n", 3, 18, '好', singletonList(Row.of("222", "333"))); + } + + @Test + public void readRLineSeparator() throws Exception { + test("111,222\r" + "222,333\r", 3, 18, '好', singletonList(Row.of("222", "333"))); + } + + @Test + public void testQuotationMark() throws Exception { + test("\"111\",222\r" + "222,333\r", 0, 18, '$', asList(Row.of("111", "222"), Row.of("222", "333"))); + test("\"111\",222\r" + "222,333\r", 3, 18, '$', singletonList(Row.of("222", "333"))); + test("\"111\",222\r" + "222,333\r", 5, 18, '$', singletonList(Row.of("222", "333"))); + test("\"111\",222\r" + "222,333\r", 6, 18, '$', singletonList(Row.of("222", "333"))); + + testOneField("\"111\"\r" + "222\r", 0, 18, '$', asList(Row.of("111"), Row.of("222"))); + testOneField("\"111\"\r" + "222\r", 3, 18, '$', singletonList(Row.of("222"))); + testOneField("\"111\"\r" + "222\r", 5, 18, '$', singletonList(Row.of("222"))); + } + + @Test + public void testSurroundEscapedDelimiter() throws Exception { + test("$11$1,222\r" + "222,333\r", 0, 18, '$', asList(Row.of("111", "222"), Row.of("222", "333"))); + test("$11$1,222\r" + "222,333\r", 3, 18, '$', singletonList(Row.of("222", "333"))); + test("$11$1,222\r" + "222,333\r", 5, 18, '$', singletonList(Row.of("222", "333"))); + test("$11$1,222\r" + "222,333\r", 6, 18, '$', singletonList(Row.of("222", "333"))); + + testOneField("123*'4**\r" + "123*'4**\n", 0, 18, '*', asList(Row.of("123'4*"), Row.of("123'4*"))); + testOneField("123*'4**\r" + "123*'4**\n", 3, 18, '*', singletonList(Row.of("123'4*"))); + testOneField("123*'4**\r" + "123*'4**\n", 4, 18, '*', singletonList(Row.of("123'4*"))); + testOneField("123*'4**\r" + "123*'4**\n", 5, 18, '*', singletonList(Row.of("123'4*"))); + + testOneField("'123''4**'\r" + "'123''4**'\n", 0, 18, '*', asList(Row.of("'123''4*'"), Row.of("'123''4*'"))); + testOneField("'123''4**'\r" + "'123''4**'\n", 3, 18, '*', singletonList(Row.of("'123''4*'"))); + testOneField("'123''4**'\r" + "'123''4**'\n", 4, 18, '*', singletonList(Row.of("'123''4*'"))); + testOneField("'123''4**'\r" + "'123''4**'\n", 5, 18, '*', singletonList(Row.of("'123''4*'"))); + testOneField("'123''4**'\r" + "'123''4**'\n", 6, 18, '*', singletonList(Row.of("'123''4*'"))); + } + + private void test(String content, long offset, long length, char escapeChar, List expected) throws Exception { + test( + content, + offset, + length, + escapeChar, + expected, + new TypeInformation[]{BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO}); + } + + private void testOneField(String content, long offset, long length, char escapeChar, List expected) throws Exception { + test( + content, + offset, + length, + escapeChar, + expected, + new TypeInformation[]{BasicTypeInfo.STRING_TYPE_INFO}); + } + + private void test( + String content, + long offset, long length, + char escapeChar, + List expected, + TypeInformation[] fieldTypes) throws Exception { + FileInputSplit split = createTempFile(content, offset, length); + + RowCsvInputFormat.Builder builder = RowCsvInputFormat.builder(new RowTypeInfo(fieldTypes), PATH) + .setEscapeCharacter(escapeChar); + + RowCsvInputFormat format = builder.build(); + format.configure(new Configuration()); + format.open(split); + + List rows = new ArrayList<>(); + while (!format.reachedEnd()) { + Row result = new Row(3); + result = format.nextRecord(result); + if (result == null) { + break; + } else { + rows.add(result); + } + } + + assertEquals(expected, rows); + } +} diff --git a/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/RowCsvInputFormatTest.java b/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/RowCsvInputFormatTest.java new file mode 100644 index 0000000000..509dd2e587 --- /dev/null +++ b/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/RowCsvInputFormatTest.java @@ -0,0 +1,756 @@ +/* + * 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.formats.csv; + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.FileInputSplit; +import org.apache.flink.core.fs.Path; +import org.apache.flink.types.Row; + +import org.junit.Test; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStreamWriter; +import java.nio.charset.StandardCharsets; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; + +import static junit.framework.TestCase.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** + * Tests for {@link RowCsvInputFormat}. + */ +public class RowCsvInputFormatTest { + + static final Path PATH = new Path("an/ignored/file/"); + + // static variables for testing the removal of \r\n to \n + private static final String FIRST_PART = "That is the first part"; + private static final String SECOND_PART = "That is the second part"; + + @Test + public void ignoreInvalidLines() throws Exception { + String fileContent = + "#description of the data\n" + + "header1|header2|header3|\n" + + "this is|1|2.0|\n" + + "//a comment\n" + + "a test|3|4.0|\n" + + "#next|5|6.0|\n"; + + FileInputSplit split = createTempFile(fileContent); + + TypeInformation[] fieldTypes = new TypeInformation[]{BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.DOUBLE_TYPE_INFO}; + RowCsvInputFormat.Builder builder = RowCsvInputFormat.builder(new RowTypeInfo(fieldTypes), PATH) + .setFieldDelimiter('|') + .setIgnoreParseErrors(false); + + RowCsvInputFormat format = builder.build(); + Configuration parameters = new Configuration(); + format.configure(parameters); + format.open(split); + + Row result = new Row(3); + try { + result = format.nextRecord(result); + fail("RuntimeException was not thrown! (Row length mismatch. 3 fields expected but was 1)"); + } catch (IOException ignored) { + } // => ok + + try { + result = format.nextRecord(result); + fail("NumberFormatException was not thrown! (For input string: \"header2\")"); + } catch (IOException ignored) { + } // => ok + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("this is", result.getField(0)); + assertEquals(1, result.getField(1)); + assertEquals(2.0, result.getField(2)); + + try { + result = format.nextRecord(result); + fail("RuntimeException was not thrown! (Row length mismatch. 3 fields expected but was 1)"); + } catch (IOException ignored) { + } // => ok + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("a test", result.getField(0)); + assertEquals(3, result.getField(1)); + assertEquals(4.0, result.getField(2)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("#next", result.getField(0)); + assertEquals(5, result.getField(1)); + assertEquals(6.0, result.getField(2)); + + result = format.nextRecord(result); + assertNull(result); + + // re-open with lenient = true + builder.setIgnoreParseErrors(true); + format = builder.build(); + format.configure(parameters); + format.open(split); + + result = new Row(3); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("#description of the data", result.getField(0)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("header1", result.getField(0)); + assertNull(result.getField(1)); + assertNull(result.getField(2)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("this is", result.getField(0)); + assertEquals(1, result.getField(1)); + assertEquals(2.0, result.getField(2)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("//a comment", result.getField(0)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("a test", result.getField(0)); + assertEquals(3, result.getField(1)); + assertEquals(4.0, result.getField(2)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("#next", result.getField(0)); + assertEquals(5, result.getField(1)); + assertEquals(6.0, result.getField(2)); + result = format.nextRecord(result); + assertNull(result); + } + + @Test + public void ignorePrefixComments() throws Exception { + String fileContent = + "#description of the data\n" + + "#successive commented line\n" + + "this is|1|2.0|\n" + + "a test|3|4.0|\n" + + "#next|5|6.0|\n"; + + FileInputSplit split = createTempFile(fileContent); + + TypeInformation[] fieldTypes = new TypeInformation[]{ + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.DOUBLE_TYPE_INFO}; + + RowCsvInputFormat.Builder builder = RowCsvInputFormat.builder(new RowTypeInfo(fieldTypes), PATH) + .setFieldDelimiter('|') + .setAllowComments(true); + + RowCsvInputFormat format = builder.build(); + format.configure(new Configuration()); + format.open(split); + + Row result = new Row(3); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("this is", result.getField(0)); + assertEquals(1, result.getField(1)); + assertEquals(2.0, result.getField(2)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("a test", result.getField(0)); + assertEquals(3, result.getField(1)); + assertEquals(4.0, result.getField(2)); + + result = format.nextRecord(result); + assertNull(result); + } + + @Test + public void readStringFields() throws Exception { + String fileContent = "abc|def|ghijk\nabc||hhg\n|||\n||"; + + FileInputSplit split = createTempFile(fileContent); + + TypeInformation[] fieldTypes = new TypeInformation[]{ + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO}; + + RowCsvInputFormat.Builder builder = RowCsvInputFormat.builder(new RowTypeInfo(fieldTypes), PATH) + .setFieldDelimiter('|'); + + RowCsvInputFormat format = builder.build(); + format.configure(new Configuration()); + format.open(split); + + Row result = new Row(3); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("abc", result.getField(0)); + assertEquals("def", result.getField(1)); + assertEquals("ghijk", result.getField(2)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("abc", result.getField(0)); + assertEquals("", result.getField(1)); + assertEquals("hhg", result.getField(2)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("", result.getField(0)); + assertEquals("", result.getField(1)); + assertEquals("", result.getField(2)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("", result.getField(0)); + assertEquals("", result.getField(1)); + assertEquals("", result.getField(2)); + + result = format.nextRecord(result); + assertNull(result); + assertTrue(format.reachedEnd()); + } + + @Test + public void readMixedQuotedStringFields() throws Exception { + String fileContent = "@a|b|c@|def|@ghijk@\nabc||@|hhg@\n|||\n"; + + FileInputSplit split = createTempFile(fileContent); + + TypeInformation[] fieldTypes = new TypeInformation[]{ + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO}; + + RowCsvInputFormat.Builder builder = RowCsvInputFormat.builder(new RowTypeInfo(fieldTypes), PATH) + .setFieldDelimiter('|') + .setQuoteCharacter('@'); + + RowCsvInputFormat format = builder.build(); + format.configure(new Configuration()); + format.open(split); + + Row result = new Row(3); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("a|b|c", result.getField(0)); + assertEquals("def", result.getField(1)); + assertEquals("ghijk", result.getField(2)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("abc", result.getField(0)); + assertEquals("", result.getField(1)); + assertEquals("|hhg", result.getField(2)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("", result.getField(0)); + assertEquals("", result.getField(1)); + assertEquals("", result.getField(2)); + + result = format.nextRecord(result); + assertNull(result); + assertTrue(format.reachedEnd()); + } + + @Test + public void testTailingEmptyFields() throws Exception { + String fileContent = "abc|def|ghijk\n" + + "abc|def|\n" + + "abc||\n" + + "|||\n" + + "||\n" + + "abc|def\n"; + + FileInputSplit split = createTempFile(fileContent); + + TypeInformation[] fieldTypes = new TypeInformation[]{ + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO}; + + RowCsvInputFormat.Builder builder = RowCsvInputFormat.builder(new RowTypeInfo(fieldTypes), PATH) + .setFieldDelimiter('|'); + + RowCsvInputFormat format = builder.build(); + format.configure(new Configuration()); + format.open(split); + + Row result = new Row(3); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("abc", result.getField(0)); + assertEquals("def", result.getField(1)); + assertEquals("ghijk", result.getField(2)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("abc", result.getField(0)); + assertEquals("def", result.getField(1)); + assertEquals("", result.getField(2)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("abc", result.getField(0)); + assertEquals("", result.getField(1)); + assertEquals("", result.getField(2)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("", result.getField(0)); + assertEquals("", result.getField(1)); + assertEquals("", result.getField(2)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("", result.getField(0)); + assertEquals("", result.getField(1)); + assertEquals("", result.getField(2)); + + try { + format.nextRecord(result); + fail("RuntimeException: Row length mismatch. 3 fields expected but was 2"); + } catch (IOException e) {} + } + + @Test + public void testIntegerFields() throws Exception { + String fileContent = "111|222|333|444|555\n666|777|888|999|000|\n"; + + FileInputSplit split = createTempFile(fileContent); + + TypeInformation[] fieldTypes = new TypeInformation[]{ + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO}; + + RowCsvInputFormat.Builder builder = RowCsvInputFormat.builder(new RowTypeInfo(fieldTypes), PATH) + .setFieldDelimiter('|'); + + RowCsvInputFormat format = builder.build(); + format.configure(new Configuration()); + format.open(split); + + Row result = new Row(5); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals(111, result.getField(0)); + assertEquals(222, result.getField(1)); + assertEquals(333, result.getField(2)); + assertEquals(444, result.getField(3)); + assertEquals(555, result.getField(4)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals(666, result.getField(0)); + assertEquals(777, result.getField(1)); + assertEquals(888, result.getField(2)); + assertEquals(999, result.getField(3)); + assertEquals(0, result.getField(4)); + + result = format.nextRecord(result); + assertNull(result); + assertTrue(format.reachedEnd()); + } + + @Test + public void testEmptyFields() throws Exception { + String fileContent = + ",,,,,,,,\n" + + ",,,,,,,\n" + + ",,,,,,,,\n" + + ",,,,,,,\n" + + ",,,,,,,,\n" + + ",,,,,,,,\n" + + ",,,,,,,\n" + + ",,,,,,,,\n"; + + FileInputSplit split = createTempFile(fileContent); + + TypeInformation[] fieldTypes = new TypeInformation[]{ + BasicTypeInfo.BOOLEAN_TYPE_INFO, + BasicTypeInfo.BYTE_TYPE_INFO, + BasicTypeInfo.DOUBLE_TYPE_INFO, + BasicTypeInfo.FLOAT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.LONG_TYPE_INFO, + BasicTypeInfo.SHORT_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO}; + + RowCsvInputFormat.Builder builder = RowCsvInputFormat.builder(new RowTypeInfo(fieldTypes), PATH) + .setFieldDelimiter(',') + .setNullLiteral(""); + + RowCsvInputFormat format = builder.build(); + format.configure(new Configuration()); + format.open(split); + + Row result = new Row(8); + int linesCnt = fileContent.split("\n").length; + + for (int i = 0; i < linesCnt; i++) { + result = format.nextRecord(result); + assertNull(result.getField(i)); + } + + // ensure no more rows + assertNull(format.nextRecord(result)); + assertTrue(format.reachedEnd()); + } + + @Test + public void testDoubleFields() throws Exception { + String fileContent = "11.1|22.2|33.3|44.4|55.5\n66.6|77.7|88.8|99.9|00.0|\n"; + + FileInputSplit split = createTempFile(fileContent); + + TypeInformation[] fieldTypes = new TypeInformation[]{ + BasicTypeInfo.DOUBLE_TYPE_INFO, + BasicTypeInfo.DOUBLE_TYPE_INFO, + BasicTypeInfo.DOUBLE_TYPE_INFO, + BasicTypeInfo.DOUBLE_TYPE_INFO, + BasicTypeInfo.DOUBLE_TYPE_INFO}; + + RowCsvInputFormat.Builder builder = RowCsvInputFormat.builder(new RowTypeInfo(fieldTypes), PATH) + .setFieldDelimiter('|'); + + RowCsvInputFormat format = builder.build(); + format.configure(new Configuration()); + format.open(split); + + Row result = new Row(5); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals(11.1, result.getField(0)); + assertEquals(22.2, result.getField(1)); + assertEquals(33.3, result.getField(2)); + assertEquals(44.4, result.getField(3)); + assertEquals(55.5, result.getField(4)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals(66.6, result.getField(0)); + assertEquals(77.7, result.getField(1)); + assertEquals(88.8, result.getField(2)); + assertEquals(99.9, result.getField(3)); + assertEquals(0.0, result.getField(4)); + + result = format.nextRecord(result); + assertNull(result); + assertTrue(format.reachedEnd()); + } + + @Test + public void testReadSparseWithPositionSetter() throws Exception { + String fileContent = "111|222|333|444|555|666|777|888|999|000|\n" + + "000|999|888|777|666|555|444|333|222|111|"; + + FileInputSplit split = createTempFile(fileContent); + + TypeInformation[] fieldTypes = new TypeInformation[]{ + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO}; + + RowCsvInputFormat.Builder builder = RowCsvInputFormat.builder(new RowTypeInfo(fieldTypes), PATH) + .setFieldDelimiter('|') + .setSelectedFields(new int[]{0, 3, 7}); + + RowCsvInputFormat format = builder.build(); + format.configure(new Configuration()); + format.open(split); + + Row result = new Row(3); + result = format.nextRecord(result); + + assertNotNull(result); + assertEquals(111, result.getField(0)); + assertEquals(444, result.getField(1)); + assertEquals(888, result.getField(2)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals(0, result.getField(0)); + assertEquals(777, result.getField(1)); + assertEquals(333, result.getField(2)); + + result = format.nextRecord(result); + assertNull(result); + assertTrue(format.reachedEnd()); + } + + @Test + public void testWindowsLineEndRemoval() throws Exception { + + // check typical use case -- linux file is correct and it is set up to linux(\n) + testRemovingTrailingCR("\n"); + + // check typical windows case -- windows file endings and file has windows file endings set up + testRemovingTrailingCR("\r\n"); + + // check problematic case windows file -- windows file endings(\r\n) + // but linux line endings (\n) set up + testRemovingTrailingCR("\r\n"); + + // check problematic case linux file -- linux file endings (\n) + // but windows file endings set up (\r\n) + // specific setup for windows line endings will expect \r\n because + // it has to be set up and is not standard. + } + + @Test + public void testQuotedStringParsingWithIncludeFields() throws Exception { + String fileContent = "\"20:41:52-1-3-2015\"|\"Re: Taskmanager memory error in Eclipse\"|" + + "\"Blahblah \"|\"blaaa\"|\"blubb\""; + File tempFile = File.createTempFile("CsvReaderQuotedString", "tmp"); + tempFile.deleteOnExit(); + tempFile.setWritable(true); + + OutputStreamWriter writer = new OutputStreamWriter(new FileOutputStream(tempFile)); + writer.write(fileContent); + writer.close(); + + TypeInformation[] fieldTypes = new TypeInformation[]{ + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO}; + + RowCsvInputFormat.Builder builder = RowCsvInputFormat + .builder(new RowTypeInfo(fieldTypes), new Path(tempFile.toURI().toString())) + .setFieldDelimiter('|') + .setSelectedFields(new int[]{0, 2}) + .setQuoteCharacter('"'); + + RowCsvInputFormat inputFormat = builder.build(); + inputFormat.configure(new Configuration()); + + FileInputSplit[] splits = inputFormat.createInputSplits(1); + inputFormat.open(splits[0]); + + Row record = inputFormat.nextRecord(new Row(2)); + assertEquals("20:41:52-1-3-2015", record.getField(0)); + assertEquals("Blahblah ", record.getField(1)); + } + + @Test + public void testQuotedStringParsingWithEscapedQuotes() throws Exception { + String fileContent = "\"\\\"Hello\\\" World\"|\"We are\\\" young\""; + File tempFile = File.createTempFile("CsvReaderQuotedString", "tmp"); + tempFile.deleteOnExit(); + tempFile.setWritable(true); + + OutputStreamWriter writer = new OutputStreamWriter(new FileOutputStream(tempFile)); + writer.write(fileContent); + writer.close(); + + TypeInformation[] fieldTypes = new TypeInformation[]{ + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO}; + + RowCsvInputFormat.Builder builder = RowCsvInputFormat + .builder(new RowTypeInfo(fieldTypes), new Path(tempFile.toURI().toString())) + .setFieldDelimiter('|') + .setQuoteCharacter('"') + .setEscapeCharacter('\\'); + + RowCsvInputFormat inputFormat = builder.build(); + inputFormat.configure(new Configuration()); + + FileInputSplit[] splits = inputFormat.createInputSplits(1); + inputFormat.open(splits[0]); + + Row record = inputFormat.nextRecord(new Row(2)); + assertEquals("\"Hello\" World", record.getField(0)); + assertEquals("We are\" young", record.getField(1)); + } + + @Test + public void testSqlTimeFields() throws Exception { + String fileContent = "1990-10-14|02:42:25|1990-10-14 02:42:25.123|1990-1-4 2:2:5\n" + + "1990-10-14|02:42:25|1990-10-14 02:42:25.123|1990-1-4 2:2:5.3\n"; + + FileInputSplit split = createTempFile(fileContent); + + TypeInformation[] fieldTypes = new TypeInformation[]{ + SqlTimeTypeInfo.DATE, + SqlTimeTypeInfo.TIME, + SqlTimeTypeInfo.TIMESTAMP, + SqlTimeTypeInfo.TIMESTAMP}; + + RowCsvInputFormat.Builder builder = RowCsvInputFormat.builder(new RowTypeInfo(fieldTypes), PATH) + .setFieldDelimiter('|'); + + RowCsvInputFormat format = builder.build(); + format.configure(new Configuration()); + format.open(split); + + Row result = new Row(4); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals(Date.valueOf("1990-10-14"), result.getField(0)); + assertEquals(Time.valueOf("02:42:25"), result.getField(1)); + assertEquals(Timestamp.valueOf("1990-10-14 02:42:25.123"), result.getField(2)); + assertEquals(Timestamp.valueOf("1990-01-04 02:02:05"), result.getField(3)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals(Date.valueOf("1990-10-14"), result.getField(0)); + assertEquals(Time.valueOf("02:42:25"), result.getField(1)); + assertEquals(Timestamp.valueOf("1990-10-14 02:42:25.123"), result.getField(2)); + assertEquals(Timestamp.valueOf("1990-01-04 02:02:05.3"), result.getField(3)); + + result = format.nextRecord(result); + assertNull(result); + assertTrue(format.reachedEnd()); + } + + @Test + public void testScanOrder() throws Exception { + String fileContent = + // first row + "111|222|333|444|555|666|777|888|999|000|\n" + + // second row + "000|999|888|777|666|555|444|333|222|111|"; + FileInputSplit split = createTempFile(fileContent); + + TypeInformation[] fieldTypes = new TypeInformation[]{ + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO}; + + RowCsvInputFormat.Builder builder = RowCsvInputFormat.builder(new RowTypeInfo(fieldTypes), PATH) + .setFieldDelimiter('|') + .setSelectedFields(new int[]{7, 3, 0}); + + RowCsvInputFormat format = builder.build(); + format.configure(new Configuration()); + format.open(split); + + Row result = new Row(3); + + // check first row + result = format.nextRecord(result); + assertNotNull(result); + assertEquals(888, result.getField(0)); + assertEquals(444, result.getField(1)); + assertEquals(111, result.getField(2)); + + // check second row + result = format.nextRecord(result); + assertNotNull(result); + assertEquals(333, result.getField(0)); + assertEquals(777, result.getField(1)); + assertEquals(0, result.getField(2)); + } + + private static FileInputSplit createTempFile(String content) throws IOException { + return createTempFile(content, 0, null); + } + + static FileInputSplit createTempFile(String content, long start, Long length) throws IOException { + File tempFile = File.createTempFile("test_contents", "tmp"); + tempFile.deleteOnExit(); + OutputStreamWriter wrt = new OutputStreamWriter(new FileOutputStream(tempFile), StandardCharsets.UTF_8); + wrt.write(content); + wrt.close(); + return new FileInputSplit(0, new Path(tempFile.toURI().toString()), start, + length == null ? tempFile.length() : length, new String[]{"localhost"}); + } + + private static void testRemovingTrailingCR(String lineBreakerInFile) throws IOException { + String fileContent = FIRST_PART + lineBreakerInFile + SECOND_PART + lineBreakerInFile; + + // create input file + File tempFile = File.createTempFile("CsvInputFormatTest", "tmp"); + tempFile.deleteOnExit(); + tempFile.setWritable(true); + + OutputStreamWriter wrt = new OutputStreamWriter(new FileOutputStream(tempFile)); + wrt.write(fileContent); + wrt.close(); + + TypeInformation[] fieldTypes = new TypeInformation[]{BasicTypeInfo.STRING_TYPE_INFO}; + + RowCsvInputFormat.Builder builder = RowCsvInputFormat + .builder(new RowTypeInfo(fieldTypes), new Path(tempFile.toURI().toString())); + + RowCsvInputFormat inputFormat = builder.build(); + + inputFormat.configure(new Configuration()); + + FileInputSplit[] splits = inputFormat.createInputSplits(1); + inputFormat.open(splits[0]); + + Row result = inputFormat.nextRecord(new Row(1)); + assertNotNull("Expecting to not return null", result); + assertEquals(FIRST_PART, result.getField(0)); + + result = inputFormat.nextRecord(result); + assertNotNull("Expecting to not return null", result); + assertEquals(SECOND_PART, result.getField(0)); + } +} -- Gitee From 1a751a3837b9bfb2907b087f04ae965550c33972 Mon Sep 17 00:00:00 2001 From: wangyang0918 Date: Wed, 25 Mar 2020 15:18:59 +0800 Subject: [PATCH 462/885] [hotfix][k8s] Remove the outdated comments and unused config option --- .../generated/kubernetes_config_configuration.html | 6 ------ .../configuration/KubernetesConfigOptions.java | 11 ----------- 2 files changed, 17 deletions(-) diff --git a/docs/_includes/generated/kubernetes_config_configuration.html b/docs/_includes/generated/kubernetes_config_configuration.html index dbea103218..b5e6f80d54 100644 --- a/docs/_includes/generated/kubernetes_config_configuration.html +++ b/docs/_includes/generated/kubernetes_config_configuration.html @@ -110,12 +110,6 @@

    - - - - - - diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java index 37ab3943bb..f3319c0336 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java @@ -101,17 +101,6 @@ public class KubernetesConfigOptions { .defaultValue("%java% %classpath% %jvmmem% %jvmopts% %logging% %class% %args% %redirects%") .withDescription("Template for the kubernetes jobmanager and taskmanager container start invocation."); - public static final ConfigOption SERVICE_CREATE_TIMEOUT = - key("kubernetes.service.create-timeout") - .stringType() - .defaultValue("1 min") - .withDescription("Timeout used for creating the service. The timeout value requires a time-unit " + - "specifier (ms/s/min/h/d)."); - - // --------------------------------------------------------------------------------- - // The following config options could be overridden by KubernetesCliOptions. - // --------------------------------------------------------------------------------- - public static final ConfigOption> JOB_MANAGER_LABELS = key("kubernetes.jobmanager.labels") .mapType() -- Gitee From 5b54a2580172a7f3c518211c1bfc5f987ebbadba Mon Sep 17 00:00:00 2001 From: wangyang0918 Date: Wed, 25 Mar 2020 15:25:07 +0800 Subject: [PATCH 463/885] [FLINK-16749][k8s] Support to set node selector for jobmanager and taskmanager pod The node-selector is a collection of key/value pairs to constrain a pod to only be able to run on particular node(s). Since affinity and anti-affinity are uncommon use case for Flink, so we leave the support in pod template. This closes #11500 . --- .../generated/kubernetes_config_configuration.html | 12 ++++++++++++ .../configuration/KubernetesConfigOptions.java | 14 ++++++++++++++ .../decorators/InitJobManagerDecorator.java | 1 + .../decorators/InitTaskManagerDecorator.java | 1 + .../parameters/KubernetesJobManagerParameters.java | 6 ++++++ .../parameters/KubernetesParameters.java | 5 +++++ .../KubernetesTaskManagerParameters.java | 6 ++++++ .../kubeclient/KubernetesJobManagerTestBase.java | 8 ++++++++ .../kubeclient/KubernetesTaskManagerTestBase.java | 8 ++++++++ .../decorators/InitJobManagerDecoratorTest.java | 5 +++++ .../decorators/InitTaskManagerDecoratorTest.java | 5 +++++ 11 files changed, 71 insertions(+) diff --git a/docs/_includes/generated/kubernetes_config_configuration.html b/docs/_includes/generated/kubernetes_config_configuration.html index b5e6f80d54..6ef559f6dc 100644 --- a/docs/_includes/generated/kubernetes_config_configuration.html +++ b/docs/_includes/generated/kubernetes_config_configuration.html @@ -92,6 +92,12 @@ + + + + + + @@ -128,5 +134,11 @@ + + + + + +
    HDFS BucketingSinkexactly onceImplementation depends on Hadoop version精确一次实现方法取决于 Hadoop 的版本
    Elasticsearchat least once至少一次
    Kafka producerat least once / exactly onceexactly once with transactional producers (v 0.11+)至少一次 / 精确一次当使用事务生产者时,保证精确一次 (v 0.11+)
    Cassandra sinkat least once / exactly onceexactly once only for idempotent updates至少一次 / 精确一次只有当更新是幂等时,保证精确一次
    AWS Kinesis Streamsat least once至少一次
    File sinksexactly once精确一次
    Socket sinksat least once至少一次
    Standard outputat least once至少一次
    Redis sinkat least once至少一次
    + +
    Log NameSize (KB)
    + {{ log.name }} + {{ (log.size / 1024) | number : '1.0-2' }}
    web.timeout
    10000600000 Long Timeout for asynchronous operations by the web monitor in milliseconds.
    state.backend.rocksdb.timer-service.factory
    "ROCKSDB"StringROCKSDB

    Enum

    Possible values: [HEAP, ROCKSDB]
    This determines the factory for timer service state implementation. Options are either HEAP (heap-based, default) or ROCKSDB for an implementation based on RocksDB .
    state.backend.rocksdb.timer-service.factory
    "ROCKSDB"StringROCKSDB

    Enum

    Possible values: [HEAP, ROCKSDB]
    This determines the factory for timer service state implementation. Options are either HEAP (heap-based, default) or ROCKSDB for an implementation based on RocksDB .

    Enum

    Possible values: [ClusterIP, NodePort, LoadBalancer]
    The type of the rest service (ClusterIP or NodePort or LoadBalancer). When set to ClusterIP, the rest service will not be created.
    kubernetes.service.create-timeout
    "1 min"StringTimeout used for creating the service. The timeout value requires a time-unit specifier (ms/s/min/h/d).
    kubernetes.taskmanager.annotations
    (none)Map The labels to be set for JobManager pod. Specified as key:value pairs separated by commas. For example, version:alphav1,deploy:test.
    kubernetes.jobmanager.node-selector
    (none)MapThe node selector to be set for JobManager pod. Specified as key:value pairs separated by commas. For example, environment:production,disk:ssd.
    kubernetes.jobmanager.service-account
    "default"Map The labels to be set for TaskManager pods. Specified as key:value pairs separated by commas. For example, version:alphav1,deploy:test.
    kubernetes.taskmanager.node-selector
    (none)MapThe node selector to be set for TaskManager pods. Specified as key:value pairs separated by commas. For example, environment:production,disk:ssd.
    diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java index f3319c0336..39b2cad4e3 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/configuration/KubernetesConfigOptions.java @@ -115,6 +115,20 @@ public class KubernetesConfigOptions { .withDescription("The labels to be set for TaskManager pods. Specified as key:value pairs separated by commas. " + "For example, version:alphav1,deploy:test."); + public static final ConfigOption> JOB_MANAGER_NODE_SELECTOR = + key("kubernetes.jobmanager.node-selector") + .mapType() + .noDefaultValue() + .withDescription("The node selector to be set for JobManager pod. Specified as key:value pairs separated by " + + "commas. For example, environment:production,disk:ssd."); + + public static final ConfigOption> TASK_MANAGER_NODE_SELECTOR = + key("kubernetes.taskmanager.node-selector") + .mapType() + .noDefaultValue() + .withDescription("The node selector to be set for TaskManager pods. Specified as key:value pairs separated by " + + "commas. For example, environment:production,disk:ssd."); + public static final ConfigOption CLUSTER_ID = key("kubernetes.cluster-id") .stringType() diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecorator.java index eac4283ea6..45fa10d2d7 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecorator.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecorator.java @@ -65,6 +65,7 @@ public class InitJobManagerDecorator extends AbstractKubernetesStepDecorator { .editOrNewSpec() .withServiceAccountName(kubernetesJobManagerParameters.getServiceAccount()) .withImagePullSecrets(kubernetesJobManagerParameters.getImagePullSecrets()) + .withNodeSelector(kubernetesJobManagerParameters.getNodeSelector()) .endSpec() .build(); diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecorator.java index 8817339270..e789b60dcf 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecorator.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecorator.java @@ -59,6 +59,7 @@ public class InitTaskManagerDecorator extends AbstractKubernetesStepDecorator { .endMetadata() .editOrNewSpec() .withImagePullSecrets(kubernetesTaskManagerParameters.getImagePullSecrets()) + .withNodeSelector(kubernetesTaskManagerParameters.getNodeSelector()) .endSpec() .build(); diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java index cff893bce7..d077c34686 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java @@ -62,6 +62,12 @@ public class KubernetesJobManagerParameters extends AbstractKubernetesParameters return Collections.unmodifiableMap(labels); } + @Override + public Map getNodeSelector() { + return Collections.unmodifiableMap( + flinkConfig.getOptional(KubernetesConfigOptions.JOB_MANAGER_NODE_SELECTOR).orElse(Collections.emptyMap())); + } + @Override public Map getEnvironments() { return ConfigurationUtils.getPrefixedKeyValuePairs(ResourceManagerOptions.CONTAINERIZED_MASTER_ENV_PREFIX, flinkConfig); diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesParameters.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesParameters.java index 45069fbce0..2f66101090 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesParameters.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesParameters.java @@ -52,6 +52,11 @@ public interface KubernetesParameters { */ Map getLabels(); + /** + * A collection of node selector to constrain a pod to only be able to run on particular node(s). + */ + Map getNodeSelector(); + /** * A collection of customized environments that are attached to the JobManager and TaskManager Container(s). */ diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParameters.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParameters.java index 28561ab88f..6929d3a809 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParameters.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesTaskManagerParameters.java @@ -68,6 +68,12 @@ public class KubernetesTaskManagerParameters extends AbstractKubernetesParameter return Collections.unmodifiableMap(labels); } + @Override + public Map getNodeSelector() { + return Collections.unmodifiableMap( + flinkConfig.getOptional(KubernetesConfigOptions.TASK_MANAGER_NODE_SELECTOR).orElse(Collections.emptyMap())); + } + @Override public Map getEnvironments() { return this.containeredTaskManagerParameters.taskManagerEnv(); diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/KubernetesJobManagerTestBase.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/KubernetesJobManagerTestBase.java index 719784b9ca..c063a6b314 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/KubernetesJobManagerTestBase.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/KubernetesJobManagerTestBase.java @@ -58,6 +58,13 @@ public class KubernetesJobManagerTestBase extends KubernetesTestBase { } }; + protected final Map nodeSelector = new HashMap() { + { + put("env", "production"); + put("disk", "ssd"); + } + }; + protected KubernetesJobManagerParameters kubernetesJobManagerParameters; protected FlinkPod baseFlinkPod; @@ -73,6 +80,7 @@ public class KubernetesJobManagerTestBase extends KubernetesTestBase { this.customizedEnvs.forEach((k, v) -> this.flinkConfig.setString(ResourceManagerOptions.CONTAINERIZED_MASTER_ENV_PREFIX + k, v)); this.flinkConfig.set(KubernetesConfigOptions.JOB_MANAGER_LABELS, userLabels); + this.flinkConfig.set(KubernetesConfigOptions.JOB_MANAGER_NODE_SELECTOR, nodeSelector); final ClusterSpecification clusterSpecification = new ClusterSpecification.ClusterSpecificationBuilder() .setMasterMemoryMB(JOB_MANAGER_MEMORY) diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/KubernetesTaskManagerTestBase.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/KubernetesTaskManagerTestBase.java index a4e3cac7ba..329db2e9b8 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/KubernetesTaskManagerTestBase.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/KubernetesTaskManagerTestBase.java @@ -60,6 +60,13 @@ public class KubernetesTaskManagerTestBase extends KubernetesTestBase { } }; + protected final Map nodeSelector = new HashMap() { + { + put("env", "production"); + put("disk", "ssd"); + } + }; + protected TaskExecutorProcessSpec taskExecutorProcessSpec; protected ContaineredTaskManagerParameters containeredTaskManagerParameters; @@ -78,6 +85,7 @@ public class KubernetesTaskManagerTestBase extends KubernetesTestBase { customizedEnvs.forEach((k, v) -> flinkConfig.setString(ResourceManagerOptions.CONTAINERIZED_TASK_MANAGER_ENV_PREFIX + k, v)); this.flinkConfig.set(KubernetesConfigOptions.TASK_MANAGER_LABELS, userLabels); + this.flinkConfig.set(KubernetesConfigOptions.TASK_MANAGER_NODE_SELECTOR, nodeSelector); taskExecutorProcessSpec = TaskExecutorProcessUtils.processSpecFromConfig(flinkConfig); containeredTaskManagerParameters = ContaineredTaskManagerParameters.create(flinkConfig, taskExecutorProcessSpec, diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecoratorTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecoratorTest.java index 9713cd62bf..78d1976f2d 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecoratorTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitJobManagerDecoratorTest.java @@ -169,4 +169,9 @@ public class InitJobManagerDecoratorTest extends KubernetesJobManagerTestBase { assertEquals(IMAGE_PULL_SECRETS, resultSecrets); } + + @Test + public void testNodeSelector() { + assertThat(this.resultPod.getSpec().getNodeSelector(), is(equalTo(nodeSelector))); + } } diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecoratorTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecoratorTest.java index 5489611e07..287548c2f3 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecoratorTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InitTaskManagerDecoratorTest.java @@ -163,4 +163,9 @@ public class InitTaskManagerDecoratorTest extends KubernetesTaskManagerTestBase assertEquals(IMAGE_PULL_SECRETS, resultSecrets); } + + @Test + public void testNodeSelector() { + assertThat(this.resultPod.getSpec().getNodeSelector(), is(equalTo(nodeSelector))); + } } -- Gitee From e9603f5ff516e0e9afeafa1287aff332360eb2f1 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Fri, 3 Apr 2020 13:55:47 +0800 Subject: [PATCH 464/885] [FLINK-16914][table-runtime-blink] Introduce array column vector in blink --- .../flink/table/dataformat/ColumnarArray.java | 297 ++++++++++++++++++ .../flink/table/dataformat/ColumnarRow.java | 3 +- .../dataformat/DataFormatConverters.java | 8 +- .../dataformat/vector/ArrayColumnVector.java | 28 ++ .../vector/VectorizedColumnBatch.java | 5 + .../typeutils/BaseArraySerializer.java | 58 +++- .../vector/VectorizedColumnBatchTest.java | 26 +- .../typeutils/BaseArraySerializerTest.java | 12 + 8 files changed, 427 insertions(+), 10 deletions(-) create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/ColumnarArray.java create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/ArrayColumnVector.java diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/ColumnarArray.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/ColumnarArray.java new file mode 100644 index 0000000000..9dbd5e6e3a --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/ColumnarArray.java @@ -0,0 +1,297 @@ +/* + * 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.table.dataformat; + +import org.apache.flink.table.dataformat.vector.ArrayColumnVector; +import org.apache.flink.table.dataformat.vector.BooleanColumnVector; +import org.apache.flink.table.dataformat.vector.ByteColumnVector; +import org.apache.flink.table.dataformat.vector.BytesColumnVector; +import org.apache.flink.table.dataformat.vector.ColumnVector; +import org.apache.flink.table.dataformat.vector.DecimalColumnVector; +import org.apache.flink.table.dataformat.vector.DoubleColumnVector; +import org.apache.flink.table.dataformat.vector.FloatColumnVector; +import org.apache.flink.table.dataformat.vector.IntColumnVector; +import org.apache.flink.table.dataformat.vector.LongColumnVector; +import org.apache.flink.table.dataformat.vector.ShortColumnVector; +import org.apache.flink.table.dataformat.vector.TimestampColumnVector; + +import java.util.Arrays; + +/** + * Columnar array to support access to vector column data. + */ +public final class ColumnarArray implements BaseArray { + + private final ColumnVector data; + private final int offset; + private final int numElements; + + public ColumnarArray(ColumnVector data, int offset, int numElements) { + this.data = data; + this.offset = offset; + this.numElements = numElements; + } + + @Override + public int numElements() { + return numElements; + } + + @Override + public boolean isNullAt(int pos) { + return data.isNullAt(offset + pos); + } + + @Override + public void setNullAt(int pos) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public boolean getBoolean(int ordinal) { + return ((BooleanColumnVector) data).getBoolean(offset + ordinal); + } + + @Override + public byte getByte(int ordinal) { + return ((ByteColumnVector) data).getByte(offset + ordinal); + } + + @Override + public short getShort(int ordinal) { + return ((ShortColumnVector) data).getShort(offset + ordinal); + } + + @Override + public int getInt(int ordinal) { + return ((IntColumnVector) data).getInt(offset + ordinal); + } + + @Override + public long getLong(int ordinal) { + return ((LongColumnVector) data).getLong(offset + ordinal); + } + + @Override + public float getFloat(int ordinal) { + return ((FloatColumnVector) data).getFloat(offset + ordinal); + } + + @Override + public double getDouble(int ordinal) { + return ((DoubleColumnVector) data).getDouble(offset + ordinal); + } + + @Override + public BinaryString getString(int ordinal) { + BytesColumnVector.Bytes byteArray = getByteArray(ordinal); + return BinaryString.fromBytes(byteArray.data, byteArray.offset, byteArray.len); + } + + @Override + public Decimal getDecimal(int ordinal, int precision, int scale) { + return ((DecimalColumnVector) data).getDecimal(offset + ordinal, precision, scale); + } + + @Override + public SqlTimestamp getTimestamp(int ordinal, int precision) { + return ((TimestampColumnVector) data).getTimestamp(offset + ordinal, precision); + } + + @Override + public BinaryGeneric getGeneric(int ordinal) { + throw new UnsupportedOperationException("GenericType is not supported."); + } + + @Override + public byte[] getBinary(int ordinal) { + BytesColumnVector.Bytes byteArray = getByteArray(ordinal); + if (byteArray.len == byteArray.data.length) { + return byteArray.data; + } else { + return Arrays.copyOfRange(byteArray.data, byteArray.offset, byteArray.len); + } + } + + @Override + public BaseArray getArray(int ordinal) { + return ((ArrayColumnVector) data).getArray(offset + ordinal); + } + + @Override + public BaseMap getMap(int ordinal) { + throw new UnsupportedOperationException("Map is not supported."); + } + + @Override + public BaseRow getRow(int ordinal, int numFields) { + throw new UnsupportedOperationException("Row is not supported."); + } + + @Override + public void setBoolean(int ordinal, boolean value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setByte(int ordinal, byte value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setShort(int ordinal, short value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setInt(int ordinal, int value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setLong(int ordinal, long value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setFloat(int ordinal, float value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setDouble(int ordinal, double value) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setDecimal(int i, Decimal value, int precision) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setTimestamp(int ordinal, SqlTimestamp value, int precision) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setNotNullAt(int pos) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setNullLong(int pos) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setNullInt(int pos) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setNullBoolean(int pos) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setNullByte(int pos) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setNullShort(int pos) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setNullFloat(int pos) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public void setNullDouble(int pos) { + throw new UnsupportedOperationException("Not support the operation!"); + } + + @Override + public boolean[] toBooleanArray() { + boolean[] res = new boolean[numElements]; + for (int i = 0; i < numElements; i++) { + res[i] = getBoolean(i); + } + return res; + } + + @Override + public byte[] toByteArray() { + byte[] res = new byte[numElements]; + for (int i = 0; i < numElements; i++) { + res[i] = getByte(i); + } + return res; + } + + @Override + public short[] toShortArray() { + short[] res = new short[numElements]; + for (int i = 0; i < numElements; i++) { + res[i] = getShort(i); + } + return res; + } + + @Override + public int[] toIntArray() { + int[] res = new int[numElements]; + for (int i = 0; i < numElements; i++) { + res[i] = getInt(i); + } + return res; + } + + @Override + public long[] toLongArray() { + long[] res = new long[numElements]; + for (int i = 0; i < numElements; i++) { + res[i] = getLong(i); + } + return res; + } + + @Override + public float[] toFloatArray() { + float[] res = new float[numElements]; + for (int i = 0; i < numElements; i++) { + res[i] = getFloat(i); + } + return res; + } + + @Override + public double[] toDoubleArray() { + double[] res = new double[numElements]; + for (int i = 0; i < numElements; i++) { + res[i] = getDouble(i); + } + return res; + } + + private BytesColumnVector.Bytes getByteArray(int ordinal) { + return ((BytesColumnVector) data).getBytes(offset + ordinal); + } +} diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/ColumnarRow.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/ColumnarRow.java index d536521878..1d6d68934b 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/ColumnarRow.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/ColumnarRow.java @@ -146,8 +146,7 @@ public final class ColumnarRow implements BaseRow { @Override public BaseArray getArray(int ordinal) { - // TODO - throw new UnsupportedOperationException("Array is not supported."); + return vectorizedColumnBatch.getArray(rowId, ordinal); } @Override diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/DataFormatConverters.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/DataFormatConverters.java index 5ed8aa4218..70cb4e32dc 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/DataFormatConverters.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/DataFormatConverters.java @@ -1118,7 +1118,7 @@ public class DataFormatConverters { T[] toExternalImpl(BaseArray value) { return (isEleIndentity && value instanceof GenericArray) ? genericArrayToJavaArray((GenericArray) value, elementType) : - binaryArrayToJavaArray((BinaryArray) value, elementType, componentClass, elementConverter); + baseArrayToJavaArray(value, elementType, componentClass, elementConverter); } @Override @@ -1153,7 +1153,7 @@ public class DataFormatConverters { } } - private static T[] binaryArrayToJavaArray(BinaryArray value, LogicalType elementType, + private static T[] baseArrayToJavaArray(BaseArray value, LogicalType elementType, Class componentClass, DataFormatConverter elementConverter) { int size = value.numElements(); T[] values = (T[]) Array.newInstance(componentClass, size); @@ -1259,8 +1259,8 @@ public class DataFormatConverters { private Map binaryMapToMap(BinaryMap value) { Map map = new HashMap<>(); - Object[] keys = binaryArrayToJavaArray(value.keyArray(), keyType, keyComponentClass, keyConverter); - Object[] values = binaryArrayToJavaArray(value.valueArray(), valueType, valueComponentClass, valueConverter); + Object[] keys = baseArrayToJavaArray(value.keyArray(), keyType, keyComponentClass, keyConverter); + Object[] values = baseArrayToJavaArray(value.valueArray(), valueType, valueComponentClass, valueConverter); for (int i = 0; i < value.numElements(); i++) { map.put(keys[i], values[i]); } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/ArrayColumnVector.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/ArrayColumnVector.java new file mode 100644 index 0000000000..90d1527905 --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/ArrayColumnVector.java @@ -0,0 +1,28 @@ +/* + * 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.table.dataformat.vector; + +import org.apache.flink.table.dataformat.BaseArray; + +/** + * Array column vector. + */ +public interface ArrayColumnVector extends ColumnVector { + BaseArray getArray(int i); +} diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/VectorizedColumnBatch.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/VectorizedColumnBatch.java index a7f0341f37..56caf25949 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/VectorizedColumnBatch.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/vector/VectorizedColumnBatch.java @@ -18,6 +18,7 @@ package org.apache.flink.table.dataformat.vector; +import org.apache.flink.table.dataformat.BaseArray; import org.apache.flink.table.dataformat.Decimal; import org.apache.flink.table.dataformat.SqlTimestamp; import org.apache.flink.table.dataformat.vector.BytesColumnVector.Bytes; @@ -116,4 +117,8 @@ public class VectorizedColumnBatch implements Serializable { public SqlTimestamp getTimestamp(int rowId, int colId, int precision) { return ((TimestampColumnVector) (columns[colId])).getTimestamp(rowId, precision); } + + public BaseArray getArray(int rowId, int colId) { + return ((ArrayColumnVector) columns[colId]).getArray(rowId); + } } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BaseArraySerializer.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BaseArraySerializer.java index c2c50fb47b..9413ddeb6c 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BaseArraySerializer.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BaseArraySerializer.java @@ -32,6 +32,7 @@ import org.apache.flink.table.dataformat.BaseArray; import org.apache.flink.table.dataformat.BinaryArray; import org.apache.flink.table.dataformat.BinaryArrayWriter; import org.apache.flink.table.dataformat.BinaryWriter; +import org.apache.flink.table.dataformat.ColumnarArray; import org.apache.flink.table.dataformat.GenericArray; import org.apache.flink.table.dataformat.TypeGetterSetters; import org.apache.flink.table.runtime.types.InternalSerializers; @@ -83,9 +84,13 @@ public class BaseArraySerializer extends TypeSerializer { @Override public BaseArray copy(BaseArray from) { - return from instanceof GenericArray ? - copyGenericArray((GenericArray) from) : - ((BinaryArray) from).copy(); + if (from instanceof GenericArray) { + return copyGenericArray((GenericArray) from); + } else if (from instanceof BinaryArray) { + return ((BinaryArray) from).copy(); + } else { + return copyColumnarArray((ColumnarArray) from); + } } @Override @@ -134,6 +139,53 @@ public class BaseArraySerializer extends TypeSerializer { return new GenericArray(arr, array.numElements(), array.isPrimitiveArray()); } + private GenericArray copyColumnarArray(ColumnarArray from) { + Object arr = null; + boolean isPrimitiveArray = true; + if (!eleType.isNullable()) { + switch (eleType.getTypeRoot()) { + case BOOLEAN: + arr = from.toBooleanArray(); + break; + case TINYINT: + arr = from.toByteArray(); + break; + case SMALLINT: + arr = from.toShortArray(); + break; + case INTEGER: + case DATE: + case TIME_WITHOUT_TIME_ZONE: + arr = from.toIntArray(); + break; + case BIGINT: + arr = from.toLongArray(); + break; + case FLOAT: + arr = from.toFloatArray(); + break; + case DOUBLE: + arr = from.toDoubleArray(); + break; + } + } + + if (arr == null) { + Object[] newArray = new Object[from.numElements()]; + for (int i = 0; i < newArray.length; i++) { + if (!from.isNullAt(i)) { + newArray[i] = eleSer.copy(TypeGetterSetters.get(from, i, eleType)); + } else { + newArray[i] = null; + } + } + arr = newArray; + isPrimitiveArray = false; + } + + return new GenericArray(arr, from.numElements(), isPrimitiveArray); + } + @Override public int getLength() { return -1; diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/vector/VectorizedColumnBatchTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/vector/VectorizedColumnBatchTest.java index d88a958301..286b6775b6 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/vector/VectorizedColumnBatchTest.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/vector/VectorizedColumnBatchTest.java @@ -18,6 +18,8 @@ package org.apache.flink.table.dataformat.vector; +import org.apache.flink.table.dataformat.BaseArray; +import org.apache.flink.table.dataformat.ColumnarArray; import org.apache.flink.table.dataformat.ColumnarRow; import org.apache.flink.table.dataformat.Decimal; import org.apache.flink.table.dataformat.SqlTimestamp; @@ -46,6 +48,7 @@ import static org.junit.Assert.assertTrue; public class VectorizedColumnBatchTest { private static final int VECTOR_SIZE = 1024; + private static final int ARRAY_SIZE = 3; @Test public void testTyped() throws IOException { @@ -200,6 +203,23 @@ public class VectorizedColumnBatchTest { vector11[i] = i; } + HeapIntVector col12Data = new HeapIntVector(VECTOR_SIZE * ARRAY_SIZE); + for (int i = 0; i < VECTOR_SIZE * ARRAY_SIZE; i++) { + col12Data.vector[i] = i; + } + ArrayColumnVector col12 = new ArrayColumnVector() { + + @Override + public boolean isNullAt(int i) { + return false; + } + + @Override + public BaseArray getArray(int i) { + return new ColumnarArray(col12Data, i * ARRAY_SIZE, ARRAY_SIZE); + } + }; + VectorizedColumnBatch batch = new VectorizedColumnBatch(new ColumnVector[]{ col0, col1, @@ -212,7 +232,8 @@ public class VectorizedColumnBatchTest { col8, col9, col10, - col11}); + col11, + col12}); batch.setNumRows(VECTOR_SIZE); for (int i = 0; i < batch.getNumRows(); i++) { @@ -230,6 +251,9 @@ public class VectorizedColumnBatchTest { assertEquals(row.getTimestamp(10, 9).getMillisecond(), i * 1000L + 123); assertEquals(row.getTimestamp(10, 9).getNanoOfMillisecond(), 456789); assertEquals(row.getDecimal(11, 10, 0).toUnscaledLong(), i); + for (int j = 0; j < ARRAY_SIZE; j++) { + assertEquals(row.getArray(12).getInt(j), i * ARRAY_SIZE + j); + } } assertEquals(VECTOR_SIZE, batch.getNumRows()); diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/typeutils/BaseArraySerializerTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/typeutils/BaseArraySerializerTest.java index 56fc2e1f18..e641907629 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/typeutils/BaseArraySerializerTest.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/typeutils/BaseArraySerializerTest.java @@ -31,13 +31,16 @@ import org.apache.flink.table.dataformat.BinaryArray; import org.apache.flink.table.dataformat.BinaryArrayWriter; import org.apache.flink.table.dataformat.BinaryGeneric; import org.apache.flink.table.dataformat.BinaryString; +import org.apache.flink.table.dataformat.ColumnarArray; import org.apache.flink.table.dataformat.GenericArray; +import org.apache.flink.table.dataformat.vector.heap.HeapBytesVector; import org.apache.flink.testutils.DeeplyEqualsChecker; import org.junit.Test; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; +import java.nio.charset.StandardCharsets; import static org.apache.flink.table.runtime.typeutils.SerializerTestUtil.MyObj; import static org.apache.flink.table.runtime.typeutils.SerializerTestUtil.MyObjSerializer; @@ -138,6 +141,7 @@ public class BaseArraySerializerTest extends SerializerTestBase { createArray("11", "haa", "ke"), createArray("11", "haa", "ke"), createArray("11", "lele", "haa", "ke"), + createColumnarArray("11", "lele", "haa", "ke"), }; } @@ -150,4 +154,12 @@ public class BaseArraySerializerTest extends SerializerTestBase { writer.complete(); return array; } + + private static ColumnarArray createColumnarArray(String... vs) { + HeapBytesVector vector = new HeapBytesVector(vs.length); + for (String v : vs) { + vector.fill(v.getBytes(StandardCharsets.UTF_8)); + } + return new ColumnarArray(vector, 0, vs.length); + } } -- Gitee From 72a7ccc91a7c382a65c0d3af3986b9700e4eef97 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Fri, 3 Apr 2020 13:56:27 +0800 Subject: [PATCH 465/885] [FLINK-16914][python] Support ArrayType in vectorized Python UDF --- flink-python/pyflink/fn_execution/coders.py | 73 +++++----- .../pyflink/table/tests/test_pandas_udf.py | 58 +++++++- flink-python/pyflink/table/types.py | 5 + .../flink/table/runtime/arrow/ArrowUtils.java | 132 +++++++++++------- .../arrow/readers/ArrayFieldReader.java | 83 +++++++++++ .../arrow/vectors/ArrowArrayColumnVector.java | 59 ++++++++ .../runtime/arrow/writers/ArrayWriter.java | 64 +++++++++ .../runtime/arrow/writers/BigIntWriter.java | 10 +- .../runtime/arrow/writers/BooleanWriter.java | 10 +- .../runtime/arrow/writers/DateWriter.java | 14 +- .../runtime/arrow/writers/DecimalWriter.java | 10 +- .../runtime/arrow/writers/DoubleWriter.java | 10 +- .../runtime/arrow/writers/FloatWriter.java | 10 +- .../runtime/arrow/writers/IntWriter.java | 10 +- .../runtime/arrow/writers/RowArrayWriter.java | 63 +++++++++ ...BigIntWriter.java => RowBigIntWriter.java} | 12 +- ...oleanWriter.java => RowBooleanWriter.java} | 12 +- ...eRowDateWriter.java => RowDateWriter.java} | 16 ++- ...cimalWriter.java => RowDecimalWriter.java} | 12 +- ...DoubleWriter.java => RowDoubleWriter.java} | 12 +- ...owFloatWriter.java => RowFloatWriter.java} | 12 +- ...aseRowIntWriter.java => RowIntWriter.java} | 12 +- ...lIntWriter.java => RowSmallIntWriter.java} | 12 +- ...eRowTimeWriter.java => RowTimeWriter.java} | 40 ++++-- ...ampWriter.java => RowTimestampWriter.java} | 31 ++-- ...nyIntWriter.java => RowTinyIntWriter.java} | 12 +- ...aryWriter.java => RowVarBinaryWriter.java} | 12 +- ...rCharWriter.java => RowVarCharWriter.java} | 14 +- .../runtime/arrow/writers/SmallIntWriter.java | 10 +- .../runtime/arrow/writers/TimeWriter.java | 38 ++--- .../arrow/writers/TimestampWriter.java | 31 ++-- .../runtime/arrow/writers/TinyIntWriter.java | 10 +- .../arrow/writers/VarBinaryWriter.java | 10 +- .../runtime/arrow/writers/VarCharWriter.java | 12 +- .../table/runtime/arrow/ArrowUtilsTest.java | 81 ++++++----- .../arrow/BaseRowArrowReaderWriterTest.java | 21 ++- .../arrow/RowArrowReaderWriterTest.java | 13 +- .../ArrowPythonScalarFunctionRunnerTest.java | 20 +-- .../table/runtime/util/StreamRecordUtils.java | 6 + 39 files changed, 729 insertions(+), 343 deletions(-) create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/ArrayFieldReader.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowArrayColumnVector.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/ArrayWriter.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowArrayWriter.java rename flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/{BaseRowBigIntWriter.java => RowBigIntWriter.java} (76%) rename flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/{BaseRowBooleanWriter.java => RowBooleanWriter.java} (80%) rename flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/{BaseRowDateWriter.java => RowDateWriter.java} (71%) rename flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/{BaseRowDecimalWriter.java => RowDecimalWriter.java} (80%) rename flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/{BaseRowDoubleWriter.java => RowDoubleWriter.java} (76%) rename flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/{BaseRowFloatWriter.java => RowFloatWriter.java} (76%) rename flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/{BaseRowIntWriter.java => RowIntWriter.java} (77%) rename flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/{BaseRowSmallIntWriter.java => RowSmallIntWriter.java} (76%) rename flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/{BaseRowTimeWriter.java => RowTimeWriter.java} (58%) rename flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/{BaseRowTimestampWriter.java => RowTimestampWriter.java} (64%) rename flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/{BaseRowTinyIntWriter.java => RowTinyIntWriter.java} (76%) rename flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/{BaseRowVarBinaryWriter.java => RowVarBinaryWriter.java} (76%) rename flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/{BaseRowVarCharWriter.java => RowVarCharWriter.java} (72%) diff --git a/flink-python/pyflink/fn_execution/coders.py b/flink-python/pyflink/fn_execution/coders.py index 33a4b6dbcf..065a1ae51c 100644 --- a/flink-python/pyflink/fn_execution/coders.py +++ b/flink-python/pyflink/fn_execution/coders.py @@ -34,7 +34,7 @@ from pyflink.fn_execution import flink_fn_execution_pb2 from pyflink.fn_execution.sdk_worker_main import pipeline_options from pyflink.table.types import Row, TinyIntType, SmallIntType, IntType, BigIntType, BooleanType, \ FloatType, DoubleType, VarCharType, VarBinaryType, DecimalType, DateType, TimeType, \ - LocalZonedTimestampType, RowType, RowField, to_arrow_type, TimestampType + LocalZonedTimestampType, RowType, RowField, to_arrow_type, TimestampType, ArrayType FLINK_SCALAR_FUNCTION_SCHEMA_CODER_URN = "flink:coder:schema:scalar_function:v1" FLINK_TABLE_FUNCTION_SCHEMA_CODER_URN = "flink:coder:schema:table_function:v1" @@ -421,44 +421,47 @@ class ArrowCoder(DeterministicCoder): return pa.schema([pa.field(n, to_arrow_type(t), t._nullable) for n, t in zip(row_type.field_names(), row_type.field_types())]) - def _to_data_type(field): - if field.type.type_name == flink_fn_execution_pb2.Schema.TINYINT: - return TinyIntType(field.type.nullable) - elif field.type.type_name == flink_fn_execution_pb2.Schema.SMALLINT: - return SmallIntType(field.type.nullable) - elif field.type.type_name == flink_fn_execution_pb2.Schema.INT: - return IntType(field.type.nullable) - elif field.type.type_name == flink_fn_execution_pb2.Schema.BIGINT: - return BigIntType(field.type.nullable) - elif field.type.type_name == flink_fn_execution_pb2.Schema.BOOLEAN: - return BooleanType(field.type.nullable) - elif field.type.type_name == flink_fn_execution_pb2.Schema.FLOAT: - return FloatType(field.type.nullable) - elif field.type.type_name == flink_fn_execution_pb2.Schema.DOUBLE: - return DoubleType(field.type.nullable) - elif field.type.type_name == flink_fn_execution_pb2.Schema.VARCHAR: - return VarCharType(0x7fffffff, field.type.nullable) - elif field.type.type_name == flink_fn_execution_pb2.Schema.VARBINARY: - return VarBinaryType(0x7fffffff, field.type.nullable) - elif field.type.type_name == flink_fn_execution_pb2.Schema.DECIMAL: - return DecimalType(field.type.decimal_info.precision, - field.type.decimal_info.scale, - field.type.nullable) - elif field.type.type_name == flink_fn_execution_pb2.Schema.DATE: - return DateType(field.type.nullable) - elif field.type.type_name == flink_fn_execution_pb2.Schema.TIME: - return TimeType(field.type.time_info.precision, field.type.nullable) - elif field.type.type_name == \ + def _to_data_type(field_type): + if field_type.type_name == flink_fn_execution_pb2.Schema.TINYINT: + return TinyIntType(field_type.nullable) + elif field_type.type_name == flink_fn_execution_pb2.Schema.SMALLINT: + return SmallIntType(field_type.nullable) + elif field_type.type_name == flink_fn_execution_pb2.Schema.INT: + return IntType(field_type.nullable) + elif field_type.type_name == flink_fn_execution_pb2.Schema.BIGINT: + return BigIntType(field_type.nullable) + elif field_type.type_name == flink_fn_execution_pb2.Schema.BOOLEAN: + return BooleanType(field_type.nullable) + elif field_type.type_name == flink_fn_execution_pb2.Schema.FLOAT: + return FloatType(field_type.nullable) + elif field_type.type_name == flink_fn_execution_pb2.Schema.DOUBLE: + return DoubleType(field_type.nullable) + elif field_type.type_name == flink_fn_execution_pb2.Schema.VARCHAR: + return VarCharType(0x7fffffff, field_type.nullable) + elif field_type.type_name == flink_fn_execution_pb2.Schema.VARBINARY: + return VarBinaryType(0x7fffffff, field_type.nullable) + elif field_type.type_name == flink_fn_execution_pb2.Schema.DECIMAL: + return DecimalType(field_type.decimal_info.precision, + field_type.decimal_info.scale, + field_type.nullable) + elif field_type.type_name == flink_fn_execution_pb2.Schema.DATE: + return DateType(field_type.nullable) + elif field_type.type_name == flink_fn_execution_pb2.Schema.TIME: + return TimeType(field_type.time_info.precision, field_type.nullable) + elif field_type.type_name == \ flink_fn_execution_pb2.Schema.LOCAL_ZONED_TIMESTAMP: - return LocalZonedTimestampType(field.type.local_zoned_timestamp_info.precision, - field.type.nullable) - elif field.type.type_name == flink_fn_execution_pb2.Schema.TIMESTAMP: - return TimestampType(field.type.timestamp_info.precision, field.type.nullable) + return LocalZonedTimestampType(field_type.local_zoned_timestamp_info.precision, + field_type.nullable) + elif field_type.type_name == flink_fn_execution_pb2.Schema.TIMESTAMP: + return TimestampType(field_type.timestamp_info.precision, field_type.nullable) + elif field_type.type_name == flink_fn_execution_pb2.Schema.ARRAY: + return ArrayType(_to_data_type(field_type.collection_element_type), + field_type.nullable) else: - raise ValueError("field_type %s is not supported." % field.type) + raise ValueError("field_type %s is not supported." % field_type) def _to_row_type(row_schema): - return RowType([RowField(f.name, _to_data_type(f)) for f in row_schema.fields]) + return RowType([RowField(f.name, _to_data_type(f.type)) for f in row_schema.fields]) timezone = pytz.timezone(pipeline_options.view_as(DebugOptions).lookup_experiment( "table.exec.timezone")) diff --git a/flink-python/pyflink/table/tests/test_pandas_udf.py b/flink-python/pyflink/table/tests/test_pandas_udf.py index 093be8991f..9751e44ae4 100644 --- a/flink-python/pyflink/table/tests/test_pandas_udf.py +++ b/flink-python/pyflink/table/tests/test_pandas_udf.py @@ -153,6 +153,18 @@ class PandasUDFITTests(object): timestamp_value) return timestamp_param + def array_func(array_param): + assert isinstance(array_param, pd.Series) + assert isinstance(array_param[0], np.ndarray), \ + 'array_param of wrong type %s !' % type(array_param[0]) + return array_param + + def nested_array_func(nested_array_param): + assert isinstance(nested_array_param, pd.Series) + assert isinstance(nested_array_param[0], np.ndarray), \ + 'nested_array_param of wrong type %s !' % type(nested_array_param[0]) + return pd.Series(nested_array_param[0]) + self.t_env.register_function( "tinyint_func", udf(tinyint_func, [DataTypes.TINYINT()], DataTypes.TINYINT(), udf_type="pandas")) @@ -207,12 +219,36 @@ class PandasUDFITTests(object): udf(timestamp_func, [DataTypes.TIMESTAMP(3)], DataTypes.TIMESTAMP(3), udf_type="pandas")) + self.t_env.register_function( + "array_str_func", + udf(array_func, [DataTypes.ARRAY(DataTypes.STRING())], + DataTypes.ARRAY(DataTypes.STRING()), udf_type="pandas")) + + self.t_env.register_function( + "array_timestamp_func", + udf(array_func, [DataTypes.ARRAY(DataTypes.TIMESTAMP(3))], + DataTypes.ARRAY(DataTypes.TIMESTAMP(3)), udf_type="pandas")) + + self.t_env.register_function( + "array_int_func", + udf(array_func, [DataTypes.ARRAY(DataTypes.INT())], + DataTypes.ARRAY(DataTypes.INT()), udf_type="pandas")) + + self.t_env.register_function( + "nested_array_func", + udf(nested_array_func, [DataTypes.ARRAY(DataTypes.ARRAY(DataTypes.STRING()))], + DataTypes.ARRAY(DataTypes.STRING()), udf_type="pandas")) + table_sink = source_sink_utils.TestAppendSink( - ['a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j', 'k', 'l', 'm', 'n', 'o', 'p'], + ['a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j', 'k', 'l', 'm', 'n', 'o', 'p', 'q', + 'r', 's', 't'], [DataTypes.TINYINT(), DataTypes.SMALLINT(), DataTypes.INT(), DataTypes.BIGINT(), DataTypes.BOOLEAN(), DataTypes.BOOLEAN(), DataTypes.FLOAT(), DataTypes.DOUBLE(), DataTypes.STRING(), DataTypes.STRING(), DataTypes.BYTES(), DataTypes.DECIMAL(38, 18), - DataTypes.DECIMAL(38, 18), DataTypes.DATE(), DataTypes.TIME(), DataTypes.TIMESTAMP(3)]) + DataTypes.DECIMAL(38, 18), DataTypes.DATE(), DataTypes.TIME(), DataTypes.TIMESTAMP(3), + DataTypes.ARRAY(DataTypes.STRING()), DataTypes.ARRAY(DataTypes.TIMESTAMP(3)), + DataTypes.ARRAY(DataTypes.INT()), + DataTypes.ARRAY(DataTypes.STRING())]) self.t_env.register_table_sink("Results", table_sink) t = self.t_env.from_elements( @@ -220,7 +256,8 @@ class PandasUDFITTests(object): bytearray(b'flink'), decimal.Decimal('1000000000000000000.05'), decimal.Decimal('1000000000000000000.05999999999999999899999999999'), datetime.date(2014, 9, 13), datetime.time(hour=1, minute=0, second=1), - timestamp_value)], + timestamp_value, ['hello', '中文', None], [timestamp_value], [1, 2], + [['hello', '中文', None]])], DataTypes.ROW( [DataTypes.FIELD("a", DataTypes.TINYINT()), DataTypes.FIELD("b", DataTypes.SMALLINT()), @@ -237,7 +274,11 @@ class PandasUDFITTests(object): DataTypes.FIELD("m", DataTypes.DECIMAL(38, 18)), DataTypes.FIELD("n", DataTypes.DATE()), DataTypes.FIELD("o", DataTypes.TIME()), - DataTypes.FIELD("p", DataTypes.TIMESTAMP(3))])) + DataTypes.FIELD("p", DataTypes.TIMESTAMP(3)), + DataTypes.FIELD("q", DataTypes.ARRAY(DataTypes.STRING())), + DataTypes.FIELD("r", DataTypes.ARRAY(DataTypes.TIMESTAMP(3))), + DataTypes.FIELD("s", DataTypes.ARRAY(DataTypes.INT())), + DataTypes.FIELD("t", DataTypes.ARRAY(DataTypes.ARRAY(DataTypes.STRING())))])) t.select("tinyint_func(a)," "smallint_func(b)," @@ -254,7 +295,11 @@ class PandasUDFITTests(object): "decimal_func(m)," "date_func(n)," "time_func(o)," - "timestamp_func(p)") \ + "timestamp_func(p)," + "array_str_func(q)," + "array_timestamp_func(r)," + "array_int_func(s)," + "nested_array_func(t)") \ .insert_into("Results") self.t_env.execute("test") actual = source_sink_utils.results() @@ -262,7 +307,8 @@ class PandasUDFITTests(object): ["1,32767,-2147483648,1,true,false,1.0,1.0,hello,中文," "[102, 108, 105, 110, 107],1000000000000000000.050000000000000000," "1000000000000000000.059999999999999999,2014-09-13,01:00:01," - "1970-01-01 00:00:00.123"]) + "1970-01-01 00:00:00.123,[hello, 中文, null],[1970-01-01 00:00:00.123]," + "[1, 2],[hello, 中文, null]"]) class BlinkPandasUDFITTests(object): diff --git a/flink-python/pyflink/table/types.py b/flink-python/pyflink/table/types.py index 6b26fa9ec8..7f4eddf670 100644 --- a/flink-python/pyflink/table/types.py +++ b/flink-python/pyflink/table/types.py @@ -2333,6 +2333,11 @@ def to_arrow_type(data_type): return pa.timestamp('us') else: return pa.timestamp('ns') + elif type(data_type) == ArrayType: + if type(data_type.element_type) == LocalZonedTimestampType: + raise ValueError("%s is not supported to be used as the element type of ArrayType." % + data_type.element_type) + return pa.list_(to_arrow_type(data_type.element_type)) else: raise ValueError("field_type %s is not supported." % data_type) diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java index d38a1e9dc6..d58cd1a4f2 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java @@ -20,7 +20,9 @@ package org.apache.flink.table.runtime.arrow; import org.apache.flink.annotation.Internal; import org.apache.flink.table.dataformat.BaseRow; +import org.apache.flink.table.dataformat.TypeGetterSetters; import org.apache.flink.table.dataformat.vector.ColumnVector; +import org.apache.flink.table.runtime.arrow.readers.ArrayFieldReader; import org.apache.flink.table.runtime.arrow.readers.ArrowFieldReader; import org.apache.flink.table.runtime.arrow.readers.BigIntFieldReader; import org.apache.flink.table.runtime.arrow.readers.BooleanFieldReader; @@ -36,6 +38,7 @@ import org.apache.flink.table.runtime.arrow.readers.TimestampFieldReader; import org.apache.flink.table.runtime.arrow.readers.TinyIntFieldReader; import org.apache.flink.table.runtime.arrow.readers.VarBinaryFieldReader; import org.apache.flink.table.runtime.arrow.readers.VarCharFieldReader; +import org.apache.flink.table.runtime.arrow.vectors.ArrowArrayColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowBigIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowBooleanColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowDateColumnVector; @@ -50,20 +53,8 @@ import org.apache.flink.table.runtime.arrow.vectors.ArrowTinyIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowVarBinaryColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowVarCharColumnVector; import org.apache.flink.table.runtime.arrow.vectors.BaseRowArrowReader; +import org.apache.flink.table.runtime.arrow.writers.ArrayWriter; import org.apache.flink.table.runtime.arrow.writers.ArrowFieldWriter; -import org.apache.flink.table.runtime.arrow.writers.BaseRowBigIntWriter; -import org.apache.flink.table.runtime.arrow.writers.BaseRowBooleanWriter; -import org.apache.flink.table.runtime.arrow.writers.BaseRowDateWriter; -import org.apache.flink.table.runtime.arrow.writers.BaseRowDecimalWriter; -import org.apache.flink.table.runtime.arrow.writers.BaseRowDoubleWriter; -import org.apache.flink.table.runtime.arrow.writers.BaseRowFloatWriter; -import org.apache.flink.table.runtime.arrow.writers.BaseRowIntWriter; -import org.apache.flink.table.runtime.arrow.writers.BaseRowSmallIntWriter; -import org.apache.flink.table.runtime.arrow.writers.BaseRowTimeWriter; -import org.apache.flink.table.runtime.arrow.writers.BaseRowTimestampWriter; -import org.apache.flink.table.runtime.arrow.writers.BaseRowTinyIntWriter; -import org.apache.flink.table.runtime.arrow.writers.BaseRowVarBinaryWriter; -import org.apache.flink.table.runtime.arrow.writers.BaseRowVarCharWriter; import org.apache.flink.table.runtime.arrow.writers.BigIntWriter; import org.apache.flink.table.runtime.arrow.writers.BooleanWriter; import org.apache.flink.table.runtime.arrow.writers.DateWriter; @@ -71,12 +62,27 @@ import org.apache.flink.table.runtime.arrow.writers.DecimalWriter; import org.apache.flink.table.runtime.arrow.writers.DoubleWriter; import org.apache.flink.table.runtime.arrow.writers.FloatWriter; import org.apache.flink.table.runtime.arrow.writers.IntWriter; +import org.apache.flink.table.runtime.arrow.writers.RowArrayWriter; +import org.apache.flink.table.runtime.arrow.writers.RowBigIntWriter; +import org.apache.flink.table.runtime.arrow.writers.RowBooleanWriter; +import org.apache.flink.table.runtime.arrow.writers.RowDateWriter; +import org.apache.flink.table.runtime.arrow.writers.RowDecimalWriter; +import org.apache.flink.table.runtime.arrow.writers.RowDoubleWriter; +import org.apache.flink.table.runtime.arrow.writers.RowFloatWriter; +import org.apache.flink.table.runtime.arrow.writers.RowIntWriter; +import org.apache.flink.table.runtime.arrow.writers.RowSmallIntWriter; +import org.apache.flink.table.runtime.arrow.writers.RowTimeWriter; +import org.apache.flink.table.runtime.arrow.writers.RowTimestampWriter; +import org.apache.flink.table.runtime.arrow.writers.RowTinyIntWriter; +import org.apache.flink.table.runtime.arrow.writers.RowVarBinaryWriter; +import org.apache.flink.table.runtime.arrow.writers.RowVarCharWriter; import org.apache.flink.table.runtime.arrow.writers.SmallIntWriter; import org.apache.flink.table.runtime.arrow.writers.TimeWriter; import org.apache.flink.table.runtime.arrow.writers.TimestampWriter; import org.apache.flink.table.runtime.arrow.writers.TinyIntWriter; import org.apache.flink.table.runtime.arrow.writers.VarBinaryWriter; import org.apache.flink.table.runtime.arrow.writers.VarCharWriter; +import org.apache.flink.table.types.logical.ArrayType; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BooleanType; import org.apache.flink.table.types.logical.DateType; @@ -116,6 +122,7 @@ import org.apache.arrow.vector.TinyIntVector; import org.apache.arrow.vector.VarBinaryVector; import org.apache.arrow.vector.VarCharVector; import org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.arrow.vector.complex.ListVector; import org.apache.arrow.vector.types.DateUnit; import org.apache.arrow.vector.types.FloatingPointPrecision; import org.apache.arrow.vector.types.TimeUnit; @@ -127,6 +134,7 @@ import org.apache.arrow.vector.types.pojo.Schema; import java.math.BigDecimal; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.stream.Collectors; @@ -143,17 +151,22 @@ public final class ArrowUtils { */ public static Schema toArrowSchema(RowType rowType) { Collection fields = rowType.getFields().stream() - .map(ArrowUtils::toArrowField) + .map(f -> ArrowUtils.toArrowField(f.getName(), f.getType())) .collect(Collectors.toCollection(ArrayList::new)); return new Schema(fields); } - private static Field toArrowField(RowType.RowField rowField) { + private static Field toArrowField(String fieldName, LogicalType logicalType) { FieldType fieldType = new FieldType( - rowField.getType().isNullable(), - rowField.getType().accept(LogicalTypeToArrowTypeConverter.INSTANCE), + logicalType.isNullable(), + logicalType.accept(LogicalTypeToArrowTypeConverter.INSTANCE), null); - return new Field(rowField.getName(), fieldType, null); + List children = null; + if (logicalType instanceof ArrayType) { + children = Collections.singletonList(toArrowField( + "element", ((ArrayType) logicalType).getElementType())); + } + return new Field(fieldName, fieldType, children); } /** @@ -173,33 +186,37 @@ public final class ArrowUtils { private static ArrowFieldWriter createRowArrowFieldWriter(FieldVector vector, LogicalType fieldType) { if (vector instanceof TinyIntVector) { - return new TinyIntWriter((TinyIntVector) vector); + return new RowTinyIntWriter((TinyIntVector) vector); } else if (vector instanceof SmallIntVector) { - return new SmallIntWriter((SmallIntVector) vector); + return new RowSmallIntWriter((SmallIntVector) vector); } else if (vector instanceof IntVector) { - return new IntWriter((IntVector) vector); + return new RowIntWriter((IntVector) vector); } else if (vector instanceof BigIntVector) { - return new BigIntWriter((BigIntVector) vector); + return new RowBigIntWriter((BigIntVector) vector); } else if (vector instanceof BitVector) { - return new BooleanWriter((BitVector) vector); + return new RowBooleanWriter((BitVector) vector); } else if (vector instanceof Float4Vector) { - return new FloatWriter((Float4Vector) vector); + return new RowFloatWriter((Float4Vector) vector); } else if (vector instanceof Float8Vector) { - return new DoubleWriter((Float8Vector) vector); + return new RowDoubleWriter((Float8Vector) vector); } else if (vector instanceof VarCharVector) { - return new VarCharWriter((VarCharVector) vector); + return new RowVarCharWriter((VarCharVector) vector); } else if (vector instanceof VarBinaryVector) { - return new VarBinaryWriter((VarBinaryVector) vector); + return new RowVarBinaryWriter((VarBinaryVector) vector); } else if (vector instanceof DecimalVector) { DecimalVector decimalVector = (DecimalVector) vector; - return new DecimalWriter(decimalVector, getPrecision(decimalVector), decimalVector.getScale()); + return new RowDecimalWriter(decimalVector, getPrecision(decimalVector), decimalVector.getScale()); } else if (vector instanceof DateDayVector) { - return new DateWriter((DateDayVector) vector); + return new RowDateWriter((DateDayVector) vector); } else if (vector instanceof TimeSecVector || vector instanceof TimeMilliVector || vector instanceof TimeMicroVector || vector instanceof TimeNanoVector) { - return new TimeWriter(vector); + return new RowTimeWriter(vector); } else if (vector instanceof TimeStampVector && ((ArrowType.Timestamp) vector.getField().getType()).getTimezone() == null) { - return new TimestampWriter(vector); + return new RowTimestampWriter(vector); + } else if (vector instanceof ListVector) { + ListVector listVector = (ListVector) vector; + LogicalType elementType = ((ArrayType) fieldType).getElementType(); + return new RowArrayWriter(listVector, createRowArrowFieldWriter(listVector.getDataVector(), elementType)); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -215,39 +232,39 @@ public final class ArrowUtils { for (int i = 0; i < vectors.size(); i++) { FieldVector vector = vectors.get(i); vector.allocateNew(); - fieldWriters[i] = createBaseRowArrowFieldWriter(vector, rowType.getTypeAt(i)); + fieldWriters[i] = createArrowFieldWriter(vector, rowType.getTypeAt(i)); } return new ArrowWriter<>(root, fieldWriters); } - private static ArrowFieldWriter createBaseRowArrowFieldWriter(FieldVector vector, LogicalType fieldType) { + private static ArrowFieldWriter createArrowFieldWriter(FieldVector vector, LogicalType fieldType) { if (vector instanceof TinyIntVector) { - return new BaseRowTinyIntWriter((TinyIntVector) vector); + return new TinyIntWriter<>((TinyIntVector) vector); } else if (vector instanceof SmallIntVector) { - return new BaseRowSmallIntWriter((SmallIntVector) vector); + return new SmallIntWriter<>((SmallIntVector) vector); } else if (vector instanceof IntVector) { - return new BaseRowIntWriter((IntVector) vector); + return new IntWriter<>((IntVector) vector); } else if (vector instanceof BigIntVector) { - return new BaseRowBigIntWriter((BigIntVector) vector); + return new BigIntWriter<>((BigIntVector) vector); } else if (vector instanceof BitVector) { - return new BaseRowBooleanWriter((BitVector) vector); + return new BooleanWriter<>((BitVector) vector); } else if (vector instanceof Float4Vector) { - return new BaseRowFloatWriter((Float4Vector) vector); + return new FloatWriter<>((Float4Vector) vector); } else if (vector instanceof Float8Vector) { - return new BaseRowDoubleWriter((Float8Vector) vector); + return new DoubleWriter<>((Float8Vector) vector); } else if (vector instanceof VarCharVector) { - return new BaseRowVarCharWriter((VarCharVector) vector); + return new VarCharWriter<>((VarCharVector) vector); } else if (vector instanceof VarBinaryVector) { - return new BaseRowVarBinaryWriter((VarBinaryVector) vector); + return new VarBinaryWriter<>((VarBinaryVector) vector); } else if (vector instanceof DecimalVector) { DecimalVector decimalVector = (DecimalVector) vector; - return new BaseRowDecimalWriter(decimalVector, getPrecision(decimalVector), decimalVector.getScale()); + return new DecimalWriter<>(decimalVector, getPrecision(decimalVector), decimalVector.getScale()); } else if (vector instanceof DateDayVector) { - return new BaseRowDateWriter((DateDayVector) vector); + return new DateWriter<>((DateDayVector) vector); } else if (vector instanceof TimeSecVector || vector instanceof TimeMilliVector || vector instanceof TimeMicroVector || vector instanceof TimeNanoVector) { - return new BaseRowTimeWriter(vector); + return new TimeWriter<>(vector); } else if (vector instanceof TimeStampVector && ((ArrowType.Timestamp) vector.getField().getType()).getTimezone() == null) { int precision; if (fieldType instanceof LocalZonedTimestampType) { @@ -255,7 +272,11 @@ public final class ArrowUtils { } else { precision = ((TimestampType) fieldType).getPrecision(); } - return new BaseRowTimestampWriter(vector, precision); + return new TimestampWriter<>(vector, precision); + } else if (vector instanceof ListVector) { + ListVector listVector = (ListVector) vector; + LogicalType elementType = ((ArrayType) fieldType).getElementType(); + return new ArrayWriter<>(listVector, createArrowFieldWriter(listVector.getDataVector(), elementType)); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -275,7 +296,7 @@ public final class ArrowUtils { return new RowArrowReader(fieldReaders.toArray(new ArrowFieldReader[0])); } - private static ArrowFieldReader createRowArrowFieldReader(FieldVector vector, LogicalType fieldType) { + public static ArrowFieldReader createRowArrowFieldReader(FieldVector vector, LogicalType fieldType) { if (vector instanceof TinyIntVector) { return new TinyIntFieldReader((TinyIntVector) vector); } else if (vector instanceof SmallIntVector) { @@ -303,6 +324,12 @@ public final class ArrowUtils { return new TimeFieldReader(vector); } else if (vector instanceof TimeStampVector && ((ArrowType.Timestamp) vector.getField().getType()).getTimezone() == null) { return new TimestampFieldReader(vector); + } else if (vector instanceof ListVector) { + ListVector listVector = (ListVector) vector; + LogicalType elementType = ((ArrayType) fieldType).getElementType(); + return new ArrayFieldReader(listVector, + createRowArrowFieldReader(listVector.getDataVector(), elementType), + elementType); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -322,7 +349,7 @@ public final class ArrowUtils { return new BaseRowArrowReader(columnVectors.toArray(new ColumnVector[0])); } - private static ColumnVector createColumnVector(FieldVector vector, LogicalType fieldType) { + public static ColumnVector createColumnVector(FieldVector vector, LogicalType fieldType) { if (vector instanceof TinyIntVector) { return new ArrowTinyIntColumnVector((TinyIntVector) vector); } else if (vector instanceof SmallIntVector) { @@ -350,6 +377,10 @@ public final class ArrowUtils { return new ArrowTimeColumnVector(vector); } else if (vector instanceof TimeStampVector && ((ArrowType.Timestamp) vector.getField().getType()).getTimezone() == null) { return new ArrowTimestampColumnVector(vector); + } else if (vector instanceof ListVector) { + ListVector listVector = (ListVector) vector; + return new ArrowArrayColumnVector(listVector, + createColumnVector(listVector.getDataVector(), ((ArrayType) fieldType).getElementType())); } else { throw new UnsupportedOperationException(String.format( "Unsupported type %s.", fieldType)); @@ -454,6 +485,11 @@ public final class ArrowUtils { } } + @Override + public ArrowType visit(ArrayType arrayType) { + return ArrowType.List.INSTANCE; + } + @Override protected ArrowType defaultMethod(LogicalType logicalType) { if (logicalType instanceof LegacyTypeInformationType) { diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/ArrayFieldReader.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/ArrayFieldReader.java new file mode 100644 index 0000000000..65cb34c44f --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/readers/ArrayFieldReader.java @@ -0,0 +1,83 @@ +/* + * 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.table.runtime.arrow.readers; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.DateType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.TimeType; +import org.apache.flink.table.types.logical.TimestampType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.flink.util.Preconditions; + +import org.apache.arrow.vector.complex.ListVector; + +import java.lang.reflect.Array; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; + + +/** + * {@link ArrowFieldReader} for Array. + */ +@Internal +public final class ArrayFieldReader extends ArrowFieldReader { + + private final ArrowFieldReader arrayData; + private final Class elementClass; + + public ArrayFieldReader(ListVector listVector, ArrowFieldReader arrayData, LogicalType elementType) { + super(listVector); + this.arrayData = Preconditions.checkNotNull(arrayData); + this.elementClass = getElementClass(elementType); + } + + @Override + public Object[] read(int index) { + if (getValueVector().isNull(index)) { + return null; + } else { + int startIndex = index * ListVector.OFFSET_WIDTH; + int start = getValueVector().getOffsetBuffer().getInt(startIndex); + int end = getValueVector().getOffsetBuffer().getInt(startIndex + ListVector.OFFSET_WIDTH); + Object[] result = (Object[]) Array.newInstance(elementClass, end - start); + for (int i = 0; i < result.length; i++) { + result[i] = arrayData.read(start + i); + } + return result; + } + } + + private Class getElementClass(LogicalType elementType) { + DataType dataType = TypeConversions.fromLogicalToDataType(elementType); + if (elementType instanceof TimestampType) { + // the default conversion class is java.time.LocalDateTime + dataType = dataType.bridgedTo(Timestamp.class); + } else if (elementType instanceof DateType) { + // the default conversion class is java.time.LocalDate + dataType = dataType.bridgedTo(Date.class); + } else if (elementType instanceof TimeType) { + // the default conversion class is java.time.LocalTime + dataType = dataType.bridgedTo(Time.class); + } + return dataType.getConversionClass(); + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowArrayColumnVector.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowArrayColumnVector.java new file mode 100644 index 0000000000..ea51f3357e --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/vectors/ArrowArrayColumnVector.java @@ -0,0 +1,59 @@ +/* + * 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.table.runtime.arrow.vectors; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.dataformat.BaseArray; +import org.apache.flink.table.dataformat.ColumnarArray; +import org.apache.flink.table.dataformat.vector.ArrayColumnVector; +import org.apache.flink.table.dataformat.vector.ColumnVector; +import org.apache.flink.util.Preconditions; + +import org.apache.arrow.vector.complex.ListVector; + +/** + * Arrow column vector for Array. + */ +@Internal +public final class ArrowArrayColumnVector implements ArrayColumnVector { + + /** + * Container which is used to store the sequence of array values of a column to read. + */ + private final ListVector listVector; + private final ColumnVector elementVector; + + public ArrowArrayColumnVector(ListVector listVector, ColumnVector elementVector) { + this.listVector = Preconditions.checkNotNull(listVector); + this.elementVector = Preconditions.checkNotNull(elementVector); + } + + @Override + public BaseArray getArray(int i) { + int index = i * ListVector.OFFSET_WIDTH; + int start = listVector.getOffsetBuffer().getInt(index); + int end = listVector.getOffsetBuffer().getInt(index + ListVector.OFFSET_WIDTH); + return new ColumnarArray(elementVector, start, end - start); + } + + @Override + public boolean isNullAt(int i) { + return listVector.isNull(i); + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/ArrayWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/ArrayWriter.java new file mode 100644 index 0000000000..950514a1bf --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/ArrayWriter.java @@ -0,0 +1,64 @@ +/* + * 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.table.runtime.arrow.writers; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.dataformat.BaseArray; +import org.apache.flink.table.dataformat.TypeGetterSetters; +import org.apache.flink.util.Preconditions; + +import org.apache.arrow.vector.complex.ListVector; + +/** + * {@link ArrowFieldWriter} for Array. + */ +@Internal +public final class ArrayWriter extends ArrowFieldWriter { + + private final ArrowFieldWriter elementWriter; + + public ArrayWriter(ListVector listVector, ArrowFieldWriter elementWriter) { + super(listVector); + this.elementWriter = Preconditions.checkNotNull(elementWriter); + } + + @Override + public void doWrite(T1 row, int ordinal) { + if (!row.isNullAt(ordinal)) { + ((ListVector) getValueVector()).startNewValue(getCount()); + BaseArray array = row.getArray(ordinal); + for (int i = 0; i < array.numElements(); i++) { + elementWriter.write((T2) array, i); + } + ((ListVector) getValueVector()).endValue(getCount(), array.numElements()); + } + } + + @Override + public void finish() { + super.finish(); + elementWriter.finish(); + } + + @Override + public void reset() { + super.reset(); + elementWriter.reset(); + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BigIntWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BigIntWriter.java index 398c74b003..442367acdd 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BigIntWriter.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BigIntWriter.java @@ -19,7 +19,7 @@ package org.apache.flink.table.runtime.arrow.writers; import org.apache.flink.annotation.Internal; -import org.apache.flink.types.Row; +import org.apache.flink.table.dataformat.TypeGetterSetters; import org.apache.arrow.vector.BigIntVector; @@ -27,18 +27,18 @@ import org.apache.arrow.vector.BigIntVector; * {@link ArrowFieldWriter} for BigInt. */ @Internal -public final class BigIntWriter extends ArrowFieldWriter { +public final class BigIntWriter extends ArrowFieldWriter { public BigIntWriter(BigIntVector bigIntVector) { super(bigIntVector); } @Override - public void doWrite(Row value, int ordinal) { - if (value.getField(ordinal) == null) { + public void doWrite(T row, int ordinal) { + if (row.isNullAt(ordinal)) { ((BigIntVector) getValueVector()).setNull(getCount()); } else { - ((BigIntVector) getValueVector()).setSafe(getCount(), (long) value.getField(ordinal)); + ((BigIntVector) getValueVector()).setSafe(getCount(), row.getLong(ordinal)); } } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BooleanWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BooleanWriter.java index 37d9fddd50..5f7169c052 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BooleanWriter.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BooleanWriter.java @@ -19,7 +19,7 @@ package org.apache.flink.table.runtime.arrow.writers; import org.apache.flink.annotation.Internal; -import org.apache.flink.types.Row; +import org.apache.flink.table.dataformat.TypeGetterSetters; import org.apache.arrow.vector.BitVector; @@ -27,17 +27,17 @@ import org.apache.arrow.vector.BitVector; * {@link ArrowFieldWriter} for Boolean. */ @Internal -public final class BooleanWriter extends ArrowFieldWriter { +public final class BooleanWriter extends ArrowFieldWriter { public BooleanWriter(BitVector bitVector) { super(bitVector); } @Override - public void doWrite(Row value, int ordinal) { - if (value.getField(ordinal) == null) { + public void doWrite(T row, int ordinal) { + if (row.isNullAt(ordinal)) { ((BitVector) getValueVector()).setNull(getCount()); - } else if ((boolean) value.getField(ordinal)) { + } else if (row.getBoolean(ordinal)) { ((BitVector) getValueVector()).setSafe(getCount(), 1); } else { ((BitVector) getValueVector()).setSafe(getCount(), 0); diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/DateWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/DateWriter.java index 27a6a91a18..9292ac2088 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/DateWriter.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/DateWriter.java @@ -19,30 +19,26 @@ package org.apache.flink.table.runtime.arrow.writers; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.runtime.typeutils.PythonTypeUtils; -import org.apache.flink.types.Row; +import org.apache.flink.table.dataformat.TypeGetterSetters; import org.apache.arrow.vector.DateDayVector; -import java.sql.Date; - /** * {@link ArrowFieldWriter} for Date. */ @Internal -public final class DateWriter extends ArrowFieldWriter { +public final class DateWriter extends ArrowFieldWriter { public DateWriter(DateDayVector dateDayVector) { super(dateDayVector); } @Override - public void doWrite(Row value, int ordinal) { - if (value.getField(ordinal) == null) { + public void doWrite(T row, int ordinal) { + if (row.isNullAt(ordinal)) { ((DateDayVector) getValueVector()).setNull(getCount()); } else { - ((DateDayVector) getValueVector()).setSafe( - getCount(), PythonTypeUtils.dateToInternal(((Date) value.getField(ordinal)))); + ((DateDayVector) getValueVector()).setSafe(getCount(), row.getInt(ordinal)); } } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/DecimalWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/DecimalWriter.java index 85da42edfb..206193feec 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/DecimalWriter.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/DecimalWriter.java @@ -19,7 +19,7 @@ package org.apache.flink.table.runtime.arrow.writers; import org.apache.flink.annotation.Internal; -import org.apache.flink.types.Row; +import org.apache.flink.table.dataformat.TypeGetterSetters; import org.apache.arrow.vector.DecimalVector; @@ -31,7 +31,7 @@ import static org.apache.flink.table.runtime.typeutils.PythonTypeUtils.fromBigDe * {@link ArrowFieldWriter} for Decimal. */ @Internal -public final class DecimalWriter extends ArrowFieldWriter { +public final class DecimalWriter extends ArrowFieldWriter { private final int precision; private final int scale; @@ -43,11 +43,11 @@ public final class DecimalWriter extends ArrowFieldWriter { } @Override - public void doWrite(Row value, int ordinal) { - if (value.getField(ordinal) == null) { + public void doWrite(T row, int ordinal) { + if (row.isNullAt(ordinal)) { ((DecimalVector) getValueVector()).setNull(getCount()); } else { - BigDecimal bigDecimal = (BigDecimal) value.getField(ordinal); + BigDecimal bigDecimal = row.getDecimal(ordinal, precision, scale).toBigDecimal(); bigDecimal = fromBigDecimal(bigDecimal, precision, scale); if (bigDecimal == null) { ((DecimalVector) getValueVector()).setNull(getCount()); diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/DoubleWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/DoubleWriter.java index 020b173b1b..22de665b3d 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/DoubleWriter.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/DoubleWriter.java @@ -19,7 +19,7 @@ package org.apache.flink.table.runtime.arrow.writers; import org.apache.flink.annotation.Internal; -import org.apache.flink.types.Row; +import org.apache.flink.table.dataformat.TypeGetterSetters; import org.apache.arrow.vector.Float8Vector; @@ -27,18 +27,18 @@ import org.apache.arrow.vector.Float8Vector; * {@link ArrowFieldWriter} for Double. */ @Internal -public final class DoubleWriter extends ArrowFieldWriter { +public final class DoubleWriter extends ArrowFieldWriter { public DoubleWriter(Float8Vector doubleVector) { super(doubleVector); } @Override - public void doWrite(Row value, int ordinal) { - if (value.getField(ordinal) == null) { + public void doWrite(T row, int ordinal) { + if (row.isNullAt(ordinal)) { ((Float8Vector) getValueVector()).setNull(getCount()); } else { - ((Float8Vector) getValueVector()).setSafe(getCount(), (double) value.getField(ordinal)); + ((Float8Vector) getValueVector()).setSafe(getCount(), row.getDouble(ordinal)); } } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/FloatWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/FloatWriter.java index 5d08f01d40..fa04b86ad8 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/FloatWriter.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/FloatWriter.java @@ -19,7 +19,7 @@ package org.apache.flink.table.runtime.arrow.writers; import org.apache.flink.annotation.Internal; -import org.apache.flink.types.Row; +import org.apache.flink.table.dataformat.TypeGetterSetters; import org.apache.arrow.vector.Float4Vector; @@ -27,18 +27,18 @@ import org.apache.arrow.vector.Float4Vector; * {@link ArrowFieldWriter} for Float. */ @Internal -public final class FloatWriter extends ArrowFieldWriter { +public final class FloatWriter extends ArrowFieldWriter { public FloatWriter(Float4Vector floatVector) { super(floatVector); } @Override - public void doWrite(Row value, int ordinal) { - if (value.getField(ordinal) == null) { + public void doWrite(T row, int ordinal) { + if (row.isNullAt(ordinal)) { ((Float4Vector) getValueVector()).setNull(getCount()); } else { - ((Float4Vector) getValueVector()).setSafe(getCount(), (float) value.getField(ordinal)); + ((Float4Vector) getValueVector()).setSafe(getCount(), row.getFloat(ordinal)); } } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/IntWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/IntWriter.java index a6ede7df7a..a5712106d0 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/IntWriter.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/IntWriter.java @@ -19,7 +19,7 @@ package org.apache.flink.table.runtime.arrow.writers; import org.apache.flink.annotation.Internal; -import org.apache.flink.types.Row; +import org.apache.flink.table.dataformat.TypeGetterSetters; import org.apache.arrow.vector.IntVector; @@ -27,18 +27,18 @@ import org.apache.arrow.vector.IntVector; * {@link ArrowFieldWriter} for Int. */ @Internal -public final class IntWriter extends ArrowFieldWriter { +public final class IntWriter extends ArrowFieldWriter { public IntWriter(IntVector intVector) { super(intVector); } @Override - public void doWrite(Row value, int ordinal) { - if (value.getField(ordinal) == null) { + public void doWrite(T row, int ordinal) { + if (row.isNullAt(ordinal)) { ((IntVector) getValueVector()).setNull(getCount()); } else { - ((IntVector) getValueVector()).setSafe(getCount(), (int) value.getField(ordinal)); + ((IntVector) getValueVector()).setSafe(getCount(), row.getInt(ordinal)); } } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowArrayWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowArrayWriter.java new file mode 100644 index 0000000000..e7dcbf115e --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowArrayWriter.java @@ -0,0 +1,63 @@ +/* + * 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.table.runtime.arrow.writers; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.types.Row; + +import org.apache.arrow.vector.complex.ListVector; + +/** + * {@link ArrowFieldWriter} for Array. + */ +@Internal +public final class RowArrayWriter extends ArrowFieldWriter { + + private final ArrowFieldWriter elementWriter; + + public RowArrayWriter(ListVector listVector, ArrowFieldWriter elementWriter) { + super(listVector); + this.elementWriter = elementWriter; + } + + @Override + public void doWrite(Row row, int ordinal) { + Object field = row.getField(ordinal); + if (field != null) { + ((ListVector) getValueVector()).startNewValue(getCount()); + Object[] array = (Object[]) field; + for (Object element : array) { + elementWriter.write(Row.of(element), 0); + } + ((ListVector) getValueVector()).endValue(getCount(), array.length); + } + } + + @Override + public void finish() { + super.finish(); + elementWriter.finish(); + } + + @Override + public void reset() { + super.reset(); + elementWriter.reset(); + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowBigIntWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowBigIntWriter.java similarity index 76% rename from flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowBigIntWriter.java rename to flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowBigIntWriter.java index 55bf5e9ce5..65064712b1 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowBigIntWriter.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowBigIntWriter.java @@ -19,7 +19,7 @@ package org.apache.flink.table.runtime.arrow.writers; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.dataformat.BaseRow; +import org.apache.flink.types.Row; import org.apache.arrow.vector.BigIntVector; @@ -27,18 +27,18 @@ import org.apache.arrow.vector.BigIntVector; * {@link ArrowFieldWriter} for BigInt. */ @Internal -public final class BaseRowBigIntWriter extends ArrowFieldWriter { +public final class RowBigIntWriter extends ArrowFieldWriter { - public BaseRowBigIntWriter(BigIntVector bigIntVector) { + public RowBigIntWriter(BigIntVector bigIntVector) { super(bigIntVector); } @Override - public void doWrite(BaseRow row, int ordinal) { - if (row.isNullAt(ordinal)) { + public void doWrite(Row value, int ordinal) { + if (value.getField(ordinal) == null) { ((BigIntVector) getValueVector()).setNull(getCount()); } else { - ((BigIntVector) getValueVector()).setSafe(getCount(), row.getLong(ordinal)); + ((BigIntVector) getValueVector()).setSafe(getCount(), (long) value.getField(ordinal)); } } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowBooleanWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowBooleanWriter.java similarity index 80% rename from flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowBooleanWriter.java rename to flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowBooleanWriter.java index ac338f50df..ccc1b7eed9 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowBooleanWriter.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowBooleanWriter.java @@ -19,7 +19,7 @@ package org.apache.flink.table.runtime.arrow.writers; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.dataformat.BaseRow; +import org.apache.flink.types.Row; import org.apache.arrow.vector.BitVector; @@ -27,17 +27,17 @@ import org.apache.arrow.vector.BitVector; * {@link ArrowFieldWriter} for Boolean. */ @Internal -public final class BaseRowBooleanWriter extends ArrowFieldWriter { +public final class RowBooleanWriter extends ArrowFieldWriter { - public BaseRowBooleanWriter(BitVector bitVector) { + public RowBooleanWriter(BitVector bitVector) { super(bitVector); } @Override - public void doWrite(BaseRow row, int ordinal) { - if (row.isNullAt(ordinal)) { + public void doWrite(Row value, int ordinal) { + if (value.getField(ordinal) == null) { ((BitVector) getValueVector()).setNull(getCount()); - } else if (row.getBoolean(ordinal)) { + } else if ((boolean) value.getField(ordinal)) { ((BitVector) getValueVector()).setSafe(getCount(), 1); } else { ((BitVector) getValueVector()).setSafe(getCount(), 0); diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowDateWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowDateWriter.java similarity index 71% rename from flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowDateWriter.java rename to flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowDateWriter.java index 1ce87590eb..adaa0ef0fe 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowDateWriter.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowDateWriter.java @@ -19,26 +19,30 @@ package org.apache.flink.table.runtime.arrow.writers; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.dataformat.BaseRow; +import org.apache.flink.table.runtime.typeutils.PythonTypeUtils; +import org.apache.flink.types.Row; import org.apache.arrow.vector.DateDayVector; +import java.sql.Date; + /** * {@link ArrowFieldWriter} for Date. */ @Internal -public final class BaseRowDateWriter extends ArrowFieldWriter { +public final class RowDateWriter extends ArrowFieldWriter { - public BaseRowDateWriter(DateDayVector dateDayVector) { + public RowDateWriter(DateDayVector dateDayVector) { super(dateDayVector); } @Override - public void doWrite(BaseRow row, int ordinal) { - if (row.isNullAt(ordinal)) { + public void doWrite(Row value, int ordinal) { + if (value.getField(ordinal) == null) { ((DateDayVector) getValueVector()).setNull(getCount()); } else { - ((DateDayVector) getValueVector()).setSafe(getCount(), row.getInt(ordinal)); + ((DateDayVector) getValueVector()).setSafe( + getCount(), PythonTypeUtils.dateToInternal(((Date) value.getField(ordinal)))); } } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowDecimalWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowDecimalWriter.java similarity index 80% rename from flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowDecimalWriter.java rename to flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowDecimalWriter.java index ee0ddad851..83edc78942 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowDecimalWriter.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowDecimalWriter.java @@ -19,7 +19,7 @@ package org.apache.flink.table.runtime.arrow.writers; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.dataformat.BaseRow; +import org.apache.flink.types.Row; import org.apache.arrow.vector.DecimalVector; @@ -31,23 +31,23 @@ import static org.apache.flink.table.runtime.typeutils.PythonTypeUtils.fromBigDe * {@link ArrowFieldWriter} for Decimal. */ @Internal -public final class BaseRowDecimalWriter extends ArrowFieldWriter { +public final class RowDecimalWriter extends ArrowFieldWriter { private final int precision; private final int scale; - public BaseRowDecimalWriter(DecimalVector decimalVector, int precision, int scale) { + public RowDecimalWriter(DecimalVector decimalVector, int precision, int scale) { super(decimalVector); this.precision = precision; this.scale = scale; } @Override - public void doWrite(BaseRow row, int ordinal) { - if (row.isNullAt(ordinal)) { + public void doWrite(Row value, int ordinal) { + if (value.getField(ordinal) == null) { ((DecimalVector) getValueVector()).setNull(getCount()); } else { - BigDecimal bigDecimal = row.getDecimal(ordinal, precision, scale).toBigDecimal(); + BigDecimal bigDecimal = (BigDecimal) value.getField(ordinal); bigDecimal = fromBigDecimal(bigDecimal, precision, scale); if (bigDecimal == null) { ((DecimalVector) getValueVector()).setNull(getCount()); diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowDoubleWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowDoubleWriter.java similarity index 76% rename from flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowDoubleWriter.java rename to flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowDoubleWriter.java index 4ead3f7758..d961df2845 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowDoubleWriter.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowDoubleWriter.java @@ -19,7 +19,7 @@ package org.apache.flink.table.runtime.arrow.writers; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.dataformat.BaseRow; +import org.apache.flink.types.Row; import org.apache.arrow.vector.Float8Vector; @@ -27,18 +27,18 @@ import org.apache.arrow.vector.Float8Vector; * {@link ArrowFieldWriter} for Double. */ @Internal -public final class BaseRowDoubleWriter extends ArrowFieldWriter { +public final class RowDoubleWriter extends ArrowFieldWriter { - public BaseRowDoubleWriter(Float8Vector doubleVector) { + public RowDoubleWriter(Float8Vector doubleVector) { super(doubleVector); } @Override - public void doWrite(BaseRow row, int ordinal) { - if (row.isNullAt(ordinal)) { + public void doWrite(Row value, int ordinal) { + if (value.getField(ordinal) == null) { ((Float8Vector) getValueVector()).setNull(getCount()); } else { - ((Float8Vector) getValueVector()).setSafe(getCount(), row.getDouble(ordinal)); + ((Float8Vector) getValueVector()).setSafe(getCount(), (double) value.getField(ordinal)); } } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowFloatWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowFloatWriter.java similarity index 76% rename from flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowFloatWriter.java rename to flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowFloatWriter.java index 7561176345..add4730b83 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowFloatWriter.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowFloatWriter.java @@ -19,7 +19,7 @@ package org.apache.flink.table.runtime.arrow.writers; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.dataformat.BaseRow; +import org.apache.flink.types.Row; import org.apache.arrow.vector.Float4Vector; @@ -27,18 +27,18 @@ import org.apache.arrow.vector.Float4Vector; * {@link ArrowFieldWriter} for Float. */ @Internal -public final class BaseRowFloatWriter extends ArrowFieldWriter { +public final class RowFloatWriter extends ArrowFieldWriter { - public BaseRowFloatWriter(Float4Vector floatVector) { + public RowFloatWriter(Float4Vector floatVector) { super(floatVector); } @Override - public void doWrite(BaseRow row, int ordinal) { - if (row.isNullAt(ordinal)) { + public void doWrite(Row value, int ordinal) { + if (value.getField(ordinal) == null) { ((Float4Vector) getValueVector()).setNull(getCount()); } else { - ((Float4Vector) getValueVector()).setSafe(getCount(), row.getFloat(ordinal)); + ((Float4Vector) getValueVector()).setSafe(getCount(), (float) value.getField(ordinal)); } } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowIntWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowIntWriter.java similarity index 77% rename from flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowIntWriter.java rename to flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowIntWriter.java index c01f0105c1..0fe6fcc342 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowIntWriter.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowIntWriter.java @@ -19,7 +19,7 @@ package org.apache.flink.table.runtime.arrow.writers; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.dataformat.BaseRow; +import org.apache.flink.types.Row; import org.apache.arrow.vector.IntVector; @@ -27,18 +27,18 @@ import org.apache.arrow.vector.IntVector; * {@link ArrowFieldWriter} for Int. */ @Internal -public final class BaseRowIntWriter extends ArrowFieldWriter { +public final class RowIntWriter extends ArrowFieldWriter { - public BaseRowIntWriter(IntVector intVector) { + public RowIntWriter(IntVector intVector) { super(intVector); } @Override - public void doWrite(BaseRow row, int ordinal) { - if (row.isNullAt(ordinal)) { + public void doWrite(Row value, int ordinal) { + if (value.getField(ordinal) == null) { ((IntVector) getValueVector()).setNull(getCount()); } else { - ((IntVector) getValueVector()).setSafe(getCount(), row.getInt(ordinal)); + ((IntVector) getValueVector()).setSafe(getCount(), (int) value.getField(ordinal)); } } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowSmallIntWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowSmallIntWriter.java similarity index 76% rename from flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowSmallIntWriter.java rename to flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowSmallIntWriter.java index 9a6a8686e6..a285d65958 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowSmallIntWriter.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowSmallIntWriter.java @@ -19,7 +19,7 @@ package org.apache.flink.table.runtime.arrow.writers; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.dataformat.BaseRow; +import org.apache.flink.types.Row; import org.apache.arrow.vector.SmallIntVector; @@ -27,18 +27,18 @@ import org.apache.arrow.vector.SmallIntVector; * {@link ArrowFieldWriter} for SmallInt. */ @Internal -public final class BaseRowSmallIntWriter extends ArrowFieldWriter { +public final class RowSmallIntWriter extends ArrowFieldWriter { - public BaseRowSmallIntWriter(SmallIntVector smallIntVector) { + public RowSmallIntWriter(SmallIntVector smallIntVector) { super(smallIntVector); } @Override - public void doWrite(BaseRow row, int ordinal) { - if (row.isNullAt(ordinal)) { + public void doWrite(Row value, int ordinal) { + if (value.getField(ordinal) == null) { ((SmallIntVector) getValueVector()).setNull(getCount()); } else { - ((SmallIntVector) getValueVector()).setSafe(getCount(), row.getShort(ordinal)); + ((SmallIntVector) getValueVector()).setSafe(getCount(), (short) value.getField(ordinal)); } } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowTimeWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowTimeWriter.java similarity index 58% rename from flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowTimeWriter.java rename to flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowTimeWriter.java index 1ccdc4948a..1791434cff 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowTimeWriter.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowTimeWriter.java @@ -19,7 +19,7 @@ package org.apache.flink.table.runtime.arrow.writers; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.dataformat.BaseRow; +import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; import org.apache.arrow.vector.BaseFixedWidthVector; @@ -29,13 +29,21 @@ import org.apache.arrow.vector.TimeNanoVector; import org.apache.arrow.vector.TimeSecVector; import org.apache.arrow.vector.ValueVector; +import java.sql.Time; +import java.util.TimeZone; + /** * {@link ArrowFieldWriter} for Time. */ @Internal -public final class BaseRowTimeWriter extends ArrowFieldWriter { +public final class RowTimeWriter extends ArrowFieldWriter { + + // The local time zone. + private static final TimeZone LOCAL_TZ = TimeZone.getDefault(); - public BaseRowTimeWriter(ValueVector valueVector) { + private static final long MILLIS_PER_DAY = 86400000L; // = 24 * 60 * 60 * 1000 + + public RowTimeWriter(ValueVector valueVector) { super(valueVector); Preconditions.checkState( valueVector instanceof TimeSecVector || @@ -45,18 +53,24 @@ public final class BaseRowTimeWriter extends ArrowFieldWriter { } @Override - public void doWrite(BaseRow row, int ordinal) { + public void doWrite(Row row, int ordinal) { ValueVector valueVector = getValueVector(); - if (row.isNullAt(ordinal)) { - ((BaseFixedWidthVector) valueVector).setNull(getCount()); - } else if (valueVector instanceof TimeSecVector) { - ((TimeSecVector) valueVector).setSafe(getCount(), row.getInt(ordinal) / 1000); - } else if (valueVector instanceof TimeMilliVector) { - ((TimeMilliVector) valueVector).setSafe(getCount(), row.getInt(ordinal)); - } else if (valueVector instanceof TimeMicroVector) { - ((TimeMicroVector) valueVector).setSafe(getCount(), row.getInt(ordinal) * 1000L); + if (row.getField(ordinal) == null) { + ((BaseFixedWidthVector) getValueVector()).setNull(getCount()); } else { - ((TimeNanoVector) valueVector).setSafe(getCount(), row.getInt(ordinal) * 1000000L); + Time time = (Time) row.getField(ordinal); + long ts = time.getTime() + LOCAL_TZ.getOffset(time.getTime()); + int timeMilli = (int) (ts % MILLIS_PER_DAY); + + if (valueVector instanceof TimeSecVector) { + ((TimeSecVector) valueVector).setSafe(getCount(), timeMilli / 1000); + } else if (valueVector instanceof TimeMilliVector) { + ((TimeMilliVector) valueVector).setSafe(getCount(), timeMilli); + } else if (valueVector instanceof TimeMicroVector) { + ((TimeMicroVector) valueVector).setSafe(getCount(), timeMilli * 1000L); + } else { + ((TimeNanoVector) valueVector).setSafe(getCount(), timeMilli * 1000000L); + } } } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowTimestampWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowTimestampWriter.java similarity index 64% rename from flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowTimestampWriter.java rename to flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowTimestampWriter.java index 41d41bd5e1..0e800805be 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowTimestampWriter.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowTimestampWriter.java @@ -19,10 +19,11 @@ package org.apache.flink.table.runtime.arrow.writers; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.dataformat.BaseRow; -import org.apache.flink.table.dataformat.SqlTimestamp; +import org.apache.flink.table.runtime.typeutils.PythonTypeUtils; +import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; +import org.apache.arrow.vector.BaseFixedWidthVector; import org.apache.arrow.vector.TimeStampMicroVector; import org.apache.arrow.vector.TimeStampMilliVector; import org.apache.arrow.vector.TimeStampNanoVector; @@ -31,36 +32,34 @@ import org.apache.arrow.vector.TimeStampVector; import org.apache.arrow.vector.ValueVector; import org.apache.arrow.vector.types.pojo.ArrowType; +import java.sql.Timestamp; + /** * {@link ArrowFieldWriter} for Timestamp. */ @Internal -public final class BaseRowTimestampWriter extends ArrowFieldWriter { - - private final int precision; +public final class RowTimestampWriter extends ArrowFieldWriter { - public BaseRowTimestampWriter(ValueVector valueVector, int precision) { + public RowTimestampWriter(ValueVector valueVector) { super(valueVector); Preconditions.checkState(valueVector instanceof TimeStampVector && ((ArrowType.Timestamp) valueVector.getField().getType()).getTimezone() == null); - this.precision = precision; } @Override - public void doWrite(BaseRow row, int ordinal) { + public void doWrite(Row row, int ordinal) { ValueVector valueVector = getValueVector(); - if (row.isNullAt(ordinal)) { - ((TimeStampVector) valueVector).setNull(getCount()); + if (row.getField(ordinal) == null) { + ((BaseFixedWidthVector) getValueVector()).setNull(getCount()); } else { - SqlTimestamp sqlTimestamp = row.getTimestamp(ordinal, precision); - + long millisecond = PythonTypeUtils.timestampToInternal((Timestamp) row.getField(ordinal)); if (valueVector instanceof TimeStampSecVector) { - ((TimeStampSecVector) valueVector).setSafe(getCount(), sqlTimestamp.getMillisecond() / 1000); + ((TimeStampSecVector) valueVector).setSafe(getCount(), millisecond / 1000); } else if (valueVector instanceof TimeStampMilliVector) { - ((TimeStampMilliVector) valueVector).setSafe(getCount(), sqlTimestamp.getMillisecond()); + ((TimeStampMilliVector) valueVector).setSafe(getCount(), millisecond); } else if (valueVector instanceof TimeStampMicroVector) { - ((TimeStampMicroVector) valueVector).setSafe(getCount(), sqlTimestamp.getMillisecond() * 1000 + sqlTimestamp.getNanoOfMillisecond() / 1000); + ((TimeStampMicroVector) valueVector).setSafe(getCount(), millisecond * 1000); } else { - ((TimeStampNanoVector) valueVector).setSafe(getCount(), sqlTimestamp.getMillisecond() * 1_000_000 + sqlTimestamp.getNanoOfMillisecond()); + ((TimeStampNanoVector) valueVector).setSafe(getCount(), millisecond * 1_000_000); } } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowTinyIntWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowTinyIntWriter.java similarity index 76% rename from flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowTinyIntWriter.java rename to flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowTinyIntWriter.java index c71bd8e4fe..889d289e38 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowTinyIntWriter.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowTinyIntWriter.java @@ -19,7 +19,7 @@ package org.apache.flink.table.runtime.arrow.writers; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.dataformat.BaseRow; +import org.apache.flink.types.Row; import org.apache.arrow.vector.TinyIntVector; @@ -27,18 +27,18 @@ import org.apache.arrow.vector.TinyIntVector; * {@link ArrowFieldWriter} for TinyInt. */ @Internal -public final class BaseRowTinyIntWriter extends ArrowFieldWriter { +public final class RowTinyIntWriter extends ArrowFieldWriter { - public BaseRowTinyIntWriter(TinyIntVector tinyIntVector) { + public RowTinyIntWriter(TinyIntVector tinyIntVector) { super(tinyIntVector); } @Override - public void doWrite(BaseRow row, int ordinal) { - if (row.isNullAt(ordinal)) { + public void doWrite(Row value, int ordinal) { + if (value.getField(ordinal) == null) { ((TinyIntVector) getValueVector()).setNull(getCount()); } else { - ((TinyIntVector) getValueVector()).setSafe(getCount(), row.getByte(ordinal)); + ((TinyIntVector) getValueVector()).setSafe(getCount(), (byte) value.getField(ordinal)); } } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowVarBinaryWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowVarBinaryWriter.java similarity index 76% rename from flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowVarBinaryWriter.java rename to flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowVarBinaryWriter.java index 2c33ac6ffa..2ba9ad8554 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowVarBinaryWriter.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowVarBinaryWriter.java @@ -19,7 +19,7 @@ package org.apache.flink.table.runtime.arrow.writers; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.dataformat.BaseRow; +import org.apache.flink.types.Row; import org.apache.arrow.vector.VarBinaryVector; @@ -27,18 +27,18 @@ import org.apache.arrow.vector.VarBinaryVector; * {@link ArrowFieldWriter} for VarBinary. */ @Internal -public final class BaseRowVarBinaryWriter extends ArrowFieldWriter { +public final class RowVarBinaryWriter extends ArrowFieldWriter { - public BaseRowVarBinaryWriter(VarBinaryVector varBinaryVector) { + public RowVarBinaryWriter(VarBinaryVector varBinaryVector) { super(varBinaryVector); } @Override - public void doWrite(BaseRow row, int ordinal) { - if (row.isNullAt(ordinal)) { + public void doWrite(Row value, int ordinal) { + if (value.getField(ordinal) == null) { ((VarBinaryVector) getValueVector()).setNull(getCount()); } else { - ((VarBinaryVector) getValueVector()).setSafe(getCount(), row.getBinary(ordinal)); + ((VarBinaryVector) getValueVector()).setSafe(getCount(), (byte[]) value.getField(ordinal)); } } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowVarCharWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowVarCharWriter.java similarity index 72% rename from flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowVarCharWriter.java rename to flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowVarCharWriter.java index 915e1869d8..49bb395241 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/BaseRowVarCharWriter.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/RowVarCharWriter.java @@ -19,7 +19,8 @@ package org.apache.flink.table.runtime.arrow.writers; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.dataformat.BaseRow; +import org.apache.flink.table.runtime.util.StringUtf8Utils; +import org.apache.flink.types.Row; import org.apache.arrow.vector.VarCharVector; @@ -27,18 +28,19 @@ import org.apache.arrow.vector.VarCharVector; * {@link ArrowFieldWriter} for VarChar. */ @Internal -public final class BaseRowVarCharWriter extends ArrowFieldWriter { +public final class RowVarCharWriter extends ArrowFieldWriter { - public BaseRowVarCharWriter(VarCharVector varCharVector) { + public RowVarCharWriter(VarCharVector varCharVector) { super(varCharVector); } @Override - public void doWrite(BaseRow row, int ordinal) { - if (row.isNullAt(ordinal)) { + public void doWrite(Row value, int ordinal) { + if (value.getField(ordinal) == null) { ((VarCharVector) getValueVector()).setNull(getCount()); } else { - ((VarCharVector) getValueVector()).setSafe(getCount(), row.getString(ordinal).getBytes()); + ((VarCharVector) getValueVector()).setSafe( + getCount(), StringUtf8Utils.encodeUTF8(((String) value.getField(ordinal)))); } } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/SmallIntWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/SmallIntWriter.java index ae38e1594c..e1a7a019a9 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/SmallIntWriter.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/SmallIntWriter.java @@ -19,7 +19,7 @@ package org.apache.flink.table.runtime.arrow.writers; import org.apache.flink.annotation.Internal; -import org.apache.flink.types.Row; +import org.apache.flink.table.dataformat.TypeGetterSetters; import org.apache.arrow.vector.SmallIntVector; @@ -27,18 +27,18 @@ import org.apache.arrow.vector.SmallIntVector; * {@link ArrowFieldWriter} for SmallInt. */ @Internal -public final class SmallIntWriter extends ArrowFieldWriter { +public final class SmallIntWriter extends ArrowFieldWriter { public SmallIntWriter(SmallIntVector smallIntVector) { super(smallIntVector); } @Override - public void doWrite(Row value, int ordinal) { - if (value.getField(ordinal) == null) { + public void doWrite(T row, int ordinal) { + if (row.isNullAt(ordinal)) { ((SmallIntVector) getValueVector()).setNull(getCount()); } else { - ((SmallIntVector) getValueVector()).setSafe(getCount(), (short) value.getField(ordinal)); + ((SmallIntVector) getValueVector()).setSafe(getCount(), row.getShort(ordinal)); } } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/TimeWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/TimeWriter.java index 0356d2076d..ece3e0a429 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/TimeWriter.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/TimeWriter.java @@ -19,7 +19,7 @@ package org.apache.flink.table.runtime.arrow.writers; import org.apache.flink.annotation.Internal; -import org.apache.flink.types.Row; +import org.apache.flink.table.dataformat.TypeGetterSetters; import org.apache.flink.util.Preconditions; import org.apache.arrow.vector.BaseFixedWidthVector; @@ -29,19 +29,11 @@ import org.apache.arrow.vector.TimeNanoVector; import org.apache.arrow.vector.TimeSecVector; import org.apache.arrow.vector.ValueVector; -import java.sql.Time; -import java.util.TimeZone; - /** * {@link ArrowFieldWriter} for Time. */ @Internal -public final class TimeWriter extends ArrowFieldWriter { - - // The local time zone. - private static final TimeZone LOCAL_TZ = TimeZone.getDefault(); - - private static final long MILLIS_PER_DAY = 86400000L; // = 24 * 60 * 60 * 1000 +public final class TimeWriter extends ArrowFieldWriter { public TimeWriter(ValueVector valueVector) { super(valueVector); @@ -53,24 +45,18 @@ public final class TimeWriter extends ArrowFieldWriter { } @Override - public void doWrite(Row row, int ordinal) { + public void doWrite(T row, int ordinal) { ValueVector valueVector = getValueVector(); - if (row.getField(ordinal) == null) { - ((BaseFixedWidthVector) getValueVector()).setNull(getCount()); + if (row.isNullAt(ordinal)) { + ((BaseFixedWidthVector) valueVector).setNull(getCount()); + } else if (valueVector instanceof TimeSecVector) { + ((TimeSecVector) valueVector).setSafe(getCount(), row.getInt(ordinal) / 1000); + } else if (valueVector instanceof TimeMilliVector) { + ((TimeMilliVector) valueVector).setSafe(getCount(), row.getInt(ordinal)); + } else if (valueVector instanceof TimeMicroVector) { + ((TimeMicroVector) valueVector).setSafe(getCount(), row.getInt(ordinal) * 1000L); } else { - Time time = (Time) row.getField(ordinal); - long ts = time.getTime() + LOCAL_TZ.getOffset(time.getTime()); - int timeMilli = (int) (ts % MILLIS_PER_DAY); - - if (valueVector instanceof TimeSecVector) { - ((TimeSecVector) valueVector).setSafe(getCount(), timeMilli / 1000); - } else if (valueVector instanceof TimeMilliVector) { - ((TimeMilliVector) valueVector).setSafe(getCount(), timeMilli); - } else if (valueVector instanceof TimeMicroVector) { - ((TimeMicroVector) valueVector).setSafe(getCount(), timeMilli * 1000L); - } else { - ((TimeNanoVector) valueVector).setSafe(getCount(), timeMilli * 1000000L); - } + ((TimeNanoVector) valueVector).setSafe(getCount(), row.getInt(ordinal) * 1000000L); } } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/TimestampWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/TimestampWriter.java index 1f2e895a9d..ac51a6d482 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/TimestampWriter.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/TimestampWriter.java @@ -19,11 +19,10 @@ package org.apache.flink.table.runtime.arrow.writers; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.runtime.typeutils.PythonTypeUtils; -import org.apache.flink.types.Row; +import org.apache.flink.table.dataformat.SqlTimestamp; +import org.apache.flink.table.dataformat.TypeGetterSetters; import org.apache.flink.util.Preconditions; -import org.apache.arrow.vector.BaseFixedWidthVector; import org.apache.arrow.vector.TimeStampMicroVector; import org.apache.arrow.vector.TimeStampMilliVector; import org.apache.arrow.vector.TimeStampNanoVector; @@ -32,34 +31,36 @@ import org.apache.arrow.vector.TimeStampVector; import org.apache.arrow.vector.ValueVector; import org.apache.arrow.vector.types.pojo.ArrowType; -import java.sql.Timestamp; - /** * {@link ArrowFieldWriter} for Timestamp. */ @Internal -public final class TimestampWriter extends ArrowFieldWriter { +public final class TimestampWriter extends ArrowFieldWriter { + + private final int precision; - public TimestampWriter(ValueVector valueVector) { + public TimestampWriter(ValueVector valueVector, int precision) { super(valueVector); Preconditions.checkState(valueVector instanceof TimeStampVector && ((ArrowType.Timestamp) valueVector.getField().getType()).getTimezone() == null); + this.precision = precision; } @Override - public void doWrite(Row row, int ordinal) { + public void doWrite(T row, int ordinal) { ValueVector valueVector = getValueVector(); - if (row.getField(ordinal) == null) { - ((BaseFixedWidthVector) getValueVector()).setNull(getCount()); + if (row.isNullAt(ordinal)) { + ((TimeStampVector) valueVector).setNull(getCount()); } else { - long millisecond = PythonTypeUtils.timestampToInternal((Timestamp) row.getField(ordinal)); + SqlTimestamp sqlTimestamp = row.getTimestamp(ordinal, precision); + if (valueVector instanceof TimeStampSecVector) { - ((TimeStampSecVector) valueVector).setSafe(getCount(), millisecond / 1000); + ((TimeStampSecVector) valueVector).setSafe(getCount(), sqlTimestamp.getMillisecond() / 1000); } else if (valueVector instanceof TimeStampMilliVector) { - ((TimeStampMilliVector) valueVector).setSafe(getCount(), millisecond); + ((TimeStampMilliVector) valueVector).setSafe(getCount(), sqlTimestamp.getMillisecond()); } else if (valueVector instanceof TimeStampMicroVector) { - ((TimeStampMicroVector) valueVector).setSafe(getCount(), millisecond * 1000); + ((TimeStampMicroVector) valueVector).setSafe(getCount(), sqlTimestamp.getMillisecond() * 1000 + sqlTimestamp.getNanoOfMillisecond() / 1000); } else { - ((TimeStampNanoVector) valueVector).setSafe(getCount(), millisecond * 1_000_000); + ((TimeStampNanoVector) valueVector).setSafe(getCount(), sqlTimestamp.getMillisecond() * 1_000_000 + sqlTimestamp.getNanoOfMillisecond()); } } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/TinyIntWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/TinyIntWriter.java index 7e4bb216a8..ea9c045ef2 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/TinyIntWriter.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/TinyIntWriter.java @@ -19,7 +19,7 @@ package org.apache.flink.table.runtime.arrow.writers; import org.apache.flink.annotation.Internal; -import org.apache.flink.types.Row; +import org.apache.flink.table.dataformat.TypeGetterSetters; import org.apache.arrow.vector.TinyIntVector; @@ -27,18 +27,18 @@ import org.apache.arrow.vector.TinyIntVector; * {@link ArrowFieldWriter} for TinyInt. */ @Internal -public final class TinyIntWriter extends ArrowFieldWriter { +public final class TinyIntWriter extends ArrowFieldWriter { public TinyIntWriter(TinyIntVector tinyIntVector) { super(tinyIntVector); } @Override - public void doWrite(Row value, int ordinal) { - if (value.getField(ordinal) == null) { + public void doWrite(T row, int ordinal) { + if (row.isNullAt(ordinal)) { ((TinyIntVector) getValueVector()).setNull(getCount()); } else { - ((TinyIntVector) getValueVector()).setSafe(getCount(), (byte) value.getField(ordinal)); + ((TinyIntVector) getValueVector()).setSafe(getCount(), row.getByte(ordinal)); } } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/VarBinaryWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/VarBinaryWriter.java index 72042e533c..d694ddd1ad 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/VarBinaryWriter.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/VarBinaryWriter.java @@ -19,7 +19,7 @@ package org.apache.flink.table.runtime.arrow.writers; import org.apache.flink.annotation.Internal; -import org.apache.flink.types.Row; +import org.apache.flink.table.dataformat.TypeGetterSetters; import org.apache.arrow.vector.VarBinaryVector; @@ -27,18 +27,18 @@ import org.apache.arrow.vector.VarBinaryVector; * {@link ArrowFieldWriter} for VarBinary. */ @Internal -public final class VarBinaryWriter extends ArrowFieldWriter { +public final class VarBinaryWriter extends ArrowFieldWriter { public VarBinaryWriter(VarBinaryVector varBinaryVector) { super(varBinaryVector); } @Override - public void doWrite(Row value, int ordinal) { - if (value.getField(ordinal) == null) { + public void doWrite(T row, int ordinal) { + if (row.isNullAt(ordinal)) { ((VarBinaryVector) getValueVector()).setNull(getCount()); } else { - ((VarBinaryVector) getValueVector()).setSafe(getCount(), (byte[]) value.getField(ordinal)); + ((VarBinaryVector) getValueVector()).setSafe(getCount(), row.getBinary(ordinal)); } } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/VarCharWriter.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/VarCharWriter.java index fcd3457d8b..e690983052 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/VarCharWriter.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/writers/VarCharWriter.java @@ -19,8 +19,7 @@ package org.apache.flink.table.runtime.arrow.writers; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.runtime.util.StringUtf8Utils; -import org.apache.flink.types.Row; +import org.apache.flink.table.dataformat.TypeGetterSetters; import org.apache.arrow.vector.VarCharVector; @@ -28,19 +27,18 @@ import org.apache.arrow.vector.VarCharVector; * {@link ArrowFieldWriter} for VarChar. */ @Internal -public final class VarCharWriter extends ArrowFieldWriter { +public final class VarCharWriter extends ArrowFieldWriter { public VarCharWriter(VarCharVector varCharVector) { super(varCharVector); } @Override - public void doWrite(Row value, int ordinal) { - if (value.getField(ordinal) == null) { + public void doWrite(T row, int ordinal) { + if (row.isNullAt(ordinal)) { ((VarCharVector) getValueVector()).setNull(getCount()); } else { - ((VarCharVector) getValueVector()).setSafe( - getCount(), StringUtf8Utils.encodeUTF8(((String) value.getField(ordinal)))); + ((VarCharVector) getValueVector()).setSafe(getCount(), row.getString(ordinal).getBytes()); } } } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java index 8190d8dfd9..944c1f0a48 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/ArrowUtilsTest.java @@ -21,6 +21,7 @@ package org.apache.flink.table.runtime.arrow; import org.apache.flink.api.java.tuple.Tuple7; import org.apache.flink.table.dataformat.BaseRow; import org.apache.flink.table.dataformat.vector.ColumnVector; +import org.apache.flink.table.runtime.arrow.readers.ArrayFieldReader; import org.apache.flink.table.runtime.arrow.readers.ArrowFieldReader; import org.apache.flink.table.runtime.arrow.readers.BigIntFieldReader; import org.apache.flink.table.runtime.arrow.readers.BooleanFieldReader; @@ -36,6 +37,7 @@ import org.apache.flink.table.runtime.arrow.readers.TimestampFieldReader; import org.apache.flink.table.runtime.arrow.readers.TinyIntFieldReader; import org.apache.flink.table.runtime.arrow.readers.VarBinaryFieldReader; import org.apache.flink.table.runtime.arrow.readers.VarCharFieldReader; +import org.apache.flink.table.runtime.arrow.vectors.ArrowArrayColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowBigIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowBooleanColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowDateColumnVector; @@ -50,20 +52,8 @@ import org.apache.flink.table.runtime.arrow.vectors.ArrowTinyIntColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowVarBinaryColumnVector; import org.apache.flink.table.runtime.arrow.vectors.ArrowVarCharColumnVector; import org.apache.flink.table.runtime.arrow.vectors.BaseRowArrowReader; +import org.apache.flink.table.runtime.arrow.writers.ArrayWriter; import org.apache.flink.table.runtime.arrow.writers.ArrowFieldWriter; -import org.apache.flink.table.runtime.arrow.writers.BaseRowBigIntWriter; -import org.apache.flink.table.runtime.arrow.writers.BaseRowBooleanWriter; -import org.apache.flink.table.runtime.arrow.writers.BaseRowDateWriter; -import org.apache.flink.table.runtime.arrow.writers.BaseRowDecimalWriter; -import org.apache.flink.table.runtime.arrow.writers.BaseRowDoubleWriter; -import org.apache.flink.table.runtime.arrow.writers.BaseRowFloatWriter; -import org.apache.flink.table.runtime.arrow.writers.BaseRowIntWriter; -import org.apache.flink.table.runtime.arrow.writers.BaseRowSmallIntWriter; -import org.apache.flink.table.runtime.arrow.writers.BaseRowTimeWriter; -import org.apache.flink.table.runtime.arrow.writers.BaseRowTimestampWriter; -import org.apache.flink.table.runtime.arrow.writers.BaseRowTinyIntWriter; -import org.apache.flink.table.runtime.arrow.writers.BaseRowVarBinaryWriter; -import org.apache.flink.table.runtime.arrow.writers.BaseRowVarCharWriter; import org.apache.flink.table.runtime.arrow.writers.BigIntWriter; import org.apache.flink.table.runtime.arrow.writers.BooleanWriter; import org.apache.flink.table.runtime.arrow.writers.DateWriter; @@ -71,12 +61,27 @@ import org.apache.flink.table.runtime.arrow.writers.DecimalWriter; import org.apache.flink.table.runtime.arrow.writers.DoubleWriter; import org.apache.flink.table.runtime.arrow.writers.FloatWriter; import org.apache.flink.table.runtime.arrow.writers.IntWriter; +import org.apache.flink.table.runtime.arrow.writers.RowArrayWriter; +import org.apache.flink.table.runtime.arrow.writers.RowBigIntWriter; +import org.apache.flink.table.runtime.arrow.writers.RowBooleanWriter; +import org.apache.flink.table.runtime.arrow.writers.RowDateWriter; +import org.apache.flink.table.runtime.arrow.writers.RowDecimalWriter; +import org.apache.flink.table.runtime.arrow.writers.RowDoubleWriter; +import org.apache.flink.table.runtime.arrow.writers.RowFloatWriter; +import org.apache.flink.table.runtime.arrow.writers.RowIntWriter; +import org.apache.flink.table.runtime.arrow.writers.RowSmallIntWriter; +import org.apache.flink.table.runtime.arrow.writers.RowTimeWriter; +import org.apache.flink.table.runtime.arrow.writers.RowTimestampWriter; +import org.apache.flink.table.runtime.arrow.writers.RowTinyIntWriter; +import org.apache.flink.table.runtime.arrow.writers.RowVarBinaryWriter; +import org.apache.flink.table.runtime.arrow.writers.RowVarCharWriter; import org.apache.flink.table.runtime.arrow.writers.SmallIntWriter; import org.apache.flink.table.runtime.arrow.writers.TimeWriter; import org.apache.flink.table.runtime.arrow.writers.TimestampWriter; import org.apache.flink.table.runtime.arrow.writers.TinyIntWriter; import org.apache.flink.table.runtime.arrow.writers.VarBinaryWriter; import org.apache.flink.table.runtime.arrow.writers.VarCharWriter; +import org.apache.flink.table.types.logical.ArrayType; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BooleanType; import org.apache.flink.table.types.logical.DateType; @@ -124,52 +129,54 @@ public class ArrowUtilsTest { public static void init() { testFields = new ArrayList<>(); testFields.add(Tuple7.of( - "f1", new TinyIntType(), new ArrowType.Int(8, true), TinyIntWriter.class, - BaseRowTinyIntWriter.class, TinyIntFieldReader.class, ArrowTinyIntColumnVector.class)); + "f1", new TinyIntType(), new ArrowType.Int(8, true), RowTinyIntWriter.class, + TinyIntWriter.class, TinyIntFieldReader.class, ArrowTinyIntColumnVector.class)); testFields.add(Tuple7.of("f2", new SmallIntType(), new ArrowType.Int(8 * 2, true), - SmallIntWriter.class, BaseRowSmallIntWriter.class, SmallIntFieldReader.class, ArrowSmallIntColumnVector.class)); + RowSmallIntWriter.class, SmallIntWriter.class, SmallIntFieldReader.class, ArrowSmallIntColumnVector.class)); testFields.add(Tuple7.of("f3", new IntType(), new ArrowType.Int(8 * 4, true), - IntWriter.class, BaseRowIntWriter.class, IntFieldReader.class, ArrowIntColumnVector.class)); + RowIntWriter.class, IntWriter.class, IntFieldReader.class, ArrowIntColumnVector.class)); testFields.add(Tuple7.of("f4", new BigIntType(), new ArrowType.Int(8 * 8, true), - BigIntWriter.class, BaseRowBigIntWriter.class, BigIntFieldReader.class, ArrowBigIntColumnVector.class)); + RowBigIntWriter.class, BigIntWriter.class, BigIntFieldReader.class, ArrowBigIntColumnVector.class)); testFields.add(Tuple7.of("f5", new BooleanType(), new ArrowType.Bool(), - BooleanWriter.class, BaseRowBooleanWriter.class, BooleanFieldReader.class, ArrowBooleanColumnVector.class)); + RowBooleanWriter.class, BooleanWriter.class, BooleanFieldReader.class, ArrowBooleanColumnVector.class)); testFields.add(Tuple7.of("f6", new FloatType(), new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE), - FloatWriter.class, BaseRowFloatWriter.class, FloatFieldReader.class, ArrowFloatColumnVector.class)); + RowFloatWriter.class, FloatWriter.class, FloatFieldReader.class, ArrowFloatColumnVector.class)); testFields.add(Tuple7.of("f7", new DoubleType(), new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE), - DoubleWriter.class, BaseRowDoubleWriter.class, DoubleFieldReader.class, ArrowDoubleColumnVector.class)); + RowDoubleWriter.class, DoubleWriter.class, DoubleFieldReader.class, ArrowDoubleColumnVector.class)); testFields.add(Tuple7.of("f8", new VarCharType(), ArrowType.Utf8.INSTANCE, - VarCharWriter.class, BaseRowVarCharWriter.class, VarCharFieldReader.class, ArrowVarCharColumnVector.class)); + RowVarCharWriter.class, VarCharWriter.class, VarCharFieldReader.class, ArrowVarCharColumnVector.class)); testFields.add(Tuple7.of("f9", new VarBinaryType(), ArrowType.Binary.INSTANCE, - VarBinaryWriter.class, BaseRowVarBinaryWriter.class, VarBinaryFieldReader.class, ArrowVarBinaryColumnVector.class)); + RowVarBinaryWriter.class, VarBinaryWriter.class, VarBinaryFieldReader.class, ArrowVarBinaryColumnVector.class)); testFields.add(Tuple7.of("f10", new DecimalType(10, 3), new ArrowType.Decimal(10, 3), - DecimalWriter.class, BaseRowDecimalWriter.class, DecimalFieldReader.class, ArrowDecimalColumnVector.class)); + RowDecimalWriter.class, DecimalWriter.class, DecimalFieldReader.class, ArrowDecimalColumnVector.class)); testFields.add(Tuple7.of("f11", new DateType(), new ArrowType.Date(DateUnit.DAY), - DateWriter.class, BaseRowDateWriter.class, DateFieldReader.class, ArrowDateColumnVector.class)); + RowDateWriter.class, DateWriter.class, DateFieldReader.class, ArrowDateColumnVector.class)); testFields.add(Tuple7.of("f13", new TimeType(0), new ArrowType.Time(TimeUnit.SECOND, 32), - TimeWriter.class, BaseRowTimeWriter.class, TimeFieldReader.class, ArrowTimeColumnVector.class)); + RowTimeWriter.class, TimeWriter.class, TimeFieldReader.class, ArrowTimeColumnVector.class)); testFields.add(Tuple7.of("f14", new TimeType(2), new ArrowType.Time(TimeUnit.MILLISECOND, 32), - TimeWriter.class, BaseRowTimeWriter.class, TimeFieldReader.class, ArrowTimeColumnVector.class)); + RowTimeWriter.class, TimeWriter.class, TimeFieldReader.class, ArrowTimeColumnVector.class)); testFields.add(Tuple7.of("f15", new TimeType(4), new ArrowType.Time(TimeUnit.MICROSECOND, 64), - TimeWriter.class, BaseRowTimeWriter.class, TimeFieldReader.class, ArrowTimeColumnVector.class)); + RowTimeWriter.class, TimeWriter.class, TimeFieldReader.class, ArrowTimeColumnVector.class)); testFields.add(Tuple7.of("f16", new TimeType(8), new ArrowType.Time(TimeUnit.NANOSECOND, 64), - TimeWriter.class, BaseRowTimeWriter.class, TimeFieldReader.class, ArrowTimeColumnVector.class)); + RowTimeWriter.class, TimeWriter.class, TimeFieldReader.class, ArrowTimeColumnVector.class)); testFields.add(Tuple7.of("f17", new LocalZonedTimestampType(0), new ArrowType.Timestamp(TimeUnit.SECOND, null), - TimestampWriter.class, BaseRowTimestampWriter.class, TimestampFieldReader.class, ArrowTimestampColumnVector.class)); + RowTimestampWriter.class, TimestampWriter.class, TimestampFieldReader.class, ArrowTimestampColumnVector.class)); testFields.add(Tuple7.of("f18", new LocalZonedTimestampType(2), new ArrowType.Timestamp(TimeUnit.MILLISECOND, null), - TimestampWriter.class, BaseRowTimestampWriter.class, TimestampFieldReader.class, ArrowTimestampColumnVector.class)); + RowTimestampWriter.class, TimestampWriter.class, TimestampFieldReader.class, ArrowTimestampColumnVector.class)); testFields.add(Tuple7.of("f19", new LocalZonedTimestampType(4), new ArrowType.Timestamp(TimeUnit.MICROSECOND, null), - TimestampWriter.class, BaseRowTimestampWriter.class, TimestampFieldReader.class, ArrowTimestampColumnVector.class)); + RowTimestampWriter.class, TimestampWriter.class, TimestampFieldReader.class, ArrowTimestampColumnVector.class)); testFields.add(Tuple7.of("f20", new LocalZonedTimestampType(8), new ArrowType.Timestamp(TimeUnit.NANOSECOND, null), - TimestampWriter.class, BaseRowTimestampWriter.class, TimestampFieldReader.class, ArrowTimestampColumnVector.class)); + RowTimestampWriter.class, TimestampWriter.class, TimestampFieldReader.class, ArrowTimestampColumnVector.class)); testFields.add(Tuple7.of("f21", new TimestampType(0), new ArrowType.Timestamp(TimeUnit.SECOND, null), - TimestampWriter.class, BaseRowTimestampWriter.class, TimestampFieldReader.class, ArrowTimestampColumnVector.class)); + RowTimestampWriter.class, TimestampWriter.class, TimestampFieldReader.class, ArrowTimestampColumnVector.class)); testFields.add(Tuple7.of("f22", new TimestampType(2), new ArrowType.Timestamp(TimeUnit.MILLISECOND, null), - TimestampWriter.class, BaseRowTimestampWriter.class, TimestampFieldReader.class, ArrowTimestampColumnVector.class)); + RowTimestampWriter.class, TimestampWriter.class, TimestampFieldReader.class, ArrowTimestampColumnVector.class)); testFields.add(Tuple7.of("f23", new TimestampType(4), new ArrowType.Timestamp(TimeUnit.MICROSECOND, null), - TimestampWriter.class, BaseRowTimestampWriter.class, TimestampFieldReader.class, ArrowTimestampColumnVector.class)); + RowTimestampWriter.class, TimestampWriter.class, TimestampFieldReader.class, ArrowTimestampColumnVector.class)); testFields.add(Tuple7.of("f24", new TimestampType(8), new ArrowType.Timestamp(TimeUnit.NANOSECOND, null), - TimestampWriter.class, BaseRowTimestampWriter.class, TimestampFieldReader.class, ArrowTimestampColumnVector.class)); + RowTimestampWriter.class, TimestampWriter.class, TimestampFieldReader.class, ArrowTimestampColumnVector.class)); + testFields.add(Tuple7.of("f25", new ArrayType(new VarCharType()), ArrowType.List.INSTANCE, + RowArrayWriter.class, ArrayWriter.class, ArrayFieldReader.class, ArrowArrayColumnVector.class)); List rowFields = new ArrayList<>(); for (Tuple7, Class, Class, Class> field : testFields) { diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java index f0428d7344..76b00f8621 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/arrow/BaseRowArrowReaderWriterTest.java @@ -22,10 +22,14 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.table.dataformat.BaseRow; import org.apache.flink.table.dataformat.BinaryRow; +import org.apache.flink.table.dataformat.BinaryString; import org.apache.flink.table.dataformat.Decimal; +import org.apache.flink.table.dataformat.GenericArray; import org.apache.flink.table.dataformat.SqlTimestamp; +import org.apache.flink.table.runtime.typeutils.BaseArraySerializer; import org.apache.flink.table.runtime.typeutils.BaseRowSerializer; import org.apache.flink.table.runtime.util.StreamRecordUtils; +import org.apache.flink.table.types.logical.ArrayType; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BooleanType; import org.apache.flink.table.types.logical.DateType; @@ -117,6 +121,7 @@ public class BaseRowArrowReaderWriterTest extends ArrowReaderWriterTestBase rowFields = new ArrayList<>(); for (int i = 0; i < fieldTypes.size(); i++) { @@ -146,18 +151,22 @@ public class BaseRowArrowReaderWriterTest extends ArrowReaderWriterTestBase { fieldTypes.add(new TimestampType(2)); fieldTypes.add(new TimestampType(4)); fieldTypes.add(new TimestampType(8)); + fieldTypes.add(new ArrayType(new VarCharType())); List rowFields = new ArrayList<>(); for (int i = 0; i < fieldTypes.size(); i++) { @@ -115,16 +117,19 @@ public class RowArrowReaderWriterTest extends ArrowReaderWriterTestBase { Row row1 = Row.of((byte) 1, (short) 2, 3, 4L, true, 1.0f, 1.0, "hello", "hello".getBytes(), new BigDecimal(1), SqlDateTimeUtils.internalToDate(100), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), - new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000)); + new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), + new String[] {null, null, null}); Row row2 = Row.of(null, (short) 2, 3, 4L, false, 1.0f, 1.0, "中文", "中文".getBytes(), new BigDecimal(1), SqlDateTimeUtils.internalToDate(100), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), - new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000)); + new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), + new String[] {"hello", "中文", null}); Row row3 = Row.of((byte) 1, null, 3, 4L, true, 1.0f, 1.0, "hello", "hello".getBytes(), new BigDecimal(1), SqlDateTimeUtils.internalToDate(100), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), SqlDateTimeUtils.internalToTime(3600000), new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), - new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000)); - Row row4 = Row.of(null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null); + new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), new Timestamp(3600000), + null); + Row row4 = Row.of(null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null); return new Row[]{row1, row2, row3, row4}; } } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/ArrowPythonScalarFunctionRunnerTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/ArrowPythonScalarFunctionRunnerTest.java index 06b28d6efa..94aba09fb4 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/ArrowPythonScalarFunctionRunnerTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/runners/python/scalar/arrow/ArrowPythonScalarFunctionRunnerTest.java @@ -25,7 +25,7 @@ import org.apache.flink.table.functions.python.PythonFunctionInfo; import org.apache.flink.table.runtime.arrow.ArrowUtils; import org.apache.flink.table.runtime.arrow.ArrowWriter; import org.apache.flink.table.runtime.arrow.writers.ArrowFieldWriter; -import org.apache.flink.table.runtime.arrow.writers.BigIntWriter; +import org.apache.flink.table.runtime.arrow.writers.RowBigIntWriter; import org.apache.flink.table.runtime.runners.python.scalar.AbstractPythonScalarFunctionRunnerTest; import org.apache.flink.table.runtime.utils.PassThroughArrowPythonScalarFunctionRunner; import org.apache.flink.table.runtime.utils.PythonTestUtils; @@ -64,7 +64,7 @@ public class ArrowPythonScalarFunctionRunnerTest extends AbstractPythonScalarFun ArrowFieldWriter[] fieldWriters = runner.arrowWriter.getFieldWriters(); assertEquals(1, fieldWriters.length); - assertTrue(fieldWriters[0] instanceof BigIntWriter); + assertTrue(fieldWriters[0] instanceof RowBigIntWriter); } @Test @@ -74,9 +74,9 @@ public class ArrowPythonScalarFunctionRunnerTest extends AbstractPythonScalarFun ArrowFieldWriter[] fieldWriters = runner.arrowWriter.getFieldWriters(); assertEquals(3, fieldWriters.length); - assertTrue(fieldWriters[0] instanceof BigIntWriter); - assertTrue(fieldWriters[1] instanceof BigIntWriter); - assertTrue(fieldWriters[2] instanceof BigIntWriter); + assertTrue(fieldWriters[0] instanceof RowBigIntWriter); + assertTrue(fieldWriters[1] instanceof RowBigIntWriter); + assertTrue(fieldWriters[2] instanceof RowBigIntWriter); } @Test @@ -86,11 +86,11 @@ public class ArrowPythonScalarFunctionRunnerTest extends AbstractPythonScalarFun ArrowFieldWriter[] fieldWriters = runner.arrowWriter.getFieldWriters(); assertEquals(5, fieldWriters.length); - assertTrue(fieldWriters[0] instanceof BigIntWriter); - assertTrue(fieldWriters[1] instanceof BigIntWriter); - assertTrue(fieldWriters[2] instanceof BigIntWriter); - assertTrue(fieldWriters[3] instanceof BigIntWriter); - assertTrue(fieldWriters[4] instanceof BigIntWriter); + assertTrue(fieldWriters[0] instanceof RowBigIntWriter); + assertTrue(fieldWriters[1] instanceof RowBigIntWriter); + assertTrue(fieldWriters[2] instanceof RowBigIntWriter); + assertTrue(fieldWriters[3] instanceof RowBigIntWriter); + assertTrue(fieldWriters[4] instanceof RowBigIntWriter); } @Test diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/util/StreamRecordUtils.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/util/StreamRecordUtils.java index 5ef8f27fb1..a8675d751b 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/util/StreamRecordUtils.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/util/StreamRecordUtils.java @@ -20,6 +20,7 @@ package org.apache.flink.table.runtime.util; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.table.dataformat.BaseArray; import org.apache.flink.table.dataformat.BaseRow; import org.apache.flink.table.dataformat.BinaryRow; import org.apache.flink.table.dataformat.BinaryRowWriter; @@ -28,6 +29,7 @@ import org.apache.flink.table.dataformat.Decimal; import org.apache.flink.table.dataformat.GenericRow; import org.apache.flink.table.dataformat.SqlTimestamp; import org.apache.flink.table.dataformat.util.BaseRowUtil; +import org.apache.flink.table.runtime.typeutils.BaseArraySerializer; import static org.apache.flink.table.dataformat.BinaryString.fromString; @@ -126,6 +128,10 @@ public class StreamRecordUtils { } else if (value instanceof Tuple2 && ((Tuple2) value).f0 instanceof SqlTimestamp) { SqlTimestamp timestamp = (SqlTimestamp) ((Tuple2) value).f0; writer.writeTimestamp(j, timestamp, (int) ((Tuple2) value).f1); + } else if (value instanceof Tuple2 && ((Tuple2) value).f0 instanceof BaseArray) { + BaseArray array = (BaseArray) ((Tuple2) value).f0; + BaseArraySerializer serializer = (BaseArraySerializer) ((Tuple2) value).f1; + writer.writeArray(j, array, serializer); } else { throw new RuntimeException("Not support yet!"); } -- Gitee From 2e74f25a8465c97924c367efc683fa6e1b6c5a0f Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 6 Apr 2020 10:57:52 +0200 Subject: [PATCH 466/885] [FLINK-16590] flink-oss-fs-hadoop: remove erroneous license references This is updated based on the output of the maven-shade-plugin on flink-oss-fs-hadoop. --- .../src/main/resources/META-INF/NOTICE | 119 +----- .../src/main/resources/licenses/LICENSE.asm | 31 -- .../main/resources/licenses/LICENSE.cddlv1.0 | 129 ------- .../main/resources/licenses/LICENSE.cddlv1.1 | 348 ------------------ .../src/main/resources/licenses/LICENSE.jzlib | 26 -- .../main/resources/licenses/LICENSE.paranamer | 28 -- .../main/resources/licenses/LICENSE.protobuf | 36 -- .../src/main/resources/licenses/LICENSE.re2j | 32 -- .../main/resources/licenses/LICENSE.stax2api | 22 -- 9 files changed, 11 insertions(+), 760 deletions(-) delete mode 100644 flink-filesystems/flink-oss-fs-hadoop/src/main/resources/licenses/LICENSE.asm delete mode 100644 flink-filesystems/flink-oss-fs-hadoop/src/main/resources/licenses/LICENSE.cddlv1.0 delete mode 100644 flink-filesystems/flink-oss-fs-hadoop/src/main/resources/licenses/LICENSE.cddlv1.1 delete mode 100644 flink-filesystems/flink-oss-fs-hadoop/src/main/resources/licenses/LICENSE.jzlib delete mode 100644 flink-filesystems/flink-oss-fs-hadoop/src/main/resources/licenses/LICENSE.paranamer delete mode 100644 flink-filesystems/flink-oss-fs-hadoop/src/main/resources/licenses/LICENSE.protobuf delete mode 100644 flink-filesystems/flink-oss-fs-hadoop/src/main/resources/licenses/LICENSE.re2j delete mode 100644 flink-filesystems/flink-oss-fs-hadoop/src/main/resources/licenses/LICENSE.stax2api diff --git a/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/META-INF/NOTICE b/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/META-INF/NOTICE index 3aea254a8b..46ca75c3b6 100644 --- a/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/META-INF/NOTICE +++ b/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/META-INF/NOTICE @@ -6,117 +6,20 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) -- com.aliyun.oss:aliyun-sdk-oss:3.4.1 -- com.aliyun:aliyun-java-sdk-core:3.4.0 -- com.aliyun:aliyun-java-sdk-ecs:4.2.0 -- com.aliyun:aliyun-java-sdk-ram:3.0.0 -- com.aliyun:aliyun-java-sdk-sts:3.0.0 -- com.fasterxml.jackson.core:jackson-annotations:2.10.1 -- com.fasterxml.jackson.core:jackson-core:2.10.1 -- com.fasterxml.jackson.core:jackson-databind:2.10.1 -- com.fasterxml.woodstox:woodstox-core:5.0.3 -- com.github.stephenc.jcip:jcip-annotations:1.0-1 -- com.google.code.gson:gson:2.2.4 -- com.google.guava:guava:11.0.2 -- com.nimbusds:nimbus-jose-jwt:4.41.1 -- commons-beanutils:commons-beanutils:1.9.3 -- commons-cli:commons-cli:1.3.1 -- commons-codec:commons-codec:1.10 -- commons-collections:commons-collections:3.2.2 -- commons-io:commons-io:2.4 -- commons-lang:commons-lang:3.3.2 -- commons-logging:commons-logging:1.1.3 -- commons-net:commons-net:3.6 -- net.minidev:accessors-smart:1.2 -- net.minidev:json-smart:2.3 -- org.apache.avro:avro:1.8.2 -- org.apache.commons:commons-compress:1.20 -- org.apache.commons:commons-configuration2:2.1.1 -- org.apache.commons:commons-lang3:3.3.2 -- org.apache.commons:commons-math3:3.5 -- org.apache.curator:curator-client:2.12.0 -- org.apache.curator:curator-framework:2.12.0 -- org.apache.curator:curator-recipes:2.12.0 -- org.apache.hadoop:hadoop-aliyun:3.1.0 -- org.apache.hadoop:hadoop-annotations:3.1.0 -- org.apache.hadoop:hadoop-auth:3.1.0 -- org.apache.hadoop:hadoop-common:3.1.0 -- org.apache.htrace:htrace-core4:4.1.0-incubating -- org.apache.httpcomponents:httpclient:4.5.3 -- org.apache.httpcomponents:httpcore:4.4.6 -- org.apache.kerby:kerb-admin:1.0.1 -- org.apache.kerby:kerb-client:1.0.1 -- org.apache.kerby:kerb-common:1.0.1 -- org.apache.kerby:kerb-core:1.0.1 -- org.apache.kerby:kerb-crypto:1.0.1 -- org.apache.kerby:kerb-identity:1.0.1 -- org.apache.kerby:kerb-server:1.0.1 -- org.apache.kerby:kerb-simplekdc:1.0.1 -- org.apache.kerby:kerb-util:1.0.1 -- org.apache.kerby:kerby-asn1:1.0.1 -- org.apache.kerby:kerby-config:1.0.1 -- org.apache.kerby:kerby-pkix:1.0.1 -- org.apache.kerby:kerby-util:1.0.1 -- org.apache.kerby:kerby-xdr:1.0.1 -- org.apache.kerby:token-provider:1.0.1 -- org.apache.zookeeper:zookeeper:3.4.10 -- org.codehaus.jackson:jackson-core-asl:1.9.2 -- org.codehaus.jackson:jackson-jaxrs:1.9.2 -- org.codehaus.jackson:jackson-mapper-asl:1.9.2 -- org.codehaus.jackson:jackson-xc:1.9.2 -- org.codehaus.jettison:jettison:1.1 -- org.eclipse.jetty:jetty-http:9.3.19.v20170502 -- org.eclipse.jetty:jetty-io:9.3.19.v20170502 -- org.eclipse.jetty:jetty-security:9.3.19.v20170502 -- org.eclipse.jetty:jetty-server:9.3.19.v20170502 -- org.eclipse.jetty:jetty-servlet:9.3.19.v20170502 -- org.eclipse.jetty:jetty-util:9.3.19.v20170502 -- org.eclipse.jetty:jetty-webapp:9.3.19.v20170502 -- org.eclipse.jetty:jetty-xml:9.3.19.v20170502 -- org.xerial.snappy:snappy-java:1.1.4 - -This project bundles the following dependencies under the BSD license. -See bundled license files for details. - -- com.google.protobuf:protobuf-java:2.5.0 -- com.jcraft:jsch:0.1.54 -- com.thoughtworks.paranamer:paranamer:2.7 -- org.codehaus.woodstox:stax2-api:3.1.4 -- org.ow2.asm:asm:5.0.4 - -This project bundles the following dependencies under the Common Development and Distribution License (CDDL) 1.0. -See bundled license files for details. - -- javax.ws.rs:jsr311-api:1.1.1 -- javax.xml.stream:stax-api:1.0-2 +- com.aliyun.oss:aliyun-sdk-oss:jar:3.4.1 +- com.aliyun:aliyun-java-sdk-core:jar:3.4.0 +- com.aliyun:aliyun-java-sdk-ecs:jar:4.2.0 +- com.aliyun:aliyun-java-sdk-ram:jar:3.0.0 +- com.aliyun:aliyun-java-sdk-sts:jar:3.0.0 +- commons-codec:commons-codec:jar:1.10 +- commons-logging:commons-logging:jar:1.1.3 +- org.apache.hadoop:hadoop-aliyun:jar:3.1.0 +- org.apache.httpcomponents:httpclient:jar:4.5.3 +- org.apache.httpcomponents:httpcore:jar:4.4.6 +- org.codehaus.jettison:jettison:jar:1.1 - stax:stax-api:1.0.1 -This project bundles the following dependencies under the Common Development and Distribution License (CDDL) 1.1. -See bundled license files for details. - -- com.sun.jersey:jersey-core:1.19 -- com.sun.jersey:jersey-json:1.9 -- com.sun.jersey:jersey-server:1.19 -- com.sun.jersey:jersey-servlet:1.19 -- com.sun.xml.bind:jaxb-impl:2.2.3-1 -- javax.activation:activation-api:1.2.0 -- javax.servlet.jsp:jsp-api:2.1 -- javax.servlet:javax.servlet-api:3.1.0 -- javax.xml.bind:jaxb-api:2.3.1 - -This project bundles the following dependencies under the Go License (https://golang.org/LICENSE). -See bundled license files for details. - -- com.google.re2j:re2j:1.1 - This project bundles the following dependencies under the JDOM license. See bundled license files for details. - org.jdom:jdom:1.1 - -This project bundles org.tukaani:xz:1.5. -This Java implementation of XZ has been put into the public domain, thus you can do -whatever you want with it. All the files in the package have been written by Lasse Collin, -but some files are heavily based on public domain code written by Igor Pavlov. - - diff --git a/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/licenses/LICENSE.asm b/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/licenses/LICENSE.asm deleted file mode 100644 index 62ffbccb68..0000000000 --- a/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/licenses/LICENSE.asm +++ /dev/null @@ -1,31 +0,0 @@ -ASM: a very small and fast Java bytecode manipulation framework - -Copyright (c) 2000-2011 INRIA, France Telecom -All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions -are met: - -1. Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - -2. Redistributions in binary form must reproduce the above copyright - notice, this list of conditions and the following disclaimer in the - documentation and/or other materials provided with the distribution. - -3. Neither the name of the copyright holders nor the names of its - contributors may be used to endorse or promote products derived from - this software without specific prior written permission. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" -AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE -IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE -ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE -LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR -CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF -SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS -INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN -CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) -ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF -THE POSSIBILITY OF SUCH DAMAGE. \ No newline at end of file diff --git a/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/licenses/LICENSE.cddlv1.0 b/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/licenses/LICENSE.cddlv1.0 deleted file mode 100644 index 468e7516a8..0000000000 --- a/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/licenses/LICENSE.cddlv1.0 +++ /dev/null @@ -1,129 +0,0 @@ -1. Definitions. - -1.1. "Contributor" means each individual or entity that creates or contributes to the creation of Modifications. - -1.2. "Contributor Version" means the combination of the Original Software, prior Modifications used by a Contributor (if any), and the Modifications made by that particular Contributor. - -1.3. "Covered Software" means (a) the Original Software, or (b) Modifications, or (c) the combination of files containing Original Software with files containing Modifications, in each case including portions thereof. - -1.4. "Executable" means the Covered Software in any form other than Source Code. - -1.5. "Initial Developer" means the individual or entity that first makes Original Software available under this License. - -1.6. "Larger Work" means a work which combines Covered Software or portions thereof with code not governed by the terms of this License. - -1.7. "License" means this document. - -1.8. "Licensable" means having the right to grant, to the maximum extent possible, whether at the time of the initial grant or subsequently acquired, any and all of the rights conveyed herein. - -1.9. "Modifications" means the Source Code and Executable form of any of the following: - -A. Any file that results from an addition to, deletion from or modification of the contents of a file containing Original Software or previous Modifications; - -B. Any new file that contains any part of the Original Software or previous Modification; or - -C. Any new file that is contributed or otherwise made available under the terms of this License. - -1.10. "Original Software" means the Source Code and Executable form of computer software code that is originally released under this License. - -1.11. "Patent Claims" means any patent claim(s), now owned or hereafter acquired, including without limitation, method, process, and apparatus claims, in any patent Licensable by grantor. - -1.12. "Source Code" means (a) the common form of computer software code in which modifications are made and (b) associated documentation included in or with such code. - -1.13. "You" (or "Your") means an individual or a legal entity exercising rights under, and complying with all of the terms of, this License. For legal entities, "You" includes any entity which controls, is controlled by, or is under common control with You. For purposes of this definition, "control" means (a) the power, direct or indirect, to cause the direction or management of such entity, whether by contract or otherwise, or (b) ownership of more than fifty percent (50%) of the outstanding shares or beneficial ownership of such entity. - -2. License Grants. - -2.1. The Initial Developer Grant. - -Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, the Initial Developer hereby grants You a world-wide, royalty-free, non-exclusive license: - -(a) under intellectual property rights (other than patent or trademark) Licensable by Initial Developer, to use, reproduce, modify, display, perform, sublicense and distribute the Original Software (or portions thereof), with or without Modifications, and/or as part of a Larger Work; and - -(b) under Patent Claims infringed by the making, using or selling of Original Software, to make, have made, use, practice, sell, and offer for sale, and/or otherwise dispose of the Original Software (or portions thereof). - -(c) The licenses granted in Sections 2.1(a) and (b) are effective on the date Initial Developer first distributes or otherwise makes the Original Software available to a third party under the terms of this License. - -(d) Notwithstanding Section 2.1(b) above, no patent license is granted: (1) for code that You delete from the Original Software, or (2) for infringements caused by: (i) the modification of the Original Software, or (ii) the combination of the Original Software with other software or devices. - -2.2. Contributor Grant. - -Conditioned upon Your compliance with Section 3.1 below and -subject to third party intellectual property claims, each -Contributor hereby grants You a world-wide, royalty-free, -non-exclusive license: - -(a) under intellectual property rights (other than patent or trademark) Licensable by Contributor to use, reproduce, modify, display, perform, sublicense and distribute the Modifications created by such Contributor (or portions thereof), either on an unmodified basis, with other Modifications, as Covered Software and/or as part of a Larger Work; and - -(b) under Patent Claims infringed by the making, using, or selling of Modifications made by that Contributor either alone and/or in combination with its Contributor Version (or portions of such combination), to make, use, sell, offer for sale, have made, and/or otherwise dispose of: (1) Modifications made by that Contributor (or portions thereof); and (2) the combination of Modifications made by that Contributor with its Contributor Version (or portions of such combination). - -(c) The licenses granted in Sections 2.2(a) and 2.2(b) are effective on the date Contributor first distributes or otherwise makes the Modifications available to a third party. -(d) Notwithstanding Section 2.2(b) above, no patent license is granted: (1) for any code that Contributor has deleted from the Contributor Version; (2) for infringements caused by: (i) third party modifications of Contributor Version, or (ii) the combination of Modifications made by that Contributor with other software (except as part of the Contributor Version) or other devices; or (3) under Patent Claims infringed by Covered Software in the absence of Modifications made by that Contributor. - -3. Distribution Obligations. - -3.1. Availability of Source Code. - -Any Covered Software that You distribute or otherwise make available in Executable form must also be made available in Source Code form and that Source Code form must be distributed only under the terms of this License. You must include a copy of this License with every copy of the Source Code form of the Covered Software You distribute or otherwise make available. You must inform recipients of any such Covered Software in Executable form as to how they can obtain such Covered Software in Source Code form in a reasonable manner on or through a medium customarily used for software exchange. - -3.2. Modifications. - -The Modifications that You create or to which You contribute are governed by the terms of this License. You represent that You believe Your Modifications are Your original creation(s) and/or You have sufficient rights to grant the rights conveyed by this License. - -3.3. Required Notices. - -You must include a notice in each of Your Modifications that identifies You as the Contributor of the Modification. You may not remove or alter any copyright, patent or trademark notices contained within the Covered Software, or any notices of licensing or any descriptive text giving attribution to any Contributor or the Initial Developer. - -3.4. Application of Additional Terms. - -You may not offer or impose any terms on any Covered Software in Source Code form that alters or restricts the applicable version of this License or the recipients' rights hereunder. You may choose to offer, and to charge a fee for, warranty, support, indemnity or liability obligations to one or more recipients of Covered Software. However, you may do so only on Your own behalf, and not on behalf of the Initial Developer or any Contributor. You must make it absolutely clear that any such warranty, support, indemnity or liability obligation is offered by You alone, and You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of warranty, support, indemnity or liability terms You offer. - -3.5. Distribution of Executable Versions. - -You may distribute the Executable form of the Covered Software under the terms of this License or under the terms of a license of Your choice, which may contain terms different from this License, provided that You are in compliance with the terms of this License and that the license for the Executable form does not attempt to limit or alter the recipient's rights in the Source Code form from the rights set forth in this License. If You distribute the Covered Software in Executable form under a different license, You must make it absolutely clear that any terms which differ from this License are offered by You alone, not by the Initial Developer or Contributor. You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of any such terms You offer. - -3.6. Larger Works. - -You may create a Larger Work by combining Covered Software with other code not governed by the terms of this License and distribute the Larger Work as a single product. In such a case, You must make sure the requirements of this License are fulfilled for the Covered Software. - -4. Versions of the License. - -4.1. New Versions. - -Sun Microsystems, Inc. is the initial license steward and may publish revised and/or new versions of this License from time to time. Each version will be given a distinguishing version number. Except as provided in Section 4.3, no one other than the license steward has the right to modify this License. - -4.2. Effect of New Versions. - -You may always continue to use, distribute or otherwise make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. If the Initial Developer includes a notice in the Original Software prohibiting it from being distributed or otherwise made available under any subsequent version of the License, You must distribute and make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. Otherwise, You may also choose to use, distribute or otherwise make the Covered Software available under the terms of any subsequent version of the License published by the license steward. - -4.3. Modified Versions. - -When You are an Initial Developer and You want to create a new license for Your Original Software, You may create and use a modified version of this License if You: (a) rename the license and remove any references to the name of the license steward (except to note that the license differs from this License); and (b) otherwise make it clear that the license contains terms which differ from this License. - -5. DISCLAIMER OF WARRANTY. - -COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN "AS IS" BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER. - -6. TERMINATION. - -6.1. This License and the rights granted hereunder will terminate automatically if You fail to comply with terms herein and fail to cure such breach within 30 days of becoming aware of the breach. Provisions which, by their nature, must remain in effect beyond the termination of this License shall survive. - -6.2. If You assert a patent infringement claim (excluding declaratory judgment actions) against Initial Developer or a Contributor (the Initial Developer or Contributor against whom You assert such claim is referred to as "Participant") alleging that the Participant Software (meaning the Contributor Version where the Participant is a Contributor or the Original Software where the Participant is the Initial Developer) directly or indirectly infringes any patent, then any and all rights granted directly or indirectly to You by such Participant, the Initial Developer (if the Initial Developer is not the Participant) and all Contributors under Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice from Participant terminate prospectively and automatically at the expiration of such 60 day notice period, unless if within such 60 day period You withdraw Your claim with respect to the Participant Software against such Participant either unilaterally or pursuant to a written agreement with Participant. - -6.3. In the event of termination under Sections 6.1 or 6.2 above, all end user licenses that have been validly granted by You or any distributor hereunder prior to termination (excluding licenses granted to You by any distributor) shall survive termination. - -7. LIMITATION OF LIABILITY. - -UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOST PROFITS, LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTY'S NEGLIGENCE TO THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU. - -8. U.S. GOVERNMENT END USERS. - -The Covered Software is a "commercial item," as that term is defined in 48 C.F.R. 2.101 (Oct. 1995), consisting of "commercial computer software" (as that term is defined at 48 C.F.R. ¤ 252.227-7014(a)(1)) and "commercial computer software documentation" as such terms are used in 48 C.F.R. 12.212 (Sept. 1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software with only those rights set forth herein. This U.S. Government Rights clause is in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision that addresses Government rights in computer software under this License. - -9. MISCELLANEOUS. - -This License represents the complete agreement concerning subject matter hereof. If any provision of this License is held to be unenforceable, such provision shall be reformed only to the extent necessary to make it enforceable. This License shall be governed by the law of the jurisdiction specified in a notice contained within the Original Software (except to the extent applicable law, if any, provides otherwise), excluding such jurisdiction's conflict-of-law provisions. Any litigation relating to this License shall be subject to the jurisdiction of the courts located in the jurisdiction and venue specified in a notice contained within the Original Software, with the losing party responsible for costs, including, without limitation, court costs and reasonable attorneys' fees and expenses. The application of the United Nations Convention on Contracts for the International Sale of Goods is expressly excluded. Any law or regulation which provides that the language of a contract shall be construed against the drafter shall not apply to this License. You agree that You alone are responsible for compliance with the United States export administration regulations (and the export control laws and regulation of any other countries) when You use, distribute or otherwise make available any Covered Software. - -10. RESPONSIBILITY FOR CLAIMS. - -As between Initial Developer and the Contributors, each party is responsible for claims and damages arising, directly or indirectly, out of its utilization of rights under this License and You agree to work with Initial Developer and Contributors to distribute such responsibility on an equitable basis. Nothing herein is intended or shall be deemed to constitute any admission of liability. diff --git a/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/licenses/LICENSE.cddlv1.1 b/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/licenses/LICENSE.cddlv1.1 deleted file mode 100644 index 75372e3d2c..0000000000 --- a/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/licenses/LICENSE.cddlv1.1 +++ /dev/null @@ -1,348 +0,0 @@ -1. Definitions. - - 1.1. "Contributor" means each individual or entity that creates or - contributes to the creation of Modifications. - - 1.2. "Contributor Version" means the combination of the Original - Software, prior Modifications used by a Contributor (if any), and - the Modifications made by that particular Contributor. - - 1.3. "Covered Software" means (a) the Original Software, or (b) - Modifications, or (c) the combination of files containing Original - Software with files containing Modifications, in each case including - portions thereof. - - 1.4. "Executable" means the Covered Software in any form other than - Source Code. - - 1.5. "Initial Developer" means the individual or entity that first - makes Original Software available under this License. - - 1.6. "Larger Work" means a work which combines Covered Software or - portions thereof with code not governed by the terms of this License. - - 1.7. "License" means this document. - - 1.8. "Licensable" means having the right to grant, to the maximum - extent possible, whether at the time of the initial grant or - subsequently acquired, any and all of the rights conveyed herein. - - 1.9. "Modifications" means the Source Code and Executable form of - any of the following: - - A. Any file that results from an addition to, deletion from or - modification of the contents of a file containing Original Software - or previous Modifications; - - B. Any new file that contains any part of the Original Software or - previous Modification; or - - C. Any new file that is contributed or otherwise made available - under the terms of this License. - - 1.10. "Original Software" means the Source Code and Executable form - of computer software code that is originally released under this - License. - - 1.11. "Patent Claims" means any patent claim(s), now owned or - hereafter acquired, including without limitation, method, process, - and apparatus claims, in any patent Licensable by grantor. - - 1.12. "Source Code" means (a) the common form of computer software - code in which modifications are made and (b) associated - documentation included in or with such code. - - 1.13. "You" (or "Your") means an individual or a legal entity - exercising rights under, and complying with all of the terms of, - this License. For legal entities, "You" includes any entity which - controls, is controlled by, or is under common control with You. For - purposes of this definition, "control" means (a) the power, direct - or indirect, to cause the direction or management of such entity, - whether by contract or otherwise, or (b) ownership of more than - fifty percent (50%) of the outstanding shares or beneficial - ownership of such entity. - -2. License Grants. - - 2.1. The Initial Developer Grant. - - Conditioned upon Your compliance with Section 3.1 below and subject - to third party intellectual property claims, the Initial Developer - hereby grants You a world-wide, royalty-free, non-exclusive license: - - (a) under intellectual property rights (other than patent or - trademark) Licensable by Initial Developer, to use, reproduce, - modify, display, perform, sublicense and distribute the Original - Software (or portions thereof), with or without Modifications, - and/or as part of a Larger Work; and - - (b) under Patent Claims infringed by the making, using or selling of - Original Software, to make, have made, use, practice, sell, and - offer for sale, and/or otherwise dispose of the Original Software - (or portions thereof). - - (c) The licenses granted in Sections 2.1(a) and (b) are effective on - the date Initial Developer first distributes or otherwise makes the - Original Software available to a third party under the terms of this - License. - - (d) Notwithstanding Section 2.1(b) above, no patent license is - granted: (1) for code that You delete from the Original Software, or - (2) for infringements caused by: (i) the modification of the - Original Software, or (ii) the combination of the Original Software - with other software or devices. - - 2.2. Contributor Grant. - - Conditioned upon Your compliance with Section 3.1 below and subject - to third party intellectual property claims, each Contributor hereby - grants You a world-wide, royalty-free, non-exclusive license: - - (a) under intellectual property rights (other than patent or - trademark) Licensable by Contributor to use, reproduce, modify, - display, perform, sublicense and distribute the Modifications - created by such Contributor (or portions thereof), either on an - unmodified basis, with other Modifications, as Covered Software - and/or as part of a Larger Work; and - - (b) under Patent Claims infringed by the making, using, or selling - of Modifications made by that Contributor either alone and/or in - combination with its Contributor Version (or portions of such - combination), to make, use, sell, offer for sale, have made, and/or - otherwise dispose of: (1) Modifications made by that Contributor (or - portions thereof); and (2) the combination of Modifications made by - that Contributor with its Contributor Version (or portions of such - combination). - - (c) The licenses granted in Sections 2.2(a) and 2.2(b) are effective - on the date Contributor first distributes or otherwise makes the - Modifications available to a third party. - - (d) Notwithstanding Section 2.2(b) above, no patent license is - granted: (1) for any code that Contributor has deleted from the - Contributor Version; (2) for infringements caused by: (i) third - party modifications of Contributor Version, or (ii) the combination - of Modifications made by that Contributor with other software - (except as part of the Contributor Version) or other devices; or (3) - under Patent Claims infringed by Covered Software in the absence of - Modifications made by that Contributor. - -3. Distribution Obligations. - - 3.1. Availability of Source Code. - - Any Covered Software that You distribute or otherwise make available - in Executable form must also be made available in Source Code form - and that Source Code form must be distributed only under the terms - of this License. You must include a copy of this License with every - copy of the Source Code form of the Covered Software You distribute - or otherwise make available. You must inform recipients of any such - Covered Software in Executable form as to how they can obtain such - Covered Software in Source Code form in a reasonable manner on or - through a medium customarily used for software exchange. - - 3.2. Modifications. - - The Modifications that You create or to which You contribute are - governed by the terms of this License. You represent that You - believe Your Modifications are Your original creation(s) and/or You - have sufficient rights to grant the rights conveyed by this License. - - 3.3. Required Notices. - - You must include a notice in each of Your Modifications that - identifies You as the Contributor of the Modification. You may not - remove or alter any copyright, patent or trademark notices contained - within the Covered Software, or any notices of licensing or any - descriptive text giving attribution to any Contributor or the - Initial Developer. - - 3.4. Application of Additional Terms. - - You may not offer or impose any terms on any Covered Software in - Source Code form that alters or restricts the applicable version of - this License or the recipients' rights hereunder. You may choose to - offer, and to charge a fee for, warranty, support, indemnity or - liability obligations to one or more recipients of Covered Software. - However, you may do so only on Your own behalf, and not on behalf of - the Initial Developer or any Contributor. You must make it - absolutely clear that any such warranty, support, indemnity or - liability obligation is offered by You alone, and You hereby agree - to indemnify the Initial Developer and every Contributor for any - liability incurred by the Initial Developer or such Contributor as a - result of warranty, support, indemnity or liability terms You offer. - - 3.5. Distribution of Executable Versions. - - You may distribute the Executable form of the Covered Software under - the terms of this License or under the terms of a license of Your - choice, which may contain terms different from this License, - provided that You are in compliance with the terms of this License - and that the license for the Executable form does not attempt to - limit or alter the recipient's rights in the Source Code form from - the rights set forth in this License. If You distribute the Covered - Software in Executable form under a different license, You must make - it absolutely clear that any terms which differ from this License - are offered by You alone, not by the Initial Developer or - Contributor. You hereby agree to indemnify the Initial Developer and - every Contributor for any liability incurred by the Initial - Developer or such Contributor as a result of any such terms You offer. - - 3.6. Larger Works. - - You may create a Larger Work by combining Covered Software with - other code not governed by the terms of this License and distribute - the Larger Work as a single product. In such a case, You must make - sure the requirements of this License are fulfilled for the Covered - Software. - -4. Versions of the License. - - 4.1. New Versions. - - Oracle is the initial license steward and may publish revised and/or - new versions of this License from time to time. Each version will be - given a distinguishing version number. Except as provided in Section - 4.3, no one other than the license steward has the right to modify - this License. - - 4.2. Effect of New Versions. - - You may always continue to use, distribute or otherwise make the - Covered Software available under the terms of the version of the - License under which You originally received the Covered Software. If - the Initial Developer includes a notice in the Original Software - prohibiting it from being distributed or otherwise made available - under any subsequent version of the License, You must distribute and - make the Covered Software available under the terms of the version - of the License under which You originally received the Covered - Software. Otherwise, You may also choose to use, distribute or - otherwise make the Covered Software available under the terms of any - subsequent version of the License published by the license steward. - - 4.3. Modified Versions. - - When You are an Initial Developer and You want to create a new - license for Your Original Software, You may create and use a - modified version of this License if You: (a) rename the license and - remove any references to the name of the license steward (except to - note that the license differs from this License); and (b) otherwise - make it clear that the license contains terms which differ from this - License. - -5. DISCLAIMER OF WARRANTY. - - COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN "AS IS" BASIS, - WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, - INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE - IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR - NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF - THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE - DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY - OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, - REPAIR OR CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN - ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED SOFTWARE IS - AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER. - -6. TERMINATION. - - 6.1. This License and the rights granted hereunder will terminate - automatically if You fail to comply with terms herein and fail to - cure such breach within 30 days of becoming aware of the breach. - Provisions which, by their nature, must remain in effect beyond the - termination of this License shall survive. - - 6.2. If You assert a patent infringement claim (excluding - declaratory judgment actions) against Initial Developer or a - Contributor (the Initial Developer or Contributor against whom You - assert such claim is referred to as "Participant") alleging that the - Participant Software (meaning the Contributor Version where the - Participant is a Contributor or the Original Software where the - Participant is the Initial Developer) directly or indirectly - infringes any patent, then any and all rights granted directly or - indirectly to You by such Participant, the Initial Developer (if the - Initial Developer is not the Participant) and all Contributors under - Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice - from Participant terminate prospectively and automatically at the - expiration of such 60 day notice period, unless if within such 60 - day period You withdraw Your claim with respect to the Participant - Software against such Participant either unilaterally or pursuant to - a written agreement with Participant. - - 6.3. If You assert a patent infringement claim against Participant - alleging that the Participant Software directly or indirectly - infringes any patent where such claim is resolved (such as by - license or settlement) prior to the initiation of patent - infringement litigation, then the reasonable value of the licenses - granted by such Participant under Sections 2.1 or 2.2 shall be taken - into account in determining the amount or value of any payment or - license. - - 6.4. In the event of termination under Sections 6.1 or 6.2 above, - all end user licenses that have been validly granted by You or any - distributor hereunder prior to termination (excluding licenses - granted to You by any distributor) shall survive termination. - -7. LIMITATION OF LIABILITY. - - UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT - (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE - INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF - COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE - TO ANY PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR - CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, WITHOUT - LIMITATION, DAMAGES FOR LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER - FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR - LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE - POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT - APPLY TO LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH - PARTY'S NEGLIGENCE TO THE EXTENT APPLICABLE LAW PROHIBITS SUCH - LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR - LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION - AND LIMITATION MAY NOT APPLY TO YOU. - -8. U.S. GOVERNMENT END USERS. - - The Covered Software is a "commercial item," as that term is defined - in 48 C.F.R. 2.101 (Oct. 1995), consisting of "commercial computer - software" (as that term is defined at 48 C.F.R. § - 252.227-7014(a)(1)) and "commercial computer software documentation" - as such terms are used in 48 C.F.R. 12.212 (Sept. 1995). Consistent - with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 - (June 1995), all U.S. Government End Users acquire Covered Software - with only those rights set forth herein. This U.S. Government Rights - clause is in lieu of, and supersedes, any other FAR, DFAR, or other - clause or provision that addresses Government rights in computer - software under this License. - -9. MISCELLANEOUS. - - This License represents the complete agreement concerning subject - matter hereof. If any provision of this License is held to be - unenforceable, such provision shall be reformed only to the extent - necessary to make it enforceable. This License shall be governed by - the law of the jurisdiction specified in a notice contained within - the Original Software (except to the extent applicable law, if any, - provides otherwise), excluding such jurisdiction's conflict-of-law - provisions. Any litigation relating to this License shall be subject - to the jurisdiction of the courts located in the jurisdiction and - venue specified in a notice contained within the Original Software, - with the losing party responsible for costs, including, without - limitation, court costs and reasonable attorneys' fees and expenses. - The application of the United Nations Convention on Contracts for - the International Sale of Goods is expressly excluded. Any law or - regulation which provides that the language of a contract shall be - construed against the drafter shall not apply to this License. You - agree that You alone are responsible for compliance with the United - States export administration regulations (and the export control - laws and regulation of any other countries) when You use, distribute - or otherwise make available any Covered Software. - -10. RESPONSIBILITY FOR CLAIMS. - - As between Initial Developer and the Contributors, each party is - responsible for claims and damages arising, directly or indirectly, - out of its utilization of rights under this License and You agree to - work with Initial Developer and Contributors to distribute such - responsibility on an equitable basis. Nothing herein is intended or - shall be deemed to constitute any admission of liability. diff --git a/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/licenses/LICENSE.jzlib b/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/licenses/LICENSE.jzlib deleted file mode 100644 index 29ad562af0..0000000000 --- a/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/licenses/LICENSE.jzlib +++ /dev/null @@ -1,26 +0,0 @@ -Copyright (c) 2000,2001,2002,2003,2004 ymnk, JCraft,Inc. All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions are met: - - 1. Redistributions of source code must retain the above copyright notice, - this list of conditions and the following disclaimer. - - 2. Redistributions in binary form must reproduce the above copyright - notice, this list of conditions and the following disclaimer in - the documentation and/or other materials provided with the distribution. - - 3. The names of the authors may not be used to endorse or promote products - derived from this software without specific prior written permission. - -THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED WARRANTIES, -INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND -FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL JCRAFT, -INC. OR ANY CONTRIBUTORS TO THIS SOFTWARE BE LIABLE FOR ANY DIRECT, INDIRECT, -INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, -OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF -LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING -NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, -EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - diff --git a/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/licenses/LICENSE.paranamer b/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/licenses/LICENSE.paranamer deleted file mode 100644 index 522176fbca..0000000000 --- a/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/licenses/LICENSE.paranamer +++ /dev/null @@ -1,28 +0,0 @@ -[ ParaNamer used to be 'Pubic Domain', but since it includes a small piece of ASM it is now the same license as that: BSD ] - - Copyright (c) 2006 Paul Hammant & ThoughtWorks Inc - All rights reserved. - - Redistribution and use in source and binary forms, with or without - modification, are permitted provided that the following conditions - are met: - 1. Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - 2. Redistributions in binary form must reproduce the above copyright - notice, this list of conditions and the following disclaimer in the - documentation and/or other materials provided with the distribution. - 3. Neither the name of the copyright holders nor the names of its - contributors may be used to endorse or promote products derived from - this software without specific prior written permission. - - THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" - AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE - IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE - ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE - LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR - CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF - SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS - INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN - CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) - ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF - THE POSSIBILITY OF SUCH DAMAGE. diff --git a/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/licenses/LICENSE.protobuf b/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/licenses/LICENSE.protobuf deleted file mode 100644 index 21645bec9d..0000000000 --- a/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/licenses/LICENSE.protobuf +++ /dev/null @@ -1,36 +0,0 @@ -Copyright 2008, Google Inc. -All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions are -met: - - * Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - - * Redistributions in binary form must reproduce the above - copyright notice, this list of conditions and the following disclaimer - in the documentation and/or other materials provided with the - distribution. - - * Neither the name of Google Inc. nor the names of its - contributors may be used to endorse or promote products derived from - this software without specific prior written permission. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - -Code generated by the Protocol Buffer compiler is owned by the owner -of the input file used when generating it. This code is not -standalone and requires a support library to be linked with it. This -support library is itself covered by the above license. - diff --git a/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/licenses/LICENSE.re2j b/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/licenses/LICENSE.re2j deleted file mode 100644 index b620ae68fe..0000000000 --- a/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/licenses/LICENSE.re2j +++ /dev/null @@ -1,32 +0,0 @@ -This is a work derived from Russ Cox's RE2 in Go, whose license -http://golang.org/LICENSE is as follows: - -Copyright (c) 2009 The Go Authors. All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions are -met: - - * Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - - * Redistributions in binary form must reproduce the above copyright - notice, this list of conditions and the following disclaimer in - the documentation and/or other materials provided with the - distribution. - - * Neither the name of Google Inc. nor the names of its contributors - may be used to endorse or promote products derived from this - software without specific prior written permission. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/licenses/LICENSE.stax2api b/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/licenses/LICENSE.stax2api deleted file mode 100644 index 0ed6361699..0000000000 --- a/flink-filesystems/flink-oss-fs-hadoop/src/main/resources/licenses/LICENSE.stax2api +++ /dev/null @@ -1,22 +0,0 @@ -Copyright woodstox stax2api contributors. - -Redistribution and use in source and binary forms, with or without modification, -are permitted provided that the following conditions are met: - -1. Redistributions of source code must retain the above copyright notice, - this list of conditions and the following disclaimer. - -2. Redistributions in binary form must reproduce the above copyright notice, - this list of conditions and the following disclaimer in the documentation - and/or other materials provided with the distribution. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND -ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED -WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. -IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, -INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, -BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, -OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, -WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) -ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE -POSSIBILITY OF SUCH DAMAGE. -- Gitee From 455549297a5779f95530595b0cb58c4539b55298 Mon Sep 17 00:00:00 2001 From: klion26 Date: Mon, 6 Apr 2020 18:07:40 +0800 Subject: [PATCH 467/885] [FLINK-17007][docs] Add section 'Handling Application Parameters' in DataStream --- docs/dev/application_parameters.md | 140 ++++++++++++++++++++++++++ docs/dev/application_parameters.zh.md | 140 ++++++++++++++++++++++++++ 2 files changed, 280 insertions(+) create mode 100644 docs/dev/application_parameters.md create mode 100644 docs/dev/application_parameters.zh.md diff --git a/docs/dev/application_parameters.md b/docs/dev/application_parameters.md new file mode 100644 index 0000000000..be6e6ec05b --- /dev/null +++ b/docs/dev/application_parameters.md @@ -0,0 +1,140 @@ +--- +title: "Handling Application Parameters" +nav-id: application_parameters +nav-show_overview: true +nav-parent_id: streaming +nav-pos: 50 +--- + + +* This will be replaced by the TOC +{:toc} + +Handling Application Parameters +------------------------------- +Almost all Flink applications, both batch and streaming, rely on external configuration parameters. +They are used to specify input and output sources (like paths or addresses), system parameters (parallelism, runtime configuration), and application specific parameters (typically used within user functions). + +Flink provides a simple utility called `ParameterTool` to provide some basic tooling for solving these problems. +Please note that you don't have to use the `ParameterTool` described here. Other frameworks such as [Commons CLI](https://commons.apache.org/proper/commons-cli/) and +[argparse4j](http://argparse4j.sourceforge.net/) also work well with Flink. + + +### Getting your configuration values into the `ParameterTool` + +The `ParameterTool` provides a set of predefined static methods for reading the configuration. The tool is internally expecting a `Map`, so it's very easy to integrate it with your own configuration style. + + +#### From `.properties` files + +The following method will read a [Properties](https://docs.oracle.com/javase/tutorial/essential/environment/properties.html) file and provide the key/value pairs: +{% highlight java %} +String propertiesFilePath = "/home/sam/flink/myjob.properties"; +ParameterTool parameter = ParameterTool.fromPropertiesFile(propertiesFilePath); + +File propertiesFile = new File(propertiesFilePath); +ParameterTool parameter = ParameterTool.fromPropertiesFile(propertiesFile); + +InputStream propertiesFileInputStream = new FileInputStream(file); +ParameterTool parameter = ParameterTool.fromPropertiesFile(propertiesFileInputStream); +{% endhighlight %} + + +#### From the command line arguments + +This allows getting arguments like `--input hdfs:///mydata --elements 42` from the command line. +{% highlight java %} +public static void main(String[] args) { + ParameterTool parameter = ParameterTool.fromArgs(args); + // .. regular code .. +{% endhighlight %} + + +#### From system properties + +When starting a JVM, you can pass system properties to it: `-Dinput=hdfs:///mydata`. You can also initialize the `ParameterTool` from these system properties: + +{% highlight java %} +ParameterTool parameter = ParameterTool.fromSystemProperties(); +{% endhighlight %} + + +### Using the parameters in your Flink program + +Now that we've got the parameters from somewhere (see above) we can use them in various ways. + +**Directly from the `ParameterTool`** + +The `ParameterTool` itself has methods for accessing the values. +{% highlight java %} +ParameterTool parameters = // ... +parameter.getRequired("input"); +parameter.get("output", "myDefaultValue"); +parameter.getLong("expectedCount", -1L); +parameter.getNumberOfParameters() +// .. there are more methods available. +{% endhighlight %} + +You can use the return values of these methods directly in the `main()` method of the client submitting the application. +For example, you could set the parallelism of a operator like this: + +{% highlight java %} +ParameterTool parameters = ParameterTool.fromArgs(args); +int parallelism = parameters.get("mapParallelism", 2); +DataSet> counts = text.flatMap(new Tokenizer()).setParallelism(parallelism); +{% endhighlight %} + +Since the `ParameterTool` is serializable, you can pass it to the functions itself: + +{% highlight java %} +ParameterTool parameters = ParameterTool.fromArgs(args); +DataSet> counts = text.flatMap(new Tokenizer(parameters)); +{% endhighlight %} + +and then use it inside the function for getting values from the command line. + +#### Register the parameters globally + +Parameters registered as global job parameters in the `ExecutionConfig` can be accessed as configuration values from the JobManager web interface and in all functions defined by the user. + +Register the parameters globally: + +{% highlight java %} +ParameterTool parameters = ParameterTool.fromArgs(args); + +// set up the execution environment +final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); +env.getConfig().setGlobalJobParameters(parameters); +{% endhighlight %} + +Access them in any rich user function: + +{% highlight java %} +public static final class Tokenizer extends RichFlatMapFunction> { + + @Override + public void flatMap(String value, Collector> out) { + ParameterTool parameters = (ParameterTool) + getRuntimeContext().getExecutionConfig().getGlobalJobParameters(); + parameters.getRequired("input"); + // .. do more .. +{% endhighlight %} + +{% top %} diff --git a/docs/dev/application_parameters.zh.md b/docs/dev/application_parameters.zh.md new file mode 100644 index 0000000000..be6e6ec05b --- /dev/null +++ b/docs/dev/application_parameters.zh.md @@ -0,0 +1,140 @@ +--- +title: "Handling Application Parameters" +nav-id: application_parameters +nav-show_overview: true +nav-parent_id: streaming +nav-pos: 50 +--- + + +* This will be replaced by the TOC +{:toc} + +Handling Application Parameters +------------------------------- +Almost all Flink applications, both batch and streaming, rely on external configuration parameters. +They are used to specify input and output sources (like paths or addresses), system parameters (parallelism, runtime configuration), and application specific parameters (typically used within user functions). + +Flink provides a simple utility called `ParameterTool` to provide some basic tooling for solving these problems. +Please note that you don't have to use the `ParameterTool` described here. Other frameworks such as [Commons CLI](https://commons.apache.org/proper/commons-cli/) and +[argparse4j](http://argparse4j.sourceforge.net/) also work well with Flink. + + +### Getting your configuration values into the `ParameterTool` + +The `ParameterTool` provides a set of predefined static methods for reading the configuration. The tool is internally expecting a `Map`, so it's very easy to integrate it with your own configuration style. + + +#### From `.properties` files + +The following method will read a [Properties](https://docs.oracle.com/javase/tutorial/essential/environment/properties.html) file and provide the key/value pairs: +{% highlight java %} +String propertiesFilePath = "/home/sam/flink/myjob.properties"; +ParameterTool parameter = ParameterTool.fromPropertiesFile(propertiesFilePath); + +File propertiesFile = new File(propertiesFilePath); +ParameterTool parameter = ParameterTool.fromPropertiesFile(propertiesFile); + +InputStream propertiesFileInputStream = new FileInputStream(file); +ParameterTool parameter = ParameterTool.fromPropertiesFile(propertiesFileInputStream); +{% endhighlight %} + + +#### From the command line arguments + +This allows getting arguments like `--input hdfs:///mydata --elements 42` from the command line. +{% highlight java %} +public static void main(String[] args) { + ParameterTool parameter = ParameterTool.fromArgs(args); + // .. regular code .. +{% endhighlight %} + + +#### From system properties + +When starting a JVM, you can pass system properties to it: `-Dinput=hdfs:///mydata`. You can also initialize the `ParameterTool` from these system properties: + +{% highlight java %} +ParameterTool parameter = ParameterTool.fromSystemProperties(); +{% endhighlight %} + + +### Using the parameters in your Flink program + +Now that we've got the parameters from somewhere (see above) we can use them in various ways. + +**Directly from the `ParameterTool`** + +The `ParameterTool` itself has methods for accessing the values. +{% highlight java %} +ParameterTool parameters = // ... +parameter.getRequired("input"); +parameter.get("output", "myDefaultValue"); +parameter.getLong("expectedCount", -1L); +parameter.getNumberOfParameters() +// .. there are more methods available. +{% endhighlight %} + +You can use the return values of these methods directly in the `main()` method of the client submitting the application. +For example, you could set the parallelism of a operator like this: + +{% highlight java %} +ParameterTool parameters = ParameterTool.fromArgs(args); +int parallelism = parameters.get("mapParallelism", 2); +DataSet> counts = text.flatMap(new Tokenizer()).setParallelism(parallelism); +{% endhighlight %} + +Since the `ParameterTool` is serializable, you can pass it to the functions itself: + +{% highlight java %} +ParameterTool parameters = ParameterTool.fromArgs(args); +DataSet> counts = text.flatMap(new Tokenizer(parameters)); +{% endhighlight %} + +and then use it inside the function for getting values from the command line. + +#### Register the parameters globally + +Parameters registered as global job parameters in the `ExecutionConfig` can be accessed as configuration values from the JobManager web interface and in all functions defined by the user. + +Register the parameters globally: + +{% highlight java %} +ParameterTool parameters = ParameterTool.fromArgs(args); + +// set up the execution environment +final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); +env.getConfig().setGlobalJobParameters(parameters); +{% endhighlight %} + +Access them in any rich user function: + +{% highlight java %} +public static final class Tokenizer extends RichFlatMapFunction> { + + @Override + public void flatMap(String value, Collector> out) { + ParameterTool parameters = (ParameterTool) + getRuntimeContext().getExecutionConfig().getGlobalJobParameters(); + parameters.getRequired("input"); + // .. do more .. +{% endhighlight %} + +{% top %} -- Gitee From aad3f7d5dec93455025a0e3e61411a15a7f716f7 Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Sat, 4 Apr 2020 10:08:37 +0200 Subject: [PATCH 468/885] [FLINK-16973][tests] Add tooling for collecting jvm crash files --- azure-pipelines.yml | 2 ++ .../tests/util/kafka/LocalStandaloneKafkaResource.java | 2 +- flink-end-to-end-tests/test-scripts/common.sh | 3 +++ tools/azure-pipelines/jobs-template.yml | 2 ++ tools/ci/maven-utils.sh | 10 ++++++++++ tools/travis_watchdog.sh | 6 ++++++ 6 files changed, 24 insertions(+), 1 deletion(-) diff --git a/azure-pipelines.yml b/azure-pipelines.yml index 89a5089bd3..e0a585372e 100644 --- a/azure-pipelines.yml +++ b/azure-pipelines.yml @@ -39,6 +39,8 @@ resources: # Container with Maven 3.2.5, SSL to have the same environment everywhere. - container: flink-build-container image: rmetzger/flink-ci:ubuntu-amd64-3528acd + # On AZP provided machines, set this flag to allow writing coredumps in docker + options: --privileged # Define variables: # - See tools/azure-pipelines/jobs-template.yml for a short summary of the caching diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/LocalStandaloneKafkaResource.java b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/LocalStandaloneKafkaResource.java index 1b1a3395c7..405690f4dd 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/LocalStandaloneKafkaResource.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/LocalStandaloneKafkaResource.java @@ -199,7 +199,7 @@ public class LocalStandaloneKafkaResource implements KafkaResource { private static boolean isKafkaRunning(final Path kafkaDir) throws IOException { try { final AtomicBoolean atomicBrokerStarted = new AtomicBoolean(false); - queryBrokerStatus(kafkaDir, line -> atomicBrokerStarted.compareAndSet(false, !line.contains("Node does not exist"))); + queryBrokerStatus(kafkaDir, line -> atomicBrokerStarted.compareAndSet(false, line.contains("dataLength ="))); return atomicBrokerStarted.get(); } catch (final IOException ioe) { // we get an exception if zookeeper isn't running diff --git a/flink-end-to-end-tests/test-scripts/common.sh b/flink-end-to-end-tests/test-scripts/common.sh index 44ce92f40f..fad1c74731 100644 --- a/flink-end-to-end-tests/test-scripts/common.sh +++ b/flink-end-to-end-tests/test-scripts/common.sh @@ -352,6 +352,7 @@ function check_logs_for_errors { | grep -v "Error while loading kafka-version.properties :null" \ | grep -v "Failed Elasticsearch item request" \ | grep -v "[Terror] modules" \ + | grep -v "HeapDumpOnOutOfMemoryError" \ | grep -ic "error" || true) if [[ ${error_count} -gt 0 ]]; then echo "Found error in log files:" @@ -401,12 +402,14 @@ function check_logs_for_exceptions { function check_logs_for_non_empty_out_files { echo "Checking for non-empty .out files..." # exclude reflective access warnings as these are expected (and currently unavoidable) on Java 9 + # exclude message about JAVA_TOOL_OPTIONS being set (https://bugs.openjdk.java.net/browse/JDK-8039152) if grep -ri -v \ -e "WARNING: An illegal reflective access" \ -e "WARNING: Illegal reflective access"\ -e "WARNING: Please consider reporting"\ -e "WARNING: Use --illegal-access"\ -e "WARNING: All illegal access"\ + -e "Picked up JAVA_TOOL_OPTIONS"\ $FLINK_DIR/log/*.out\ | grep "." \ > /dev/null; then diff --git a/tools/azure-pipelines/jobs-template.yml b/tools/azure-pipelines/jobs-template.yml index 85217901a6..e036ebc7f2 100644 --- a/tools/azure-pipelines/jobs-template.yml +++ b/tools/azure-pipelines/jobs-template.yml @@ -117,6 +117,8 @@ jobs: echo "##vso[task.setvariable variable=PATH]$JAVA_HOME_11_X64/bin:$PATH" displayName: "Set to jdk11" condition: eq('${{parameters.jdk}}', 'jdk11') + - script: sudo sysctl -w kernel.core_pattern=core.%p + displayName: Set coredump pattern # Test - script: STAGE=test ${{parameters.environment}} ./tools/azure_controller.sh $(module) displayName: Test - $(module) diff --git a/tools/ci/maven-utils.sh b/tools/ci/maven-utils.sh index f36e374998..1ea3d28de8 100755 --- a/tools/ci/maven-utils.sh +++ b/tools/ci/maven-utils.sh @@ -67,3 +67,13 @@ function setup_maven { echo "Installed Maven ${MAVEN_VERSION} to ${M2_HOME}" } + +function collect_coredumps { + local SEARCHDIR=$1 + local TARGET_DIR=$2 + echo "Searching for .dump, .dumpstream and related files in '$SEARCHDIR'" + for file in `find $SEARCHDIR -type f -regextype posix-extended -iregex '.*\.hprof|.*\.dump|.*\.dumpstream|.*hs.*\.log|.*/core(.[0-9]+)?$'`; do + echo "Moving '$file' to target directory ('$TARGET_DIR')" + mv $file $TARGET_DIR/ + done +} diff --git a/tools/travis_watchdog.sh b/tools/travis_watchdog.sh index bb05113b4f..296789891e 100755 --- a/tools/travis_watchdog.sh +++ b/tools/travis_watchdog.sh @@ -97,6 +97,10 @@ if [ ! -z "$TF_BUILD" ] ; then ARTIFACTS_FILE=${BUILD_BUILDNUMBER}.tar.gz fi +# enable coredumps +ulimit -c unlimited +export JAVA_TOOL_OPTIONS="-XX:+HeapDumpOnOutOfMemoryError" + if [ $TEST == $STAGE_PYTHON ]; then CMD=$PYTHON_TEST CMD_PID=$PYTHON_PID @@ -276,6 +280,8 @@ case $TEST in ;; esac +collect_coredumps `pwd` $ARTIFACTS_DIR + upload_artifacts_s3 # since we are in flink/tools/artifacts -- Gitee From 693cb6adc42d75d1db720b45013430a4c6817d4a Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Fri, 3 Apr 2020 11:08:56 +0800 Subject: [PATCH 469/885] [FLINK-16537][network] Implement ResultPartition state recovery for unaligned checkpoint During state recovery for unaligned checkpoint, the partition state should also be recovered besides with existing operator states. The ResultPartition would request buffer from local pool and then interact with ChannelStateReader to fill in the state data. The filled buffer would be inserted into respective ResultSubpartition queue in normal way. It should guarantee that op can not process any inputs before finishing all the output recovery to avoid mis-order issue. --- .../channel/ChannelStateReader.java | 5 +- .../api/writer/ResultPartitionWriter.java | 7 + .../partition/PipelinedSubpartition.java | 22 ++- .../partition/PipelinedSubpartitionView.java | 4 +- .../io/network/partition/ResultPartition.java | 8 ++ .../network/partition/ResultSubpartition.java | 4 + ...tifyingResultPartitionWriterDecorator.java | 6 + .../network/api/writer/RecordWriterTest.java | 62 +++++++++ .../buffer/BufferBuilderAndConsumerTest.java | 10 +- .../partition/MockResultPartitionWriter.java | 5 + .../NoOpBufferAvailablityListener.java | 2 +- .../partition/ResultPartitionTest.java | 125 ++++++++++++++++++ .../streaming/runtime/tasks/StreamTask.java | 9 ++ .../runtime/tasks/StreamTaskTest.java | 42 ++++++ 14 files changed, 300 insertions(+), 11 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateReader.java index 49321cc1d4..0753e7a533 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateReader.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateReader.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.checkpoint.channel; import org.apache.flink.annotation.Internal; import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferBuilder; import java.io.IOException; @@ -42,7 +43,7 @@ public interface ChannelStateReader extends AutoCloseable { * Put data into the supplied buffer to be injected into * {@link org.apache.flink.runtime.io.network.partition.ResultSubpartition ResultSubpartition}. */ - ReadResult readOutputData(ResultSubpartitionInfo info, Buffer buffer) throws IOException; + ReadResult readOutputData(ResultSubpartitionInfo info, BufferBuilder bufferBuilder) throws IOException; @Override void close() throws Exception; @@ -55,7 +56,7 @@ public interface ChannelStateReader extends AutoCloseable { } @Override - public ReadResult readOutputData(ResultSubpartitionInfo info, Buffer buffer) { + public ReadResult readOutputData(ResultSubpartitionInfo info, BufferBuilder bufferBuilder) { return ReadResult.NO_MORE_DATA; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java index 75cd5fb9d3..2c1717d41e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.io.network.api.writer; +import org.apache.flink.runtime.checkpoint.channel.ChannelStateReader; import org.apache.flink.runtime.io.AvailabilityProvider; import org.apache.flink.runtime.io.network.buffer.BufferBuilder; import org.apache.flink.runtime.io.network.buffer.BufferConsumer; @@ -42,6 +43,12 @@ public interface ResultPartitionWriter extends AutoCloseable, AvailabilityProvid */ void setup() throws IOException; + /** + * Loads the previous output states with the given reader for unaligned checkpoint. + * It should be done before task processing the inputs. + */ + void initializeState(ChannelStateReader stateReader) throws IOException, InterruptedException; + ResultPartitionID getPartitionId(); int getNumberOfSubpartitions(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java index ecf695696a..070089d19c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java @@ -19,9 +19,12 @@ package org.apache.flink.runtime.io.network.partition; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.runtime.checkpoint.channel.ChannelStateReader; +import org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult; import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferBuilder; import org.apache.flink.runtime.io.network.buffer.BufferConsumer; import org.slf4j.Logger; @@ -52,7 +55,7 @@ import static org.apache.flink.util.Preconditions.checkState; * {@link PipelinedSubpartitionView#notifyDataAvailable() notification} for any * {@link BufferConsumer} present in the queue. */ -class PipelinedSubpartition extends ResultSubpartition { +public class PipelinedSubpartition extends ResultSubpartition { private static final Logger LOG = LoggerFactory.getLogger(PipelinedSubpartition.class); @@ -89,6 +92,23 @@ class PipelinedSubpartition extends ResultSubpartition { super(index, parent); } + @Override + public void initializeState(ChannelStateReader stateReader) throws IOException, InterruptedException { + for (ReadResult readResult = ReadResult.HAS_MORE_DATA; readResult == ReadResult.HAS_MORE_DATA;) { + BufferBuilder bufferBuilder = parent.getBufferPool().requestBufferBuilderBlocking(); + BufferConsumer bufferConsumer = bufferBuilder.createBufferConsumer(); + readResult = stateReader.readOutputData(subpartitionInfo, bufferBuilder); + + // check whether there are some states data filled in this time + if (bufferConsumer.isDataAvailable()) { + add(bufferConsumer); + bufferBuilder.finish(); + } else { + bufferConsumer.close(); + } + } + } + @Override public boolean add(BufferConsumer bufferConsumer) { return add(bufferConsumer, false); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionView.java index febbfbd0c0..ee837d54b5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionView.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionView.java @@ -29,7 +29,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull; /** * View over a pipelined in-memory only subpartition. */ -class PipelinedSubpartitionView implements ResultSubpartitionView { +public class PipelinedSubpartitionView implements ResultSubpartitionView { /** The subpartition this view belongs to. */ private final PipelinedSubpartition parent; @@ -39,7 +39,7 @@ class PipelinedSubpartitionView implements ResultSubpartitionView { /** Flag indicating whether this view has been released. */ private final AtomicBoolean isReleased; - PipelinedSubpartitionView(PipelinedSubpartition parent, BufferAvailabilityListener listener) { + public PipelinedSubpartitionView(PipelinedSubpartition parent, BufferAvailabilityListener listener) { this.parent = checkNotNull(parent); this.availabilityListener = checkNotNull(listener); this.isReleased = new AtomicBoolean(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java index ccd3fa9ab9..bb925fb58d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.io.network.partition; +import org.apache.flink.runtime.checkpoint.channel.ChannelStateReader; import org.apache.flink.runtime.executiongraph.IntermediateResultPartition; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.io.network.buffer.Buffer; @@ -150,6 +151,13 @@ public class ResultPartition implements ResultPartitionWriter, BufferPoolOwner { partitionManager.registerResultPartition(this); } + @Override + public void initializeState(ChannelStateReader stateReader) throws IOException, InterruptedException { + for (ResultSubpartition subpartition : subpartitions) { + subpartition.initializeState(stateReader); + } + } + public String getOwningTaskName() { return owningTaskName; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java index d139df00e2..d0256a12ba 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.io.network.partition; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.runtime.checkpoint.channel.ChannelStateReader; import org.apache.flink.runtime.checkpoint.channel.ResultSubpartitionInfo; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferConsumer; @@ -76,6 +77,9 @@ public abstract class ResultSubpartition { parent.onConsumedSubpartition(index); } + public void initializeState(ChannelStateReader stateReader) throws IOException, InterruptedException { + } + /** * Adds the given buffer. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ConsumableNotifyingResultPartitionWriterDecorator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ConsumableNotifyingResultPartitionWriterDecorator.java index 8b1d97d7c9..ada45cbeb2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ConsumableNotifyingResultPartitionWriterDecorator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ConsumableNotifyingResultPartitionWriterDecorator.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.taskmanager; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.checkpoint.channel.ChannelStateReader; import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.io.network.buffer.BufferBuilder; @@ -88,6 +89,11 @@ public class ConsumableNotifyingResultPartitionWriterDecorator implements Result partitionWriter.setup(); } + @Override + public void initializeState(ChannelStateReader stateReader) throws IOException, InterruptedException { + partitionWriter.initializeState(stateReader); + } + @Override public boolean addBufferConsumer(BufferConsumer bufferConsumer, int subpartitionIndex) throws IOException { boolean success = partitionWriter.addBufferConsumer(bufferConsumer, subpartitionIndex); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterTest.java index 4964d93f47..867f5910ea 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterTest.java @@ -25,6 +25,7 @@ import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.core.memory.MemorySegmentFactory; import org.apache.flink.runtime.checkpoint.CheckpointOptions; +import org.apache.flink.runtime.checkpoint.channel.ChannelStateReader; import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.io.network.api.CheckpointBarrier; import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; @@ -34,6 +35,7 @@ import org.apache.flink.runtime.io.network.api.serialization.RecordSerializer.Se import org.apache.flink.runtime.io.network.api.serialization.SpillingAdaptiveSpanningRecordDeserializer; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferBuilder; +import org.apache.flink.runtime.io.network.buffer.BufferBuilderAndConsumerTest; import org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils; import org.apache.flink.runtime.io.network.buffer.BufferConsumer; import org.apache.flink.runtime.io.network.buffer.BufferPool; @@ -41,8 +43,15 @@ import org.apache.flink.runtime.io.network.buffer.BufferProvider; import org.apache.flink.runtime.io.network.buffer.BufferRecycler; import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; import org.apache.flink.runtime.io.network.partition.MockResultPartitionWriter; +import org.apache.flink.runtime.io.network.partition.NoOpBufferAvailablityListener; import org.apache.flink.runtime.io.network.partition.NoOpResultPartitionConsumableNotifier; +import org.apache.flink.runtime.io.network.partition.PipelinedSubpartition; +import org.apache.flink.runtime.io.network.partition.PipelinedSubpartitionView; +import org.apache.flink.runtime.io.network.partition.ResultPartition; import org.apache.flink.runtime.io.network.partition.ResultPartitionBuilder; +import org.apache.flink.runtime.io.network.partition.ResultPartitionTest; +import org.apache.flink.runtime.io.network.partition.ResultSubpartition; +import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.runtime.io.network.util.DeserializationUtils; import org.apache.flink.runtime.io.network.util.TestPooledBufferProvider; @@ -464,6 +473,59 @@ public class RecordWriterTest { } } + @Test + public void testEmitRecordWithPartitionStateRecovery() throws Exception { + final int totalBuffers = 10; // enough for both states and normal records + final int totalStates = 2; + final int[] states = {1, 2, 3, 4}; + final int[] records = {5, 6, 7, 8}; + final int bufferSize = states.length * Integer.BYTES; + + final NetworkBufferPool globalPool = new NetworkBufferPool(totalBuffers, bufferSize, 1); + final ChannelStateReader stateReader = new ResultPartitionTest.FiniteChannelStateReader(totalStates, states); + final ResultPartition partition = new ResultPartitionBuilder() + .setNetworkBufferPool(globalPool) + .build(); + final RecordWriter recordWriter = new RecordWriterBuilder().build(partition); + + try { + partition.setup(); + partition.initializeState(stateReader); + + for (int record: records) { + // the record length 4 is also written into buffer for every emit + recordWriter.broadcastEmit(new IntValue(record)); + } + + // every buffer can contain 2 int records with 2 int length(4) + final int[][] expectedRecordsInBuffer = {{4, 5, 4, 6}, {4, 7, 4, 8}}; + + for (ResultSubpartition subpartition : partition.getAllPartitions()) { + // create the view to consume all the buffers with states and records + final ResultSubpartitionView view = new PipelinedSubpartitionView( + (PipelinedSubpartition) subpartition, + new NoOpBufferAvailablityListener()); + + int numConsumedBuffers = 0; + ResultSubpartition.BufferAndBacklog bufferAndBacklog; + while ((bufferAndBacklog = view.getNextBuffer()) != null) { + Buffer buffer = bufferAndBacklog.buffer(); + int[] expected = numConsumedBuffers < totalStates ? states : expectedRecordsInBuffer[numConsumedBuffers - totalStates]; + BufferBuilderAndConsumerTest.assertContent(buffer, partition.getBufferPool(), expected); + + buffer.recycleBuffer(); + numConsumedBuffers++; + } + + assertEquals(totalStates + expectedRecordsInBuffer.length, numConsumedBuffers); + } + } finally { + // cleanup + globalPool.destroyAllBufferPools(); + globalPool.destroy(); + } + } + private void verifyBroadcastBufferOrEventIndependence(boolean broadcastEvent) throws Exception { @SuppressWarnings("unchecked") ArrayDeque[] queues = new ArrayDeque[]{new ArrayDeque(), new ArrayDeque()}; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderAndConsumerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderAndConsumerTest.java index 3975a71f72..1033c5e3fd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderAndConsumerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderAndConsumerTest.java @@ -164,7 +164,7 @@ public class BufferBuilderAndConsumerTest { public void buildEmptyBuffer() { Buffer buffer = buildSingleBuffer(createBufferBuilder()); assertEquals(0, buffer.getSize()); - assertContent(buffer); + assertContent(buffer, FreeingBufferRecycler.INSTANCE); } @Test @@ -240,7 +240,7 @@ public class BufferBuilderAndConsumerTest { assertTrue(bufferConsumer.isFinished()); } - private static ByteBuffer toByteBuffer(int... data) { + public static ByteBuffer toByteBuffer(int... data) { ByteBuffer byteBuffer = ByteBuffer.allocate(data.length * Integer.BYTES); byteBuffer.asIntBuffer().put(data); return byteBuffer; @@ -250,18 +250,18 @@ public class BufferBuilderAndConsumerTest { assertFalse(actualConsumer.isFinished()); Buffer buffer = actualConsumer.build(); assertFalse(buffer.isRecycled()); - assertContent(buffer, expected); + assertContent(buffer, FreeingBufferRecycler.INSTANCE, expected); assertEquals(expected.length * Integer.BYTES, buffer.getSize()); buffer.recycleBuffer(); } - private static void assertContent(Buffer actualBuffer, int... expected) { + public static void assertContent(Buffer actualBuffer, BufferRecycler recycler, int... expected) { IntBuffer actualIntBuffer = actualBuffer.getNioBufferReadable().asIntBuffer(); int[] actual = new int[actualIntBuffer.limit()]; actualIntBuffer.get(actual); assertArrayEquals(expected, actual); - assertEquals(FreeingBufferRecycler.INSTANCE, actualBuffer.getRecycler()); + assertEquals(recycler, actualBuffer.getRecycler()); } private static BufferBuilder createBufferBuilder() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MockResultPartitionWriter.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MockResultPartitionWriter.java index fd6c1f8e37..9fd8205685 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MockResultPartitionWriter.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MockResultPartitionWriter.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.io.network.partition; +import org.apache.flink.runtime.checkpoint.channel.ChannelStateReader; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.io.network.buffer.BufferBuilder; import org.apache.flink.runtime.io.network.buffer.BufferConsumer; @@ -38,6 +39,10 @@ public class MockResultPartitionWriter implements ResultPartitionWriter { public void setup() { } + @Override + public void initializeState(ChannelStateReader stateReader) { + } + @Override public ResultPartitionID getPartitionId() { return partitionId; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/NoOpBufferAvailablityListener.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/NoOpBufferAvailablityListener.java index 41629754d7..7fbd43e9e3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/NoOpBufferAvailablityListener.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/NoOpBufferAvailablityListener.java @@ -21,7 +21,7 @@ package org.apache.flink.runtime.io.network.partition; /** * Test implementation of {@link BufferAvailabilityListener}. */ -class NoOpBufferAvailablityListener implements BufferAvailabilityListener { +public class NoOpBufferAvailablityListener implements BufferAvailabilityListener { @Override public void notifyDataAvailable() { } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java index 011aa72862..f3e512fb83 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java @@ -19,13 +19,17 @@ package org.apache.flink.runtime.io.network.partition; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.checkpoint.channel.ChannelStateReader; +import org.apache.flink.runtime.checkpoint.channel.InputChannelInfo; import org.apache.flink.runtime.checkpoint.channel.ResultSubpartitionInfo; import org.apache.flink.runtime.io.disk.FileChannelManager; import org.apache.flink.runtime.io.disk.FileChannelManagerImpl; import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; import org.apache.flink.runtime.io.network.NettyShuffleEnvironmentBuilder; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; +import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferBuilder; +import org.apache.flink.runtime.io.network.buffer.BufferBuilderAndConsumerTest; import org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils; import org.apache.flink.runtime.io.network.buffer.BufferConsumer; import org.apache.flink.runtime.io.network.buffer.BufferPool; @@ -42,6 +46,11 @@ import org.junit.Test; import java.io.IOException; import java.util.Collections; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledFinishedBufferConsumer; import static org.apache.flink.runtime.io.network.partition.PartitionTestUtils.createPartition; @@ -407,4 +416,120 @@ public class ResultPartitionTest { jobId, notifier)[0]; } + + @Test + public void testInitializeEmptyState() throws Exception { + final int totalBuffers = 2; + final NetworkBufferPool globalPool = new NetworkBufferPool(totalBuffers, 1, 1); + final ResultPartition partition = new ResultPartitionBuilder() + .setNetworkBufferPool(globalPool) + .build(); + final ChannelStateReader stateReader = ChannelStateReader.NO_OP; + try { + partition.setup(); + partition.initializeState(stateReader); + + for (ResultSubpartition subpartition : partition.getAllPartitions()) { + // no buffers are added into the queue for empty states + assertEquals(0, subpartition.getTotalNumberOfBuffers()); + } + + // destroy the local pool to verify that all the requested buffers by partition are recycled + partition.getBufferPool().lazyDestroy(); + assertEquals(totalBuffers, globalPool.getNumberOfAvailableMemorySegments()); + } finally { + // cleanup + globalPool.destroyAllBufferPools(); + globalPool.destroy(); + } + } + + @Test + public void testInitializeMoreStateThanBuffer() throws Exception { + final int totalBuffers = 2; // the total buffers are less than the requirement from total states + final int totalStates = 5; + final int[] states = {1, 2, 3, 4}; + final int bufferSize = states.length * Integer.BYTES; + + final NetworkBufferPool globalPool = new NetworkBufferPool(totalBuffers, bufferSize, 1); + final ChannelStateReader stateReader = new FiniteChannelStateReader(totalStates, states); + final ResultPartition partition = new ResultPartitionBuilder() + .setNetworkBufferPool(globalPool) + .build(); + final ExecutorService executor = Executors.newFixedThreadPool(1); + + try { + final Callable partitionConsumeTask = () -> { + for (ResultSubpartition subpartition : partition.getAllPartitions()) { + final ResultSubpartitionView view = new PipelinedSubpartitionView( + (PipelinedSubpartition) subpartition, + new NoOpBufferAvailablityListener()); + + int numConsumedBuffers = 0; + while (numConsumedBuffers != totalStates) { + ResultSubpartition.BufferAndBacklog bufferAndBacklog = view.getNextBuffer(); + if (bufferAndBacklog != null) { + Buffer buffer = bufferAndBacklog.buffer(); + BufferBuilderAndConsumerTest.assertContent(buffer, partition.getBufferPool(), states); + buffer.recycleBuffer(); + numConsumedBuffers++; + } else { + Thread.sleep(5); + } + } + } + return null; + }; + Future result = executor.submit(partitionConsumeTask); + + partition.setup(); + partition.initializeState(stateReader); + + // wait the partition consume task finish + result.get(20, TimeUnit.SECONDS); + + // destroy the local pool to verify that all the requested buffers by partition are recycled + partition.getBufferPool().lazyDestroy(); + assertEquals(totalBuffers, globalPool.getNumberOfAvailableMemorySegments()); + } finally { + // cleanup + executor.shutdown(); + globalPool.destroyAllBufferPools(); + globalPool.destroy(); + } + } + + /** + * The {@link ChannelStateReader} instance for restoring the specific number of states. + */ + public static final class FiniteChannelStateReader implements ChannelStateReader { + private final int totalStates; + private int numRestoredStates; + private final int[] states; + + public FiniteChannelStateReader(int totalStates, int[] states) { + this.totalStates = totalStates; + this.states = states; + } + + @Override + public ReadResult readInputData(InputChannelInfo info, Buffer buffer) { + return ReadResult.NO_MORE_DATA; + } + + @Override + public ReadResult readOutputData(ResultSubpartitionInfo info, BufferBuilder bufferBuilder) { + bufferBuilder.appendAndCommit(BufferBuilderAndConsumerTest.toByteBuffer(states)); + + if (++numRestoredStates < totalStates) { + return ReadResult.HAS_MORE_DATA; + } else { + return ReadResult.NO_MORE_DATA; + } + } + + @Override + public void close() { + } + } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java index d3a79b6eb1..e699994380 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java @@ -28,6 +28,7 @@ import org.apache.flink.runtime.checkpoint.CheckpointFailureReason; import org.apache.flink.runtime.checkpoint.CheckpointMetaData; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; import org.apache.flink.runtime.checkpoint.CheckpointOptions; +import org.apache.flink.runtime.checkpoint.channel.ChannelStateReader; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.execution.CancelTaskException; import org.apache.flink.runtime.execution.Environment; @@ -434,6 +435,14 @@ public abstract class StreamTask> // so that we avoid race conditions in the case that initializeState() // registers a timer, that fires before the open() is called. operatorChain.initializeStateAndOpenOperators(createStreamTaskStateInitializer()); + + ResultPartitionWriter[] writers = getEnvironment().getAllWriters(); + if (writers != null) { + //TODO we should get proper state reader from getEnvironment().getTaskStateManager().getChannelStateReader() + for (ResultPartitionWriter writer : writers) { + writer.initializeState(ChannelStateReader.NO_OP); + } + } }); isRunning = true; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java index 3fa9508ab6..c610a4fa7f 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java @@ -33,6 +33,7 @@ import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.runtime.checkpoint.StateObjectCollection; import org.apache.flink.runtime.checkpoint.SubtaskState; import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; +import org.apache.flink.runtime.checkpoint.channel.ChannelStateReader; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.concurrent.TestingUncaughtExceptionHandler; import org.apache.flink.runtime.execution.CancelTaskException; @@ -44,6 +45,7 @@ import org.apache.flink.runtime.io.network.NettyShuffleEnvironmentBuilder; import org.apache.flink.runtime.io.network.api.writer.AvailabilityTestResultPartitionWriter; import org.apache.flink.runtime.io.network.api.writer.RecordWriter; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; +import org.apache.flink.runtime.io.network.partition.MockResultPartitionWriter; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.operators.testutils.DummyEnvironment; @@ -895,6 +897,30 @@ public class StreamTaskTest extends TestLogger { } } + @Test + public void testInitializeResultPartitionState() throws Exception { + int numWriters = 2; + RecoveryResultPartition[] partitions = new RecoveryResultPartition[numWriters]; + for (int i = 0; i < numWriters; i++) { + partitions[i] = new RecoveryResultPartition(); + } + + MockEnvironment mockEnvironment = new MockEnvironmentBuilder().build(); + mockEnvironment.addOutputs(Arrays.asList(partitions)); + StreamTask task = new MockStreamTaskBuilder(mockEnvironment).build(); + + try { + task.beforeInvoke(); + + // output recovery should be done before task processing + for (RecoveryResultPartition resultPartition : partitions) { + assertTrue(resultPartition.isStateInitialized()); + } + } finally { + task.cleanUpInvoke(); + } + } + /** * Tests that some StreamTask methods are called only in the main task's thread. * Currently, the main task's thread is the thread that creates the task. @@ -1723,4 +1749,20 @@ public class StreamTaskTest extends TestLogger { throw new UnsupportedOperationException(); } } + + private static class RecoveryResultPartition extends MockResultPartitionWriter { + private boolean isStateInitialized; + + RecoveryResultPartition() { + } + + @Override + public void initializeState(ChannelStateReader stateReader) { + isStateInitialized = true; + } + + boolean isStateInitialized() { + return isStateInitialized; + } + } } -- Gitee From a9f3e2ffa17967c965a7d5b2cc91abe0e8a61ef8 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Tue, 7 Apr 2020 20:27:53 +0200 Subject: [PATCH 470/885] [FLINK-17010] Remove --quiet from s3 get operation The reason this failed with --quiet is that the test redirects stdout to get the sorted results: get_complete_result > "${TEST_DATA_DIR}/complete_result" 5f2f7d637d8073b1034f8c1124f52604745936eb removed the --quiet parameter from the s3 fetch command, which changed what is printed to stdout. The s3 command is printing which file it is downloading and this log message also ends up in the "result" output that is being sorted and checked. --- flink-end-to-end-tests/test-scripts/common_s3_operations.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-end-to-end-tests/test-scripts/common_s3_operations.sh b/flink-end-to-end-tests/test-scripts/common_s3_operations.sh index 7c8745e113..f784c7d58b 100644 --- a/flink-end-to-end-tests/test-scripts/common_s3_operations.sh +++ b/flink-end-to-end-tests/test-scripts/common_s3_operations.sh @@ -100,7 +100,7 @@ function s3_get_by_full_path_and_filename_prefix() { args="$args --recursive" fi local relative_dir=${1#$TEST_INFRA_DIR} - aws_cli s3 cp "s3://$IT_CASE_S3_BUCKET/$2" "/hostdir/${relative_dir}" $args + aws_cli s3 cp --quiet "s3://$IT_CASE_S3_BUCKET/$2" "/hostdir/${relative_dir}" $args } ################################### -- Gitee From 989bc02518d1be05d4f2260d9c4a67098df19063 Mon Sep 17 00:00:00 2001 From: "bowen.li" Date: Thu, 26 Mar 2020 14:15:39 -0700 Subject: [PATCH 471/885] [FLINK-16817][core] StringUtils.arrayToString() doesn't convert array of byte array correctly --- .../main/java/org/apache/flink/util/StringUtils.java | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/flink-core/src/main/java/org/apache/flink/util/StringUtils.java b/flink-core/src/main/java/org/apache/flink/util/StringUtils.java index a968a8af0e..8bc977b03f 100644 --- a/flink-core/src/main/java/org/apache/flink/util/StringUtils.java +++ b/flink-core/src/main/java/org/apache/flink/util/StringUtils.java @@ -142,6 +142,17 @@ public final class StringUtils { if (array instanceof Object[]) { return Arrays.toString((Object[]) array); } + // for array of byte array + if (array instanceof byte[][]) { + byte[][] b = (byte[][]) array; + String[] strs = new String[b.length]; + + for (int i = 0; i < b.length; i++) { + strs[i] = Arrays.toString(b[i]); + } + + return Arrays.toString(strs); + } if (array instanceof byte[]) { return Arrays.toString((byte[]) array); } -- Gitee From 3fd568a635ae23e530babe26c98425266e975663 Mon Sep 17 00:00:00 2001 From: "bowen.li" Date: Mon, 6 Apr 2020 21:53:32 -0700 Subject: [PATCH 472/885] [FLINK-16811][jdbc] introduce row converter API to JDBCDialect closes #11652 --- .../api/java/io/jdbc/JDBCInputFormat.java | 18 +++++-- .../api/java/io/jdbc/JDBCTableSource.java | 2 + .../api/java/io/jdbc/dialect/JDBCDialect.java | 9 ++++ .../java/io/jdbc/dialect/JDBCDialects.java | 20 ++++++++ .../converter/AbstractJDBCRowConverter.java | 49 +++++++++++++++++++ .../row/converter/DerbyRowConverter.java | 31 ++++++++++++ .../row/converter/JDBCRowConverter.java | 39 +++++++++++++++ .../row/converter/MySQLRowConverter.java | 31 ++++++++++++ .../row/converter/PostgresRowConverter.java | 31 ++++++++++++ .../flink/api/java/io/jdbc/JDBCFullTest.java | 5 +- .../api/java/io/jdbc/JDBCInputFormatTest.java | 22 +++++++++ .../api/java/io/jdbc/JdbcTestFixture.java | 5 ++ 12 files changed, 256 insertions(+), 6 deletions(-) create mode 100644 flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/source/row/converter/AbstractJDBCRowConverter.java create mode 100644 flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/source/row/converter/DerbyRowConverter.java create mode 100644 flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/source/row/converter/JDBCRowConverter.java create mode 100644 flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/source/row/converter/MySQLRowConverter.java create mode 100644 flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/source/row/converter/PostgresRowConverter.java diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java index 3f149178f2..8114a675a4 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.io.DefaultInputSplitAssigner; import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.common.io.RichInputFormat; import org.apache.flink.api.common.io.statistics.BaseStatistics; +import org.apache.flink.api.java.io.jdbc.source.row.converter.JDBCRowConverter; import org.apache.flink.api.java.io.jdbc.split.ParameterValuesProvider; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -111,6 +112,7 @@ public class JDBCInputFormat extends RichInputFormat implements private int resultSetType; private int resultSetConcurrency; private RowTypeInfo rowTypeInfo; + private JDBCRowConverter rowConverter; private transient Connection dbConn; private transient PreparedStatement statement; @@ -282,19 +284,17 @@ public class JDBCInputFormat extends RichInputFormat implements /** * Stores the next resultSet row in a tuple. * - * @param row row to be reused. + * @param reuse row to be reused. * @return row containing next {@link Row} * @throws java.io.IOException */ @Override - public Row nextRecord(Row row) throws IOException { + public Row nextRecord(Row reuse) throws IOException { try { if (!hasNext) { return null; } - for (int pos = 0; pos < row.getArity(); pos++) { - row.setField(pos, resultSet.getObject(pos + 1)); - } + Row row = rowConverter.convert(resultSet, reuse); //update hasNext after we've read the record hasNext = resultSet.next(); return row; @@ -403,6 +403,11 @@ public class JDBCInputFormat extends RichInputFormat implements return this; } + public JDBCInputFormatBuilder setRowConverter(JDBCRowConverter rowConverter) { + format.rowConverter = rowConverter; + return this; + } + public JDBCInputFormatBuilder setFetchSize(int fetchSize) { Preconditions.checkArgument(fetchSize == Integer.MIN_VALUE || fetchSize > 0, "Illegal value %s for fetchSize, has to be positive or Integer.MIN_VALUE.", fetchSize); @@ -434,6 +439,9 @@ public class JDBCInputFormat extends RichInputFormat implements if (format.rowTypeInfo == null) { throw new IllegalArgumentException("No " + RowTypeInfo.class.getSimpleName() + " supplied"); } + if (format.rowConverter == null) { + throw new IllegalArgumentException("No row converter supplied"); + } if (format.parameterValues == null) { LOG.debug("No input splitting configured (data will be read with parallelism 1)."); } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCTableSource.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCTableSource.java index 1775be775f..05e4796f94 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCTableSource.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCTableSource.java @@ -31,6 +31,7 @@ import org.apache.flink.table.sources.ProjectableTableSource; import org.apache.flink.table.sources.StreamTableSource; import org.apache.flink.table.sources.TableSource; import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.utils.TableConnectorUtils; import org.apache.flink.types.Row; @@ -177,6 +178,7 @@ public class JDBCTableSource implements " BETWEEN ? AND ?"; } builder.setQuery(query); + builder.setRowConverter(dialect.getRowConverter((RowType) producedDataType.getLogicalType())); return builder.finish(); } diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialect.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialect.java index 5dba839edd..627b6af6b6 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialect.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialect.java @@ -18,8 +18,10 @@ package org.apache.flink.api.java.io.jdbc.dialect; +import org.apache.flink.api.java.io.jdbc.source.row.converter.JDBCRowConverter; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.types.logical.RowType; import java.io.Serializable; import java.util.Arrays; @@ -38,6 +40,13 @@ public interface JDBCDialect extends Serializable { */ boolean canHandle(String url); + /** + * Get a row converter for the database according to the given row type. + * @param rowType the given row type + * @return a row converter for the database + */ + JDBCRowConverter getRowConverter(RowType rowType); + /** * Check if this dialect instance support a specific data type in table schema. * @param schema the table schema. diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialects.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialects.java index b09e36f73c..141aea25ff 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialects.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialects.java @@ -18,11 +18,16 @@ package org.apache.flink.api.java.io.jdbc.dialect; +import org.apache.flink.api.java.io.jdbc.source.row.converter.DerbyRowConverter; +import org.apache.flink.api.java.io.jdbc.source.row.converter.JDBCRowConverter; +import org.apache.flink.api.java.io.jdbc.source.row.converter.MySQLRowConverter; +import org.apache.flink.api.java.io.jdbc.source.row.converter.PostgresRowConverter; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.DecimalType; import org.apache.flink.table.types.logical.LogicalTypeRoot; +import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.TimestampType; import org.apache.flink.table.types.logical.VarBinaryType; @@ -142,6 +147,11 @@ public final class JDBCDialects { return url.startsWith("jdbc:derby:"); } + @Override + public JDBCRowConverter getRowConverter(RowType rowType) { + return new DerbyRowConverter(rowType); + } + @Override public Optional defaultDriverName() { return Optional.of("org.apache.derby.jdbc.EmbeddedDriver"); @@ -225,6 +235,11 @@ public final class JDBCDialects { return url.startsWith("jdbc:mysql:"); } + @Override + public JDBCRowConverter getRowConverter(RowType rowType) { + return new MySQLRowConverter(rowType); + } + @Override public Optional defaultDriverName() { return Optional.of("com.mysql.jdbc.Driver"); @@ -326,6 +341,11 @@ public final class JDBCDialects { return url.startsWith("jdbc:postgresql:"); } + @Override + public JDBCRowConverter getRowConverter(RowType rowType) { + return new PostgresRowConverter(rowType); + } + @Override public Optional defaultDriverName() { return Optional.of("org.postgresql.Driver"); diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/source/row/converter/AbstractJDBCRowConverter.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/source/row/converter/AbstractJDBCRowConverter.java new file mode 100644 index 0000000000..f03b7d7784 --- /dev/null +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/source/row/converter/AbstractJDBCRowConverter.java @@ -0,0 +1,49 @@ +/* + * 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.api.java.io.jdbc.source.row.converter; + +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.Row; + +import java.sql.ResultSet; +import java.sql.SQLException; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Base for all row converters. + */ +public abstract class AbstractJDBCRowConverter implements JDBCRowConverter { + + protected final RowType rowType; + + public AbstractJDBCRowConverter(RowType rowType) { + this.rowType = checkNotNull(rowType); + } + + @Override + public Row convert(ResultSet resultSet, Row reuse) throws SQLException { + for (int pos = 0; pos < rowType.getFieldCount(); pos++) { + Object v = resultSet.getObject(pos + 1); + reuse.setField(pos, v); + } + + return reuse; + } +} diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/source/row/converter/DerbyRowConverter.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/source/row/converter/DerbyRowConverter.java new file mode 100644 index 0000000000..ff84a84497 --- /dev/null +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/source/row/converter/DerbyRowConverter.java @@ -0,0 +1,31 @@ +/* + * 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.api.java.io.jdbc.source.row.converter; + +import org.apache.flink.table.types.logical.RowType; + +/** + * Row converter for Derby. + */ +public class DerbyRowConverter extends AbstractJDBCRowConverter { + + public DerbyRowConverter(RowType rowType) { + super(rowType); + } +} diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/source/row/converter/JDBCRowConverter.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/source/row/converter/JDBCRowConverter.java new file mode 100644 index 0000000000..b29011fdf9 --- /dev/null +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/source/row/converter/JDBCRowConverter.java @@ -0,0 +1,39 @@ +/* + * 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.api.java.io.jdbc.source.row.converter; + +import org.apache.flink.types.Row; + +import java.io.Serializable; +import java.sql.ResultSet; +import java.sql.SQLException; + +/** + * Convert row from JDBC result set to a Flink row. + */ +public interface JDBCRowConverter extends Serializable { + + /** + * Convert data retrieved from {@link ResultSet} to {@link Row}. + * + * @param resultSet ResultSet from JDBC + * @param reuse The row to set + */ + Row convert(ResultSet resultSet, Row reuse) throws SQLException; +} diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/source/row/converter/MySQLRowConverter.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/source/row/converter/MySQLRowConverter.java new file mode 100644 index 0000000000..f5a87957a9 --- /dev/null +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/source/row/converter/MySQLRowConverter.java @@ -0,0 +1,31 @@ +/* + * 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.api.java.io.jdbc.source.row.converter; + +import org.apache.flink.table.types.logical.RowType; + +/** + * Row converter for MySQL. + */ +public class MySQLRowConverter extends AbstractJDBCRowConverter { + + public MySQLRowConverter(RowType rowType) { + super(rowType); + } +} diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/source/row/converter/PostgresRowConverter.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/source/row/converter/PostgresRowConverter.java new file mode 100644 index 0000000000..102f079b83 --- /dev/null +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/source/row/converter/PostgresRowConverter.java @@ -0,0 +1,31 @@ +/* + * 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.api.java.io.jdbc.source.row.converter; + +import org.apache.flink.table.types.logical.RowType; + +/** + * Row converter for Postgres. + */ +public class PostgresRowConverter extends AbstractJDBCRowConverter { + + public PostgresRowConverter(RowType rowType) { + super(rowType); + } +} diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCFullTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCFullTest.java index 28ac1a1cc4..69e3527ff6 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCFullTest.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCFullTest.java @@ -21,6 +21,7 @@ package org.apache.flink.api.java.io.jdbc; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.io.jdbc.JDBCInputFormat.JDBCInputFormatBuilder; +import org.apache.flink.api.java.io.jdbc.dialect.JDBCDialects; import org.apache.flink.api.java.io.jdbc.split.NumericBetweenParametersProvider; import org.apache.flink.types.Row; @@ -38,6 +39,7 @@ import java.sql.Statement; import java.sql.Types; import static org.apache.flink.api.java.io.jdbc.JdbcTestFixture.OUTPUT_TABLE; +import static org.apache.flink.api.java.io.jdbc.JdbcTestFixture.ROW_TYPE; import static org.apache.flink.api.java.io.jdbc.JdbcTestFixture.ROW_TYPE_INFO; import static org.hamcrest.core.StringContains.containsString; @@ -83,7 +85,8 @@ public class JDBCFullTest extends JDBCDataTestBase { .setDrivername(getDbMetadata().getDriverClass()) .setDBUrl(getDbMetadata().getUrl()) .setQuery(JdbcTestFixture.SELECT_ALL_BOOKS) - .setRowTypeInfo(ROW_TYPE_INFO); + .setRowTypeInfo(ROW_TYPE_INFO) + .setRowConverter(JDBCDialects.get(getDbMetadata().getUrl()).get().getRowConverter(ROW_TYPE)); if (exploitParallelism) { final int fetchSize = 1; diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java index ca4f8e388a..1ade12a5b4 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java @@ -19,6 +19,7 @@ package org.apache.flink.api.java.io.jdbc; import org.apache.flink.api.java.io.jdbc.JdbcTestFixture.TestEntry; +import org.apache.flink.api.java.io.jdbc.dialect.JDBCDialects; import org.apache.flink.api.java.io.jdbc.split.GenericParameterValuesProvider; import org.apache.flink.api.java.io.jdbc.split.NumericBetweenParametersProvider; import org.apache.flink.api.java.io.jdbc.split.ParameterValuesProvider; @@ -35,6 +36,7 @@ import java.sql.DriverManager; import java.sql.ResultSet; import java.sql.SQLException; +import static org.apache.flink.api.java.io.jdbc.JdbcTestFixture.ROW_TYPE; import static org.apache.flink.api.java.io.jdbc.JdbcTestFixture.ROW_TYPE_INFO; import static org.apache.flink.api.java.io.jdbc.JdbcTestFixture.SELECT_ALL_BOOKS; import static org.apache.flink.api.java.io.jdbc.JdbcTestFixture.SELECT_EMPTY; @@ -127,6 +129,8 @@ public class JDBCInputFormatTest extends JDBCDataTestBase { .setQuery(SELECT_ALL_BOOKS) .setRowTypeInfo(ROW_TYPE_INFO) .setFetchSize(Integer.MIN_VALUE) + .setRowConverter( + JDBCDialects.get(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()).get().getRowConverter(ROW_TYPE)) .finish(); } @@ -137,6 +141,8 @@ public class JDBCInputFormatTest extends JDBCDataTestBase { .setDBUrl(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()) .setQuery(SELECT_ALL_BOOKS) .setRowTypeInfo(ROW_TYPE_INFO) + .setRowConverter( + JDBCDialects.get(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()).get().getRowConverter(ROW_TYPE)) .finish(); jdbcInputFormat.openInputFormat(); @@ -155,6 +161,8 @@ public class JDBCInputFormatTest extends JDBCDataTestBase { .setQuery(SELECT_ALL_BOOKS) .setRowTypeInfo(ROW_TYPE_INFO) .setFetchSize(desiredFetchSize) + .setRowConverter( + JDBCDialects.get(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()).get().getRowConverter(ROW_TYPE)) .finish(); jdbcInputFormat.openInputFormat(); Assert.assertEquals(desiredFetchSize, jdbcInputFormat.getStatement().getFetchSize()); @@ -168,6 +176,8 @@ public class JDBCInputFormatTest extends JDBCDataTestBase { .setDBUrl(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()) .setQuery(SELECT_ALL_BOOKS) .setRowTypeInfo(ROW_TYPE_INFO) + .setRowConverter( + JDBCDialects.get(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()).get().getRowConverter(ROW_TYPE)) .finish(); jdbcInputFormat.openInputFormat(); @@ -188,6 +198,8 @@ public class JDBCInputFormatTest extends JDBCDataTestBase { .setQuery(SELECT_ALL_BOOKS) .setRowTypeInfo(ROW_TYPE_INFO) .setAutoCommit(desiredAutoCommit) + .setRowConverter( + JDBCDialects.get(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()).get().getRowConverter(ROW_TYPE)) .finish(); jdbcInputFormat.openInputFormat(); @@ -203,6 +215,8 @@ public class JDBCInputFormatTest extends JDBCDataTestBase { .setQuery(SELECT_ALL_BOOKS) .setRowTypeInfo(ROW_TYPE_INFO) .setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE) + .setRowConverter( + JDBCDialects.get(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()).get().getRowConverter(ROW_TYPE)) .finish(); //this query does not exploit parallelism Assert.assertEquals(1, jdbcInputFormat.createInputSplits(1).length); @@ -235,6 +249,8 @@ public class JDBCInputFormatTest extends JDBCDataTestBase { .setRowTypeInfo(ROW_TYPE_INFO) .setParametersProvider(pramProvider) .setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE) + .setRowConverter( + JDBCDialects.get(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()).get().getRowConverter(ROW_TYPE)) .finish(); jdbcInputFormat.openInputFormat(); @@ -271,6 +287,8 @@ public class JDBCInputFormatTest extends JDBCDataTestBase { .setRowTypeInfo(ROW_TYPE_INFO) .setParametersProvider(pramProvider) .setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE) + .setRowConverter( + JDBCDialects.get(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()).get().getRowConverter(ROW_TYPE)) .finish(); jdbcInputFormat.openInputFormat(); @@ -307,6 +325,8 @@ public class JDBCInputFormatTest extends JDBCDataTestBase { .setRowTypeInfo(ROW_TYPE_INFO) .setParametersProvider(paramProvider) .setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE) + .setRowConverter( + JDBCDialects.get(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()).get().getRowConverter(ROW_TYPE)) .finish(); jdbcInputFormat.openInputFormat(); @@ -346,6 +366,8 @@ public class JDBCInputFormatTest extends JDBCDataTestBase { .setQuery(SELECT_EMPTY) .setRowTypeInfo(ROW_TYPE_INFO) .setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE) + .setRowConverter( + JDBCDialects.get(JdbcTestFixture.DERBY_EBOOKSHOP_DB.getUrl()).get().getRowConverter(ROW_TYPE)) .finish(); try { jdbcInputFormat.openInputFormat(); diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTestFixture.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTestFixture.java index 8dd5402153..a5af85c96f 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTestFixture.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JdbcTestFixture.java @@ -20,6 +20,7 @@ package org.apache.flink.api.java.io.jdbc; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.table.types.logical.RowType; import java.io.OutputStream; import java.io.Serializable; @@ -28,6 +29,8 @@ import java.sql.DriverManager; import java.sql.SQLException; import java.sql.Statement; +import static org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType; + /** * Test data and helper objects for JDBC tests. */ @@ -103,6 +106,8 @@ public class JdbcTestFixture { BasicTypeInfo.DOUBLE_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); + static final RowType ROW_TYPE = (RowType) fromLegacyInfoToDataType(ROW_TYPE_INFO).getLogicalType(); + private static String getCreateQuery(String tableName) { return "CREATE TABLE " + tableName + " (" + "id INT NOT NULL DEFAULT 0," + -- Gitee From 980e31dcc29ec6cc60ed59569f1f1cb7c47747b7 Mon Sep 17 00:00:00 2001 From: godfreyhe Date: Tue, 7 Apr 2020 21:15:21 +0800 Subject: [PATCH 473/885] [FLINK-16535][table] rename BatchTableSink#emitDataSet to BatchTableSink#consumeDataSet, and return DataSink --- .../flink/api/java/io/jdbc/JDBCAppendTableSink.java | 5 +++-- .../client/gateway/local/CollectBatchTableSink.java | 5 +++-- .../org/apache/flink/table/sinks/BatchTableSink.java | 12 +++++++++--- .../org/apache/flink/table/sinks/CsvTableSink.java | 4 ++-- .../factories/utils/TestCollectionTableFactory.scala | 3 ++- .../flink/table/api/internal/BatchTableEnvImpl.scala | 2 +- .../factories/utils/TestCollectionTableFactory.scala | 3 ++- .../runtime/batch/sql/PartitionableSinkITCase.scala | 6 ++++-- .../table/utils/MemoryTableSourceSinkUtil.scala | 3 ++- .../common/table/SpendReportTableSink.java | 5 +++-- 10 files changed, 31 insertions(+), 17 deletions(-) diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSink.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSink.java index fde2585ef6..412a0ce4a0 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSink.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCAppendTableSink.java @@ -21,6 +21,7 @@ package org.apache.flink.api.java.io.jdbc; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.operators.DataSink; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; @@ -69,8 +70,8 @@ public class JDBCAppendTableSink implements AppendStreamTableSink, BatchTab } @Override - public void emitDataSet(DataSet dataSet) { - dataSet.output(outputFormat); + public DataSink consumeDataSet(DataSet dataSet) { + return dataSet.output(outputFormat); } @Override diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/CollectBatchTableSink.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/CollectBatchTableSink.java index 801f1f7ca9..b2ef69525e 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/CollectBatchTableSink.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/CollectBatchTableSink.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.Utils; +import org.apache.flink.api.java.operators.DataSink; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.sinks.BatchTableSink; import org.apache.flink.table.sinks.OutputFormatTableSink; @@ -68,8 +69,8 @@ public class CollectBatchTableSink extends OutputFormatTableSink implements } @Override - public void emitDataSet(DataSet dataSet) { - dataSet + public DataSink consumeDataSet(DataSet dataSet) { + return dataSet .output(new Utils.CollectHelper<>(accumulatorName, serializer)) .name("SQL Client Batch Collect Sink"); } diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/BatchTableSink.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/BatchTableSink.java index f9413b7560..29221b4a4e 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/BatchTableSink.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/BatchTableSink.java @@ -18,10 +18,13 @@ package org.apache.flink.table.sinks; +import org.apache.flink.api.common.Plan; import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.operators.DataSink; import org.apache.flink.table.api.Table; -/** Defines an external {@link TableSink} to emit a batch {@link Table}. +/** + * Defines an external {@link TableSink} to emit a batch {@link Table}. * * @param Type of {@link DataSet} that this {@link TableSink} expects and supports. * @@ -30,6 +33,9 @@ import org.apache.flink.table.api.Table; @Deprecated public interface BatchTableSink extends TableSink { - /** Emits the DataSet. */ - void emitDataSet(DataSet dataSet); + /** + * Consumes the DataSet and return the {@link DataSink}. + * The returned {@link DataSink} will be used to generate {@link Plan}. + */ + DataSink consumeDataSet(DataSet dataSet); } diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSink.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSink.java index dfe5e8763f..4ea61ee30f 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSink.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSink.java @@ -112,7 +112,7 @@ public class CsvTableSink implements BatchTableSink, AppendStreamTableSink< } @Override - public void emitDataSet(DataSet dataSet) { + public DataSink consumeDataSet(DataSet dataSet) { MapOperator csvRows = dataSet.map(new CsvFormatter(fieldDelim == null ? "," : fieldDelim)); @@ -128,7 +128,7 @@ public class CsvTableSink implements BatchTableSink, AppendStreamTableSink< sink.setParallelism(numFiles); } - sink.name(TableConnectorUtils.generateRuntimeName(CsvTableSink.class, fieldNames)); + return sink.name(TableConnectorUtils.generateRuntimeName(CsvTableSink.class, fieldNames)); } @Override diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/factories/utils/TestCollectionTableFactory.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/factories/utils/TestCollectionTableFactory.scala index 228ea3c5f6..f1ae9428d0 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/factories/utils/TestCollectionTableFactory.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/factories/utils/TestCollectionTableFactory.scala @@ -22,6 +22,7 @@ import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.common.typeutils.TypeSerializer import org.apache.flink.api.java.io.{CollectionInputFormat, LocalCollectionOutputFormat} +import org.apache.flink.api.java.operators.DataSink import org.apache.flink.api.java.{DataSet, ExecutionEnvironment} import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.datastream.{DataStream, DataStreamSink, DataStreamSource} @@ -165,7 +166,7 @@ object TestCollectionTableFactory { class CollectionTableSink(val schema: TableSchema) extends BatchTableSink[Row] with AppendStreamTableSink[Row] { - override def emitDataSet(dataSet: DataSet[Row]): Unit = { + override def consumeDataSet(dataSet: DataSet[Row]): DataSink[_] = { dataSet.output(new LocalCollectionOutputFormat[Row](RESULT)).setParallelism(1) } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/BatchTableEnvImpl.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/BatchTableEnvImpl.scala index 9fe93cb340..19b26fcfe0 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/BatchTableEnvImpl.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/BatchTableEnvImpl.scala @@ -130,7 +130,7 @@ abstract class BatchTableEnvImpl( // translate the Table into a DataSet and provide the type that the TableSink expects. val result: DataSet[T] = translate(table)(outputType) // Give the DataSet to the TableSink to emit it. - batchSink.emitDataSet(result) + batchSink.consumeDataSet(result) case boundedSink: OutputFormatTableSink[T] => val outputType = fromDataTypeToLegacyInfo(sink.getConsumedDataType) .asInstanceOf[TypeInformation[T]] diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/factories/utils/TestCollectionTableFactory.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/factories/utils/TestCollectionTableFactory.scala index bd31d46419..ded1cbdbcf 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/factories/utils/TestCollectionTableFactory.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/factories/utils/TestCollectionTableFactory.scala @@ -22,6 +22,7 @@ import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.common.typeutils.TypeSerializer import org.apache.flink.api.java.io.{CollectionInputFormat, LocalCollectionOutputFormat} +import org.apache.flink.api.java.operators.DataSink import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.api.java.{DataSet, ExecutionEnvironment} import org.apache.flink.configuration.Configuration @@ -176,7 +177,7 @@ object TestCollectionTableFactory { class CollectionTableSink(val outputType: RowTypeInfo) extends BatchTableSink[Row] with AppendStreamTableSink[Row] { - override def emitDataSet(dataSet: DataSet[Row]): Unit = { + override def consumeDataSet(dataSet: DataSet[Row]): DataSink[_] = { dataSet.output(new LocalCollectionOutputFormat[Row](RESULT)).setParallelism(1) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/PartitionableSinkITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/PartitionableSinkITCase.scala index 90a6f247f7..9032ed135e 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/PartitionableSinkITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/PartitionableSinkITCase.scala @@ -27,11 +27,12 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo.{INT_TYPE_INFO, LONG_T import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java import org.apache.flink.api.java.DataSet +import org.apache.flink.api.java.operators.DataSink import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.api.scala.ExecutionEnvironment import org.apache.flink.configuration.Configuration import org.apache.flink.table.api.scala.BatchTableEnvironment -import org.apache.flink.table.api.{DataTypes, SqlDialect, TableSchema} +import org.apache.flink.table.api.{DataTypes, TableSchema} import org.apache.flink.table.factories.utils.TestCollectionTableFactory.TestCollectionInputFormat import org.apache.flink.table.runtime.batch.sql.PartitionableSinkITCase._ import org.apache.flink.table.sinks.{BatchTableSink, PartitionableTableSink, TableSink} @@ -39,6 +40,7 @@ import org.apache.flink.table.sources.BatchTableSource import org.apache.flink.table.types.logical.{BigIntType, IntType, VarCharType} import org.apache.flink.test.util.AbstractTestBase import org.apache.flink.types.Row + import org.junit.Assert.assertEquals import org.junit.rules.ExpectedException import org.junit.{Before, Rule, Test} @@ -154,7 +156,7 @@ class PartitionableSinkITCase extends AbstractTestBase { staticPartitions } - override def emitDataSet(dataSet: DataSet[Row]): Unit = { + override def consumeDataSet(dataSet: DataSet[Row]): DataSink[_] = { dataSet.map(new MapFunction[Row, String] { override def map(value: Row): String = value.toString }).output(new CollectionOutputFormat) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MemoryTableSourceSinkUtil.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MemoryTableSourceSinkUtil.scala index a835bba752..034e9ca3ea 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MemoryTableSourceSinkUtil.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MemoryTableSourceSinkUtil.scala @@ -20,6 +20,7 @@ package org.apache.flink.table.utils import org.apache.flink.api.common.io.{OutputFormat, RichOutputFormat} import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.operators.DataSink import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.api.java.{DataSet, ExecutionEnvironment} import org.apache.flink.configuration.Configuration @@ -111,7 +112,7 @@ object MemoryTableSourceSinkUtil { new UnsafeMemoryAppendTableSink } - override def emitDataSet(dataSet: DataSet[Row]): Unit = { + override def consumeDataSet(dataSet: DataSet[Row]): DataSink[_] = { dataSet .output(new MemoryCollectionOutputFormat) .name(TableConnectorUtils.generateRuntimeName(this.getClass, getTableSchema.getFieldNames)) diff --git a/flink-walkthroughs/flink-walkthrough-common/src/main/java/org/apache/flink/walkthrough/common/table/SpendReportTableSink.java b/flink-walkthroughs/flink-walkthrough-common/src/main/java/org/apache/flink/walkthrough/common/table/SpendReportTableSink.java index a143306d07..5073d90e1f 100644 --- a/flink-walkthroughs/flink-walkthrough-common/src/main/java/org/apache/flink/walkthrough/common/table/SpendReportTableSink.java +++ b/flink-walkthroughs/flink-walkthrough-common/src/main/java/org/apache/flink/walkthrough/common/table/SpendReportTableSink.java @@ -22,6 +22,7 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.operators.DataSink; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.table.api.TableSchema; @@ -51,8 +52,8 @@ public class SpendReportTableSink implements AppendStreamTableSink, BatchTa } @Override - public void emitDataSet(DataSet dataSet) { - dataSet + public DataSink consumeDataSet(DataSet dataSet) { + return dataSet .map(SpendReportTableSink::format) .output(new LoggerOutputFormat()); } -- Gitee From dd6d40fd82b5ac945a95783f1a50cd35fdeadbb1 Mon Sep 17 00:00:00 2001 From: Zhenghua Gao Date: Wed, 8 Apr 2020 11:12:12 +0800 Subject: [PATCH 474/885] [FLINK-16632][table-planner-blink] Fix SqlDateTimeUtils#toSqlTimestamp(String, String) may yield incorrect result This closes #11654 --- .../expressions/TemporalTypesTest.scala | 20 +++++++- .../runtime/functions/SqlDateTimeUtils.java | 47 ++++++++++++++----- 2 files changed, 54 insertions(+), 13 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala index 42eb09702b..dfb977f847 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala @@ -118,8 +118,11 @@ class TemporalTypesTest extends ExpressionTestBase { testSqlApi( "CAST('1999-9-10 05:20:10.123456' AS TIMESTAMP)", - "1999-09-10 05:20:10.123456" - ) + "1999-09-10 05:20:10.123456") + + testSqlApi( + "CAST('1999-9-10' AS TIMESTAMP)", + "1999-09-10 00:00:00.000000") } @Test @@ -1122,6 +1125,19 @@ class TemporalTypesTest extends ExpressionTestBase { testSqlApi("TO_TIMESTAMP('abc')", "null") + // TO_TIMESTAMP should complement YEAR/MONTH/DAY/HOUR/MINUTE/SECOND/NANO_OF_SECOND + testSqlApi( + "TO_TIMESTAMP('2000020210', 'yyyyMMddHH')", + "2000-02-02 10:00:00.000") + + testSqlApi( + "TO_TIMESTAMP('20000202 59:59.1234567', 'yyyyMMdd mm:ss.SSSSSSS')", + "2000-02-02 00:59:59.1234567") + + testSqlApi( + "TO_TIMESTAMP('1234567', 'SSSSSSS')", + "1970-01-01 00:00:00.1234567") + // CAST between two TIMESTAMPs testSqlApi( "CAST(TIMESTAMP '1970-01-01 00:00:00.123456789' AS TIMESTAMP(6))", diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlDateTimeUtils.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlDateTimeUtils.java index a64ff98812..42f3308f1a 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlDateTimeUtils.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlDateTimeUtils.java @@ -45,9 +45,18 @@ import java.time.ZonedDateTime; import java.time.format.DateTimeFormatter; import java.time.format.DateTimeParseException; import java.time.temporal.ChronoUnit; +import java.time.temporal.TemporalAccessor; import java.util.Date; import java.util.TimeZone; +import static java.time.temporal.ChronoField.DAY_OF_MONTH; +import static java.time.temporal.ChronoField.HOUR_OF_DAY; +import static java.time.temporal.ChronoField.MINUTE_OF_HOUR; +import static java.time.temporal.ChronoField.MONTH_OF_YEAR; +import static java.time.temporal.ChronoField.NANO_OF_SECOND; +import static java.time.temporal.ChronoField.SECOND_OF_MINUTE; +import static java.time.temporal.ChronoField.YEAR; + /** * Utility functions for datetime types: date, time, timestamp. * Currently, it is a bit messy putting date time functions in various classes because @@ -245,19 +254,35 @@ public class SqlDateTimeUtils { DateTimeFormatter formatter = DATETIME_FORMATTER_CACHE.get(format); try { - if (dateStr.length() == 10) { - // Just a LocalDate - LocalDate ld = LocalDate.parse(dateStr, formatter); - return SqlTimestamp.fromLocalDateTime(LocalDateTime.of(ld, LocalTime.MIDNIGHT)); - } else { - LocalDateTime ldt = LocalDateTime.parse(dateStr, formatter); - return SqlTimestamp.fromLocalDateTime(ldt); - } + TemporalAccessor accessor = formatter.parse(dateStr); + // complement year with 1970 + int year = accessor.isSupported(YEAR) ? accessor.get(YEAR) : 1970; + // complement month with 1 + int month = accessor.isSupported(MONTH_OF_YEAR) ? accessor.get(MONTH_OF_YEAR) : 1; + // complement day with 1 + int day = accessor.isSupported(DAY_OF_MONTH) ? accessor.get(DAY_OF_MONTH) : 1; + // complement hour with 0 + int hour = accessor.isSupported(HOUR_OF_DAY) ? accessor.get(HOUR_OF_DAY) : 0; + // complement minute with 0 + int minute = accessor.isSupported(MINUTE_OF_HOUR) ? accessor.get(MINUTE_OF_HOUR) : 0; + // complement second with 0 + int second = accessor.isSupported(SECOND_OF_MINUTE) ? accessor.get(SECOND_OF_MINUTE) : 0; + // complement nano_of_second with 0 + int nanoOfSecond = accessor.isSupported(NANO_OF_SECOND) ? accessor.get(NANO_OF_SECOND) : 0; + LocalDateTime ldt = LocalDateTime.of(year, month, day, hour, minute, second, nanoOfSecond); + return SqlTimestamp.fromLocalDateTime(ldt); } catch (DateTimeParseException e) { - // fall back to support cases like '1999-9-10 05:20:10' + // fall back to support cases like '1999-9-10 05:20:10' or '1999-9-10' try { - Timestamp ts = Timestamp.valueOf(dateStr); - return SqlTimestamp.fromTimestamp(ts); + dateStr = dateStr.trim(); + int space = dateStr.indexOf(' '); + if (space >= 0) { + Timestamp ts = Timestamp.valueOf(dateStr); + return SqlTimestamp.fromTimestamp(ts); + } else { + java.sql.Date dt = java.sql.Date.valueOf(dateStr); + return SqlTimestamp.fromLocalDateTime(LocalDateTime.of(dt.toLocalDate(), LocalTime.MIDNIGHT)); + } } catch (IllegalArgumentException ie) { return null; } -- Gitee From 5303a215213335c6574e83911ee49bceac851d25 Mon Sep 17 00:00:00 2001 From: "kevin.cyj" Date: Wed, 8 Apr 2020 11:42:48 +0800 Subject: [PATCH 475/885] [FLINK-17040][tests] Fix SavepointWriterITCase failure because of UnsupportedOperationException --- .../apache/flink/state/api/runtime/SavepointEnvironment.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/SavepointEnvironment.java b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/SavepointEnvironment.java index 95ab72dcd4..71d252c965 100644 --- a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/SavepointEnvironment.java +++ b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/SavepointEnvironment.java @@ -239,7 +239,7 @@ public class SavepointEnvironment implements Environment { @Override public ResultPartitionWriter[] getAllWriters() { - throw new UnsupportedOperationException(ERROR_MSG); + return new ResultPartitionWriter[0]; } @Override -- Gitee From 18787230ea0bb3b502a8002b9f4f0fa0afb85f63 Mon Sep 17 00:00:00 2001 From: "wenlong.lwl" Date: Fri, 27 Mar 2020 21:18:01 +0800 Subject: [PATCH 476/885] [FLINK-16864][metrics] Add IdleTime metric for task This pr adds an IdleTime metric which measures idle time of a task including the time cost for mail processor to wait for new mail and the time cost in record writer to waiting a new buffer. 1. when a job can not catch up with the speed of data generating, the vertex which idle time is near to zero is the bottle neck of the job. 2. when a job is not busy, idle time can be used to guide user how much he can scale down the job. --- docs/monitoring/metrics.md | 5 ++ docs/monitoring/metrics.zh.md | 5 ++ .../api/runtime/SavepointEnvironment.java | 3 +- .../api/writer/BroadcastRecordWriter.java | 8 +- .../writer/ChannelSelectorRecordWriter.java | 4 +- .../io/network/api/writer/RecordWriter.java | 27 ++++++- .../api/writer/ResultPartitionWriter.java | 8 ++ .../io/network/buffer/BufferProvider.java | 7 ++ .../io/network/buffer/LocalBufferPool.java | 5 ++ .../io/network/partition/ResultPartition.java | 6 ++ .../flink/runtime/metrics/MetricNames.java | 2 + .../metrics/groups/TaskIOMetricGroup.java | 7 ++ ...tifyingResultPartitionWriterDecorator.java | 5 ++ ...stractCollectingResultPartitionWriter.java | 5 ++ .../network/api/writer/RecordWriterTest.java | 81 ++++++++++++++++++- .../io/network/buffer/NoOpBufferPool.java | 5 ++ .../partition/MockResultPartitionWriter.java | 5 ++ .../util/TestPooledBufferProvider.java | 9 +++ .../metrics/groups/TaskIOMetricGroupTest.java | 2 + .../streaming/runtime/tasks/StreamTask.java | 1 + .../tasks/mailbox/MailboxProcessor.java | 23 +++++- .../mailbox/TaskMailboxProcessorTest.java | 48 +++++++++++ 22 files changed, 261 insertions(+), 10 deletions(-) diff --git a/docs/monitoring/metrics.md b/docs/monitoring/metrics.md index 53e0bedac6..203089230e 100644 --- a/docs/monitoring/metrics.md +++ b/docs/monitoring/metrics.md @@ -1440,6 +1440,11 @@ Certain RocksDB native metrics are available but disabled by default, you can fi Whether the task is back-pressured. Gauge + + idleTimeMsPerSecond + The time (in milliseconds) this task is idle (either has no data to process or it is back pressured) per second. + Meter + Task/Operator numRecordsIn diff --git a/docs/monitoring/metrics.zh.md b/docs/monitoring/metrics.zh.md index 58763c58eb..7deb0ad5d4 100644 --- a/docs/monitoring/metrics.zh.md +++ b/docs/monitoring/metrics.zh.md @@ -1440,6 +1440,11 @@ Certain RocksDB native metrics are available but disabled by default, you can fi Whether the task is back-pressured. Gauge + + idleTimeMsPerSecond + The time (in milliseconds) this task is idle (either has no data to process or it is back pressured) per second. + Meter + Task/Operator numRecordsIn diff --git a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/SavepointEnvironment.java b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/SavepointEnvironment.java index 71d252c965..4fbb78d3de 100644 --- a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/SavepointEnvironment.java +++ b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/SavepointEnvironment.java @@ -43,6 +43,7 @@ import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; import org.apache.flink.runtime.jobgraph.tasks.TaskOperatorEventGateway; import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; +import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.operators.coordination.OperatorEvent; import org.apache.flink.runtime.query.KvStateRegistry; import org.apache.flink.runtime.query.TaskKvStateRegistry; @@ -139,7 +140,7 @@ public class SavepointEnvironment implements Environment { @Override public TaskMetricGroup getMetricGroup() { - throw new UnsupportedOperationException(ERROR_MSG); + return UnregisteredMetricGroups.createUnregisteredTaskMetricGroup(); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/BroadcastRecordWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/BroadcastRecordWriter.java index b4999835d7..93bc2c4a2f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/BroadcastRecordWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/BroadcastRecordWriter.java @@ -84,7 +84,7 @@ public final class BroadcastRecordWriter extends R if (bufferBuilder != null) { for (int index = 0; index < numberOfChannels; index++) { if (index != targetChannelIndex) { - targetPartition.addBufferConsumer(bufferBuilder.createBufferConsumer(), index); + addBufferConsumer(bufferBuilder.createBufferConsumer(), index); } } } @@ -128,13 +128,13 @@ public final class BroadcastRecordWriter extends R public BufferBuilder requestNewBufferBuilder(int targetChannel) throws IOException, InterruptedException { checkState(bufferBuilder == null || bufferBuilder.isFinished()); - BufferBuilder builder = targetPartition.getBufferBuilder(); + BufferBuilder builder = getBufferBuilder(); if (randomTriggered) { - targetPartition.addBufferConsumer(builder.createBufferConsumer(), targetChannel); + addBufferConsumer(builder.createBufferConsumer(), targetChannel); } else { try (BufferConsumer bufferConsumer = builder.createBufferConsumer()) { for (int channel = 0; channel < numberOfChannels; channel++) { - targetPartition.addBufferConsumer(bufferConsumer.copy(), channel); + addBufferConsumer(bufferConsumer.copy(), channel); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ChannelSelectorRecordWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ChannelSelectorRecordWriter.java index 5f5e5964b9..8393409660 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ChannelSelectorRecordWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ChannelSelectorRecordWriter.java @@ -100,8 +100,8 @@ public final class ChannelSelectorRecordWriter ext public BufferBuilder requestNewBufferBuilder(int targetChannel) throws IOException, InterruptedException { checkState(bufferBuilders[targetChannel] == null || bufferBuilders[targetChannel].isFinished()); - BufferBuilder bufferBuilder = targetPartition.getBufferBuilder(); - targetPartition.addBufferConsumer(bufferBuilder.createBufferConsumer(), targetChannel); + BufferBuilder bufferBuilder = getBufferBuilder(); + addBufferConsumer(bufferBuilder.createBufferConsumer(), targetChannel); bufferBuilders[targetChannel] = bufferBuilder; return bufferBuilder; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java index 6c680f5981..0895bf4823 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java @@ -21,6 +21,8 @@ package org.apache.flink.runtime.io.network.api.writer; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.core.io.IOReadableWritable; import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MeterView; import org.apache.flink.metrics.SimpleCounter; import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.io.AvailabilityProvider; @@ -66,7 +68,7 @@ public abstract class RecordWriter implements Avai private static final Logger LOG = LoggerFactory.getLogger(RecordWriter.class); - protected final ResultPartitionWriter targetPartition; + private final ResultPartitionWriter targetPartition; protected final int numberOfChannels; @@ -78,6 +80,8 @@ public abstract class RecordWriter implements Avai private Counter numBuffersOut = new SimpleCounter(); + protected Meter idleTimeMsPerSecond = new MeterView(new SimpleCounter()); + private final boolean flushAlways; /** The thread that periodically flushes the output, to give an upper latency bound. */ @@ -182,6 +186,7 @@ public abstract class RecordWriter implements Avai public void setMetricGroup(TaskIOMetricGroup metrics) { numBytesOut = metrics.getNumBytesOutCounter(); numBuffersOut = metrics.getNumBuffersOutCounter(); + idleTimeMsPerSecond = metrics.getIdleTimeMsPerSecond(); } protected void finishBufferBuilder(BufferBuilder bufferBuilder) { @@ -276,6 +281,26 @@ public abstract class RecordWriter implements Avai } } + protected void addBufferConsumer(BufferConsumer consumer, int targetChannel) throws IOException { + targetPartition.addBufferConsumer(consumer, targetChannel); + } + + @VisibleForTesting + public BufferBuilder getBufferBuilder() throws IOException, InterruptedException { + BufferBuilder builder = targetPartition.tryGetBufferBuilder(); + if (builder == null) { + long start = System.currentTimeMillis(); + builder = targetPartition.getBufferBuilder(); + idleTimeMsPerSecond.markEvent(System.currentTimeMillis() - start); + } + return builder; + } + + @VisibleForTesting + public Meter getIdleTimeMsPerSecond() { + return idleTimeMsPerSecond; + } + // ------------------------------------------------------------------------ /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java index 2c1717d41e..2efc84949a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java @@ -60,6 +60,14 @@ public interface ResultPartitionWriter extends AutoCloseable, AvailabilityProvid */ BufferBuilder getBufferBuilder() throws IOException, InterruptedException; + + /** + * Try to request a {@link BufferBuilder} from this partition for writing data. + * + *

    Returns null if no buffer is available or the buffer provider has been destroyed. + */ + BufferBuilder tryGetBufferBuilder() throws IOException; + /** * Adds the bufferConsumer to the subpartition with the given index. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferProvider.java index 5e17b3087f..05518044a3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferProvider.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferProvider.java @@ -37,6 +37,13 @@ public interface BufferProvider extends AvailabilityProvider { */ Buffer requestBuffer() throws IOException; + /** + * Returns a {@link BufferBuilder} instance from the buffer provider. + * + *

    Returns null if no buffer is available or the buffer provider has been destroyed. + */ + BufferBuilder requestBufferBuilder() throws IOException; + /** * Returns a {@link BufferBuilder} instance from the buffer provider. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java index d7aef4c30c..a6af47176b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java @@ -204,6 +204,11 @@ class LocalBufferPool implements BufferPool { return toBuffer(requestMemorySegment()); } + @Override + public BufferBuilder requestBufferBuilder() throws IOException { + return toBufferBuilder(requestMemorySegment()); + } + @Override public BufferBuilder requestBufferBuilderBlocking() throws IOException, InterruptedException { return toBufferBuilder(requestMemorySegmentBlocking()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java index bb925fb58d..b41fabc2b7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java @@ -207,6 +207,12 @@ public class ResultPartition implements ResultPartitionWriter, BufferPoolOwner { return bufferPool.requestBufferBuilderBlocking(); } + @Override + public BufferBuilder tryGetBufferBuilder() throws IOException { + BufferBuilder bufferBuilder = bufferPool.requestBufferBuilder(); + return bufferBuilder; + } + @Override public boolean addBufferConsumer(BufferConsumer bufferConsumer, int subpartitionIndex) throws IOException { checkNotNull(bufferConsumer); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricNames.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricNames.java index 51702ba53c..9cad0ed316 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricNames.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricNames.java @@ -71,4 +71,6 @@ public class MetricNames { public static String currentInputWatermarkName(int index) { return String.format(IO_CURRENT_INPUT_WATERMARK_PATERN, index); } + + public static final String TASK_IDLE_TIME = "idleTimeMs" + SUFFIX_RATE; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TaskIOMetricGroup.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TaskIOMetricGroup.java index d3fa829915..e2b984a5c4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TaskIOMetricGroup.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TaskIOMetricGroup.java @@ -45,6 +45,7 @@ public class TaskIOMetricGroup extends ProxyMetricGroup { private final Meter numRecordsInRate; private final Meter numRecordsOutRate; private final Meter numBuffersOutRate; + private final Meter idleTimePerSecond; public TaskIOMetricGroup(TaskMetricGroup parent) { super(parent); @@ -61,6 +62,8 @@ public class TaskIOMetricGroup extends ProxyMetricGroup { this.numBuffersOut = counter(MetricNames.IO_NUM_BUFFERS_OUT); this.numBuffersOutRate = meter(MetricNames.IO_NUM_BUFFERS_OUT_RATE, new MeterView(numBuffersOut)); + + this.idleTimePerSecond = meter(MetricNames.TASK_IDLE_TIME, new MeterView(new SimpleCounter())); } public IOMetrics createSnapshot() { @@ -91,6 +94,10 @@ public class TaskIOMetricGroup extends ProxyMetricGroup { return numBuffersOut; } + public Meter getIdleTimeMsPerSecond() { + return idleTimePerSecond; + } + // ============================================================================================ // Metric Reuse // ============================================================================================ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ConsumableNotifyingResultPartitionWriterDecorator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ConsumableNotifyingResultPartitionWriterDecorator.java index ada45cbeb2..3e675333e6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ConsumableNotifyingResultPartitionWriterDecorator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ConsumableNotifyingResultPartitionWriterDecorator.java @@ -69,6 +69,11 @@ public class ConsumableNotifyingResultPartitionWriterDecorator implements Result return partitionWriter.getBufferBuilder(); } + @Override + public BufferBuilder tryGetBufferBuilder() throws IOException { + return partitionWriter.tryGetBufferBuilder(); + } + @Override public ResultPartitionID getPartitionId() { return partitionWriter.getPartitionId(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/AbstractCollectingResultPartitionWriter.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/AbstractCollectingResultPartitionWriter.java index 6ad31e30fa..4751dc29bd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/AbstractCollectingResultPartitionWriter.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/AbstractCollectingResultPartitionWriter.java @@ -49,6 +49,11 @@ public abstract class AbstractCollectingResultPartitionWriter extends MockResult return bufferProvider.requestBufferBuilderBlocking(); } + @Override + public BufferBuilder tryGetBufferBuilder() throws IOException { + return bufferProvider.requestBufferBuilder(); + } + @Override public synchronized boolean addBufferConsumer(BufferConsumer bufferConsumer, int targetChannel) throws IOException { checkState(targetChannel < getNumberOfSubpartitions()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterTest.java index 867f5910ea..98ff3b2e87 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterTest.java @@ -65,6 +65,7 @@ import org.apache.flink.testutils.serialization.types.Util; import org.apache.flink.types.IntValue; import org.apache.flink.util.XORShiftRandom; +import org.hamcrest.Matchers; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -83,11 +84,13 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicReference; import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.buildSingleBuffer; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; @@ -154,8 +157,21 @@ public class RecordWriterTest { } }; + Answer nonBlockingRequest = new Answer() { + @Override + public BufferBuilder answer(InvocationOnMock invocation) throws Throwable { + sync.countDown(); + + if (sync.getCount() == 1) { + return new BufferBuilder(memorySegment, recycler); + } + return null; + } + }; + BufferProvider bufferProvider = mock(BufferProvider.class); when(bufferProvider.requestBufferBuilderBlocking()).thenAnswer(request); + when(bufferProvider.requestBufferBuilder()).thenAnswer(nonBlockingRequest); ResultPartitionWriter partitionWriter = new RecyclingPartitionWriter(bufferProvider); @@ -192,7 +208,8 @@ public class RecordWriterTest { recordWriter.clearBuffers(); // Verify that buffer have been requested twice - verify(bufferProvider, times(2)).requestBufferBuilderBlocking(); + verify(bufferProvider, times(1)).requestBufferBuilderBlocking(); + verify(bufferProvider, times(2)).requestBufferBuilder(); // Verify that the written out buffer has only been recycled once // (by the partition writer). @@ -467,6 +484,7 @@ public class RecordWriterTest { buffer.recycleBuffer(); assertTrue(recordWriter.getAvailableFuture().isDone()); assertEquals(recordWriter.AVAILABLE, recordWriter.getAvailableFuture()); + } finally { localPool.lazyDestroy(); globalPool.destroy(); @@ -526,6 +544,57 @@ public class RecordWriterTest { } } + @Test + public void testIdleTime() throws IOException, InterruptedException { + // setup + final NetworkBufferPool globalPool = new NetworkBufferPool(10, 128, 2); + final BufferPool localPool = globalPool.createBufferPool(1, 1); + final ResultPartitionWriter resultPartition = new ResultPartitionBuilder() + .setBufferPoolFactory(p -> localPool) + .build(); + resultPartition.setup(); + final ResultPartitionWriter partitionWrapper = new ConsumableNotifyingResultPartitionWriterDecorator( + new NoOpTaskActions(), + new JobID(), + resultPartition, + new NoOpResultPartitionConsumableNotifier()); + final RecordWriter recordWriter = createRecordWriter(partitionWrapper); + BufferBuilder builder = recordWriter.getBufferBuilder(); + assertEquals(0, recordWriter.getIdleTimeMsPerSecond().getCount()); + + final Object runningLock = new Object(); + AtomicReference asyncRequestResult = new AtomicReference<>(); + final Thread requestThread = new Thread(new Runnable() { + @Override + public void run() { + try { + // notify that the request thread start to run. + synchronized (runningLock) { + runningLock.notify(); + } + // wait for buffer. + asyncRequestResult.set(recordWriter.getBufferBuilder()); + } catch (Exception e) { + } + } + }); + requestThread.start(); + + // wait until request thread start to run. + synchronized (runningLock) { + runningLock.wait(); + } + Thread.sleep(10); + //recycle the buffer + final Buffer buffer = BufferBuilderTestUtils.buildSingleBuffer(builder); + + buffer.recycleBuffer(); + requestThread.join(); + assertThat(recordWriter.getIdleTimeMsPerSecond().getCount(), Matchers.greaterThanOrEqualTo(10L)); + assertNotNull(asyncRequestResult.get()); + + } + private void verifyBroadcastBufferOrEventIndependence(boolean broadcastEvent) throws Exception { @SuppressWarnings("unchecked") ArrayDeque[] queues = new ArrayDeque[]{new ArrayDeque(), new ArrayDeque()}; @@ -614,6 +683,11 @@ public class RecordWriterTest { return bufferProvider.requestBufferBuilderBlocking(); } + @Override + public BufferBuilder tryGetBufferBuilder() throws IOException { + return bufferProvider.requestBufferBuilder(); + } + @Override public boolean addBufferConsumer(BufferConsumer buffer, int targetChannel) throws IOException { return queues[targetChannel].add(buffer); @@ -646,6 +720,11 @@ public class RecordWriterTest { public BufferBuilder getBufferBuilder() throws IOException, InterruptedException { return bufferProvider.requestBufferBuilderBlocking(); } + + @Override + public BufferBuilder tryGetBufferBuilder() throws IOException { + return bufferProvider.requestBufferBuilder(); + } } private static class ByteArrayIO implements IOReadableWritable { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NoOpBufferPool.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NoOpBufferPool.java index b8a866c4ac..aa237440bf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NoOpBufferPool.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NoOpBufferPool.java @@ -39,6 +39,11 @@ public class NoOpBufferPool implements BufferPool { throw new UnsupportedOperationException(); } + @Override + public BufferBuilder requestBufferBuilder() throws IOException { + throw new UnsupportedOperationException(); + } + @Override public BufferBuilder requestBufferBuilderBlocking() throws IOException, InterruptedException { throw new UnsupportedOperationException(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MockResultPartitionWriter.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MockResultPartitionWriter.java index 9fd8205685..2f9019338f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MockResultPartitionWriter.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MockResultPartitionWriter.java @@ -69,6 +69,11 @@ public class MockResultPartitionWriter implements ResultPartitionWriter { throw new UnsupportedOperationException(); } + @Override + public BufferBuilder tryGetBufferBuilder() throws IOException { + throw new UnsupportedOperationException(); + } + @Override public void flushAll() { } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestPooledBufferProvider.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestPooledBufferProvider.java index d7c94398ef..e0ed290c81 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestPooledBufferProvider.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestPooledBufferProvider.java @@ -66,6 +66,15 @@ public class TestPooledBufferProvider implements BufferProvider { return bufferFactory.create(); } + @Override + public BufferBuilder requestBufferBuilder() throws IOException { + Buffer buffer = requestBuffer(); + if (buffer != null) { + return new BufferBuilder(buffer.getMemorySegment(), buffer.getRecycler()); + } + return null; + } + private Buffer requestBufferBlocking() throws IOException, InterruptedException { Buffer buffer = buffers.poll(); if (buffer != null) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/TaskIOMetricGroupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/TaskIOMetricGroupTest.java index 36c1fda1b0..8578d069d1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/TaskIOMetricGroupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/TaskIOMetricGroupTest.java @@ -54,6 +54,7 @@ public class TaskIOMetricGroupTest { taskIO.getNumBytesInCounter().inc(100L); taskIO.getNumBytesOutCounter().inc(250L); taskIO.getNumBuffersOutCounter().inc(3L); + taskIO.getIdleTimeMsPerSecond().markEvent(2L); IOMetrics io = taskIO.createSnapshot(); assertEquals(32L, io.getNumRecordsIn()); @@ -61,5 +62,6 @@ public class TaskIOMetricGroupTest { assertEquals(100L, io.getNumBytesIn()); assertEquals(250L, io.getNumBytesOut()); assertEquals(3L, taskIO.getNumBuffersOutCounter().getCount()); + assertEquals(2L, taskIO.getIdleTimeMsPerSecond().getCount()); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java index e699994380..b566ded47d 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java @@ -267,6 +267,7 @@ public abstract class StreamTask> this.recordWriter = createRecordWriterDelegate(configuration, environment); this.actionExecutor = Preconditions.checkNotNull(actionExecutor); this.mailboxProcessor = new MailboxProcessor(this::processInput, mailbox, actionExecutor); + this.mailboxProcessor.initMetric(environment.getMetricGroup()); this.asyncExceptionHandler = new StreamTaskAsyncExceptionHandler(environment); this.asyncOperationsThreadPool = Executors.newCachedThreadPool( new ExecutorThreadFactory("AsyncOperations", uncaughtExceptionHandler)); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxProcessor.java index 382b6699dd..d23e843d69 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxProcessor.java @@ -19,6 +19,10 @@ package org.apache.flink.streaming.runtime.tasks.mailbox; import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MeterView; +import org.apache.flink.metrics.SimpleCounter; +import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; import org.apache.flink.streaming.api.operators.MailboxExecutor; import org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor; import org.apache.flink.util.ExceptionUtils; @@ -82,6 +86,8 @@ public class MailboxProcessor implements Closeable { private final StreamTaskActionExecutor actionExecutor; + private Meter idleTime = new MeterView(new SimpleCounter()); + public MailboxProcessor(MailboxDefaultAction mailboxDefaultAction) { this(mailboxDefaultAction, StreamTaskActionExecutor.IMMEDIATE); } @@ -128,6 +134,10 @@ public class MailboxProcessor implements Closeable { return new MailboxExecutorImpl(mailbox, priority, actionExecutor); } + public void initMetric(TaskMetricGroup metricGroup) { + idleTime = metricGroup.getIOMetricGroup().getIdleTimeMsPerSecond(); + } + /** * Lifecycle method to close the mailbox for action submission. */ @@ -269,7 +279,13 @@ public class MailboxProcessor implements Closeable { // If the default action is currently not available, we can run a blocking mailbox execution until the default // action becomes available again. while (isDefaultActionUnavailable() && isMailboxLoopRunning()) { - mailbox.take(MIN_PRIORITY).run(); + maybeMail = mailbox.tryTake(MIN_PRIORITY); + if (!maybeMail.isPresent()) { + long start = System.currentTimeMillis(); + maybeMail = Optional.of(mailbox.take(MIN_PRIORITY)); + idleTime.markEvent(System.currentTimeMillis() - start); + } + maybeMail.get().run(); } return isMailboxLoopRunning(); @@ -301,6 +317,11 @@ public class MailboxProcessor implements Closeable { return mailboxLoopRunning; } + @VisibleForTesting + public Meter getIdleTime() { + return idleTime; + } + /** * Helper method to make sure that the mailbox loop will check the control flow flags in the next iteration. */ diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxProcessorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxProcessorTest.java index 9740907629..b649f6be76 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxProcessorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxProcessorTest.java @@ -23,6 +23,7 @@ import org.apache.flink.streaming.api.operators.MailboxExecutor; import org.apache.flink.util.FlinkException; import org.apache.flink.util.function.RunnableWithException; +import org.hamcrest.Matchers; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -236,6 +237,53 @@ public class TaskMailboxProcessorTest { mailboxProcessor.allActionsCompleted(); } + @Test + public void testIdleTime() throws InterruptedException { + final AtomicReference suspendedActionRef = new AtomicReference<>(); + final int totalSwitches = 10; + + MailboxThread mailboxThread = new MailboxThread() { + int count = 0; + + @Override + public void runDefaultAction(Controller controller) { + // If this is violated, it means that the default action was invoked while we assumed suspension + Assert.assertTrue(suspendedActionRef.compareAndSet(null, controller.suspendDefaultAction())); + ++count; + if (count == totalSwitches) { + controller.allActionsCompleted(); + } + } + }; + mailboxThread.start(); + final MailboxProcessor mailboxProcessor = mailboxThread.getMailboxProcessor(); + + final Thread asyncUnblocker = new Thread(() -> { + while (!Thread.currentThread().isInterrupted()) { + final MailboxDefaultAction.Suspension resume = + suspendedActionRef.getAndSet(null); + + if (resume != null) { + try { + Thread.sleep(1); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + mailboxProcessor.getMailboxExecutor(DEFAULT_PRIORITY).execute(resume::resume, "resume"); + } + } + }); + + asyncUnblocker.start(); + mailboxThread.signalStart(); + mailboxThread.join(); + asyncUnblocker.interrupt(); + asyncUnblocker.join(); + + Assert.assertThat(mailboxProcessor.getIdleTime().getCount(), Matchers.greaterThanOrEqualTo(10L)); + + } + private static MailboxProcessor start(MailboxThread mailboxThread) { mailboxThread.start(); final MailboxProcessor mailboxProcessor = mailboxThread.getMailboxProcessor(); -- Gitee From 86e9ec070ba28be6f71624a2039214177f7e135d Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Mon, 30 Mar 2020 10:49:43 +0200 Subject: [PATCH 477/885] [FLINK-16856][scripts] Manually calculate lines of logging output 'head' with a negative '-n' argument isn't standardized. --- flink-dist/src/main/flink-bin/bin/taskmanager.sh | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/flink-dist/src/main/flink-bin/bin/taskmanager.sh b/flink-dist/src/main/flink-bin/bin/taskmanager.sh index 4ffceb0559..fd1e1d4fe0 100755 --- a/flink-dist/src/main/flink-bin/bin/taskmanager.sh +++ b/flink-dist/src/main/flink-bin/bin/taskmanager.sh @@ -49,7 +49,9 @@ if [[ $STARTSTOP == "start" ]] || [[ $STARTSTOP == "start-foreground" ]]; then # Startup parameters java_utils_output=$(runBashJavaUtilsCmd GET_TM_RESOURCE_PARAMS ${FLINK_CONF_DIR}) - logging_output=$(echo "${java_utils_output}" | head -n -2) + + num_lines=$(echo "${java_utils_output}" | wc -l) + logging_output=$(echo "${java_utils_output}" | head -n $((${num_lines} - 2))) params_output=$(echo "${java_utils_output}" | tail -n 2) jvm_params=$(extractExecutionParams "$(echo "$params_output" | head -n 1)") -- Gitee From 562e7711914e58f152288b512bbf05ec3a1cfbfa Mon Sep 17 00:00:00 2001 From: Canbin Zheng Date: Wed, 8 Apr 2020 17:31:57 +0800 Subject: [PATCH 478/885] [FLINK-16602][k8s] Rework the internal & external Service 1.The REST service serves REST traffic while the internal service serves internal requests from TMs to JM. 2.The REST service is always created but the internal service is only created in non-high availability setup. Co-authored-by: felixzheng This closes #11456 . --- .../kubernetes/cli/KubernetesSessionCli.java | 2 +- .../kubeclient/Fabric8FlinkKubeClient.java | 42 ++++----- .../kubeclient/FlinkKubeClient.java | 9 -- .../decorators/AbstractServiceDecorator.java | 87 ------------------- .../decorators/ExternalServiceDecorator.java | 43 +++++---- .../decorators/InternalServiceDecorator.java | 68 ++++++++------- .../KubernetesJobManagerParameters.java | 5 ++ .../flink/kubernetes/utils/Constants.java | 2 + .../ExternalServiceDecoratorTest.java | 16 ++-- .../InternalServiceDecoratorTest.java | 18 ++-- .../KubernetesJobManagerFactoryTest.java | 6 +- 11 files changed, 111 insertions(+), 187 deletions(-) delete mode 100644 flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/AbstractServiceDecorator.java diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/cli/KubernetesSessionCli.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/cli/KubernetesSessionCli.java index 568ff2e591..074555fed1 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/cli/KubernetesSessionCli.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/cli/KubernetesSessionCli.java @@ -101,7 +101,7 @@ public class KubernetesSessionCli { final FlinkKubeClient kubeClient = KubeClientFactory.fromConfiguration(configuration); // Retrieve or create a session cluster. - if (clusterId != null && kubeClient.getInternalService(clusterId) != null) { + if (clusterId != null && kubeClient.getRestService(clusterId) != null) { clusterClient = kubernetesClusterDescriptor.retrieve(clusterId).getClusterClient(); } else { clusterClient = kubernetesClusterDescriptor diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java index df0af2a3a4..d5a5134cb7 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/Fabric8FlinkKubeClient.java @@ -127,8 +127,7 @@ public class Fabric8FlinkKubeClient implements FlinkKubeClient { @Nullable public Endpoint getRestEndpoint(String clusterId) { int restPort = this.flinkConfig.getInteger(RestOptions.PORT); - final KubernetesConfigOptions.ServiceExposedType serviceExposedType = - flinkConfig.get(KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE); + final KubernetesConfigOptions.ServiceExposedType serviceExposedType = flinkConfig.get(KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE); // Return the service.namespace directly when use ClusterIP. if (serviceExposedType == KubernetesConfigOptions.ServiceExposedType.ClusterIP) { @@ -193,16 +192,24 @@ public class Fabric8FlinkKubeClient implements FlinkKubeClient { LOG.error("A Kubernetes exception occurred.", e); } - @Nullable - @Override - public KubernetesService getInternalService(String clusterId) { - return getService(KubernetesUtils.getInternalServiceName(clusterId)); - } - @Override @Nullable public KubernetesService getRestService(String clusterId) { - return getService(KubernetesUtils.getRestServiceName(clusterId)); + final String serviceName = KubernetesUtils.getRestServiceName(clusterId); + + final Service service = this.internalClient + .services() + .inNamespace(nameSpace) + .withName(serviceName) + .fromServer() + .get(); + + if (service == null) { + LOG.debug("Service {} does not exist", serviceName); + return null; + } + + return new KubernetesService(service); } @Override @@ -256,23 +263,6 @@ public class Fabric8FlinkKubeClient implements FlinkKubeClient { resource.getMetadata().setOwnerReferences(Collections.singletonList(deploymentOwnerReference))); } - private KubernetesService getService(String serviceName) { - final Service service = this - .internalClient - .services() - .inNamespace(nameSpace) - .withName(serviceName) - .fromServer() - .get(); - - if (service == null) { - LOG.debug("Service {} does not exist", serviceName); - return null; - } - - return new KubernetesService(service); - } - /** * To get nodePort of configured ports. */ diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkKubeClient.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkKubeClient.java index 1fc01cd5d3..866de7eef2 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkKubeClient.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/FlinkKubeClient.java @@ -56,15 +56,6 @@ public interface FlinkKubeClient extends AutoCloseable { */ void stopAndCleanupCluster(String clusterId); - /** - * Get the kubernetes internal service of the given flink clusterId. - * - * @param clusterId cluster id - * @return Return the internal service of the specified cluster id. Return null if the service does not exist. - */ - @Nullable - KubernetesService getInternalService(String clusterId); - /** * Get the kubernetes rest service of the given flink clusterId. * diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/AbstractServiceDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/AbstractServiceDecorator.java deleted file mode 100644 index 444de3aa39..0000000000 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/AbstractServiceDecorator.java +++ /dev/null @@ -1,87 +0,0 @@ -/* - * 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.kubernetes.kubeclient.decorators; - -import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; -import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters; -import org.apache.flink.kubernetes.utils.Constants; - -import io.fabric8.kubernetes.api.model.HasMetadata; -import io.fabric8.kubernetes.api.model.Service; -import io.fabric8.kubernetes.api.model.ServiceBuilder; -import io.fabric8.kubernetes.api.model.ServicePort; -import io.fabric8.kubernetes.api.model.ServicePortBuilder; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * An abstract class containing some common implementations for the internal/external Services. - */ -public abstract class AbstractServiceDecorator extends AbstractKubernetesStepDecorator { - - protected final KubernetesJobManagerParameters kubernetesJobManagerParameters; - - public AbstractServiceDecorator(KubernetesJobManagerParameters kubernetesJobManagerParameters) { - this.kubernetesJobManagerParameters = checkNotNull(kubernetesJobManagerParameters); - } - - @Override - public List buildAccompanyingKubernetesResources() throws IOException { - final Service service = new ServiceBuilder() - .withApiVersion(Constants.API_VERSION) - .withNewMetadata() - .withName(getServiceName()) - .withLabels(kubernetesJobManagerParameters.getCommonLabels()) - .endMetadata() - .withNewSpec() - .withType(getServiceType().name()) - .withPorts(getServicePorts()) - .withSelector(kubernetesJobManagerParameters.getLabels()) - .endSpec() - .build(); - - return Collections.singletonList(service); - } - - protected abstract KubernetesConfigOptions.ServiceExposedType getServiceType(); - - protected abstract String getServiceName(); - - protected List getServicePorts() { - final List servicePorts = new ArrayList<>(); - - servicePorts.add(getServicePort( - Constants.REST_PORT_NAME, - kubernetesJobManagerParameters.getRestPort())); - - return servicePorts; - } - - protected static ServicePort getServicePort(String name, int port) { - return new ServicePortBuilder() - .withName(name) - .withPort(port) - .build(); - } -} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecorator.java index 3c05915e30..570fa2c1a4 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecorator.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecorator.java @@ -18,41 +18,52 @@ package org.apache.flink.kubernetes.kubeclient.decorators; -import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters; +import org.apache.flink.kubernetes.utils.Constants; import org.apache.flink.kubernetes.utils.KubernetesUtils; import io.fabric8.kubernetes.api.model.HasMetadata; +import io.fabric8.kubernetes.api.model.Service; +import io.fabric8.kubernetes.api.model.ServiceBuilder; import java.io.IOException; import java.util.Collections; import java.util.List; +import static org.apache.flink.util.Preconditions.checkNotNull; + /** * Creates an external Service to expose the rest port of the Flink JobManager(s). */ -public class ExternalServiceDecorator extends AbstractServiceDecorator { +public class ExternalServiceDecorator extends AbstractKubernetesStepDecorator { + + private final KubernetesJobManagerParameters kubernetesJobManagerParameters; public ExternalServiceDecorator(KubernetesJobManagerParameters kubernetesJobManagerParameters) { - super(kubernetesJobManagerParameters); + this.kubernetesJobManagerParameters = checkNotNull(kubernetesJobManagerParameters); } @Override public List buildAccompanyingKubernetesResources() throws IOException { - if (kubernetesJobManagerParameters.getRestServiceExposedType() == KubernetesConfigOptions.ServiceExposedType.ClusterIP) { - return Collections.emptyList(); - } + final String serviceName = + KubernetesUtils.getRestServiceName(kubernetesJobManagerParameters.getClusterId()); - return super.buildAccompanyingKubernetesResources(); - } + final Service externalService = new ServiceBuilder() + .withApiVersion(Constants.API_VERSION) + .withNewMetadata() + .withName(serviceName) + .withLabels(kubernetesJobManagerParameters.getCommonLabels()) + .endMetadata() + .withNewSpec() + .withType(kubernetesJobManagerParameters.getRestServiceExposedType().name()) + .withSelector(kubernetesJobManagerParameters.getLabels()) + .addNewPort() + .withName(Constants.REST_PORT_NAME) + .withPort(kubernetesJobManagerParameters.getRestPort()) + .endPort() + .endSpec() + .build(); - @Override - protected KubernetesConfigOptions.ServiceExposedType getServiceType() { - return kubernetesJobManagerParameters.getRestServiceExposedType(); - } - - @Override - protected String getServiceName() { - return KubernetesUtils.getRestServiceName(kubernetesJobManagerParameters.getClusterId()); + return Collections.singletonList(externalService); } } diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InternalServiceDecorator.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InternalServiceDecorator.java index 8ca4b12537..6c9377005f 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InternalServiceDecorator.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/decorators/InternalServiceDecorator.java @@ -19,63 +19,67 @@ package org.apache.flink.kubernetes.kubeclient.decorators; import org.apache.flink.configuration.JobManagerOptions; -import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; import org.apache.flink.kubernetes.kubeclient.parameters.KubernetesJobManagerParameters; import org.apache.flink.kubernetes.utils.Constants; import org.apache.flink.kubernetes.utils.KubernetesUtils; import io.fabric8.kubernetes.api.model.HasMetadata; -import io.fabric8.kubernetes.api.model.ServicePort; +import io.fabric8.kubernetes.api.model.Service; +import io.fabric8.kubernetes.api.model.ServiceBuilder; import java.io.IOException; +import java.util.Collections; import java.util.List; +import static org.apache.flink.util.Preconditions.checkNotNull; + /** * Creates an internal Service which forwards the requests from the TaskManager(s) to the * active JobManager. * Note that only the non-HA scenario relies on this Service for internal communication, since * in the HA mode, the TaskManager(s) directly connects to the JobManager via IP address. */ -public class InternalServiceDecorator extends AbstractServiceDecorator { +public class InternalServiceDecorator extends AbstractKubernetesStepDecorator { + + private final KubernetesJobManagerParameters kubernetesJobManagerParameters; public InternalServiceDecorator(KubernetesJobManagerParameters kubernetesJobManagerParameters) { - super(kubernetesJobManagerParameters); + this.kubernetesJobManagerParameters = checkNotNull(kubernetesJobManagerParameters); } @Override public List buildAccompanyingKubernetesResources() throws IOException { - final String serviceName = getServiceName(); + if (!kubernetesJobManagerParameters.isInternalServiceEnabled()) { + return Collections.emptyList(); + } + + final String serviceName = KubernetesUtils.getInternalServiceName(kubernetesJobManagerParameters.getClusterId()); + + final Service headlessService = new ServiceBuilder() + .withApiVersion(Constants.API_VERSION) + .withNewMetadata() + .withName(serviceName) + .withLabels(kubernetesJobManagerParameters.getCommonLabels()) + .endMetadata() + .withNewSpec() + .withClusterIP(Constants.HEADLESS_SERVICE_CLUSTER_IP) + .withSelector(kubernetesJobManagerParameters.getLabels()) + .addNewPort() + .withName(Constants.JOB_MANAGER_RPC_PORT_NAME) + .withPort(kubernetesJobManagerParameters.getRPCPort()) + .endPort() + .addNewPort() + .withName(Constants.BLOB_SERVER_PORT_NAME) + .withPort(kubernetesJobManagerParameters.getBlobServerPort()) + .endPort() + .endSpec() + .build(); // Set job manager address to namespaced service name final String namespace = kubernetesJobManagerParameters.getNamespace(); - kubernetesJobManagerParameters.getFlinkConfiguration() - .setString(JobManagerOptions.ADDRESS, serviceName + "." + namespace); - - return super.buildAccompanyingKubernetesResources(); - } - - @Override - protected List getServicePorts() { - final List servicePorts = super.getServicePorts(); - - servicePorts.add(getServicePort( - Constants.JOB_MANAGER_RPC_PORT_NAME, - kubernetesJobManagerParameters.getRPCPort())); - servicePorts.add(getServicePort( - Constants.BLOB_SERVER_PORT_NAME, - kubernetesJobManagerParameters.getBlobServerPort())); + kubernetesJobManagerParameters.getFlinkConfiguration().setString(JobManagerOptions.ADDRESS, serviceName + "." + namespace); - return servicePorts; - } - - @Override - protected KubernetesConfigOptions.ServiceExposedType getServiceType() { - return KubernetesConfigOptions.ServiceExposedType.ClusterIP; - } - - @Override - protected String getServiceName() { - return KubernetesUtils.getInternalServiceName(kubernetesJobManagerParameters.getClusterId()); + return Collections.singletonList(headlessService); } } diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java index d077c34686..89f15f3824 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/kubeclient/parameters/KubernetesJobManagerParameters.java @@ -29,6 +29,7 @@ import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; import org.apache.flink.kubernetes.configuration.KubernetesConfigOptionsInternal; import org.apache.flink.kubernetes.utils.Constants; import org.apache.flink.kubernetes.utils.KubernetesUtils; +import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; import java.util.Collections; import java.util.HashMap; @@ -118,4 +119,8 @@ public class KubernetesJobManagerParameters extends AbstractKubernetesParameters public KubernetesConfigOptions.ServiceExposedType getRestServiceExposedType() { return flinkConfig.get(KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE); } + + public boolean isInternalServiceEnabled() { + return !HighAvailabilityMode.isHighAvailabilityModeActivated(flinkConfig); + } } diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/Constants.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/Constants.java index 65a078f7d2..e817a4d249 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/Constants.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/Constants.java @@ -73,4 +73,6 @@ public class Constants { public static final String ENV_FLINK_POD_IP_ADDRESS = "_POD_IP_ADDRESS"; public static final String POD_IP_FIELD_PATH = "status.podIP"; + + public static final String HEADLESS_SERVICE_CLUSTER_IP = "None"; } diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecoratorTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecoratorTest.java index dc97770ecd..7fd7a52825 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecoratorTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/ExternalServiceDecoratorTest.java @@ -35,7 +35,6 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import static junit.framework.TestCase.assertTrue; import static org.junit.Assert.assertEquals; /** @@ -65,9 +64,9 @@ public class ExternalServiceDecoratorTest extends KubernetesJobManagerTestBase { final Map expectedLabels = getCommonLabels(); assertEquals(expectedLabels, restService.getMetadata().getLabels()); - assertEquals("LoadBalancer", restService.getSpec().getType()); + assertEquals(KubernetesConfigOptions.ServiceExposedType.LoadBalancer.name(), restService.getSpec().getType()); - List expectedServicePorts = Collections.singletonList( + final List expectedServicePorts = Collections.singletonList( new ServicePortBuilder() .withName(Constants.REST_PORT_NAME) .withPort(REST_PORT) @@ -81,14 +80,13 @@ public class ExternalServiceDecoratorTest extends KubernetesJobManagerTestBase { @Test public void testSetServiceExposedType() throws IOException { - this.flinkConfig.set(KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE, - KubernetesConfigOptions.ServiceExposedType.NodePort); - List resources = this.externalServiceDecorator.buildAccompanyingKubernetesResources(); + this.flinkConfig.set(KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE, KubernetesConfigOptions.ServiceExposedType.NodePort); + final List resources = this.externalServiceDecorator.buildAccompanyingKubernetesResources(); assertEquals(KubernetesConfigOptions.ServiceExposedType.NodePort.name(), ((Service) resources.get(0)).getSpec().getType()); - this.flinkConfig.set(KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE, - KubernetesConfigOptions.ServiceExposedType.ClusterIP); - assertTrue(this.externalServiceDecorator.buildAccompanyingKubernetesResources().isEmpty()); + this.flinkConfig.set(KubernetesConfigOptions.REST_SERVICE_EXPOSED_TYPE, KubernetesConfigOptions.ServiceExposedType.ClusterIP); + final List servicesWithClusterIP = this.externalServiceDecorator.buildAccompanyingKubernetesResources(); + assertEquals(KubernetesConfigOptions.ServiceExposedType.ClusterIP.name(), ((Service) servicesWithClusterIP.get(0)).getSpec().getType()); } } diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InternalServiceDecoratorTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InternalServiceDecoratorTest.java index cb591cb1fe..f0f79ede49 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InternalServiceDecoratorTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/decorators/InternalServiceDecoratorTest.java @@ -18,10 +18,12 @@ package org.apache.flink.kubernetes.kubeclient.decorators; +import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.kubernetes.kubeclient.KubernetesJobManagerTestBase; import org.apache.flink.kubernetes.utils.Constants; import org.apache.flink.kubernetes.utils.KubernetesUtils; +import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; import io.fabric8.kubernetes.api.model.HasMetadata; import io.fabric8.kubernetes.api.model.Service; @@ -36,6 +38,7 @@ import java.util.List; import java.util.Map; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; /** * General tests for the {@link InternalServiceDecorator}. @@ -68,13 +71,10 @@ public class InternalServiceDecoratorTest extends KubernetesJobManagerTestBase { final Map expectedLabels = getCommonLabels(); assertEquals(expectedLabels, internalService.getMetadata().getLabels()); - assertEquals("ClusterIP", internalService.getSpec().getType()); + assertNull(internalService.getSpec().getType()); + assertEquals("None", internalService.getSpec().getClusterIP()); List expectedServicePorts = Arrays.asList( - new ServicePortBuilder() - .withName(Constants.REST_PORT_NAME) - .withPort(REST_PORT) - .build(), new ServicePortBuilder() .withName(Constants.JOB_MANAGER_RPC_PORT_NAME) .withPort(RPC_PORT) @@ -89,4 +89,12 @@ public class InternalServiceDecoratorTest extends KubernetesJobManagerTestBase { expectedLabels.putAll(userLabels); assertEquals(expectedLabels, internalService.getSpec().getSelector()); } + + @Test + public void testDisableInternalService() throws IOException { + this.flinkConfig.setString(HighAvailabilityOptions.HA_MODE, HighAvailabilityMode.ZOOKEEPER.name()); + + final List resources = this.internalServiceDecorator.buildAccompanyingKubernetesResources(); + assertEquals(0, resources.size()); + } } diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactoryTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactoryTest.java index 16c688a7df..07a5d56d6b 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactoryTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/factory/KubernetesJobManagerFactoryTest.java @@ -51,6 +51,7 @@ import static org.apache.flink.configuration.GlobalConfiguration.FLINK_CONF_FILE import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; /** @@ -179,8 +180,9 @@ public class KubernetesJobManagerFactoryTest extends KubernetesJobManagerTestBas final Service resultInternalService = internalServiceCandidates.get(0); assertEquals(2, resultInternalService.getMetadata().getLabels().size()); - assertEquals(resultInternalService.getSpec().getType(), "ClusterIP"); - assertEquals(3, resultInternalService.getSpec().getPorts().size()); + assertNull(resultInternalService.getSpec().getType()); + assertEquals(Constants.HEADLESS_SERVICE_CLUSTER_IP, resultInternalService.getSpec().getClusterIP()); + assertEquals(2, resultInternalService.getSpec().getPorts().size()); assertEquals(5, resultInternalService.getSpec().getSelector().size()); final Service resultRestService = restServiceCandidates.get(0); -- Gitee From c2c9297f9fc45c052bfb245bbca2d91d8da28c7f Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Wed, 8 Apr 2020 14:31:55 +0800 Subject: [PATCH 479/885] [FLINK-14162][runtime] SchedulerOperations#allocateSlotsAndDeploy always deploys tasks in a bulk --- .../runtime/scheduler/DefaultScheduler.java | 26 +------------------ .../scheduler/SchedulerOperations.java | 1 + .../LazyFromSourcesSchedulingStrategy.java | 5 +++- ...LazyFromSourcesSchedulingStrategyTest.java | 13 +++++++--- .../strategy/TestingSchedulerOperations.java | 4 --- 5 files changed, 16 insertions(+), 33 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java index 2049ec8c31..c21396f516 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java @@ -42,7 +42,6 @@ import org.apache.flink.runtime.jobmaster.slotpool.ThrowingSlotProvider; import org.apache.flink.runtime.metrics.groups.JobManagerJobMetricGroup; import org.apache.flink.runtime.rest.handler.legacy.backpressure.BackPressureStatsTracker; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; -import org.apache.flink.runtime.scheduler.strategy.LazyFromSourcesSchedulingStrategy; import org.apache.flink.runtime.scheduler.strategy.SchedulingStrategy; import org.apache.flink.runtime.scheduler.strategy.SchedulingStrategyFactory; import org.apache.flink.runtime.shuffle.ShuffleMaster; @@ -302,11 +301,7 @@ public class DefaultScheduler extends SchedulerBase implements SchedulerOperatio deploymentOptionsByVertex, slotExecutionVertexAssignments); - if (isDeployIndividually()) { - deployIndividually(deploymentHandles); - } else { - waitForAllSlotsAndDeploy(deploymentHandles); - } + waitForAllSlotsAndDeploy(deploymentHandles); } private void validateDeploymentOptions(final Collection deploymentOptions) { @@ -351,25 +346,6 @@ public class DefaultScheduler extends SchedulerBase implements SchedulerOperatio .collect(Collectors.toList()); } - /** - * HACK: See FLINK-14162 - * for details. - */ - private boolean isDeployIndividually() { - return schedulingStrategy instanceof LazyFromSourcesSchedulingStrategy; - } - - private void deployIndividually(final List deploymentHandles) { - for (final DeploymentHandle deploymentHandle : deploymentHandles) { - FutureUtils.assertNoException( - deploymentHandle - .getSlotExecutionVertexAssignment() - .getLogicalSlotFuture() - .handle(assignResourceOrHandleError(deploymentHandle)) - .handle(deployOrHandleError(deploymentHandle))); - } - } - private void waitForAllSlotsAndDeploy(final List deploymentHandles) { FutureUtils.assertNoException( assignAllResources(deploymentHandles).handle(deployAll(deploymentHandles))); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerOperations.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerOperations.java index 3875ce6e11..075e10ae96 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerOperations.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerOperations.java @@ -29,6 +29,7 @@ public interface SchedulerOperations { /** * Allocate slots and deploy the vertex when slots are returned. + * Vertices will be deployed only after all of them have been assigned slots. * The given order will be respected, i.e. tasks with smaller indices will be deployed earlier. * Only vertices in CREATED state will be accepted. Errors will happen if scheduling Non-CREATED vertices. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/LazyFromSourcesSchedulingStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/LazyFromSourcesSchedulingStrategy.java index e34ddf849c..a475b1f4b0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/LazyFromSourcesSchedulingStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/LazyFromSourcesSchedulingStrategy.java @@ -27,6 +27,7 @@ import org.apache.flink.util.IterableUtils; import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -143,7 +144,9 @@ public class LazyFromSourcesSchedulingStrategy implements SchedulingStrategy { verticesToDeploy, deploymentOptions::get); - schedulerOperations.allocateSlotsAndDeploy(vertexDeploymentOptions); + for (ExecutionVertexDeploymentOption deploymentOption : vertexDeploymentOptions) { + schedulerOperations.allocateSlotsAndDeploy(Collections.singletonList(deploymentOption)); + } } private Predicate> isInputConstraintSatisfied() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/LazyFromSourcesSchedulingStrategyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/LazyFromSourcesSchedulingStrategyTest.java index 532bd0d8f6..a4ffe99525 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/LazyFromSourcesSchedulingStrategyTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/LazyFromSourcesSchedulingStrategyTest.java @@ -28,13 +28,16 @@ import org.apache.flink.util.TestLogger; import org.junit.Test; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Set; import java.util.stream.Collectors; import static org.apache.flink.api.common.InputDependencyConstraint.ALL; import static org.apache.flink.runtime.io.network.partition.ResultPartitionType.PIPELINED; +import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; /** * Unit tests for {@link LazyFromSourcesSchedulingStrategy}. @@ -351,9 +354,13 @@ public class LazyFromSourcesSchedulingStrategyTest extends TestLogger { } private void assertLatestScheduledVerticesAreEqualTo(final List expected) { - assertEquals( - idsFromVertices(expected), - idsFromDeploymentOptions(testingSchedulerOperation.getLatestScheduledVertices())); + final List> deploymentOptions = testingSchedulerOperation.getScheduledVertices(); + assertThat(expected.size(), lessThanOrEqualTo(deploymentOptions.size())); + for (int i = 0; i < expected.size(); i++) { + assertEquals( + idsFromVertices(Collections.singletonList(expected.get(expected.size() - i - 1))), + idsFromDeploymentOptions(deploymentOptions.get(deploymentOptions.size() - i - 1))); + } } private static List idsFromVertices(final List vertices) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/TestingSchedulerOperations.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/TestingSchedulerOperations.java index 0ed581320e..512d3503d0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/TestingSchedulerOperations.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/TestingSchedulerOperations.java @@ -40,8 +40,4 @@ public class TestingSchedulerOperations implements SchedulerOperations { List> getScheduledVertices() { return Collections.unmodifiableList(scheduledVertices); } - - List getLatestScheduledVertices() { - return Collections.unmodifiableList(scheduledVertices.get(scheduledVertices.size() - 1)); - } } -- Gitee From 288750a76f64269de7f5eb2aee72feb92b048036 Mon Sep 17 00:00:00 2001 From: ifndef-SleePy Date: Fri, 3 Apr 2020 17:23:23 +0800 Subject: [PATCH 480/885] [FLINK-16945][checkpointing] Execute CheckpointFailureManager.FailJobCallback directly in main thread executor --- .../executiongraph/ExecutionGraph.java | 6 +- ...ecutionGraphCheckpointCoordinatorTest.java | 122 +++++++++++++++--- 2 files changed, 108 insertions(+), 20 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index 7a64f91532..04022cc34b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -472,12 +472,14 @@ public class ExecutionGraph implements AccessExecutionGraph { new CheckpointFailureManager.FailJobCallback() { @Override public void failJob(Throwable cause) { - getJobMasterMainThreadExecutor().execute(() -> failGlobal(cause)); + assertRunningInJobMasterMainThread(); + failGlobal(cause); } @Override public void failJobDueToTaskFailure(Throwable cause, ExecutionAttemptID failingTask) { - getJobMasterMainThreadExecutor().execute(() -> failGlobalIfExecutionIsStillRunning(cause, failingTask)); + assertRunningInJobMasterMainThread(); + failGlobalIfExecutionIsStillRunning(cause, failingTask); } } ); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java index e9623be119..78b798d64e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java @@ -20,16 +20,24 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor; import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; +import org.apache.flink.runtime.concurrent.Executors; +import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.Execution; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.ExecutionGraph; +import org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils; +import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.executiongraph.ExecutionVertex; import org.apache.flink.runtime.executiongraph.TestingExecutionGraphBuilder; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; +import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; +import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint; import org.apache.flink.runtime.state.memory.MemoryStateBackend; import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.util.TestLogger; @@ -37,11 +45,14 @@ import org.apache.flink.util.TestLogger; import org.hamcrest.Matchers; import org.junit.Test; +import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicLong; import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; /** @@ -131,9 +142,82 @@ public class ExecutionGraphCheckpointCoordinatorTest extends TestLogger { assertThat(storeShutdownFuture.get(), is(JobStatus.FINISHED)); } + /** + * The case is designed to check the race condition between {@link ExecutionGraph} and + * {@link CheckpointCoordinator}. There should be no checkpoint accepted after + * {@link CheckpointFailureManager} decides to fail the {@link ExecutionGraph}. + */ + @Test + public void testNoCheckpointAcceptedWhileFailingExecutionGraph() throws Exception { + CheckpointIDCounter counter = new TestingCheckpointIDCounter(new CompletableFuture<>()); + + TestingCompletedCheckpointStore store = new TestingCompletedCheckpointStore(new CompletableFuture<>()); + + final ManuallyTriggeredScheduledExecutor mainThreadExecutor = new ManuallyTriggeredScheduledExecutor(); + ExecutionGraph graph = createExecutionGraphAndEnableCheckpointing( + counter, + store, + new ComponentMainThreadExecutorServiceAdapter(mainThreadExecutor, Thread.currentThread())); + + final CheckpointCoordinator checkpointCoordinator = graph.getCheckpointCoordinator(); + + assertThat(checkpointCoordinator, Matchers.notNullValue()); + assertThat(checkpointCoordinator.isShutdown(), is(false)); + + graph.scheduleForExecution(); + ExecutionGraphTestUtils.switchToRunning(graph); + + // trigger a normal checkpoint which we will fail/decline later + checkpointCoordinator.triggerCheckpoint(System.currentTimeMillis(), true); + mainThreadExecutor.triggerAll(); + final long checkpointId = checkpointCoordinator.getPendingCheckpoints().keySet().iterator().next(); + + // trigger a forced checkpoint which could avoid failing the pre-checking + // this checkpoint would be acknowledged after the first one declined + checkpointCoordinator.triggerCheckpoint( + System.currentTimeMillis(), + new CheckpointProperties( + true, + CheckpointType.CHECKPOINT, + true, + true, + true, + true, + true), + null, + false, + false); + mainThreadExecutor.triggerAll(); + + // now we decline the first checkpoint and acknowledge the second checkpoint + // the first declined message should fail the execution graph and abort all pending checkpoints + // so the second acknowledged checkpoint should be abandoned when the ack arrived + final ExecutionAttemptID attemptId = + graph.getAllExecutionVertices().iterator().next().getCurrentExecutionAttempt().getAttemptId(); + final DeclineCheckpoint decline = new DeclineCheckpoint(graph.getJobID(), attemptId, checkpointId); + final AcknowledgeCheckpoint ack = new AcknowledgeCheckpoint(graph.getJobID(), attemptId, checkpointId + 1); + checkpointCoordinator.receiveDeclineMessage(decline, "localhost"); + checkpointCoordinator.receiveAcknowledgeMessage(ack, "localhost"); + + mainThreadExecutor.triggerAll(); + + assertEquals(0, store.completedCheckpoints.size()); + } + + private ExecutionGraph createExecutionGraphAndEnableCheckpointing( + CheckpointIDCounter counter, + CompletedCheckpointStore store) throws Exception { + + return createExecutionGraphAndEnableCheckpointing( + counter, + store, + ComponentMainThreadExecutorServiceAdapter.forMainThread()); + } + private ExecutionGraph createExecutionGraphAndEnableCheckpointing( CheckpointIDCounter counter, - CompletedCheckpointStore store) throws Exception { + CompletedCheckpointStore store, + ComponentMainThreadExecutor mainThreadExecutor) throws Exception { final Time timeout = Time.days(1L); JobVertex jobVertex = new JobVertex("MockVertex"); @@ -144,11 +228,14 @@ public class ExecutionGraphCheckpointCoordinatorTest extends TestLogger { .setJobGraph(new JobGraph(jobVertex)) .setRpcTimeout(timeout) .setAllocationTimeout(timeout) + .setIoExecutor(Executors.directExecutor()) .build(); + final ExecutionJobVertex executionJobVertex = executionGraph.getVerticesTopologically().iterator().next(); + CheckpointCoordinatorConfiguration chkConfig = new CheckpointCoordinatorConfiguration( - 100, - 100, + 1000000, + 1000000, 100, 1, CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION, @@ -158,22 +245,24 @@ public class ExecutionGraphCheckpointCoordinatorTest extends TestLogger { executionGraph.enableCheckpointing( chkConfig, - Collections.emptyList(), - Collections.emptyList(), - Collections.emptyList(), + Collections.singletonList(executionJobVertex), + Collections.singletonList(executionJobVertex), + Collections.singletonList(executionJobVertex), Collections.emptyList(), counter, store, new MemoryStateBackend(), CheckpointStatsTrackerTest.createTestTracker()); - executionGraph.start(ComponentMainThreadExecutorServiceAdapter.forMainThread()); + executionGraph.start(mainThreadExecutor); return executionGraph; } private static final class TestingCheckpointIDCounter implements CheckpointIDCounter { + private final AtomicLong counter = new AtomicLong(0); + private final CompletableFuture shutdownStatus; private TestingCheckpointIDCounter(CompletableFuture shutdownStatus) { @@ -190,17 +279,17 @@ public class ExecutionGraphCheckpointCoordinatorTest extends TestLogger { @Override public long getAndIncrement() { - throw new UnsupportedOperationException("Not implemented."); + return counter.getAndIncrement(); } @Override public long get() { - throw new UnsupportedOperationException("Not implemented."); + return counter.get(); } @Override public void setCount(long newId) { - throw new UnsupportedOperationException("Not implemented."); + counter.set(newId); } } @@ -208,6 +297,8 @@ public class ExecutionGraphCheckpointCoordinatorTest extends TestLogger { private final CompletableFuture shutdownStatus; + private final List completedCheckpoints = new ArrayList<>(); + private TestingCompletedCheckpointStore(CompletableFuture shutdownStatus) { this.shutdownStatus = shutdownStatus; } @@ -219,12 +310,7 @@ public class ExecutionGraphCheckpointCoordinatorTest extends TestLogger { @Override public void addCheckpoint(CompletedCheckpoint checkpoint) { - throw new UnsupportedOperationException("Not implemented."); - } - - @Override - public CompletedCheckpoint getLatestCheckpoint(boolean isPreferCheckpointForRecovery) { - throw new UnsupportedOperationException("Not implemented."); + completedCheckpoints.add(checkpoint); } @Override @@ -234,12 +320,12 @@ public class ExecutionGraphCheckpointCoordinatorTest extends TestLogger { @Override public List getAllCheckpoints() { - throw new UnsupportedOperationException("Not implemented."); + return completedCheckpoints; } @Override public int getNumberOfRetainedCheckpoints() { - throw new UnsupportedOperationException("Not implemented."); + return completedCheckpoints.size(); } @Override -- Gitee From 5e5fddb81d212bb5b9f6f48868e6dc21715d8ada Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 8 Apr 2020 12:10:25 +0200 Subject: [PATCH 481/885] [FLINK-9429] run-single-test.sh: fix order of "imports" Some e2e tests that rely on FLINK_VERSION being set, such as test_quickstarts.sh, didn't work with run-single-test.sh before. The reason the tests didn't work locally is that run-single-test.sh has the order of "imports" wrong. i.e. we have: source "${END_TO_END_DIR}/test-scripts/test-runner-common.sh" source "${END_TO_END_DIR}/../tools/ci/maven-utils.sh" while run-nightly-tests.sh has: source "${END_TO_END_DIR}/test-scripts/test-runner-common.sh" source "${END_TO_END_DIR}/../tools/ci/maven-utils.sh" This leads to this line failing in test-runner-common.sh because mvn_run is not available: export FLINK_VERSION=$(MVN_RUN_VERBOSE=false run_mvn --file ${END_TO_END_DIR}/pom.xml org.apache.maven.plugins:maven-help-plugin:3.1.0:evaluate -Dexpression=project.version -q -DforceStdout) --- flink-end-to-end-tests/run-single-test.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-end-to-end-tests/run-single-test.sh b/flink-end-to-end-tests/run-single-test.sh index 9ec56ed574..f84fd46cf1 100755 --- a/flink-end-to-end-tests/run-single-test.sh +++ b/flink-end-to-end-tests/run-single-test.sh @@ -46,8 +46,8 @@ if [ -z "$FLINK_DIR" ] ; then exit 1 fi -source "${END_TO_END_DIR}/test-scripts/test-runner-common.sh" source "${END_TO_END_DIR}/../tools/ci/maven-utils.sh" +source "${END_TO_END_DIR}/test-scripts/test-runner-common.sh" cleanup_tmp_files -- Gitee From 7cf0f6880896bb30bec2f8fc0c6193bfa35c4f04 Mon Sep 17 00:00:00 2001 From: qqibrow Date: Wed, 25 Mar 2020 20:57:24 -0700 Subject: [PATCH 482/885] [FLINK-16555] Reject Enum as key type hashCode() of Enum type calls Object.hashCode(), therefore keyBy on Enum type could lead to incorrect data routing. --- .../streaming/api/datastream/KeyedStream.java | 12 ++++++- .../flink/streaming/api/DataStreamTest.java | 33 +++++++++++++++---- 2 files changed, 38 insertions(+), 7 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java index d2bcfdadae..c586b5c501 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java @@ -32,6 +32,7 @@ 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.EnumTypeInfo; import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; import org.apache.flink.api.java.typeutils.PojoTypeInfo; import org.apache.flink.api.java.typeutils.TupleTypeInfoBase; @@ -210,6 +211,7 @@ public class KeyedStream extends DataStream { * the {@link Object#hashCode()} implementation. *

  • it is an array of any type (see {@link PrimitiveArrayTypeInfo}, {@link BasicArrayTypeInfo}, * {@link ObjectArrayTypeInfo}).
  • + *
  • it is enum type
  • * , * {@code true} otherwise. */ @@ -217,13 +219,21 @@ public class KeyedStream extends DataStream { try { return (type instanceof PojoTypeInfo) ? !type.getTypeClass().getMethod("hashCode").getDeclaringClass().equals(Object.class) - : !(type instanceof PrimitiveArrayTypeInfo || type instanceof BasicArrayTypeInfo || type instanceof ObjectArrayTypeInfo); + : !(isArrayType(type) || isEnumType(type)); } catch (NoSuchMethodException ignored) { // this should never happen as we are just searching for the hashCode() method. } return false; } + private static boolean isArrayType(TypeInformation type) { + return type instanceof PrimitiveArrayTypeInfo || type instanceof BasicArrayTypeInfo || type instanceof ObjectArrayTypeInfo; + } + + private static boolean isEnumType(TypeInformation type) { + return type instanceof EnumTypeInfo; + } + // ------------------------------------------------------------------------ // properties // ------------------------------------------------------------------------ diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java index 41db863310..a174a6ee4d 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java @@ -33,6 +33,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.EnumTypeInfo; import org.apache.flink.api.java.typeutils.GenericTypeInfo; import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; import org.apache.flink.api.java.typeutils.TupleTypeInfo; @@ -1192,7 +1193,7 @@ public class DataStreamTest extends TestLogger { } }; - testKeyRejection(keySelector, PrimitiveArrayTypeInfo.INT_PRIMITIVE_ARRAY_TYPE_INFO); + assertArrayKeyRejection(keySelector, PrimitiveArrayTypeInfo.INT_PRIMITIVE_ARRAY_TYPE_INFO); } @Test @@ -1207,7 +1208,7 @@ public class DataStreamTest extends TestLogger { } }; - testKeyRejection(keySelector, BasicArrayTypeInfo.INT_ARRAY_TYPE_INFO); + assertArrayKeyRejection(keySelector, BasicArrayTypeInfo.INT_ARRAY_TYPE_INFO); } @Test @@ -1229,15 +1230,14 @@ public class DataStreamTest extends TestLogger { ObjectArrayTypeInfo keyTypeInfo = ObjectArrayTypeInfo.getInfoFor( Object[].class, new GenericTypeInfo<>(Object.class)); - testKeyRejection(keySelector, keyTypeInfo); + assertArrayKeyRejection(keySelector, keyTypeInfo); } - private void testKeyRejection(KeySelector, K> keySelector, TypeInformation expectedKeyType) { + private void assertArrayKeyRejection(KeySelector, K> keySelector, TypeInformation expectedKeyType) { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataStream> input = env.fromElements( - new Tuple2<>(new Integer[] {1, 2}, "barfoo") - ); + new Tuple2<>(new Integer[] {1, 2}, "barfoo")); Assert.assertEquals(expectedKeyType, TypeExtractor.getKeySelectorTypes(keySelector, input.getType())); @@ -1248,6 +1248,23 @@ public class DataStreamTest extends TestLogger { input.keyBy(keySelector); } + @Test + public void testEnumKeyRejection() { + KeySelector, TestEnum> keySelector = value -> value.f0; + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream> input = env.fromElements( + Tuple2.of(TestEnum.FOO, "Foo"), + Tuple2.of(TestEnum.BAR, "Bar")); + + expectedException.expect(InvalidProgramException.class); + expectedException.expectMessage(new StringStartsWith("Type " + EnumTypeInfo.of(TestEnum.class) + " cannot be used as key.")); + + input.keyBy(keySelector); + } + + //////////////// Composite Key Tests : POJOs //////////////// @Test @@ -1514,6 +1531,10 @@ public class DataStreamTest extends TestLogger { } } + private enum TestEnum { + FOO, BAR + } + private class DummyOutputSelector implements OutputSelector { @Override public Iterable select(Integer value) { -- Gitee From c51b65aa0e7ad9f88b536ba9aa5cc45c1d56bd8e Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Tue, 7 Apr 2020 15:40:19 +0200 Subject: [PATCH 483/885] [FLINK-17035][runtime] Add new methods to TestingSchedulingTopology - Enable adding single vertex to TestingSchedulingTopology. - Enable setting 'containsCoLocationConstraints' flag. --- .../strategy/TestingSchedulingTopology.java | 35 ++++++++++++++++++- 1 file changed, 34 insertions(+), 1 deletion(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/TestingSchedulingTopology.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/TestingSchedulingTopology.java index 8979ba6e9c..5f290a2e5e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/TestingSchedulingTopology.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/TestingSchedulingTopology.java @@ -46,6 +46,8 @@ public class TestingSchedulingTopology private final Map schedulingResultPartitions = new HashMap<>(); + private boolean containsCoLocationConstraints; + @Override public Iterable getVertices() { return Collections.unmodifiableCollection(schedulingExecutionVertices.values()); @@ -53,7 +55,11 @@ public class TestingSchedulingTopology @Override public boolean containsCoLocationConstraints() { - return false; + return containsCoLocationConstraints; + } + + public void setContainsCoLocationConstraints(final boolean containsCoLocationConstraints) { + this.containsCoLocationConstraints = containsCoLocationConstraints; } @Override @@ -95,6 +101,33 @@ public class TestingSchedulingTopology return new SchedulingExecutionVerticesBuilder(); } + public TestingSchedulingExecutionVertex newVertex() { + final TestingSchedulingExecutionVertex newVertex = new TestingSchedulingExecutionVertex(new JobVertexID(), 0); + addSchedulingExecutionVertex(newVertex); + return newVertex; + } + + public TestingSchedulingTopology connect( + TestingSchedulingExecutionVertex producer, + TestingSchedulingExecutionVertex consumer, + ResultPartitionType resultPartitionType) { + + final TestingSchedulingResultPartition resultPartition = new TestingSchedulingResultPartition.Builder() + .withResultPartitionType(resultPartitionType) + .build(); + + resultPartition.addConsumer(consumer); + resultPartition.setProducer(producer); + + producer.addProducedPartition(resultPartition); + consumer.addConsumedPartition(resultPartition); + + updateVertexResultPartitions(producer); + updateVertexResultPartitions(consumer); + + return this; + } + public ProducerConsumerConnectionBuilder connectPointwise( final List producers, final List consumers) { -- Gitee From 10af1cc667dda02e931d4a61b3cc1101302c44f0 Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Tue, 7 Apr 2020 15:38:07 +0200 Subject: [PATCH 484/885] [FLINK-17035][runtime] Replace FailoverTopology with SchedulingTopology --- .../executiongraph/ExecutionGraph.java | 5 - ...aptedRestartPipelinedRegionStrategyNG.java | 2 +- .../flip1/ExecutionFailureHandler.java | 14 +- .../failover/flip1/FailoverRegion.java | 7 +- .../flip1/FailoverResultPartition.java | 30 --- .../failover/flip1/FailoverStrategy.java | 3 +- .../failover/flip1/FailoverTopology.java | 29 --- .../failover/flip1/FailoverVertex.java | 30 --- .../flip1/RestartAllFailoverStrategy.java | 10 +- ...estartPipelinedRegionFailoverStrategy.java | 27 ++- .../runtime/scheduler/DefaultScheduler.java | 4 +- .../runtime/scheduler/SchedulerBase.java | 9 - .../adapter/DefaultExecutionTopology.java | 6 +- .../adapter/DefaultExecutionVertex.java | 6 +- .../adapter/DefaultResultPartition.java | 6 +- .../flip1/ExecutionFailureHandlerTest.java | 17 +- .../flip1/RestartAllFailoverStrategyTest.java | 16 +- ...nedRegionFailoverStrategyBuildingTest.java | 222 ++++++++---------- ...rtPipelinedRegionFailoverStrategyTest.java | 111 +++++---- .../failover/flip1/TestFailoverTopology.java | 174 -------------- 20 files changed, 217 insertions(+), 511 deletions(-) delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverResultPartition.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverTopology.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverVertex.java delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/TestFailoverTopology.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index 04022cc34b..d3aa0e2ee9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -49,7 +49,6 @@ import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.execution.SuppressRestartsException; import org.apache.flink.runtime.executiongraph.failover.FailoverStrategy; -import org.apache.flink.runtime.executiongraph.failover.flip1.FailoverTopology; import org.apache.flink.runtime.executiongraph.failover.flip1.ResultPartitionAvailabilityChecker; import org.apache.flink.runtime.executiongraph.failover.flip1.partitionrelease.PartitionReleaseStrategy; import org.apache.flink.runtime.executiongraph.restart.ExecutionGraphRestartCallback; @@ -418,10 +417,6 @@ public class ExecutionGraph implements AccessExecutionGraph { return executionTopology; } - public FailoverTopology getFailoverTopology() { - return executionTopology; - } - public ScheduleMode getScheduleMode() { return scheduleMode; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/AdaptedRestartPipelinedRegionStrategyNG.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/AdaptedRestartPipelinedRegionStrategyNG.java index 21d503ee22..36dfa13fc4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/AdaptedRestartPipelinedRegionStrategyNG.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/AdaptedRestartPipelinedRegionStrategyNG.java @@ -294,7 +294,7 @@ public class AdaptedRestartPipelinedRegionStrategyNG extends FailoverStrategy { // currently it's safe to add it here, as this method is invoked only once in production code. checkState(restartPipelinedRegionFailoverStrategy == null, "notifyNewVertices() must be called only once"); this.restartPipelinedRegionFailoverStrategy = new RestartPipelinedRegionFailoverStrategy( - executionGraph.getFailoverTopology(), + executionGraph.getSchedulingTopology(), executionGraph.getResultPartitionAvailabilityChecker()); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandler.java index a766a3e5e8..c89717edb1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandler.java @@ -20,6 +20,8 @@ package org.apache.flink.runtime.executiongraph.failover.flip1; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.runtime.JobException; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.runtime.scheduler.strategy.SchedulingExecutionVertex; +import org.apache.flink.runtime.scheduler.strategy.SchedulingTopology; import org.apache.flink.runtime.throwable.ThrowableClassifier; import org.apache.flink.runtime.throwable.ThrowableType; import org.apache.flink.util.IterableUtils; @@ -36,7 +38,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull; */ public class ExecutionFailureHandler { - private final FailoverTopology failoverTopology; + private final SchedulingTopology schedulingTopology; /** Strategy to judge which tasks should be restarted. */ private final FailoverStrategy failoverStrategy; @@ -50,16 +52,16 @@ public class ExecutionFailureHandler { /** * Creates the handler to deal with task failures. * - * @param failoverTopology contains the topology info for failover + * @param schedulingTopology contains the topology info for failover * @param failoverStrategy helps to decide tasks to restart on task failures * @param restartBackoffTimeStrategy helps to decide whether to restart failed tasks and the restarting delay */ public ExecutionFailureHandler( - final FailoverTopology failoverTopology, + final SchedulingTopology schedulingTopology, final FailoverStrategy failoverStrategy, final RestartBackoffTimeStrategy restartBackoffTimeStrategy) { - this.failoverTopology = checkNotNull(failoverTopology); + this.schedulingTopology = checkNotNull(schedulingTopology); this.failoverStrategy = checkNotNull(failoverStrategy); this.restartBackoffTimeStrategy = checkNotNull(restartBackoffTimeStrategy); } @@ -86,8 +88,8 @@ public class ExecutionFailureHandler { public FailureHandlingResult getGlobalFailureHandlingResult(final Throwable cause) { return handleFailure( cause, - IterableUtils.toStream(failoverTopology.getVertices()) - .map(FailoverVertex::getId) + IterableUtils.toStream(schedulingTopology.getVertices()) + .map(SchedulingExecutionVertex::getId) .collect(Collectors.toSet())); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverRegion.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverRegion.java index 2e416a4fe8..9e0369b789 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverRegion.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverRegion.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.executiongraph.failover.flip1; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.runtime.scheduler.strategy.SchedulingExecutionVertex; import java.util.HashSet; import java.util.Set; @@ -34,14 +35,14 @@ public class FailoverRegion { private final Set executionVertexIDs; /** All vertices in this region. */ - private final Set> executionVertices; + private final Set> executionVertices; /** * Creates a new failover region containing a set of vertices. * * @param executionVertices to be contained in this region */ - public FailoverRegion(Set> executionVertices) { + public FailoverRegion(Set> executionVertices) { this.executionVertices = checkNotNull(executionVertices); this.executionVertexIDs = new HashSet<>(); executionVertices.forEach(v -> this.executionVertexIDs.add(v.getId())); @@ -61,7 +62,7 @@ public class FailoverRegion { * * @return all vertices in this region */ - public Set> getAllExecutionVertices() { + public Set> getAllExecutionVertices() { return executionVertices; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverResultPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverResultPartition.java deleted file mode 100644 index 39f2435168..0000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverResultPartition.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * 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.runtime.executiongraph.failover.flip1; - -import org.apache.flink.runtime.executiongraph.IntermediateResultPartition; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; -import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; -import org.apache.flink.runtime.topology.Result; - -/** - * Represents a {@link IntermediateResultPartition} produced by a {@link FailoverVertex}. - */ -public interface FailoverResultPartition, R extends FailoverResultPartition> - extends Result { -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverStrategy.java index 798d88b622..11e9431410 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverStrategy.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.executiongraph.failover.flip1; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.runtime.scheduler.strategy.SchedulingTopology; import java.util.Set; @@ -52,7 +53,7 @@ public interface FailoverStrategy { * @return The instantiated failover strategy. */ FailoverStrategy create( - FailoverTopology topology, + SchedulingTopology topology, ResultPartitionAvailabilityChecker resultPartitionAvailabilityChecker); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverTopology.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverTopology.java deleted file mode 100644 index b9d0b6f281..0000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverTopology.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * 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.runtime.executiongraph.failover.flip1; - -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; -import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; -import org.apache.flink.runtime.topology.Topology; - -/** - * Represents a topology for failover. - */ -public interface FailoverTopology, R extends FailoverResultPartition> - extends Topology { -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverVertex.java deleted file mode 100644 index 62f2eb7ef4..0000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverVertex.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * 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.runtime.executiongraph.failover.flip1; - -import org.apache.flink.runtime.executiongraph.ExecutionVertex; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; -import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; -import org.apache.flink.runtime.topology.Vertex; - -/** - * Represents an {@link ExecutionVertex}. - */ -public interface FailoverVertex, R extends FailoverResultPartition> - extends Vertex { -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartAllFailoverStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartAllFailoverStrategy.java index f7f0d5770f..fef759d0f4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartAllFailoverStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartAllFailoverStrategy.java @@ -18,6 +18,8 @@ package org.apache.flink.runtime.executiongraph.failover.flip1; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.runtime.scheduler.strategy.SchedulingExecutionVertex; +import org.apache.flink.runtime.scheduler.strategy.SchedulingTopology; import org.apache.flink.util.IterableUtils; import java.util.Set; @@ -30,9 +32,9 @@ import static org.apache.flink.util.Preconditions.checkNotNull; */ public class RestartAllFailoverStrategy implements FailoverStrategy { - private final FailoverTopology topology; + private final SchedulingTopology topology; - public RestartAllFailoverStrategy(final FailoverTopology topology) { + public RestartAllFailoverStrategy(final SchedulingTopology topology) { this.topology = checkNotNull(topology); } @@ -46,7 +48,7 @@ public class RestartAllFailoverStrategy implements FailoverStrategy { @Override public Set getTasksNeedingRestart(ExecutionVertexID executionVertexId, Throwable cause) { return IterableUtils.toStream(topology.getVertices()) - .map(FailoverVertex::getId) + .map(SchedulingExecutionVertex::getId) .collect(Collectors.toSet()); } @@ -57,7 +59,7 @@ public class RestartAllFailoverStrategy implements FailoverStrategy { @Override public FailoverStrategy create( - final FailoverTopology topology, + final SchedulingTopology topology, final ResultPartitionAvailabilityChecker resultPartitionAvailabilityChecker) { return new RestartAllFailoverStrategy(topology); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartPipelinedRegionFailoverStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartPipelinedRegionFailoverStrategy.java index cdc6754298..b0990216f9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartPipelinedRegionFailoverStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartPipelinedRegionFailoverStrategy.java @@ -21,6 +21,9 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.runtime.io.network.partition.PartitionException; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.runtime.scheduler.strategy.SchedulingExecutionVertex; +import org.apache.flink.runtime.scheduler.strategy.SchedulingResultPartition; +import org.apache.flink.runtime.scheduler.strategy.SchedulingTopology; import org.apache.flink.util.ExceptionUtils; import org.slf4j.Logger; @@ -48,7 +51,7 @@ public class RestartPipelinedRegionFailoverStrategy implements FailoverStrategy private static final Logger LOG = LoggerFactory.getLogger(RestartPipelinedRegionFailoverStrategy.class); /** The topology containing info about all the vertices and result partitions. */ - private final FailoverTopology topology; + private final SchedulingTopology topology; /** All failover regions. */ private final Set regions; @@ -66,7 +69,7 @@ public class RestartPipelinedRegionFailoverStrategy implements FailoverStrategy * @param topology containing info about all the vertices and result partitions */ @VisibleForTesting - public RestartPipelinedRegionFailoverStrategy(FailoverTopology topology) { + public RestartPipelinedRegionFailoverStrategy(SchedulingTopology topology) { this(topology, resultPartitionID -> true); } @@ -77,7 +80,7 @@ public class RestartPipelinedRegionFailoverStrategy implements FailoverStrategy * @param resultPartitionAvailabilityChecker helps to query result partition availability */ public RestartPipelinedRegionFailoverStrategy( - FailoverTopology topology, + SchedulingTopology topology, ResultPartitionAvailabilityChecker resultPartitionAvailabilityChecker) { this.topology = checkNotNull(topology); @@ -95,15 +98,15 @@ public class RestartPipelinedRegionFailoverStrategy implements FailoverStrategy // ------------------------------------------------------------------------ private void buildFailoverRegions() { - final Set>> distinctRegions = + final Set>> distinctRegions = PipelinedRegionComputeUtil.computePipelinedRegions(topology); // creating all the failover regions and register them - for (Set> regionVertices : distinctRegions) { + for (Set> regionVertices : distinctRegions) { LOG.debug("Creating a failover region with {} vertices.", regionVertices.size()); final FailoverRegion failoverRegion = new FailoverRegion(regionVertices); regions.add(failoverRegion); - for (FailoverVertex vertex : regionVertices) { + for (SchedulingExecutionVertex vertex : regionVertices) { vertexToRegionMap.put(vertex.getId(), failoverRegion); } } @@ -187,8 +190,8 @@ public class RestartPipelinedRegionFailoverStrategy implements FailoverStrategy regionsToRestart.add(regionToRestart); // if a needed input result partition is not available, its producer region is involved - for (FailoverVertex vertex : regionToRestart.getAllExecutionVertices()) { - for (FailoverResultPartition consumedPartition : vertex.getConsumedResults()) { + for (SchedulingExecutionVertex vertex : regionToRestart.getAllExecutionVertices()) { + for (SchedulingResultPartition consumedPartition : vertex.getConsumedResults()) { if (!resultPartitionAvailabilityChecker.isAvailable(consumedPartition.getId())) { FailoverRegion producerRegion = vertexToRegionMap.get(consumedPartition.getProducer().getId()); if (!visitedRegions.contains(producerRegion)) { @@ -200,9 +203,9 @@ public class RestartPipelinedRegionFailoverStrategy implements FailoverStrategy } // all consumer regions of an involved region should be involved - for (FailoverVertex vertex : regionToRestart.getAllExecutionVertices()) { - for (FailoverResultPartition producedPartition : vertex.getProducedResults()) { - for (FailoverVertex consumerVertex : producedPartition.getConsumers()) { + for (SchedulingExecutionVertex vertex : regionToRestart.getAllExecutionVertices()) { + for (SchedulingResultPartition producedPartition : vertex.getProducedResults()) { + for (SchedulingExecutionVertex consumerVertex : producedPartition.getConsumers()) { FailoverRegion consumerRegion = vertexToRegionMap.get(consumerVertex.getId()); if (!visitedRegions.contains(consumerRegion)) { visitedRegions.add(consumerRegion); @@ -268,7 +271,7 @@ public class RestartPipelinedRegionFailoverStrategy implements FailoverStrategy @Override public FailoverStrategy create( - final FailoverTopology topology, + final SchedulingTopology topology, final ResultPartitionAvailabilityChecker resultPartitionAvailabilityChecker) { return new RestartPipelinedRegionFailoverStrategy(topology, resultPartitionAvailabilityChecker); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java index c21396f516..15c30a4690 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java @@ -141,12 +141,12 @@ public class DefaultScheduler extends SchedulerBase implements SchedulerOperatio this.executionVertexOperations = checkNotNull(executionVertexOperations); final FailoverStrategy failoverStrategy = failoverStrategyFactory.create( - getFailoverTopology(), + getSchedulingTopology(), getResultPartitionAvailabilityChecker()); log.info("Using failover strategy {} for {} ({}).", failoverStrategy, jobGraph.getName(), jobGraph.getJobID()); this.executionFailureHandler = new ExecutionFailureHandler( - getFailoverTopology(), + getSchedulingTopology(), failoverStrategy, restartBackoffTimeStrategy); this.schedulingStrategy = schedulingStrategyFactory.createInstance(this, getSchedulingTopology()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java index 11f1204b2e..cb06d4a4ab 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java @@ -54,7 +54,6 @@ import org.apache.flink.runtime.executiongraph.JobStatusListener; import org.apache.flink.runtime.executiongraph.failover.FailoverStrategy; import org.apache.flink.runtime.executiongraph.failover.FailoverStrategyLoader; import org.apache.flink.runtime.executiongraph.failover.NoOpFailoverStrategy; -import org.apache.flink.runtime.executiongraph.failover.flip1.FailoverTopology; import org.apache.flink.runtime.executiongraph.failover.flip1.ResultPartitionAvailabilityChecker; import org.apache.flink.runtime.executiongraph.restart.RestartStrategy; import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory; @@ -134,8 +133,6 @@ public abstract class SchedulerBase implements SchedulerNG { private final SchedulingTopology schedulingTopology; - private final FailoverTopology failoverTopology; - private final InputsLocationsRetriever inputsLocationsRetriever; private final BackPressureStatsTracker backPressureStatsTracker; @@ -222,7 +219,6 @@ public abstract class SchedulerBase implements SchedulerNG { this.executionGraph = createAndRestoreExecutionGraph(jobManagerJobMetricGroup, checkNotNull(shuffleMaster), checkNotNull(partitionTracker)); this.schedulingTopology = executionGraph.getSchedulingTopology(); - this.failoverTopology = executionGraph.getFailoverTopology(); this.inputsLocationsRetriever = new ExecutionGraphToInputsLocationsRetrieverAdapter(executionGraph); } @@ -287,7 +283,6 @@ public abstract class SchedulerBase implements SchedulerNG { * execution graph and accessors should be preferred over direct access: *