Return-Path: X-Original-To: apmail-hive-commits-archive@www.apache.org Delivered-To: apmail-hive-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 0537B18F7A for ; Thu, 25 Feb 2016 00:33:22 +0000 (UTC) Received: (qmail 93433 invoked by uid 500); 25 Feb 2016 00:33:21 -0000 Delivered-To: apmail-hive-commits-archive@hive.apache.org Received: (qmail 93386 invoked by uid 500); 25 Feb 2016 00:33:21 -0000 Mailing-List: contact commits-help@hive.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hive-dev@hive.apache.org Delivered-To: mailing list commits@hive.apache.org Received: (qmail 93375 invoked by uid 99); 25 Feb 2016 00:33:21 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 25 Feb 2016 00:33:21 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 99657E8EC9; Thu, 25 Feb 2016 00:33:21 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: vikram@apache.org To: commits@hive.apache.org Message-Id: <39141c45fc67468f9f0a47ea3ce1ca80@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: hive git commit: HIVE-12064: prevent transactional=false (Wei Zhang via Vikram Dixit K) Date: Thu, 25 Feb 2016 00:33:21 +0000 (UTC) Repository: hive Updated Branches: refs/heads/branch-2.0 f8f201450 -> c8ed4fedf HIVE-12064: prevent transactional=false (Wei Zhang via Vikram Dixit K) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/c8ed4fed Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/c8ed4fed Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/c8ed4fed Branch: refs/heads/branch-2.0 Commit: c8ed4fedfccbd3bd10b588021826e5e4955895a9 Parents: f8f2014 Author: vikram Authored: Wed Feb 24 16:26:24 2016 -0800 Committer: vikram Committed: Wed Feb 24 16:26:24 2016 -0800 ---------------------------------------------------------------------- .../streaming/mutate/StreamingTestUtils.java | 9 +- .../streaming/mutate/TestMutations.java | 6 +- .../hive/metastore/TestHiveMetaStore.java | 128 ++++++++++++- .../hive/ql/txn/compactor/TestCompactor.java | 3 +- .../hadoop/hive/metastore/HiveMetaStore.java | 1 + .../TransactionalValidationListener.java | 179 +++++++++++++++++++ .../queries/clientnegative/create_not_acid.q | 6 + .../clientnegative/delete_not_bucketed.q | 7 - .../clientnegative/update_not_bucketed.q | 7 - .../clientnegative/create_not_acid.q.out | 5 + .../clientnegative/delete_not_bucketed.q.out | 9 - .../clientnegative/update_not_bucketed.q.out | 9 - 12 files changed, 330 insertions(+), 39 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hive/blob/c8ed4fed/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/StreamingTestUtils.java ---------------------------------------------------------------------- diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/StreamingTestUtils.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/StreamingTestUtils.java index e874877f..87a72b5 100644 --- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/StreamingTestUtils.java +++ b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/StreamingTestUtils.java @@ -45,7 +45,7 @@ import org.apache.hadoop.hive.metastore.api.SerDeInfo; import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.txn.TxnDbUtil; -import org.apache.hadoop.hive.ql.io.HiveInputFormat; +import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat; import org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat; import org.apache.hadoop.hive.ql.io.orc.OrcSerde; import org.apache.hadoop.hive.serde.serdeConstants; @@ -183,7 +183,7 @@ public class StreamingTestUtils { table.setParameters(tableParams); sd = new StorageDescriptor(); - sd.setInputFormat(HiveInputFormat.class.getName()); + sd.setInputFormat(OrcInputFormat.class.getName()); sd.setOutputFormat(OrcOutputFormat.class.getName()); sd.setNumBuckets(1); table.setSd(sd); @@ -207,6 +207,11 @@ public class StreamingTestUtils { return this; } + public TableBuilder bucketCols(List columnNames) { + sd.setBucketCols(columnNames); + return this; + } + public TableBuilder addColumn(String columnName, String columnType) { columnNames.add(columnName); columnTypes.add(columnType); http://git-wip-us.apache.org/repos/asf/hive/blob/c8ed4fed/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/TestMutations.java ---------------------------------------------------------------------- diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/TestMutations.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/TestMutations.java index 703cef6..f1de1df 100644 --- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/TestMutations.java +++ b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/TestMutations.java @@ -92,12 +92,14 @@ public class TestMutations { .name("partitioned") .addColumn("id", "int") .addColumn("msg", "string") - .partitionKeys("continent", "country"); + .partitionKeys("continent", "country") + .bucketCols(Collections.singletonList("string")); unpartitionedTableBuilder = tableBuilder(database) .name("unpartitioned") .addColumn("id", "int") - .addColumn("msg", "string"); + .addColumn("msg", "string") + .bucketCols(Collections.singletonList("string")); } @Test http://git-wip-us.apache.org/repos/asf/hive/blob/c8ed4fed/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java ---------------------------------------------------------------------- diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java index 8601df0..a910a27 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java @@ -23,6 +23,7 @@ import java.sql.PreparedStatement; import java.sql.SQLException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.LinkedHashMap; @@ -69,6 +70,7 @@ import org.apache.hadoop.hive.metastore.api.StringColumnStatsData; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.api.Type; import org.apache.hadoop.hive.metastore.api.UnknownDBException; +import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.io.HiveInputFormat; import org.apache.hadoop.hive.ql.io.HiveOutputFormat; @@ -77,6 +79,7 @@ import org.apache.hadoop.hive.serde.serdeConstants; import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe; import org.apache.hadoop.util.StringUtils; import org.apache.thrift.TException; +import org.junit.Assert; import org.junit.Test; import com.google.common.collect.Lists; @@ -2182,7 +2185,7 @@ public abstract class TestHiveMetaStore extends TestCase { } assertNotNull(me); assertTrue("NoSuchObject exception", me.getMessage().contains( - "database/table does not exist")); + "invDBName.invTableName table not found")); client.dropTable(dbName, tblName); client.dropDatabase(dbName); @@ -2803,6 +2806,120 @@ public abstract class TestHiveMetaStore extends TestCase { createTable(dbName, tableName, null, null, null, sd, 0); } + @Test + public void testTransactionalValidation() throws Throwable { + String tblName = "acidTable"; + String owner = "acid"; + Map fields = new HashMap(); + fields.put("name", serdeConstants.STRING_TYPE_NAME); + fields.put("income", serdeConstants.INT_TYPE_NAME); + + Type type = createType("Person", fields); + + Map params = new HashMap(); + params.put("transactional", ""); + + Map serdParams = new HashMap(); + serdParams.put(serdeConstants.SERIALIZATION_FORMAT, "1"); + StorageDescriptor sd = createStorageDescriptor(tblName, type.getFields(), params, serdParams); + sd.setNumBuckets(0); + sd.unsetBucketCols(); + + /// CREATE TABLE scenarios + + // Fail - No "transactional" property is specified + try { + Table t = createTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tblName, owner, params, null, sd, 0); + Assert.assertTrue("Expected exception", false); + } catch (MetaException e) { + Assert.assertEquals("'transactional' property of TBLPROPERTIES may only have value 'true'", e.getMessage()); + } + + // Fail - "transactional" property is set to an invalid value + try { + params.clear(); + params.put("transactional", "foobar"); + Table t = createTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tblName, owner, params, null, sd, 0); + Assert.assertTrue("Expected exception", false); + } catch (MetaException e) { + Assert.assertEquals("'transactional' property of TBLPROPERTIES may only have value 'true'", e.getMessage()); + } + + // Fail - "transactional" is set to true, but the table is not bucketed + try { + params.clear(); + params.put("transactional", "true"); + Table t = createTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tblName, owner, params, null, sd, 0); + Assert.assertTrue("Expected exception", false); + } catch (MetaException e) { + Assert.assertEquals("The table must be bucketed and stored using an ACID compliant format (such as ORC)", e.getMessage()); + } + + // Fail - "transactional" is set to true, and the table is bucketed, but doesn't use ORC + try { + params.clear(); + params.put("transactional", "true"); + List bucketCols = new ArrayList(); + bucketCols.add("income"); + sd.setBucketCols(bucketCols); + Table t = createTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tblName, owner, params, null, sd, 0); + Assert.assertTrue("Expected exception", false); + } catch (MetaException e) { + Assert.assertEquals("The table must be bucketed and stored using an ACID compliant format (such as ORC)", e.getMessage()); + } + + // Succeed - "transactional" is set to true, and the table is bucketed, and uses ORC + params.clear(); + params.put("transactional", "true"); + List bucketCols = new ArrayList(); + bucketCols.add("income"); + sd.setBucketCols(bucketCols); + sd.setInputFormat("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"); + sd.setOutputFormat("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat"); + Table t = createTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tblName, owner, params, null, sd, 0); + Assert.assertTrue("CREATE TABLE should succeed", "true".equals(t.getParameters().get(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL))); + + /// ALTER TABLE scenarios + + // Fail - trying to set "transactional" to "false" is not allowed + try { + params.clear(); + params.put("transactional", "false"); + t = new Table(); + t.setParameters(params); + client.alter_table(MetaStoreUtils.DEFAULT_DATABASE_NAME, tblName, t); + Assert.assertTrue("Expected exception", false); + } catch (MetaException e) { + Assert.assertEquals("TBLPROPERTIES with 'transactional'='true' cannot be unset", e.getMessage()); + } + + // Fail - trying to set "transactional" to "true" but doesn't satisfy bucketing and Input/OutputFormat requirement + try { + tblName += "1"; + params.clear(); + sd.unsetBucketCols(); + t = createTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tblName, owner, params, null, sd, 0); + params.put("transactional", "true"); + t.setParameters(params); + client.alter_table(MetaStoreUtils.DEFAULT_DATABASE_NAME, tblName, t); + Assert.assertTrue("Expected exception", false); + } catch (MetaException e) { + Assert.assertEquals("The table must be bucketed and stored using an ACID compliant format (such as ORC)", e.getMessage()); + } + + // Succeed - trying to set "transactional" to "true", and satisfies bucketing and Input/OutputFormat requirement + tblName += "2"; + params.clear(); + sd.setNumBuckets(1); + sd.setBucketCols(bucketCols); + t = createTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tblName, owner, params, null, sd, 0); + params.put("transactional", "true"); + t.setParameters(params); + t.setPartitionKeys(Collections.EMPTY_LIST); + client.alter_table(MetaStoreUtils.DEFAULT_DATABASE_NAME, tblName, t); + Assert.assertTrue("ALTER TABLE should succeed", "true".equals(t.getParameters().get(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL))); + } + private Table createTable(String dbName, String tblName, String owner, Map tableParams, Map partitionKeys, StorageDescriptor sd, int lastAccessTime) throws Exception { @@ -2829,6 +2946,15 @@ public abstract class TestHiveMetaStore extends TestCase { tbl.setLastAccessTime(lastAccessTime); client.createTable(tbl); + + if (isThriftClient) { + // the createTable() above does not update the location in the 'tbl' + // object when the client is a thrift client and ALTER TABLE relies + // on the location being present in the 'tbl' object - so get the table + // from the metastore + tbl = client.getTable(dbName, tblName); + } + return tbl; } http://git-wip-us.apache.org/repos/asf/hive/blob/c8ed4fed/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java ---------------------------------------------------------------------- diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java index 226a1fa..26f7c25 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java @@ -363,8 +363,7 @@ public class TestCompactor { executeStatementOnDriver("CREATE EXTERNAL TABLE " + tblNameStg + "(a INT, b STRING)" + " ROW FORMAT DELIMITED FIELDS TERMINATED BY '\\t' LINES TERMINATED BY '\\n'" + " STORED AS TEXTFILE" + - " LOCATION '" + stagingFolder.newFolder().toURI().getPath() + "'" + - " TBLPROPERTIES ('transactional'='true')", driver); + " LOCATION '" + stagingFolder.newFolder().toURI().getPath() + "'", driver); executeStatementOnDriver("load data local inpath '" + BASIC_FILE_NAME + "' overwrite into table " + tblNameStg, driver); http://git-wip-us.apache.org/repos/asf/hive/blob/c8ed4fed/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java ---------------------------------------------------------------------- diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java index 6b3f8d0..b1eb7ed 100644 --- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java +++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java @@ -388,6 +388,7 @@ public class HiveMetaStore extends ThriftHiveMetastore { preListeners = MetaStoreUtils.getMetaStoreListeners(MetaStorePreEventListener.class, hiveConf, hiveConf.getVar(HiveConf.ConfVars.METASTORE_PRE_EVENT_LISTENERS)); + preListeners.add(0, new TransactionalValidationListener(hiveConf)); listeners = MetaStoreUtils.getMetaStoreListeners(MetaStoreEventListener.class, hiveConf, hiveConf.getVar(HiveConf.ConfVars.METASTORE_EVENT_LISTENERS)); listeners.add(new SessionPropertiesListener(hiveConf)); http://git-wip-us.apache.org/repos/asf/hive/blob/c8ed4fed/metastore/src/java/org/apache/hadoop/hive/metastore/TransactionalValidationListener.java ---------------------------------------------------------------------- diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/TransactionalValidationListener.java b/metastore/src/java/org/apache/hadoop/hive/metastore/TransactionalValidationListener.java new file mode 100644 index 0000000..96158f8 --- /dev/null +++ b/metastore/src/java/org/apache/hadoop/hive/metastore/TransactionalValidationListener.java @@ -0,0 +1,179 @@ +/** + * 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.hadoop.conf.Configuration; +import org.apache.hadoop.hive.metastore.api.*; +import org.apache.hadoop.hive.metastore.events.PreAlterTableEvent; +import org.apache.hadoop.hive.metastore.events.PreCreateTableEvent; +import org.apache.hadoop.hive.metastore.events.PreEventContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +final class TransactionalValidationListener extends MetaStorePreEventListener { + public static final Logger LOG = LoggerFactory.getLogger(TransactionalValidationListener.class); + + TransactionalValidationListener(Configuration conf) { + super(conf); + } + + public void onEvent(PreEventContext context) throws MetaException, NoSuchObjectException, + InvalidOperationException { + switch (context.getEventType()) { + case CREATE_TABLE: + handle((PreCreateTableEvent) context); + break; + case ALTER_TABLE: + handle((PreAlterTableEvent) context); + break; + default: + //no validation required.. + } + } + + private void handle(PreAlterTableEvent context) throws MetaException { + handleAlterTableTransactionalProp(context); + } + + private void handle(PreCreateTableEvent context) throws MetaException { + handleCreateTableTransactionalProp(context); + } + + /** + * once a table is marked transactional, you cannot go back. Enforce this. + */ + private void handleAlterTableTransactionalProp(PreAlterTableEvent context) throws MetaException { + Table newTable = context.getNewTable(); + Map parameters = newTable.getParameters(); + if (parameters == null || parameters.isEmpty()) { + return; + } + Set keys = new HashSet<>(parameters.keySet()); + String transactionalValue = null; + boolean transactionalValuePresent = false; + for (String key : keys) { + if(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL.equalsIgnoreCase(key)) { + transactionalValuePresent = true; + transactionalValue = parameters.get(key); + parameters.remove(key); + } + } + if (transactionalValuePresent) { + //normalize prop name + parameters.put(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL, transactionalValue); + } + if ("true".equalsIgnoreCase(transactionalValue)) { + if (!conformToAcid(newTable)) { + throw new MetaException("The table must be bucketed and stored using an ACID compliant" + + " format (such as ORC)"); + } + return; + } + Table oldTable = context.getOldTable(); + String oldTransactionalValue = null; + for (String key : oldTable.getParameters().keySet()) { + if (hive_metastoreConstants.TABLE_IS_TRANSACTIONAL.equalsIgnoreCase(key)) { + oldTransactionalValue = oldTable.getParameters().get(key); + } + } + if (oldTransactionalValue == null ? transactionalValue == null + : oldTransactionalValue.equalsIgnoreCase(transactionalValue)) { + //this covers backward compat cases where this prop may have been set already + return; + } + // if here, there is attempt to set transactional to something other than 'true' + // and NOT the same value it was before + throw new MetaException("TBLPROPERTIES with 'transactional'='true' cannot be unset"); + } + + /** + * Normalize case and make sure: + * 1. 'true' is the only value to be set for 'transactional' (if set at all) + * 2. If set to 'true', we should also enforce bucketing and ORC format + */ + private void handleCreateTableTransactionalProp(PreCreateTableEvent context) throws MetaException { + Table newTable = context.getTable(); + Map parameters = newTable.getParameters(); + if (parameters == null || parameters.isEmpty()) { + return; + } + String transactionalValue = null; + boolean transactionalPropFound = false; + Set keys = new HashSet<>(parameters.keySet()); + for(String key : keys) { + if(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL.equalsIgnoreCase(key)) { + transactionalPropFound = true; + transactionalValue = parameters.get(key); + parameters.remove(key); + } + } + + if (!transactionalPropFound) { + return; + } + + if ("false".equalsIgnoreCase(transactionalValue)) { + // just drop transactional=false. For backward compatibility in case someone has scripts + // with transactional=false + LOG.info("'transactional'='false' is no longer a valid property and will be ignored"); + return; + } + + if ("true".equalsIgnoreCase(transactionalValue)) { + if (!conformToAcid(newTable)) { + throw new MetaException("The table must be bucketed and stored using an ACID compliant" + + " format (such as ORC)"); + } + + // normalize prop name + parameters.put(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL, Boolean.TRUE.toString()); + return; + } + + // transactional prop is found, but the value is not in expected range + throw new MetaException("'transactional' property of TBLPROPERTIES may only have value 'true'"); + } + + // Check if table is bucketed and InputFormatClass/OutputFormatClass should implement + // AcidInputFormat/AcidOutputFormat + private boolean conformToAcid(Table table) throws MetaException { + StorageDescriptor sd = table.getSd(); + if (sd.getBucketColsSize() < 1) { + return false; + } + + try { + Class inputFormatClass = Class.forName(sd.getInputFormat()); + Class outputFormatClass = Class.forName(sd.getOutputFormat()); + + if (inputFormatClass == null || outputFormatClass == null || + !Class.forName("org.apache.hadoop.hive.ql.io.AcidInputFormat").isAssignableFrom(inputFormatClass) || + !Class.forName("org.apache.hadoop.hive.ql.io.AcidOutputFormat").isAssignableFrom(outputFormatClass)) { + return false; + } + } catch (ClassNotFoundException e) { + throw new MetaException("Invalid input/output format for table"); + } + + return true; + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/hive/blob/c8ed4fed/ql/src/test/queries/clientnegative/create_not_acid.q ---------------------------------------------------------------------- diff --git a/ql/src/test/queries/clientnegative/create_not_acid.q b/ql/src/test/queries/clientnegative/create_not_acid.q new file mode 100644 index 0000000..8d6c9ac --- /dev/null +++ b/ql/src/test/queries/clientnegative/create_not_acid.q @@ -0,0 +1,6 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; + + +create table acid_notbucketed(a int, b varchar(128)) stored as orc TBLPROPERTIES ('transactional'='true'); + http://git-wip-us.apache.org/repos/asf/hive/blob/c8ed4fed/ql/src/test/queries/clientnegative/delete_not_bucketed.q ---------------------------------------------------------------------- diff --git a/ql/src/test/queries/clientnegative/delete_not_bucketed.q b/ql/src/test/queries/clientnegative/delete_not_bucketed.q deleted file mode 100644 index d575a8f..0000000 --- a/ql/src/test/queries/clientnegative/delete_not_bucketed.q +++ /dev/null @@ -1,7 +0,0 @@ -set hive.support.concurrency=true; -set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; - - -create table acid_notbucketed(a int, b varchar(128)) stored as orc TBLPROPERTIES ('transactional'='true'); - -delete from acid_notbucketed where a = 3; http://git-wip-us.apache.org/repos/asf/hive/blob/c8ed4fed/ql/src/test/queries/clientnegative/update_not_bucketed.q ---------------------------------------------------------------------- diff --git a/ql/src/test/queries/clientnegative/update_not_bucketed.q b/ql/src/test/queries/clientnegative/update_not_bucketed.q deleted file mode 100644 index d7d0da4..0000000 --- a/ql/src/test/queries/clientnegative/update_not_bucketed.q +++ /dev/null @@ -1,7 +0,0 @@ -set hive.support.concurrency=true; -set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; - - -create table acid_notbucketed(a int, b varchar(128)) partitioned by (ds string) stored as orc TBLPROPERTIES ('transactional'='true'); - -update acid_notbucketed set b = 'fred' where a = 3; http://git-wip-us.apache.org/repos/asf/hive/blob/c8ed4fed/ql/src/test/results/clientnegative/create_not_acid.q.out ---------------------------------------------------------------------- diff --git a/ql/src/test/results/clientnegative/create_not_acid.q.out b/ql/src/test/results/clientnegative/create_not_acid.q.out new file mode 100644 index 0000000..bb8f6c9 --- /dev/null +++ b/ql/src/test/results/clientnegative/create_not_acid.q.out @@ -0,0 +1,5 @@ +PREHOOK: query: create table acid_notbucketed(a int, b varchar(128)) stored as orc TBLPROPERTIES ('transactional'='true') +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@acid_notbucketed +FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. MetaException(message:The table must be bucketed and stored using an ACID compliant format (such as ORC)) http://git-wip-us.apache.org/repos/asf/hive/blob/c8ed4fed/ql/src/test/results/clientnegative/delete_not_bucketed.q.out ---------------------------------------------------------------------- diff --git a/ql/src/test/results/clientnegative/delete_not_bucketed.q.out b/ql/src/test/results/clientnegative/delete_not_bucketed.q.out deleted file mode 100644 index d0ba680..0000000 --- a/ql/src/test/results/clientnegative/delete_not_bucketed.q.out +++ /dev/null @@ -1,9 +0,0 @@ -PREHOOK: query: create table acid_notbucketed(a int, b varchar(128)) stored as orc TBLPROPERTIES ('transactional'='true') -PREHOOK: type: CREATETABLE -PREHOOK: Output: database:default -PREHOOK: Output: default@acid_notbucketed -POSTHOOK: query: create table acid_notbucketed(a int, b varchar(128)) stored as orc TBLPROPERTIES ('transactional'='true') -POSTHOOK: type: CREATETABLE -POSTHOOK: Output: database:default -POSTHOOK: Output: default@acid_notbucketed -FAILED: SemanticException [Error 10297]: Attempt to do update or delete on table acid_notbucketed that does not use an AcidOutputFormat or is not bucketed http://git-wip-us.apache.org/repos/asf/hive/blob/c8ed4fed/ql/src/test/results/clientnegative/update_not_bucketed.q.out ---------------------------------------------------------------------- diff --git a/ql/src/test/results/clientnegative/update_not_bucketed.q.out b/ql/src/test/results/clientnegative/update_not_bucketed.q.out deleted file mode 100644 index 8ebf41d..0000000 --- a/ql/src/test/results/clientnegative/update_not_bucketed.q.out +++ /dev/null @@ -1,9 +0,0 @@ -PREHOOK: query: create table acid_notbucketed(a int, b varchar(128)) partitioned by (ds string) stored as orc TBLPROPERTIES ('transactional'='true') -PREHOOK: type: CREATETABLE -PREHOOK: Output: database:default -PREHOOK: Output: default@acid_notbucketed -POSTHOOK: query: create table acid_notbucketed(a int, b varchar(128)) partitioned by (ds string) stored as orc TBLPROPERTIES ('transactional'='true') -POSTHOOK: type: CREATETABLE -POSTHOOK: Output: database:default -POSTHOOK: Output: default@acid_notbucketed -FAILED: SemanticException [Error 10297]: Attempt to do update or delete on table acid_notbucketed that does not use an AcidOutputFormat or is not bucketed