twalthr commented on code in PR #28129:
URL: https://github.com/apache/flink/pull/28129#discussion_r3209755684


##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/codegen/JsonParseReuseTest.java:
##########
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.planner.codegen;
+
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.TableResult;
+import org.apache.flink.table.runtime.generated.CompileUtils;
+import org.apache.flink.testutils.logging.LoggerAuditingExtension;
+import org.apache.flink.types.Row;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.slf4j.event.Level;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests that multiple JSON function calls on the same input reuse the parsed 
JSON. */
+class JsonParseReuseTest {
+
+    @RegisterExtension
+    private final LoggerAuditingExtension codeLog =
+            new LoggerAuditingExtension(CompileUtils.class, Level.DEBUG);
+

Review Comment:
   How about we access the Transformation (like e.g. in 
org.apache.flink.table.planner.plan.nodes.exec.TransformationsTest#toLegacySourceTransformation)
 and access the code string in the StreamOperatorFactory directly? Sounds a bit 
cleaner to me than intercepting loggers.



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGeneratorContext.scala:
##########
@@ -145,6 +145,11 @@ class CodeGeneratorContext(
   private val localRefScopes =
     mutable.ArrayBuffer(mutable.LinkedHashMap.empty[Int, GeneratedExpression])
 
+  // cache of pre-parsed JSON variables keyed by input result term, so that 
multiple
+  // JSON function calls on the same input share a single parse
+  private val reusableParsedJsonExprs: mutable.Map[String, String] =

Review Comment:
   do we really need a new map in context? can't we reuse existing 
`reusablePerRecordStatements` or `Members`?



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/JsonQueryCallGen.scala:
##########
@@ -50,8 +51,21 @@ class JsonQueryCallGen extends CallGenerator {
           } else {
             JsonQueryReturnType.STRING
           }
+          val inputTerm = s"${argTerms.head}.toString()"
+
+          val (parsedTerm, parseCode) = ctx.getReusableParsedJson(inputTerm) 
match {

Review Comment:
   could you paste an example code what will be constructed with these 
optimizations?



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to