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/02/13 08:51:19 UTC

[GitHub] [flink] dawidwys opened a new pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

dawidwys opened a new pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081
 
 
   ## What is the purpose of the change
   
   This PR adds a Java Expression DSL as described in FLIP-55: https://cwiki.apache.org/confluence/display/FLINK/FLIP-55%3A+Introduction+of+a+Table+API+Java+Expression+DSL
   
   ## Brief change log
   
   *(for example:)*
     - Introduced Java Expression DSL equivalent to the Scala's one
     - Added `interval(Period)` and `interval(Duration)` expressions as a substitute for `.millis`, `.months`, `.quarters` etc.
     - Added prefix and/or expressions that accept multiple predicates. This required also changes in translation from `Expression` to `PlannerExpression`.
   
   
   ## Verifying this change
   
   - Added a test that check if there are equivalent methods in both Scala's and Java's APIs.
   - Adjusted a test `JavaTableEnvironmentITCase.java` to use the Java's DSL
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (yes / **no**)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (**yes** / no)
     - The serializers: (yes / **no** / don't know)
     - The runtime per-record code paths (performance sensitive): (yes / **no** / don't know)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: (yes / **no** / don't know)
     - The S3 file system connector: (yes / **no** / don't know)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (yes / **no**)
     - If yes, how is the feature documented? (not applicable / docs / JavaDocs / **not documented**) - will be part of a next PR
   

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r385595579
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/utils/factories/CalculatedTableFactory.java
 ##########
 @@ -95,42 +94,87 @@ public FunctionTableCallVisitor(String[] leftTableFieldNames) {
 			}
 
 			CallExpression tableCall = (CallExpression) children.get(0);
-			TableFunctionDefinition tableFunctionDefinition =
-				(TableFunctionDefinition) tableCall.getFunctionDefinition();
-			return createFunctionCall(tableFunctionDefinition, aliases, tableCall.getResolvedChildren());
+			return createFunctionCall(tableCall, aliases);
+		}
+
+		private CalculatedQueryOperation<?> createFunctionCall(CallExpression call, List<String> aliases) {
+			FunctionDefinition definition = call.getFunctionDefinition();
+			if (definition instanceof TableFunctionDefinition) {
+				return createFunctionCall(
+					((TableFunctionDefinition) definition).getTableFunction(),
+					call.getFunctionIdentifier().orElse(FunctionIdentifier.of(definition.toString())),
+					call.getOutputDataType(),
+					aliases,
+					call.getResolvedChildren());
+			} else if (definition instanceof TableFunction<?>) {
 
 Review comment:
   Check function kind instead?

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380048015
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,527 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.internal.BaseExpressions;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.ExpressionVisitor;
+import org.apache.flink.table.expressions.TimePointUnit;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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 java.util.Arrays;
+import java.util.List;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral;
+
+/**
+ * Entry point for a Java Table Expression DSL.
+ */
+@PublicEvolving
+public final class Expressions {
+
+	/**
+	 * Creates an unresolved reference to a table's column.
+	 */
+	//CHECKSTYLE.OFF: MethodName
+	public static ApiExpression $(String name) {
+		return new ApiExpression(ApiExpressionUtils.unresolvedRef(name));
+	}
+	//CHECKSTYLE.ON: MethodName
+
+	/**
+	 * Creates a SQL literal. The data type is derived from the Java class.
+	 */
+	public static ApiExpression lit(Object v) {
+		return new ApiExpression(valueLiteral(v));
+	}
+
+	/**
+	 * Creates a SQL literal of a given {@link DataType}.
 
 Review comment:
   Mention that
   - the other `lit` is recommended
   - `LogicalType#supportsInputConversion` must support the 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] flinkbot commented on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot commented on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585618441
 
 
   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 16d4e9e6fbeb230b8aa05976e4909299ebc929bf (Thu Feb 13 08:53:10 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] dawidwys commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
dawidwys commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r381281782
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/UnresolvedCallExpression.java
 ##########
 @@ -61,8 +60,10 @@ public UnresolvedCallExpression(
 			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 = Preconditions.checkNotNull(args)
 
 Review comment:
   I opened #11141 for it.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380043664
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,527 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.internal.BaseExpressions;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.ExpressionVisitor;
+import org.apache.flink.table.expressions.TimePointUnit;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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 java.util.Arrays;
+import java.util.List;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral;
+
+/**
+ * Entry point for a Java Table Expression DSL.
+ */
+@PublicEvolving
+public final class Expressions {
+
+	/**
+	 * Creates an unresolved reference to a table's column.
 
 Review comment:
   add an example maybe within a `.select(...)`

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380066320
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,527 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.internal.BaseExpressions;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.ExpressionVisitor;
+import org.apache.flink.table.expressions.TimePointUnit;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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 java.util.Arrays;
+import java.util.List;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral;
+
+/**
+ * Entry point for a Java Table Expression DSL.
+ */
+@PublicEvolving
+public final class Expressions {
+
+	/**
+	 * Creates an unresolved reference to a table's column.
+	 */
+	//CHECKSTYLE.OFF: MethodName
+	public static ApiExpression $(String name) {
+		return new ApiExpression(ApiExpressionUtils.unresolvedRef(name));
+	}
+	//CHECKSTYLE.ON: MethodName
+
+	/**
+	 * Creates a SQL literal. The data type is derived from the Java class.
+	 */
+	public static ApiExpression lit(Object v) {
+		return new ApiExpression(valueLiteral(v));
+	}
+
+	/**
+	 * Creates a SQL literal of a given {@link DataType}.
+	 */
+	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.
+	 *
+	 * @see #withColumns(Object, Object...)
+	 * @see #withoutColumns(Object, Object...)
+	 */
+	public static ApiExpression range(String start, String end) {
+		return apiCall(BuiltInFunctionDefinitions.RANGE_TO, valueLiteral(start), valueLiteral(end));
+	}
+
+	/**
+	 * Indicates an index based range, which can be used in columns selection.
+	 *
+	 * @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 apiCall(BuiltInFunctionDefinitions.AND, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Boolean OR in three-valued logic.
+	 */
+	public static ApiExpression or(Object predicate0, Object predicate1, Object... predicates) {
+		return apiCall(BuiltInFunctionDefinitions.OR, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 <code>leftEnd >= rightStart && rightEnd >= leftStart</code>.
+	 *
+	 * <p>It evaluates: leftEnd >= rightStart && rightEnd >= leftStart
+	 *
+	 * <p>e.g. temporalOverlaps("2:55:00".toTime, 1.hour, "3:30:00".toTime, 2.hour) leads to true
+	 */
+	public static ApiExpression temporalOverlaps(
+			Object leftTimePoint,
+			Object leftTemporal,
+			Object rightTimePoint,
+			Object rightTemporal) {
+		return apiCall(
+			BuiltInFunctionDefinitions.TEMPORAL_OVERLAPS,
+			objectToExpression(leftTimePoint),
+			objectToExpression(leftTemporal),
+			objectToExpression(rightTimePoint),
+			objectToExpression(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.
+	 *
+	 * <p>For example 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,
+			objectToExpression(timestamp),
+			objectToExpression(format));
+	}
+
+	/**
+	 * Returns the (signed) number of {@link TimePointUnit} between timePoint1 and timePoint2.
+	 *
+	 * <p>For example, {@code timestampDiff(TimePointUnit.DAY, '2016-06-15'.toDate, '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),
+			objectToExpression(timePoint1),
+			objectToExpression(timePoint2));
+	}
+
+	/**
+	 * Creates an array of literals.
+	 */
+	public static ApiExpression array(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.ARRAY,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a row of expressions.
+	 */
+	public static ApiExpression row(Object head, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.ROW, Stream.concat(
+			Stream.of(head),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a map of expressions.
 
 Review comment:
   This should have an example. Because this key, value stuff might be confusing.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r385005863
 
 

 ##########
 File path: flink-table/flink-table-api-scala/src/main/scala/org/apache/flink/table/api/expressionDsl.scala
 ##########
 @@ -1284,47 +490,85 @@ 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: _*)
+
+  /**
+   * 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
   // ----------------------------------------------------------------------------------------------
 
+  /**
+   * 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 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 minus(v: Expression): Expression = {
 
 Review comment:
   should we name it negative? I don't know.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148755202",
       "triggerID" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149801561",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149806101",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149869824",
       "triggerID" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149885015",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149970663",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149981047",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150432772",
       "triggerID" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150669044",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5643",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d510ce90e5cd8a13256c84c855562c403091a14",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150861986",
       "triggerID" : "8d510ce90e5cd8a13256c84c855562c403091a14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3786dab008166eff376617dabda4dedeba7ca961",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3786dab008166eff376617dabda4dedeba7ca961",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8d510ce90e5cd8a13256c84c855562c403091a14 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/150861986) 
   * 3786dab008166eff376617dabda4dedeba7ca961 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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380076793
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java
 ##########
 @@ -0,0 +1,1146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.Table;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.TimeIntervalUnit;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.tableRef;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.typeLiteral;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.unresolvedCall;
+import static org.apache.flink.table.expressions.utils.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.
+ */
+@PublicEvolving
+public abstract class BaseExpressions<InT, OutT> {
 
 Review comment:
   add JavaDocs for generics, why we need them esp for Scala.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148755202 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   Hash:93f931c7ef96767509f62982e48b4853d3a74dfc Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/149801561 TriggerType:PUSH TriggerID:93f931c7ef96767509f62982e48b4853d3a74dfc
   Hash:93f931c7ef96767509f62982e48b4853d3a74dfc Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362 TriggerType:PUSH TriggerID:93f931c7ef96767509f62982e48b4853d3a74dfc
   Hash:b79c6583d86289244513a44c260c5c5d6b520f48 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363 TriggerType:PUSH TriggerID:b79c6583d86289244513a44c260c5c5d6b520f48
   Hash:b79c6583d86289244513a44c260c5c5d6b520f48 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149806101 TriggerType:PUSH TriggerID:b79c6583d86289244513a44c260c5c5d6b520f48
   -->
   ## CI report:
   
   * 16d4e9e6fbeb230b8aa05976e4909299ebc929bf Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148755202) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128) 
   * 93f931c7ef96767509f62982e48b4853d3a74dfc Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/149801561) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362) 
   * b79c6583d86289244513a44c260c5c5d6b520f48 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149806101) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363) 
   
   <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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r385598339
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/utils/factories/CalculatedTableFactory.java
 ##########
 @@ -95,42 +94,87 @@ public FunctionTableCallVisitor(String[] leftTableFieldNames) {
 			}
 
 			CallExpression tableCall = (CallExpression) children.get(0);
-			TableFunctionDefinition tableFunctionDefinition =
-				(TableFunctionDefinition) tableCall.getFunctionDefinition();
-			return createFunctionCall(tableFunctionDefinition, aliases, tableCall.getResolvedChildren());
+			return createFunctionCall(tableCall, aliases);
+		}
+
+		private CalculatedQueryOperation<?> createFunctionCall(CallExpression call, List<String> aliases) {
+			FunctionDefinition definition = call.getFunctionDefinition();
+			if (definition instanceof TableFunctionDefinition) {
+				return createFunctionCall(
+					((TableFunctionDefinition) definition).getTableFunction(),
+					call.getFunctionIdentifier().orElse(FunctionIdentifier.of(definition.toString())),
+					call.getOutputDataType(),
+					aliases,
+					call.getResolvedChildren());
+			} else if (definition instanceof TableFunction<?>) {
+				return createFunctionCall(
+					(TableFunction<?>) definition,
+					call.getFunctionIdentifier().orElse(FunctionIdentifier.of(definition.toString())),
+					call.getOutputDataType(),
+					aliases,
+					call.getResolvedChildren());
+			} else {
+				return defaultMethod(call);
+			}
 		}
 
 		private CalculatedQueryOperation<?> createFunctionCall(
-				TableFunctionDefinition tableFunctionDefinition,
+				TableFunction<?> tableFunction,
+				FunctionIdentifier identifier,
+				DataType resultType,
 				List<String> aliases,
 				List<ResolvedExpression> parameters) {
-			TypeInformation<?> resultType = tableFunctionDefinition.getResultType();
 
-			int callArity = resultType.getTotalFields();
-			int aliasesSize = aliases.size();
+			final TableSchema tableSchema = adjustNames(
+				extractSchema(resultType),
+				aliases,
+				identifier);
 
-			String[] fieldNames;
+			return new CalculatedQueryOperation(
+				tableFunction,
 
 Review comment:
   Pass the function definition as a marker for `QueryOperationConverter`. Otherwise it will still use the old stack. We should use `BridgingSqlFunction` there. We should be able to use `relBuilder.functionScan`.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148755202",
       "triggerID" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149801561",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149806101",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149869824",
       "triggerID" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149885015",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149970663",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149981047",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150432772",
       "triggerID" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150669044",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5643",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d510ce90e5cd8a13256c84c855562c403091a14",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150861986",
       "triggerID" : "8d510ce90e5cd8a13256c84c855562c403091a14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3786dab008166eff376617dabda4dedeba7ca961",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3786dab008166eff376617dabda4dedeba7ca961",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0e3ee0c3b8b58815b2f771d8b4320e7483c41159",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0e3ee0c3b8b58815b2f771d8b4320e7483c41159",
       "triggerType" : "PUSH"
     }, {
       "hash" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5706",
       "triggerID" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150885626",
       "triggerID" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f9502902a9c1a3203a22fdeb91f47551c3e862d1",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150974862",
       "triggerID" : "f9502902a9c1a3203a22fdeb91f47551c3e862d1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f9502902a9c1a3203a22fdeb91f47551c3e862d1",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5721",
       "triggerID" : "f9502902a9c1a3203a22fdeb91f47551c3e862d1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3786dab008166eff376617dabda4dedeba7ca961 UNKNOWN
   * 0e3ee0c3b8b58815b2f771d8b4320e7483c41159 UNKNOWN
   * 156ddaa5298b8059b4038c486e00770a263c25f7 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/150885626) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5706) 
   * f9502902a9c1a3203a22fdeb91f47551c3e862d1 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/150974862) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5721) 
   
   <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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
dawidwys commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r385216930
 
 

 ##########
 File path: flink-table/flink-table-api-scala/src/test/scala/org/apache/flink/table/api/ExpressionsConsistencyCheckTest.scala
 ##########
 @@ -0,0 +1,294 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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" -> "multipliedBy", // *
+    "$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 rowsInterval
 
 Review comment:
   That's a type in the comment.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148755202 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   Hash:93f931c7ef96767509f62982e48b4853d3a74dfc Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/149801561 TriggerType:PUSH TriggerID:93f931c7ef96767509f62982e48b4853d3a74dfc
   Hash:93f931c7ef96767509f62982e48b4853d3a74dfc Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362 TriggerType:PUSH TriggerID:93f931c7ef96767509f62982e48b4853d3a74dfc
   Hash:b79c6583d86289244513a44c260c5c5d6b520f48 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363 TriggerType:PUSH TriggerID:b79c6583d86289244513a44c260c5c5d6b520f48
   Hash:b79c6583d86289244513a44c260c5c5d6b520f48 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149806101 TriggerType:PUSH TriggerID:b79c6583d86289244513a44c260c5c5d6b520f48
   Hash:91b982ff6a238b4db2bc89e81e53327d731e9378 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/149869824 TriggerType:PUSH TriggerID:91b982ff6a238b4db2bc89e81e53327d731e9378
   Hash:01b03edabdb80d7a941b10ecccb2670973d8b64a Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/149885015 TriggerType:PUSH TriggerID:01b03edabdb80d7a941b10ecccb2670973d8b64a
   -->
   ## CI report:
   
   * 16d4e9e6fbeb230b8aa05976e4909299ebc929bf Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148755202) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128) 
   * 93f931c7ef96767509f62982e48b4853d3a74dfc Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/149801561) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362) 
   * b79c6583d86289244513a44c260c5c5d6b520f48 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149806101) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363) 
   * 91b982ff6a238b4db2bc89e81e53327d731e9378 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/149869824) 
   * 01b03edabdb80d7a941b10ecccb2670973d8b64a Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/149885015) 
   
   <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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148755202",
       "triggerID" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149801561",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149806101",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149869824",
       "triggerID" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149885015",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149970663",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149981047",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150432772",
       "triggerID" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150669044",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5643",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d510ce90e5cd8a13256c84c855562c403091a14",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150861986",
       "triggerID" : "8d510ce90e5cd8a13256c84c855562c403091a14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3786dab008166eff376617dabda4dedeba7ca961",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3786dab008166eff376617dabda4dedeba7ca961",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0e3ee0c3b8b58815b2f771d8b4320e7483c41159",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0e3ee0c3b8b58815b2f771d8b4320e7483c41159",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8d510ce90e5cd8a13256c84c855562c403091a14 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/150861986) 
   * 3786dab008166eff376617dabda4dedeba7ca961 UNKNOWN
   * 0e3ee0c3b8b58815b2f771d8b4320e7483c41159 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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380098061
 
 

 ##########
 File path: flink-table/flink-table-api-scala/src/test/scala/org/apache/flink/table/api/ExpressionsConsistencyCheckTest.scala
 ##########
 @@ -0,0 +1,301 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.Expression
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils.{unresolvedCall, unresolvedRef, valueLiteral}
+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._
+
+class ExpressionsConsistencyCheckTest {
 
 Review comment:
   Add explanation what we test here.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380077712
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java
 ##########
 @@ -0,0 +1,1146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.Table;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.TimeIntervalUnit;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.tableRef;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.typeLiteral;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.unresolvedCall;
+import static org.apache.flink.table.expressions.utils.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.
+ */
+@PublicEvolving
+public abstract class BaseExpressions<InT, OutT> {
+	protected abstract Expression toExpr();
+
+	protected abstract OutT 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
+	 * @return field with an alias
+	 */
+	public OutT 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.
+	 */
+	public OutT and(InT other) {
+		return toApiSpecificExpression(unresolvedCall(AND, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Boolean OR in three-valued logic.
+	 */
+	public OutT or(InT other) {
+		return toApiSpecificExpression(unresolvedCall(OR, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Greater than.
+	 */
+	public OutT isGreater(InT other) {
+		return toApiSpecificExpression(unresolvedCall(GREATER_THAN, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Greater than or equal.
+	 */
+	public OutT isGreaterOrEqual(InT other) {
+		return toApiSpecificExpression(unresolvedCall(GREATER_THAN_OR_EQUAL, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Less than.
+	 */
+	public OutT isLess(InT other) {
+		return toApiSpecificExpression(unresolvedCall(LESS_THAN, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Less than or equal.
+	 */
+	public OutT isLessOrEqual(InT other) {
+		return toApiSpecificExpression(unresolvedCall(LESS_THAN_OR_EQUAL, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Equals.
+	 */
+	public OutT isEqual(InT other) {
+		return toApiSpecificExpression(unresolvedCall(EQUALS, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Not equal.
+	 */
+	public OutT isNotEqual(InT other) {
+		return toApiSpecificExpression(unresolvedCall(NOT_EQUALS, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Returns left plus right.
+	 */
+	public OutT plus(InT other) {
+		return toApiSpecificExpression(unresolvedCall(PLUS, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Returns left minus right.
+	 */
+	public OutT minus(InT other) {
+		return toApiSpecificExpression(unresolvedCall(MINUS, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Returns left divided by right.
+	 */
+	public OutT dividedBy(InT other) {
+		return toApiSpecificExpression(unresolvedCall(DIVIDE, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Returns left multiplied by right.
+	 */
+	public OutT multipliedBy(InT 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
+	 * @return boolean or null
+	 */
+	public OutT between(InT lowerBound, InT 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
+	 * @return boolean or null
+	 */
+	public OutT notBetween(InT lowerBound, InT upperBound) {
+		return toApiSpecificExpression(unresolvedCall(
+			NOT_BETWEEN,
+			toExpr(),
+			objectToExpression(lowerBound),
+			objectToExpression(upperBound)));
+	}
+
+	/**
+	 * Returns true if the given expression is null.
+	 */
+	public OutT isNull() {
+		return toApiSpecificExpression(unresolvedCall(IS_NULL, toExpr()));
+	}
+
+	/**
+	 * Returns true if the given expression is not null.
+	 */
+	public OutT isNotNull() {
+		return toApiSpecificExpression(unresolvedCall(IS_NOT_NULL, toExpr()));
+	}
+
+	/**
+	 * Returns true if given boolean expression is true. False otherwise (for null and false).
+	 */
+	public OutT isTrue() {
+		return toApiSpecificExpression(unresolvedCall(IS_TRUE, toExpr()));
+	}
+
+	/**
+	 * Returns true if given boolean expression is false. False otherwise (for null and true).
+	 */
+	public OutT isFalse() {
+		return toApiSpecificExpression(unresolvedCall(IS_FALSE, toExpr()));
+	}
+
+	/**
+	 * Returns true if given boolean expression is not true (for null and false). False otherwise.
+	 */
+	public OutT isNotTrue() {
+		return toApiSpecificExpression(unresolvedCall(IS_NOT_TRUE, toExpr()));
+	}
+
+	/**
+	 * Returns true if given boolean expression is not false (for null and true). False otherwise.
+	 */
+	public OutT 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.
+	 *
+	 * <p>For example:
+	 * <pre>
+	 * {@code
+	 * orders
 
 Review comment:
   update and indent?

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148755202",
       "triggerID" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149801561",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149806101",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149869824",
       "triggerID" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149885015",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149970663",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149981047",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3c118088adedb8b818d0371495d313e9fe038968 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/149981047) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408) 
   
   <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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380086335
 
 

 ##########
 File path: flink-table/flink-table-api-scala/src/main/scala/org/apache/flink/table/api/expressionDsl.scala
 ##########
 @@ -355,190 +155,10 @@ trait ImplicitExpressionOperations {
     * @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
 
 Review comment:
   should we offer an infix notation for `ifThenElse`? I think it would be nice to fluently evaluate a boolean predicate.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148755202 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   Hash:93f931c7ef96767509f62982e48b4853d3a74dfc Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/149801561 TriggerType:PUSH TriggerID:93f931c7ef96767509f62982e48b4853d3a74dfc
   Hash:93f931c7ef96767509f62982e48b4853d3a74dfc Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362 TriggerType:PUSH TriggerID:93f931c7ef96767509f62982e48b4853d3a74dfc
   Hash:b79c6583d86289244513a44c260c5c5d6b520f48 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363 TriggerType:PUSH TriggerID:b79c6583d86289244513a44c260c5c5d6b520f48
   Hash:b79c6583d86289244513a44c260c5c5d6b520f48 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149806101 TriggerType:PUSH TriggerID:b79c6583d86289244513a44c260c5c5d6b520f48
   Hash:91b982ff6a238b4db2bc89e81e53327d731e9378 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/149869824 TriggerType:PUSH TriggerID:91b982ff6a238b4db2bc89e81e53327d731e9378
   Hash:01b03edabdb80d7a941b10ecccb2670973d8b64a Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/149885015 TriggerType:PUSH TriggerID:01b03edabdb80d7a941b10ecccb2670973d8b64a
   Hash:01b03edabdb80d7a941b10ecccb2670973d8b64a Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391 TriggerType:PUSH TriggerID:01b03edabdb80d7a941b10ecccb2670973d8b64a
   -->
   ## CI report:
   
   * 16d4e9e6fbeb230b8aa05976e4909299ebc929bf Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148755202) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128) 
   * 93f931c7ef96767509f62982e48b4853d3a74dfc Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/149801561) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362) 
   * b79c6583d86289244513a44c260c5c5d6b520f48 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149806101) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363) 
   * 91b982ff6a238b4db2bc89e81e53327d731e9378 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/149869824) 
   * 01b03edabdb80d7a941b10ecccb2670973d8b64a Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/149885015) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391) 
   
   <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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380099235
 
 

 ##########
 File path: flink-table/flink-table-api-scala/src/test/scala/org/apache/flink/table/api/ExpressionsConsistencyCheckTest.scala
 ##########
 @@ -0,0 +1,301 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.Expression
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils.{unresolvedCall, unresolvedRef, valueLiteral}
+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._
+
+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" -> "multipliedBy", // *
+    "$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 WithOperations
+    //-----------------------------------------------------------------------------------
+    "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 rowsInterval
+    "rows",
+
+    // not supported in java
+    "unary_$plus", // unary_+
+    "unary_$minus", // unary_-
 
 Review comment:
   should we support this in Java?

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r385009604
 
 

 ##########
 File path: flink-table/flink-table-api-scala/src/test/scala/org/apache/flink/table/api/ExpressionsConsistencyCheckTest.scala
 ##########
 @@ -0,0 +1,294 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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" -> "multipliedBy", // *
+    "$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 rowsInterval
 
 Review comment:
   Can we call it `rowInterval()`?

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
dawidwys commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r385215093
 
 

 ##########
 File path: flink-table/flink-table-api-scala/src/main/scala/org/apache/flink/table/api/expressionDsl.scala
 ##########
 @@ -1160,6 +359,13 @@ trait ImplicitExpressionConversions {
     }
   }
 
+  implicit class FieldExpression(val sc: StringContext) {
+    def $(args: Any*): Expression = unresolvedRef(sc.s(args: _*))
+  }
+
+  implicit def apiExpressionToExpression(expr: ApiExpression): Expression =
 
 Review comment:
   It's not necessary. Let's remove it.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380067128
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,527 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.internal.BaseExpressions;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.ExpressionVisitor;
+import org.apache.flink.table.expressions.TimePointUnit;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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 java.util.Arrays;
+import java.util.List;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral;
+
+/**
+ * Entry point for a Java Table Expression DSL.
+ */
+@PublicEvolving
+public final class Expressions {
+
+	/**
+	 * Creates an unresolved reference to a table's column.
+	 */
+	//CHECKSTYLE.OFF: MethodName
+	public static ApiExpression $(String name) {
+		return new ApiExpression(ApiExpressionUtils.unresolvedRef(name));
+	}
+	//CHECKSTYLE.ON: MethodName
+
+	/**
+	 * Creates a SQL literal. The data type is derived from the Java class.
+	 */
+	public static ApiExpression lit(Object v) {
+		return new ApiExpression(valueLiteral(v));
+	}
+
+	/**
+	 * Creates a SQL literal of a given {@link DataType}.
+	 */
+	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.
+	 *
+	 * @see #withColumns(Object, Object...)
+	 * @see #withoutColumns(Object, Object...)
+	 */
+	public static ApiExpression range(String start, String end) {
+		return apiCall(BuiltInFunctionDefinitions.RANGE_TO, valueLiteral(start), valueLiteral(end));
+	}
+
+	/**
+	 * Indicates an index based range, which can be used in columns selection.
+	 *
+	 * @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 apiCall(BuiltInFunctionDefinitions.AND, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Boolean OR in three-valued logic.
+	 */
+	public static ApiExpression or(Object predicate0, Object predicate1, Object... predicates) {
+		return apiCall(BuiltInFunctionDefinitions.OR, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 <code>leftEnd >= rightStart && rightEnd >= leftStart</code>.
+	 *
+	 * <p>It evaluates: leftEnd >= rightStart && rightEnd >= leftStart
+	 *
+	 * <p>e.g. temporalOverlaps("2:55:00".toTime, 1.hour, "3:30:00".toTime, 2.hour) leads to true
+	 */
+	public static ApiExpression temporalOverlaps(
+			Object leftTimePoint,
+			Object leftTemporal,
+			Object rightTimePoint,
+			Object rightTemporal) {
+		return apiCall(
+			BuiltInFunctionDefinitions.TEMPORAL_OVERLAPS,
+			objectToExpression(leftTimePoint),
+			objectToExpression(leftTemporal),
+			objectToExpression(rightTimePoint),
+			objectToExpression(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.
+	 *
+	 * <p>For example 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,
+			objectToExpression(timestamp),
+			objectToExpression(format));
+	}
+
+	/**
+	 * Returns the (signed) number of {@link TimePointUnit} between timePoint1 and timePoint2.
+	 *
+	 * <p>For example, {@code timestampDiff(TimePointUnit.DAY, '2016-06-15'.toDate, '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),
+			objectToExpression(timePoint1),
+			objectToExpression(timePoint2));
+	}
+
+	/**
+	 * Creates an array of literals.
+	 */
+	public static ApiExpression array(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.ARRAY,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a row of expressions.
+	 */
+	public static ApiExpression row(Object head, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.ROW, Stream.concat(
+			Stream.of(head),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a map of expressions.
+	 */
+	public static ApiExpression map(Object key, Object value, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.MAP, Stream.concat(
 
 Review comment:
   nit: overload `apiCall` to support one or two minimum args?

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r384996469
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java
 ##########
 @@ -0,0 +1,1286 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 <InType>  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 <OutType> 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<InType, OutType> {
+	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 org.apache.flink.table.api.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 org.apache.flink.table.api.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 multipliedBy(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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>For example:
+	 * <pre>
+	 * {@code
+	 * orders
+	 *  .groupBy($("a"))
+	 *  .select($("a"), $("b").sum().distinct().as("d"))
+	 * }
+	 * </pre>
+	 */
+	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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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()));
+	}
+
+	// scalar functions
+
+	/**
+	 * 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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>e.g. "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.
+	 *
+	 * <p>e.g. "hi".lpad(4, '??') returns "??hi",  "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.
+	 *
+	 * <p>e.g. "hi".rpad(4, '??') returns "hi??",  "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.
+	 *
+	 * <p>For example:
+	 *
+	 * <pre>
+	 * {@code
+	 * table
+	 * .window(Over partitionBy 'c orderBy 'rowtime preceding 2.rows following CURRENT_ROW as 'w)
 
 Review comment:
   nit: indent with two spaces

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380070544
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,527 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.internal.BaseExpressions;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.ExpressionVisitor;
+import org.apache.flink.table.expressions.TimePointUnit;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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 java.util.Arrays;
+import java.util.List;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral;
+
+/**
+ * Entry point for a Java Table Expression DSL.
+ */
+@PublicEvolving
+public final class Expressions {
+
+	/**
+	 * Creates an unresolved reference to a table's column.
+	 */
+	//CHECKSTYLE.OFF: MethodName
+	public static ApiExpression $(String name) {
+		return new ApiExpression(ApiExpressionUtils.unresolvedRef(name));
+	}
+	//CHECKSTYLE.ON: MethodName
+
+	/**
+	 * Creates a SQL literal. The data type is derived from the Java class.
+	 */
+	public static ApiExpression lit(Object v) {
+		return new ApiExpression(valueLiteral(v));
+	}
+
+	/**
+	 * Creates a SQL literal of a given {@link DataType}.
+	 */
+	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.
+	 *
+	 * @see #withColumns(Object, Object...)
+	 * @see #withoutColumns(Object, Object...)
+	 */
+	public static ApiExpression range(String start, String end) {
+		return apiCall(BuiltInFunctionDefinitions.RANGE_TO, valueLiteral(start), valueLiteral(end));
+	}
+
+	/**
+	 * Indicates an index based range, which can be used in columns selection.
+	 *
+	 * @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 apiCall(BuiltInFunctionDefinitions.AND, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Boolean OR in three-valued logic.
+	 */
+	public static ApiExpression or(Object predicate0, Object predicate1, Object... predicates) {
+		return apiCall(BuiltInFunctionDefinitions.OR, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 <code>leftEnd >= rightStart && rightEnd >= leftStart</code>.
+	 *
+	 * <p>It evaluates: leftEnd >= rightStart && rightEnd >= leftStart
+	 *
+	 * <p>e.g. temporalOverlaps("2:55:00".toTime, 1.hour, "3:30:00".toTime, 2.hour) leads to true
+	 */
+	public static ApiExpression temporalOverlaps(
+			Object leftTimePoint,
+			Object leftTemporal,
+			Object rightTimePoint,
+			Object rightTemporal) {
+		return apiCall(
+			BuiltInFunctionDefinitions.TEMPORAL_OVERLAPS,
+			objectToExpression(leftTimePoint),
+			objectToExpression(leftTemporal),
+			objectToExpression(rightTimePoint),
+			objectToExpression(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.
+	 *
+	 * <p>For example 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,
+			objectToExpression(timestamp),
+			objectToExpression(format));
+	}
+
+	/**
+	 * Returns the (signed) number of {@link TimePointUnit} between timePoint1 and timePoint2.
+	 *
+	 * <p>For example, {@code timestampDiff(TimePointUnit.DAY, '2016-06-15'.toDate, '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),
+			objectToExpression(timePoint1),
+			objectToExpression(timePoint2));
+	}
+
+	/**
+	 * Creates an array of literals.
+	 */
+	public static ApiExpression array(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.ARRAY,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a row of expressions.
+	 */
+	public static ApiExpression row(Object head, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.ROW, Stream.concat(
+			Stream.of(head),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a map of expressions.
+	 */
+	public static ApiExpression map(Object key, Object value, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.MAP, Stream.concat(
+			Stream.of(key, value),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 apiCall(
+			BuiltInFunctionDefinitions.CONCAT,
+			Stream.concat(
+				Stream.of(string),
+				Stream.of(strings)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Calculates the arc tangent of a given coordinate.
+	 */
+	public static ApiExpression atan2(Object y, Object x) {
+		return apiCall(BuiltInFunctionDefinitions.ATAN2, objectToExpression(y), objectToExpression(x));
+	}
+
+	/**
+	 * Returns the string that results from concatenating the arguments and separator.
+	 * Returns NULL If the separator is NULL.
+	 *
+	 * <p>Note: this user-public static ApiExpressionined 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 apiCall(BuiltInFunctionDefinitions.CONCAT_WS, Stream.concat(
+			Stream.of(separator, string),
+			Stream.of(strings)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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.
+	 *
+	 * <p>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, objectToExpression(value));
+	}
+
+	/**
+	 * Calculates the logarithm of the given value to the given base.
+	 */
+	public static ApiExpression log(Object base, Object value) {
+		return apiCall(BuiltInFunctionDefinitions.LOG, objectToExpression(base), objectToExpression(value));
+	}
+
+	/**
+	 * Ternary conditional operator that decides which of two other expressions should be evaluated
+	 * based on a evaluated boolean condition.
+	 *
+	 * <p>e.g. ifThenElse(42 > 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,
+			objectToExpression(condition),
+			objectToExpression(ifTrue),
+			objectToExpression(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.
+	 *
+	 * <p>A range can either be index-based or name-based. Indices start at 1 and boundaries are
+	 * inclusive.
+	 *
+	 * <p>e.g. withColumns('b to 'c) or withColumns('*)
+	 */
+	public static ApiExpression withColumns(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.WITH_COLUMNS,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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.
+	 *
+	 * <p>A range can either be index-based or name-based. Indices start at 1 and boundaries are
+	 * inclusive.
+	 *
+	 * <p>e.g. withoutColumns('b to 'c) or withoutColumns('c)
+	 */
+	public static ApiExpression withoutColumns(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.WITHOUT_COLUMNS,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * A call to a function that will be looked up in a catalog.
+	 */
+	public static ApiExpression call(String functionName, Object... params) {
+		return new ApiExpression(ApiExpressionUtils.lookupCall(
+			functionName,
+			Arrays.stream(params).map(ApiExpressionUtils::objectToExpression).toArray(Expression[]::new)));
+	}
+
+	/**
+	 * A call to an inline function. For functions registered in a catalog use {@link #call(String, Object...)}.
+	 */
+	public static ApiExpression call(FunctionDefinition scalarFunction, Object... params) {
 
 Review comment:
   not only scalar function

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r384996806
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java
 ##########
 @@ -0,0 +1,1286 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 <InType>  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 <OutType> 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<InType, OutType> {
+	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 org.apache.flink.table.api.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 org.apache.flink.table.api.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 multipliedBy(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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>For example:
+	 * <pre>
+	 * {@code
+	 * orders
+	 *  .groupBy($("a"))
+	 *  .select($("a"), $("b").sum().distinct().as("d"))
+	 * }
+	 * </pre>
+	 */
+	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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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()));
+	}
+
+	// scalar functions
+
+	/**
+	 * 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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>e.g. "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.
+	 *
+	 * <p>e.g. "hi".lpad(4, '??') returns "??hi",  "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.
+	 *
+	 * <p>e.g. "hi".rpad(4, '??') returns "hi??",  "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.
+	 *
+	 * <p>For example:
+	 *
+	 * <pre>
+	 * {@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)
+	 * }</pre>
+	 */
+	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).
+	 *
+	 * <p>e.g. "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.
+	 *
+	 * <p>e.g. "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.
+	 *
+	 * <p>e.g. "2006-06-05".toDate.extract(DAY) leads to 5
 
 Review comment:
   update to Java?

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
dawidwys commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380103016
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,527 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.internal.BaseExpressions;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.ExpressionVisitor;
+import org.apache.flink.table.expressions.TimePointUnit;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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 java.util.Arrays;
+import java.util.List;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral;
+
+/**
+ * Entry point for a Java Table Expression DSL.
+ */
+@PublicEvolving
+public final class Expressions {
+
+	/**
+	 * Creates an unresolved reference to a table's column.
+	 */
+	//CHECKSTYLE.OFF: MethodName
+	public static ApiExpression $(String name) {
+		return new ApiExpression(ApiExpressionUtils.unresolvedRef(name));
+	}
+	//CHECKSTYLE.ON: MethodName
+
+	/**
+	 * Creates a SQL literal. The data type is derived from the Java class.
+	 */
+	public static ApiExpression lit(Object v) {
+		return new ApiExpression(valueLiteral(v));
+	}
+
+	/**
+	 * Creates a SQL literal of a given {@link DataType}.
+	 */
+	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.
+	 *
+	 * @see #withColumns(Object, Object...)
+	 * @see #withoutColumns(Object, Object...)
+	 */
+	public static ApiExpression range(String start, String end) {
+		return apiCall(BuiltInFunctionDefinitions.RANGE_TO, valueLiteral(start), valueLiteral(end));
+	}
+
+	/**
+	 * Indicates an index based range, which can be used in columns selection.
+	 *
+	 * @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 apiCall(BuiltInFunctionDefinitions.AND, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Boolean OR in three-valued logic.
+	 */
+	public static ApiExpression or(Object predicate0, Object predicate1, Object... predicates) {
+		return apiCall(BuiltInFunctionDefinitions.OR, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 <code>leftEnd >= rightStart && rightEnd >= leftStart</code>.
+	 *
+	 * <p>It evaluates: leftEnd >= rightStart && rightEnd >= leftStart
+	 *
+	 * <p>e.g. temporalOverlaps("2:55:00".toTime, 1.hour, "3:30:00".toTime, 2.hour) leads to true
+	 */
+	public static ApiExpression temporalOverlaps(
+			Object leftTimePoint,
+			Object leftTemporal,
+			Object rightTimePoint,
+			Object rightTemporal) {
+		return apiCall(
+			BuiltInFunctionDefinitions.TEMPORAL_OVERLAPS,
+			objectToExpression(leftTimePoint),
+			objectToExpression(leftTemporal),
+			objectToExpression(rightTimePoint),
+			objectToExpression(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.
+	 *
+	 * <p>For example 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,
+			objectToExpression(timestamp),
+			objectToExpression(format));
+	}
+
+	/**
+	 * Returns the (signed) number of {@link TimePointUnit} between timePoint1 and timePoint2.
+	 *
+	 * <p>For example, {@code timestampDiff(TimePointUnit.DAY, '2016-06-15'.toDate, '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),
+			objectToExpression(timePoint1),
+			objectToExpression(timePoint2));
+	}
+
+	/**
+	 * Creates an array of literals.
+	 */
+	public static ApiExpression array(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.ARRAY,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a row of expressions.
+	 */
+	public static ApiExpression row(Object head, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.ROW, Stream.concat(
+			Stream.of(head),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a map of expressions.
+	 */
+	public static ApiExpression map(Object key, Object value, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.MAP, Stream.concat(
+			Stream.of(key, value),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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() {
 
 Review comment:
   I think the reason why it is not a constant is that it is usually a function in other RDBMS:
   https://docs.oracle.com/cd/E17952_01/mysql-5.0-en/mathematical-functions.html
   https://docs.microsoft.com/en-us/sql/t-sql/functions/pi-transact-sql?view=sql-server-ver15
   
   Whereas the `UNBOUNDED_RANGE` are really symbols/part of the over clause syntax.
   As far as I can tell, the intention was to be as close to the sql as possible.
   
   So that we have:
   ```
   table.select(pi()) 
   //but
   table.window(Over.orderBy($("h")).preceding(CURRENT_RANGE).following(UNBOUNDED_RANGE).as("w")).select(...)
   ```
   
   The same as in SQL:
   ```
   SELECT PI() FROM table;
   //but
   SELECT ... OVER (order by h preceding CURRENT_RANGE following UNBOUNDED_RANGE) AS w FROM table
   ```

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380079587
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableImpl.java
 ##########
 @@ -563,6 +559,16 @@ private TableImpl createTable(QueryOperation operation) {
 		return new TableImpl(tableEnvironment, operation, operationTreeBuilder, lookupResolver);
 	}
 
+	private List<Expression> preprocessExpressions(List<Expression> expressions) {
 
 Review comment:
   nit: `preProcess..`

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380070079
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,527 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.internal.BaseExpressions;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.ExpressionVisitor;
+import org.apache.flink.table.expressions.TimePointUnit;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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 java.util.Arrays;
+import java.util.List;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral;
+
+/**
+ * Entry point for a Java Table Expression DSL.
+ */
+@PublicEvolving
+public final class Expressions {
+
+	/**
+	 * Creates an unresolved reference to a table's column.
+	 */
+	//CHECKSTYLE.OFF: MethodName
+	public static ApiExpression $(String name) {
+		return new ApiExpression(ApiExpressionUtils.unresolvedRef(name));
+	}
+	//CHECKSTYLE.ON: MethodName
+
+	/**
+	 * Creates a SQL literal. The data type is derived from the Java class.
+	 */
+	public static ApiExpression lit(Object v) {
+		return new ApiExpression(valueLiteral(v));
+	}
+
+	/**
+	 * Creates a SQL literal of a given {@link DataType}.
+	 */
+	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.
+	 *
+	 * @see #withColumns(Object, Object...)
+	 * @see #withoutColumns(Object, Object...)
+	 */
+	public static ApiExpression range(String start, String end) {
+		return apiCall(BuiltInFunctionDefinitions.RANGE_TO, valueLiteral(start), valueLiteral(end));
+	}
+
+	/**
+	 * Indicates an index based range, which can be used in columns selection.
+	 *
+	 * @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 apiCall(BuiltInFunctionDefinitions.AND, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Boolean OR in three-valued logic.
+	 */
+	public static ApiExpression or(Object predicate0, Object predicate1, Object... predicates) {
+		return apiCall(BuiltInFunctionDefinitions.OR, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 <code>leftEnd >= rightStart && rightEnd >= leftStart</code>.
+	 *
+	 * <p>It evaluates: leftEnd >= rightStart && rightEnd >= leftStart
+	 *
+	 * <p>e.g. temporalOverlaps("2:55:00".toTime, 1.hour, "3:30:00".toTime, 2.hour) leads to true
+	 */
+	public static ApiExpression temporalOverlaps(
+			Object leftTimePoint,
+			Object leftTemporal,
+			Object rightTimePoint,
+			Object rightTemporal) {
+		return apiCall(
+			BuiltInFunctionDefinitions.TEMPORAL_OVERLAPS,
+			objectToExpression(leftTimePoint),
+			objectToExpression(leftTemporal),
+			objectToExpression(rightTimePoint),
+			objectToExpression(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.
+	 *
+	 * <p>For example 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,
+			objectToExpression(timestamp),
+			objectToExpression(format));
+	}
+
+	/**
+	 * Returns the (signed) number of {@link TimePointUnit} between timePoint1 and timePoint2.
+	 *
+	 * <p>For example, {@code timestampDiff(TimePointUnit.DAY, '2016-06-15'.toDate, '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),
+			objectToExpression(timePoint1),
+			objectToExpression(timePoint2));
+	}
+
+	/**
+	 * Creates an array of literals.
+	 */
+	public static ApiExpression array(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.ARRAY,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a row of expressions.
+	 */
+	public static ApiExpression row(Object head, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.ROW, Stream.concat(
+			Stream.of(head),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a map of expressions.
+	 */
+	public static ApiExpression map(Object key, Object value, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.MAP, Stream.concat(
+			Stream.of(key, value),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 apiCall(
+			BuiltInFunctionDefinitions.CONCAT,
+			Stream.concat(
+				Stream.of(string),
+				Stream.of(strings)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Calculates the arc tangent of a given coordinate.
+	 */
+	public static ApiExpression atan2(Object y, Object x) {
+		return apiCall(BuiltInFunctionDefinitions.ATAN2, objectToExpression(y), objectToExpression(x));
+	}
+
+	/**
+	 * Returns the string that results from concatenating the arguments and separator.
+	 * Returns NULL If the separator is NULL.
+	 *
+	 * <p>Note: this user-public static ApiExpressionined 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 apiCall(BuiltInFunctionDefinitions.CONCAT_WS, Stream.concat(
+			Stream.of(separator, string),
+			Stream.of(strings)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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.
+	 *
+	 * <p>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, objectToExpression(value));
+	}
+
+	/**
+	 * Calculates the logarithm of the given value to the given base.
+	 */
+	public static ApiExpression log(Object base, Object value) {
+		return apiCall(BuiltInFunctionDefinitions.LOG, objectToExpression(base), objectToExpression(value));
+	}
+
+	/**
+	 * Ternary conditional operator that decides which of two other expressions should be evaluated
+	 * based on a evaluated boolean condition.
+	 *
+	 * <p>e.g. ifThenElse(42 > 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,
+			objectToExpression(condition),
+			objectToExpression(ifTrue),
+			objectToExpression(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.
+	 *
+	 * <p>A range can either be index-based or name-based. Indices start at 1 and boundaries are
+	 * inclusive.
+	 *
+	 * <p>e.g. withColumns('b to 'c) or withColumns('*)
+	 */
+	public static ApiExpression withColumns(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.WITH_COLUMNS,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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.
+	 *
+	 * <p>A range can either be index-based or name-based. Indices start at 1 and boundaries are
+	 * inclusive.
+	 *
+	 * <p>e.g. withoutColumns('b to 'c) or withoutColumns('c)
+	 */
+	public static ApiExpression withoutColumns(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.WITHOUT_COLUMNS,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * A call to a function that will be looked up in a catalog.
 
 Review comment:
   Add examples and explain the resolution logic of system functions and catalog functions etc. It can be fully qualified here right?

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r385592020
 
 

 ##########
 File path: 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]].
 
 Review comment:
   Sure, I just want to warn people not to use this class anymore. 

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380065878
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,527 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.internal.BaseExpressions;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.ExpressionVisitor;
+import org.apache.flink.table.expressions.TimePointUnit;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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 java.util.Arrays;
+import java.util.List;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral;
+
+/**
+ * Entry point for a Java Table Expression DSL.
+ */
+@PublicEvolving
+public final class Expressions {
+
+	/**
+	 * Creates an unresolved reference to a table's column.
+	 */
+	//CHECKSTYLE.OFF: MethodName
+	public static ApiExpression $(String name) {
+		return new ApiExpression(ApiExpressionUtils.unresolvedRef(name));
+	}
+	//CHECKSTYLE.ON: MethodName
+
+	/**
+	 * Creates a SQL literal. The data type is derived from the Java class.
+	 */
+	public static ApiExpression lit(Object v) {
+		return new ApiExpression(valueLiteral(v));
+	}
+
+	/**
+	 * Creates a SQL literal of a given {@link DataType}.
+	 */
+	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.
+	 *
+	 * @see #withColumns(Object, Object...)
+	 * @see #withoutColumns(Object, Object...)
+	 */
+	public static ApiExpression range(String start, String end) {
+		return apiCall(BuiltInFunctionDefinitions.RANGE_TO, valueLiteral(start), valueLiteral(end));
+	}
+
+	/**
+	 * Indicates an index based range, which can be used in columns selection.
+	 *
+	 * @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 apiCall(BuiltInFunctionDefinitions.AND, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Boolean OR in three-valued logic.
+	 */
+	public static ApiExpression or(Object predicate0, Object predicate1, Object... predicates) {
+		return apiCall(BuiltInFunctionDefinitions.OR, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 <code>leftEnd >= rightStart && rightEnd >= leftStart</code>.
+	 *
+	 * <p>It evaluates: leftEnd >= rightStart && rightEnd >= leftStart
+	 *
+	 * <p>e.g. temporalOverlaps("2:55:00".toTime, 1.hour, "3:30:00".toTime, 2.hour) leads to true
+	 */
+	public static ApiExpression temporalOverlaps(
+			Object leftTimePoint,
+			Object leftTemporal,
+			Object rightTimePoint,
+			Object rightTemporal) {
+		return apiCall(
+			BuiltInFunctionDefinitions.TEMPORAL_OVERLAPS,
+			objectToExpression(leftTimePoint),
+			objectToExpression(leftTemporal),
+			objectToExpression(rightTimePoint),
+			objectToExpression(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.
+	 *
+	 * <p>For example 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,
+			objectToExpression(timestamp),
+			objectToExpression(format));
+	}
+
+	/**
+	 * Returns the (signed) number of {@link TimePointUnit} between timePoint1 and timePoint2.
+	 *
+	 * <p>For example, {@code timestampDiff(TimePointUnit.DAY, '2016-06-15'.toDate, '2016-06-18'.toDate} leads
 
 Review comment:
   and here?

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
dawidwys commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r385211381
 
 

 ##########
 File path: 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]].
 
 Review comment:
   How about we do it along with methods in `Table`\`TableEnvironment`? We still need to implement the expressionDsl for python to fully drop it.

----------------------------------------------------------------
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 issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
dawidwys commented on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-589577543
 
 
   In the end I had to rebase this on top of #11156 and #11170. The reason is I had to first solve looking up functions in a catalog before I could expose all the expressions because of test failures.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148755202 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   Hash:93f931c7ef96767509f62982e48b4853d3a74dfc Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/149801561 TriggerType:PUSH TriggerID:93f931c7ef96767509f62982e48b4853d3a74dfc
   Hash:93f931c7ef96767509f62982e48b4853d3a74dfc Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362 TriggerType:PUSH TriggerID:93f931c7ef96767509f62982e48b4853d3a74dfc
   Hash:b79c6583d86289244513a44c260c5c5d6b520f48 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363 TriggerType:PUSH TriggerID:b79c6583d86289244513a44c260c5c5d6b520f48
   Hash:b79c6583d86289244513a44c260c5c5d6b520f48 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/149806101 TriggerType:PUSH TriggerID:b79c6583d86289244513a44c260c5c5d6b520f48
   -->
   ## CI report:
   
   * 16d4e9e6fbeb230b8aa05976e4909299ebc929bf Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148755202) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128) 
   * 93f931c7ef96767509f62982e48b4853d3a74dfc Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/149801561) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362) 
   * b79c6583d86289244513a44c260c5c5d6b520f48 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/149806101) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363) 
   
   <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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148755202",
       "triggerID" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149801561",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149806101",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149869824",
       "triggerID" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149885015",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149970663",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149981047",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3c118088adedb8b818d0371495d313e9fe038968 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/149981047) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408) 
   * 0cd224f4da41acc6973bbd4fb27f11e201832e6e 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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r385023076
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/expressions/resolver/ExpressionResolverTest.java
 ##########
 @@ -0,0 +1,391 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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;
+import org.apache.flink.table.catalog.DataTypeFactory;
+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.operations.CatalogQueryOperation;
+import org.apache.flink.table.operations.QueryOperation;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.inference.TypeInferenceUtil;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeParser;
+
+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.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.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.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType;
+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:
+ *
+ * <p>- BuiltinFunctionDefinitions.EQUALS
+ * - BuiltinFunctionDefinitions.IS_NULL
+ *
+ * <p>Pseudo functions that are executed during expression resolution e.g.:
+ * - BuiltinFunctionDefinitions.WITH_COLUMNS
+ * - BuiltinFunctionDefinitions.WITHOUT_COLUMNS
+ * - BuiltinFunctionDefinitions.RANGE_TO
+ * - BuiltinFunctionDefinitions.FLATTEN
+ *
+ * <p>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<TestSpec> 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 calls")
+				.inputSchemas(
+					TableSchema.builder()
+						.field("f0", DataTypes.INT())
+						.build()
+				)
+				.lookupFunction("func", new ScalarFunc())
+				.select(call("func", 1, $("f0")))
+				.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("Catalog calls")
+				.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 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
+	public TestSpec testSpec;
+
+	@Test
+	public void testResolvingExpressions() {
+		List<ResolvedExpression> resolvedExpressions = testSpec.getResolver()
+			.resolve(Arrays.asList(testSpec.expressions));
+		assertThat(
+			resolvedExpressions,
+			equalTo(testSpec.expectedExpressions));
+	}
+
+	/**
+	 * Test scalar function.
+	 */
+	@FunctionHint(input = @DataTypeHint(inputGroup = InputGroup.ANY), isVarArgs = true, output = @DataTypeHint(value = "INTEGER NOT NULL", bridgedTo = int.class))
 
 Review comment:
   nit: line wrap per hint argument

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380181541
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/rules/UnwrapApiExpressionRule.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.expressions.resolver.rules;
+
+import org.apache.flink.table.api.Expressions;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Unwraps all {@link Expressions.ApiExpression}.
+ */
+final class UnwrapApiExpressionRule implements ResolverRule {
 
 Review comment:
   I thought we mark all classes and interfaces.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380078629
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java
 ##########
 @@ -0,0 +1,1146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.Table;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.TimeIntervalUnit;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.tableRef;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.typeLiteral;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.unresolvedCall;
+import static org.apache.flink.table.expressions.utils.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.
+ */
+@PublicEvolving
+public abstract class BaseExpressions<InT, OutT> {
+	protected abstract Expression toExpr();
+
+	protected abstract OutT 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
+	 * @return field with an alias
+	 */
+	public OutT 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.
+	 */
+	public OutT and(InT other) {
+		return toApiSpecificExpression(unresolvedCall(AND, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Boolean OR in three-valued logic.
+	 */
+	public OutT or(InT other) {
+		return toApiSpecificExpression(unresolvedCall(OR, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Greater than.
+	 */
+	public OutT isGreater(InT other) {
+		return toApiSpecificExpression(unresolvedCall(GREATER_THAN, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Greater than or equal.
+	 */
+	public OutT isGreaterOrEqual(InT other) {
+		return toApiSpecificExpression(unresolvedCall(GREATER_THAN_OR_EQUAL, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Less than.
+	 */
+	public OutT isLess(InT other) {
+		return toApiSpecificExpression(unresolvedCall(LESS_THAN, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Less than or equal.
+	 */
+	public OutT isLessOrEqual(InT other) {
+		return toApiSpecificExpression(unresolvedCall(LESS_THAN_OR_EQUAL, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Equals.
+	 */
+	public OutT isEqual(InT other) {
+		return toApiSpecificExpression(unresolvedCall(EQUALS, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Not equal.
+	 */
+	public OutT isNotEqual(InT other) {
+		return toApiSpecificExpression(unresolvedCall(NOT_EQUALS, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Returns left plus right.
+	 */
+	public OutT plus(InT other) {
+		return toApiSpecificExpression(unresolvedCall(PLUS, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Returns left minus right.
+	 */
+	public OutT minus(InT other) {
+		return toApiSpecificExpression(unresolvedCall(MINUS, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Returns left divided by right.
+	 */
+	public OutT dividedBy(InT other) {
+		return toApiSpecificExpression(unresolvedCall(DIVIDE, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Returns left multiplied by right.
+	 */
+	public OutT multipliedBy(InT 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
+	 * @return boolean or null
+	 */
+	public OutT between(InT lowerBound, InT 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
+	 * @return boolean or null
+	 */
+	public OutT notBetween(InT lowerBound, InT upperBound) {
+		return toApiSpecificExpression(unresolvedCall(
+			NOT_BETWEEN,
+			toExpr(),
+			objectToExpression(lowerBound),
+			objectToExpression(upperBound)));
+	}
+
+	/**
+	 * Returns true if the given expression is null.
+	 */
+	public OutT isNull() {
+		return toApiSpecificExpression(unresolvedCall(IS_NULL, toExpr()));
+	}
+
+	/**
+	 * Returns true if the given expression is not null.
+	 */
+	public OutT isNotNull() {
+		return toApiSpecificExpression(unresolvedCall(IS_NOT_NULL, toExpr()));
+	}
+
+	/**
+	 * Returns true if given boolean expression is true. False otherwise (for null and false).
+	 */
+	public OutT isTrue() {
+		return toApiSpecificExpression(unresolvedCall(IS_TRUE, toExpr()));
+	}
+
+	/**
+	 * Returns true if given boolean expression is false. False otherwise (for null and true).
+	 */
+	public OutT isFalse() {
+		return toApiSpecificExpression(unresolvedCall(IS_FALSE, toExpr()));
+	}
+
+	/**
+	 * Returns true if given boolean expression is not true (for null and false). False otherwise.
+	 */
+	public OutT isNotTrue() {
+		return toApiSpecificExpression(unresolvedCall(IS_NOT_TRUE, toExpr()));
+	}
+
+	/**
+	 * Returns true if given boolean expression is not false (for null and true). False otherwise.
+	 */
+	public OutT 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.
+	 *
+	 * <p>For example:
+	 * <pre>
+	 * {@code
+	 * orders
+	 * .groupBy('a)
+	 * .select('a, 'b.sum.distinct as 'd)
+	 * }
+	 * </pre>
+	 */
+	public OutT 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 OutT 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 OutT sum0() {
+		return toApiSpecificExpression(unresolvedCall(SUM0, toExpr()));
+	}
+
+	/**
+	 * Returns the minimum value of field across all input values.
+	 */
+	public OutT min() {
+		return toApiSpecificExpression(unresolvedCall(MIN, toExpr()));
+	}
+
+	/**
+	 * Returns the maximum value of field across all input values.
+	 */
+	public OutT max() {
+		return toApiSpecificExpression(unresolvedCall(MAX, toExpr()));
+	}
+
+	/**
+	 * Returns the number of input rows for which the field is not null.
+	 */
+	public OutT count() {
+		return toApiSpecificExpression(unresolvedCall(COUNT, toExpr()));
+	}
+
+	/**
+	 * Returns the average (arithmetic mean) of the numeric field across all input values.
+	 */
+	public OutT avg() {
+		return toApiSpecificExpression(unresolvedCall(AVG, toExpr()));
+	}
+
+	/**
+	 * Returns the population standard deviation of an expression (the square root of varPop()).
+	 */
+	public OutT stddevPop() {
+		return toApiSpecificExpression(unresolvedCall(STDDEV_POP, toExpr()));
+	}
+
+	/**
+	 * Returns the sample standard deviation of an expression (the square root of varSamp()).
+	 */
+	public OutT stddevSamp() {
+		return toApiSpecificExpression(unresolvedCall(STDDEV_SAMP, toExpr()));
+	}
+
+	/**
+	 * Returns the population standard variance of an expression.
+	 */
+	public OutT varPop() {
+		return toApiSpecificExpression(unresolvedCall(VAR_POP, toExpr()));
+	}
+
+	/**
+	 * Returns the sample variance of a given expression.
+	 */
+	public OutT varSamp() {
+		return toApiSpecificExpression(unresolvedCall(VAR_SAMP, toExpr()));
+	}
+
+	/**
+	 * Returns multiset aggregate of a given expression.
+	 */
+	public OutT collect() {
+		return toApiSpecificExpression(unresolvedCall(COLLECT, toExpr()));
+	}
+
+	/**
+	 * Converts a value to a given data type.
+	 *
+	 * <p>e.g. "42".cast(DataTypes.INT()) leads to 42.
+	 *
+	 * @return casted expression
+	 */
+	public OutT 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 OutT cast(TypeInformation<?> toType) {
+		return toApiSpecificExpression(unresolvedCall(CAST, toExpr(), typeLiteral(fromLegacyInfoToDataType(toType))));
+	}
+
+	/**
+	 * Specifies ascending order of an expression i.e. a field for orderBy unresolvedCall.
+	 *
+	 * @return ascend expression
+	 */
+	public OutT asc() {
+		return toApiSpecificExpression(unresolvedCall(ORDER_ASC, toExpr()));
+	}
+
+	/**
+	 * Specifies descending order of an expression i.e. a field for orderBy unresolvedCall.
+	 *
+	 * @return descend expression
+	 */
+	public OutT 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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>e.g. "42".in(1, 2, 3) leads to false.
+	 */
+	@SafeVarargs
+	public final OutT in(InT... 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.
+	 *
+	 * <p>Note: This operation is not supported in a streaming environment yet.
+	 */
+	public OutT 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 OutT start() {
+		return toApiSpecificExpression(unresolvedCall(WINDOW_START, toExpr()));
+	}
+
+	/**
+	 * Returns the end time (exclusive) of a window when applied on a window reference.
+	 *
+	 * <p>e.g. if a window ends at 10:59:59.999 this property will return 11:00:00.000.
+	 */
+	public OutT end() {
+		return toApiSpecificExpression(unresolvedCall(WINDOW_END, toExpr()));
+	}
+
+	// scalar functions
+
+	/**
+	 * Calculates the remainder of division the given number by another one.
+	 */
+	public OutT mod(InT other) {
+		return toApiSpecificExpression(unresolvedCall(MOD, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Calculates the Euler's number raised to the given power.
+	 */
+	public OutT exp() {
+		return toApiSpecificExpression(unresolvedCall(EXP, toExpr()));
+	}
+
+	/**
+	 * Calculates the base 10 logarithm of the given value.
+	 */
+	public OutT log10() {
+		return toApiSpecificExpression(unresolvedCall(LOG10, toExpr()));
+	}
+
+	/**
+	 * Calculates the base 2 logarithm of the given value.
+	 */
+	public OutT log2() {
+		return toApiSpecificExpression(unresolvedCall(LOG2, toExpr()));
+	}
+
+	/**
+	 * Calculates the natural logarithm of the given value.
+	 */
+	public OutT ln() {
+		return toApiSpecificExpression(unresolvedCall(LN, toExpr()));
+	}
+
+	/**
+	 * Calculates the natural logarithm of the given value.
+	 */
+	public OutT log() {
+		return toApiSpecificExpression(unresolvedCall(LOG, toExpr()));
+	}
+
+	/**
+	 * Calculates the logarithm of the given value to the given base.
+	 */
+	public OutT log(InT base) {
+		return toApiSpecificExpression(unresolvedCall(LOG, objectToExpression(base), toExpr()));
+	}
+
+	/**
+	 * Calculates the given number raised to the power of the other value.
+	 */
+	public OutT power(InT other) {
+		return toApiSpecificExpression(unresolvedCall(POWER, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Calculates the hyperbolic cosine of a given value.
+	 */
+	public OutT cosh() {
+		return toApiSpecificExpression(unresolvedCall(COSH, toExpr()));
+	}
+
+	/**
+	 * Calculates the square root of a given value.
+	 */
+	public OutT sqrt() {
+		return toApiSpecificExpression(unresolvedCall(SQRT, toExpr()));
+	}
+
+	/**
+	 * Calculates the absolute value of given value.
+	 */
+	public OutT abs() {
+		return toApiSpecificExpression(unresolvedCall(ABS, toExpr()));
+	}
+
+	/**
+	 * Calculates the largest integer less than or equal to a given number.
+	 */
+	public OutT floor() {
+		return toApiSpecificExpression(unresolvedCall(FLOOR, toExpr()));
+	}
+
+	/**
+	 * Calculates the hyperbolic sine of a given value.
+	 */
+	public OutT sinh() {
+		return toApiSpecificExpression(unresolvedCall(SINH, toExpr()));
+	}
+
+	/**
+	 * Calculates the smallest integer greater than or equal to a given number.
+	 */
+	public OutT ceil() {
+		return toApiSpecificExpression(unresolvedCall(CEIL, toExpr()));
+	}
+
+	/**
+	 * Calculates the sine of a given number.
+	 */
+	public OutT sin() {
+		return toApiSpecificExpression(unresolvedCall(SIN, toExpr()));
+	}
+
+	/**
+	 * Calculates the cosine of a given number.
+	 */
+	public OutT cos() {
+		return toApiSpecificExpression(unresolvedCall(COS, toExpr()));
+	}
+
+	/**
+	 * Calculates the tangent of a given number.
+	 */
+	public OutT tan() {
+		return toApiSpecificExpression(unresolvedCall(TAN, toExpr()));
+	}
+
+	/**
+	 * Calculates the cotangent of a given number.
+	 */
+	public OutT cot() {
+		return toApiSpecificExpression(unresolvedCall(COT, toExpr()));
+	}
+
+	/**
+	 * Calculates the arc sine of a given number.
+	 */
+	public OutT asin() {
+		return toApiSpecificExpression(unresolvedCall(ASIN, toExpr()));
+	}
+
+	/**
+	 * Calculates the arc cosine of a given number.
+	 */
+	public OutT acos() {
+		return toApiSpecificExpression(unresolvedCall(ACOS, toExpr()));
+	}
+
+	/**
+	 * Calculates the arc tangent of a given number.
+	 */
+	public OutT atan() {
+		return toApiSpecificExpression(unresolvedCall(ATAN, toExpr()));
+	}
+
+	/**
+	 * Calculates the hyperbolic tangent of a given number.
+	 */
+	public OutT tanh() {
+		return toApiSpecificExpression(unresolvedCall(TANH, toExpr()));
+	}
+
+	/**
+	 * Converts numeric from radians to degrees.
+	 */
+	public OutT degrees() {
+		return toApiSpecificExpression(unresolvedCall(DEGREES, toExpr()));
+	}
+
+	/**
+	 * Converts numeric from degrees to radians.
+	 */
+	public OutT radians() {
+		return toApiSpecificExpression(unresolvedCall(RADIANS, toExpr()));
+	}
+
+	/**
+	 * Calculates the signum of a given number.
+	 */
+	public OutT sign() {
+		return toApiSpecificExpression(unresolvedCall(SIGN, toExpr()));
+	}
+
+	/**
+	 * Rounds the given number to integer places right to the decimal point.
+	 */
+	public OutT round(InT 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 OutT 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.
+	 *
+	 * <p>E.g. a numeric 20 leads to "14", a numeric 100 leads to "64", and a string "hello,world" leads
+	 * to "68656c6c6f2c776f726c64".
+	 */
+	public OutT 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 OutT truncate(InT 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 OutT 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
+	 * @return substring
 
 Review comment:
   nit: remove the `@return` everywhere?

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r384986198
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,562 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.catalog.DataTypeFactory;
+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.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;
+
+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()}
+ *
+ * <p>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.
+ *
+ * <p>For fluent definition of expressions and easier readability, we recommend to add a
+ * star import to the methods of this class:
+ *
+ * <pre>
+ * import static org.apache.flink.table.api.Expressions.*;
+ * </pre>
+ *
+ * <p>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.
+	 *
+	 * <p>Example
+	 * <pre>{@code
+	 *   tab.select($("key"), $("value"))
+	 * }
+	 * </pre>
+	 */
+	//CHECKSTYLE.OFF: MethodName
+	public static ApiExpression $(String name) {
+		return new ApiExpression(unresolvedRef(name));
+	}
+	//CHECKSTYLE.ON: MethodName
+
+	/**
+	 * Creates a SQL literal.
+	 *
+	 * <p>The data type is derived from the object's class and its value.
+	 *
+	 * <p>For example:
+	 * <ul>
+	 *     <li>{@code lit(12)} leads to {@code INT}</li>
+	 *     <li>{@code lit("abc")} leads to {@code CHAR(3)}</li>
+	 *     <li>{@code lit(new BigDecimal("123.45"))} leads to {@code DECIMAL(5, 2)}</li>
+	 * </ul>
+	 *
+	 * <p>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}.
+	 *
+	 * <p>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.
+	 *
+	 * <p>Example:
+	 * <pre>{@code
+	 * Table table = ...
+	 * table.withColumns(range(b, c))
 
 Review comment:
   Should be within a `select()`.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380180450
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,527 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.internal.BaseExpressions;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.ExpressionVisitor;
+import org.apache.flink.table.expressions.TimePointUnit;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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 java.util.Arrays;
+import java.util.List;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral;
+
+/**
+ * Entry point for a Java Table Expression DSL.
+ */
+@PublicEvolving
+public final class Expressions {
+
+	/**
+	 * Creates an unresolved reference to a table's column.
+	 */
+	//CHECKSTYLE.OFF: MethodName
+	public static ApiExpression $(String name) {
+		return new ApiExpression(ApiExpressionUtils.unresolvedRef(name));
+	}
+	//CHECKSTYLE.ON: MethodName
+
+	/**
+	 * Creates a SQL literal. The data type is derived from the Java class.
+	 */
+	public static ApiExpression lit(Object v) {
+		return new ApiExpression(valueLiteral(v));
+	}
+
+	/**
+	 * Creates a SQL literal of a given {@link DataType}.
+	 */
+	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.
+	 *
+	 * @see #withColumns(Object, Object...)
+	 * @see #withoutColumns(Object, Object...)
+	 */
+	public static ApiExpression range(String start, String end) {
+		return apiCall(BuiltInFunctionDefinitions.RANGE_TO, valueLiteral(start), valueLiteral(end));
+	}
+
+	/**
+	 * Indicates an index based range, which can be used in columns selection.
+	 *
+	 * @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 apiCall(BuiltInFunctionDefinitions.AND, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Boolean OR in three-valued logic.
+	 */
+	public static ApiExpression or(Object predicate0, Object predicate1, Object... predicates) {
+		return apiCall(BuiltInFunctionDefinitions.OR, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 <code>leftEnd >= rightStart && rightEnd >= leftStart</code>.
+	 *
+	 * <p>It evaluates: leftEnd >= rightStart && rightEnd >= leftStart
+	 *
+	 * <p>e.g. temporalOverlaps("2:55:00".toTime, 1.hour, "3:30:00".toTime, 2.hour) leads to true
+	 */
+	public static ApiExpression temporalOverlaps(
+			Object leftTimePoint,
+			Object leftTemporal,
+			Object rightTimePoint,
+			Object rightTemporal) {
+		return apiCall(
+			BuiltInFunctionDefinitions.TEMPORAL_OVERLAPS,
+			objectToExpression(leftTimePoint),
+			objectToExpression(leftTemporal),
+			objectToExpression(rightTimePoint),
+			objectToExpression(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.
+	 *
+	 * <p>For example 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,
+			objectToExpression(timestamp),
+			objectToExpression(format));
+	}
+
+	/**
+	 * Returns the (signed) number of {@link TimePointUnit} between timePoint1 and timePoint2.
+	 *
+	 * <p>For example, {@code timestampDiff(TimePointUnit.DAY, '2016-06-15'.toDate, '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),
+			objectToExpression(timePoint1),
+			objectToExpression(timePoint2));
+	}
+
+	/**
+	 * Creates an array of literals.
+	 */
+	public static ApiExpression array(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.ARRAY,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a row of expressions.
+	 */
+	public static ApiExpression row(Object head, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.ROW, Stream.concat(
+			Stream.of(head),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a map of expressions.
+	 */
+	public static ApiExpression map(Object key, Object value, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.MAP, Stream.concat(
+			Stream.of(key, value),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 apiCall(
+			BuiltInFunctionDefinitions.CONCAT,
+			Stream.concat(
+				Stream.of(string),
+				Stream.of(strings)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Calculates the arc tangent of a given coordinate.
+	 */
+	public static ApiExpression atan2(Object y, Object x) {
+		return apiCall(BuiltInFunctionDefinitions.ATAN2, objectToExpression(y), objectToExpression(x));
+	}
+
+	/**
+	 * Returns the string that results from concatenating the arguments and separator.
+	 * Returns NULL If the separator is NULL.
+	 *
+	 * <p>Note: this user-public static ApiExpressionined 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 apiCall(BuiltInFunctionDefinitions.CONCAT_WS, Stream.concat(
+			Stream.of(separator, string),
+			Stream.of(strings)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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.
+	 *
+	 * <p>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, objectToExpression(value));
+	}
+
+	/**
+	 * Calculates the logarithm of the given value to the given base.
+	 */
+	public static ApiExpression log(Object base, Object value) {
+		return apiCall(BuiltInFunctionDefinitions.LOG, objectToExpression(base), objectToExpression(value));
+	}
+
+	/**
+	 * Ternary conditional operator that decides which of two other expressions should be evaluated
+	 * based on a evaluated boolean condition.
+	 *
+	 * <p>e.g. ifThenElse(42 > 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,
+			objectToExpression(condition),
+			objectToExpression(ifTrue),
+			objectToExpression(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.
+	 *
+	 * <p>A range can either be index-based or name-based. Indices start at 1 and boundaries are
+	 * inclusive.
+	 *
+	 * <p>e.g. withColumns('b to 'c) or withColumns('*)
+	 */
+	public static ApiExpression withColumns(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.WITH_COLUMNS,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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.
+	 *
+	 * <p>A range can either be index-based or name-based. Indices start at 1 and boundaries are
+	 * inclusive.
+	 *
+	 * <p>e.g. withoutColumns('b to 'c) or withoutColumns('c)
+	 */
+	public static ApiExpression withoutColumns(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.WITHOUT_COLUMNS,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * A call to a function that will be looked up in a catalog.
+	 */
+	public static ApiExpression call(String functionName, Object... params) {
+		return new ApiExpression(ApiExpressionUtils.lookupCall(
+			functionName,
+			Arrays.stream(params).map(ApiExpressionUtils::objectToExpression).toArray(Expression[]::new)));
+	}
+
+	/**
+	 * A call to an inline function. For functions registered in a catalog use {@link #call(String, Object...)}.
+	 */
+	public static ApiExpression call(FunctionDefinition scalarFunction, Object... params) {
+		return apiCall(
+			scalarFunction,
+			Arrays.stream(params).map(ApiExpressionUtils::objectToExpression).toArray(Expression[]::new));
+	}
+
+	private static ApiExpression apiCall(FunctionDefinition functionDefinition, Expression... args) {
+		return new ApiExpression(new UnresolvedCallExpression(functionDefinition, Arrays.asList(args)));
+	}
+
+	/**
+	 * Java API class that gives access to expressions operations.
+	 */
+	public static final class ApiExpression extends BaseExpressions<Object, ApiExpression> implements Expression {
 
 Review comment:
   Sure, ApiExpression must be public. I was just suggesting to have a hidden constructor and a top-level class.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380076418
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java
 ##########
 @@ -0,0 +1,1146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.Table;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.TimeIntervalUnit;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.tableRef;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.typeLiteral;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.unresolvedCall;
+import static org.apache.flink.table.expressions.utils.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.
+ */
+@PublicEvolving
+public abstract class BaseExpressions<InT, OutT> {
+	protected abstract Expression toExpr();
+
+	protected abstract OutT 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
+	 * @return field with an alias
+	 */
+	public OutT 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.
+	 */
+	public OutT and(InT other) {
 
 Review comment:
   mention that this is infix notation and there also exists a prefix notation.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148755202",
       "triggerID" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149801561",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149806101",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149869824",
       "triggerID" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149885015",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149970663",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149981047",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150432772",
       "triggerID" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150669044",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5643",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d510ce90e5cd8a13256c84c855562c403091a14",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150861986",
       "triggerID" : "8d510ce90e5cd8a13256c84c855562c403091a14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3786dab008166eff376617dabda4dedeba7ca961",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3786dab008166eff376617dabda4dedeba7ca961",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0e3ee0c3b8b58815b2f771d8b4320e7483c41159",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0e3ee0c3b8b58815b2f771d8b4320e7483c41159",
       "triggerType" : "PUSH"
     }, {
       "hash" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8d510ce90e5cd8a13256c84c855562c403091a14 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/150861986) 
   * 3786dab008166eff376617dabda4dedeba7ca961 UNKNOWN
   * 0e3ee0c3b8b58815b2f771d8b4320e7483c41159 UNKNOWN
   * 156ddaa5298b8059b4038c486e00770a263c25f7 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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380081874
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/LookupCallResolver.java
 ##########
 @@ -63,6 +64,17 @@ public Expression visit(UnresolvedCallExpression unresolvedCall) {
 			.collect(Collectors.toList());
 	}
 
+	@Override
+	public Expression visitNonApiExpression(Expression other) {
+		// LookupCallResolver might be called outside of ExpressionResolver, thus we need to additionally
 
 Review comment:
   Can't we merge `LookupCallResolver` and `ExpressionResolver` into one entity? Just an idea...

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380080705
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/LookupCallExpression.java
 ##########
 @@ -43,7 +42,10 @@
 
 	public LookupCallExpression(String unresolvedFunction, List<Expression> args) {
 		this.unresolvedName = Preconditions.checkNotNull(unresolvedFunction);
-		this.args = Collections.unmodifiableList(new ArrayList<>(Preconditions.checkNotNull(args)));
+		this.args = Preconditions.checkNotNull(args)
 
 Review comment:
   Can't we do this earlier? In `ApiExpressionUtils`?

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380089754
 
 

 ##########
 File path: flink-table/flink-table-api-scala/src/main/scala/org/apache/flink/table/api/expressionDsl.scala
 ##########
 @@ -1527,6 +874,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 = {
 
 Review comment:
   How about other infix notation expressions for the Scala API? `lit()`, `call()` should also be present here. We should synchronize with `Expressions`? And add a comment that contributors should synchronize in the future.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
dawidwys commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380168356
 
 

 ##########
 File path: flink-table/flink-table-api-scala/src/main/scala/org/apache/flink/table/api/expressionDsl.scala
 ##########
 @@ -355,190 +155,10 @@ trait ImplicitExpressionOperations {
     * @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
 
 Review comment:
   Any suggestions for a name?

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380179639
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,527 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.internal.BaseExpressions;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.ExpressionVisitor;
+import org.apache.flink.table.expressions.TimePointUnit;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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 java.util.Arrays;
+import java.util.List;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral;
+
+/**
+ * Entry point for a Java Table Expression DSL.
+ */
+@PublicEvolving
+public final class Expressions {
+
+	/**
+	 * Creates an unresolved reference to a table's column.
+	 */
+	//CHECKSTYLE.OFF: MethodName
+	public static ApiExpression $(String name) {
+		return new ApiExpression(ApiExpressionUtils.unresolvedRef(name));
+	}
+	//CHECKSTYLE.ON: MethodName
+
+	/**
+	 * Creates a SQL literal. The data type is derived from the Java class.
+	 */
+	public static ApiExpression lit(Object v) {
+		return new ApiExpression(valueLiteral(v));
+	}
+
+	/**
+	 * Creates a SQL literal of a given {@link DataType}.
+	 */
+	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.
+	 *
+	 * @see #withColumns(Object, Object...)
+	 * @see #withoutColumns(Object, Object...)
+	 */
+	public static ApiExpression range(String start, String end) {
+		return apiCall(BuiltInFunctionDefinitions.RANGE_TO, valueLiteral(start), valueLiteral(end));
+	}
+
+	/**
+	 * Indicates an index based range, which can be used in columns selection.
+	 *
+	 * @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 apiCall(BuiltInFunctionDefinitions.AND, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Boolean OR in three-valued logic.
+	 */
+	public static ApiExpression or(Object predicate0, Object predicate1, Object... predicates) {
+		return apiCall(BuiltInFunctionDefinitions.OR, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 <code>leftEnd >= rightStart && rightEnd >= leftStart</code>.
+	 *
+	 * <p>It evaluates: leftEnd >= rightStart && rightEnd >= leftStart
+	 *
+	 * <p>e.g. temporalOverlaps("2:55:00".toTime, 1.hour, "3:30:00".toTime, 2.hour) leads to true
+	 */
+	public static ApiExpression temporalOverlaps(
+			Object leftTimePoint,
+			Object leftTemporal,
+			Object rightTimePoint,
+			Object rightTemporal) {
+		return apiCall(
+			BuiltInFunctionDefinitions.TEMPORAL_OVERLAPS,
+			objectToExpression(leftTimePoint),
+			objectToExpression(leftTemporal),
+			objectToExpression(rightTimePoint),
+			objectToExpression(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.
+	 *
+	 * <p>For example 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,
+			objectToExpression(timestamp),
+			objectToExpression(format));
+	}
+
+	/**
+	 * Returns the (signed) number of {@link TimePointUnit} between timePoint1 and timePoint2.
+	 *
+	 * <p>For example, {@code timestampDiff(TimePointUnit.DAY, '2016-06-15'.toDate, '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),
+			objectToExpression(timePoint1),
+			objectToExpression(timePoint2));
+	}
+
+	/**
+	 * Creates an array of literals.
+	 */
+	public static ApiExpression array(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.ARRAY,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a row of expressions.
+	 */
+	public static ApiExpression row(Object head, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.ROW, Stream.concat(
+			Stream.of(head),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a map of expressions.
+	 */
+	public static ApiExpression map(Object key, Object value, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.MAP, Stream.concat(
+			Stream.of(key, value),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 apiCall(
+			BuiltInFunctionDefinitions.CONCAT,
+			Stream.concat(
+				Stream.of(string),
+				Stream.of(strings)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Calculates the arc tangent of a given coordinate.
+	 */
+	public static ApiExpression atan2(Object y, Object x) {
+		return apiCall(BuiltInFunctionDefinitions.ATAN2, objectToExpression(y), objectToExpression(x));
+	}
+
+	/**
+	 * Returns the string that results from concatenating the arguments and separator.
+	 * Returns NULL If the separator is NULL.
+	 *
+	 * <p>Note: this user-public static ApiExpressionined 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 apiCall(BuiltInFunctionDefinitions.CONCAT_WS, Stream.concat(
+			Stream.of(separator, string),
+			Stream.of(strings)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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.
+	 *
+	 * <p>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, objectToExpression(value));
+	}
+
+	/**
+	 * Calculates the logarithm of the given value to the given base.
+	 */
+	public static ApiExpression log(Object base, Object value) {
+		return apiCall(BuiltInFunctionDefinitions.LOG, objectToExpression(base), objectToExpression(value));
+	}
+
+	/**
+	 * Ternary conditional operator that decides which of two other expressions should be evaluated
+	 * based on a evaluated boolean condition.
+	 *
+	 * <p>e.g. ifThenElse(42 > 5, "A", "B") leads to "A"
 
 Review comment:
   True, the example is misleading. It should use a field instead.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/148755202 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   -->
   ## CI report:
   
   * 16d4e9e6fbeb230b8aa05976e4909299ebc929bf Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/148755202) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128) 
   
   <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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r385597494
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/utils/factories/CalculatedTableFactory.java
 ##########
 @@ -95,42 +94,87 @@ public FunctionTableCallVisitor(String[] leftTableFieldNames) {
 			}
 
 			CallExpression tableCall = (CallExpression) children.get(0);
-			TableFunctionDefinition tableFunctionDefinition =
-				(TableFunctionDefinition) tableCall.getFunctionDefinition();
-			return createFunctionCall(tableFunctionDefinition, aliases, tableCall.getResolvedChildren());
+			return createFunctionCall(tableCall, aliases);
+		}
+
+		private CalculatedQueryOperation<?> createFunctionCall(CallExpression call, List<String> aliases) {
+			FunctionDefinition definition = call.getFunctionDefinition();
+			if (definition instanceof TableFunctionDefinition) {
+				return createFunctionCall(
+					((TableFunctionDefinition) definition).getTableFunction(),
+					call.getFunctionIdentifier().orElse(FunctionIdentifier.of(definition.toString())),
+					call.getOutputDataType(),
+					aliases,
+					call.getResolvedChildren());
+			} else if (definition instanceof TableFunction<?>) {
+				return createFunctionCall(
+					(TableFunction<?>) definition,
+					call.getFunctionIdentifier().orElse(FunctionIdentifier.of(definition.toString())),
+					call.getOutputDataType(),
+					aliases,
+					call.getResolvedChildren());
+			} else {
+				return defaultMethod(call);
+			}
 		}
 
 		private CalculatedQueryOperation<?> createFunctionCall(
-				TableFunctionDefinition tableFunctionDefinition,
+				TableFunction<?> tableFunction,
+				FunctionIdentifier identifier,
+				DataType resultType,
 				List<String> aliases,
 				List<ResolvedExpression> parameters) {
-			TypeInformation<?> resultType = tableFunctionDefinition.getResultType();
 
-			int callArity = resultType.getTotalFields();
-			int aliasesSize = aliases.size();
+			final TableSchema tableSchema = adjustNames(
+				extractSchema(resultType),
+				aliases,
+				identifier);
 
-			String[] fieldNames;
+			return new CalculatedQueryOperation(
+				tableFunction,
+				parameters,
+				TypeConversions.fromDataTypeToLegacyInfo(resultType),
 
 Review comment:
   Can we store the DataType instead to not loose information here?

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380073034
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,527 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.internal.BaseExpressions;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.ExpressionVisitor;
+import org.apache.flink.table.expressions.TimePointUnit;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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 java.util.Arrays;
+import java.util.List;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral;
+
+/**
+ * Entry point for a Java Table Expression DSL.
+ */
+@PublicEvolving
+public final class Expressions {
+
+	/**
+	 * Creates an unresolved reference to a table's column.
+	 */
+	//CHECKSTYLE.OFF: MethodName
+	public static ApiExpression $(String name) {
+		return new ApiExpression(ApiExpressionUtils.unresolvedRef(name));
+	}
+	//CHECKSTYLE.ON: MethodName
+
+	/**
+	 * Creates a SQL literal. The data type is derived from the Java class.
+	 */
+	public static ApiExpression lit(Object v) {
+		return new ApiExpression(valueLiteral(v));
+	}
+
+	/**
+	 * Creates a SQL literal of a given {@link DataType}.
+	 */
+	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.
+	 *
+	 * @see #withColumns(Object, Object...)
+	 * @see #withoutColumns(Object, Object...)
+	 */
+	public static ApiExpression range(String start, String end) {
+		return apiCall(BuiltInFunctionDefinitions.RANGE_TO, valueLiteral(start), valueLiteral(end));
+	}
+
+	/**
+	 * Indicates an index based range, which can be used in columns selection.
+	 *
+	 * @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 apiCall(BuiltInFunctionDefinitions.AND, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Boolean OR in three-valued logic.
+	 */
+	public static ApiExpression or(Object predicate0, Object predicate1, Object... predicates) {
+		return apiCall(BuiltInFunctionDefinitions.OR, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 <code>leftEnd >= rightStart && rightEnd >= leftStart</code>.
+	 *
+	 * <p>It evaluates: leftEnd >= rightStart && rightEnd >= leftStart
+	 *
+	 * <p>e.g. temporalOverlaps("2:55:00".toTime, 1.hour, "3:30:00".toTime, 2.hour) leads to true
+	 */
+	public static ApiExpression temporalOverlaps(
+			Object leftTimePoint,
+			Object leftTemporal,
+			Object rightTimePoint,
+			Object rightTemporal) {
+		return apiCall(
+			BuiltInFunctionDefinitions.TEMPORAL_OVERLAPS,
+			objectToExpression(leftTimePoint),
+			objectToExpression(leftTemporal),
+			objectToExpression(rightTimePoint),
+			objectToExpression(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.
+	 *
+	 * <p>For example 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,
+			objectToExpression(timestamp),
+			objectToExpression(format));
+	}
+
+	/**
+	 * Returns the (signed) number of {@link TimePointUnit} between timePoint1 and timePoint2.
+	 *
+	 * <p>For example, {@code timestampDiff(TimePointUnit.DAY, '2016-06-15'.toDate, '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),
+			objectToExpression(timePoint1),
+			objectToExpression(timePoint2));
+	}
+
+	/**
+	 * Creates an array of literals.
+	 */
+	public static ApiExpression array(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.ARRAY,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a row of expressions.
+	 */
+	public static ApiExpression row(Object head, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.ROW, Stream.concat(
+			Stream.of(head),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a map of expressions.
+	 */
+	public static ApiExpression map(Object key, Object value, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.MAP, Stream.concat(
+			Stream.of(key, value),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 apiCall(
+			BuiltInFunctionDefinitions.CONCAT,
+			Stream.concat(
+				Stream.of(string),
+				Stream.of(strings)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Calculates the arc tangent of a given coordinate.
+	 */
+	public static ApiExpression atan2(Object y, Object x) {
+		return apiCall(BuiltInFunctionDefinitions.ATAN2, objectToExpression(y), objectToExpression(x));
+	}
+
+	/**
+	 * Returns the string that results from concatenating the arguments and separator.
+	 * Returns NULL If the separator is NULL.
+	 *
+	 * <p>Note: this user-public static ApiExpressionined 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 apiCall(BuiltInFunctionDefinitions.CONCAT_WS, Stream.concat(
+			Stream.of(separator, string),
+			Stream.of(strings)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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.
+	 *
+	 * <p>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, objectToExpression(value));
+	}
+
+	/**
+	 * Calculates the logarithm of the given value to the given base.
+	 */
+	public static ApiExpression log(Object base, Object value) {
+		return apiCall(BuiltInFunctionDefinitions.LOG, objectToExpression(base), objectToExpression(value));
+	}
+
+	/**
+	 * Ternary conditional operator that decides which of two other expressions should be evaluated
+	 * based on a evaluated boolean condition.
+	 *
+	 * <p>e.g. ifThenElse(42 > 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,
+			objectToExpression(condition),
+			objectToExpression(ifTrue),
+			objectToExpression(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.
+	 *
+	 * <p>A range can either be index-based or name-based. Indices start at 1 and boundaries are
+	 * inclusive.
+	 *
+	 * <p>e.g. withColumns('b to 'c) or withColumns('*)
+	 */
+	public static ApiExpression withColumns(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.WITH_COLUMNS,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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.
+	 *
+	 * <p>A range can either be index-based or name-based. Indices start at 1 and boundaries are
+	 * inclusive.
+	 *
+	 * <p>e.g. withoutColumns('b to 'c) or withoutColumns('c)
+	 */
+	public static ApiExpression withoutColumns(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.WITHOUT_COLUMNS,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * A call to a function that will be looked up in a catalog.
+	 */
+	public static ApiExpression call(String functionName, Object... params) {
+		return new ApiExpression(ApiExpressionUtils.lookupCall(
+			functionName,
+			Arrays.stream(params).map(ApiExpressionUtils::objectToExpression).toArray(Expression[]::new)));
+	}
+
+	/**
+	 * A call to an inline function. For functions registered in a catalog use {@link #call(String, Object...)}.
+	 */
+	public static ApiExpression call(FunctionDefinition scalarFunction, Object... params) {
+		return apiCall(
+			scalarFunction,
+			Arrays.stream(params).map(ApiExpressionUtils::objectToExpression).toArray(Expression[]::new));
+	}
+
+	private static ApiExpression apiCall(FunctionDefinition functionDefinition, Expression... args) {
+		return new ApiExpression(new UnresolvedCallExpression(functionDefinition, Arrays.asList(args)));
+	}
+
+	/**
+	 * Java API class that gives access to expressions operations.
+	 */
+	public static final class ApiExpression extends BaseExpressions<Object, ApiExpression> implements Expression {
 
 Review comment:
   Move to top-level class with default scoped constructor? Otherwise I see `Expressions.ApiExpression expr = ...` variable declarations coming.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380097085
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/api/ExpressionsTest.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.table.api;
+
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.time.Duration;
+import java.time.Period;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for Java Expressions DSL.
+ */
+public class ExpressionsTest extends TestLogger {
 
 Review comment:
   Use parameterized test without logger? I'm sure we will add more tests here in the future.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148755202 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   Hash:93f931c7ef96767509f62982e48b4853d3a74dfc Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/149801561 TriggerType:PUSH TriggerID:93f931c7ef96767509f62982e48b4853d3a74dfc
   Hash:93f931c7ef96767509f62982e48b4853d3a74dfc Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362 TriggerType:PUSH TriggerID:93f931c7ef96767509f62982e48b4853d3a74dfc
   Hash:b79c6583d86289244513a44c260c5c5d6b520f48 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363 TriggerType:PUSH TriggerID:b79c6583d86289244513a44c260c5c5d6b520f48
   Hash:b79c6583d86289244513a44c260c5c5d6b520f48 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149806101 TriggerType:PUSH TriggerID:b79c6583d86289244513a44c260c5c5d6b520f48
   Hash:91b982ff6a238b4db2bc89e81e53327d731e9378 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:91b982ff6a238b4db2bc89e81e53327d731e9378
   -->
   ## CI report:
   
   * 16d4e9e6fbeb230b8aa05976e4909299ebc929bf Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148755202) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128) 
   * 93f931c7ef96767509f62982e48b4853d3a74dfc Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/149801561) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362) 
   * b79c6583d86289244513a44c260c5c5d6b520f48 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149806101) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363) 
   * 91b982ff6a238b4db2bc89e81e53327d731e9378 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] flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148755202",
       "triggerID" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149801561",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149806101",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149869824",
       "triggerID" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149885015",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149970663",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149981047",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150432772",
       "triggerID" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150669044",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5643",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d510ce90e5cd8a13256c84c855562c403091a14",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150861986",
       "triggerID" : "8d510ce90e5cd8a13256c84c855562c403091a14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3786dab008166eff376617dabda4dedeba7ca961",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3786dab008166eff376617dabda4dedeba7ca961",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0e3ee0c3b8b58815b2f771d8b4320e7483c41159",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0e3ee0c3b8b58815b2f771d8b4320e7483c41159",
       "triggerType" : "PUSH"
     }, {
       "hash" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5706",
       "triggerID" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150885626",
       "triggerID" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f9502902a9c1a3203a22fdeb91f47551c3e862d1",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150974862",
       "triggerID" : "f9502902a9c1a3203a22fdeb91f47551c3e862d1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f9502902a9c1a3203a22fdeb91f47551c3e862d1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5721",
       "triggerID" : "f9502902a9c1a3203a22fdeb91f47551c3e862d1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3786dab008166eff376617dabda4dedeba7ca961 UNKNOWN
   * 0e3ee0c3b8b58815b2f771d8b4320e7483c41159 UNKNOWN
   * f9502902a9c1a3203a22fdeb91f47551c3e862d1 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/150974862) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5721) 
   
   <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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r385017585
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/expressions/resolver/ExpressionResolverTest.java
 ##########
 @@ -0,0 +1,391 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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;
+import org.apache.flink.table.catalog.DataTypeFactory;
+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.operations.CatalogQueryOperation;
+import org.apache.flink.table.operations.QueryOperation;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.inference.TypeInferenceUtil;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeParser;
+
+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.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.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.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType;
+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:
+ *
+ * <p>- BuiltinFunctionDefinitions.EQUALS
+ * - BuiltinFunctionDefinitions.IS_NULL
+ *
+ * <p>Pseudo functions that are executed during expression resolution e.g.:
+ * - BuiltinFunctionDefinitions.WITH_COLUMNS
+ * - BuiltinFunctionDefinitions.WITHOUT_COLUMNS
+ * - BuiltinFunctionDefinitions.RANGE_TO
+ * - BuiltinFunctionDefinitions.FLATTEN
+ *
+ * <p>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<TestSpec> 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 calls")
+				.inputSchemas(
+					TableSchema.builder()
+						.field("f0", DataTypes.INT())
+						.build()
+				)
+				.lookupFunction("func", new ScalarFunc())
+				.select(call("func", 1, $("f0")))
+				.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("Catalog calls")
+				.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 calls")
 
 Review comment:
   "Deeply nested user-defined inline calls"

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r385016722
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/expressions/resolver/ExpressionResolverTest.java
 ##########
 @@ -0,0 +1,391 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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;
+import org.apache.flink.table.catalog.DataTypeFactory;
+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.operations.CatalogQueryOperation;
+import org.apache.flink.table.operations.QueryOperation;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.inference.TypeInferenceUtil;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeParser;
+
+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.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.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.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType;
+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:
+ *
+ * <p>- BuiltinFunctionDefinitions.EQUALS
+ * - BuiltinFunctionDefinitions.IS_NULL
+ *
+ * <p>Pseudo functions that are executed during expression resolution e.g.:
+ * - BuiltinFunctionDefinitions.WITH_COLUMNS
+ * - BuiltinFunctionDefinitions.WITHOUT_COLUMNS
+ * - BuiltinFunctionDefinitions.RANGE_TO
+ * - BuiltinFunctionDefinitions.FLATTEN
+ *
+ * <p>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<TestSpec> 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 calls")
+				.inputSchemas(
+					TableSchema.builder()
+						.field("f0", DataTypes.INT())
+						.build()
+				)
+				.lookupFunction("func", new ScalarFunc())
+				.select(call("func", 1, $("f0")))
+				.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("Catalog calls")
 
 Review comment:
   "Lookup catalog function call"

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380059258
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,527 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.internal.BaseExpressions;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.ExpressionVisitor;
+import org.apache.flink.table.expressions.TimePointUnit;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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 java.util.Arrays;
+import java.util.List;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral;
+
+/**
+ * Entry point for a Java Table Expression DSL.
 
 Review comment:
   `Check the documentation for more programming language specific APIs, for example, by using Scala implicits.`

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148755202",
       "triggerID" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149801561",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149806101",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149869824",
       "triggerID" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149885015",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149970663",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149981047",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3c118088adedb8b818d0371495d313e9fe038968 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/149981047) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408) 
   
   <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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148755202",
       "triggerID" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149801561",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149806101",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149869824",
       "triggerID" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149885015",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149970663",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149981047",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150432772",
       "triggerID" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150669044",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5643",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d510ce90e5cd8a13256c84c855562c403091a14",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150861986",
       "triggerID" : "8d510ce90e5cd8a13256c84c855562c403091a14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3786dab008166eff376617dabda4dedeba7ca961",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3786dab008166eff376617dabda4dedeba7ca961",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0e3ee0c3b8b58815b2f771d8b4320e7483c41159",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0e3ee0c3b8b58815b2f771d8b4320e7483c41159",
       "triggerType" : "PUSH"
     }, {
       "hash" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5706",
       "triggerID" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150885626",
       "triggerID" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3786dab008166eff376617dabda4dedeba7ca961 UNKNOWN
   * 0e3ee0c3b8b58815b2f771d8b4320e7483c41159 UNKNOWN
   * 156ddaa5298b8059b4038c486e00770a263c25f7 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/150885626) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5706) 
   
   <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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380074212
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,527 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.internal.BaseExpressions;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.ExpressionVisitor;
+import org.apache.flink.table.expressions.TimePointUnit;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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 java.util.Arrays;
+import java.util.List;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral;
+
+/**
+ * Entry point for a Java Table Expression DSL.
+ */
+@PublicEvolving
+public final class Expressions {
+
+	/**
+	 * Creates an unresolved reference to a table's column.
+	 */
+	//CHECKSTYLE.OFF: MethodName
+	public static ApiExpression $(String name) {
+		return new ApiExpression(ApiExpressionUtils.unresolvedRef(name));
+	}
+	//CHECKSTYLE.ON: MethodName
+
+	/**
+	 * Creates a SQL literal. The data type is derived from the Java class.
+	 */
+	public static ApiExpression lit(Object v) {
+		return new ApiExpression(valueLiteral(v));
+	}
+
+	/**
+	 * Creates a SQL literal of a given {@link DataType}.
+	 */
+	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.
+	 *
+	 * @see #withColumns(Object, Object...)
+	 * @see #withoutColumns(Object, Object...)
+	 */
+	public static ApiExpression range(String start, String end) {
+		return apiCall(BuiltInFunctionDefinitions.RANGE_TO, valueLiteral(start), valueLiteral(end));
+	}
+
+	/**
+	 * Indicates an index based range, which can be used in columns selection.
+	 *
+	 * @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 apiCall(BuiltInFunctionDefinitions.AND, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Boolean OR in three-valued logic.
+	 */
+	public static ApiExpression or(Object predicate0, Object predicate1, Object... predicates) {
+		return apiCall(BuiltInFunctionDefinitions.OR, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 <code>leftEnd >= rightStart && rightEnd >= leftStart</code>.
+	 *
+	 * <p>It evaluates: leftEnd >= rightStart && rightEnd >= leftStart
+	 *
+	 * <p>e.g. temporalOverlaps("2:55:00".toTime, 1.hour, "3:30:00".toTime, 2.hour) leads to true
+	 */
+	public static ApiExpression temporalOverlaps(
+			Object leftTimePoint,
+			Object leftTemporal,
+			Object rightTimePoint,
+			Object rightTemporal) {
+		return apiCall(
+			BuiltInFunctionDefinitions.TEMPORAL_OVERLAPS,
+			objectToExpression(leftTimePoint),
+			objectToExpression(leftTemporal),
+			objectToExpression(rightTimePoint),
+			objectToExpression(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.
+	 *
+	 * <p>For example 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,
+			objectToExpression(timestamp),
+			objectToExpression(format));
+	}
+
+	/**
+	 * Returns the (signed) number of {@link TimePointUnit} between timePoint1 and timePoint2.
+	 *
+	 * <p>For example, {@code timestampDiff(TimePointUnit.DAY, '2016-06-15'.toDate, '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),
+			objectToExpression(timePoint1),
+			objectToExpression(timePoint2));
+	}
+
+	/**
+	 * Creates an array of literals.
+	 */
+	public static ApiExpression array(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.ARRAY,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a row of expressions.
+	 */
+	public static ApiExpression row(Object head, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.ROW, Stream.concat(
+			Stream.of(head),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a map of expressions.
+	 */
+	public static ApiExpression map(Object key, Object value, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.MAP, Stream.concat(
+			Stream.of(key, value),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 apiCall(
+			BuiltInFunctionDefinitions.CONCAT,
+			Stream.concat(
+				Stream.of(string),
+				Stream.of(strings)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Calculates the arc tangent of a given coordinate.
+	 */
+	public static ApiExpression atan2(Object y, Object x) {
+		return apiCall(BuiltInFunctionDefinitions.ATAN2, objectToExpression(y), objectToExpression(x));
+	}
+
+	/**
+	 * Returns the string that results from concatenating the arguments and separator.
+	 * Returns NULL If the separator is NULL.
+	 *
+	 * <p>Note: this user-public static ApiExpressionined 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 apiCall(BuiltInFunctionDefinitions.CONCAT_WS, Stream.concat(
+			Stream.of(separator, string),
+			Stream.of(strings)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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.
+	 *
+	 * <p>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, objectToExpression(value));
+	}
+
+	/**
+	 * Calculates the logarithm of the given value to the given base.
+	 */
+	public static ApiExpression log(Object base, Object value) {
+		return apiCall(BuiltInFunctionDefinitions.LOG, objectToExpression(base), objectToExpression(value));
+	}
+
+	/**
+	 * Ternary conditional operator that decides which of two other expressions should be evaluated
+	 * based on a evaluated boolean condition.
+	 *
+	 * <p>e.g. ifThenElse(42 > 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,
+			objectToExpression(condition),
+			objectToExpression(ifTrue),
+			objectToExpression(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.
+	 *
+	 * <p>A range can either be index-based or name-based. Indices start at 1 and boundaries are
+	 * inclusive.
+	 *
+	 * <p>e.g. withColumns('b to 'c) or withColumns('*)
+	 */
+	public static ApiExpression withColumns(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.WITH_COLUMNS,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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.
+	 *
+	 * <p>A range can either be index-based or name-based. Indices start at 1 and boundaries are
+	 * inclusive.
+	 *
+	 * <p>e.g. withoutColumns('b to 'c) or withoutColumns('c)
+	 */
+	public static ApiExpression withoutColumns(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.WITHOUT_COLUMNS,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * A call to a function that will be looked up in a catalog.
+	 */
+	public static ApiExpression call(String functionName, Object... params) {
+		return new ApiExpression(ApiExpressionUtils.lookupCall(
+			functionName,
+			Arrays.stream(params).map(ApiExpressionUtils::objectToExpression).toArray(Expression[]::new)));
+	}
+
+	/**
+	 * A call to an inline function. For functions registered in a catalog use {@link #call(String, Object...)}.
+	 */
+	public static ApiExpression call(FunctionDefinition scalarFunction, Object... params) {
+		return apiCall(
+			scalarFunction,
+			Arrays.stream(params).map(ApiExpressionUtils::objectToExpression).toArray(Expression[]::new));
+	}
+
+	private static ApiExpression apiCall(FunctionDefinition functionDefinition, Expression... args) {
+		return new ApiExpression(new UnresolvedCallExpression(functionDefinition, Arrays.asList(args)));
+	}
+
+	/**
+	 * Java API class that gives access to expressions operations.
+	 */
+	public static final class ApiExpression extends BaseExpressions<Object, ApiExpression> implements Expression {
+		private final Expression wrappedExpression;
+
+		@Override
+		public String asSummaryString() {
+			return wrappedExpression.asSummaryString();
+		}
+
+		private ApiExpression(Expression wrappedExpression) {
+			if (wrappedExpression instanceof ApiExpression) {
+				throw new UnsupportedOperationException("This is a bug. Please file a JIRA.");
 
 Review comment:
   nit: `Please file an issue.` You never know if we change our issue tracking in the future :D

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r384986407
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,562 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.catalog.DataTypeFactory;
+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.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;
+
+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()}
+ *
+ * <p>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.
+ *
+ * <p>For fluent definition of expressions and easier readability, we recommend to add a
+ * star import to the methods of this class:
+ *
+ * <pre>
+ * import static org.apache.flink.table.api.Expressions.*;
+ * </pre>
+ *
+ * <p>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.
+	 *
+	 * <p>Example
+	 * <pre>{@code
+	 *   tab.select($("key"), $("value"))
+	 * }
+	 * </pre>
+	 */
+	//CHECKSTYLE.OFF: MethodName
+	public static ApiExpression $(String name) {
+		return new ApiExpression(unresolvedRef(name));
+	}
+	//CHECKSTYLE.ON: MethodName
+
+	/**
+	 * Creates a SQL literal.
+	 *
+	 * <p>The data type is derived from the object's class and its value.
+	 *
+	 * <p>For example:
+	 * <ul>
+	 *     <li>{@code lit(12)} leads to {@code INT}</li>
+	 *     <li>{@code lit("abc")} leads to {@code CHAR(3)}</li>
+	 *     <li>{@code lit(new BigDecimal("123.45"))} leads to {@code DECIMAL(5, 2)}</li>
+	 * </ul>
+	 *
+	 * <p>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}.
+	 *
+	 * <p>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.
+	 *
+	 * <p>Example:
+	 * <pre>{@code
+	 * Table table = ...
+	 * table.withColumns(range(b, c))
+	 * }</pre>
+	 *
+	 * @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.
+	 *
+	 * <p>Example:
+	 * <pre>{@code
+	 * Table table = ...
+	 * table.withColumns(range(3, 4))
 
 Review comment:
   within `select()`

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380043275
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,527 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.internal.BaseExpressions;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.ExpressionVisitor;
+import org.apache.flink.table.expressions.TimePointUnit;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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 java.util.Arrays;
+import java.util.List;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral;
+
+/**
+ * Entry point for a Java Table Expression DSL.
 
 Review comment:
   ```
   Entry point of the Table API Expression DSL such as: {@code $("myField").plus(10).abs()}
   
   <p>This class contains static methods for referencing table columns, creating literals, and building more complex {@link Expression} chains. {@link ApiExpression}s are pure API entities that are further translated into {@link ResolvedExpression}s under the hood.
   
   <p>For fluent definition of expressions and easier readability, we recommend to add a star import to the methods of this class:
   
   <pre>
   import static org.apache.flink.table.api.Expressions.*;
   <pre>
   ```

----------------------------------------------------------------
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 issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
dawidwys commented on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-588966463
 
 
   I think I addressed most of the comments. Some of the comments I decided to extract to separate PRs. This meant though I had to change the history of the commits quite a bit.
   
   Therefore I decided to squash the updates into respective commits rather than having them as separate commits on top of the old ones.
   
   Moreover I removed the expression that creates a `LookupCallExpression` as this requires some changes in the `ExpressionResolver`. I will reintroduce that in a separate PR.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148755202",
       "triggerID" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149801561",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149806101",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149869824",
       "triggerID" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149885015",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149970663",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149981047",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150432772",
       "triggerID" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150669044",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5643",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d510ce90e5cd8a13256c84c855562c403091a14",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150861986",
       "triggerID" : "8d510ce90e5cd8a13256c84c855562c403091a14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3786dab008166eff376617dabda4dedeba7ca961",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3786dab008166eff376617dabda4dedeba7ca961",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0e3ee0c3b8b58815b2f771d8b4320e7483c41159",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0e3ee0c3b8b58815b2f771d8b4320e7483c41159",
       "triggerType" : "PUSH"
     }, {
       "hash" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5706",
       "triggerID" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150885626",
       "triggerID" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 8d510ce90e5cd8a13256c84c855562c403091a14 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/150861986) 
   * 3786dab008166eff376617dabda4dedeba7ca961 UNKNOWN
   * 0e3ee0c3b8b58815b2f771d8b4320e7483c41159 UNKNOWN
   * 156ddaa5298b8059b4038c486e00770a263c25f7 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/150885626) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5706) 
   
   <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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r384995250
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java
 ##########
 @@ -0,0 +1,1286 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 <InType>  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 <OutType> 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<InType, OutType> {
+	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 org.apache.flink.table.api.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 org.apache.flink.table.api.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 multipliedBy(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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>For example:
+	 * <pre>
+	 * {@code
+	 * orders
+	 *  .groupBy($("a"))
+	 *  .select($("a"), $("b").sum().distinct().as("d"))
+	 * }
+	 * </pre>
+	 */
+	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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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()));
+	}
+
+	// scalar functions
 
 Review comment:
   nit: remove it or make it more visible like?
   ```
   // ------
   // Scalar function
   // ----
   ```

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
dawidwys commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r385216930
 
 

 ##########
 File path: flink-table/flink-table-api-scala/src/test/scala/org/apache/flink/table/api/ExpressionsConsistencyCheckTest.scala
 ##########
 @@ -0,0 +1,294 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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" -> "multipliedBy", // *
+    "$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 rowsInterval
 
 Review comment:
   That's a typo in the comment. I fixed it.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r384991195
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java
 ##########
 @@ -0,0 +1,1286 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 <InType>  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 <OutType> 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
 
 Review comment:
   nit: `In Scala the`

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148755202",
       "triggerID" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149801561",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149806101",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149869824",
       "triggerID" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149885015",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149970663",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149981047",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150432772",
       "triggerID" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150669044",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5643",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d510ce90e5cd8a13256c84c855562c403091a14",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150861986",
       "triggerID" : "8d510ce90e5cd8a13256c84c855562c403091a14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3786dab008166eff376617dabda4dedeba7ca961",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3786dab008166eff376617dabda4dedeba7ca961",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0e3ee0c3b8b58815b2f771d8b4320e7483c41159",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0e3ee0c3b8b58815b2f771d8b4320e7483c41159",
       "triggerType" : "PUSH"
     }, {
       "hash" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5706",
       "triggerID" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150885626",
       "triggerID" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f9502902a9c1a3203a22fdeb91f47551c3e862d1",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150974862",
       "triggerID" : "f9502902a9c1a3203a22fdeb91f47551c3e862d1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f9502902a9c1a3203a22fdeb91f47551c3e862d1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5721",
       "triggerID" : "f9502902a9c1a3203a22fdeb91f47551c3e862d1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2264ba7fa5063309c8cb58cc4c32694b90164f0d",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150998989",
       "triggerID" : "2264ba7fa5063309c8cb58cc4c32694b90164f0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2264ba7fa5063309c8cb58cc4c32694b90164f0d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5728",
       "triggerID" : "2264ba7fa5063309c8cb58cc4c32694b90164f0d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3786dab008166eff376617dabda4dedeba7ca961 UNKNOWN
   * 0e3ee0c3b8b58815b2f771d8b4320e7483c41159 UNKNOWN
   * 2264ba7fa5063309c8cb58cc4c32694b90164f0d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/150998989) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5728) 
   
   <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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148755202 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   -->
   ## CI report:
   
   * 16d4e9e6fbeb230b8aa05976e4909299ebc929bf Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148755202) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128) 
   
   <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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r385003920
 
 

 ##########
 File path: flink-table/flink-table-api-scala/src/main/scala/org/apache/flink/table/api/expressionDsl.scala
 ##########
 @@ -1160,6 +359,13 @@ trait ImplicitExpressionConversions {
     }
   }
 
+  implicit class FieldExpression(val sc: StringContext) {
+    def $(args: Any*): Expression = unresolvedRef(sc.s(args: _*))
 
 Review comment:
   Add Scala Doc to method and class.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148755202",
       "triggerID" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149801561",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149806101",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149869824",
       "triggerID" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149885015",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149970663",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149981047",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150432772",
       "triggerID" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150669044",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0cd224f4da41acc6973bbd4fb27f11e201832e6e Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/150432772) 
   * 4bcbe8c726aa4cda0ac0214e5997c9c0d38640be Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/150669044) 
   
   <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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r385006297
 
 

 ##########
 File path: flink-table/flink-table-api-scala/src/main/scala/org/apache/flink/table/api/expressionDsl.scala
 ##########
 @@ -1471,9 +715,22 @@ trait ImplicitExpressionConversions {
     *
     * Note: this user-defined 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
 
 Review comment:
   nit: `Note: This function does not`

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148755202 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   Hash:93f931c7ef96767509f62982e48b4853d3a74dfc Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/149801561 TriggerType:PUSH TriggerID:93f931c7ef96767509f62982e48b4853d3a74dfc
   Hash:93f931c7ef96767509f62982e48b4853d3a74dfc Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362 TriggerType:PUSH TriggerID:93f931c7ef96767509f62982e48b4853d3a74dfc
   Hash:b79c6583d86289244513a44c260c5c5d6b520f48 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363 TriggerType:PUSH TriggerID:b79c6583d86289244513a44c260c5c5d6b520f48
   Hash:b79c6583d86289244513a44c260c5c5d6b520f48 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149806101 TriggerType:PUSH TriggerID:b79c6583d86289244513a44c260c5c5d6b520f48
   Hash:91b982ff6a238b4db2bc89e81e53327d731e9378 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/149869824 TriggerType:PUSH TriggerID:91b982ff6a238b4db2bc89e81e53327d731e9378
   Hash:01b03edabdb80d7a941b10ecccb2670973d8b64a Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:01b03edabdb80d7a941b10ecccb2670973d8b64a
   -->
   ## CI report:
   
   * 16d4e9e6fbeb230b8aa05976e4909299ebc929bf Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148755202) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128) 
   * 93f931c7ef96767509f62982e48b4853d3a74dfc Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/149801561) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362) 
   * b79c6583d86289244513a44c260c5c5d6b520f48 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149806101) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363) 
   * 91b982ff6a238b4db2bc89e81e53327d731e9378 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/149869824) 
   * 01b03edabdb80d7a941b10ecccb2670973d8b64a 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] flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148755202 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   Hash:93f931c7ef96767509f62982e48b4853d3a74dfc Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/149801561 TriggerType:PUSH TriggerID:93f931c7ef96767509f62982e48b4853d3a74dfc
   Hash:93f931c7ef96767509f62982e48b4853d3a74dfc Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362 TriggerType:PUSH TriggerID:93f931c7ef96767509f62982e48b4853d3a74dfc
   Hash:b79c6583d86289244513a44c260c5c5d6b520f48 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363 TriggerType:PUSH TriggerID:b79c6583d86289244513a44c260c5c5d6b520f48
   Hash:b79c6583d86289244513a44c260c5c5d6b520f48 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149806101 TriggerType:PUSH TriggerID:b79c6583d86289244513a44c260c5c5d6b520f48
   Hash:91b982ff6a238b4db2bc89e81e53327d731e9378 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/149869824 TriggerType:PUSH TriggerID:91b982ff6a238b4db2bc89e81e53327d731e9378
   Hash:01b03edabdb80d7a941b10ecccb2670973d8b64a Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149885015 TriggerType:PUSH TriggerID:01b03edabdb80d7a941b10ecccb2670973d8b64a
   Hash:01b03edabdb80d7a941b10ecccb2670973d8b64a Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391 TriggerType:PUSH TriggerID:01b03edabdb80d7a941b10ecccb2670973d8b64a
   Hash:2590f92a3dbe4302baf85b67b090ffb75b05d52b Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:2590f92a3dbe4302baf85b67b090ffb75b05d52b
   -->
   ## CI report:
   
   * 16d4e9e6fbeb230b8aa05976e4909299ebc929bf Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148755202) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128) 
   * 93f931c7ef96767509f62982e48b4853d3a74dfc Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/149801561) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362) 
   * b79c6583d86289244513a44c260c5c5d6b520f48 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149806101) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363) 
   * 91b982ff6a238b4db2bc89e81e53327d731e9378 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/149869824) 
   * 01b03edabdb80d7a941b10ecccb2670973d8b64a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149885015) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391) 
   * 2590f92a3dbe4302baf85b67b090ffb75b05d52b 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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380082553
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/rules/ExpandColumnFunctionsRule.java
 ##########
 @@ -140,7 +140,7 @@ public ColumnFunctionsExpander(List<UnresolvedReferenceExpression> inputFieldRef
 	/**
 	 * Expands a single column expression to it's real parent's input references.
 	 */
-	class ColumnsExpressionExpander extends ApiExpressionDefaultVisitor<List<UnresolvedReferenceExpression>> {
+	static class ColumnsExpressionExpander extends ApiExpressionDefaultVisitor<List<UnresolvedReferenceExpression>> {
 
 Review comment:
   nit: make it private as well?

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148755202",
       "triggerID" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149801561",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149806101",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149869824",
       "triggerID" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149885015",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149970663",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149981047",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150432772",
       "triggerID" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150669044",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5643",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d510ce90e5cd8a13256c84c855562c403091a14",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150861986",
       "triggerID" : "8d510ce90e5cd8a13256c84c855562c403091a14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3786dab008166eff376617dabda4dedeba7ca961",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3786dab008166eff376617dabda4dedeba7ca961",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0e3ee0c3b8b58815b2f771d8b4320e7483c41159",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0e3ee0c3b8b58815b2f771d8b4320e7483c41159",
       "triggerType" : "PUSH"
     }, {
       "hash" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5706",
       "triggerID" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150885626",
       "triggerID" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f9502902a9c1a3203a22fdeb91f47551c3e862d1",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f9502902a9c1a3203a22fdeb91f47551c3e862d1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3786dab008166eff376617dabda4dedeba7ca961 UNKNOWN
   * 0e3ee0c3b8b58815b2f771d8b4320e7483c41159 UNKNOWN
   * 156ddaa5298b8059b4038c486e00770a263c25f7 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/150885626) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5706) 
   * f9502902a9c1a3203a22fdeb91f47551c3e862d1 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] dawidwys commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
dawidwys commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380155009
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableImpl.java
 ##########
 @@ -563,6 +559,16 @@ private TableImpl createTable(QueryOperation operation) {
 		return new TableImpl(tableEnvironment, operation, operationTreeBuilder, lookupResolver);
 	}
 
+	private List<Expression> preprocessExpressions(List<Expression> expressions) {
 
 Review comment:
   why? preprocess is a verb
   
   `preProcessExpressions` would miss a verb

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r384996692
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java
 ##########
 @@ -0,0 +1,1286 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 <InType>  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 <OutType> 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<InType, OutType> {
+	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 org.apache.flink.table.api.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 org.apache.flink.table.api.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 multipliedBy(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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>For example:
+	 * <pre>
+	 * {@code
+	 * orders
+	 *  .groupBy($("a"))
+	 *  .select($("a"), $("b").sum().distinct().as("d"))
+	 * }
+	 * </pre>
+	 */
+	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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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()));
+	}
+
+	// scalar functions
+
+	/**
+	 * 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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>e.g. "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.
+	 *
+	 * <p>e.g. "hi".lpad(4, '??') returns "??hi",  "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.
+	 *
+	 * <p>e.g. "hi".rpad(4, '??') returns "hi??",  "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.
+	 *
+	 * <p>For example:
+	 *
+	 * <pre>
+	 * {@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)
+	 * }</pre>
+	 */
+	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).
+	 *
+	 * <p>e.g. "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.
+	 *
+	 * <p>e.g. "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.
+	 *
+	 * <p>e.g. "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.
+	 *
+	 * <p>e.g. "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.
+	 *
+	 * <p>e.g. "12:44:31".toDate.ceil(MINUTE) leads to 12:45:00
 
 Review comment:
   update to Java

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r384996770
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java
 ##########
 @@ -0,0 +1,1286 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 <InType>  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 <OutType> 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<InType, OutType> {
+	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 org.apache.flink.table.api.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 org.apache.flink.table.api.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 multipliedBy(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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>For example:
+	 * <pre>
+	 * {@code
+	 * orders
+	 *  .groupBy($("a"))
+	 *  .select($("a"), $("b").sum().distinct().as("d"))
+	 * }
+	 * </pre>
+	 */
+	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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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()));
+	}
+
+	// scalar functions
+
+	/**
+	 * 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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>e.g. "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.
+	 *
+	 * <p>e.g. "hi".lpad(4, '??') returns "??hi",  "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.
+	 *
+	 * <p>e.g. "hi".rpad(4, '??') returns "hi??",  "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.
+	 *
+	 * <p>For example:
+	 *
+	 * <pre>
+	 * {@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)
+	 * }</pre>
+	 */
+	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).
+	 *
+	 * <p>e.g. "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.
+	 *
+	 * <p>e.g. "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.
+	 *
+	 * <p>e.g. "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.
+	 *
+	 * <p>e.g. "12:44:31".toDate.floor(MINUTE) leads to 12:44:00
 
 Review comment:
   update to Java?

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r385016651
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/expressions/resolver/ExpressionResolverTest.java
 ##########
 @@ -0,0 +1,391 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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;
+import org.apache.flink.table.catalog.DataTypeFactory;
+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.operations.CatalogQueryOperation;
+import org.apache.flink.table.operations.QueryOperation;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.inference.TypeInferenceUtil;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeParser;
+
+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.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.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.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType;
+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:
+ *
+ * <p>- BuiltinFunctionDefinitions.EQUALS
+ * - BuiltinFunctionDefinitions.IS_NULL
+ *
+ * <p>Pseudo functions that are executed during expression resolution e.g.:
+ * - BuiltinFunctionDefinitions.WITH_COLUMNS
+ * - BuiltinFunctionDefinitions.WITHOUT_COLUMNS
+ * - BuiltinFunctionDefinitions.RANGE_TO
+ * - BuiltinFunctionDefinitions.FLATTEN
+ *
+ * <p>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<TestSpec> 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 calls")
 
 Review comment:
   "Lookup system function call"

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r385008937
 
 

 ##########
 File path: flink-table/flink-table-api-scala/src/test/scala/org/apache/flink/table/api/ExpressionsConsistencyCheckTest.scala
 ##########
 @@ -0,0 +1,294 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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" -> "multipliedBy", // *
 
 Review comment:
   should we also call it `times`? It would be shorter but not in sync with `dividedBy` anymore.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380186145
 
 

 ##########
 File path: flink-table/flink-table-api-scala/src/main/scala/org/apache/flink/table/api/expressionDsl.scala
 ##########
 @@ -355,190 +155,10 @@ trait ImplicitExpressionOperations {
     * @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
 
 Review comment:
   difficult, here are some inspirations: https://en.wikipedia.org/wiki/%3F:
   
   How about "then" or "ifThenElse" again: `$("f").equalTo(12).then(12, null)`

----------------------------------------------------------------
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 issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
dawidwys commented on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-593116938
 
 
   The remaining commens/issues apply to calling user defined functions with the new type inference stack. I will move the `call(UserDefinedFunction call, ...)` method and all related code to a separate PR and merge the base expressions.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
dawidwys commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380138604
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/rules/UnwrapApiExpressionRule.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.expressions.resolver.rules;
+
+import org.apache.flink.table.api.Expressions;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Unwraps all {@link Expressions.ApiExpression}.
+ */
+final class UnwrapApiExpressionRule implements ResolverRule {
 
 Review comment:
   Isn't it enough that the class has the default scope?

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148755202 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   Hash:93f931c7ef96767509f62982e48b4853d3a74dfc Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:93f931c7ef96767509f62982e48b4853d3a74dfc
   -->
   ## CI report:
   
   * 16d4e9e6fbeb230b8aa05976e4909299ebc929bf Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148755202) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128) 
   * 93f931c7ef96767509f62982e48b4853d3a74dfc 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] flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148755202 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   Hash:93f931c7ef96767509f62982e48b4853d3a74dfc Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/149801561 TriggerType:PUSH TriggerID:93f931c7ef96767509f62982e48b4853d3a74dfc
   Hash:93f931c7ef96767509f62982e48b4853d3a74dfc Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362 TriggerType:PUSH TriggerID:93f931c7ef96767509f62982e48b4853d3a74dfc
   Hash:b79c6583d86289244513a44c260c5c5d6b520f48 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363 TriggerType:PUSH TriggerID:b79c6583d86289244513a44c260c5c5d6b520f48
   Hash:b79c6583d86289244513a44c260c5c5d6b520f48 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149806101 TriggerType:PUSH TriggerID:b79c6583d86289244513a44c260c5c5d6b520f48
   Hash:91b982ff6a238b4db2bc89e81e53327d731e9378 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/149869824 TriggerType:PUSH TriggerID:91b982ff6a238b4db2bc89e81e53327d731e9378
   Hash:01b03edabdb80d7a941b10ecccb2670973d8b64a Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149885015 TriggerType:PUSH TriggerID:01b03edabdb80d7a941b10ecccb2670973d8b64a
   Hash:01b03edabdb80d7a941b10ecccb2670973d8b64a Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391 TriggerType:PUSH TriggerID:01b03edabdb80d7a941b10ecccb2670973d8b64a
   -->
   ## CI report:
   
   * 16d4e9e6fbeb230b8aa05976e4909299ebc929bf Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148755202) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128) 
   * 93f931c7ef96767509f62982e48b4853d3a74dfc Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/149801561) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362) 
   * b79c6583d86289244513a44c260c5c5d6b520f48 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149806101) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363) 
   * 91b982ff6a238b4db2bc89e81e53327d731e9378 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/149869824) 
   * 01b03edabdb80d7a941b10ecccb2670973d8b64a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149885015) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391) 
   
   <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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380065132
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,527 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.internal.BaseExpressions;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.ExpressionVisitor;
+import org.apache.flink.table.expressions.TimePointUnit;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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 java.util.Arrays;
+import java.util.List;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral;
+
+/**
+ * Entry point for a Java Table Expression DSL.
+ */
+@PublicEvolving
+public final class Expressions {
+
+	/**
+	 * Creates an unresolved reference to a table's column.
+	 */
+	//CHECKSTYLE.OFF: MethodName
+	public static ApiExpression $(String name) {
+		return new ApiExpression(ApiExpressionUtils.unresolvedRef(name));
+	}
+	//CHECKSTYLE.ON: MethodName
+
+	/**
+	 * Creates a SQL literal. The data type is derived from the Java class.
+	 */
+	public static ApiExpression lit(Object v) {
+		return new ApiExpression(valueLiteral(v));
+	}
+
+	/**
+	 * Creates a SQL literal of a given {@link DataType}.
+	 */
+	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.
+	 *
+	 * @see #withColumns(Object, Object...)
+	 * @see #withoutColumns(Object, Object...)
+	 */
+	public static ApiExpression range(String start, String end) {
+		return apiCall(BuiltInFunctionDefinitions.RANGE_TO, valueLiteral(start), valueLiteral(end));
+	}
+
+	/**
+	 * Indicates an index based range, which can be used in columns selection.
+	 *
+	 * @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 apiCall(BuiltInFunctionDefinitions.AND, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Boolean OR in three-valued logic.
+	 */
+	public static ApiExpression or(Object predicate0, Object predicate1, Object... predicates) {
+		return apiCall(BuiltInFunctionDefinitions.OR, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 <code>leftEnd >= rightStart && rightEnd >= leftStart</code>.
+	 *
+	 * <p>It evaluates: leftEnd >= rightStart && rightEnd >= leftStart
+	 *
+	 * <p>e.g. temporalOverlaps("2:55:00".toTime, 1.hour, "3:30:00".toTime, 2.hour) leads to true
 
 Review comment:
   update example with Expression DSL?

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r385024146
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/expressions/resolver/ExpressionResolverTest.java
 ##########
 @@ -0,0 +1,391 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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;
+import org.apache.flink.table.catalog.DataTypeFactory;
+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.operations.CatalogQueryOperation;
+import org.apache.flink.table.operations.QueryOperation;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.inference.TypeInferenceUtil;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeParser;
+
+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.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.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.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType;
+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:
+ *
+ * <p>- BuiltinFunctionDefinitions.EQUALS
+ * - BuiltinFunctionDefinitions.IS_NULL
+ *
+ * <p>Pseudo functions that are executed during expression resolution e.g.:
+ * - BuiltinFunctionDefinitions.WITH_COLUMNS
+ * - BuiltinFunctionDefinitions.WITHOUT_COLUMNS
+ * - BuiltinFunctionDefinitions.RANGE_TO
+ * - BuiltinFunctionDefinitions.FLATTEN
+ *
+ * <p>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<TestSpec> 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 calls")
+				.inputSchemas(
+					TableSchema.builder()
+						.field("f0", DataTypes.INT())
+						.build()
+				)
+				.lookupFunction("func", new ScalarFunc())
+				.select(call("func", 1, $("f0")))
+				.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("Catalog calls")
+				.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 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
+	public TestSpec testSpec;
+
+	@Test
+	public void testResolvingExpressions() {
+		List<ResolvedExpression> resolvedExpressions = testSpec.getResolver()
+			.resolve(Arrays.asList(testSpec.expressions));
+		assertThat(
+			resolvedExpressions,
+			equalTo(testSpec.expectedExpressions));
+	}
+
+	/**
+	 * 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;
+		}
+	}
+
+	private static class TestSpec {
+		private final String description;
+		private TableSchema[] schemas;
+		private Expression[] expressions;
+		private List<ResolvedExpression> expectedExpressions;
+		private Map<FunctionIdentifier, FunctionDefinition> 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() {
+			DataTypeFactory dataTypeFactory = new DataTypeFactory() {
 
 Review comment:
   Use `org.apache.flink.table.types.inference.utils.DataTypeFactoryMock` I already moved it to a more prominent location in my other PR.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r385601394
 
 

 ##########
 File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/UserDefinedFunctionConverter.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.functions.FunctionIdentifier;
+import org.apache.flink.table.functions.UserDefinedFunction;
+import org.apache.flink.table.planner.functions.bridging.BridgingSqlFunction;
+
+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
+ * that use the new type stack.
+ */
+public class UserDefinedFunctionConverter implements CallExpressionConvertRule {
+	@Override
+	public Optional<RexNode> convert(
+			CallExpression call,
+			ConvertContext context) {
+		if (!(call.getFunctionDefinition() instanceof UserDefinedFunction)) {
+			return Optional.empty();
+		}
+
+		switch (call.getFunctionDefinition().getKind()) {
+			case SCALAR:
+			case TABLE:
+				List<RexNode> args = call.getChildren().stream().map(context::toRexNode).collect(Collectors.toList());
+				return Optional.of(context.getRelBuilder().call(
+					BridgingSqlFunction.of(
+						context.getDataTypeFactory(),
+						context.getTypeFactory(),
+						SqlKind.OTHER_FUNCTION,
+						call.getFunctionIdentifier()
+							.orElse(FunctionIdentifier.of(call.getFunctionDefinition().getClass().getName())),
 
 Review comment:
   Should we integrate that into the call expression? Otherwise we have custom logic at a couple of places.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
dawidwys commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r385227403
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/LookupCallExpression.java
 ##########
 @@ -43,7 +42,7 @@
 
 	LookupCallExpression(String unresolvedFunction, List<Expression> args) {
 		this.unresolvedName = Preconditions.checkNotNull(unresolvedFunction);
-		this.args = Collections.unmodifiableList(new ArrayList<>(Preconditions.checkNotNull(args)));
+		this.args = Collections.unmodifiableList(Preconditions.checkNotNull(args));
 
 Review comment:
   Honestly I don't know if it makes sense to be so defensive here. 
   
   Usually the arguments are constructed either with `Arrays.asList` or via stream API which produce unmodifiable lists anyway.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148755202 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   Hash:93f931c7ef96767509f62982e48b4853d3a74dfc Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/149801561 TriggerType:PUSH TriggerID:93f931c7ef96767509f62982e48b4853d3a74dfc
   Hash:b79c6583d86289244513a44c260c5c5d6b520f48 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:b79c6583d86289244513a44c260c5c5d6b520f48
   -->
   ## CI report:
   
   * 16d4e9e6fbeb230b8aa05976e4909299ebc929bf Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148755202) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128) 
   * 93f931c7ef96767509f62982e48b4853d3a74dfc Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/149801561) 
   * b79c6583d86289244513a44c260c5c5d6b520f48 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] flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148755202",
       "triggerID" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149801561",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149806101",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149869824",
       "triggerID" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149885015",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149970663",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149981047",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150432772",
       "triggerID" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0cd224f4da41acc6973bbd4fb27f11e201832e6e Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/150432772) 
   
   <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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r384990126
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,562 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.catalog.DataTypeFactory;
+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.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;
+
+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()}
+ *
+ * <p>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.
+ *
+ * <p>For fluent definition of expressions and easier readability, we recommend to add a
+ * star import to the methods of this class:
+ *
+ * <pre>
+ * import static org.apache.flink.table.api.Expressions.*;
+ * </pre>
+ *
+ * <p>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.
+	 *
+	 * <p>Example
+	 * <pre>{@code
+	 *   tab.select($("key"), $("value"))
+	 * }
+	 * </pre>
+	 */
+	//CHECKSTYLE.OFF: MethodName
+	public static ApiExpression $(String name) {
+		return new ApiExpression(unresolvedRef(name));
+	}
+	//CHECKSTYLE.ON: MethodName
+
+	/**
+	 * Creates a SQL literal.
+	 *
+	 * <p>The data type is derived from the object's class and its value.
+	 *
+	 * <p>For example:
+	 * <ul>
+	 *     <li>{@code lit(12)} leads to {@code INT}</li>
+	 *     <li>{@code lit("abc")} leads to {@code CHAR(3)}</li>
+	 *     <li>{@code lit(new BigDecimal("123.45"))} leads to {@code DECIMAL(5, 2)}</li>
+	 * </ul>
+	 *
+	 * <p>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}.
+	 *
+	 * <p>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.
+	 *
+	 * <p>Example:
+	 * <pre>{@code
+	 * Table table = ...
+	 * table.withColumns(range(b, c))
+	 * }</pre>
+	 *
+	 * @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.
+	 *
+	 * <p>Example:
+	 * <pre>{@code
+	 * Table table = ...
+	 * table.withColumns(range(3, 4))
+	 * }</pre>
+	 *
+	 * @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 <code>leftEnd >= rightStart && rightEnd >= leftStart</code>.
+	 *
+	 * <p>It evaluates: leftEnd >= rightStart && rightEnd >= leftStart
+	 *
+	 * <p>e.g.
+	 * <pre>{@code
+	 * temporalOverlaps(
+	 *      lit("2:55:00").toTime(),
+	 *      interval(Duration.ofHour(1)),
+	 *      lit("3:30:00").toTime(),
+	 *      interval(Duration.ofHour(2))
+	 * }</pre>
+	 * 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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <pre>{@code
+	 *  table.select(
+	 *      map(
+	 *          "key1", 1,
+	 *          "key2", 2,
+	 *          "key3", 3
+	 *      ))
+	 * }</pre>
+	 *
+	 * <p>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 minus(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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>A range can either be index-based or name-based. Indices start at 1 and boundaries are
+	 * inclusive.
+	 *
+	 * <p>e.g. in Scala: 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.
+	 *
+	 * <p>A range can either be index-based or name-based. Indices start at 1 and boundaries are
+	 * inclusive.
+	 *
+	 * <p>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:
+	 * <ul>
+	 *     <li>System functions - which are identified with one part names</li>
+	 *     <li>Catalog functions - which are identified always with three parts names
+	 *     (catalog, database, function)</li>
+	 * </ul>
+	 *
+	 * <p>Moreover each function can either be a temporary function or permanent one
+	 * (which is stored in an external catalog).
+	 *
+	 * <p>Based on that two properties the resolution order for looking up a function based on
+	 * the provided {@code functionName} is following:
+	 * <ul>
+	 *     <li>Temporary system function</li>
+	 *     <li>System function</li>
+	 *     <li>Temporary catalog function</li>
+	 *     <li>Catalog function</li>
+	 * </ul>
+	 *
+	 * @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)));
+	}
+
+	/**
+	 * 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...)}.
+	 *
+	 * <p><b>NOTE:</b>This call uses the new type inference stack. It means it will use the result of
 
 Review comment:
   nit: I don't know if we should add this comment. I think it is better suited in the website docs. The JavaDocs of the scalar/table function are also updated already.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r384999779
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/LookupCallExpression.java
 ##########
 @@ -43,7 +42,7 @@
 
 	LookupCallExpression(String unresolvedFunction, List<Expression> args) {
 		this.unresolvedName = Preconditions.checkNotNull(unresolvedFunction);
-		this.args = Collections.unmodifiableList(new ArrayList<>(Preconditions.checkNotNull(args)));
+		this.args = Collections.unmodifiableList(Preconditions.checkNotNull(args));
 
 Review comment:
   In theory one can still modify the list from the outside. But I don't have a string opinion here.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148755202",
       "triggerID" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149801561",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149806101",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149869824",
       "triggerID" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149885015",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149970663",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149981047",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150432772",
       "triggerID" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150669044",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5643",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d510ce90e5cd8a13256c84c855562c403091a14",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150861986",
       "triggerID" : "8d510ce90e5cd8a13256c84c855562c403091a14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3786dab008166eff376617dabda4dedeba7ca961",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3786dab008166eff376617dabda4dedeba7ca961",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0e3ee0c3b8b58815b2f771d8b4320e7483c41159",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0e3ee0c3b8b58815b2f771d8b4320e7483c41159",
       "triggerType" : "PUSH"
     }, {
       "hash" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5706",
       "triggerID" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150885626",
       "triggerID" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3786dab008166eff376617dabda4dedeba7ca961 UNKNOWN
   * 0e3ee0c3b8b58815b2f771d8b4320e7483c41159 UNKNOWN
   * 156ddaa5298b8059b4038c486e00770a263c25f7 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/150885626) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5706) 
   
   <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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r385005375
 
 

 ##########
 File path: flink-table/flink-table-api-scala/src/main/scala/org/apache/flink/table/api/expressionDsl.scala
 ##########
 @@ -1284,47 +490,85 @@ 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: _*)
+
+  /**
+   * 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
   // ----------------------------------------------------------------------------------------------
 
+  /**
+   * 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 BigDecimal("123.45"))` leads to `DECIMAL(5, 2)`
 
 Review comment:
   `java.math.BigDecimal` because Scala decimals are not supported. Or we convert them ourselves in this method?

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380048495
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,527 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.internal.BaseExpressions;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.ExpressionVisitor;
+import org.apache.flink.table.expressions.TimePointUnit;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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 java.util.Arrays;
+import java.util.List;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral;
+
+/**
+ * Entry point for a Java Table Expression DSL.
+ */
+@PublicEvolving
+public final class Expressions {
+
+	/**
+	 * Creates an unresolved reference to a table's column.
+	 */
+	//CHECKSTYLE.OFF: MethodName
+	public static ApiExpression $(String name) {
+		return new ApiExpression(ApiExpressionUtils.unresolvedRef(name));
+	}
+	//CHECKSTYLE.ON: MethodName
+
+	/**
+	 * Creates a SQL literal. The data type is derived from the Java class.
+	 */
+	public static ApiExpression lit(Object v) {
+		return new ApiExpression(valueLiteral(v));
+	}
+
+	/**
+	 * Creates a SQL literal of a given {@link DataType}.
+	 */
+	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.
+	 *
+	 * @see #withColumns(Object, Object...)
+	 * @see #withoutColumns(Object, Object...)
+	 */
+	public static ApiExpression range(String start, String end) {
 
 Review comment:
   Example?

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380065284
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,527 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.internal.BaseExpressions;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.ExpressionVisitor;
+import org.apache.flink.table.expressions.TimePointUnit;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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 java.util.Arrays;
+import java.util.List;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral;
+
+/**
+ * Entry point for a Java Table Expression DSL.
+ */
+@PublicEvolving
+public final class Expressions {
+
+	/**
+	 * Creates an unresolved reference to a table's column.
+	 */
+	//CHECKSTYLE.OFF: MethodName
+	public static ApiExpression $(String name) {
+		return new ApiExpression(ApiExpressionUtils.unresolvedRef(name));
+	}
+	//CHECKSTYLE.ON: MethodName
+
+	/**
+	 * Creates a SQL literal. The data type is derived from the Java class.
+	 */
+	public static ApiExpression lit(Object v) {
+		return new ApiExpression(valueLiteral(v));
+	}
+
+	/**
+	 * Creates a SQL literal of a given {@link DataType}.
+	 */
+	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.
+	 *
+	 * @see #withColumns(Object, Object...)
+	 * @see #withoutColumns(Object, Object...)
+	 */
+	public static ApiExpression range(String start, String end) {
+		return apiCall(BuiltInFunctionDefinitions.RANGE_TO, valueLiteral(start), valueLiteral(end));
+	}
+
+	/**
+	 * Indicates an index based range, which can be used in columns selection.
+	 *
+	 * @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 apiCall(BuiltInFunctionDefinitions.AND, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Boolean OR in three-valued logic.
+	 */
+	public static ApiExpression or(Object predicate0, Object predicate1, Object... predicates) {
+		return apiCall(BuiltInFunctionDefinitions.OR, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 <code>leftEnd >= rightStart && rightEnd >= leftStart</code>.
+	 *
+	 * <p>It evaluates: leftEnd >= rightStart && rightEnd >= leftStart
+	 *
+	 * <p>e.g. temporalOverlaps("2:55:00".toTime, 1.hour, "3:30:00".toTime, 2.hour) leads to true
+	 */
+	public static ApiExpression temporalOverlaps(
+			Object leftTimePoint,
+			Object leftTemporal,
+			Object rightTimePoint,
+			Object rightTemporal) {
+		return apiCall(
+			BuiltInFunctionDefinitions.TEMPORAL_OVERLAPS,
+			objectToExpression(leftTimePoint),
+			objectToExpression(leftTemporal),
+			objectToExpression(rightTimePoint),
+			objectToExpression(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.
+	 *
+	 * <p>For example dataFormat('time, "%Y, %d %M") results in strings formatted as "2017, 05 May".
 
 Review comment:
   update example with Expression DSL?

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380071163
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,527 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.internal.BaseExpressions;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.ExpressionVisitor;
+import org.apache.flink.table.expressions.TimePointUnit;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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 java.util.Arrays;
+import java.util.List;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral;
+
+/**
+ * Entry point for a Java Table Expression DSL.
+ */
+@PublicEvolving
+public final class Expressions {
+
+	/**
+	 * Creates an unresolved reference to a table's column.
+	 */
+	//CHECKSTYLE.OFF: MethodName
+	public static ApiExpression $(String name) {
+		return new ApiExpression(ApiExpressionUtils.unresolvedRef(name));
+	}
+	//CHECKSTYLE.ON: MethodName
+
+	/**
+	 * Creates a SQL literal. The data type is derived from the Java class.
+	 */
+	public static ApiExpression lit(Object v) {
+		return new ApiExpression(valueLiteral(v));
+	}
+
+	/**
+	 * Creates a SQL literal of a given {@link DataType}.
+	 */
+	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.
+	 *
+	 * @see #withColumns(Object, Object...)
+	 * @see #withoutColumns(Object, Object...)
+	 */
+	public static ApiExpression range(String start, String end) {
+		return apiCall(BuiltInFunctionDefinitions.RANGE_TO, valueLiteral(start), valueLiteral(end));
+	}
+
+	/**
+	 * Indicates an index based range, which can be used in columns selection.
+	 *
+	 * @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 apiCall(BuiltInFunctionDefinitions.AND, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Boolean OR in three-valued logic.
+	 */
+	public static ApiExpression or(Object predicate0, Object predicate1, Object... predicates) {
+		return apiCall(BuiltInFunctionDefinitions.OR, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 <code>leftEnd >= rightStart && rightEnd >= leftStart</code>.
+	 *
+	 * <p>It evaluates: leftEnd >= rightStart && rightEnd >= leftStart
+	 *
+	 * <p>e.g. temporalOverlaps("2:55:00".toTime, 1.hour, "3:30:00".toTime, 2.hour) leads to true
+	 */
+	public static ApiExpression temporalOverlaps(
+			Object leftTimePoint,
+			Object leftTemporal,
+			Object rightTimePoint,
+			Object rightTemporal) {
+		return apiCall(
+			BuiltInFunctionDefinitions.TEMPORAL_OVERLAPS,
+			objectToExpression(leftTimePoint),
+			objectToExpression(leftTemporal),
+			objectToExpression(rightTimePoint),
+			objectToExpression(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.
+	 *
+	 * <p>For example 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,
+			objectToExpression(timestamp),
+			objectToExpression(format));
+	}
+
+	/**
+	 * Returns the (signed) number of {@link TimePointUnit} between timePoint1 and timePoint2.
+	 *
+	 * <p>For example, {@code timestampDiff(TimePointUnit.DAY, '2016-06-15'.toDate, '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),
+			objectToExpression(timePoint1),
+			objectToExpression(timePoint2));
+	}
+
+	/**
+	 * Creates an array of literals.
+	 */
+	public static ApiExpression array(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.ARRAY,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a row of expressions.
+	 */
+	public static ApiExpression row(Object head, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.ROW, Stream.concat(
+			Stream.of(head),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a map of expressions.
+	 */
+	public static ApiExpression map(Object key, Object value, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.MAP, Stream.concat(
+			Stream.of(key, value),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 apiCall(
+			BuiltInFunctionDefinitions.CONCAT,
+			Stream.concat(
+				Stream.of(string),
+				Stream.of(strings)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Calculates the arc tangent of a given coordinate.
+	 */
+	public static ApiExpression atan2(Object y, Object x) {
+		return apiCall(BuiltInFunctionDefinitions.ATAN2, objectToExpression(y), objectToExpression(x));
+	}
+
+	/**
+	 * Returns the string that results from concatenating the arguments and separator.
+	 * Returns NULL If the separator is NULL.
+	 *
+	 * <p>Note: this user-public static ApiExpressionined 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 apiCall(BuiltInFunctionDefinitions.CONCAT_WS, Stream.concat(
+			Stream.of(separator, string),
+			Stream.of(strings)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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.
+	 *
+	 * <p>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, objectToExpression(value));
+	}
+
+	/**
+	 * Calculates the logarithm of the given value to the given base.
+	 */
+	public static ApiExpression log(Object base, Object value) {
+		return apiCall(BuiltInFunctionDefinitions.LOG, objectToExpression(base), objectToExpression(value));
+	}
+
+	/**
+	 * Ternary conditional operator that decides which of two other expressions should be evaluated
+	 * based on a evaluated boolean condition.
+	 *
+	 * <p>e.g. ifThenElse(42 > 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,
+			objectToExpression(condition),
+			objectToExpression(ifTrue),
+			objectToExpression(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.
+	 *
+	 * <p>A range can either be index-based or name-based. Indices start at 1 and boundaries are
+	 * inclusive.
+	 *
+	 * <p>e.g. withColumns('b to 'c) or withColumns('*)
+	 */
+	public static ApiExpression withColumns(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.WITH_COLUMNS,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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.
+	 *
+	 * <p>A range can either be index-based or name-based. Indices start at 1 and boundaries are
+	 * inclusive.
+	 *
+	 * <p>e.g. withoutColumns('b to 'c) or withoutColumns('c)
+	 */
+	public static ApiExpression withoutColumns(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.WITHOUT_COLUMNS,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * A call to a function that will be looked up in a catalog.
+	 */
+	public static ApiExpression call(String functionName, Object... params) {
+		return new ApiExpression(ApiExpressionUtils.lookupCall(
+			functionName,
+			Arrays.stream(params).map(ApiExpressionUtils::objectToExpression).toArray(Expression[]::new)));
+	}
+
+	/**
+	 * A call to an inline function. For functions registered in a catalog use {@link #call(String, Object...)}.
 
 Review comment:
   `A call to an unregistered, inline function. For functions that have been registered before and are identified by a name, use ...`

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380046217
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,527 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.internal.BaseExpressions;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.ExpressionVisitor;
+import org.apache.flink.table.expressions.TimePointUnit;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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 java.util.Arrays;
+import java.util.List;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral;
+
+/**
+ * Entry point for a Java Table Expression DSL.
+ */
+@PublicEvolving
+public final class Expressions {
+
+	/**
+	 * Creates an unresolved reference to a table's column.
+	 */
+	//CHECKSTYLE.OFF: MethodName
+	public static ApiExpression $(String name) {
+		return new ApiExpression(ApiExpressionUtils.unresolvedRef(name));
+	}
+	//CHECKSTYLE.ON: MethodName
+
+	/**
+	 * Creates a SQL literal. The data type is derived from the Java class.
 
 Review comment:
   ```
   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.
   
   ```

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
dawidwys commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380103448
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,527 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.internal.BaseExpressions;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.ExpressionVisitor;
+import org.apache.flink.table.expressions.TimePointUnit;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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 java.util.Arrays;
+import java.util.List;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral;
+
+/**
+ * Entry point for a Java Table Expression DSL.
+ */
+@PublicEvolving
+public final class Expressions {
+
+	/**
+	 * Creates an unresolved reference to a table's column.
+	 */
+	//CHECKSTYLE.OFF: MethodName
+	public static ApiExpression $(String name) {
+		return new ApiExpression(ApiExpressionUtils.unresolvedRef(name));
+	}
+	//CHECKSTYLE.ON: MethodName
+
+	/**
+	 * Creates a SQL literal. The data type is derived from the Java class.
+	 */
+	public static ApiExpression lit(Object v) {
+		return new ApiExpression(valueLiteral(v));
+	}
+
+	/**
+	 * Creates a SQL literal of a given {@link DataType}.
+	 */
+	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.
+	 *
+	 * @see #withColumns(Object, Object...)
+	 * @see #withoutColumns(Object, Object...)
+	 */
+	public static ApiExpression range(String start, String end) {
+		return apiCall(BuiltInFunctionDefinitions.RANGE_TO, valueLiteral(start), valueLiteral(end));
+	}
+
+	/**
+	 * Indicates an index based range, which can be used in columns selection.
+	 *
+	 * @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 apiCall(BuiltInFunctionDefinitions.AND, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Boolean OR in three-valued logic.
+	 */
+	public static ApiExpression or(Object predicate0, Object predicate1, Object... predicates) {
+		return apiCall(BuiltInFunctionDefinitions.OR, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 <code>leftEnd >= rightStart && rightEnd >= leftStart</code>.
+	 *
+	 * <p>It evaluates: leftEnd >= rightStart && rightEnd >= leftStart
+	 *
+	 * <p>e.g. temporalOverlaps("2:55:00".toTime, 1.hour, "3:30:00".toTime, 2.hour) leads to true
+	 */
+	public static ApiExpression temporalOverlaps(
+			Object leftTimePoint,
+			Object leftTemporal,
+			Object rightTimePoint,
+			Object rightTemporal) {
+		return apiCall(
+			BuiltInFunctionDefinitions.TEMPORAL_OVERLAPS,
+			objectToExpression(leftTimePoint),
+			objectToExpression(leftTemporal),
+			objectToExpression(rightTimePoint),
+			objectToExpression(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.
+	 *
+	 * <p>For example 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,
+			objectToExpression(timestamp),
+			objectToExpression(format));
+	}
+
+	/**
+	 * Returns the (signed) number of {@link TimePointUnit} between timePoint1 and timePoint2.
+	 *
+	 * <p>For example, {@code timestampDiff(TimePointUnit.DAY, '2016-06-15'.toDate, '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),
+			objectToExpression(timePoint1),
+			objectToExpression(timePoint2));
+	}
+
+	/**
+	 * Creates an array of literals.
+	 */
+	public static ApiExpression array(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.ARRAY,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a row of expressions.
+	 */
+	public static ApiExpression row(Object head, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.ROW, Stream.concat(
+			Stream.of(head),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a map of expressions.
+	 */
+	public static ApiExpression map(Object key, Object value, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.MAP, Stream.concat(
+			Stream.of(key, value),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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() {
 
 Review comment:
   I'm fine though with changing it to constants as this was also my first thought.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380048570
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,527 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.internal.BaseExpressions;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.ExpressionVisitor;
+import org.apache.flink.table.expressions.TimePointUnit;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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 java.util.Arrays;
+import java.util.List;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral;
+
+/**
+ * Entry point for a Java Table Expression DSL.
+ */
+@PublicEvolving
+public final class Expressions {
+
+	/**
+	 * Creates an unresolved reference to a table's column.
+	 */
+	//CHECKSTYLE.OFF: MethodName
+	public static ApiExpression $(String name) {
+		return new ApiExpression(ApiExpressionUtils.unresolvedRef(name));
+	}
+	//CHECKSTYLE.ON: MethodName
+
+	/**
+	 * Creates a SQL literal. The data type is derived from the Java class.
+	 */
+	public static ApiExpression lit(Object v) {
+		return new ApiExpression(valueLiteral(v));
+	}
+
+	/**
+	 * Creates a SQL literal of a given {@link DataType}.
+	 */
+	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.
+	 *
+	 * @see #withColumns(Object, Object...)
+	 * @see #withoutColumns(Object, Object...)
+	 */
+	public static ApiExpression range(String start, String end) {
+		return apiCall(BuiltInFunctionDefinitions.RANGE_TO, valueLiteral(start), valueLiteral(end));
+	}
+
+	/**
+	 * Indicates an index based range, which can be used in columns selection.
+	 *
+	 * @see #withColumns(Object, Object...)
+	 * @see #withoutColumns(Object, Object...)
+	 */
+	public static ApiExpression range(int start, int end) {
 
 Review comment:
   Example?

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148755202 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   Hash:93f931c7ef96767509f62982e48b4853d3a74dfc Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/149801561 TriggerType:PUSH TriggerID:93f931c7ef96767509f62982e48b4853d3a74dfc
   Hash:93f931c7ef96767509f62982e48b4853d3a74dfc Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362 TriggerType:PUSH TriggerID:93f931c7ef96767509f62982e48b4853d3a74dfc
   Hash:b79c6583d86289244513a44c260c5c5d6b520f48 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363 TriggerType:PUSH TriggerID:b79c6583d86289244513a44c260c5c5d6b520f48
   Hash:b79c6583d86289244513a44c260c5c5d6b520f48 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149806101 TriggerType:PUSH TriggerID:b79c6583d86289244513a44c260c5c5d6b520f48
   Hash:91b982ff6a238b4db2bc89e81e53327d731e9378 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/149869824 TriggerType:PUSH TriggerID:91b982ff6a238b4db2bc89e81e53327d731e9378
   Hash:01b03edabdb80d7a941b10ecccb2670973d8b64a Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149885015 TriggerType:PUSH TriggerID:01b03edabdb80d7a941b10ecccb2670973d8b64a
   Hash:01b03edabdb80d7a941b10ecccb2670973d8b64a Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391 TriggerType:PUSH TriggerID:01b03edabdb80d7a941b10ecccb2670973d8b64a
   Hash:2590f92a3dbe4302baf85b67b090ffb75b05d52b Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404 TriggerType:PUSH TriggerID:2590f92a3dbe4302baf85b67b090ffb75b05d52b
   Hash:2590f92a3dbe4302baf85b67b090ffb75b05d52b Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/149970663 TriggerType:PUSH TriggerID:2590f92a3dbe4302baf85b67b090ffb75b05d52b
   Hash:3c118088adedb8b818d0371495d313e9fe038968 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/149981047 TriggerType:PUSH TriggerID:3c118088adedb8b818d0371495d313e9fe038968
   Hash:3c118088adedb8b818d0371495d313e9fe038968 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408 TriggerType:PUSH TriggerID:3c118088adedb8b818d0371495d313e9fe038968
   -->
   ## CI report:
   
   * 16d4e9e6fbeb230b8aa05976e4909299ebc929bf Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148755202) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128) 
   * 93f931c7ef96767509f62982e48b4853d3a74dfc Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/149801561) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362) 
   * b79c6583d86289244513a44c260c5c5d6b520f48 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149806101) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363) 
   * 91b982ff6a238b4db2bc89e81e53327d731e9378 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/149869824) 
   * 01b03edabdb80d7a941b10ecccb2670973d8b64a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149885015) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391) 
   * 2590f92a3dbe4302baf85b67b090ffb75b05d52b Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/149970663) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404) 
   * 3c118088adedb8b818d0371495d313e9fe038968 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/149981047) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408) 
   
   <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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r385010800
 
 

 ##########
 File path: flink-table/flink-table-api-scala/src/test/scala/org/apache/flink/table/api/ExpressionsConsistencyCheckTest.scala
 ##########
 @@ -0,0 +1,294 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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" -> "multipliedBy", // *
+    "$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 rowsInterval
+    "rows",
+
+    // users in java should use static minus()
+    "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_-
+    "minus"
+  )
+
+  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]))
+  }
+
 
 Review comment:
   nit: remove empty line

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r384996539
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java
 ##########
 @@ -0,0 +1,1286 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 <InType>  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 <OutType> 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<InType, OutType> {
+	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 org.apache.flink.table.api.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 org.apache.flink.table.api.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 multipliedBy(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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>For example:
+	 * <pre>
+	 * {@code
+	 * orders
+	 *  .groupBy($("a"))
+	 *  .select($("a"), $("b").sum().distinct().as("d"))
+	 * }
+	 * </pre>
+	 */
+	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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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()));
+	}
+
+	// scalar functions
+
+	/**
+	 * 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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>e.g. "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.
+	 *
+	 * <p>e.g. "hi".lpad(4, '??') returns "??hi",  "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.
+	 *
+	 * <p>e.g. "hi".rpad(4, '??') returns "hi??",  "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.
+	 *
+	 * <p>For example:
+	 *
+	 * <pre>
+	 * {@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)
+	 * }</pre>
+	 */
+	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).
+	 *
+	 * <p>e.g. "xxxxxtest".overlay("xxxx", 6) leads to "xxxxxxxxx"
 
 Review comment:
   update to Java

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148755202 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   Hash:93f931c7ef96767509f62982e48b4853d3a74dfc Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/149801561 TriggerType:PUSH TriggerID:93f931c7ef96767509f62982e48b4853d3a74dfc
   Hash:93f931c7ef96767509f62982e48b4853d3a74dfc Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362 TriggerType:PUSH TriggerID:93f931c7ef96767509f62982e48b4853d3a74dfc
   Hash:b79c6583d86289244513a44c260c5c5d6b520f48 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363 TriggerType:PUSH TriggerID:b79c6583d86289244513a44c260c5c5d6b520f48
   Hash:b79c6583d86289244513a44c260c5c5d6b520f48 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149806101 TriggerType:PUSH TriggerID:b79c6583d86289244513a44c260c5c5d6b520f48
   Hash:91b982ff6a238b4db2bc89e81e53327d731e9378 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/149869824 TriggerType:PUSH TriggerID:91b982ff6a238b4db2bc89e81e53327d731e9378
   Hash:01b03edabdb80d7a941b10ecccb2670973d8b64a Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149885015 TriggerType:PUSH TriggerID:01b03edabdb80d7a941b10ecccb2670973d8b64a
   Hash:01b03edabdb80d7a941b10ecccb2670973d8b64a Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391 TriggerType:PUSH TriggerID:01b03edabdb80d7a941b10ecccb2670973d8b64a
   Hash:2590f92a3dbe4302baf85b67b090ffb75b05d52b Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404 TriggerType:PUSH TriggerID:2590f92a3dbe4302baf85b67b090ffb75b05d52b
   Hash:2590f92a3dbe4302baf85b67b090ffb75b05d52b Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/149970663 TriggerType:PUSH TriggerID:2590f92a3dbe4302baf85b67b090ffb75b05d52b
   Hash:3c118088adedb8b818d0371495d313e9fe038968 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/149981047 TriggerType:PUSH TriggerID:3c118088adedb8b818d0371495d313e9fe038968
   Hash:3c118088adedb8b818d0371495d313e9fe038968 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408 TriggerType:PUSH TriggerID:3c118088adedb8b818d0371495d313e9fe038968
   -->
   ## CI report:
   
   * 16d4e9e6fbeb230b8aa05976e4909299ebc929bf Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148755202) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128) 
   * 93f931c7ef96767509f62982e48b4853d3a74dfc Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/149801561) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362) 
   * b79c6583d86289244513a44c260c5c5d6b520f48 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149806101) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363) 
   * 91b982ff6a238b4db2bc89e81e53327d731e9378 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/149869824) 
   * 01b03edabdb80d7a941b10ecccb2670973d8b64a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149885015) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391) 
   * 2590f92a3dbe4302baf85b67b090ffb75b05d52b Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/149970663) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404) 
   * 3c118088adedb8b818d0371495d313e9fe038968 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/149981047) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408) 
   
   <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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
dawidwys commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380205913
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/LookupCallResolver.java
 ##########
 @@ -63,6 +64,17 @@ public Expression visit(UnresolvedCallExpression unresolvedCall) {
 			.collect(Collectors.toList());
 	}
 
+	@Override
+	public Expression visitNonApiExpression(Expression other) {
+		// LookupCallResolver might be called outside of ExpressionResolver, thus we need to additionally
 
 Review comment:
   I think it's not that simple. We call the `LookupCallResolver` externally primarily to separate functions into scalar and aggregating functions so that we can split expressions into ones that go into groupBy clause and those that go into select clause. Therefore before that step we might not have an input we can use to fully resolve expressions using `ExpressionResolver`.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148755202",
       "triggerID" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149801561",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149806101",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149869824",
       "triggerID" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149885015",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149970663",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149981047",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150432772",
       "triggerID" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150669044",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5643",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d510ce90e5cd8a13256c84c855562c403091a14",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150861986",
       "triggerID" : "8d510ce90e5cd8a13256c84c855562c403091a14",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4bcbe8c726aa4cda0ac0214e5997c9c0d38640be Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/150669044) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5643) 
   * 8d510ce90e5cd8a13256c84c855562c403091a14 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/150861986) 
   
   <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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
dawidwys commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380196151
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,527 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.internal.BaseExpressions;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.ExpressionVisitor;
+import org.apache.flink.table.expressions.TimePointUnit;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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 java.util.Arrays;
+import java.util.List;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral;
+
+/**
+ * Entry point for a Java Table Expression DSL.
+ */
+@PublicEvolving
+public final class Expressions {
+
+	/**
+	 * Creates an unresolved reference to a table's column.
+	 */
+	//CHECKSTYLE.OFF: MethodName
+	public static ApiExpression $(String name) {
+		return new ApiExpression(ApiExpressionUtils.unresolvedRef(name));
+	}
+	//CHECKSTYLE.ON: MethodName
+
+	/**
+	 * Creates a SQL literal. The data type is derived from the Java class.
+	 */
+	public static ApiExpression lit(Object v) {
+		return new ApiExpression(valueLiteral(v));
+	}
+
+	/**
+	 * Creates a SQL literal of a given {@link DataType}.
+	 */
+	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.
+	 *
+	 * @see #withColumns(Object, Object...)
+	 * @see #withoutColumns(Object, Object...)
+	 */
+	public static ApiExpression range(String start, String end) {
+		return apiCall(BuiltInFunctionDefinitions.RANGE_TO, valueLiteral(start), valueLiteral(end));
+	}
+
+	/**
+	 * Indicates an index based range, which can be used in columns selection.
+	 *
+	 * @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 apiCall(BuiltInFunctionDefinitions.AND, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Boolean OR in three-valued logic.
+	 */
+	public static ApiExpression or(Object predicate0, Object predicate1, Object... predicates) {
+		return apiCall(BuiltInFunctionDefinitions.OR, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 <code>leftEnd >= rightStart && rightEnd >= leftStart</code>.
+	 *
+	 * <p>It evaluates: leftEnd >= rightStart && rightEnd >= leftStart
+	 *
+	 * <p>e.g. temporalOverlaps("2:55:00".toTime, 1.hour, "3:30:00".toTime, 2.hour) leads to true
+	 */
+	public static ApiExpression temporalOverlaps(
+			Object leftTimePoint,
+			Object leftTemporal,
+			Object rightTimePoint,
+			Object rightTemporal) {
+		return apiCall(
+			BuiltInFunctionDefinitions.TEMPORAL_OVERLAPS,
+			objectToExpression(leftTimePoint),
+			objectToExpression(leftTemporal),
+			objectToExpression(rightTimePoint),
+			objectToExpression(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.
+	 *
+	 * <p>For example 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,
+			objectToExpression(timestamp),
+			objectToExpression(format));
+	}
+
+	/**
+	 * Returns the (signed) number of {@link TimePointUnit} between timePoint1 and timePoint2.
+	 *
+	 * <p>For example, {@code timestampDiff(TimePointUnit.DAY, '2016-06-15'.toDate, '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),
+			objectToExpression(timePoint1),
+			objectToExpression(timePoint2));
+	}
+
+	/**
+	 * Creates an array of literals.
+	 */
+	public static ApiExpression array(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.ARRAY,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a row of expressions.
+	 */
+	public static ApiExpression row(Object head, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.ROW, Stream.concat(
+			Stream.of(head),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a map of expressions.
+	 */
+	public static ApiExpression map(Object key, Object value, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.MAP, Stream.concat(
+			Stream.of(key, value),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 apiCall(
+			BuiltInFunctionDefinitions.CONCAT,
+			Stream.concat(
+				Stream.of(string),
+				Stream.of(strings)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Calculates the arc tangent of a given coordinate.
+	 */
+	public static ApiExpression atan2(Object y, Object x) {
+		return apiCall(BuiltInFunctionDefinitions.ATAN2, objectToExpression(y), objectToExpression(x));
+	}
+
+	/**
+	 * Returns the string that results from concatenating the arguments and separator.
+	 * Returns NULL If the separator is NULL.
+	 *
+	 * <p>Note: this user-public static ApiExpressionined 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 apiCall(BuiltInFunctionDefinitions.CONCAT_WS, Stream.concat(
+			Stream.of(separator, string),
+			Stream.of(strings)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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.
+	 *
+	 * <p>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, objectToExpression(value));
+	}
+
+	/**
+	 * Calculates the logarithm of the given value to the given base.
+	 */
+	public static ApiExpression log(Object base, Object value) {
+		return apiCall(BuiltInFunctionDefinitions.LOG, objectToExpression(base), objectToExpression(value));
+	}
+
+	/**
+	 * Ternary conditional operator that decides which of two other expressions should be evaluated
+	 * based on a evaluated boolean condition.
+	 *
+	 * <p>e.g. ifThenElse(42 > 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,
+			objectToExpression(condition),
+			objectToExpression(ifTrue),
+			objectToExpression(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.
+	 *
+	 * <p>A range can either be index-based or name-based. Indices start at 1 and boundaries are
+	 * inclusive.
+	 *
+	 * <p>e.g. withColumns('b to 'c) or withColumns('*)
+	 */
+	public static ApiExpression withColumns(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.WITH_COLUMNS,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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.
+	 *
+	 * <p>A range can either be index-based or name-based. Indices start at 1 and boundaries are
+	 * inclusive.
+	 *
+	 * <p>e.g. withoutColumns('b to 'c) or withoutColumns('c)
+	 */
+	public static ApiExpression withoutColumns(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.WITHOUT_COLUMNS,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * A call to a function that will be looked up in a catalog.
+	 */
+	public static ApiExpression call(String functionName, Object... params) {
+		return new ApiExpression(ApiExpressionUtils.lookupCall(
+			functionName,
+			Arrays.stream(params).map(ApiExpressionUtils::objectToExpression).toArray(Expression[]::new)));
+	}
+
+	/**
+	 * A call to an inline function. For functions registered in a catalog use {@link #call(String, Object...)}.
+	 */
+	public static ApiExpression call(FunctionDefinition scalarFunction, Object... params) {
+		return apiCall(
+			scalarFunction,
+			Arrays.stream(params).map(ApiExpressionUtils::objectToExpression).toArray(Expression[]::new));
+	}
+
+	private static ApiExpression apiCall(FunctionDefinition functionDefinition, Expression... args) {
+		return new ApiExpression(new UnresolvedCallExpression(functionDefinition, Arrays.asList(args)));
+	}
+
+	/**
+	 * Java API class that gives access to expressions operations.
+	 */
+	public static final class ApiExpression extends BaseExpressions<Object, ApiExpression> implements Expression {
 
 Review comment:
   Ah, misread that, sorry. Will do

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148755202",
       "triggerID" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149801561",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149806101",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149869824",
       "triggerID" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149885015",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149970663",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149981047",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150432772",
       "triggerID" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150669044",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5643",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d510ce90e5cd8a13256c84c855562c403091a14",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150861986",
       "triggerID" : "8d510ce90e5cd8a13256c84c855562c403091a14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3786dab008166eff376617dabda4dedeba7ca961",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3786dab008166eff376617dabda4dedeba7ca961",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0e3ee0c3b8b58815b2f771d8b4320e7483c41159",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0e3ee0c3b8b58815b2f771d8b4320e7483c41159",
       "triggerType" : "PUSH"
     }, {
       "hash" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5706",
       "triggerID" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150885626",
       "triggerID" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f9502902a9c1a3203a22fdeb91f47551c3e862d1",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150974862",
       "triggerID" : "f9502902a9c1a3203a22fdeb91f47551c3e862d1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f9502902a9c1a3203a22fdeb91f47551c3e862d1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5721",
       "triggerID" : "f9502902a9c1a3203a22fdeb91f47551c3e862d1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2264ba7fa5063309c8cb58cc4c32694b90164f0d",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150998989",
       "triggerID" : "2264ba7fa5063309c8cb58cc4c32694b90164f0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2264ba7fa5063309c8cb58cc4c32694b90164f0d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5728",
       "triggerID" : "2264ba7fa5063309c8cb58cc4c32694b90164f0d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3786dab008166eff376617dabda4dedeba7ca961 UNKNOWN
   * 0e3ee0c3b8b58815b2f771d8b4320e7483c41159 UNKNOWN
   * 2264ba7fa5063309c8cb58cc4c32694b90164f0d Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/150998989) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5728) 
   
   <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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148755202 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   Hash:93f931c7ef96767509f62982e48b4853d3a74dfc Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/149801561 TriggerType:PUSH TriggerID:93f931c7ef96767509f62982e48b4853d3a74dfc
   Hash:93f931c7ef96767509f62982e48b4853d3a74dfc Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362 TriggerType:PUSH TriggerID:93f931c7ef96767509f62982e48b4853d3a74dfc
   Hash:b79c6583d86289244513a44c260c5c5d6b520f48 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363 TriggerType:PUSH TriggerID:b79c6583d86289244513a44c260c5c5d6b520f48
   Hash:b79c6583d86289244513a44c260c5c5d6b520f48 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149806101 TriggerType:PUSH TriggerID:b79c6583d86289244513a44c260c5c5d6b520f48
   Hash:91b982ff6a238b4db2bc89e81e53327d731e9378 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/149869824 TriggerType:PUSH TriggerID:91b982ff6a238b4db2bc89e81e53327d731e9378
   -->
   ## CI report:
   
   * 16d4e9e6fbeb230b8aa05976e4909299ebc929bf Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148755202) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128) 
   * 93f931c7ef96767509f62982e48b4853d3a74dfc Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/149801561) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362) 
   * b79c6583d86289244513a44c260c5c5d6b520f48 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149806101) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363) 
   * 91b982ff6a238b4db2bc89e81e53327d731e9378 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/149869824) 
   
   <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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148755202",
       "triggerID" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149801561",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149806101",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149869824",
       "triggerID" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149885015",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149970663",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149981047",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150432772",
       "triggerID" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0cd224f4da41acc6973bbd4fb27f11e201832e6e Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/150432772) 
   * 4bcbe8c726aa4cda0ac0214e5997c9c0d38640be 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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380080780
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/UnresolvedCallExpression.java
 ##########
 @@ -61,8 +60,10 @@ public UnresolvedCallExpression(
 			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 = Preconditions.checkNotNull(args)
 
 Review comment:
   Can't we do this earlier? In `ApiExpressionUtils`?

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380068156
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,527 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.internal.BaseExpressions;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.ExpressionVisitor;
+import org.apache.flink.table.expressions.TimePointUnit;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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 java.util.Arrays;
+import java.util.List;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral;
+
+/**
+ * Entry point for a Java Table Expression DSL.
+ */
+@PublicEvolving
+public final class Expressions {
+
+	/**
+	 * Creates an unresolved reference to a table's column.
+	 */
+	//CHECKSTYLE.OFF: MethodName
+	public static ApiExpression $(String name) {
+		return new ApiExpression(ApiExpressionUtils.unresolvedRef(name));
+	}
+	//CHECKSTYLE.ON: MethodName
+
+	/**
+	 * Creates a SQL literal. The data type is derived from the Java class.
+	 */
+	public static ApiExpression lit(Object v) {
+		return new ApiExpression(valueLiteral(v));
+	}
+
+	/**
+	 * Creates a SQL literal of a given {@link DataType}.
+	 */
+	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.
+	 *
+	 * @see #withColumns(Object, Object...)
+	 * @see #withoutColumns(Object, Object...)
+	 */
+	public static ApiExpression range(String start, String end) {
+		return apiCall(BuiltInFunctionDefinitions.RANGE_TO, valueLiteral(start), valueLiteral(end));
+	}
+
+	/**
+	 * Indicates an index based range, which can be used in columns selection.
+	 *
+	 * @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 apiCall(BuiltInFunctionDefinitions.AND, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Boolean OR in three-valued logic.
+	 */
+	public static ApiExpression or(Object predicate0, Object predicate1, Object... predicates) {
+		return apiCall(BuiltInFunctionDefinitions.OR, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 <code>leftEnd >= rightStart && rightEnd >= leftStart</code>.
+	 *
+	 * <p>It evaluates: leftEnd >= rightStart && rightEnd >= leftStart
+	 *
+	 * <p>e.g. temporalOverlaps("2:55:00".toTime, 1.hour, "3:30:00".toTime, 2.hour) leads to true
+	 */
+	public static ApiExpression temporalOverlaps(
+			Object leftTimePoint,
+			Object leftTemporal,
+			Object rightTimePoint,
+			Object rightTemporal) {
+		return apiCall(
+			BuiltInFunctionDefinitions.TEMPORAL_OVERLAPS,
+			objectToExpression(leftTimePoint),
+			objectToExpression(leftTemporal),
+			objectToExpression(rightTimePoint),
+			objectToExpression(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.
+	 *
+	 * <p>For example 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,
+			objectToExpression(timestamp),
+			objectToExpression(format));
+	}
+
+	/**
+	 * Returns the (signed) number of {@link TimePointUnit} between timePoint1 and timePoint2.
+	 *
+	 * <p>For example, {@code timestampDiff(TimePointUnit.DAY, '2016-06-15'.toDate, '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),
+			objectToExpression(timePoint1),
+			objectToExpression(timePoint2));
+	}
+
+	/**
+	 * Creates an array of literals.
+	 */
+	public static ApiExpression array(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.ARRAY,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a row of expressions.
+	 */
+	public static ApiExpression row(Object head, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.ROW, Stream.concat(
+			Stream.of(head),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a map of expressions.
+	 */
+	public static ApiExpression map(Object key, Object value, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.MAP, Stream.concat(
+			Stream.of(key, value),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 apiCall(
+			BuiltInFunctionDefinitions.CONCAT,
+			Stream.concat(
+				Stream.of(string),
+				Stream.of(strings)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Calculates the arc tangent of a given coordinate.
+	 */
+	public static ApiExpression atan2(Object y, Object x) {
+		return apiCall(BuiltInFunctionDefinitions.ATAN2, objectToExpression(y), objectToExpression(x));
+	}
+
+	/**
+	 * Returns the string that results from concatenating the arguments and separator.
+	 * Returns NULL If the separator is NULL.
+	 *
+	 * <p>Note: this user-public static ApiExpressionined 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 apiCall(BuiltInFunctionDefinitions.CONCAT_WS, Stream.concat(
+			Stream.of(separator, string),
+			Stream.of(strings)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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.
+	 *
+	 * <p>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
 
 Review comment:
   nit: indent

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380069485
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,527 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.internal.BaseExpressions;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.ExpressionVisitor;
+import org.apache.flink.table.expressions.TimePointUnit;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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 java.util.Arrays;
+import java.util.List;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral;
+
+/**
+ * Entry point for a Java Table Expression DSL.
+ */
+@PublicEvolving
+public final class Expressions {
+
+	/**
+	 * Creates an unresolved reference to a table's column.
+	 */
+	//CHECKSTYLE.OFF: MethodName
+	public static ApiExpression $(String name) {
+		return new ApiExpression(ApiExpressionUtils.unresolvedRef(name));
+	}
+	//CHECKSTYLE.ON: MethodName
+
+	/**
+	 * Creates a SQL literal. The data type is derived from the Java class.
+	 */
+	public static ApiExpression lit(Object v) {
+		return new ApiExpression(valueLiteral(v));
+	}
+
+	/**
+	 * Creates a SQL literal of a given {@link DataType}.
+	 */
+	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.
+	 *
+	 * @see #withColumns(Object, Object...)
+	 * @see #withoutColumns(Object, Object...)
+	 */
+	public static ApiExpression range(String start, String end) {
+		return apiCall(BuiltInFunctionDefinitions.RANGE_TO, valueLiteral(start), valueLiteral(end));
+	}
+
+	/**
+	 * Indicates an index based range, which can be used in columns selection.
+	 *
+	 * @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 apiCall(BuiltInFunctionDefinitions.AND, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Boolean OR in three-valued logic.
+	 */
+	public static ApiExpression or(Object predicate0, Object predicate1, Object... predicates) {
+		return apiCall(BuiltInFunctionDefinitions.OR, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 <code>leftEnd >= rightStart && rightEnd >= leftStart</code>.
+	 *
+	 * <p>It evaluates: leftEnd >= rightStart && rightEnd >= leftStart
+	 *
+	 * <p>e.g. temporalOverlaps("2:55:00".toTime, 1.hour, "3:30:00".toTime, 2.hour) leads to true
+	 */
+	public static ApiExpression temporalOverlaps(
+			Object leftTimePoint,
+			Object leftTemporal,
+			Object rightTimePoint,
+			Object rightTemporal) {
+		return apiCall(
+			BuiltInFunctionDefinitions.TEMPORAL_OVERLAPS,
+			objectToExpression(leftTimePoint),
+			objectToExpression(leftTemporal),
+			objectToExpression(rightTimePoint),
+			objectToExpression(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.
+	 *
+	 * <p>For example 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,
+			objectToExpression(timestamp),
+			objectToExpression(format));
+	}
+
+	/**
+	 * Returns the (signed) number of {@link TimePointUnit} between timePoint1 and timePoint2.
+	 *
+	 * <p>For example, {@code timestampDiff(TimePointUnit.DAY, '2016-06-15'.toDate, '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),
+			objectToExpression(timePoint1),
+			objectToExpression(timePoint2));
+	}
+
+	/**
+	 * Creates an array of literals.
+	 */
+	public static ApiExpression array(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.ARRAY,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a row of expressions.
+	 */
+	public static ApiExpression row(Object head, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.ROW, Stream.concat(
+			Stream.of(head),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a map of expressions.
+	 */
+	public static ApiExpression map(Object key, Object value, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.MAP, Stream.concat(
+			Stream.of(key, value),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 apiCall(
+			BuiltInFunctionDefinitions.CONCAT,
+			Stream.concat(
+				Stream.of(string),
+				Stream.of(strings)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Calculates the arc tangent of a given coordinate.
+	 */
+	public static ApiExpression atan2(Object y, Object x) {
+		return apiCall(BuiltInFunctionDefinitions.ATAN2, objectToExpression(y), objectToExpression(x));
+	}
+
+	/**
+	 * Returns the string that results from concatenating the arguments and separator.
+	 * Returns NULL If the separator is NULL.
+	 *
+	 * <p>Note: this user-public static ApiExpressionined 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 apiCall(BuiltInFunctionDefinitions.CONCAT_WS, Stream.concat(
+			Stream.of(separator, string),
+			Stream.of(strings)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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.
+	 *
+	 * <p>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, objectToExpression(value));
+	}
+
+	/**
+	 * Calculates the logarithm of the given value to the given base.
+	 */
+	public static ApiExpression log(Object base, Object value) {
+		return apiCall(BuiltInFunctionDefinitions.LOG, objectToExpression(base), objectToExpression(value));
+	}
+
+	/**
+	 * Ternary conditional operator that decides which of two other expressions should be evaluated
+	 * based on a evaluated boolean condition.
+	 *
+	 * <p>e.g. ifThenElse(42 > 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,
+			objectToExpression(condition),
+			objectToExpression(ifTrue),
+			objectToExpression(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.
+	 *
+	 * <p>A range can either be index-based or name-based. Indices start at 1 and boundaries are
+	 * inclusive.
+	 *
+	 * <p>e.g. withColumns('b to 'c) or withColumns('*)
+	 */
+	public static ApiExpression withColumns(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.WITH_COLUMNS,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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.
+	 *
+	 * <p>A range can either be index-based or name-based. Indices start at 1 and boundaries are
+	 * inclusive.
+	 *
+	 * <p>e.g. withoutColumns('b to 'c) or withoutColumns('c)
+	 */
+	public static ApiExpression withoutColumns(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.WITHOUT_COLUMNS,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * A call to a function that will be looked up in a catalog.
+	 */
+	public static ApiExpression call(String functionName, Object... params) {
+		return new ApiExpression(ApiExpressionUtils.lookupCall(
+			functionName,
+			Arrays.stream(params).map(ApiExpressionUtils::objectToExpression).toArray(Expression[]::new)));
 
 Review comment:
   nit: let's have a couple of helper methods here to avoid the stream magic and keep the implementation for these method minimal.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r385602063
 
 

 ##########
 File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/UserDefinedFunctionConverter.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.functions.FunctionIdentifier;
+import org.apache.flink.table.functions.UserDefinedFunction;
+import org.apache.flink.table.planner.functions.bridging.BridgingSqlFunction;
+
+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
+ * that use the new type stack.
 
 Review comment:
   nit: At some point `new type stack` is not `new` anymore I would rather just annotate the legacy.

----------------------------------------------------------------
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 closed pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
dawidwys closed pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081
 
 
   

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r384985161
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,562 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.catalog.DataTypeFactory;
+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.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;
+
+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()}
+ *
+ * <p>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.
+ *
+ * <p>For fluent definition of expressions and easier readability, we recommend to add a
+ * star import to the methods of this class:
+ *
+ * <pre>
+ * import static org.apache.flink.table.api.Expressions.*;
+ * </pre>
+ *
+ * <p>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.
+	 *
+	 * <p>Example
 
 Review comment:
   nit: `Example:`

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
dawidwys commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r385227403
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/LookupCallExpression.java
 ##########
 @@ -43,7 +42,7 @@
 
 	LookupCallExpression(String unresolvedFunction, List<Expression> args) {
 		this.unresolvedName = Preconditions.checkNotNull(unresolvedFunction);
-		this.args = Collections.unmodifiableList(new ArrayList<>(Preconditions.checkNotNull(args)));
+		this.args = Collections.unmodifiableList(Preconditions.checkNotNull(args));
 
 Review comment:
   Honestly I don't know if it makes sense to be so defensive here. 
   
   Usually the arguments are constructed either with `Arrays.asList` or via stream API which produce unmodifiable lists anyway. I was tempted to remove even the `unmodifiableList`.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r385603661
 
 

 ##########
 File path: flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/table/FunctionITCase.java
 ##########
 @@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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.Test;
+
+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 catalog and system in stream table environment.
+ */
+public class FunctionITCase extends StreamingTestBase {
+
+	@Test
+	public void testPrimitiveScalarFunction() throws Exception {
+		final List<Row> sourceData = Arrays.asList(
+			Row.of(1, 1L, 1L),
+			Row.of(2, 2L, 1L),
+			Row.of(3, 3L, 1L)
+		);
+
+		final List<Row> 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')");
 
 Review comment:
   I think if we would test a decimal here, the comments I mentioned above would be visible.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r384996242
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java
 ##########
 @@ -0,0 +1,1286 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 <InType>  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 <OutType> 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<InType, OutType> {
+	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 org.apache.flink.table.api.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 org.apache.flink.table.api.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 multipliedBy(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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>For example:
+	 * <pre>
+	 * {@code
+	 * orders
+	 *  .groupBy($("a"))
+	 *  .select($("a"), $("b").sum().distinct().as("d"))
+	 * }
+	 * </pre>
+	 */
+	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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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()));
+	}
+
+	// scalar functions
+
+	/**
+	 * 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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>e.g. "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.
+	 *
+	 * <p>e.g. "hi".lpad(4, '??') returns "??hi",  "hi".lpad(1, '??') returns "h"
 
 Review comment:
   update to Java? `'??'` is invalid anyway

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148755202",
       "triggerID" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149801561",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149806101",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149869824",
       "triggerID" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149885015",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149970663",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149981047",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150432772",
       "triggerID" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3c118088adedb8b818d0371495d313e9fe038968 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/149981047) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408) 
   * 0cd224f4da41acc6973bbd4fb27f11e201832e6e Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/150432772) 
   
   <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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380038479
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,527 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.internal.BaseExpressions;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.ExpressionVisitor;
+import org.apache.flink.table.expressions.TimePointUnit;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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 java.util.Arrays;
+import java.util.List;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral;
+
+/**
+ * Entry point for a Java Table Expression DSL.
+ */
+@PublicEvolving
+public final class Expressions {
+
+	/**
+	 * Creates an unresolved reference to a table's column.
+	 */
+	//CHECKSTYLE.OFF: MethodName
+	public static ApiExpression $(String name) {
+		return new ApiExpression(ApiExpressionUtils.unresolvedRef(name));
+	}
+	//CHECKSTYLE.ON: MethodName
+
+	/**
+	 * Creates a SQL literal. The data type is derived from the Java class.
+	 */
+	public static ApiExpression lit(Object v) {
+		return new ApiExpression(valueLiteral(v));
+	}
+
+	/**
+	 * Creates a SQL literal of a given {@link DataType}.
+	 */
+	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.
+	 *
+	 * @see #withColumns(Object, Object...)
+	 * @see #withoutColumns(Object, Object...)
+	 */
+	public static ApiExpression range(String start, String end) {
+		return apiCall(BuiltInFunctionDefinitions.RANGE_TO, valueLiteral(start), valueLiteral(end));
+	}
+
+	/**
+	 * Indicates an index based range, which can be used in columns selection.
+	 *
+	 * @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 apiCall(BuiltInFunctionDefinitions.AND, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Boolean OR in three-valued logic.
+	 */
+	public static ApiExpression or(Object predicate0, Object predicate1, Object... predicates) {
+		return apiCall(BuiltInFunctionDefinitions.OR, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 <code>leftEnd >= rightStart && rightEnd >= leftStart</code>.
+	 *
+	 * <p>It evaluates: leftEnd >= rightStart && rightEnd >= leftStart
+	 *
+	 * <p>e.g. temporalOverlaps("2:55:00".toTime, 1.hour, "3:30:00".toTime, 2.hour) leads to true
+	 */
+	public static ApiExpression temporalOverlaps(
+			Object leftTimePoint,
+			Object leftTemporal,
+			Object rightTimePoint,
+			Object rightTemporal) {
+		return apiCall(
+			BuiltInFunctionDefinitions.TEMPORAL_OVERLAPS,
+			objectToExpression(leftTimePoint),
+			objectToExpression(leftTemporal),
+			objectToExpression(rightTimePoint),
+			objectToExpression(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.
+	 *
+	 * <p>For example 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,
+			objectToExpression(timestamp),
+			objectToExpression(format));
+	}
+
+	/**
+	 * Returns the (signed) number of {@link TimePointUnit} between timePoint1 and timePoint2.
+	 *
+	 * <p>For example, {@code timestampDiff(TimePointUnit.DAY, '2016-06-15'.toDate, '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),
+			objectToExpression(timePoint1),
+			objectToExpression(timePoint2));
+	}
+
+	/**
+	 * Creates an array of literals.
+	 */
+	public static ApiExpression array(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.ARRAY,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a row of expressions.
+	 */
+	public static ApiExpression row(Object head, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.ROW, Stream.concat(
+			Stream.of(head),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a map of expressions.
+	 */
+	public static ApiExpression map(Object key, Object value, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.MAP, Stream.concat(
+			Stream.of(key, value),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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() {
 
 Review comment:
   Shouldn't PI and E also be constant fields? Should we deprecate those two method?

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
dawidwys commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380120928
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,527 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.internal.BaseExpressions;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.ExpressionVisitor;
+import org.apache.flink.table.expressions.TimePointUnit;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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 java.util.Arrays;
+import java.util.List;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral;
+
+/**
+ * Entry point for a Java Table Expression DSL.
+ */
+@PublicEvolving
+public final class Expressions {
+
+	/**
+	 * Creates an unresolved reference to a table's column.
+	 */
+	//CHECKSTYLE.OFF: MethodName
+	public static ApiExpression $(String name) {
+		return new ApiExpression(ApiExpressionUtils.unresolvedRef(name));
+	}
+	//CHECKSTYLE.ON: MethodName
+
+	/**
+	 * Creates a SQL literal. The data type is derived from the Java class.
+	 */
+	public static ApiExpression lit(Object v) {
+		return new ApiExpression(valueLiteral(v));
+	}
+
+	/**
+	 * Creates a SQL literal of a given {@link DataType}.
+	 */
+	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.
+	 *
+	 * @see #withColumns(Object, Object...)
+	 * @see #withoutColumns(Object, Object...)
+	 */
+	public static ApiExpression range(String start, String end) {
+		return apiCall(BuiltInFunctionDefinitions.RANGE_TO, valueLiteral(start), valueLiteral(end));
+	}
+
+	/**
+	 * Indicates an index based range, which can be used in columns selection.
+	 *
+	 * @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 apiCall(BuiltInFunctionDefinitions.AND, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Boolean OR in three-valued logic.
+	 */
+	public static ApiExpression or(Object predicate0, Object predicate1, Object... predicates) {
+		return apiCall(BuiltInFunctionDefinitions.OR, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 <code>leftEnd >= rightStart && rightEnd >= leftStart</code>.
+	 *
+	 * <p>It evaluates: leftEnd >= rightStart && rightEnd >= leftStart
+	 *
+	 * <p>e.g. temporalOverlaps("2:55:00".toTime, 1.hour, "3:30:00".toTime, 2.hour) leads to true
+	 */
+	public static ApiExpression temporalOverlaps(
+			Object leftTimePoint,
+			Object leftTemporal,
+			Object rightTimePoint,
+			Object rightTemporal) {
+		return apiCall(
+			BuiltInFunctionDefinitions.TEMPORAL_OVERLAPS,
+			objectToExpression(leftTimePoint),
+			objectToExpression(leftTemporal),
+			objectToExpression(rightTimePoint),
+			objectToExpression(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.
+	 *
+	 * <p>For example 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,
+			objectToExpression(timestamp),
+			objectToExpression(format));
+	}
+
+	/**
+	 * Returns the (signed) number of {@link TimePointUnit} between timePoint1 and timePoint2.
+	 *
+	 * <p>For example, {@code timestampDiff(TimePointUnit.DAY, '2016-06-15'.toDate, '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),
+			objectToExpression(timePoint1),
+			objectToExpression(timePoint2));
+	}
+
+	/**
+	 * Creates an array of literals.
+	 */
+	public static ApiExpression array(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.ARRAY,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a row of expressions.
+	 */
+	public static ApiExpression row(Object head, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.ROW, Stream.concat(
+			Stream.of(head),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a map of expressions.
+	 */
+	public static ApiExpression map(Object key, Object value, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.MAP, Stream.concat(
+			Stream.of(key, value),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 apiCall(
+			BuiltInFunctionDefinitions.CONCAT,
+			Stream.concat(
+				Stream.of(string),
+				Stream.of(strings)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Calculates the arc tangent of a given coordinate.
+	 */
+	public static ApiExpression atan2(Object y, Object x) {
+		return apiCall(BuiltInFunctionDefinitions.ATAN2, objectToExpression(y), objectToExpression(x));
+	}
+
+	/**
+	 * Returns the string that results from concatenating the arguments and separator.
+	 * Returns NULL If the separator is NULL.
+	 *
+	 * <p>Note: this user-public static ApiExpressionined 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 apiCall(BuiltInFunctionDefinitions.CONCAT_WS, Stream.concat(
+			Stream.of(separator, string),
+			Stream.of(strings)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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.
+	 *
+	 * <p>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, objectToExpression(value));
+	}
+
+	/**
+	 * Calculates the logarithm of the given value to the given base.
+	 */
+	public static ApiExpression log(Object base, Object value) {
+		return apiCall(BuiltInFunctionDefinitions.LOG, objectToExpression(base), objectToExpression(value));
+	}
+
+	/**
+	 * Ternary conditional operator that decides which of two other expressions should be evaluated
+	 * based on a evaluated boolean condition.
+	 *
+	 * <p>e.g. ifThenElse(42 > 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,
+			objectToExpression(condition),
+			objectToExpression(ifTrue),
+			objectToExpression(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.
+	 *
+	 * <p>A range can either be index-based or name-based. Indices start at 1 and boundaries are
+	 * inclusive.
+	 *
+	 * <p>e.g. withColumns('b to 'c) or withColumns('*)
+	 */
+	public static ApiExpression withColumns(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.WITH_COLUMNS,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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.
+	 *
+	 * <p>A range can either be index-based or name-based. Indices start at 1 and boundaries are
+	 * inclusive.
+	 *
+	 * <p>e.g. withoutColumns('b to 'c) or withoutColumns('c)
+	 */
+	public static ApiExpression withoutColumns(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.WITHOUT_COLUMNS,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * A call to a function that will be looked up in a catalog.
+	 */
+	public static ApiExpression call(String functionName, Object... params) {
+		return new ApiExpression(ApiExpressionUtils.lookupCall(
+			functionName,
+			Arrays.stream(params).map(ApiExpressionUtils::objectToExpression).toArray(Expression[]::new)));
+	}
+
+	/**
+	 * A call to an inline function. For functions registered in a catalog use {@link #call(String, Object...)}.
+	 */
+	public static ApiExpression call(FunctionDefinition scalarFunction, Object... params) {
+		return apiCall(
+			scalarFunction,
+			Arrays.stream(params).map(ApiExpressionUtils::objectToExpression).toArray(Expression[]::new));
+	}
+
+	private static ApiExpression apiCall(FunctionDefinition functionDefinition, Expression... args) {
+		return new ApiExpression(new UnresolvedCallExpression(functionDefinition, Arrays.asList(args)));
+	}
+
+	/**
+	 * Java API class that gives access to expressions operations.
+	 */
+	public static final class ApiExpression extends BaseExpressions<Object, ApiExpression> implements Expression {
 
 Review comment:
   Unfortunately this will not work. The Java's Expression DSL return the `ApiExpression` e.g.
   ```
   class Expressions {
      ApiExpression $(String name);
   }
   ```
   thus the `ApiExpression` must be public so that users can access methods defined in `BaseExpression`. We could return `BaseExpressions` instead, but I don't think it would be any better.
   
   Moreover I think the original idea was that if we have some non-static Java specific methods we can put them in `ApiExpression`.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
dawidwys commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380115348
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,527 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.internal.BaseExpressions;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.ExpressionVisitor;
+import org.apache.flink.table.expressions.TimePointUnit;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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 java.util.Arrays;
+import java.util.List;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral;
+
+/**
+ * Entry point for a Java Table Expression DSL.
+ */
+@PublicEvolving
+public final class Expressions {
+
+	/**
+	 * Creates an unresolved reference to a table's column.
+	 */
+	//CHECKSTYLE.OFF: MethodName
+	public static ApiExpression $(String name) {
+		return new ApiExpression(ApiExpressionUtils.unresolvedRef(name));
+	}
+	//CHECKSTYLE.ON: MethodName
+
+	/**
+	 * Creates a SQL literal. The data type is derived from the Java class.
+	 */
+	public static ApiExpression lit(Object v) {
+		return new ApiExpression(valueLiteral(v));
+	}
+
+	/**
+	 * Creates a SQL literal of a given {@link DataType}.
+	 */
+	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.
+	 *
+	 * @see #withColumns(Object, Object...)
+	 * @see #withoutColumns(Object, Object...)
+	 */
+	public static ApiExpression range(String start, String end) {
+		return apiCall(BuiltInFunctionDefinitions.RANGE_TO, valueLiteral(start), valueLiteral(end));
+	}
+
+	/**
+	 * Indicates an index based range, which can be used in columns selection.
+	 *
+	 * @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 apiCall(BuiltInFunctionDefinitions.AND, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Boolean OR in three-valued logic.
+	 */
+	public static ApiExpression or(Object predicate0, Object predicate1, Object... predicates) {
+		return apiCall(BuiltInFunctionDefinitions.OR, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 <code>leftEnd >= rightStart && rightEnd >= leftStart</code>.
+	 *
+	 * <p>It evaluates: leftEnd >= rightStart && rightEnd >= leftStart
+	 *
+	 * <p>e.g. temporalOverlaps("2:55:00".toTime, 1.hour, "3:30:00".toTime, 2.hour) leads to true
+	 */
+	public static ApiExpression temporalOverlaps(
+			Object leftTimePoint,
+			Object leftTemporal,
+			Object rightTimePoint,
+			Object rightTemporal) {
+		return apiCall(
+			BuiltInFunctionDefinitions.TEMPORAL_OVERLAPS,
+			objectToExpression(leftTimePoint),
+			objectToExpression(leftTemporal),
+			objectToExpression(rightTimePoint),
+			objectToExpression(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.
+	 *
+	 * <p>For example 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,
+			objectToExpression(timestamp),
+			objectToExpression(format));
+	}
+
+	/**
+	 * Returns the (signed) number of {@link TimePointUnit} between timePoint1 and timePoint2.
+	 *
+	 * <p>For example, {@code timestampDiff(TimePointUnit.DAY, '2016-06-15'.toDate, '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),
+			objectToExpression(timePoint1),
+			objectToExpression(timePoint2));
+	}
+
+	/**
+	 * Creates an array of literals.
+	 */
+	public static ApiExpression array(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.ARRAY,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a row of expressions.
+	 */
+	public static ApiExpression row(Object head, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.ROW, Stream.concat(
+			Stream.of(head),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a map of expressions.
+	 */
+	public static ApiExpression map(Object key, Object value, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.MAP, Stream.concat(
+			Stream.of(key, value),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 apiCall(
+			BuiltInFunctionDefinitions.CONCAT,
+			Stream.concat(
+				Stream.of(string),
+				Stream.of(strings)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Calculates the arc tangent of a given coordinate.
+	 */
+	public static ApiExpression atan2(Object y, Object x) {
+		return apiCall(BuiltInFunctionDefinitions.ATAN2, objectToExpression(y), objectToExpression(x));
+	}
+
+	/**
+	 * Returns the string that results from concatenating the arguments and separator.
+	 * Returns NULL If the separator is NULL.
+	 *
+	 * <p>Note: this user-public static ApiExpressionined 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 apiCall(BuiltInFunctionDefinitions.CONCAT_WS, Stream.concat(
+			Stream.of(separator, string),
+			Stream.of(strings)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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.
+	 *
+	 * <p>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, objectToExpression(value));
+	}
+
+	/**
+	 * Calculates the logarithm of the given value to the given base.
+	 */
+	public static ApiExpression log(Object base, Object value) {
+		return apiCall(BuiltInFunctionDefinitions.LOG, objectToExpression(base), objectToExpression(value));
+	}
+
+	/**
+	 * Ternary conditional operator that decides which of two other expressions should be evaluated
+	 * based on a evaluated boolean condition.
+	 *
+	 * <p>e.g. ifThenElse(42 > 5, "A", "B") leads to "A"
 
 Review comment:
   This is a correct example? Do you think I should update to `lit(42) > lit(5)` ?

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot commented on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   -->
   ## CI report:
   
   * 16d4e9e6fbeb230b8aa05976e4909299ebc929bf 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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r384985750
 
 

 ##########
 File path: 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 expressions operations.
 
 Review comment:
   nit: `expression operations`

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r384988596
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,562 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.catalog.DataTypeFactory;
+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.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;
+
+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()}
+ *
+ * <p>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.
+ *
+ * <p>For fluent definition of expressions and easier readability, we recommend to add a
+ * star import to the methods of this class:
+ *
+ * <pre>
+ * import static org.apache.flink.table.api.Expressions.*;
+ * </pre>
+ *
+ * <p>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.
+	 *
+	 * <p>Example
+	 * <pre>{@code
+	 *   tab.select($("key"), $("value"))
+	 * }
+	 * </pre>
+	 */
+	//CHECKSTYLE.OFF: MethodName
+	public static ApiExpression $(String name) {
+		return new ApiExpression(unresolvedRef(name));
+	}
+	//CHECKSTYLE.ON: MethodName
+
+	/**
+	 * Creates a SQL literal.
+	 *
+	 * <p>The data type is derived from the object's class and its value.
+	 *
+	 * <p>For example:
+	 * <ul>
+	 *     <li>{@code lit(12)} leads to {@code INT}</li>
+	 *     <li>{@code lit("abc")} leads to {@code CHAR(3)}</li>
+	 *     <li>{@code lit(new BigDecimal("123.45"))} leads to {@code DECIMAL(5, 2)}</li>
+	 * </ul>
+	 *
+	 * <p>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}.
+	 *
+	 * <p>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.
+	 *
+	 * <p>Example:
+	 * <pre>{@code
+	 * Table table = ...
+	 * table.withColumns(range(b, c))
+	 * }</pre>
+	 *
+	 * @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.
+	 *
+	 * <p>Example:
+	 * <pre>{@code
+	 * Table table = ...
+	 * table.withColumns(range(3, 4))
+	 * }</pre>
+	 *
+	 * @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 <code>leftEnd >= rightStart && rightEnd >= leftStart</code>.
+	 *
+	 * <p>It evaluates: leftEnd >= rightStart && rightEnd >= leftStart
+	 *
+	 * <p>e.g.
+	 * <pre>{@code
+	 * temporalOverlaps(
+	 *      lit("2:55:00").toTime(),
+	 *      interval(Duration.ofHour(1)),
+	 *      lit("3:30:00").toTime(),
+	 *      interval(Duration.ofHour(2))
+	 * }</pre>
+	 * 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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <pre>{@code
+	 *  table.select(
+	 *      map(
+	 *          "key1", 1,
+	 *          "key2", 2,
+	 *          "key3", 3
+	 *      ))
+	 * }</pre>
+	 *
+	 * <p>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 minus(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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>A range can either be index-based or name-based. Indices start at 1 and boundaries are
+	 * inclusive.
+	 *
+	 * <p>e.g. in Scala: withColumns(range("b", "c")) or withoutColumns($("*"))
 
 Review comment:
   remove `in Scala`?

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r385002478
 
 

 ##########
 File path: 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]].
 
 Review comment:
   Let's deprecate the parser.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380068316
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,527 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.internal.BaseExpressions;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.ExpressionVisitor;
+import org.apache.flink.table.expressions.TimePointUnit;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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 java.util.Arrays;
+import java.util.List;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral;
+
+/**
+ * Entry point for a Java Table Expression DSL.
+ */
+@PublicEvolving
+public final class Expressions {
+
+	/**
+	 * Creates an unresolved reference to a table's column.
+	 */
+	//CHECKSTYLE.OFF: MethodName
+	public static ApiExpression $(String name) {
+		return new ApiExpression(ApiExpressionUtils.unresolvedRef(name));
+	}
+	//CHECKSTYLE.ON: MethodName
+
+	/**
+	 * Creates a SQL literal. The data type is derived from the Java class.
+	 */
+	public static ApiExpression lit(Object v) {
+		return new ApiExpression(valueLiteral(v));
+	}
+
+	/**
+	 * Creates a SQL literal of a given {@link DataType}.
+	 */
+	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.
+	 *
+	 * @see #withColumns(Object, Object...)
+	 * @see #withoutColumns(Object, Object...)
+	 */
+	public static ApiExpression range(String start, String end) {
+		return apiCall(BuiltInFunctionDefinitions.RANGE_TO, valueLiteral(start), valueLiteral(end));
+	}
+
+	/**
+	 * Indicates an index based range, which can be used in columns selection.
+	 *
+	 * @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 apiCall(BuiltInFunctionDefinitions.AND, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Boolean OR in three-valued logic.
+	 */
+	public static ApiExpression or(Object predicate0, Object predicate1, Object... predicates) {
+		return apiCall(BuiltInFunctionDefinitions.OR, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 <code>leftEnd >= rightStart && rightEnd >= leftStart</code>.
+	 *
+	 * <p>It evaluates: leftEnd >= rightStart && rightEnd >= leftStart
+	 *
+	 * <p>e.g. temporalOverlaps("2:55:00".toTime, 1.hour, "3:30:00".toTime, 2.hour) leads to true
+	 */
+	public static ApiExpression temporalOverlaps(
+			Object leftTimePoint,
+			Object leftTemporal,
+			Object rightTimePoint,
+			Object rightTemporal) {
+		return apiCall(
+			BuiltInFunctionDefinitions.TEMPORAL_OVERLAPS,
+			objectToExpression(leftTimePoint),
+			objectToExpression(leftTemporal),
+			objectToExpression(rightTimePoint),
+			objectToExpression(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.
+	 *
+	 * <p>For example 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,
+			objectToExpression(timestamp),
+			objectToExpression(format));
+	}
+
+	/**
+	 * Returns the (signed) number of {@link TimePointUnit} between timePoint1 and timePoint2.
+	 *
+	 * <p>For example, {@code timestampDiff(TimePointUnit.DAY, '2016-06-15'.toDate, '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),
+			objectToExpression(timePoint1),
+			objectToExpression(timePoint2));
+	}
+
+	/**
+	 * Creates an array of literals.
+	 */
+	public static ApiExpression array(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.ARRAY,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a row of expressions.
+	 */
+	public static ApiExpression row(Object head, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.ROW, Stream.concat(
+			Stream.of(head),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a map of expressions.
+	 */
+	public static ApiExpression map(Object key, Object value, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.MAP, Stream.concat(
+			Stream.of(key, value),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 apiCall(
+			BuiltInFunctionDefinitions.CONCAT,
+			Stream.concat(
+				Stream.of(string),
+				Stream.of(strings)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Calculates the arc tangent of a given coordinate.
+	 */
+	public static ApiExpression atan2(Object y, Object x) {
+		return apiCall(BuiltInFunctionDefinitions.ATAN2, objectToExpression(y), objectToExpression(x));
+	}
+
+	/**
+	 * Returns the string that results from concatenating the arguments and separator.
+	 * Returns NULL If the separator is NULL.
+	 *
+	 * <p>Note: this user-public static ApiExpressionined 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 apiCall(BuiltInFunctionDefinitions.CONCAT_WS, Stream.concat(
+			Stream.of(separator, string),
+			Stream.of(strings)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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.
+	 *
+	 * <p>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, objectToExpression(value));
+	}
+
+	/**
+	 * Calculates the logarithm of the given value to the given base.
+	 */
+	public static ApiExpression log(Object base, Object value) {
+		return apiCall(BuiltInFunctionDefinitions.LOG, objectToExpression(base), objectToExpression(value));
+	}
+
+	/**
+	 * Ternary conditional operator that decides which of two other expressions should be evaluated
+	 * based on a evaluated boolean condition.
+	 *
+	 * <p>e.g. ifThenElse(42 > 5, "A", "B") leads to "A"
 
 Review comment:
   update here and in following examples...

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
dawidwys commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r385216930
 
 

 ##########
 File path: flink-table/flink-table-api-scala/src/test/scala/org/apache/flink/table/api/ExpressionsConsistencyCheckTest.scala
 ##########
 @@ -0,0 +1,294 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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" -> "multipliedBy", // *
+    "$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 rowsInterval
 
 Review comment:
   That's a typo in the comment.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380083014
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/rules/UnwrapApiExpressionRule.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.expressions.resolver.rules;
+
+import org.apache.flink.table.api.Expressions;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Unwraps all {@link Expressions.ApiExpression}.
+ */
+final class UnwrapApiExpressionRule implements ResolverRule {
 
 Review comment:
   `@Internal`

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148755202",
       "triggerID" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149801561",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149806101",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149869824",
       "triggerID" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149885015",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149970663",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149981047",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150432772",
       "triggerID" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150669044",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5643",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4bcbe8c726aa4cda0ac0214e5997c9c0d38640be Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/150669044) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5643) 
   
   <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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148755202",
       "triggerID" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149801561",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149806101",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149869824",
       "triggerID" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149885015",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149970663",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149981047",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150432772",
       "triggerID" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150669044",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5643",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d510ce90e5cd8a13256c84c855562c403091a14",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150861986",
       "triggerID" : "8d510ce90e5cd8a13256c84c855562c403091a14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3786dab008166eff376617dabda4dedeba7ca961",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3786dab008166eff376617dabda4dedeba7ca961",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0e3ee0c3b8b58815b2f771d8b4320e7483c41159",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0e3ee0c3b8b58815b2f771d8b4320e7483c41159",
       "triggerType" : "PUSH"
     }, {
       "hash" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5706",
       "triggerID" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150885626",
       "triggerID" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f9502902a9c1a3203a22fdeb91f47551c3e862d1",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150974862",
       "triggerID" : "f9502902a9c1a3203a22fdeb91f47551c3e862d1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f9502902a9c1a3203a22fdeb91f47551c3e862d1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5721",
       "triggerID" : "f9502902a9c1a3203a22fdeb91f47551c3e862d1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2264ba7fa5063309c8cb58cc4c32694b90164f0d",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150998989",
       "triggerID" : "2264ba7fa5063309c8cb58cc4c32694b90164f0d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2264ba7fa5063309c8cb58cc4c32694b90164f0d",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5728",
       "triggerID" : "2264ba7fa5063309c8cb58cc4c32694b90164f0d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3786dab008166eff376617dabda4dedeba7ca961 UNKNOWN
   * 0e3ee0c3b8b58815b2f771d8b4320e7483c41159 UNKNOWN
   * f9502902a9c1a3203a22fdeb91f47551c3e862d1 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/150974862) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5721) 
   * 2264ba7fa5063309c8cb58cc4c32694b90164f0d Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/150998989) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5728) 
   
   <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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128",
       "triggerID" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148755202",
       "triggerID" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149801561",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149806101",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149869824",
       "triggerID" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149885015",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149970663",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149981047",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3c118088adedb8b818d0371495d313e9fe038968 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/149981047) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408) 
   
   <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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148755202 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   Hash:93f931c7ef96767509f62982e48b4853d3a74dfc Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/149801561 TriggerType:PUSH TriggerID:93f931c7ef96767509f62982e48b4853d3a74dfc
   Hash:93f931c7ef96767509f62982e48b4853d3a74dfc Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362 TriggerType:PUSH TriggerID:93f931c7ef96767509f62982e48b4853d3a74dfc
   Hash:b79c6583d86289244513a44c260c5c5d6b520f48 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363 TriggerType:PUSH TriggerID:b79c6583d86289244513a44c260c5c5d6b520f48
   Hash:b79c6583d86289244513a44c260c5c5d6b520f48 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149806101 TriggerType:PUSH TriggerID:b79c6583d86289244513a44c260c5c5d6b520f48
   Hash:91b982ff6a238b4db2bc89e81e53327d731e9378 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/149869824 TriggerType:PUSH TriggerID:91b982ff6a238b4db2bc89e81e53327d731e9378
   Hash:01b03edabdb80d7a941b10ecccb2670973d8b64a Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149885015 TriggerType:PUSH TriggerID:01b03edabdb80d7a941b10ecccb2670973d8b64a
   Hash:01b03edabdb80d7a941b10ecccb2670973d8b64a Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391 TriggerType:PUSH TriggerID:01b03edabdb80d7a941b10ecccb2670973d8b64a
   Hash:2590f92a3dbe4302baf85b67b090ffb75b05d52b Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404 TriggerType:PUSH TriggerID:2590f92a3dbe4302baf85b67b090ffb75b05d52b
   Hash:2590f92a3dbe4302baf85b67b090ffb75b05d52b Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/149970663 TriggerType:PUSH TriggerID:2590f92a3dbe4302baf85b67b090ffb75b05d52b
   Hash:3c118088adedb8b818d0371495d313e9fe038968 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/149981047 TriggerType:PUSH TriggerID:3c118088adedb8b818d0371495d313e9fe038968
   -->
   ## CI report:
   
   * 16d4e9e6fbeb230b8aa05976e4909299ebc929bf Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148755202) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128) 
   * 93f931c7ef96767509f62982e48b4853d3a74dfc Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/149801561) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362) 
   * b79c6583d86289244513a44c260c5c5d6b520f48 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149806101) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363) 
   * 91b982ff6a238b4db2bc89e81e53327d731e9378 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/149869824) 
   * 01b03edabdb80d7a941b10ecccb2670973d8b64a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149885015) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391) 
   * 2590f92a3dbe4302baf85b67b090ffb75b05d52b Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/149970663) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404) 
   * 3c118088adedb8b818d0371495d313e9fe038968 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/149981047) 
   
   <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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r384996045
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java
 ##########
 @@ -0,0 +1,1286 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 <InType>  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 <OutType> 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<InType, OutType> {
+	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 org.apache.flink.table.api.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 org.apache.flink.table.api.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 multipliedBy(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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>For example:
+	 * <pre>
+	 * {@code
+	 * orders
+	 *  .groupBy($("a"))
+	 *  .select($("a"), $("b").sum().distinct().as("d"))
+	 * }
+	 * </pre>
+	 */
+	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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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()));
+	}
+
+	// scalar functions
+
+	/**
+	 * 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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>e.g. "a".position("bbbbba") leads to 6
 
 Review comment:
   update to Java?

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380096106
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -280,6 +282,40 @@ public static ApiExpression rowInterval(Long rows) {
 		return new ApiExpression(valueLiteral(rows));
 	}
 
+
+	/**
+	 * Creates a SQL INTERVAL literal corresponding to the given {@link Duration}.
+	 * Equivalent to {@code lit(Duration)}.
 
 Review comment:
   this logic is different from `lit(Duration)` because the `ValueDataTypeConverter` analyzes the values and generates suitable intervals: `INTERVAL DAY(2) TO SECOND(3) != INTERVAL SECOND(3)`
   Either we don't limit intervals in the API and let the planner fail (this is also what we do in `lit()`) or we call `valueLiteral()` first and check for `INTERVAL SECOND(3)` afterwards.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380181067
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableImpl.java
 ##########
 @@ -563,6 +559,16 @@ private TableImpl createTable(QueryOperation operation) {
 		return new TableImpl(tableEnvironment, operation, operationTreeBuilder, lookupResolver);
 	}
 
+	private List<Expression> preprocessExpressions(List<Expression> expressions) {
 
 Review comment:
   sorry, my fault. I thought it is written `pre-process`. Very nit picking :D

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380077918
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java
 ##########
 @@ -0,0 +1,1146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.Table;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.TimeIntervalUnit;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.tableRef;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.typeLiteral;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.unresolvedCall;
+import static org.apache.flink.table.expressions.utils.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.
+ */
+@PublicEvolving
+public abstract class BaseExpressions<InT, OutT> {
+	protected abstract Expression toExpr();
+
+	protected abstract OutT 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
+	 * @return field with an alias
+	 */
+	public OutT 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.
+	 */
+	public OutT and(InT other) {
+		return toApiSpecificExpression(unresolvedCall(AND, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Boolean OR in three-valued logic.
+	 */
+	public OutT or(InT other) {
+		return toApiSpecificExpression(unresolvedCall(OR, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Greater than.
+	 */
+	public OutT isGreater(InT other) {
+		return toApiSpecificExpression(unresolvedCall(GREATER_THAN, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Greater than or equal.
+	 */
+	public OutT isGreaterOrEqual(InT other) {
+		return toApiSpecificExpression(unresolvedCall(GREATER_THAN_OR_EQUAL, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Less than.
+	 */
+	public OutT isLess(InT other) {
+		return toApiSpecificExpression(unresolvedCall(LESS_THAN, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Less than or equal.
+	 */
+	public OutT isLessOrEqual(InT other) {
+		return toApiSpecificExpression(unresolvedCall(LESS_THAN_OR_EQUAL, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Equals.
+	 */
+	public OutT isEqual(InT other) {
+		return toApiSpecificExpression(unresolvedCall(EQUALS, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Not equal.
+	 */
+	public OutT isNotEqual(InT other) {
+		return toApiSpecificExpression(unresolvedCall(NOT_EQUALS, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Returns left plus right.
+	 */
+	public OutT plus(InT other) {
+		return toApiSpecificExpression(unresolvedCall(PLUS, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Returns left minus right.
+	 */
+	public OutT minus(InT other) {
+		return toApiSpecificExpression(unresolvedCall(MINUS, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Returns left divided by right.
+	 */
+	public OutT dividedBy(InT other) {
+		return toApiSpecificExpression(unresolvedCall(DIVIDE, toExpr(), objectToExpression(other)));
+	}
+
+	/**
+	 * Returns left multiplied by right.
+	 */
+	public OutT multipliedBy(InT 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
+	 * @return boolean or null
+	 */
+	public OutT between(InT lowerBound, InT 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
+	 * @return boolean or null
+	 */
+	public OutT notBetween(InT lowerBound, InT upperBound) {
+		return toApiSpecificExpression(unresolvedCall(
+			NOT_BETWEEN,
+			toExpr(),
+			objectToExpression(lowerBound),
+			objectToExpression(upperBound)));
+	}
+
+	/**
+	 * Returns true if the given expression is null.
+	 */
+	public OutT isNull() {
+		return toApiSpecificExpression(unresolvedCall(IS_NULL, toExpr()));
+	}
+
+	/**
+	 * Returns true if the given expression is not null.
+	 */
+	public OutT isNotNull() {
+		return toApiSpecificExpression(unresolvedCall(IS_NOT_NULL, toExpr()));
+	}
+
+	/**
+	 * Returns true if given boolean expression is true. False otherwise (for null and false).
+	 */
+	public OutT isTrue() {
+		return toApiSpecificExpression(unresolvedCall(IS_TRUE, toExpr()));
+	}
+
+	/**
+	 * Returns true if given boolean expression is false. False otherwise (for null and true).
+	 */
+	public OutT isFalse() {
+		return toApiSpecificExpression(unresolvedCall(IS_FALSE, toExpr()));
+	}
+
+	/**
+	 * Returns true if given boolean expression is not true (for null and false). False otherwise.
+	 */
+	public OutT isNotTrue() {
+		return toApiSpecificExpression(unresolvedCall(IS_NOT_TRUE, toExpr()));
+	}
+
+	/**
+	 * Returns true if given boolean expression is not false (for null and true). False otherwise.
+	 */
+	public OutT 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.
+	 *
+	 * <p>For example:
+	 * <pre>
+	 * {@code
+	 * orders
+	 * .groupBy('a)
+	 * .select('a, 'b.sum.distinct as 'd)
+	 * }
+	 * </pre>
+	 */
+	public OutT 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 OutT 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 OutT sum0() {
+		return toApiSpecificExpression(unresolvedCall(SUM0, toExpr()));
+	}
+
+	/**
+	 * Returns the minimum value of field across all input values.
+	 */
+	public OutT min() {
+		return toApiSpecificExpression(unresolvedCall(MIN, toExpr()));
+	}
+
+	/**
+	 * Returns the maximum value of field across all input values.
+	 */
+	public OutT max() {
+		return toApiSpecificExpression(unresolvedCall(MAX, toExpr()));
+	}
+
+	/**
+	 * Returns the number of input rows for which the field is not null.
+	 */
+	public OutT count() {
+		return toApiSpecificExpression(unresolvedCall(COUNT, toExpr()));
+	}
+
+	/**
+	 * Returns the average (arithmetic mean) of the numeric field across all input values.
+	 */
+	public OutT avg() {
+		return toApiSpecificExpression(unresolvedCall(AVG, toExpr()));
+	}
+
+	/**
+	 * Returns the population standard deviation of an expression (the square root of varPop()).
+	 */
+	public OutT stddevPop() {
+		return toApiSpecificExpression(unresolvedCall(STDDEV_POP, toExpr()));
+	}
+
+	/**
+	 * Returns the sample standard deviation of an expression (the square root of varSamp()).
+	 */
+	public OutT stddevSamp() {
+		return toApiSpecificExpression(unresolvedCall(STDDEV_SAMP, toExpr()));
+	}
+
+	/**
+	 * Returns the population standard variance of an expression.
+	 */
+	public OutT varPop() {
+		return toApiSpecificExpression(unresolvedCall(VAR_POP, toExpr()));
+	}
+
+	/**
+	 * Returns the sample variance of a given expression.
+	 */
+	public OutT varSamp() {
+		return toApiSpecificExpression(unresolvedCall(VAR_SAMP, toExpr()));
+	}
+
+	/**
+	 * Returns multiset aggregate of a given expression.
+	 */
+	public OutT collect() {
+		return toApiSpecificExpression(unresolvedCall(COLLECT, toExpr()));
+	}
+
+	/**
+	 * Converts a value to a given data type.
+	 *
+	 * <p>e.g. "42".cast(DataTypes.INT()) leads to 42.
+	 *
+	 * @return casted expression
+	 */
+	public OutT 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
 
 Review comment:
   nit: indent

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380077497
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java
 ##########
 @@ -0,0 +1,1146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.Table;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.TimeIntervalUnit;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.tableRef;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.typeLiteral;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.unresolvedCall;
+import static org.apache.flink.table.expressions.utils.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.
+ */
+@PublicEvolving
+public abstract class BaseExpressions<InT, OutT> {
 
 Review comment:
   nit: `InType` instead of `T`?

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r385009963
 
 

 ##########
 File path: flink-table/flink-table-api-scala/src/test/scala/org/apache/flink/table/api/ExpressionsConsistencyCheckTest.scala
 ##########
 @@ -0,0 +1,294 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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" -> "multipliedBy", // *
+    "$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 rowsInterval
+    "rows",
+
+    // users in java should use static minus()
+    "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_-
 
 Review comment:
   nit: space

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
dawidwys commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380115348
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java
 ##########
 @@ -0,0 +1,527 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.internal.BaseExpressions;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.ExpressionVisitor;
+import org.apache.flink.table.expressions.TimePointUnit;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
+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 java.util.Arrays;
+import java.util.List;
+import java.util.stream.Stream;
+
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.objectToExpression;
+import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral;
+
+/**
+ * Entry point for a Java Table Expression DSL.
+ */
+@PublicEvolving
+public final class Expressions {
+
+	/**
+	 * Creates an unresolved reference to a table's column.
+	 */
+	//CHECKSTYLE.OFF: MethodName
+	public static ApiExpression $(String name) {
+		return new ApiExpression(ApiExpressionUtils.unresolvedRef(name));
+	}
+	//CHECKSTYLE.ON: MethodName
+
+	/**
+	 * Creates a SQL literal. The data type is derived from the Java class.
+	 */
+	public static ApiExpression lit(Object v) {
+		return new ApiExpression(valueLiteral(v));
+	}
+
+	/**
+	 * Creates a SQL literal of a given {@link DataType}.
+	 */
+	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.
+	 *
+	 * @see #withColumns(Object, Object...)
+	 * @see #withoutColumns(Object, Object...)
+	 */
+	public static ApiExpression range(String start, String end) {
+		return apiCall(BuiltInFunctionDefinitions.RANGE_TO, valueLiteral(start), valueLiteral(end));
+	}
+
+	/**
+	 * Indicates an index based range, which can be used in columns selection.
+	 *
+	 * @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 apiCall(BuiltInFunctionDefinitions.AND, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Boolean OR in three-valued logic.
+	 */
+	public static ApiExpression or(Object predicate0, Object predicate1, Object... predicates) {
+		return apiCall(BuiltInFunctionDefinitions.OR, Stream.concat(
+			Stream.of(predicate0, predicate1),
+			Stream.of(predicates)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 <code>leftEnd >= rightStart && rightEnd >= leftStart</code>.
+	 *
+	 * <p>It evaluates: leftEnd >= rightStart && rightEnd >= leftStart
+	 *
+	 * <p>e.g. temporalOverlaps("2:55:00".toTime, 1.hour, "3:30:00".toTime, 2.hour) leads to true
+	 */
+	public static ApiExpression temporalOverlaps(
+			Object leftTimePoint,
+			Object leftTemporal,
+			Object rightTimePoint,
+			Object rightTemporal) {
+		return apiCall(
+			BuiltInFunctionDefinitions.TEMPORAL_OVERLAPS,
+			objectToExpression(leftTimePoint),
+			objectToExpression(leftTemporal),
+			objectToExpression(rightTimePoint),
+			objectToExpression(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.
+	 *
+	 * <p>For example 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,
+			objectToExpression(timestamp),
+			objectToExpression(format));
+	}
+
+	/**
+	 * Returns the (signed) number of {@link TimePointUnit} between timePoint1 and timePoint2.
+	 *
+	 * <p>For example, {@code timestampDiff(TimePointUnit.DAY, '2016-06-15'.toDate, '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),
+			objectToExpression(timePoint1),
+			objectToExpression(timePoint2));
+	}
+
+	/**
+	 * Creates an array of literals.
+	 */
+	public static ApiExpression array(Object head, Object... tail) {
+		return apiCall(
+			BuiltInFunctionDefinitions.ARRAY,
+			Stream.concat(
+				Stream.of(head),
+				Stream.of(tail)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a row of expressions.
+	 */
+	public static ApiExpression row(Object head, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.ROW, Stream.concat(
+			Stream.of(head),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Creates a map of expressions.
+	 */
+	public static ApiExpression map(Object key, Object value, Object... tail) {
+		return apiCall(BuiltInFunctionDefinitions.MAP, Stream.concat(
+			Stream.of(key, value),
+			Stream.of(tail)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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 apiCall(
+			BuiltInFunctionDefinitions.CONCAT,
+			Stream.concat(
+				Stream.of(string),
+				Stream.of(strings)
+			).map(ApiExpressionUtils::objectToExpression)
+				.toArray(Expression[]::new));
+	}
+
+	/**
+	 * Calculates the arc tangent of a given coordinate.
+	 */
+	public static ApiExpression atan2(Object y, Object x) {
+		return apiCall(BuiltInFunctionDefinitions.ATAN2, objectToExpression(y), objectToExpression(x));
+	}
+
+	/**
+	 * Returns the string that results from concatenating the arguments and separator.
+	 * Returns NULL If the separator is NULL.
+	 *
+	 * <p>Note: this user-public static ApiExpressionined 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 apiCall(BuiltInFunctionDefinitions.CONCAT_WS, Stream.concat(
+			Stream.of(separator, string),
+			Stream.of(strings)
+		).map(ApiExpressionUtils::objectToExpression)
+			.toArray(Expression[]::new));
+	}
+
+	/**
+	 * 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.
+	 *
+	 * <p>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, objectToExpression(value));
+	}
+
+	/**
+	 * Calculates the logarithm of the given value to the given base.
+	 */
+	public static ApiExpression log(Object base, Object value) {
+		return apiCall(BuiltInFunctionDefinitions.LOG, objectToExpression(base), objectToExpression(value));
+	}
+
+	/**
+	 * Ternary conditional operator that decides which of two other expressions should be evaluated
+	 * based on a evaluated boolean condition.
+	 *
+	 * <p>e.g. ifThenElse(42 > 5, "A", "B") leads to "A"
 
 Review comment:
   This is a correct example. Do you think I should update to `lit(42) > lit(5)` ?

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r385004404
 
 

 ##########
 File path: flink-table/flink-table-api-scala/src/main/scala/org/apache/flink/table/api/expressionDsl.scala
 ##########
 @@ -1160,6 +359,13 @@ trait ImplicitExpressionConversions {
     }
   }
 
+  implicit class FieldExpression(val sc: StringContext) {
+    def $(args: Any*): Expression = unresolvedRef(sc.s(args: _*))
+  }
+
+  implicit def apiExpressionToExpression(expr: ApiExpression): Expression =
 
 Review comment:
   Is this really necessary or just convenience? A ScalaDoc would help.

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   Hash:16d4e9e6fbeb230b8aa05976e4909299ebc929bf Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148755202 TriggerType:PUSH TriggerID:16d4e9e6fbeb230b8aa05976e4909299ebc929bf
   Hash:93f931c7ef96767509f62982e48b4853d3a74dfc Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/149801561 TriggerType:PUSH TriggerID:93f931c7ef96767509f62982e48b4853d3a74dfc
   Hash:93f931c7ef96767509f62982e48b4853d3a74dfc Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362 TriggerType:PUSH TriggerID:93f931c7ef96767509f62982e48b4853d3a74dfc
   Hash:b79c6583d86289244513a44c260c5c5d6b520f48 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363 TriggerType:PUSH TriggerID:b79c6583d86289244513a44c260c5c5d6b520f48
   Hash:b79c6583d86289244513a44c260c5c5d6b520f48 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149806101 TriggerType:PUSH TriggerID:b79c6583d86289244513a44c260c5c5d6b520f48
   Hash:91b982ff6a238b4db2bc89e81e53327d731e9378 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/149869824 TriggerType:PUSH TriggerID:91b982ff6a238b4db2bc89e81e53327d731e9378
   Hash:01b03edabdb80d7a941b10ecccb2670973d8b64a Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149885015 TriggerType:PUSH TriggerID:01b03edabdb80d7a941b10ecccb2670973d8b64a
   Hash:01b03edabdb80d7a941b10ecccb2670973d8b64a Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391 TriggerType:PUSH TriggerID:01b03edabdb80d7a941b10ecccb2670973d8b64a
   Hash:2590f92a3dbe4302baf85b67b090ffb75b05d52b Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404 TriggerType:PUSH TriggerID:2590f92a3dbe4302baf85b67b090ffb75b05d52b
   Hash:2590f92a3dbe4302baf85b67b090ffb75b05d52b Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/149970663 TriggerType:PUSH TriggerID:2590f92a3dbe4302baf85b67b090ffb75b05d52b
   Hash:3c118088adedb8b818d0371495d313e9fe038968 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:3c118088adedb8b818d0371495d313e9fe038968
   -->
   ## CI report:
   
   * 16d4e9e6fbeb230b8aa05976e4909299ebc929bf Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148755202) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128) 
   * 93f931c7ef96767509f62982e48b4853d3a74dfc Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/149801561) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362) 
   * b79c6583d86289244513a44c260c5c5d6b520f48 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149806101) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363) 
   * 91b982ff6a238b4db2bc89e81e53327d731e9378 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/149869824) 
   * 01b03edabdb80d7a941b10ecccb2670973d8b64a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149885015) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391) 
   * 2590f92a3dbe4302baf85b67b090ffb75b05d52b Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/149970663) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404) 
   * 3c118088adedb8b818d0371495d313e9fe038968 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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r380099911
 
 

 ##########
 File path: flink-table/flink-table-api-scala/src/test/scala/org/apache/flink/table/api/ExpressionsConsistencyCheckTest.scala
 ##########
 @@ -0,0 +1,301 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.Expression
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils.{unresolvedCall, unresolvedRef, valueLiteral}
+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._
+
+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" -> "multipliedBy", // *
+    "$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 WithOperations
+    //-----------------------------------------------------------------------------------
+    "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 rowsInterval
+    "rows",
+
+    // not supported in java
+    "unary_$plus", // unary_+
+    "unary_$minus", // unary_-
+
+    //-----------------------------------------------------------------------------------
+    //  Interval conversion - deprecated in favor of interval(int, Resolution), interval(String,
 
 Review comment:
   Is this comment correct?

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r384996362
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java
 ##########
 @@ -0,0 +1,1286 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 <InType>  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 <OutType> 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<InType, OutType> {
+	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 org.apache.flink.table.api.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 org.apache.flink.table.api.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 multipliedBy(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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>For example:
+	 * <pre>
+	 * {@code
+	 * orders
+	 *  .groupBy($("a"))
+	 *  .select($("a"), $("b").sum().distinct().as("d"))
+	 * }
+	 * </pre>
+	 */
+	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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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()));
+	}
+
+	// scalar functions
+
+	/**
+	 * 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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>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.
+	 *
+	 * <p>e.g. "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.
+	 *
+	 * <p>e.g. "hi".lpad(4, '??') returns "??hi",  "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.
+	 *
+	 * <p>e.g. "hi".rpad(4, '??') returns "hi??",  "hi".rpad(1, '??') returns "h"
 
 Review comment:
   update to Java

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r384992220
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java
 ##########
 @@ -0,0 +1,1286 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 <InType>  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 <OutType> 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<InType, OutType> {
+	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 org.apache.flink.table.api.Expressions#and(Object, Object, Object...)
 
 Review comment:
   very very nit: I would import the class for better readability

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148755202",
       "triggerID" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149801561",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149806101",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149869824",
       "triggerID" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149885015",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149970663",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149981047",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150432772",
       "triggerID" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150669044",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5643",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d510ce90e5cd8a13256c84c855562c403091a14",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8d510ce90e5cd8a13256c84c855562c403091a14",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4bcbe8c726aa4cda0ac0214e5997c9c0d38640be Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/150669044) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5643) 
   * 8d510ce90e5cd8a13256c84c855562c403091a14 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] flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148755202",
       "triggerID" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149801561",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149806101",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149869824",
       "triggerID" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149885015",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149970663",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149981047",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150432772",
       "triggerID" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150669044",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5643",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d510ce90e5cd8a13256c84c855562c403091a14",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150861986",
       "triggerID" : "8d510ce90e5cd8a13256c84c855562c403091a14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3786dab008166eff376617dabda4dedeba7ca961",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3786dab008166eff376617dabda4dedeba7ca961",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0e3ee0c3b8b58815b2f771d8b4320e7483c41159",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0e3ee0c3b8b58815b2f771d8b4320e7483c41159",
       "triggerType" : "PUSH"
     }, {
       "hash" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5706",
       "triggerID" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150885626",
       "triggerID" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f9502902a9c1a3203a22fdeb91f47551c3e862d1",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150974862",
       "triggerID" : "f9502902a9c1a3203a22fdeb91f47551c3e862d1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f9502902a9c1a3203a22fdeb91f47551c3e862d1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5721",
       "triggerID" : "f9502902a9c1a3203a22fdeb91f47551c3e862d1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3786dab008166eff376617dabda4dedeba7ca961 UNKNOWN
   * 0e3ee0c3b8b58815b2f771d8b4320e7483c41159 UNKNOWN
   * f9502902a9c1a3203a22fdeb91f47551c3e862d1 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/150974862) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5721) 
   
   <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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
twalthr commented on a change in pull request #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#discussion_r385607049
 
 

 ##########
 File path: flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/expressions/resolver/ExpressionResolverTest.java
 ##########
 @@ -236,7 +232,10 @@ public void testResolvingExpressions() {
 	/**
 	 * Test scalar function.
 	 */
-	@FunctionHint(input = @DataTypeHint(inputGroup = InputGroup.ANY), isVarArgs = true, output = @DataTypeHint(value = "INTEGER NOT NULL", bridgedTo = int.class))
+	@FunctionHint(
 
 Review comment:
   I thought more about:
   ```
   @FunctionHint(
   		input = @DataTypeHint(inputGroup = InputGroup.ANY),
   		isVarArgs = true,
           output = @DataTypeHint(value = "INTEGER NOT NULL", bridgedTo = int.class))
   ```

----------------------------------------------------------------
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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128",
       "triggerID" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148755202",
       "triggerID" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149801561",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149806101",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149869824",
       "triggerID" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149885015",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149970663",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149981047",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 16d4e9e6fbeb230b8aa05976e4909299ebc929bf Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148755202) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5128) 
   * 93f931c7ef96767509f62982e48b4853d3a74dfc Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/149801561) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362) 
   * b79c6583d86289244513a44c260c5c5d6b520f48 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149806101) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363) 
   * 91b982ff6a238b4db2bc89e81e53327d731e9378 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/149869824) 
   * 01b03edabdb80d7a941b10ecccb2670973d8b64a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149885015) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391) 
   * 2590f92a3dbe4302baf85b67b090ffb75b05d52b Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/149970663) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404) 
   * 3c118088adedb8b818d0371495d313e9fe038968 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/149981047) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408) 
   
   <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 #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148755202",
       "triggerID" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149801561",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149806101",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149869824",
       "triggerID" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149885015",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149970663",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149981047",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150432772",
       "triggerID" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150669044",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5643",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d510ce90e5cd8a13256c84c855562c403091a14",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150861986",
       "triggerID" : "8d510ce90e5cd8a13256c84c855562c403091a14",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3786dab008166eff376617dabda4dedeba7ca961",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3786dab008166eff376617dabda4dedeba7ca961",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0e3ee0c3b8b58815b2f771d8b4320e7483c41159",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0e3ee0c3b8b58815b2f771d8b4320e7483c41159",
       "triggerType" : "PUSH"
     }, {
       "hash" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5706",
       "triggerID" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150885626",
       "triggerID" : "156ddaa5298b8059b4038c486e00770a263c25f7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f9502902a9c1a3203a22fdeb91f47551c3e862d1",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150974862",
       "triggerID" : "f9502902a9c1a3203a22fdeb91f47551c3e862d1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f9502902a9c1a3203a22fdeb91f47551c3e862d1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5721",
       "triggerID" : "f9502902a9c1a3203a22fdeb91f47551c3e862d1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2264ba7fa5063309c8cb58cc4c32694b90164f0d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2264ba7fa5063309c8cb58cc4c32694b90164f0d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3786dab008166eff376617dabda4dedeba7ca961 UNKNOWN
   * 0e3ee0c3b8b58815b2f771d8b4320e7483c41159 UNKNOWN
   * f9502902a9c1a3203a22fdeb91f47551c3e862d1 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/150974862) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5721) 
   * 2264ba7fa5063309c8cb58cc4c32694b90164f0d 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] flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11081: [FLINK-16033][table-api] Introduced Java Table API Expression DSL
URL: https://github.com/apache/flink/pull/11081#issuecomment-585639356
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/148755202",
       "triggerID" : "16d4e9e6fbeb230b8aa05976e4909299ebc929bf",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149801561",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5362",
       "triggerID" : "93f931c7ef96767509f62982e48b4853d3a74dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5363",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149806101",
       "triggerID" : "b79c6583d86289244513a44c260c5c5d6b520f48",
       "triggerType" : "PUSH"
     }, {
       "hash" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149869824",
       "triggerID" : "91b982ff6a238b4db2bc89e81e53327d731e9378",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149885015",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5391",
       "triggerID" : "01b03edabdb80d7a941b10ecccb2670973d8b64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5404",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149970663",
       "triggerID" : "2590f92a3dbe4302baf85b67b090ffb75b05d52b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/149981047",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3c118088adedb8b818d0371495d313e9fe038968",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5408",
       "triggerID" : "3c118088adedb8b818d0371495d313e9fe038968",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150432772",
       "triggerID" : "0cd224f4da41acc6973bbd4fb27f11e201832e6e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/flink-ci/flink/builds/150669044",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5643",
       "triggerID" : "4bcbe8c726aa4cda0ac0214e5997c9c0d38640be",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4bcbe8c726aa4cda0ac0214e5997c9c0d38640be Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/150669044) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5643) 
   
   <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