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/06/17 05:21:05 UTC

[GitHub] [flink] luoyuxia opened a new pull request, #20000: [FLINK-28096][hive] Hive dialect support set variables

luoyuxia opened a new pull request, #20000:
URL: https://github.com/apache/flink/pull/20000

   <!--
   *Thank you very much for contributing to Apache Flink - we are happy that you want to help us improve Flink. To help the community review your contribution in the best possible way, please go through the checklist below, which will get the contribution into a shape in which it can be best reviewed.*
   
   *Please understand that we do not do this to make contributions to Flink a hassle. In order to uphold a high standard of quality for code contributions, while at the same time managing a large number of contributions, we need contributors to prepare the contributions well, and give reviewers enough contextual information for the review. Please also understand that contributions that do not follow this guide will take longer to review and thus typically be picked up with lower priority by the community.*
   
   ## Contribution Checklist
   
     - Make sure that the pull request corresponds to a [JIRA issue](https://issues.apache.org/jira/projects/FLINK/issues). Exceptions are made for typos in JavaDoc or documentation files, which need no JIRA issue.
     
     - Name the pull request in the form "[FLINK-XXXX] [component] Title of the pull request", where *FLINK-XXXX* should be replaced by the actual issue number. Skip *component* if you are unsure about which is the best component.
     Typo fixes that have no associated JIRA issue should be named following this pattern: `[hotfix] [docs] Fix typo in event time introduction` or `[hotfix] [javadocs] Expand JavaDoc for PuncuatedWatermarkGenerator`.
   
     - Fill out the template below to describe the changes contributed by the pull request. That will give reviewers the context they need to do the review.
     
     - Make sure that the change passes the automated tests, i.e., `mvn clean verify` passes. You can set up Azure Pipelines CI to do that following [this guide](https://cwiki.apache.org/confluence/display/FLINK/Azure+Pipelines#AzurePipelines-Tutorial:SettingupAzurePipelinesforaforkoftheFlinkrepository).
   
     - Each pull request should address only one issue, not mix up code from multiple issues.
     
     - Each commit in the pull request has a meaningful commit message (including the JIRA id)
   
     - Once all items of the checklist are addressed, remove the above text and this checklist, leaving only the filled out template below.
   
   
   **(The sections below can be removed for hotfixes of typos)**
   -->
   
   ## What is the purpose of the change
   To make Hive dialect support set variables.
   
   
   ## Brief change log
     - When it's for set command, extract the key and value, and then set the value according what kind of variable it belongs.
   
   
   ## Verifying this change
   UT
   
   
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): no
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: no
     - The serializers:  no
     - The runtime per-record code paths (performance sensitive): no
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: no
     - The S3 file system connector: no
   
   ## Documentation
   
     - Does this pull request introduce a new feature? no
     - If yes, how is the feature documented? N/A
   


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


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

Posted by GitBox <gi...@apache.org>.
fsk119 commented on code in PR #20000:
URL: https://github.com/apache/flink/pull/20000#discussion_r927266121


##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/copy/HiveSetProcessor.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 {@link org.apache.hadoop.hive.ql.processors.SetProcessor}. */

Review Comment:
   Sorry. I made a mistake. It's better use `{@link SetProcessor}`



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


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

Posted by GitBox <gi...@apache.org>.
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


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

Posted by GitBox <gi...@apache.org>.
wuchong commented on PR #20000:
URL: https://github.com/apache/flink/pull/20000#issuecomment-1192328719

   The compile phase is failed. 


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


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

Posted by GitBox <gi...@apache.org>.
luoyuxia commented on code in PR #20000:
URL: https://github.com/apache/flink/pull/20000#discussion_r933919454


##########
flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectITCase.java:
##########
@@ -918,6 +918,72 @@ public void testMacro() throws Exception {
                         badMacroName);
     }
 
+    @Test
+    public void testSetCommand() throws Exception {
+        // test set system:
+        tableEnv.executeSql("set system:xxx=5");
+        assertThat(System.getProperty("xxx")).isEqualTo("5");
+
+        // test set hiveconf:
+        tableEnv.executeSql("set hiveconf:yyy=${system:xxx}");
+        assertThat(hiveCatalog.getHiveConf().get("yyy")).isEqualTo("5");
+
+        // test set hivevar:
+        tableEnv.executeSql("set hivevar:a=1");
+        tableEnv.executeSql("set hiveconf:zzz=${hivevar:a}");
+        assertThat(hiveCatalog.getHiveConf().get("zzz")).isEqualTo("1");
+
+        // test the hivevar still exists when we renew the sql parser
+        tableEnv.getConfig().setSqlDialect(SqlDialect.DEFAULT);
+        tableEnv.executeSql("show tables");
+        tableEnv.getConfig().setSqlDialect(SqlDialect.HIVE);
+        tableEnv.executeSql("set hiveconf:zzz1=${hivevar:a}");

Review Comment:
   1: We maynot use `set` statement to switch dialect directly for the `SetOperation` is executed in SqlClient.
   2: The Flink's `SetOperation` will be executed in SqlClient , so we can't set it in tableEnv. But the test [set.q](https://github.com/apache/flink/blob/master/flink-table/flink-sql-client/src/test/resources/sql/set.q) in flink-sql-client has covered the case that set flink config in HiveDialect.



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


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

Posted by GitBox <gi...@apache.org>.
flinkbot commented on PR #20000:
URL: https://github.com/apache/flink/pull/20000#issuecomment-1158504230

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "18202b0c22d4f4781b2ba71ae1868face56bedf3",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "18202b0c22d4f4781b2ba71ae1868face56bedf3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 18202b0c22d4f4781b2ba71ae1868face56bedf3 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

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

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


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

Posted by GitBox <gi...@apache.org>.
fsk119 commented on code in PR #20000:
URL: https://github.com/apache/flink/pull/20000#discussion_r928861091


##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/HiveParser.java:
##########
@@ -82,6 +88,7 @@ public class HiveParser extends ParserImpl {
     private static final Method getCurrentTSMethod =
             HiveReflectionUtils.tryGetMethod(
                     SessionState.class, "getQueryCurrentTimestamp", new Class[0]);
+    private static final String HIVE_VARIABLE_PREFIX = "__hive.variable__";

Review Comment:
   It's better to move to HiveInternalOptions and describe this in the descriptions.
   
   I think it's better to use `__hive.variables__`? WDYT?



##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/HiveParser.java:
##########
@@ -207,6 +225,66 @@ public List<Operation> parse(String statement) {
         }
     }
 
+    private Optional<Operation> tryProcessHiveNonSqlStatement(HiveConf hiveConf, String statement) {
+        String[] commandTokens = statement.split("\\s+");
+        HiveCommand hiveCommand = HiveCommand.find(commandTokens);
+        if (hiveCommand != null) {
+            if (hiveCommand == HiveCommand.SET) {
+                return Optional.of(
+                        processSetCmd(
+                                hiveConf, statement.substring(commandTokens[0].length()).trim()));
+            } else {
+                throw new UnsupportedOperationException();
+            }
+        }
+        return Optional.empty();
+    }
+
+    private Operation processSetCmd(HiveConf hiveConf, String setCmdArgs) {
+        String nwcmd = setCmdArgs.trim();
+        // the set command may end with ";" as it won't be removed by Flink SQL CLI,
+        // so, we need to remove ";"
+        if (nwcmd.endsWith(";")) {
+            nwcmd = nwcmd.substring(0, nwcmd.length() - 1);
+        }
+
+        if (nwcmd.equals("")) {
+            String options =
+                    HiveSetProcessor.dumpOptions(
+                            hiveConf.getChangedProperties(), hiveConf, hiveVariables);
+            // todo show the options
+            throw new UnsupportedOperationException("Command 'set' isn't supported currently.");
+        }
+        if (nwcmd.equals("-v")) {
+            String options =
+                    HiveSetProcessor.dumpOptions(
+                            hiveConf.getAllProperties(), hiveConf, hiveVariables);
+            throw new UnsupportedOperationException("Command 'set -v' isn't supported currently.");
+        }
+
+        String[] part = new String[2];
+        int eqIndex = nwcmd.indexOf('=');
+        if (nwcmd.contains("=")) {
+            if (eqIndex == nwcmd.length() - 1) { // x=
+                part[0] = nwcmd.substring(0, nwcmd.length() - 1);
+                part[1] = "";
+            } else { // x=y
+                part[0] = nwcmd.substring(0, eqIndex).trim();
+                part[1] = nwcmd.substring(eqIndex + 1).trim();
+            }
+            if (part[0].equals("silent")) {
+                throw new UnsupportedOperationException("Unsupported command 'set silent'.");
+            }
+            HiveSetProcessor.setVariable(hiveConf, tableConfig, hiveVariables, part[0], part[1]);

Review Comment:
   It better to let SessionContext to do the set.



##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/HiveParser.java:
##########
@@ -207,6 +225,66 @@ public List<Operation> parse(String statement) {
         }
     }
 
+    private Optional<Operation> tryProcessHiveNonSqlStatement(HiveConf hiveConf, String statement) {
+        String[] commandTokens = statement.split("\\s+");
+        HiveCommand hiveCommand = HiveCommand.find(commandTokens);
+        if (hiveCommand != null) {
+            if (hiveCommand == HiveCommand.SET) {
+                return Optional.of(
+                        processSetCmd(
+                                hiveConf, statement.substring(commandTokens[0].length()).trim()));
+            } else {
+                throw new UnsupportedOperationException();

Review Comment:
   nit: It's better to tell users what is unsupported.



##########
flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectITCase.java:
##########
@@ -903,6 +904,36 @@ public void testMacro() throws Exception {
                         badMacroName);
     }
 
+    @Test
+    public void testSetCommand() throws Exception {
+        // test set system:
+        tableEnv.executeSql("set system:xxx=5");
+        assertThat(System.getProperty("xxx")).isEqualTo("5");
+        // test set hiveconf:
+        tableEnv.executeSql("set hiveconf:yyy=${system:xxx}");
+        assertThat(hiveCatalog.getHiveConf().get("yyy")).isEqualTo("5");
+        // test set hivevar:
+        tableEnv.executeSql("set hivevar:a=1");
+        tableEnv.executeSql("set hiveconf:zzz=${hivevar:a}");

Review Comment:
   I think we should renew a sql parser to test whether the test config memorizes hive-vars.
   
   For example,
   
   ```
           tableEnv.getConfig().setSqlDialect(SqlDialect.DEFAULT);
           tableEnv.executeSql("show tables");
           tableEnv.getConfig().setSqlDialect(SqlDialect.HIVE);
   ```



##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/HiveParser.java:
##########
@@ -207,6 +225,66 @@ public List<Operation> parse(String statement) {
         }
     }
 
+    private Optional<Operation> tryProcessHiveNonSqlStatement(HiveConf hiveConf, String statement) {
+        String[] commandTokens = statement.split("\\s+");
+        HiveCommand hiveCommand = HiveCommand.find(commandTokens);
+        if (hiveCommand != null) {
+            if (hiveCommand == HiveCommand.SET) {
+                return Optional.of(
+                        processSetCmd(
+                                hiveConf, statement.substring(commandTokens[0].length()).trim()));
+            } else {
+                throw new UnsupportedOperationException();
+            }
+        }
+        return Optional.empty();
+    }
+
+    private Operation processSetCmd(HiveConf hiveConf, String setCmdArgs) {
+        String nwcmd = setCmdArgs.trim();
+        // the set command may end with ";" as it won't be removed by Flink SQL CLI,
+        // so, we need to remove ";"
+        if (nwcmd.endsWith(";")) {
+            nwcmd = nwcmd.substring(0, nwcmd.length() - 1);
+        }
+
+        if (nwcmd.equals("")) {
+            String options =
+                    HiveSetProcessor.dumpOptions(
+                            hiveConf.getChangedProperties(), hiveConf, hiveVariables);

Review Comment:
   remove if we don't support this.



##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/copy/HiveSetProcessor.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.flink.table.api.TableConfig;
+
+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.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+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 {@link org.apache.hadoop.hive.ql.processors.SetProcessor}. */
+public class HiveSetProcessor {
+
+    private static final String[] PASSWORD_STRINGS = new String[] {"password", "paswd", "pswd"};
+
+    public static void setVariable(
+            HiveConf hiveConf,
+            TableConfig tableConfig,
+            Map<String, String> hiveVariables,
+            String varname,
+            String varvalue) {
+        if (varname.startsWith(ENV_PREFIX)) {
+            throw new UnsupportedOperationException("env:* variables can not be set.");
+        } else if (varname.startsWith(SYSTEM_PREFIX)) {
+            String propName = varname.substring(SYSTEM_PREFIX.length());
+            System.getProperties()
+                    .setProperty(
+                            propName,
+                            new VariableSubstitution(() -> hiveVariables)
+                                    .substitute(hiveConf, varvalue));
+        } else if (varname.startsWith(HIVECONF_PREFIX)) {
+            String propName = varname.substring(HIVECONF_PREFIX.length());
+            setConf(hiveConf, hiveVariables, varname, propName, varvalue);
+        } else if (varname.startsWith(HIVEVAR_PREFIX)) {
+            String propName = varname.substring(HIVEVAR_PREFIX.length());
+            hiveVariables.put(
+                    propName,
+                    new VariableSubstitution(() -> hiveVariables).substitute(hiveConf, varvalue));
+        } else if (varname.startsWith(METACONF_PREFIX)) {
+            String propName = varname.substring(METACONF_PREFIX.length());
+            try {
+                Hive hive = Hive.get(hiveConf);
+                hive.setMetaConf(
+                        propName,
+                        new VariableSubstitution(() -> hiveVariables)
+                                .substitute(hiveConf, varvalue));
+            } catch (HiveException e) {
+                throw new FlinkHiveException(
+                        String.format("'SET %s=%s' FAILED.", varname, varvalue), e);
+            }
+        } else {
+            setConf(hiveConf, hiveVariables, varname, varname, varvalue);
+            // we also try to set the value to Flink's table config. Otherwise, we have no way to
+            // change the table config of Flink when using Hive dialect.
+            String value =
+                    new VariableSubstitution(() -> hiveVariables).substitute(hiveConf, varvalue);
+            tableConfig.set(varname, value);
+        }
+    }
+
+    private static void setConf(
+            HiveConf hiveConf,
+            Map<String, String> hiveVariables,
+            String varname,
+            String key,
+            String varvalue) {
+        String value = new VariableSubstitution(() -> hiveVariables).substitute(hiveConf, varvalue);
+        if (hiveConf.getBoolVar(HiveConf.ConfVars.HIVECONFVALIDATION)) {
+            HiveConf.ConfVars confVars = HiveConf.getConfVars(key);
+            if (confVars != null) {
+                if (!confVars.isType(value)) {
+                    String message =
+                            String.format(
+                                    "'SET %s=%s' FAILED because %s expects %s type value.",
+                                    varname, varvalue, key, confVars.typeString());
+                    throw new IllegalArgumentException(message);
+                }
+                String fail = confVars.validate(value);
+                if (fail != null) {
+                    String message =
+                            String.format(
+                                    "'SET %s=%s' FAILED in validation : %s.",
+                                    varname, varvalue, fail);
+                    throw new IllegalArgumentException(message);
+                }
+            }
+        }
+        hiveConf.verifyAndSet(key, value);
+    }
+
+    public static String getVariable(

Review Comment:
   Do you want to support to show variables using `SET`? If not, I think we can add this when needs.



##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/HiveParser.java:
##########
@@ -207,6 +225,66 @@ public List<Operation> parse(String statement) {
         }
     }
 
+    private Optional<Operation> tryProcessHiveNonSqlStatement(HiveConf hiveConf, String statement) {
+        String[] commandTokens = statement.split("\\s+");
+        HiveCommand hiveCommand = HiveCommand.find(commandTokens);
+        if (hiveCommand != null) {
+            if (hiveCommand == HiveCommand.SET) {
+                return Optional.of(
+                        processSetCmd(
+                                hiveConf, statement.substring(commandTokens[0].length()).trim()));
+            } else {
+                throw new UnsupportedOperationException();
+            }
+        }
+        return Optional.empty();
+    }
+
+    private Operation processSetCmd(HiveConf hiveConf, String setCmdArgs) {
+        String nwcmd = setCmdArgs.trim();
+        // the set command may end with ";" as it won't be removed by Flink SQL CLI,
+        // so, we need to remove ";"
+        if (nwcmd.endsWith(";")) {
+            nwcmd = nwcmd.substring(0, nwcmd.length() - 1);
+        }
+
+        if (nwcmd.equals("")) {
+            String options =
+                    HiveSetProcessor.dumpOptions(
+                            hiveConf.getChangedProperties(), hiveConf, hiveVariables);
+            // todo show the options
+            throw new UnsupportedOperationException("Command 'set' isn't supported currently.");
+        }
+        if (nwcmd.equals("-v")) {
+            String options =
+                    HiveSetProcessor.dumpOptions(
+                            hiveConf.getAllProperties(), hiveConf, hiveVariables);
+            throw new UnsupportedOperationException("Command 'set -v' isn't supported currently.");
+        }
+
+        String[] part = new String[2];
+        int eqIndex = nwcmd.indexOf('=');
+        if (nwcmd.contains("=")) {
+            if (eqIndex == nwcmd.length() - 1) { // x=
+                part[0] = nwcmd.substring(0, nwcmd.length() - 1);
+                part[1] = "";
+            } else { // x=y
+                part[0] = nwcmd.substring(0, eqIndex).trim();
+                part[1] = nwcmd.substring(eqIndex + 1).trim();
+            }
+            if (part[0].equals("silent")) {
+                throw new UnsupportedOperationException("Unsupported command 'set silent'.");
+            }
+            HiveSetProcessor.setVariable(hiveConf, tableConfig, hiveVariables, part[0], part[1]);
+            return new NopOperation();
+        }
+
+        // todo show the option
+        String option = HiveSetProcessor.getVariable(hiveConf, hiveVariables, nwcmd);
+        // for the variable
+        throw new UnsupportedOperationException("Unsupported SET command which misses '='.");

Review Comment:
   Do we need this? I think we can 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.

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

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


[GitHub] [flink] wuchong merged pull request #20000: [FLINK-28096][hive] Hive dialect support set variables

Posted by GitBox <gi...@apache.org>.
wuchong merged PR #20000:
URL: https://github.com/apache/flink/pull/20000


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


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

Posted by GitBox <gi...@apache.org>.
wuchong commented on code in PR #20000:
URL: https://github.com/apache/flink/pull/20000#discussion_r933836257


##########
flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectITCase.java:
##########
@@ -918,6 +918,72 @@ public void testMacro() throws Exception {
                         badMacroName);
     }
 
+    @Test
+    public void testSetCommand() throws Exception {
+        // test set system:
+        tableEnv.executeSql("set system:xxx=5");
+        assertThat(System.getProperty("xxx")).isEqualTo("5");
+
+        // test set hiveconf:
+        tableEnv.executeSql("set hiveconf:yyy=${system:xxx}");
+        assertThat(hiveCatalog.getHiveConf().get("yyy")).isEqualTo("5");
+
+        // test set hivevar:
+        tableEnv.executeSql("set hivevar:a=1");
+        tableEnv.executeSql("set hiveconf:zzz=${hivevar:a}");
+        assertThat(hiveCatalog.getHiveConf().get("zzz")).isEqualTo("1");
+
+        // test the hivevar still exists when we renew the sql parser
+        tableEnv.getConfig().setSqlDialect(SqlDialect.DEFAULT);
+        tableEnv.executeSql("show tables");
+        tableEnv.getConfig().setSqlDialect(SqlDialect.HIVE);
+        tableEnv.executeSql("set hiveconf:zzz1=${hivevar:a}");

Review Comment:
   1. Use `set` statement to switch dialect? 
   2. Set a flink config under hive dialect and check it works.



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