You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/01/06 16:17:11 UTC

[GitHub] [flink] twalthr opened a new pull request #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces

twalthr opened a new pull request #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces
URL: https://github.com/apache/flink/pull/10781
 
 
   ## What is the purpose of the change
   
   This PR connects Flink's new type inference to Calcite's type inference. It ensures that both Table API and SQL have exactly the same behavior and similar exception messages.
   
   Note: This PR does not contain tests yet. They will be added as part of FLINK-15487 for testing end-to-end functionality.
   
   ## Brief change log
   
   - Implementation of `SqlFunction` and `SqlAggFunction`
   
   ## Verifying this change
   
   Manually verified until code generation with some additional changes that will be part of FLINK-15487.
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): no
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: no
     - The serializers: no
     - The runtime per-record code paths (performance sensitive): no
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: no
     - The S3 file system connector: no
   
   ## Documentation
   
     - Does this pull request introduce a new feature? yes
     - If yes, how is the feature documented? JavaDocs
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] twalthr closed pull request #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces

Posted by GitBox <gi...@apache.org>.
twalthr closed pull request #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces
URL: https://github.com/apache/flink/pull/10781
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces
URL: https://github.com/apache/flink/pull/10781#issuecomment-571250359
 
 
   <!--
   Meta data
   Hash:85dc4bd6420b9354e6cf3f2fd9387077ed631894 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/143281261 TriggerType:PUSH TriggerID:85dc4bd6420b9354e6cf3f2fd9387077ed631894
   Hash:85dc4bd6420b9354e6cf3f2fd9387077ed631894 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4139 TriggerType:PUSH TriggerID:85dc4bd6420b9354e6cf3f2fd9387077ed631894
   -->
   ## CI report:
   
   * 85dc4bd6420b9354e6cf3f2fd9387077ed631894 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/143281261) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4139) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces
URL: https://github.com/apache/flink/pull/10781#issuecomment-571250359
 
 
   <!--
   Meta data
   Hash:85dc4bd6420b9354e6cf3f2fd9387077ed631894 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/143281261 TriggerType:PUSH TriggerID:85dc4bd6420b9354e6cf3f2fd9387077ed631894
   Hash:85dc4bd6420b9354e6cf3f2fd9387077ed631894 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4139 TriggerType:PUSH TriggerID:85dc4bd6420b9354e6cf3f2fd9387077ed631894
   Hash:982a7d1256e933e02ebcf4bcb994ab676ff8510b Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/143718894 TriggerType:PUSH TriggerID:982a7d1256e933e02ebcf4bcb994ab676ff8510b
   Hash:982a7d1256e933e02ebcf4bcb994ab676ff8510b Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4220 TriggerType:PUSH TriggerID:982a7d1256e933e02ebcf4bcb994ab676ff8510b
   -->
   ## CI report:
   
   * 85dc4bd6420b9354e6cf3f2fd9387077ed631894 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/143281261) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4139) 
   * 982a7d1256e933e02ebcf4bcb994ab676ff8510b Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/143718894) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4220) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] dawidwys commented on a change in pull request #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces

Posted by GitBox <gi...@apache.org>.
dawidwys commented on a change in pull request #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces
URL: https://github.com/apache/flink/pull/10781#discussion_r364259992
 
 

 ##########
 File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandChecker.java
 ##########
 @@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.functions.inference;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.functions.FunctionDefinition;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.inference.CallContext;
+import org.apache.flink.table.types.inference.TypeInference;
+import org.apache.flink.table.types.inference.TypeInferenceUtil;
+import org.apache.flink.table.types.inference.utils.AdaptedCallContext;
+import org.apache.flink.table.types.logical.LogicalType;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperandCountRange;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorNamespace;
+
+import java.util.List;
+
+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;
+
+/**
+ * A {@link SqlOperandTypeChecker} backed by {@link TypeInference}.
+ *
+ * <p>Note: This class must be kept in sync with {@link TypeInferenceUtil}.
+ */
+@Internal
+public final class TypeInferenceOperandChecker implements SqlOperandTypeChecker {
+
+	private final FunctionDefinition definition;
+
+	private final TypeInference typeInference;
+
+	private final SqlOperandCountRange countRange;
+
+	public TypeInferenceOperandChecker(
+			FunctionDefinition definition,
+			TypeInference typeInference) {
+		this.definition = definition;
+		this.typeInference = typeInference;
+		this.countRange = new ArgumentCountRange(typeInference.getInputTypeStrategy().getArgumentCount());
+	}
+
+	@Override
+	public boolean checkOperandTypes(SqlCallBinding callBinding, boolean throwOnFailure) {
+		final CallContext callContext = new CallBindingCallContext(definition, callBinding, null);
+		try {
+			return checkOperandTypesOrError(callBinding, callContext);
+		}
+		catch (ValidationException e) {
+			if (throwOnFailure) {
+				throw createInvalidCallException(callContext, e);
+			}
+			return false;
+		} catch (Throwable t) {
+			throw createUnexpectedException(callContext, t);
+		}
+	}
+
+	@Override
+	public SqlOperandCountRange getOperandCountRange() {
+		return countRange;
+	}
+
+	@Override
+	public String getAllowedSignatures(SqlOperator op, String opName) {
+		return TypeInferenceUtil.generateSignature(opName, definition, typeInference);
+	}
+
+	@Override
+	public Consistency getConsistency() {
+		return Consistency.NONE;
+	}
+
+	@Override
+	public boolean isOptional(int i) {
+		return false;
+	}
+
+	// --------------------------------------------------------------------------------------------
+
+	private boolean checkOperandTypesOrError(SqlCallBinding callBinding, CallContext callContext) {
+		final AdaptedCallContext adaptedCallContext;
+		try {
+			adaptedCallContext = adaptArguments(
+				typeInference,
+				callContext,
+				null);
+		} catch (ValidationException e) {
+			throw createInvalidInputException(
+				typeInference,
+				callContext,
+				e);
+		}
+
+		insertImplicitCasts(callBinding, adaptedCallContext.getArgumentDataTypes());
+
+		return true;
+	}
+
+	private void insertImplicitCasts(SqlCallBinding callBinding, List<DataType> expectedDataTypes) {
+		final FlinkTypeFactory flinkTypeFactory = (FlinkTypeFactory) callBinding.getTypeFactory();
+		final List<SqlNode> 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 SqlNode castedOperand = castTo(operands.get(i), expectedRelDataType);
+			callBinding.getCall().setOperand(i, castedOperand);
+			updateInferredType(callBinding.getValidator(), castedOperand, expectedRelDataType);
+		}
+	}
+
+	/**
+	 * Adopted from {@link org.apache.calcite.sql.validate.implicit.AbstractTypeCoercion}.
+	 */
+	private SqlNode castTo(SqlNode node, RelDataType type) {
+		return SqlStdOperatorTable.CAST.createCall(
+			SqlParserPos.ZERO,
+			node,
+			SqlTypeUtil.convertTypeToSpec(type).withNullable(type.isNullable()));
+	}
+
+	/**
+	 * Adopted from {@link org.apache.calcite.sql.validate.implicit.AbstractTypeCoercion}.
+	 */
+	@Deprecated
 
 Review comment:
   Why is it deprecated?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces
URL: https://github.com/apache/flink/pull/10781#issuecomment-571250359
 
 
   <!--
   Meta data
   Hash:85dc4bd6420b9354e6cf3f2fd9387077ed631894 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/143281261 TriggerType:PUSH TriggerID:85dc4bd6420b9354e6cf3f2fd9387077ed631894
   Hash:85dc4bd6420b9354e6cf3f2fd9387077ed631894 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4139 TriggerType:PUSH TriggerID:85dc4bd6420b9354e6cf3f2fd9387077ed631894
   Hash:982a7d1256e933e02ebcf4bcb994ab676ff8510b Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/143718894 TriggerType:PUSH TriggerID:982a7d1256e933e02ebcf4bcb994ab676ff8510b
   Hash:982a7d1256e933e02ebcf4bcb994ab676ff8510b Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4220 TriggerType:PUSH TriggerID:982a7d1256e933e02ebcf4bcb994ab676ff8510b
   -->
   ## CI report:
   
   * 85dc4bd6420b9354e6cf3f2fd9387077ed631894 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/143281261) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4139) 
   * 982a7d1256e933e02ebcf4bcb994ab676ff8510b Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/143718894) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4220) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] dawidwys commented on a change in pull request #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces

Posted by GitBox <gi...@apache.org>.
dawidwys commented on a change in pull request #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces
URL: https://github.com/apache/flink/pull/10781#discussion_r364174578
 
 

 ##########
 File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/inference/AbstractSqlCallContext.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.table.planner.functions.inference;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.catalog.DataTypeLookup;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.functions.FunctionDefinition;
+import org.apache.flink.table.types.inference.CallContext;
+
+import org.apache.calcite.sql.SqlIntervalLiteral;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.util.DateString;
+import org.apache.calcite.util.TimeString;
+import org.apache.calcite.util.TimestampString;
+
+import java.time.Duration;
+import java.time.Period;
+
+/**
+ * A {@link CallContext} backed by {@link SqlOperatorBinding}.
+ */
+@Internal
+public abstract class AbstractSqlCallContext implements CallContext {
+
+	private final DataTypeLookup lookup;
+
+	private final FunctionDefinition definition;
+
+	private final String name;
+
+	protected AbstractSqlCallContext(
+			DataTypeLookup lookup,
+			FunctionDefinition definition,
+			String name) {
+		this.lookup = lookup;
+		this.definition = definition;
+		this.name = name;
+	}
+
+	@Override
+	public DataTypeLookup getDataTypeLookup() {
+		return lookup;
+	}
+
+	@Override
+	public FunctionDefinition getFunctionDefinition() {
+		return definition;
+	}
+
+	@Override
+	public String getName() {
+		return name;
+	}
+
+	// --------------------------------------------------------------------------------------------
+
+	/**
+	 * Helper interface for abstracting accessing literals.
+	 */
+	protected interface LiteralValueAccessor {
+		<T> T getValueAs(Class<T> clazz);
+	}
+
+	/**
+	 * Bridges to {@link ValueLiteralExpression#getValueAs(Class)}.
+	 */
+	@SuppressWarnings("unchecked")
+	protected static <T, V> T getLiteralValueAs(LiteralValueAccessor accessor, Class<T> clazz) {
 
 Review comment:
   Unnecessary `V` type.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] dawidwys commented on a change in pull request #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces

Posted by GitBox <gi...@apache.org>.
dawidwys commented on a change in pull request #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces
URL: https://github.com/apache/flink/pull/10781#discussion_r364712338
 
 

 ##########
 File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandChecker.java
 ##########
 @@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.functions.inference;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.functions.FunctionDefinition;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.inference.CallContext;
+import org.apache.flink.table.types.inference.TypeInference;
+import org.apache.flink.table.types.inference.TypeInferenceUtil;
+import org.apache.flink.table.types.inference.utils.AdaptedCallContext;
+import org.apache.flink.table.types.logical.LogicalType;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperandCountRange;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorNamespace;
+
+import java.util.List;
+
+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;
+
+/**
+ * A {@link SqlOperandTypeChecker} backed by {@link TypeInference}.
+ *
+ * <p>Note: This class must be kept in sync with {@link TypeInferenceUtil}.
+ */
+@Internal
+public final class TypeInferenceOperandChecker implements SqlOperandTypeChecker {
+
+	private final FunctionDefinition definition;
+
+	private final TypeInference typeInference;
+
+	private final SqlOperandCountRange countRange;
+
+	public TypeInferenceOperandChecker(
+			FunctionDefinition definition,
+			TypeInference typeInference) {
+		this.definition = definition;
+		this.typeInference = typeInference;
+		this.countRange = new ArgumentCountRange(typeInference.getInputTypeStrategy().getArgumentCount());
+	}
+
+	@Override
+	public boolean checkOperandTypes(SqlCallBinding callBinding, boolean throwOnFailure) {
+		final CallContext callContext = new CallBindingCallContext(definition, callBinding, null);
+		try {
+			return checkOperandTypesOrError(callBinding, callContext);
+		}
+		catch (ValidationException e) {
+			if (throwOnFailure) {
+				throw createInvalidCallException(callContext, e);
+			}
+			return false;
+		} catch (Throwable t) {
+			throw createUnexpectedException(callContext, t);
+		}
+	}
+
+	@Override
+	public SqlOperandCountRange getOperandCountRange() {
+		return countRange;
+	}
+
+	@Override
+	public String getAllowedSignatures(SqlOperator op, String opName) {
+		return TypeInferenceUtil.generateSignature(opName, definition, typeInference);
+	}
+
+	@Override
+	public Consistency getConsistency() {
+		return Consistency.NONE;
+	}
+
+	@Override
+	public boolean isOptional(int i) {
+		return false;
+	}
+
+	// --------------------------------------------------------------------------------------------
+
+	private boolean checkOperandTypesOrError(SqlCallBinding callBinding, CallContext callContext) {
+		final AdaptedCallContext adaptedCallContext;
+		try {
+			adaptedCallContext = adaptArguments(
+				typeInference,
+				callContext,
+				null);
+		} catch (ValidationException e) {
+			throw createInvalidInputException(
+				typeInference,
+				callContext,
+				e);
+		}
+
+		insertImplicitCasts(callBinding, adaptedCallContext.getArgumentDataTypes());
+
+		return true;
+	}
+
+	private void insertImplicitCasts(SqlCallBinding callBinding, List<DataType> expectedDataTypes) {
+		final FlinkTypeFactory flinkTypeFactory = (FlinkTypeFactory) callBinding.getTypeFactory();
+		final List<SqlNode> 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 SqlNode castedOperand = castTo(operands.get(i), expectedRelDataType);
+			callBinding.getCall().setOperand(i, castedOperand);
+			updateInferredType(callBinding.getValidator(), castedOperand, expectedRelDataType);
+		}
+	}
+
+	/**
+	 * Adopted from {@link org.apache.calcite.sql.validate.implicit.AbstractTypeCoercion}.
+	 */
+	private SqlNode castTo(SqlNode node, RelDataType type) {
+		return SqlStdOperatorTable.CAST.createCall(
+			SqlParserPos.ZERO,
+			node,
+			SqlTypeUtil.convertTypeToSpec(type).withNullable(type.isNullable()));
+	}
+
+	/**
+	 * Adopted from {@link org.apache.calcite.sql.validate.implicit.AbstractTypeCoercion}.
+	 */
+	@Deprecated
 
 Review comment:
   I think it will be better to remove it. Esp. as in Calcite it is actually not deprecated. There is only `  @SuppressWarnings("deprecation")` annotation.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces
URL: https://github.com/apache/flink/pull/10781#issuecomment-571250359
 
 
   <!--
   Meta data
   Hash:85dc4bd6420b9354e6cf3f2fd9387077ed631894 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/143281261 TriggerType:PUSH TriggerID:85dc4bd6420b9354e6cf3f2fd9387077ed631894
   Hash:85dc4bd6420b9354e6cf3f2fd9387077ed631894 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4139 TriggerType:PUSH TriggerID:85dc4bd6420b9354e6cf3f2fd9387077ed631894
   -->
   ## CI report:
   
   * 85dc4bd6420b9354e6cf3f2fd9387077ed631894 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/143281261) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4139) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces
URL: https://github.com/apache/flink/pull/10781#issuecomment-571250359
 
 
   <!--
   Meta data
   Hash:85dc4bd6420b9354e6cf3f2fd9387077ed631894 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/143281261 TriggerType:PUSH TriggerID:85dc4bd6420b9354e6cf3f2fd9387077ed631894
   Hash:85dc4bd6420b9354e6cf3f2fd9387077ed631894 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4139 TriggerType:PUSH TriggerID:85dc4bd6420b9354e6cf3f2fd9387077ed631894
   -->
   ## CI report:
   
   * 85dc4bd6420b9354e6cf3f2fd9387077ed631894 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/143281261) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4139) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] twalthr commented on issue #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces

Posted by GitBox <gi...@apache.org>.
twalthr commented on issue #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces
URL: https://github.com/apache/flink/pull/10781#issuecomment-572514036
 
 
   Thank you @dawidwys. I will merge this in my next batch if there are no objections.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot commented on issue #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces

Posted by GitBox <gi...@apache.org>.
flinkbot commented on issue #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces
URL: https://github.com/apache/flink/pull/10781#issuecomment-571250359
 
 
   <!--
   Meta data
   Hash:85dc4bd6420b9354e6cf3f2fd9387077ed631894 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:85dc4bd6420b9354e6cf3f2fd9387077ed631894
   -->
   ## CI report:
   
   * 85dc4bd6420b9354e6cf3f2fd9387077ed631894 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] twalthr commented on a change in pull request #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces
URL: https://github.com/apache/flink/pull/10781#discussion_r364677631
 
 

 ##########
 File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/inference/TypeInferenceOperandChecker.java
 ##########
 @@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.functions.inference;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.functions.FunctionDefinition;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.inference.CallContext;
+import org.apache.flink.table.types.inference.TypeInference;
+import org.apache.flink.table.types.inference.TypeInferenceUtil;
+import org.apache.flink.table.types.inference.utils.AdaptedCallContext;
+import org.apache.flink.table.types.logical.LogicalType;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperandCountRange;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorNamespace;
+
+import java.util.List;
+
+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;
+
+/**
+ * A {@link SqlOperandTypeChecker} backed by {@link TypeInference}.
+ *
+ * <p>Note: This class must be kept in sync with {@link TypeInferenceUtil}.
+ */
+@Internal
+public final class TypeInferenceOperandChecker implements SqlOperandTypeChecker {
+
+	private final FunctionDefinition definition;
+
+	private final TypeInference typeInference;
+
+	private final SqlOperandCountRange countRange;
+
+	public TypeInferenceOperandChecker(
+			FunctionDefinition definition,
+			TypeInference typeInference) {
+		this.definition = definition;
+		this.typeInference = typeInference;
+		this.countRange = new ArgumentCountRange(typeInference.getInputTypeStrategy().getArgumentCount());
+	}
+
+	@Override
+	public boolean checkOperandTypes(SqlCallBinding callBinding, boolean throwOnFailure) {
+		final CallContext callContext = new CallBindingCallContext(definition, callBinding, null);
+		try {
+			return checkOperandTypesOrError(callBinding, callContext);
+		}
+		catch (ValidationException e) {
+			if (throwOnFailure) {
+				throw createInvalidCallException(callContext, e);
+			}
+			return false;
+		} catch (Throwable t) {
+			throw createUnexpectedException(callContext, t);
+		}
+	}
+
+	@Override
+	public SqlOperandCountRange getOperandCountRange() {
+		return countRange;
+	}
+
+	@Override
+	public String getAllowedSignatures(SqlOperator op, String opName) {
+		return TypeInferenceUtil.generateSignature(opName, definition, typeInference);
+	}
+
+	@Override
+	public Consistency getConsistency() {
+		return Consistency.NONE;
+	}
+
+	@Override
+	public boolean isOptional(int i) {
+		return false;
+	}
+
+	// --------------------------------------------------------------------------------------------
+
+	private boolean checkOperandTypesOrError(SqlCallBinding callBinding, CallContext callContext) {
+		final AdaptedCallContext adaptedCallContext;
+		try {
+			adaptedCallContext = adaptArguments(
+				typeInference,
+				callContext,
+				null);
+		} catch (ValidationException e) {
+			throw createInvalidInputException(
+				typeInference,
+				callContext,
+				e);
+		}
+
+		insertImplicitCasts(callBinding, adaptedCallContext.getArgumentDataTypes());
+
+		return true;
+	}
+
+	private void insertImplicitCasts(SqlCallBinding callBinding, List<DataType> expectedDataTypes) {
+		final FlinkTypeFactory flinkTypeFactory = (FlinkTypeFactory) callBinding.getTypeFactory();
+		final List<SqlNode> 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 SqlNode castedOperand = castTo(operands.get(i), expectedRelDataType);
+			callBinding.getCall().setOperand(i, castedOperand);
+			updateInferredType(callBinding.getValidator(), castedOperand, expectedRelDataType);
+		}
+	}
+
+	/**
+	 * Adopted from {@link org.apache.calcite.sql.validate.implicit.AbstractTypeCoercion}.
+	 */
+	private SqlNode castTo(SqlNode node, RelDataType type) {
+		return SqlStdOperatorTable.CAST.createCall(
+			SqlParserPos.ZERO,
+			node,
+			SqlTypeUtil.convertTypeToSpec(type).withNullable(type.isNullable()));
+	}
+
+	/**
+	 * Adopted from {@link org.apache.calcite.sql.validate.implicit.AbstractTypeCoercion}.
+	 */
+	@Deprecated
 
 Review comment:
   The deprecated annotation was also adopted from Calcite. But I will remove it as it causes confusion. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] dawidwys commented on a change in pull request #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces

Posted by GitBox <gi...@apache.org>.
dawidwys commented on a change in pull request #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces
URL: https://github.com/apache/flink/pull/10781#discussion_r364145797
 
 

 ##########
 File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/ConstantArgumentCount.java
 ##########
 @@ -60,16 +60,20 @@ public static ArgumentCount from(int minCount) {
 	}
 
 	public static ArgumentCount any() {
-		return new ConstantArgumentCount(0, OPEN_INTERVAL);
+		return new ConstantArgumentCount(OPEN_INTERVAL, OPEN_INTERVAL);
 
 Review comment:
   Is this necessary? Does it make sense to have a special handling for a non present min value? 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] twalthr commented on a change in pull request #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces
URL: https://github.com/apache/flink/pull/10781#discussion_r364680851
 
 

 ##########
 File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/ConstantArgumentCount.java
 ##########
 @@ -60,16 +60,20 @@ public static ArgumentCount from(int minCount) {
 	}
 
 	public static ArgumentCount any() {
-		return new ConstantArgumentCount(0, OPEN_INTERVAL);
+		return new ConstantArgumentCount(OPEN_INTERVAL, OPEN_INTERVAL);
 
 Review comment:
   I will skip this change for now. I remember there was some issue while mapping to Calcite but I'm sure we will notice it when we have more tests.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot commented on issue #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces

Posted by GitBox <gi...@apache.org>.
flinkbot commented on issue #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces
URL: https://github.com/apache/flink/pull/10781#issuecomment-571203038
 
 
   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit cbb7375c1f8b09d99956c52936c8a2fa6af72b2f (Mon Jan 06 16:18:58 UTC 2020)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces
URL: https://github.com/apache/flink/pull/10781#issuecomment-571250359
 
 
   <!--
   Meta data
   Hash:85dc4bd6420b9354e6cf3f2fd9387077ed631894 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/143281261 TriggerType:PUSH TriggerID:85dc4bd6420b9354e6cf3f2fd9387077ed631894
   Hash:85dc4bd6420b9354e6cf3f2fd9387077ed631894 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4139 TriggerType:PUSH TriggerID:85dc4bd6420b9354e6cf3f2fd9387077ed631894
   Hash:982a7d1256e933e02ebcf4bcb994ab676ff8510b Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/143718894 TriggerType:PUSH TriggerID:982a7d1256e933e02ebcf4bcb994ab676ff8510b
   Hash:982a7d1256e933e02ebcf4bcb994ab676ff8510b Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4220 TriggerType:PUSH TriggerID:982a7d1256e933e02ebcf4bcb994ab676ff8510b
   -->
   ## CI report:
   
   * 85dc4bd6420b9354e6cf3f2fd9387077ed631894 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/143281261) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4139) 
   * 982a7d1256e933e02ebcf4bcb994ab676ff8510b Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/143718894) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4220) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10781: [FLINK-15281][table-planner-blink] Map Flink's TypeInference to Calcite's interfaces
URL: https://github.com/apache/flink/pull/10781#issuecomment-571250359
 
 
   <!--
   Meta data
   Hash:85dc4bd6420b9354e6cf3f2fd9387077ed631894 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/143281261 TriggerType:PUSH TriggerID:85dc4bd6420b9354e6cf3f2fd9387077ed631894
   Hash:85dc4bd6420b9354e6cf3f2fd9387077ed631894 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4139 TriggerType:PUSH TriggerID:85dc4bd6420b9354e6cf3f2fd9387077ed631894
   Hash:982a7d1256e933e02ebcf4bcb994ab676ff8510b Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:982a7d1256e933e02ebcf4bcb994ab676ff8510b
   -->
   ## CI report:
   
   * 85dc4bd6420b9354e6cf3f2fd9387077ed631894 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/143281261) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4139) 
   * 982a7d1256e933e02ebcf4bcb994ab676ff8510b UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services