libenchao commented on code in PR #23849:
URL: https://github.com/apache/flink/pull/23849#discussion_r1416754135


##########
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/context/SessionContext.java:
##########
@@ -136,6 +158,10 @@ public URLClassLoader getUserClassloader() {
         return userClassloader;
     }
 
+    public PlanCacheManager getPlanCacheManager() {

Review Comment:
   `@Nullable`



##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableConfig.java:
##########
@@ -475,4 +476,23 @@ public void setRootConfiguration(ReadableConfig 
rootConfiguration) {
     public static TableConfig getDefault() {
         return new TableConfig();
     }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        TableConfig that = (TableConfig) o;
+        return Objects.equals(plannerConfig, that.plannerConfig)
+                && Objects.equals(configuration, that.configuration)
+                && Objects.equals(rootConfiguration, that.rootConfiguration);
+    }
+
+    @Override
+    public int hashCode() {

Review Comment:
   Then changes in `TableConfig` can be reverted now.



##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableResultImpl.java:
##########
@@ -224,13 +234,23 @@ public Builder setPrintStyle(PrintStyle printStyle) {
             return this;
         }
 
+        /**
+         * Specifies cached plan which associates the submitted Flink job.
+         *
+         * @param cachedPlan a {@link CachedPlan} for the submitted Flink job.
+         */
+        public Builder setPlanValue(CachedPlan cachedPlan) {

Review Comment:
   `setCachedPlan` is better?



##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java:
##########
@@ -1064,6 +1076,8 @@ private TableResultInternal 
executeQueryOperation(QueryOperation operation) {
         try {
             JobClient jobClient = execEnv.executeAsync(pipeline);
             ResultProvider resultProvider = 
sinkOperation.getSelectResultProvider();
+            // We must reset resultProvider as we have to reuse it between 
different jobs.

Review Comment:
   ```suggestion
               // We must reset resultProvider as we might reuse it between 
different jobs.
   ```



##########
docs/content/docs/dev/table/olap_quickstart.md:
##########
@@ -151,10 +151,11 @@ In OLAP scenario, appropriate configurations that can 
greatly help users improve
 
 #### SQL&Table Options
 
-| Parameters                                                                   
                                  | Default | Recommended |
-|:---------------------------------------------------------------------------------------------------------------|:--------|:------------|
-| [table.optimizer.join-reorder-enabled]({{<ref 
"docs/dev/table/config#table-optimizer-join-reorder-enabled">}}) | false | true 
|
-| [pipeline.object-reuse]({{< ref 
"docs/deployment/config#pipeline-object-reuse" >}})                            
| false | true |
+| Parameters                                                                   
                                                      | Default   | Recommended 
|
+|:-----------------------------------------------------------------------------------------------------------------------------------|:----------|:------------|
+| [table.optimizer.join-reorder-enabled]({{<ref 
"docs/dev/table/config#table-optimizer-join-reorder-enabled">}})                
     | false     | true        |
+| [pipeline.object-reuse]({{< ref 
"docs/deployment/config#pipeline-object-reuse" >}})                             
                   | false     | true        |
+| [sql-gateway.session.plan-cache.enabled]({{<ref 
"docs/dev/table/sql-gateway/overview/#sql-gateway-session-plan-cache-enabled">}})
  | false     | true        |

Review Comment:
   ```suggestion
   | [sql-gateway.session.plan-cache.enabled]({{<ref 
"docs/dev/table/sql-gateway/overview#sql-gateway-session-plan-cache-enabled">}})
  | false     | true        |
   ```



##########
docs/content/docs/dev/table/sql-gateway/overview.md:
##########
@@ -190,6 +190,24 @@ $ ./sql-gateway -Dkey=value
             <td>Integer</td>
             <td>The maximum number of the active session for sql gateway 
service.</td>
         </tr>
+        <tr>
+            <td><h5>sql-gateway.session.plan-cache.enabled</h5></td>
+            <td style="word-wrap: break-word;">false</td>
+            <td>Boolean</td>
+            <td>When it is true, sql gateway will cache and reuse plans for 
queries.</td>

Review Comment:
   ```suggestion
               <td>When it is true, sql gateway will cache and reuse plans for 
queries per session.</td>
   ```



##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/PlanCacheManager.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.internal;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.guava31.com.google.common.cache.Cache;
+import org.apache.flink.shaded.guava31.com.google.common.cache.CacheBuilder;
+import org.apache.flink.shaded.guava31.com.google.common.cache.CacheStats;
+
+import java.time.Duration;
+import java.util.Optional;
+
+/** This manages all the plan caches. */
+@Internal
+public class PlanCacheManager {
+
+    private final Cache<String, CachedPlan> planCache;
+
+    public PlanCacheManager(long maximumCapacity, Duration ttl) {
+        planCache =
+                CacheBuilder.newBuilder()
+                        .maximumSize(maximumCapacity)
+                        .expireAfterWrite(ttl)
+                        .recordStats()
+                        .build();
+    }
+
+    public Optional<CachedPlan> getPlan(String query) {
+        CachedPlan cachedPlan = planCache.getIfPresent(query);
+        return Optional.ofNullable(cachedPlan);
+    }
+
+    public void putPlan(String query, CachedPlan cachedPlan) {
+        Preconditions.checkNotNull(query, "query can not be null");
+        Preconditions.checkNotNull(cachedPlan, "cachedPlan can not be null");
+        planCache.put(query, cachedPlan);
+    }
+
+    public void invalidateAll() {
+        planCache.invalidateAll();
+    }
+
+    public CacheStats getCacheStats() {

Review Comment:
   It seems that this is only used in tests, please add `@VisibleForTesting` 
annotation.



##########
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/service/session/Session.java:
##########
@@ -74,6 +75,10 @@ public OperationExecutor createExecutor(Configuration 
executionConfig) {
         return sessionContext.createOperationExecutor(executionConfig);
     }
 
+    public PlanCacheManager getPlanCacheManager() {

Review Comment:
   `@Nullable`



##########
flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/SqlGatewayServiceStatementITCase.java:
##########
@@ -52,18 +59,49 @@
 /** Test {@link SqlGatewayService}#executeStatement. */
 public class SqlGatewayServiceStatementITCase extends 
AbstractSqlGatewayStatementITCase {
 
-    private final SessionEnvironment defaultSessionEnvironment =
+    private static final SessionEnvironment DEFAULT_SESSION_ENVIRONMENT =
+            SessionEnvironment.newBuilder()
+                    .setSessionEndpointVersion(MockedEndpointVersion.V1)
+                    .build();
+
+    private static final SessionEnvironment 
SESSION_ENVIRONMENT_WITH_PLAN_CACHE_ENABLED =
             SessionEnvironment.newBuilder()
                     .setSessionEndpointVersion(MockedEndpointVersion.V1)
+                    .addSessionConfig(
+                            Collections.singletonMap(
+                                    
SQL_GATEWAY_SESSION_PLAN_CACHE_ENABLED.key(), "true"))
                     .build();
 
     private SessionHandle sessionHandle;
 
+    @Parameters(name = "parameters={0}")
+    public static List<TestParameters> parameters() throws Exception {
+        return listFlinkSqlTests().stream()
+                .map(path -> new StatementTestParameters(path, 
path.endsWith("repeated_dql.q")))
+                .collect(Collectors.toList());
+    }
+
     @BeforeEach
     @Override
     public void before(@TempDir Path temporaryFolder) throws Exception {
         super.before(temporaryFolder);
-        sessionHandle = service.openSession(defaultSessionEnvironment);
+        SessionEnvironment sessionEnvironment =
+                isPlanCacheEnabled()
+                        ? SESSION_ENVIRONMENT_WITH_PLAN_CACHE_ENABLED
+                        : DEFAULT_SESSION_ENVIRONMENT;
+        sessionHandle = service.openSession(sessionEnvironment);
+    }
+
+    @AfterEach
+    public void after() {
+        if (isPlanCacheEnabled()) {
+            CacheStats cacheStats =
+                    ((SqlGatewayServiceImpl) service)
+                            .getSession(sessionHandle)
+                            .getPlanCacheManager()
+                            .getCacheStats();
+            assertThat(cacheStats).isEqualTo(new CacheStats(4, 14, 0, 0, 0, 
0));

Review Comment:
   The design of this test is very fragile, since adding more tests in `*.iq` 
will all break this test, and they would be forced to change this test 
expectation as well. I'll think again about how we can test 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.

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