You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ra...@apache.org on 2023/01/05 18:34:05 UTC

[hive] branch master updated: HIVE-26837 CTLT with hive.create.as.external.legacy as true creates managed table instead of external table (#3854)

This is an automated email from the ASF dual-hosted git repository.

rameshkumar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
     new c7db232125e HIVE-26837 CTLT with hive.create.as.external.legacy as true creates managed table instead of external table (#3854)
c7db232125e is described below

commit c7db232125e753c76861ca816cdf8d5535dbaa7a
Author: Ramesh Kumar <ra...@gmail.com>
AuthorDate: Thu Jan 5 10:33:52 2023 -0800

    HIVE-26837 CTLT with hive.create.as.external.legacy as true creates managed table instead of external table (#3854)
---
 .../apache/hadoop/hive/ql/TestCtltExternalDir.java |  76 +++++++++++++++
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java     |   4 +
 .../clientpositive/ctlt_translate_external.q       |  20 ++++
 .../llap/ctlt_translate_external.q.out             | 108 +++++++++++++++++++++
 4 files changed, 208 insertions(+)

diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestCtltExternalDir.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestCtltExternalDir.java
new file mode 100644
index 00000000000..e46b2323a40
--- /dev/null
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestCtltExternalDir.java
@@ -0,0 +1,76 @@
+/*
+ * 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.ql;
+
+import java.io.File;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.utils.TestTxnDbUtil;
+import org.apache.hadoop.hive.ql.QTestMiniClusters.MiniClusterType;
+
+import org.junit.Assert;
+import org.junit.Test;
+import static org.junit.Assert.fail;
+
+public class TestCtltExternalDir extends BaseTestQueries {
+  public TestCtltExternalDir() {
+    File logDirFile = new File(logDir);
+    if (!(logDirFile.exists() || logDirFile.mkdirs())) {
+      fail("Could not create " + logDir);
+    }
+  }
+
+  @Test public void testCtltLocation() throws Exception {
+    String[] testNames = new String[] { "ctlt_translate_external.q" };
+
+    File[] qfiles = setupQFiles(testNames);
+
+    String whRootExternal = "/tmp/wh_ext";
+
+    QTestUtil qt = new QTestUtil(
+        QTestArguments.QTestArgumentsBuilder.instance().withOutDir(resDir + "/llap").withLogDir(logDir).withClusterType(MiniClusterType.LLAP_LOCAL).withConfDir(null).withInitScript("").withCleanupScript("")
+            .withLlapIo(false).build());
+
+    HiveConf hiveConf = qt.getConf();
+    hiveConf.setVar(ConfVars.HIVE_METASTORE_WAREHOUSE_EXTERNAL, whRootExternal);
+    TestTxnDbUtil.setConfValues(hiveConf);
+    TestTxnDbUtil.cleanDb(hiveConf);
+    TestTxnDbUtil.prepDb(hiveConf);
+    qt.postInit();
+    qt.newSession();
+    qt.setInputFile(qfiles[0]);
+    qt.clearTestSideEffects();
+
+    boolean success = QTestRunnerUtils.queryListRunnerSingleThreaded(qfiles, new QTestUtil[] { qt });
+    if (success) {
+      IMetaStoreClient hmsClient = new HiveMetaStoreClient(hiveConf);
+      Table table = hmsClient.getTable("default", "test_ext1");
+      FileSystem fs = FileSystem.get(hiveConf);
+      String location = table.getSd().getLocation();
+      Assert.assertEquals("Not an external table", "file:" + whRootExternal + "/test_ext1", location);
+    } else {
+      fail("One or more queries failed");
+    }
+  }
+}
\ No newline at end of file
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 57702036cdb..41a947dc985 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -13979,6 +13979,10 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
           updateDefaultTblProps(likeTable.getParameters(), tblProps, null);
         }
       }
+      if (likeTable.getTableType() == TableType.EXTERNAL_TABLE &&
+          HiveConf.getBoolVar(conf, ConfVars.CREATE_TABLE_AS_EXTERNAL)) {
+        isExt = true;
+      }
       CreateTableLikeDesc crtTblLikeDesc = new CreateTableLikeDesc(dbDotTab, isExt, isTemporary,
           storageFormat.getInputFormat(), storageFormat.getOutputFormat(), location,
           storageFormat.getSerde(), storageFormat.getSerdeProps(), tblProps, ifNotExists,
diff --git a/ql/src/test/queries/clientpositive/ctlt_translate_external.q b/ql/src/test/queries/clientpositive/ctlt_translate_external.q
new file mode 100644
index 00000000000..3319fa00738
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/ctlt_translate_external.q
@@ -0,0 +1,20 @@
+set hive.fetch.task.conversion=none;
+set hive.compute.query.using.stats=false;
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+set hive.strict.managed.tables=true;
+set hive.create.as.acid=true;
+set hive.create.as.insert.only=true;
+set hive.default.fileformat.managed=ORC;
+
+create table test_mm(empno int, name string) partitioned by(dept string) stored as orc;
+
+-- CTAS with external legacy config
+set hive.create.as.external.legacy=true;
+create table test_external like test_mm;
+desc formatted test_external;
+
+create external table test_ext(empno int, name string) stored as ORC;
+
+create table test_ext1 like test_ext;
+desc formatted test_ext1;
diff --git a/ql/src/test/results/clientpositive/llap/ctlt_translate_external.q.out b/ql/src/test/results/clientpositive/llap/ctlt_translate_external.q.out
new file mode 100644
index 00000000000..a5b6b69fa3e
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/ctlt_translate_external.q.out
@@ -0,0 +1,108 @@
+PREHOOK: query: create table test_mm(empno int, name string) partitioned by(dept string) stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@test_mm
+POSTHOOK: query: create table test_mm(empno int, name string) partitioned by(dept string) stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@test_mm
+PREHOOK: query: create table test_external like test_mm
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@test_external
+POSTHOOK: query: create table test_external like test_mm
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@test_external
+PREHOOK: query: desc formatted test_external
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@test_external
+POSTHOOK: query: desc formatted test_external
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@test_external
+# col_name            	data_type           	comment             
+empno               	int                 	                    
+name                	string              	                    
+	 	 
+# Partition Information	 	 
+# col_name            	data_type           	comment             
+dept                	string              	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	MANAGED_TABLE       	 
+Table Parameters:	 	 
+	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\"}
+	bucketing_version   	2                   
+	numFiles            	0                   
+	numPartitions       	0                   
+	numRows             	0                   
+	rawDataSize         	0                   
+	totalSize           	0                   
+	transactional       	true                
+	transactional_properties	default             
+#### A masked pattern was here ####
+	 	 
+# Storage Information	 	 
+SerDe Library:      	org.apache.hadoop.hive.ql.io.orc.OrcSerde	 
+InputFormat:        	org.apache.hadoop.hive.ql.io.orc.OrcInputFormat	 
+OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat	 
+Compressed:         	No                  	 
+Num Buckets:        	-1                  	 
+Bucket Columns:     	[]                  	 
+Sort Columns:       	[]                  	 
+PREHOOK: query: create external table test_ext(empno int, name string) stored as ORC
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@test_ext
+POSTHOOK: query: create external table test_ext(empno int, name string) stored as ORC
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@test_ext
+PREHOOK: query: create table test_ext1 like test_ext
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@test_ext1
+POSTHOOK: query: create table test_ext1 like test_ext
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@test_ext1
+PREHOOK: query: desc formatted test_ext1
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@test_ext1
+POSTHOOK: query: desc formatted test_ext1
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@test_ext1
+# col_name            	data_type           	comment             
+empno               	int                 	                    
+name                	string              	                    
+	 	 
+# Detailed Table Information	 	 
+Database:           	default             	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	EXTERNAL_TABLE      	 
+Table Parameters:	 	 
+	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"empno\":\"true\",\"name\":\"true\"}}
+	EXTERNAL            	TRUE                
+	bucketing_version   	2                   
+	numFiles            	0                   
+	numRows             	0                   
+	rawDataSize         	0                   
+	totalSize           	0                   
+#### A masked pattern was here ####
+	 	 
+# Storage Information	 	 
+SerDe Library:      	org.apache.hadoop.hive.ql.io.orc.OrcSerde	 
+InputFormat:        	org.apache.hadoop.hive.ql.io.orc.OrcInputFormat	 
+OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat	 
+Compressed:         	No                  	 
+Num Buckets:        	-1                  	 
+Bucket Columns:     	[]                  	 
+Sort Columns:       	[]                  	 
+Storage Desc Params:	 	 
+	serialization.format	1