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 2022/07/20 08:41:29 UTC

[GitHub] [flink] fsk119 commented on a diff in pull request #20000: [FLINK-28096][hive] Hive dialect support set variables

fsk119 commented on code in PR #20000:
URL: https://github.com/apache/flink/pull/20000#discussion_r925271157


##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/copy/HiveParserSetProcessor.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.delegation.hive.copy;
+
+import org.apache.flink.connectors.hive.FlinkHiveException;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.VariableSubstitution;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+
+import java.util.Map;
+
+import static org.apache.hadoop.hive.conf.SystemVariables.ENV_PREFIX;
+import static org.apache.hadoop.hive.conf.SystemVariables.HIVECONF_PREFIX;
+import static org.apache.hadoop.hive.conf.SystemVariables.HIVEVAR_PREFIX;
+import static org.apache.hadoop.hive.conf.SystemVariables.METACONF_PREFIX;
+import static org.apache.hadoop.hive.conf.SystemVariables.SYSTEM_PREFIX;
+
+/** Counterpart of hive's org.apache.hadoop.hive.ql.processors.SetProcessor. */
+public class HiveParserSetProcessor {

Review Comment:
   Rename to `HiveSetProcessor` because HiveServer2 Endpoint also requires this.



##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/copy/HiveParserSetProcessor.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.delegation.hive.copy;
+
+import org.apache.flink.connectors.hive.FlinkHiveException;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.VariableSubstitution;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+
+import java.util.Map;
+
+import static org.apache.hadoop.hive.conf.SystemVariables.ENV_PREFIX;
+import static org.apache.hadoop.hive.conf.SystemVariables.HIVECONF_PREFIX;
+import static org.apache.hadoop.hive.conf.SystemVariables.HIVEVAR_PREFIX;
+import static org.apache.hadoop.hive.conf.SystemVariables.METACONF_PREFIX;
+import static org.apache.hadoop.hive.conf.SystemVariables.SYSTEM_PREFIX;
+
+/** Counterpart of hive's org.apache.hadoop.hive.ql.processors.SetProcessor. */

Review Comment:
   use `{@link org.apache.hadoop.hive.ql.processors.SetProcessor}`.



##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/HiveParser.java:
##########
@@ -207,6 +232,23 @@ public List<Operation> parse(String statement) {
         }
     }
 
+    private Operation processSetCmd(HiveConf hiveConf, Matcher setCmdMatcher) {
+        if (setCmdMatcher.group("verbose") != null) {
+            throw new UnsupportedOperationException("Unsupported command 'set -v'.");
+        }
+        String key = setCmdMatcher.group("key");
+        String value = setCmdMatcher.group("val");
+        if (FLINK_RESERVED_KEYWORDS_FOR_SET_CMD.contains(key)) {
+            // try to let Flink CLI handle it
+            return new SetOperation(key, value);
+        } else {
+            if (!key.equals("silent")) {
+                HiveParserSetProcessor.setVariable(hiveConf, hiveVariables, key, value);
+            }
+            return new NopOperation();

Review Comment:
   We'd better to also set the config into table config because users may set some optimization options here.



##########
flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectITCase.java:
##########
@@ -794,6 +795,33 @@ public void testShowPartitions() throws Exception {
         assertThat(partitions.toString()).contains("dt=2020-04-30 01:02:03/country=china");
     }
 
+    @Test
+    public void testSetCommand() throws Exception {

Review Comment:
   Is the user able to `SET key=value -v`? Also, add a case to check whether SET works.



##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/HiveParser.java:
##########
@@ -207,6 +232,23 @@ public List<Operation> parse(String statement) {
         }
     }
 
+    private Operation processSetCmd(HiveConf hiveConf, Matcher setCmdMatcher) {
+        if (setCmdMatcher.group("verbose") != null) {
+            throw new UnsupportedOperationException("Unsupported command 'set -v'.");
+        }
+        String key = setCmdMatcher.group("key");
+        String value = setCmdMatcher.group("val");
+        if (FLINK_RESERVED_KEYWORDS_FOR_SET_CMD.contains(key)) {
+            // try to let Flink CLI handle it
+            return new SetOperation(key, value);
+        } else {
+            if (!key.equals("silent")) {

Review Comment:
   It's better if we can throw exceptions to notify the users we don't support to set silent.



##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/HiveParser.java:
##########
@@ -83,6 +91,17 @@ public class HiveParser extends ParserImpl {
             HiveReflectionUtils.tryGetMethod(
                     SessionState.class, "getQueryCurrentTimestamp", new Class[0]);
 
+    private static final Pattern SET_CMD_PATTERN1 =

Review Comment:
   SET_CMD_PATTERN



##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/HiveParser.java:
##########
@@ -83,6 +91,17 @@ public class HiveParser extends ParserImpl {
             HiveReflectionUtils.tryGetMethod(
                     SessionState.class, "getQueryCurrentTimestamp", new Class[0]);
 
+    private static final Pattern SET_CMD_PATTERN1 =
+            // set key=val; set key=; set -v;
+            Pattern.compile(
+                    "SET(\\s+(((?<key>[^\\s]+)\\s*=\\s*(?<val>[^;\\s]*))|(?<verbose>-v))\\s*;?)",
+                    Pattern.CASE_INSENSITIVE | Pattern.DOTALL);
+
+    private static final Set<String> FLINK_RESERVED_KEYWORDS_FOR_SET_CMD =
+            Sets.newHashSet("table.sql-dialect");

Review Comment:
   Use `TABLE_SQL_DIALECT.key()`
   
   BTW, it's better if we can use `Collections.singleton` here.



##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/HiveParser.java:
##########
@@ -190,6 +209,12 @@ public List<Operation> parse(String statement) {
             LOG.warn("Current catalog is not HiveCatalog. Falling back to Flink's planner.");
             return super.parse(statement);
         }
+        // check whether it's set command
+        Matcher setCmdMatcher = SET_CMD_PATTERN1.matcher(statement);
+        if (setCmdMatcher.find()) {
+            return Collections.singletonList(
+                    processSetCmd(((HiveCatalog) currentCatalog).getHiveConf(), setCmdMatcher));
+        }

Review Comment:
   It seems Hive also supports other kinds of commands, e.g. RESET/COMPILE. How about we reuse the HiveCommand to notify users of unsupported commands.
   
   ```
          HiveCommand hiveCommand = HiveCommand.find(statement.split("\\s+"), false);
           if (hiveCommand != null) {
               return Collections.singletonList(
                       processCmd(hiveCommand, statement, (HiveCatalog) currentCatalog));
           }
   
   ```
   
   ```
       private Operation processCmd(HiveCommand command, String statement, HiveCatalog catalog) {
           if (command != HiveCommand.SET) {
               throw new UnsupportedOperationException();
           }
           return processSetCmd(catalog.getHiveConf(), statement);
       }
   ```



##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/HiveParser.java:
##########
@@ -83,6 +91,17 @@ public class HiveParser extends ParserImpl {
             HiveReflectionUtils.tryGetMethod(
                     SessionState.class, "getQueryCurrentTimestamp", new Class[0]);
 
+    private static final Pattern SET_CMD_PATTERN1 =
+            // set key=val; set key=; set -v;
+            Pattern.compile(
+                    "SET(\\s+(((?<key>[^\\s]+)\\s*=\\s*(?<val>[^;\\s]*))|(?<verbose>-v))\\s*;?)",
+                    Pattern.CASE_INSENSITIVE | Pattern.DOTALL);
+
+    private static final Set<String> FLINK_RESERVED_KEYWORDS_FOR_SET_CMD =
+            Sets.newHashSet("table.sql-dialect");
+
+    private static final Map<String, String> hiveVariables = new HashMap<>();

Review Comment:
   Don't use static variables because it is visible to all sessions on the gateway side.



-- 
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-unsubscribe@flink.apache.org

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