You are viewing a plain text version of this content. The canonical link for it is here.
Posted to gitbox@hive.apache.org by "akshat0395 (via GitHub)" <gi...@apache.org> on 2023/05/31 17:16:34 UTC

[GitHub] [hive] akshat0395 commented on a diff in pull request #4060: HIVE-27032: Introduce liquibase for HMS schema evolution

akshat0395 commented on code in PR #4060:
URL: https://github.com/apache/hive/pull/4060#discussion_r1212044143


##########
common/src/java/org/apache/hadoop/hive/conf/HiveConf.java:
##########
@@ -1228,7 +1228,7 @@ public static enum ConfVars {
         + "To enable auto create also set hive.metastore.schema.verification=false. Auto creation is not "
         + "recommended for production use cases, run schematool command instead." ),
     /**
-     * @deprecated Use MetastoreConf.SCHEMA_VERIFICATION
+     * @deprecated No longer in user, neither {@link org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars#SCHEMA_VERIFICATION}

Review Comment:
   nit: Typo in user, should be use



##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/tools/schematool/liquibase/XmlScriptScanner.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.hadoop.hive.metastore.tools.schematool.liquibase;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hive.metastore.HiveMetaException;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * This {@link ScriptScanner} implementation call can scan Liquibase XML format changesets for tables.
+ */
+class XmlScriptScanner implements ScriptScanner {
+
+  private final Pattern pattern = Pattern.compile("<createTable(?:.|\\s)*?tableName=\"(.+?)\"|<renameTable(?:.|\\s)*?oldTableName=\"(.+?)\"(?:.|\\s)*?newTableName=\"(.+?)\"|<dropTable(?:.|\\s)*?tableName=\"(.+?)\"");
+
+  /**
+   * @param dbType not in use
+   */
+  @Override
+  public void findTablesInScript(String scriptPath, String dbType, Set<String> tableList) throws HiveMetaException {

Review Comment:
   dbType param is not used here, can this be removed



##########
beeline/src/java/org/apache/hive/beeline/schematool/tasks/BeelineScriptExecutor.java:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.hive.beeline.schematool.tasks;
+
+import org.apache.commons.io.output.NullOutputStream;
+import org.apache.hadoop.hive.metastore.tools.schematool.CommandBuilder;
+import org.apache.hadoop.hive.metastore.tools.schematool.HiveSchemaHelper;
+import org.apache.hadoop.hive.metastore.tools.schematool.commandparser.NestedScriptParser;
+import org.apache.hadoop.hive.metastore.tools.schematool.scriptexecution.ScriptExecutor;
+import org.apache.hive.beeline.BeeLine;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.PrintStream;
+
+/**
+ * {@link BeeLine} based {@link ScriptExecutor} implementation. Able to execute the given scripts using {@link BeeLine} internally.
+ * Can be used to execute scripts against the Hive schema.
+ */
+class BeelineScriptExecutor implements ScriptExecutor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BeelineScriptExecutor.class);
+  private static final String LINE_SEPARATOR = "line.separator";
+
+  private final NestedScriptParser dbCommandParser;
+  private final CommandBuilder commandBuilder;
+
+  @Override
+  public void execSql(String scriptDir, String sqlScriptFile) throws IOException {
+    // expand the nested script
+    // If the metaDbType is set, this is setting up the information
+    // schema in Hive. That specifically means that the sql commands need
+    // to be adjusted for the underlying RDBMS (correct quotation
+    // strings, etc).
+    String sqlCommands = dbCommandParser.buildCommand(scriptDir, sqlScriptFile, true);
+    File tmpFile = File.createTempFile("schematool", ".sql");
+    tmpFile.deleteOnExit();
+
+    // write out the buffer into a file. Add beeline commands for autocommit and close
+    FileWriter fstream = new FileWriter(tmpFile.getPath());
+    try (BufferedWriter out = new BufferedWriter(fstream)) {
+      if (!commandBuilder.getConnectionInfo().getDbType().equalsIgnoreCase(HiveSchemaHelper.DB_HIVE)) {
+        out.write("!autocommit off" + System.getProperty(LINE_SEPARATOR));
+        out.write(sqlCommands);
+        out.write("!commit" + System.getProperty(LINE_SEPARATOR));
+      } else {
+        out.write("!autocommit on" + System.getProperty(LINE_SEPARATOR));
+        out.write(sqlCommands);
+      }
+      out.write("!closeall" + System.getProperty(LINE_SEPARATOR));
+    }
+    execSql(tmpFile.getPath());
+  }
+
+  public void execSql(String sqlScriptFile) throws IOException {
+    // run the script using Beeline
+    try (BeeLine beeLine = new BeeLine()) {
+      if (!commandBuilder.isVerbose()) {
+        beeLine.setOutputStream(new PrintStream(new NullOutputStream()));
+        beeLine.getOpts().setSilent(true);
+      }
+      beeLine.getOpts().setAllowMultiLineCommand(false);
+      beeLine.getOpts().setIsolation("TRANSACTION_READ_COMMITTED");

Review Comment:
   We should create an enum/constant for this, WDYT @veghlaci05 



##########
standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/SchemaInfo.java:
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.hadoop.hive.metastore;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.tools.schematool.HiveSchemaHelper;
+
+import java.io.File;
+import java.util.List;
+
+/**
+ * Provides information about the HMS or Hive schema
+ */
+public abstract class SchemaInfo {
+
+  /**
+   * This must be updated accordingly in case of breaking schema changes. 
+   * For example: adding a new column to an HMS table.
+   */
+  private static final String MIN_HMS_SCHEMA_VERSION = "4.0.0";

Review Comment:
   Should we make these configurable? @veghlaci05 



-- 
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: gitbox-unsubscribe@hive.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org