twalthr commented on a change in pull request #17186:
URL: https://github.com/apache/flink/pull/17186#discussion_r708393288



##########
File path: 
flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/RepeatingSequenceInputTypeStrategy.java
##########
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.types.inference.strategies;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.functions.FunctionDefinition;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.inference.ArgumentCount;
+import org.apache.flink.table.types.inference.ArgumentTypeStrategy;
+import org.apache.flink.table.types.inference.CallContext;
+import org.apache.flink.table.types.inference.InputTypeStrategy;
+import org.apache.flink.table.types.inference.Signature;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * {@link InputTypeStrategy} composed of an arbitrarily often repeating list 
of {@link
+ * ArgumentTypeStrategy}s.
+ */
+@Internal
+public class RepeatingSequenceInputTypeStrategy implements InputTypeStrategy {
+
+    private final List<ArgumentTypeStrategy> argumentStrategies;
+
+    public RepeatingSequenceInputTypeStrategy(List<ArgumentTypeStrategy> 
argumentStrategies) {
+        this.argumentStrategies = argumentStrategies;
+    }
+
+    @Override
+    public ArgumentCount getArgumentCount() {
+        return new ArgumentCount() {
+            @Override
+            public boolean isValidCount(int count) {
+                return count % argumentStrategies.size() == 0;
+            }
+
+            @Override
+            public Optional<Integer> getMinCount() {
+                return Optional.empty();
+            }
+
+            @Override
+            public Optional<Integer> getMaxCount() {
+                return Optional.empty();
+            }
+        };
+    }
+
+    @Override
+    public Optional<List<DataType>> inferInputTypes(
+            CallContext callContext, boolean throwOnFailure) {
+        final List<DataType> dataTypes = callContext.getArgumentDataTypes();
+        final List<DataType> inferredDataTypes = new 
ArrayList<>(dataTypes.size());
+
+        for (int i = 0; i < callContext.getArgumentDataTypes().size(); i++) {
+            final ArgumentTypeStrategy argumentStrategy =
+                    argumentStrategies.get(i % argumentStrategies.size());
+
+            final Optional<DataType> inferredDataType =
+                    argumentStrategy.inferArgumentType(callContext, i, 
throwOnFailure);
+            if (!inferredDataType.isPresent()) {
+                return Optional.empty();
+            }
+
+            inferredDataTypes.add(inferredDataType.get());
+        }
+
+        return Optional.of(inferredDataTypes);
+    }
+
+    @Override
+    public List<Signature> getExpectedSignatures(FunctionDefinition 
definition) {
+        final List<Signature.Argument> arguments = new ArrayList<>();
+        for (int i = 0; i < argumentStrategies.size(); i++) {
+            final Signature.Argument argument =
+                    argumentStrategies.get(i).getExpectedArgument(definition, 
i);
+
+            final Signature.Argument newArgument;
+            if (i == 0) {
+                newArgument = Signature.Argument.of(String.format("[%s", 
argument.getType()));
+            } else if (i == argumentStrategies.size() - 1) {
+                newArgument = Signature.Argument.of(String.format("%s]…", 
argument.getType()));

Review comment:
       nit: to be consistent with varying sequence we should use `...` instead 
of `…`

##########
File path: 
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/JsonFunctionsITCase.java
##########
@@ -54,6 +77,7 @@
         testCases.add(jsonValue());
         testCases.addAll(isJson());
         testCases.addAll(jsonQuery());
+        testCases.addAll(jsonObjectSpec());

Review comment:
       nit: why `Spec` here?

##########
File path: 
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/ExpressionReducer.scala
##########
@@ -78,6 +79,8 @@ class ExpressionReducer(
            (SqlTypeName.MAP, _) |
            (SqlTypeName.MULTISET, _) => None
 
+      case (_, call: RexCall) if call.getOperator == JSON_OBJECT => None

Review comment:
       add a comment why we exclude this function

##########
File path: 
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/JsonFunctionsITCase.java
##########
@@ -420,8 +431,146 @@ private static TestSpec jsonValue() throws Exception {
                                 "No results for path"));
     }
 
+    private static List<TestSpec> jsonObjectSpec() {
+        final Map<String, String> mapData = new HashMap<>();
+        mapData.put("M1", "V1");
+        mapData.put("M2", "V2");
+
+        final Map<String, Integer> multisetData = new HashMap<>();
+        multisetData.put("M1", 1);
+        multisetData.put("M2", 2);
+
+        return Arrays.asList(
+                TestSpec.forFunction(BuiltInFunctionDefinitions.JSON_OBJECT)
+                        .onFieldsWithData(0)
+                        .testResult(
+                                jsonObject(JsonOnNull.NULL),
+                                "JSON_OBJECT()",
+                                "{}",
+                                STRING().notNull(),
+                                VARCHAR(2000).notNull())
+                        .testResult(
+                                jsonObject(JsonOnNull.NULL, "K", 
nullOf(STRING())),
+                                "JSON_OBJECT(KEY 'K' VALUE CAST(NULL AS 
STRING) NULL ON NULL)",
+                                "{\"K\":null}",
+                                STRING().notNull(),
+                                VARCHAR(2000).notNull())
+                        .testResult(
+                                jsonObject(JsonOnNull.ABSENT, "K", 
nullOf(STRING())),
+                                "JSON_OBJECT(KEY 'K' VALUE CAST(NULL AS 
STRING) ABSENT ON NULL)",
+                                "{}",
+                                STRING().notNull(),
+                                VARCHAR(2000).notNull()),
+                TestSpec.forFunction(BuiltInFunctionDefinitions.JSON_OBJECT)
+                        .onFieldsWithData(
+                                "V",
+                                true,
+                                1,
+                                1.23d,
+                                1.23,
+                                LocalDateTime.parse("1990-06-02T13:37:42.001"),
+                                Instant.parse("1990-06-02T13:37:42.001Z"),
+                                Arrays.asList("A1", "A2", "A3"),
+                                Row.of("R1"),

Review comment:
       we should add at least a second field, maybe with a timestamp?




-- 
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.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to