diff --git common/src/java/org/apache/hadoop/hive/conf/HiveConf.java common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index 371cb0f..4c3b151 100644 --- common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -18,26 +18,6 @@ package org.apache.hadoop.hive.conf; -import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.IOException; -import java.io.InputStream; -import java.io.PrintStream; -import java.net.URL; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedHashSet; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Properties; -import java.util.Set; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -import javax.security.auth.login.LoginException; - import org.apache.commons.lang.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -47,6 +27,14 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Shell; +import javax.security.auth.login.LoginException; +import java.io.*; +import java.net.URL; +import java.util.*; +import java.util.Map.Entry; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + /** * Hive Configuration. */ @@ -692,6 +680,27 @@ HIVE_ZOOKEEPER_NAMESPACE("hive.zookeeper.namespace", "hive_zookeeper_namespace"), HIVE_ZOOKEEPER_CLEAN_EXTRA_NODES("hive.zookeeper.clean.extra.nodes", false), + // Transactions + HIVE_TXN_MANAGER("hive.txn.manager", + "org.apache.hadoop.hive.ql.lockmgr.DummyTxnManager"), + HIVE_TXN_JDBC_DRIVER("hive.txn.driver", ""), + HIVE_TXN_JDBC_CONNECT_STRING("hive.txn.connection.string", ""), + // time after which transactions are declared aborted if the client has + // not sent a heartbeat, in seconds. + HIVE_TXN_TIMEOUT("hive.txn.timeout", 300), + + // Maximum number of transactions that can be fetched in one call to + // open_txns(). + // Increasing this will decrease the number of delta files created when + // streaming data into Hive. But it will also increase the number of + // open transactions at any given time, possibly impacting read + // performance. + HIVE_TXN_MAX_OPEN_BATCH("hive.txn.max.open.batch", 1000), + + // Hack to handle setting up the transaction/lock database for qfile + // testing. + HIVE_TXN_TESTING("hive.txn.testing", false), + // For HBase storage handler HIVE_HBASE_WAL_ENABLED("hive.hbase.wal.enabled", true), diff --git metastore/if/hive_metastore.thrift metastore/if/hive_metastore.thrift index e4e816d..82e1e90 100755 --- metastore/if/hive_metastore.thrift +++ metastore/if/hive_metastore.thrift @@ -70,6 +70,31 @@ enum PartitionEventType { LOAD_DONE = 1, } +// Enums for transaction and lock management +enum TxnState { + COMMITTED = 1, + ABORTED = 2, + OPEN = 3, +} + +enum LockLevel { + DB = 1, + TABLE = 2, + PARTITION = 3, +} + +enum LockState { + ACQUIRED = 1, + WAITING = 2, + ABORT = 3, +} + +enum LockType { + SHARED_READ = 1, + SHARED_WRITE = 2, + EXCLUSIVE = 3, +} + struct HiveObjectRef{ 1: HiveObjectType objectType, 2: string dbName, @@ -300,6 +325,76 @@ struct AddPartitionsRequest { 5: optional bool needResult=true } +// Structs for transaction and locks +struct TxnInfo { + 1: required i64 id, + 2: required TxnState state, +} + +struct GetOpenTxnsInfoResponse { + 1: required i64 txn_high_water_mark, + 2: required list open_txns, +} + +struct GetOpenTxnsResponse { + 1: required i64 txn_high_water_mark, + 2: required set open_txns, +} + +struct OpenTxnRequest { + 1: required i32 num_txns, +} + +struct OpenTxnsResponse { + 1: required list txn_ids, +} + +struct AbortTxnRequest { + 1: required i64 txnid, +} + +struct CommitTxnRequest { + 1: required i64 txnid, +} + +struct LockComponent { + 1: required LockType type, + 2: required LockLevel level, + 3: required string dbname, + 4: required string tablename, + 5: required string partitionname, + 6: optional string lock_object_data, +} + +struct LockRequest { + 1: required list component, + 2: optional i64 txnid, +} + +struct LockResponse { + 1: required i64 lockid, + 2: required LockState state, +} + +struct CheckLockRequest { + 1: required i64 lockid, +} + +struct UnlockRequest { + 1: required i64 lockid, +} + +struct HeartbeatRequest { + 1: optional i64 lockid, + 2: optional i64 txnid +} + +struct CleanAbortedTransactionRequest { + 1: required string dbname, + 2: required string tablename, + 3: required string partitionname, +} + exception MetaException { 1: string message } @@ -348,6 +443,23 @@ exception InvalidInputException { 1: string message } +// Transaction and lock exceptions +exception NoSuchTxnException { + 1: string message +} + +exception TxnAbortedException { + 1: string message +} + +exception TxnOpenException { + 1: string message +} + +exception NoSuchLockException { + 1: string message +} + /** * This interface is live. */ @@ -665,6 +777,22 @@ service ThriftHiveMetastore extends fb303.FacebookService // method to cancel delegation token obtained from metastore server void cancel_delegation_token(1:string token_str_form) throws (1:MetaException o1) + + // Transaction and lock management calls + // Get just list of open transactions + GetOpenTxnsResponse get_open_txns() + // Get list of open transactions with state (open, aborted) + GetOpenTxnsInfoResponse get_open_txns_info() + OpenTxnsResponse open_txns(1:OpenTxnRequest rqst) + void abort_txn(1:AbortTxnRequest rqst) throws (1:NoSuchTxnException o1) + void commit_txn(1:CommitTxnRequest rqst) throws (1:NoSuchTxnException o1, 2:TxnAbortedException o2) + LockResponse lock(1:LockRequest rqst) throws (1:NoSuchTxnException o1, 2:TxnAbortedException o2) + LockResponse check_lock(1:CheckLockRequest rqst) + throws (1:NoSuchTxnException o1, 2:TxnAbortedException o2, 3:NoSuchLockException o3) + void unlock(1:UnlockRequest rqst) throws (1:NoSuchLockException o1, 2:TxnOpenException o2) + void heartbeat(1:HeartbeatRequest ids) throws (1:NoSuchLockException o1, 2:NoSuchTxnException o2, 3:TxnAbortedException o3) + void timeout_txns() + void clean_aborted_txns(1:CleanAbortedTransactionRequest o1) } // * Note about the DDL_TIME: When creating or altering a table or a partition, diff --git metastore/scripts/upgrade/derby/hive-txn-schema-0.13.0.derby.sql metastore/scripts/upgrade/derby/hive-txn-schema-0.13.0.derby.sql new file mode 100644 index 0000000..cc5554d --- /dev/null +++ metastore/scripts/upgrade/derby/hive-txn-schema-0.13.0.derby.sql @@ -0,0 +1,63 @@ +-- 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. + +-- +-- Tables for transaction management +-- +CREATE SCHEMA HIVETXNS; +SET SCHEMA HIVETXNS; + +CREATE TABLE TXNS ( + TXN_ID bigint PRIMARY KEY, + TXN_STATE char(1) NOT NULL, + TXN_STARTED bigint NOT NULL, + TXN_LAST_HEARTBEAT bigint NOT NULL, + TXN_FINALIZED bigint +); + +CREATE TABLE TXN_COMPONENTS ( + TC_TXNID bigint REFERENCES TXNS (TXN_ID), + TC_DATABASE varchar(128) NOT NULL, + TC_TABLE varchar(128), + TC_PARTITION varchar(767) +); + +CREATE TABLE NEXT_TXN_ID ( + NTXN_NEXT bigint NOT NULL +); +INSERT INTO NEXT_TXN_ID VALUES(1); + +CREATE TABLE HIVE_LOCKS ( + HL_LOCK_ID bigint NOT NULL, + HL_TXNID bigint, + HL_DB varchar(128) NOT NULL, + HL_TABLE varchar(128), + HL_PARTITION varchar(767), + HL_LOCK_STATE char(1) NOT NULL, + HL_LOCK_TYPE char(1) NOT NULL, + HL_LAST_HEARTBEAT bigint NOT NULL, + HL_ACQUIRED_AT bigint, + HL_OBJECT_DATA CLOB +); + +CREATE INDEX HL_LOCK_ID_INDEX ON HIVE_LOCKS (HL_LOCK_ID); +CREATE INDEX HL_TXNID_INDEX ON HIVE_LOCKS (HL_TXNID); + +CREATE TABLE NEXT_LOCK_ID ( + NL_NEXT bigint NOT NULL +); +INSERT INTO NEXT_LOCK_ID VALUES(1); + + diff --git metastore/scripts/upgrade/derby/upgrade-0.12.0-to-0.13.0.derby.sql metastore/scripts/upgrade/derby/upgrade-0.12.0-to-0.13.0.derby.sql index 1f1e830..3b481e3 100644 --- metastore/scripts/upgrade/derby/upgrade-0.12.0-to-0.13.0.derby.sql +++ metastore/scripts/upgrade/derby/upgrade-0.12.0-to-0.13.0.derby.sql @@ -1,2 +1,3 @@ -- Upgrade MetaStore schema from 0.11.0 to 0.12.0 +RUN 'hive-txn-schema-0.13.0.derby.sql'; UPDATE "APP".VERSION SET SCHEMA_VERSION='0.13.0', VERSION_COMMENT='Hive release version 0.13.0' where VER_ID=1; diff --git metastore/scripts/upgrade/mysql/hive-schema-0.13.0.mysql.sql metastore/scripts/upgrade/mysql/hive-schema-0.13.0.mysql.sql index 2cd8db8..b0415b1 100644 --- metastore/scripts/upgrade/mysql/hive-schema-0.13.0.mysql.sql +++ metastore/scripts/upgrade/mysql/hive-schema-0.13.0.mysql.sql @@ -761,7 +761,7 @@ CREATE TABLE IF NOT EXISTS `VERSION` ( PRIMARY KEY (`VER_ID`) ) ENGINE=InnoDB DEFAULT CHARSET=latin1; -INSERT INTO VERSION (VER_ID, SCHEMA_VERSION, VERSION_COMMENT) VALUES (1, '0.13.0', 'Hive release version 0.13.0'); +INSERT INTO VERSION (VER_ID, SCHEMA_VERSION, VERSION_COMMENT) VALUES (1, '0.12.0', 'Hive release version 0.12.0'); /*!40101 SET character_set_client = @saved_cs_client */; /*!40103 SET TIME_ZONE=@OLD_TIME_ZONE */; diff --git metastore/scripts/upgrade/mysql/hive-txn-schema-0.13.0.mysql.sql metastore/scripts/upgrade/mysql/hive-txn-schema-0.13.0.mysql.sql new file mode 100644 index 0000000..079c8a8 --- /dev/null +++ metastore/scripts/upgrade/mysql/hive-txn-schema-0.13.0.mysql.sql @@ -0,0 +1,65 @@ +-- 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. + +-- +-- Tables for transaction management +-- + +CREATE DATABASE HIVETXNS DEFAULT CHARACTER SET = latin1; +USE HIVETXNS; + +CREATE TABLE TXNS ( + TXN_ID bigint PRIMARY KEY, + TXN_STATE char(1) NOT NULL, -- 'a' = aborted, 'o' = open + TXN_STARTED bigint NOT NULL, + TXN_LAST_HEARTBEAT bigint NOT NULL, + TXN_FINALIZED bigint -- will be null unless txn_state = 'a' +) ENGINE=InnoDB; + +CREATE TABLE TXN_COMPONENTS ( + TC_TXNID bigint, + TC_DATABASE varchar(128) NOT NULL, + TC_TABLE varchar(128), + TC_PARTITION varchar(767), + FOREIGN KEY (TC_TXNID) REFERENCES TXNS (TXN_ID) +) ENGINE=InnoDB; + +CREATE TABLE NEXT_TXN_ID ( + NTXN_NEXT bigint NOT NULL +) ENGINE=InnoDB; +INSERT INTO NEXT_TXN_ID VALUES(1); + +CREATE TABLE HIVE_LOCKS ( + HL_LOCK_ID bigint NOT NULL, + HL_TXNID bigint, + HL_DB varchar(128) NOT NULL, + HL_TABLE varchar(128), + HL_PARTITION varchar(767), + HL_LOCK_STATE char(1) not null, -- 'a' = acquired, 'w' = waiting + HL_LOCK_TYPE char(1) not null, -- 'r' = read, 'w' = write, 'x' = exclusive + HL_LAST_HEARTBEAT bigint NOT NULL, + HL_ACQUIRED_AT bigint, -- will be null until lock is acquired + HL_OBJECT_DATA text, -- will be null unless debug is on + KEY HIVE_LOCK_LOCK_ID_INDEX (HL_LOCK_ID), + KEY HIVE_LOCK_TXNID_INDEX (HL_TXNID) +) ENGINE=InnoDB; + +CREATE INDEX HL_LOCK_ID_IDX ON HIVE_LOCKS (HL_LOCK_ID) USING HASH; +CREATE INDEX HL_TXNID_IDX ON HIVE_LOCKS (HL_TXNID) USING HASH; + +CREATE TABLE NEXT_LOCK_ID ( + NL_NEXT bigint NOT NULL +) ENGINE=InnoDB; +INSERT INTO NEXT_LOCK_ID VALUES(1); diff --git metastore/scripts/upgrade/mysql/upgrade-0.12.0-to-0.13.0.mysql.sql metastore/scripts/upgrade/mysql/upgrade-0.12.0-to-0.13.0.mysql.sql index 0296ac5..e730672 100644 --- metastore/scripts/upgrade/mysql/upgrade-0.12.0-to-0.13.0.mysql.sql +++ metastore/scripts/upgrade/mysql/upgrade-0.12.0-to-0.13.0.mysql.sql @@ -1,6 +1,9 @@ SELECT 'Upgrading MetaStore schema from 0.12.0 to 0.13.0' AS ' '; +<<<<<<< HEAD +======= SOURCE 015-HIVE-5700.mysql.sql; +>>>>>>> origin/trunk UPDATE VERSION SET SCHEMA_VERSION='0.13.0', VERSION_COMMENT='Hive release version 0.13.0' where VER_ID=1; SELECT 'Finished upgrading MetaStore schema from 0.12.0 to 0.13.0' AS ' '; diff --git metastore/scripts/upgrade/oracle/hive-txn-schema-0.13.0.oracle.sql metastore/scripts/upgrade/oracle/hive-txn-schema-0.13.0.oracle.sql new file mode 100644 index 0000000..df4e0f9 --- /dev/null +++ metastore/scripts/upgrade/oracle/hive-txn-schema-0.13.0.oracle.sql @@ -0,0 +1,62 @@ +-- 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. + +-- +-- Tables for transaction management +-- + +-- Unlike in MySQL and others, we don't put the tables in a separate schema +-- in Oracle since users are tied to schemas in Oracle and we don't want to +-- create a separate user account for this. +CREATE TABLE TXNS ( + TXN_ID NUMBER(10) PRIMARY KEY, + TXN_STATE char(1) NOT NULL, -- 'a' = aborted, 'o' = open + TXN_STARTED NUMBER(10) NOT NULL, + TXN_LAST_HEARTBEAT NUMBER(10) NOT NULL, + TXN_FINALIZED NUMBER(10) -- will be null unless txn_state = 'a' +); + +CREATE TABLE TXN_COMPONENTS ( + TC_TXNID NUMBER(10) REFERENCES TXNS (TXN_ID), + TC_DATABASE VARCHAR2(128) NOT NULL, + TC_TABLE VARCHAR2(128), + TC_PARTITION VARCHAR2(767) NULL +); + +CREATE TABLE NEXT_TXN_ID ( + NTXN_NEXT NUMBER(10) NOT NULL +); +INSERT INTO NEXT_TXN_ID VALUES(1); + +CREATE TABLE HIVE_LOCKS ( + HL_LOCK_ID NUMBER(10) NOT NULL, + HL_TXNID NUMBER(10), + HL_DB VARCHAR2(128) NOT NULL, + HL_TABLE VARCHAR2(128), + HL_PARTITION VARCHAR2(767), + HL_LOCK_STATE CHAR(1) NOT NULL, -- 'a' = acquired, 'w' = waiting + HL_LOCK_TYPE CHAR(1) NOT NULL, -- 'r' = read, 'w' = write, 'x' = exclusive + HL_LAST_HEARTBEAT NUMBER(10) NOT NULL, + HL_ACQUIRED_AT NUMBER(10), -- will be null until lock is acquired + HL_OBJECT_DATA CLOB -- will be null unless debug is on +); + +CREATE INDEX HL_LOCK_ID_INDEX ON HIVE_LOCKS (HL_LOCK_ID); +CREATE INDEX HL_TXNID_INDEX ON HIVE_LOCKS (HL_TXNID); + +CREATE TABLE NEXT_LOCK_ID ( + NL_NEXT NUMBER(10) NOT NULL +); +INSERT INTO NEXT_LOCK_ID VALUES(1); diff --git metastore/scripts/upgrade/oracle/upgrade-0.12.0-to-0.13.0.oracle.sql metastore/scripts/upgrade/oracle/upgrade-0.12.0-to-0.13.0.oracle.sql index 4898537..e4fe60f 100644 --- metastore/scripts/upgrade/oracle/upgrade-0.12.0-to-0.13.0.oracle.sql +++ metastore/scripts/upgrade/oracle/upgrade-0.12.0-to-0.13.0.oracle.sql @@ -1,6 +1,9 @@ SELECT 'Upgrading MetaStore schema from 0.12.0 to 0.13.0' AS Status from dual; +<<<<<<< HEAD +======= @015-HIVE-5700.oracle.sql; +>>>>>>> origin/trunk UPDATE VERSION SET SCHEMA_VERSION='0.13.0', VERSION_COMMENT='Hive release version 0.13.0' where VER_ID=1; SELECT 'Finished upgrading MetaStore schema from 0.12.0 to 0.13.0' AS Status from dual; diff --git metastore/scripts/upgrade/postgres/hive-txn-schema-0.13.0.postgres.sql metastore/scripts/upgrade/postgres/hive-txn-schema-0.13.0.postgres.sql new file mode 100644 index 0000000..5d4f4f9 --- /dev/null +++ metastore/scripts/upgrade/postgres/hive-txn-schema-0.13.0.postgres.sql @@ -0,0 +1,62 @@ +-- 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. + +-- +-- Tables for transaction management +-- + +CREATE SCHEMA HIVETXNS; +SET search_path TO HIVETXNS; + +CREATE TABLE TXNS ( + TXN_ID bigint PRIMARY KEY, + TXN_STATE char(1) NOT NULL, -- 'a' = aborted, 'o' = open + TXN_STARTED bigint NOT NULL, + TXN_LAST_HEARTBEAT bigint NOT NULL, + TXN_FINALIZED bigint -- will be null unless txn_state = 'a' +); + +CREATE TABLE TXN_COMPONENTS ( + TC_TXNID bigint REFERENCES TXNS (TXN_ID), + TC_DATABASE varchar(128) NOT NULL, + TC_TABLE varchar(128), + TC_PARTITION varchar(767) DEFAULT NULL +); + +CREATE TABLE NEXT_TXN_ID ( + NTXN_NEXT bigint NOT NULL +); +INSERT INTO NEXT_TXN_ID VALUES(1); + +CREATE TABLE HIVE_LOCKS ( + HL_LOCK_ID bigint NOT NULL, + HL_TXNID bigint, + HL_DB varchar(128) NOT NULL, + HL_TABLE varchar(128), + HL_PARTITION varchar(767) DEFAULT NULL, + HL_LOCK_STATE char(1) NOT NULL, -- 'a' = acquired, 'w' = waiting + HL_LOCK_TYPE char(1) NOT NULL, -- 'r' = read, 'w' = write, 'x' = exclusive + HL_LAST_HEARTBEAT bigint NOT NULL, + HL_ACQUIRED_AT bigint, -- will be null until lock is acquired + HL_OBJECT_DATA text -- will be null unless debug is on +); + +CREATE INDEX HL_LOCK_ID_INDEX ON HIVE_LOCKS USING hash (HL_LOCK_ID); +CREATE INDEX HL_TXNID_INDEX ON HIVE_LOCKS USING hash (HL_TXNID); + +CREATE TABLE NEXT_LOCK_ID ( + NL_NEXT bigint NOT NULL +); +INSERT INTO NEXT_LOCK_ID VALUES(1); diff --git metastore/scripts/upgrade/postgres/upgrade-0.12.0-to-0.13.0.postgres.sql metastore/scripts/upgrade/postgres/upgrade-0.12.0-to-0.13.0.postgres.sql index e96d4fd..eec9da4 100644 --- metastore/scripts/upgrade/postgres/upgrade-0.12.0-to-0.13.0.postgres.sql +++ metastore/scripts/upgrade/postgres/upgrade-0.12.0-to-0.13.0.postgres.sql @@ -1,8 +1,10 @@ SELECT 'Upgrading MetaStore schema from 0.12.0 to 0.13.0'; +<<<<<<< HEAD +\i hive-txn-schema-0.13.0.postgres.sql; +======= \i 015-HIVE-5700.postgres.sql; +>>>>>>> origin/trunk UPDATE "VERSION" SET "SCHEMA_VERSION"='0.13.0', "VERSION_COMMENT"='Hive release version 0.13.0' where "VER_ID"=1; SELECT 'Finished upgrading MetaStore schema from 0.12.0 to 0.13.0'; - - diff --git metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java index 58f9957..c0b142c 100644 --- metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java +++ metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java @@ -18,6 +18,8 @@ package org.apache.hadoop.hive.metastore; +import com.facebook.fb303.FacebookBase; +import com.facebook.fb303.fb_status; import static org.apache.commons.lang.StringUtils.join; import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_COMMENT; import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME; @@ -58,44 +60,9 @@ import org.apache.hadoop.hive.common.metrics.Metrics; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hadoop.hive.metastore.api.AddPartitionsRequest; -import org.apache.hadoop.hive.metastore.api.AddPartitionsResult; -import org.apache.hadoop.hive.metastore.api.AlreadyExistsException; -import org.apache.hadoop.hive.metastore.api.ColumnStatistics; -import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc; -import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj; -import org.apache.hadoop.hive.metastore.api.ConfigValSecurityException; -import org.apache.hadoop.hive.metastore.api.Database; -import org.apache.hadoop.hive.metastore.api.EnvironmentContext; -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege; -import org.apache.hadoop.hive.metastore.api.HiveObjectRef; -import org.apache.hadoop.hive.metastore.api.HiveObjectType; -import org.apache.hadoop.hive.metastore.api.Index; -import org.apache.hadoop.hive.metastore.api.IndexAlreadyExistsException; -import org.apache.hadoop.hive.metastore.api.InvalidInputException; -import org.apache.hadoop.hive.metastore.api.InvalidObjectException; -import org.apache.hadoop.hive.metastore.api.InvalidOperationException; -import org.apache.hadoop.hive.metastore.api.InvalidPartitionException; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; -import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.hadoop.hive.metastore.api.PartitionEventType; -import org.apache.hadoop.hive.metastore.api.PartitionsByExprRequest; -import org.apache.hadoop.hive.metastore.api.PartitionsByExprResult; -import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet; -import org.apache.hadoop.hive.metastore.api.PrincipalType; -import org.apache.hadoop.hive.metastore.api.PrivilegeBag; -import org.apache.hadoop.hive.metastore.api.PrivilegeGrantInfo; -import org.apache.hadoop.hive.metastore.api.Role; -import org.apache.hadoop.hive.metastore.api.SkewedInfo; -import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.api.*; +import org.apache.hadoop.hive.metastore.txn.TxnHandler; import org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore; -import org.apache.hadoop.hive.metastore.api.Type; -import org.apache.hadoop.hive.metastore.api.UnknownDBException; -import org.apache.hadoop.hive.metastore.api.UnknownPartitionException; -import org.apache.hadoop.hive.metastore.api.UnknownTableException; -import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.metastore.events.AddPartitionEvent; import org.apache.hadoop.hive.metastore.events.AlterPartitionEvent; import org.apache.hadoop.hive.metastore.events.AlterTableEvent; @@ -138,14 +105,15 @@ import org.apache.thrift.protocol.TBinaryProtocol; import org.apache.thrift.server.TServer; import org.apache.thrift.server.TThreadPoolServer; -import org.apache.thrift.transport.TFramedTransport; -import org.apache.thrift.transport.TServerSocket; -import org.apache.thrift.transport.TServerTransport; -import org.apache.thrift.transport.TTransport; -import org.apache.thrift.transport.TTransportFactory; +import org.apache.thrift.transport.*; -import com.facebook.fb303.FacebookBase; -import com.facebook.fb303.fb_status; +import java.io.IOException; +import java.util.*; +import java.util.Map.Entry; +import java.util.regex.Pattern; + +import static org.apache.commons.lang.StringUtils.join; +import static org.apache.hadoop.hive.metastore.MetaStoreUtils.*; /** * TODO:pc remove application logic to a separate interface. @@ -201,6 +169,8 @@ protected synchronized RawStore initialValue() { return null; } }; + // Helper class to handle transaction operations + private TxnHandler txnHandler; // Thread local configuration is needed as many threads could make changes // to the conf using the connection hook @@ -4126,6 +4096,79 @@ private static MetaException newMetaException(Exception e) { me.initCause(e); return me; } + + // Transaction and locking methods + @Override + public GetOpenTxnsResponse get_open_txns() { + initTxnHandler(); + return txnHandler.getOpenTxns(); + } + + // Transaction and locking methods + @Override + public GetOpenTxnsInfoResponse get_open_txns_info() { + initTxnHandler(); + return txnHandler.getOpenTxnsInfo(); + } + + @Override + public OpenTxnsResponse open_txns(OpenTxnRequest rqst) { + initTxnHandler(); + return txnHandler.openTxns(rqst); + } + + @Override + public void abort_txn(AbortTxnRequest rqst) throws NoSuchTxnException { + initTxnHandler(); + txnHandler.abortTxn(rqst); + } + + @Override + public void commit_txn(CommitTxnRequest rqst) + throws NoSuchTxnException, TxnAbortedException { + initTxnHandler(); + txnHandler.commitTxn(rqst); + } + + @Override + public LockResponse lock(LockRequest rqst) + throws NoSuchTxnException, TxnAbortedException { + initTxnHandler(); + return txnHandler.lock(rqst); + } + + @Override + public LockResponse check_lock(CheckLockRequest rqst) + throws NoSuchTxnException, TxnAbortedException, NoSuchLockException { + initTxnHandler(); + return txnHandler.checkLock(rqst); + } + + @Override + public void unlock(UnlockRequest rqst) + throws NoSuchLockException, TxnOpenException { + initTxnHandler(); + txnHandler.unlock(rqst); + } + + @Override + public void heartbeat(HeartbeatRequest ids) + throws NoSuchLockException, NoSuchTxnException, TxnAbortedException { + initTxnHandler(); + txnHandler.heartbeat(ids); + } + + @Override + public void timeout_txns() { + } + + @Override + public void clean_aborted_txns(CleanAbortedTransactionRequest tpi) { + } + + private final void initTxnHandler() { + if (txnHandler == null) txnHandler = new TxnHandler(hiveConf); + } } public static IHMSHandler newHMSHandler(String name, HiveConf hiveConf) throws MetaException { diff --git metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java index ed05790..cbbe24d 100644 --- metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java +++ metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java @@ -45,37 +45,8 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hadoop.hive.metastore.api.AddPartitionsRequest; -import org.apache.hadoop.hive.metastore.api.AddPartitionsResult; -import org.apache.hadoop.hive.metastore.api.AlreadyExistsException; -import org.apache.hadoop.hive.metastore.api.ColumnStatistics; -import org.apache.hadoop.hive.metastore.api.ConfigValSecurityException; -import org.apache.hadoop.hive.metastore.api.Database; -import org.apache.hadoop.hive.metastore.api.EnvironmentContext; -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege; -import org.apache.hadoop.hive.metastore.api.HiveObjectRef; -import org.apache.hadoop.hive.metastore.api.Index; -import org.apache.hadoop.hive.metastore.api.InvalidInputException; -import org.apache.hadoop.hive.metastore.api.InvalidObjectException; -import org.apache.hadoop.hive.metastore.api.InvalidOperationException; -import org.apache.hadoop.hive.metastore.api.InvalidPartitionException; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; -import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.hadoop.hive.metastore.api.PartitionEventType; -import org.apache.hadoop.hive.metastore.api.PartitionsByExprRequest; -import org.apache.hadoop.hive.metastore.api.PartitionsByExprResult; -import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet; -import org.apache.hadoop.hive.metastore.api.PrincipalType; -import org.apache.hadoop.hive.metastore.api.PrivilegeBag; -import org.apache.hadoop.hive.metastore.api.Role; -import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.api.*; import org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore; -import org.apache.hadoop.hive.metastore.api.Type; -import org.apache.hadoop.hive.metastore.api.UnknownDBException; -import org.apache.hadoop.hive.metastore.api.UnknownPartitionException; -import org.apache.hadoop.hive.metastore.api.UnknownTableException; import org.apache.hadoop.hive.shims.HadoopShims; import org.apache.hadoop.hive.shims.ShimLoader; import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge; @@ -89,6 +60,20 @@ import org.apache.thrift.transport.TTransport; import org.apache.thrift.transport.TTransportException; +import javax.security.auth.login.LoginException; +import java.io.IOException; +import java.lang.reflect.InvocationHandler; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.lang.reflect.Proxy; +import java.net.URI; +import java.net.URISyntaxException; +import java.nio.ByteBuffer; +import java.util.*; + +import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME; +import static org.apache.hadoop.hive.metastore.MetaStoreUtils.isIndexTable; + /** * Hive Metastore Client. */ @@ -635,7 +620,6 @@ public boolean dropPartition(String db_name, String tbl_name, List part_ * @param name * @param dbname * @throws NoSuchObjectException - * @throws ExistingDependentsException * @throws MetaException * @throws TException * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#drop_table(java.lang.String, @@ -659,7 +643,6 @@ public void dropTable(String tableName, boolean deleteData) * @param deleteData * delete the underlying data or just delete the table in metadata * @throws NoSuchObjectException - * @throws ExistingDependentsException * @throws MetaException * @throws TException * @see org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface#drop_table(java.lang.String, @@ -1404,6 +1387,83 @@ public void cancelDelegationToken(String tokenStrForm) throws MetaException, TEx client.cancel_delegation_token(tokenStrForm); } + public static class ValidTxnListImpl implements ValidTxnList { + + private GetOpenTxnsResponse txns; + + ValidTxnListImpl(GetOpenTxnsResponse t) { + txns = t; + } + + @Override + public boolean isTxnCommitted(long txnid) { + if (txns.getTxn_high_water_mark() < txnid) return false; + return !txns.getOpen_txns().contains(txnid); + } + + @Override + public GetOpenTxnsResponse getOpenTxns() { + return txns; + } + } + + @Override + public ValidTxnList getValidTxns() throws TException { + GetOpenTxnsResponse txns = client.get_open_txns(); + return new ValidTxnListImpl(txns); + } + + @Override + public long openTxn() throws TException { + OpenTxnsResponse txns = client.open_txns(new OpenTxnRequest(1)); + return txns.getTxn_ids().get(0); + } + + @Override + public OpenTxnsResponse openTxns(int numTxns) throws TException { + return client.open_txns(new OpenTxnRequest(numTxns)); + } + + @Override + public void rollbackTxn(long txnid) throws NoSuchTxnException, TException { + client.abort_txn(new AbortTxnRequest(txnid)); + } + + @Override + public void commitTxn(long txnid) + throws NoSuchTxnException, TxnAbortedException, TException { + client.commit_txn(new CommitTxnRequest(txnid)); + } + + @Override + public LockResponse lock(LockRequest request) + throws NoSuchTxnException, TxnAbortedException, TException { + return client.lock(request); + } + + @Override + public LockResponse checkLock(long lockid) + throws NoSuchTxnException, TxnAbortedException, NoSuchLockException, + TException { + return client.check_lock(new CheckLockRequest(lockid)); + } + + @Override + public void unlock(long lockid) + throws NoSuchLockException, TxnOpenException, TException { + client.unlock(new UnlockRequest(lockid)); + } + + @Override + public void heartbeat(long txnid, long lockid) + throws NoSuchLockException, NoSuchTxnException, TxnAbortedException, + TException { + HeartbeatRequest hb = new HeartbeatRequest(); + hb.setLockid(lockid); + hb.setTxnid(txnid); + client.heartbeat(hb); + } + /** * Creates a synchronized wrapper for any {@link IMetaStoreClient}. * This may be used by multi-threaded applications until we have diff --git metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java index 4288781..17b22c3 100644 --- metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java +++ metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java @@ -18,36 +18,13 @@ package org.apache.hadoop.hive.metastore; +import org.apache.hadoop.hive.metastore.api.*; +import org.apache.thrift.TException; + import java.util.List; import java.util.Map; import java.util.Set; -import org.apache.hadoop.hive.metastore.api.AlreadyExistsException; -import org.apache.hadoop.hive.metastore.api.ColumnStatistics; -import org.apache.hadoop.hive.metastore.api.ConfigValSecurityException; -import org.apache.hadoop.hive.metastore.api.Database; -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege; -import org.apache.hadoop.hive.metastore.api.HiveObjectRef; -import org.apache.hadoop.hive.metastore.api.Index; -import org.apache.hadoop.hive.metastore.api.InvalidInputException; -import org.apache.hadoop.hive.metastore.api.InvalidObjectException; -import org.apache.hadoop.hive.metastore.api.InvalidOperationException; -import org.apache.hadoop.hive.metastore.api.InvalidPartitionException; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; -import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.hadoop.hive.metastore.api.PartitionEventType; -import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet; -import org.apache.hadoop.hive.metastore.api.PrincipalType; -import org.apache.hadoop.hive.metastore.api.PrivilegeBag; -import org.apache.hadoop.hive.metastore.api.Role; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.api.UnknownDBException; -import org.apache.hadoop.hive.metastore.api.UnknownPartitionException; -import org.apache.hadoop.hive.metastore.api.UnknownTableException; -import org.apache.thrift.TException; - /** * TODO Unnecessary when the server sides for both dbstore and filestore are * merged @@ -157,7 +134,6 @@ * The table wasn't found. * @throws TException * A thrift communication error occurred - * @throws ExistingDependentsException */ public void dropTable(String dbname, String tableName, boolean deleteData, boolean ignoreUknownTab) throws MetaException, TException, @@ -359,7 +335,9 @@ public Partition getPartition(String tblName, String dbName, List partVals) throws NoSuchObjectException, MetaException, TException; /** - * @param partition + * @param partitionSpecs + * @param sourceDb + * @param sourceTable * @param destdb * @param destTableName * @return partition object @@ -1016,6 +994,181 @@ public String getDelegationToken(String owner, String renewerKerberosPrincipalNa */ public void cancelDelegationToken(String tokenStrForm) throws MetaException, TException; + // Transaction and locking methods + public interface ValidTxnList { + /** + * Indicates whether a given transaction has been committed and should be + * viewed as valid for read. + * @param txnid id for the transaction + * @return true if committed, false otherwise + */ + public boolean isTxnCommitted(long txnid); + + /** + * Get at the underlying OpenTxn structure. This is useful if the user + * wishes to get a list of all open transactions for more efficient + * filtering. + * @return open transactions + */ + public GetOpenTxnsResponse getOpenTxns(); + } + + /** + * Get a structure that details valid transactions. + * @return list of valid transactions + * @throws TException + */ + public ValidTxnList getValidTxns() throws TException; + + /** + * Initiate a transaction. + * @return transaction identifier + * @throws TException + */ + public long openTxn() throws TException; + + /** + * Initiate a batch of transactions. It is not guaranteed that the + * requested number of transactions will be instantiated. The system has a + * maximum number instantiated per request, controlled by hive.txn.max + * .batch.open in hive-site.xml. If the user requests more than this + * value, only the configured max will be returned. + * + *

Increasing the number of transactions requested in the batch will + * allow applications that stream data into Hive to place more commits in a + * single file, thus reducing load on the namenode and making reads of the + * data more efficient. However, opening more transactions in a batch will + * also result in readers needing to keep a larger list of open + * transactions to ignore, potentially slowing their reads. Users will + * need to test in their system to understand the optimal number of + * transactions to request in a batch. + *

+ * @param numTxns number of requested transactions to open + * @return list of opened txn ids. As noted above, this may be less than + * requested, so the user should check how many were returned rather than + * optimistically assuming that the result matches the request. + * @throws TException + */ + public OpenTxnsResponse openTxns(int numTxns) throws TException; + + /** + * Rollback a transaction. This will also unlock any locks associated with + * this transaction. + * @param txnid id of transaction to be rolled back. + * @throws NoSuchTxnException if the requested transaction does not exist. + * Note that this can result from the transaction having timed out and been + * deleted. + * @throws TException + */ + public void rollbackTxn(long txnid) throws NoSuchTxnException, TException; + + /** + * Commit a transaction. This will also unlock any locks associated with + * this transaction. + * @param txnid id of transaction to be committed. + * @throws NoSuchTxnException if the requested transaction does not exist. + * This can result fro the transaction having timed out and been deleted by + * the compactor. + * @throws TxnAbortedException if the requested transaction has been + * aborted. This can result from the transaction timing out. + * @throws TException + */ + public void commitTxn(long txnid) + throws NoSuchTxnException, TxnAbortedException, TException; + + /** + * Request a set of locks. All locks needed for a particular query, DML, + * or DDL operation should be batched together and requested in one lock + * call. This avoids deadlocks. It also avoids blocking other users who + * only require some of the locks required by this user. + * + *

If the operation requires a transaction (INSERT, UPDATE, + * or DELETE) that transaction id must be provided as part this lock + * request. All locks associated with a transaction will be released when + * that transaction is committed or rolled back.

+ * * + *

Once a lock is acquired, {@link #heartbeat(long, long)} must be called + * on a regular basis to avoid the lock being timed out by the system.

+ * @param request The lock request. {@link LockRequestBuilder} can be used + * construct this request. + * @return a lock response, which will provide two things, + * the id of the lock (to be used in all further calls regarding this lock) + * as well as a state of the lock. If the state is ACQUIRED then the user + * can proceed. If it is WAITING the user should wait and call + * {@link #checkLock(long)} before proceeding. All components of the lock + * will have the same state. + * @throws NoSuchTxnException if the requested transaction does not exist. + * This can result fro the transaction having timed out and been deleted by + * the compactor. + * @throws TxnAbortedException if the requested transaction has been + * aborted. This can result from the transaction timing out. + * @throws TException + */ + public LockResponse lock(LockRequest request) + throws NoSuchTxnException, TxnAbortedException, TException; + + /** + * Check the status of a set of locks requested via a + * {@link #lock(org.apache.hadoop.hive.metastore.api.LockRequest)} call. + * Once a lock is acquired, {@link #heartbeat(long, long)} must be called + * on a regular basis to avoid the lock being timed out by the system. + * @param lockid lock id returned by lock(). + * @return a lock response, which will provide two things, + * the id of the lock (to be used in all further calls regarding this lock) + * as well as a state of the lock. If the state is ACQUIRED then the user + * can proceed. If it is WAITING the user should wait and call + * this method again before proceeding. All components of the lock + * will have the same state. + * @throws NoSuchTxnException if the requested transaction does not exist. + * This can result fro the transaction having timed out and been deleted by + * the compactor. + * @throws TxnAbortedException if the requested transaction has been + * aborted. This can result from the transaction timing out. + * @throws NoSuchLockException if the requested lockid does not exist. + * This can result from the lock timing out and being unlocked by the system. + * @throws TException + */ + public LockResponse checkLock(long lockid) + throws NoSuchTxnException, TxnAbortedException, NoSuchLockException, + TException; + + /** + * Unlock a set of locks. This can only be called when the locks are not + * assocaited with a transaction. + * @param lockid lock id returned by + * {@link #lock(org.apache.hadoop.hive.metastore.api.LockRequest)} + * @throws NoSuchLockException if the requested lockid does not exist. + * This can result from the lock timing out and being unlocked by the system. + * @throws TxnOpenException if the locks are are associated with a + * transaction. + * @throws TException + */ + public void unlock(long lockid) + throws NoSuchLockException, TxnOpenException, TException; + + /** + * Send a heartbeat to indicate that the client holding these locks (if + * any) and that opened this transaction (if one exists) is still alive. + * The default timeout for transactions and locks is 300 seconds, + * though it is configurable. To determine how often to heartbeat you will + * need to ask your system administrator how the metastore thrift service + * has been configured. + * @param txnid the id of the open transaction. If no transaction is open + * (it is a DDL or query) then this can be set to 0. + * @param lockid the id of the locks obtained. If no locks have been + * obtained then this can be set to 0. + * @throws NoSuchTxnException if the requested transaction does not exist. + * This can result fro the transaction having timed out and been deleted by + * the compactor. + * @throws TxnAbortedException if the requested transaction has been + * aborted. This can result from the transaction timing out. + * @throws NoSuchLockException if the requested lockid does not exist. + * This can result from the lock timing out and being unlocked by the system. + * @throws TException + */ + public void heartbeat(long txnid, long lockid) + throws NoSuchLockException, NoSuchTxnException, TxnAbortedException, + TException; public class IncompatibleMetastoreException extends MetaException { public IncompatibleMetastoreException(String message) { diff --git metastore/src/java/org/apache/hadoop/hive/metastore/LockComponentBuilder.java metastore/src/java/org/apache/hadoop/hive/metastore/LockComponentBuilder.java new file mode 100644 index 0000000..7556682 --- /dev/null +++ metastore/src/java/org/apache/hadoop/hive/metastore/LockComponentBuilder.java @@ -0,0 +1,119 @@ +/** + * 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.hive.metastore.api.LockComponent; +import org.apache.hadoop.hive.metastore.api.LockLevel; +import org.apache.hadoop.hive.metastore.api.LockType; + +/** + * A builder for {@link LockComponent}s + */ +public class LockComponentBuilder { + private LockComponent component; + private boolean tableNameSet; + private boolean partNameSet; + + public LockComponentBuilder() { + component = new LockComponent(); + tableNameSet = partNameSet = false; + } + + /** + * Set the lock to be exclusive. + * @return reference to this builder + */ + public LockComponentBuilder setExclusive() { + component.setType(LockType.EXCLUSIVE); + return this; + } + + /** + * Set the lock to be semi-shared. + * @return reference to this builder + */ + public LockComponentBuilder setSemiShared() { + component.setType(LockType.SHARED_WRITE); + return this; + } + + /** + * Set the lock to be shared. + * @return reference to this builder + */ + public LockComponentBuilder setShared() { + component.setType(LockType.SHARED_READ); + return this; + } + + /** + * Set the database name. + * @param dbName database name + * @return reference to this builder + */ + public LockComponentBuilder setDbName(String dbName) { + component.setDbname(dbName); + return this; + } + + /** + * Set the table name. + * @param tableName table name + * @return reference to this builder + */ + public LockComponentBuilder setTableName(String tableName) { + component.setTablename(tableName); + tableNameSet = true; + return this; + } + + /** + * Set the partition name. + * @param partitionName partition name + * @return reference to this builder + */ + public LockComponentBuilder setPartitionName(String partitionName) { + component.setPartitionname(partitionName); + partNameSet = true; + return this; + } + + /** + * Set the lock object data. + * @param objectData object data, as a string since I'm not clear how bytes + * are moved in thrift. It's up to the user to serialize + * this into a string in an acceptable way. + * @return reference to this builder + */ + public LockComponentBuilder setObjectData(String objectData) { + component.setLock_object_data(objectData); + return this; + } + + /** + * Get the constructed lock component. + * @return lock component. + */ + public LockComponent build() { + LockLevel level = LockLevel.DB; + if (tableNameSet) level = LockLevel.TABLE; + if (partNameSet) level = LockLevel.PARTITION; + component.setLevel(level); + return component; + } +} diff --git metastore/src/java/org/apache/hadoop/hive/metastore/LockRequestBuilder.java metastore/src/java/org/apache/hadoop/hive/metastore/LockRequestBuilder.java new file mode 100644 index 0000000..300b262 --- /dev/null +++ metastore/src/java/org/apache/hadoop/hive/metastore/LockRequestBuilder.java @@ -0,0 +1,140 @@ +/** + * 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.hive.metastore.api.LockComponent; +import org.apache.hadoop.hive.metastore.api.LockRequest; +import org.apache.hadoop.hive.metastore.api.LockType; + +import java.util.HashMap; +import java.util.Map; + +/** + * Builder class to make constructing {@link LockRequest} easier. + */ +public class LockRequestBuilder { + + private LockRequest req; + private LockTrie trie; + + public LockRequestBuilder() { + req = new LockRequest(); + trie = new LockTrie(); + } + + /** + * Get the constructed LockRequest. + * @return lock request + */ + public LockRequest build() { + trie.addLocksToRequest(req); + return req; + } + + /** + * Set the transaction id. + * @param txnid transaction id + * @return reference to this builder + */ + public LockRequestBuilder setTransactionId(long txnid) { + req.setTxnid(txnid); + return this; + } + + /** + * Add a lock component to the lock request + * @param component to add + * @return reference to this builder + */ + public LockRequestBuilder addLockComponent(LockComponent component) { + trie.add(component); + return this; + } + + // For reasons that are completely incomprehensible to me the semantic + // analyzers often ask for multiple locks on the same entity (for example + // a shared_read and an exlcusive lock). The db locking system gets confused + // by this and dead locks on it. To resolve that, we'll make sure in the + // request that multiple locks are coalesced and promoted to the higher + // level of locking. To do this we put all locks components in trie based + // on dbname, tablename, partition name and handle the promotion as new + // requests come in. This structure depends on the fact that null is a + // valid key in a HashMap. So a database lock will map to (dbname, null, + // null). + private static class LockTrie { + Map trie; + + LockTrie() { + trie = new HashMap(); + } + + public void add(LockComponent comp) { + TableTrie tabs = trie.get(comp.getDbname()); + if (tabs == null) { + tabs = new TableTrie(); + trie.put(comp.getDbname(), tabs); + } + setTable(comp, tabs); + } + + public void addLocksToRequest(LockRequest request) { + for (TableTrie tab : trie.values()) { + for (PartTrie part : tab.values()) { + for (LockComponent lock : part.values()) { + request.addToComponent(lock); + } + } + } + } + + private void setTable(LockComponent comp, TableTrie tabs) { + PartTrie parts = tabs.get(comp.getTablename()); + if (parts == null) { + parts = new PartTrie(); + tabs.put(comp.getTablename(), parts); + } + setPart(comp, parts); + } + + private void setPart(LockComponent comp, PartTrie parts) { + LockComponent existing = parts.get(comp.getPartitionname()); + if (existing == null) { + // No existing lock for this partition. + parts.put(comp.getPartitionname(), comp); + } else if (existing.getType() != LockType.EXCLUSIVE && + (comp.getType() == LockType.EXCLUSIVE || + comp.getType() == LockType.SHARED_WRITE)) { + // We only need to promote if comp.type is > existing.type. For + // efficiency we check if existing is exclusive (in which case we + // need never promote) or if comp is exclusive or shared_write (in + // which case we can promote even though they may both be shared + // write). If comp is shared_read there's never a need to promote. + parts.put(comp.getPartitionname(), comp); + } + } + + private static class TableTrie extends HashMap { + } + + private static class PartTrie extends HashMap { + } + + + + } +} diff --git metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java new file mode 100644 index 0000000..3f02c16 --- /dev/null +++ metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java @@ -0,0 +1,174 @@ +/** + * 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.txn; + +import org.apache.hadoop.hive.conf.HiveConf; + +import java.sql.Connection; +import java.sql.Driver; +import java.sql.ResultSet; +import java.sql.Statement; +import java.util.Properties; + +/** + * Utility methods for creating and destroying txn database/schema. Placed + * here in a separate class so it can be shared across unit tests. + */ +public class TxnDbUtil { + private final static String jdbcString = "jdbc:derby:;databaseName=metastore_db;create=true"; + private final static String jdbcDriver = "org.apache.derby.jdbc.EmbeddedDriver"; + private final static String txnMgr = "org.apache.hadoop.hive.ql.lockmgr.DbTxnManager"; + + /** + * Set up the configuration so it will use the DbTxnManager, concurrency will be set to true, + * and the JDBC configs will be set for putting the transaction and lock info in the embedded + * metastore. + * @param conf HiveConf to add these values to. + */ + public static void setConfValues(HiveConf conf) { + conf.setVar(HiveConf.ConfVars.HIVE_TXN_JDBC_DRIVER, jdbcDriver); + conf.setVar(HiveConf.ConfVars.HIVE_TXN_JDBC_CONNECT_STRING, jdbcString); + conf.setVar(HiveConf.ConfVars.HIVE_TXN_MANAGER, txnMgr); + conf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, true); + } + + public static void prepDb() throws Exception { + // This is a bogus hack because it copies the contents of the SQL file + // intended for creating derby databases, and thus will inexorably get + // out of date with it. I'm open to any suggestions on how to make this + // read the file in a build friendly way. + Driver driver = (Driver)Class.forName(jdbcDriver).newInstance(); + Connection conn = driver.connect(jdbcString, new Properties()); + Statement s = conn.createStatement(); + s.execute("CREATE SCHEMA HIVETXNS"); + s.execute("SET SCHEMA HIVETXNS"); + s.execute("CREATE TABLE TXNS (" + + " TXN_ID bigint PRIMARY KEY," + + " TXN_STATE char(1) NOT NULL," + + " TXN_STARTED bigint NOT NULL," + + " TXN_LAST_HEARTBEAT bigint NOT NULL," + + " TXN_FINALIZED bigint)"); + s.execute("CREATE TABLE TXN_COMPONENTS (" + + " TC_TXNID bigint REFERENCES TXNS (TXN_ID)," + + " TC_DATABASE varchar(128) NOT NULL," + + " TC_TABLE varchar(128)," + + " TC_PARTITION varchar(767))"); + s.execute("CREATE TABLE NEXT_TXN_ID (" + + " NTXN_NEXT bigint NOT NULL)"); + s.execute("INSERT INTO NEXT_TXN_ID VALUES(1)"); + s.execute("CREATE TABLE HIVE_LOCKS (" + + " HL_LOCK_EXT_ID bigint NOT NULL," + + " HL_LOCK_INT_ID bigint NOT NULL," + + " HL_TXNID bigint," + + " HL_DB varchar(128) NOT NULL," + + " HL_TABLE varchar(128)," + + " HL_PARTITION varchar(767)," + + " HL_LOCK_STATE char(1) NOT NULL," + + " HL_LOCK_TYPE char(1) NOT NULL," + + " HL_LAST_HEARTBEAT bigint NOT NULL," + + " HL_ACQUIRED_AT bigint," + + " HL_OBJECT_DATA CLOB," + + " PRIMARY KEY(HL_LOCK_EXT_ID, HL_LOCK_INT_ID))"); + s.execute("CREATE INDEX HL_LOCK_ID_INDEX ON HIVE_LOCKS (HL_LOCK_EXT_ID)"); + s.execute("CREATE INDEX HL_TXNID_INDEX ON HIVE_LOCKS (HL_TXNID)"); + + s.execute("CREATE TABLE NEXT_LOCK_ID (" + + " NL_NEXT bigint NOT NULL)"); + s.execute("INSERT INTO NEXT_LOCK_ID VALUES(1)"); + conn.commit(); + conn.close(); + } + + public static void cleanDb() throws Exception { + Driver driver = (Driver)Class.forName(jdbcDriver).newInstance(); + Connection conn = driver.connect(jdbcString, new Properties()); + Statement s = conn.createStatement(); + try { + s.execute("SET SCHEMA HIVETXNS"); + } catch (Exception e) { + } + // We want to try these, whether they succeed or fail. + try { + s.execute("DROP INDEX HL_LOCK_ID_INDEX"); + } catch (Exception e) { + System.err.println("Unable to drop index HL_LOCK_ID_INDEX " + + e.getMessage()); + } + try { + s.execute("DROP INDEX HL_TXNID_INDEX"); + } catch (Exception e) { + System.err.println("Unable to drop index HL_TXNID_INDEX " + + e.getMessage()); + } + try { + s.execute("DROP TABLE TXN_COMPONENTS"); + } catch (Exception e) { + System.err.println("Unable to drop table TXN_COMPONENTS " + + e.getMessage()); + } + try { + s.execute("DROP TABLE TXNS"); + } catch (Exception e) { + System.err.println("Unable to drop table TXNS " + + e.getMessage()); + } + try { + s.execute("DROP TABLE NEXT_TXN_ID"); + } catch (Exception e) { + System.err.println("Unable to drop table NEXT_TXN_ID " + + e.getMessage()); + } + try { + s.execute("DROP TABLE HIVE_LOCKS"); + } catch (Exception e) { + System.err.println("Unable to drop table HIVE_LOCKS " + + e.getMessage()); + } + try { + s.execute("DROP TABLE NEXT_LOCK_ID"); + } catch (Exception e) { + } + try { + s.execute("DROP SCHEMA HIVETXNS RESTRICT"); + } catch (Exception e) { + System.err.println("Unable to drop schema " + e.getMessage()); + + } + conn.close(); + } + + /** + * A tool to count the number of partitions, tables, + * and databases locked by a particular lockId. + * @param lockId lock id to look for lock components + * @return number of components, or 0 if there is no lock + */ + public static int countLockComponents(long lockId) throws Exception { + Driver driver = (Driver)Class.forName(jdbcDriver).newInstance(); + Connection conn = driver.connect(jdbcString, new Properties()); + Statement s = conn.createStatement(); + s.execute("SET SCHEMA HIVETXNS"); + ResultSet rs = s.executeQuery("select count(*) from hive_locks where " + + "hl_lock_ext_id = " + lockId); + if (!rs.next()) return 0; + int rc = rs.getInt(1); + conn.rollback(); + conn.close(); + return rc; + } +} diff --git metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java new file mode 100644 index 0000000..4ab308b --- /dev/null +++ metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java @@ -0,0 +1,1142 @@ +/** + * 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.txn; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.*; + +import java.sql.*; +import java.util.*; + +/** + * A handler to answer transaction related calls that come into the metastore + * server. + */ +public class TxnHandler { + static final private String CLASS_NAME = TxnHandler.class.getName(); + static final private Log LOG = LogFactory.getLog(CLASS_NAME); + + private Connection dbConn; + // Transaction timeout, in milliseconds. + private long timeout; + LockInfoLRU lockInfoCache; + private HiveConf conf; + + public TxnHandler(HiveConf conf) { + this.conf = conf; + + checkQFileTestHack(); + + // Find the JDBC driver + String driverName = HiveConf.getVar(conf, + HiveConf.ConfVars.HIVE_TXN_JDBC_DRIVER); + if (driverName == null || driverName.equals("")) { + String msg = "JDBC driver for transaction db not set in configuration " + + "file, need to set " + HiveConf.ConfVars.HIVE_TXN_JDBC_DRIVER.varname; + LOG.error(msg); + throw new RuntimeException(msg); + } + Driver driver; + try { + LOG.info("Going to load transaction JDBC driver " + driverName); + driver = (Driver)Class.forName(driverName).newInstance(); + } catch (InstantiationException e) { + throw new RuntimeException("Unable to instantiate driver " + driverName + + ", " + e.getMessage(), e); + } catch (IllegalAccessException e) { + throw new RuntimeException("Unable to access driver " + driverName + + ", " + e.getMessage(), e); + } catch (ClassNotFoundException e) { + throw new RuntimeException("Unable to find driver " + driverName + + ", " + e.getMessage(), e); + } + + String connString = HiveConf + .getVar(conf, HiveConf.ConfVars.HIVE_TXN_JDBC_CONNECT_STRING); + try { + LOG.info("Connecting to transaction db with connection string " + + connString); + dbConn = driver.connect(connString, new Properties()); + dbConn.setAutoCommit(false); + if (dbConn.getAutoCommit()) { + throw new RuntimeException("Transaction database doesn't support " + + "transactions!"); + } + Statement stmt = dbConn.createStatement(); + LOG.debug("Going to execute "); + stmt.execute("SET SCHEMA HIVETXNS"); + } catch (SQLException e) { + throw new RuntimeException("Unable to connect to transaction manager " + + "using " + connString + ", " + e.getMessage(), e); + } + + timeout = HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVE_TXN_TIMEOUT) * + 1000; + buildJumpTable(); + // TODO should probably make configurable + lockInfoCache = new LockInfoLRU(1000); + } + + public GetOpenTxnsInfoResponse getOpenTxnsInfo() { + // We need to figure out the current transaction number and the list of + // open transactions. To avoid needing a transaction on the underlying + // database we'll look at the current transaction number first. If it + // subsequently shows up in the open list that's ok. + try { + Statement stmt = dbConn.createStatement(); + LOG.debug("Going to execute query "); + rs = stmt.executeQuery("select txn_id, txn_state from txns"); + while (rs.next()) { + char c = rs.getString(2).charAt(0); + TxnState state; + switch (c) { + case 'a': + state = TxnState.ABORTED; + break; + + case 'o': + state = TxnState.OPEN; + break; + + default: + throw new RuntimeException("Unexpected transaction state " + c + + " found in txns table"); + } + txnInfo.add(new TxnInfo(rs.getLong(1), state)); + } + stmt.close(); + LOG.debug("Going to rollback"); + dbConn.rollback(); + return new GetOpenTxnsInfoResponse(hwm, txnInfo); + } catch (SQLException e) { + try { + LOG.debug("Going to rollback"); + dbConn.rollback(); + } catch (SQLException e1) { + } + throw new RuntimeException("Unable to select from transaction database, " + + e.getMessage(), e); + } + } + + public GetOpenTxnsResponse getOpenTxns() { + // We need to figure out the current transaction number and the list of + // open transactions. To avoid needing a transaction on the underlying + // database we'll look at the current transaction number first. If it + // subsequently shows up in the open list that's ok. + try { + Statement stmt = dbConn.createStatement(); + LOG.debug("Going to execute query "); + rs = stmt.executeQuery("select txn_id from txns"); + while (rs.next()) { + openList.add(rs.getLong(1)); + } + stmt.close(); + LOG.debug("Going to rollback"); + dbConn.rollback(); + return new GetOpenTxnsResponse(hwm, openList); + } catch (SQLException e) { + try { + LOG.debug("Going to rollback"); + dbConn.rollback(); + } catch (SQLException e1) { + } + throw new RuntimeException("Unable to select from transaction database, " + + e.getMessage(), e); + } + } + + public OpenTxnsResponse openTxns(OpenTxnRequest rqst) { + int numTxns = rqst.getNum_txns(); + try { + // Make sure the user has not requested an insane amount of txns. + int maxTxns = HiveConf.getIntVar(conf, + HiveConf.ConfVars.HIVE_TXN_MAX_OPEN_BATCH); + if (numTxns > maxTxns) numTxns = maxTxns; + + Statement stmt = dbConn.createStatement(); + LOG.debug("Going to execute query "); + ResultSet rs = stmt.executeQuery("select nl_next from next_lock_id " + + "for update"); + if (!rs.next()) { + LOG.debug("Going to rollback"); + dbConn.rollback(); + throw new RuntimeException("Transaction tables not properly " + + "initialized, no record found in next_lock_id"); + } + long extLockId = rs.getLong(1); + String s = "update next_lock_id set nl_next = " + (extLockId + 1); + LOG.debug("Going to execute update <" + s + ">"); + stmt.executeUpdate(s); + List lockInfoList = + new ArrayList(rqst .getComponentSize()); + long intLockId = 0; + for (LockComponent lc : rqst.getComponent()) { + intLockId++; + String dbName = lc.getDbname(); + String tblName = lc.getTablename(); + String partName = lc.getPartitionname(); + LockType lockType = lc.getType(); + char lockChar = 'z'; + switch (lockType) { + case EXCLUSIVE: lockChar = 'e'; break; + case SHARED_READ: lockChar = 'r'; break; + case SHARED_WRITE: lockChar = 'w'; break; + } + String lockObjectData = lc.getLock_object_data(); + long now = System.currentTimeMillis(); + s = "insert into hive_locks " + + " (hl_lock_ext_id, hl_lock_int_id, hl_txnid, hl_db, hl_table, " + + "hl_partition, hl_lock_state, hl_lock_type, hl_last_heartbeat, " + + "hl_object_data)" + " values (" + extLockId + ", " + + + intLockId + "," + (txnid >= 0 ? txnid : "null") + ", '" + + dbName + "', " + (tblName == null ? "null" : "'" + tblName + "'" ) + + ", " + (partName == null ? "null" : "'" + partName + "'") + + ", 'w', " + "'" + lockChar + "', " + now + ", " + + (lockObjectData == null ? "null" : "'" + lockObjectData + "'") + + ")"; + LOG.debug("Going to execute update <" + s + ">"); + stmt.executeUpdate(s); + lockInfoList.add(new LockInfo(extLockId, intLockId, txnid, dbName, + tblName, partName)); + } + lockInfoCache.put(extLockId, lockInfoList); + LOG.debug("Going to commit"); + dbConn.commit(); + return checkLock(extLockId, txnid); + } catch (SQLException e) { + try { + LOG.debug("Going to rollback"); + dbConn.rollback(); + } catch (SQLException e1) { + } + throw new RuntimeException("Unable to connect to transaction database " + + e.getMessage()); + } catch (NoSuchLockException e) { + // This should never happen, as we just added the lock id + throw new RuntimeException("Couldn't find a lock we just created!"); + } + } + + public LockResponse checkLock(CheckLockRequest rqst) + throws NoSuchTxnException, NoSuchLockException, TxnAbortedException { + long extLockId = rqst.getLockid(); + // Clean up timed out locks + timeOutLocks(); + + // Heartbeat on the lockid first, to assure that our lock is still valid. + // Then look up the lock info (hopefully in the cache). If these locks + // are associated with a transaction then heartbeat on that as well. + heartbeatLock(extLockId); + // TODO This isn't quite optimal, in the case where the cache has been + // flushed we'll double hit the db, once for txnid and once for reading all + // the lockinfo. Should fix this. + long txnid = getTxnIdFromLockId(extLockId); + if (txnid > 0) heartbeatTxn(txnid); + return checkLock(extLockId, txnid); + } + + public void unlock(UnlockRequest rqst) + throws NoSuchLockException, TxnOpenException { + // Odd as it seems, we need to heartbeat first because this touches the + // lock table and assures that our locks our still valid. If they are + // not, this will throw an exception and the heartbeat will fail. + long extLockId = rqst.getLockid(); + heartbeatLock(extLockId); + long txnid = getTxnIdFromLockId(extLockId); + // If there is a valid txnid, throw an exception, + // as locks associated with transactions should be unlocked only when the + // transaction is committed or aborted. + if (txnid > 0) { + try { + LOG.debug("Going to rollback"); + dbConn.rollback(); + } catch (SQLException e1) { + } + throw new TxnOpenException("Unlocking locks associated with transaction" + + " not permitted. Lockid " + extLockId + " is associated with " + + "transaction " + txnid); + } + try { + Statement stmt = dbConn.createStatement(); + String s = "delete from hive_locks where hl_lock_ext_id = " + extLockId; + LOG.debug("Going to execute update <" + s + ">"); + int rc = stmt.executeUpdate(s); + if (rc < 1) { + LOG.debug("Going to rollback"); + dbConn.rollback(); + throw new NoSuchLockException("No such lock: " + extLockId); + } + LOG.debug("Going to commit"); + dbConn.commit(); + } catch (SQLException e) { + try { + LOG.debug("Going to rollback"); + dbConn.rollback(); + } catch (SQLException e1) { + } + throw new RuntimeException("Unable to connect to transaction database"); + } + } + + public void heartbeat(HeartbeatRequest ids) + throws NoSuchTxnException, NoSuchLockException, TxnAbortedException { + heartbeatLock(ids.getLockid()); + heartbeatTxn(ids.getTxnid()); + try { + LOG.debug("Going to commit"); + dbConn.commit(); + } catch (SQLException e) { + } + } + + /** + * For testing only, do not use. + */ + int numLocksInLockTable() throws SQLException { + Statement stmt = dbConn.createStatement(); + String s = "select count(*) from hive_locks"; + LOG.debug("Going to execute query <" + s + ">"); + ResultSet rs = stmt.executeQuery(s); + rs.next(); + int rc = rs.getInt(1); + // Necessary to clean up the transaction in the db. + dbConn.rollback(); + return rc; + } + + /** + * For testing only, do not use. + */ + long setTimeout(long milliseconds) { + long previous_timeout = timeout; + timeout = milliseconds; + return previous_timeout; + } + + private static class LockInfo { + long extLockId; + long intLockId; + long txnId; + String db; + String table; + String partition; + LockState state; + LockType type; + + // Assumes the result set is set to a valid row + LockInfo(ResultSet rs) throws SQLException { + extLockId = rs.getLong("hl_lock_ext_id"); // can't be null + intLockId = rs.getLong("hl_lock_int_id"); // can't be null + db = rs.getString("hl_db"); // can't be null + String t = rs.getString("hl_table"); + table = (rs.wasNull() ? null : t); + String p = rs.getString("hl_partition"); + partition = (rs.wasNull() ? null : p); + switch (rs.getString("hl_lock_state").charAt(0)) { + case 'w': state = LockState.WAITING; break; + case 'a': state = LockState.ACQUIRED; break; + } + switch (rs.getString("hl_lock_type").charAt(0)) { + case 'e': type = LockType.EXCLUSIVE; break; + case 'r': type = LockType.SHARED_READ; break; + case 'w': type = LockType.SHARED_WRITE; break; + } + } + + LockInfo(long elid, long ilid, long tid, String d, String t, String p) { + extLockId = elid; + intLockId = ilid; + txnId = tid; + db = d; + table = t; + partition = p; + } + + public boolean equals(Object other) { + if (!(other instanceof LockInfo)) return false; + LockInfo o = (LockInfo)other; + // Lock ids are unique across the system. + return extLockId == o.extLockId && intLockId == o.intLockId; + } + + @Override + public String toString() { + return "extLockId:" + Long.toString(extLockId) + " intLockId:" + + intLockId + " txnId:" + Long.toString + (txnId) + " db:" + db + " table:" + table + " partition:" + + partition + " state:" + (state == null ? "null" : state.toString()) + + " type:" + (type == null ? "null" : type.toString()); + } + } + + private static class LockInfoComparator implements Comparator { + public boolean equals(Object other) { + return this == other; + } + + public int compare(LockInfo info1, LockInfo info2) { + // We sort by state (acquired vs waiting) and then by extLockId. + if (info1.state == LockState.ACQUIRED && + info2.state != LockState .ACQUIRED) { + return -1; + } + if (info1.state != LockState.ACQUIRED && + info2.state == LockState .ACQUIRED) { + return 1; + } + if (info1.extLockId < info2.extLockId) { + return -1; + } else if (info1.extLockId > info2.extLockId) { + return 1; + } else { + if (info1.intLockId < info2.intLockId) { + return -1; + } else if (info1.intLockId > info2.intLockId) { + return 1; + } else { + return 0; + } + } + } + } + + private static class LockInfoLRU extends LinkedHashMap> { + int maxSize; + + LockInfoLRU(int size) { + // Size +2 for capacity so that we have room for 1 extra (before + // removeEldestEntry is called) and one more for buffer so we don't + // ever rehash. + super(size + 2, 1.0f, true); + maxSize = size; + } + + protected boolean removeEldestEntry(Map.Entry eldest) { + return size() > maxSize; + } + } + + // A jump table to figure out whether to wait, acquire, + // or keep looking . Since + // java doesn't have function pointers (grumble grumble) we store a + // character that we'll use to determine which function to call. + // The table maps the lock type of the lock we are looking to acquire to + // the lock type of the lock we are checking to the lock state of the lock + // we are checking to the desired action. + private Map>> jumpTable; + + private void checkQFileTestHack() { + boolean hackOn = HiveConf.getBoolVar(conf, + HiveConf.ConfVars.HIVE_TXN_TESTING, false); + if (hackOn) { + LOG.info("Hacking in canned values for transaction manager"); + // Set up the transaction/locking db in the derby metastore + /* + String jdbcString = + "jdbc:derby:;databaseName=metastore_db;create=true"; + String jdbcDriver = "org.apache.derby.jdbc.EmbeddedDriver"; + conf.setVar(HiveConf.ConfVars.HIVE_TXN_MANAGER, + "org.apache.hadoop.hive.ql.lockmgr.DbTxnManager"); + conf.setVar(HiveConf.ConfVars.HIVE_TXN_JDBC_DRIVER, + jdbcDriver); + conf.setVar(HiveConf.ConfVars.HIVE_TXN_JDBC_CONNECT_STRING, + jdbcString); + */ + TxnDbUtil.setConfValues(conf); + try { + TxnDbUtil.prepDb(); + } catch (Exception e) { + if (!e.getMessage().contains("Schema 'HIVETXNS' already exists")) { + throw new RuntimeException("Unable to set up transaction database for" + + " testing: " + e.getMessage()); + } + } + } + } + + private LockResponse checkLock(long extLockId, long txnid) + throws NoSuchLockException, NoSuchTxnException, TxnAbortedException { + List locksBeingChecked = getLockInfoFromLockId(extLockId); + LockResponse response = new LockResponse(); + response.setLockid(extLockId); + + long now = System.currentTimeMillis(); + try { + LOG.debug("Setting savepoint"); + Savepoint save = dbConn.setSavepoint(); + Statement stmt = dbConn.createStatement(); + StringBuffer query = new StringBuffer("select hl_lock_ext_id, " + + "hl_lock_int_id, hl_db, hl_table, hl_partition, hl_lock_state, " + + "hl_lock_type from hive_locks where hl_db in ("); + + Set strings = new HashSet(locksBeingChecked.size()); + for (LockInfo info : locksBeingChecked) { + strings.add(info.db); + } + boolean first = true; + for (String s : strings) { + if (first) first = false; + else query.append(", "); + query.append('\''); + query.append(s); + query.append('\''); + } + query.append(")"); + + // If any of the table requests are null, then I need to pull all the + // table locks for this db. + boolean sawNull = false; + strings.clear(); + for (LockInfo info : locksBeingChecked) { + if (info.table == null) { + sawNull = true; + break; + } else { + strings.add(info.table); + } + } + if (!sawNull) { + query.append(" and (hl_table is null or hl_table in("); + first = true; + for (String s : strings) { + if (first) first = false; + else query.append(", "); + query.append('\''); + query.append(s); + query.append('\''); + } + query.append("))"); + + // If any of the partition requests are null, then I need to pull all + // partition locks for this table. + sawNull = false; + strings.clear(); + for (LockInfo info : locksBeingChecked) { + if (info.partition == null) { + sawNull = true; + break; + } else { + strings.add(info.partition); + } + } + if (!sawNull) { + query.append(" and (hl_partition is null or hl_partition in("); + first = true; + for (String s : strings) { + if (first) first = false; + else query.append(", "); + query.append('\''); + query.append(s); + query.append('\''); + } + query.append("))"); + } + } + query.append(" for update"); + + LOG.debug("Going to execute query <" + query.toString() + ">"); + ResultSet rs = stmt.executeQuery(query.toString()); + SortedSet lockSet = new TreeSet(new LockInfoComparator()); + while (rs.next()) { + lockSet.add(new LockInfo(rs)); + } + // Turn the tree set into an array so we can move back and forth easily + // in it. + LockInfo[] locks = (LockInfo[])lockSet.toArray(new LockInfo[1]); + + for (LockInfo info : locksBeingChecked) { + // Find the lock record we're checking + int index = -1; + for (int i = 0; i < locks.length; i++) { + if (locks[i].equals(info)) { + index = i; + break; + } + } + + // If we didn't find the lock, then it must not be in the table + if (index == -1) { + LOG.debug("Going to rollback"); + dbConn.rollback(); + throw new RuntimeException("How did we get here, " + + "we heartbeated our lock before we started!"); + } + + + // If we've found it and it's already been marked acquired, + // then just look at the other locks. + if (locks[index].state == LockState.ACQUIRED) { + continue; + } + + // Look at everything in front of this lock to see if it should block + // it or not. + for (int i = index - 1; i >= 0; i--) { + // Check if we're operating on the same database, if not, move on + if (!locks[index].db.equals(locks[i].db)) { + continue; + } + + // If table is null on either of these, then they are claiming to + // lock the whole database and we need to check it. Otherwise, + // check if they are operating on the same table, if not, move on. + if (locks[index].table != null && locks[i].table != null + && !locks[index].table.equals(locks[i].table)) { + continue; + } + + // If partition is null on either of these, then they are claiming to + // lock the whole table and we need to check it. Otherwise, + // check if they are operating on the same partition, if not, move on. + if (locks[index].partition != null && locks[i].partition != null + && !locks[index].partition.equals(locks[i].partition)) { + continue; + } + + // We've found something that matches what we're trying to lock, + // so figure out if we can lock it too. + switch (jumpTable.get(locks[index].type).get(locks[i].type).get + (locks[i].state)) { + case 'a': + acquire(stmt, extLockId, info.intLockId); + break; + case 'w': + wait(save); + LOG.debug("Going to commit"); + dbConn.commit(); + response.setState(LockState.WAITING); + return response; + case 'c': + continue; + } + } + + // If we've arrived here it means there's nothing in the way of the + // lock, so acquire the lock. + acquire(stmt, extLockId, info.intLockId); + } + + // We acquired all of the locks, so commit and return acquired. + LOG.debug("Going to commit"); + dbConn.commit(); + response.setState(LockState.ACQUIRED); + return response; + } catch (SQLException e) { + try { + LOG.debug("Going to rollback"); + dbConn.rollback(); + } catch (SQLException e1) { + } + throw new RuntimeException("Unable to connect to transaction database " + + e.getMessage()); + } + } + + private void wait(Savepoint save) throws SQLException { + // Need to rollback because we did a select for update but we didn't + // actually update anything. Also, we may have locked some locks as + // acquired that we now want to not acquire. It's ok to rollback because + // once we see one wait, we're done, we won't look for more. + // Only rollback to savepoint because we want to commit our heartbeat + // changes. + LOG.debug("Going to rollback to savepoint"); + dbConn.rollback(save); + } + + private void acquire(Statement stmt, long extLockId, long intLockId) + throws SQLException, NoSuchLockException { + long now = System.currentTimeMillis(); + String s = "update hive_locks set hl_lock_state = 'a', " + + "hl_last_heartbeat = " + now + " where hl_lock_ext_id = " + extLockId + + " and hl_lock_int_id = " + intLockId; + LOG.debug("Going to execute update <" + s + ">"); + int rc = stmt.executeUpdate(s); + if (rc < 1) { + LOG.debug("Going to rollback"); + dbConn.rollback(); + throw new NoSuchLockException("No such lock: (" + extLockId + "," + + + intLockId + ")"); + } + // We update the database, but we don't commit because there may be other + // locks together with this, and we only want to acquire one if we can + // acquire all. + } + + // Heartbeats on the lock table. This does not call commit as it assumes + // someone downstream will. However, it does lock rows in the lock table. + private void heartbeatLock(long extLockId) throws NoSuchLockException { + // If the lock id is 0, then there are no locks in this heartbeat + if (extLockId == 0) return; + try { + Statement stmt = dbConn.createStatement(); + long now = System.currentTimeMillis(); + + String s = "update hive_locks set hl_last_heartbeat = " + + now + " where hl_lock_ext_id = " + extLockId; + LOG.debug("Going to execute update <" + s + ">"); + int rc = stmt.executeUpdate(s); + if (rc < 1) { + LOG.debug("Going to rollback"); + dbConn.rollback(); + throw new NoSuchLockException("No such lock: " + extLockId); + } + } catch (SQLException e) { + try { + LOG.debug("Going to rollback"); + dbConn.rollback(); + } catch (SQLException e1) { + } + throw new RuntimeException("Unable to connect to transaction database"); + } + } + + // Heartbeats on the txn table. This does not call commit as it assumes + // someone downstream will. However, it does lock rows in the txn table. + private void heartbeatTxn(long txnid) + throws NoSuchTxnException, TxnAbortedException { + // If the txnid is 0, then there are no transactions in this heartbeat + if (txnid == 0) return; + try { + Statement stmt = dbConn.createStatement(); + long now = System.currentTimeMillis(); + // We need to check whether this transaction is valid and open + String s = "select txn_state from txns where txn_id = " + + txnid + "for update"; + LOG.debug("Going to execute query <" + s + ">"); + ResultSet rs = stmt.executeQuery(s); + if (!rs.next()) { + LOG.debug("Going to rollback"); + dbConn.rollback(); + throw new NoSuchTxnException("No such transaction: " + txnid); + } + if (rs.getString(1).charAt(0) == 'a') { + LOG.debug("Going to rollback"); + dbConn.rollback(); + throw new TxnAbortedException("Transaction " + txnid + + " already aborted"); + } + s = "update txns set txn_last_heartbeat = " + now + + " where txn_id = " + txnid; + LOG.debug("Going to execute update <" + s + ">"); + stmt.executeUpdate(s); + + } catch (SQLException e) { + try { + LOG.debug("Going to rollback"); + dbConn.rollback(); + } catch (SQLException e1) { + } + throw new RuntimeException("Unable to connect to transaction database " + + e.getMessage()); + } + } + + // NEVER call this function without first calling heartbeat(long, long) + private long getTxnIdFromLockId(long extLockId) throws NoSuchLockException { + // Try to look in our cache first. We do not need to worry about our + // cache being out of date with the database because we have already done + // a heartbeat that touched the database and would have failed if our + // locks had timed out. + List lockInfoList = lockInfoCache.get(extLockId); + if (lockInfoList != null) { + return lockInfoList.get(0).txnId; + } + try { + Statement stmt = dbConn.createStatement(); + String s = "select hl_txnid from hive_locks where hl_lock_ext_id = " + + extLockId; + LOG.debug("Going to execute query <" + s + ">"); + ResultSet rs = stmt.executeQuery(s); + if (!rs.next()) { + throw new RuntimeException("This should never happen! We already " + + "checked the lock existed but now we can't find it!"); + } + long txnid = rs.getLong(1); + return (rs.wasNull() ? -1 : txnid); + } catch (SQLException e) { + throw new RuntimeException("Unable to connect to transaction database"); + } + } + + // NEVER call this function without first calling heartbeat(long, long) + private List getLockInfoFromLockId(long extLockId) throws + NoSuchLockException { + // Try to look in our cache first. We do not need to worry about our + // cache being out of date with the database because we have already done + // a heartbeat that touched the database and would have failed if our + // locks or transaction had timed out. + List lockInfoList = lockInfoCache.get(extLockId); + if (lockInfoList != null) { + return lockInfoList; + } + try { + Statement stmt = dbConn.createStatement(); + String s = "select hl_lock_ext_id, hl_lock_int_id, hl_db, hl_table, " + + "hl_partition, hl_lock_state, hl_lock_type from hive_locks where " + + "hl_lock_ext_id = " + extLockId; + LOG.debug("Going to execute query <" + s + ">"); + ResultSet rs = stmt.executeQuery(s); + if (!rs.next()) { + throw new RuntimeException("This should never happen! We already " + + "checked the lock existed but now we can't find it!"); + } + List ourLockInfo = new ArrayList(); + while (rs.next()) { + ourLockInfo.add(new LockInfo(rs)); + } + lockInfoCache.put(extLockId, ourLockInfo); + return ourLockInfo; + } catch (SQLException e) { + throw new RuntimeException("Unable to connect to transaction database " + + e.getMessage()); + } + } + + // Clean time out locks from the database. This does a commit, + // and thus should be done before any calls to heartbeat that will leave + // open transactions. + private void timeOutLocks() { + try { + long now = System.currentTimeMillis(); + Statement stmt = dbConn.createStatement(); + // Remove any timed out locks from the table. + String s = "delete from hive_locks where hl_last_heartbeat < " + + (now - timeout); + LOG.debug("Going to execute update <" + s + ">"); + stmt.executeUpdate(s); + LOG.debug("Going to commit"); + dbConn.commit(); + } catch (SQLException e) { + try { + LOG.debug("Going to rollback"); + dbConn.rollback(); + } catch (SQLException e1) { + } + throw new RuntimeException("Unable to connect to transaction database " + + e.getMessage()); + } + } + + private void buildJumpTable() { + jumpTable = + new HashMap>>(3); + + // SR: Lock we are trying to acquire is shared read + Map> m = + new HashMap>(3); + jumpTable.put(LockType.SHARED_READ, m); + + // SR.SR: Lock we are examining is shared read + Map m2 = new HashMap(2); + m.put(LockType.SHARED_READ, m2); + + // SR.SR.acquired Lock we are examining is acquired; We can acquire + // because two shared reads can acquire together and there must be + // nothing in front of this one to prevent acquisition. + m2.put(LockState.ACQUIRED, 'a'); + + // SR.SR.wait Lock we are examining is waiting. In this case we keep + // looking, as it's possible that something in front is blocking it or + // that the other locker hasn't checked yet and he could lock as well. + m2.put(LockState.WAITING, 'c'); + + // SR.SW: Lock we are examining is shared write + m2 = new HashMap(2); + m.put(LockType.SHARED_WRITE, m2); + + // SR.SW.acquired Lock we are examining is acquired; We can acquire + // because a read can share with a write, and there must be + // nothing in front of this one to prevent acquisition. + m2.put(LockState.ACQUIRED, 'a'); + + // SR.SW.wait Lock we are examining is waiting. In this case we keep + // looking, as it's possible that something in front is blocking it or + // that the other locker hasn't checked yet and he could lock as well or + // that something is blocking it that would not block a read. + m2.put(LockState.WAITING, 'c'); + + // SR.E: Lock we are examining is exclusive + m2 = new HashMap(2); + m.put(LockType.EXCLUSIVE, m2); + + // No matter whether it has acquired or not, we cannot pass an exclusive. + m2.put(LockState.ACQUIRED, 'w'); + m2.put(LockState.WAITING, 'w'); + + // SW: Lock we are trying to acquire is shared write + m = new HashMap>(3); + jumpTable.put(LockType.SHARED_WRITE, m); + + // SW.SR: Lock we are examining is shared read + m2 = new HashMap(2); + m.put(LockType.SHARED_READ, m2); + + // SW.SR.acquired Lock we are examining is acquired; We need to keep + // looking, because there may or may not be another shared write in front + // that would block us. + m2.put(LockState.ACQUIRED, 'c'); + + // SW.SR.wait Lock we are examining is waiting. In this case we keep + // looking, as it's possible that something in front is blocking it or + // that the other locker hasn't checked yet and he could lock as well. + m2.put(LockState.WAITING, 'c'); + + // SW.SW: Lock we are examining is shared write + m2 = new HashMap(2); + m.put(LockType.SHARED_WRITE, m2); + + // Regardless of acquired or waiting, one shared write cannot pass another. + m2.put(LockState.ACQUIRED, 'w'); + m2.put(LockState.WAITING, 'w'); + + // SW.E: Lock we are examining is exclusive + m2 = new HashMap(2); + m.put(LockType.EXCLUSIVE, m2); + + // No matter whether it has acquired or not, we cannot pass an exclusive. + m2.put(LockState.ACQUIRED, 'w'); + m2.put(LockState.WAITING, 'w'); + + // E: Lock we are trying to acquire is exclusive + m = new HashMap>(3); + jumpTable.put(LockType.EXCLUSIVE, m); + + // E.SR: Lock we are examining is shared read + m2 = new HashMap(2); + m.put(LockType.SHARED_READ, m2); + + // Exclusives can never pass + m2.put(LockState.ACQUIRED, 'w'); + m2.put(LockState.WAITING, 'w'); + + // E.SW: Lock we are examining is shared write + m2 = new HashMap(2); + m.put(LockType.SHARED_WRITE, m2); + + // Exclusives can never pass + m2.put(LockState.ACQUIRED, 'w'); + m2.put(LockState.WAITING, 'w'); + + // E.E: Lock we are examining is exclusive + m2 = new HashMap(2); + m.put(LockType.EXCLUSIVE, m2); + + // No matter whether it has acquired or not, we cannot pass an exclusive. + m2.put(LockState.ACQUIRED, 'w'); + m2.put(LockState.WAITING, 'w'); + } +} diff --git metastore/src/test/org/apache/hadoop/hive/metastore/TestLockRequestBuilder.java metastore/src/test/org/apache/hadoop/hive/metastore/TestLockRequestBuilder.java new file mode 100644 index 0000000..c985960 --- /dev/null +++ metastore/src/test/org/apache/hadoop/hive/metastore/TestLockRequestBuilder.java @@ -0,0 +1,561 @@ +/** + * 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 junit.framework.Assert; +import org.apache.hadoop.hive.metastore.api.LockComponent; +import org.apache.hadoop.hive.metastore.api.LockLevel; +import org.apache.hadoop.hive.metastore.api.LockRequest; +import org.apache.hadoop.hive.metastore.api.LockType; +import org.junit.Test; + +import java.util.List; + +/** + * Tests for LockRequestBuilder. + */ +public class TestLockRequestBuilder { + + // Test that database and table don't coalesce. + @Test + public void testDbTable() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", null, null); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", null); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(2, locks.size()); + } + + // Test that database and table don't coalesce. + @Test + public void testTablePartition() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", null); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", "mypartition"); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(2, locks.size()); + } + + // Test that 2 separate databases don't coalesce. + @Test + public void testTwoSeparateDbs() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", null, null); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "yourdb", null, null); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(2, locks.size()); + } + + // Test that 2 exclusive db locks coalesce to one + @Test + public void testExExDb() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", null, null); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", null, null); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + } + + // Test that existing exclusive db with new shared_write coalesces to + // exclusive + @Test + public void testExSWDb() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", null, null); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", null, null); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.EXCLUSIVE, locks.get(0).getType()); + } + + // Test that existing exclusive db with new shared_read coalesces to + // exclusive + @Test + public void testExSRDb() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", null, null); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, + "mydb", null, null); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.EXCLUSIVE, locks.get(0).getType()); + } + + // Test that existing shared_write db with new exclusive coalesces to + // exclusive + @Test + public void testSWExDb() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", null, null); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", null, null); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.EXCLUSIVE, locks.get(0).getType()); + } + + // Test that existing shared_write db with new shared_write coalesces to + // shared_write + @Test + public void testSWSWDb() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", null, null); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", null, null); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.SHARED_WRITE, locks.get(0).getType()); + } + + // Test that existing shared_write db with new shared_read coalesces to + // shared_write + @Test + public void testSWSRDb() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", null, null); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, + "mydb", null, null); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.SHARED_WRITE, locks.get(0).getType()); + } + + // Test that existing shared_read db with new exclusive coalesces to + // exclusive + @Test + public void testSRExDb() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, + "mydb", null, null); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", null, null); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.EXCLUSIVE, locks.get(0).getType()); + } + + // Test that existing shared_read db with new shared_write coalesces to + // shared_write + @Test + public void testSRSWDb() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, + "mydb", null, null); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", null, null); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.SHARED_WRITE, locks.get(0).getType()); + } + + // Test that existing shared_read db with new shared_read coalesces to + // shared_read + @Test + public void testSRSRDb() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, + "mydb", null, null); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, + "mydb", null, null); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.SHARED_READ, locks.get(0).getType()); + } + + // Test that 2 separate tables don't coalesce. + @Test + public void testTwoSeparateTables() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", null); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "yourtable", null); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(2, locks.size()); + } + + // Test that 2 exclusive table locks coalesce to one + @Test + public void testExExTable() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", null); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", null); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + } + + // Test that existing exclusive table with new shared_write coalesces to + // exclusive + @Test + public void testExSWTable() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", null); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", "mytable", null); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.EXCLUSIVE, locks.get(0).getType()); + } + + // Test that existing exclusive table with new shared_read coalesces to + // exclusive + @Test + public void testExSRTable() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", null); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, + "mydb", "mytable", null); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.EXCLUSIVE, locks.get(0).getType()); + } + + // Test that existing shared_write table with new exclusive coalesces to + // exclusive + @Test + public void testSWExTable() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", "mytable", null); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", null); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.EXCLUSIVE, locks.get(0).getType()); + } + + // Test that existing shared_write table with new shared_write coalesces to + // shared_write + @Test + public void testSWSWTable() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", "mytable", null); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", "mytable", null); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.SHARED_WRITE, locks.get(0).getType()); + } + + // Test that existing shared_write table with new shared_read coalesces to + // shared_write + @Test + public void testSWSRTable() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", "mytable", null); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, + "mydb", "mytable", null); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.SHARED_WRITE, locks.get(0).getType()); + } + + // Test that existing shared_read table with new exclusive coalesces to + // exclusive + @Test + public void testSRExTable() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, + "mydb", "mytable", null); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", null); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.EXCLUSIVE, locks.get(0).getType()); + } + + // Test that existing shared_read table with new shared_write coalesces to + // shared_write + @Test + public void testSRSWTable() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, + "mydb", "mytable", null); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", "mytable", null); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.SHARED_WRITE, locks.get(0).getType()); + } + + // Test that existing shared_read table with new shared_read coalesces to + // shared_read + @Test + public void testSRSRTable() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, + "mydb", "mytable", null); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, + "mydb", "mytable", null); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.SHARED_READ, locks.get(0).getType()); + } + + // Test that 2 separate partitions don't coalesce. + @Test + public void testTwoSeparatePartitions() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", "mypart"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", "yourpart"); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(2, locks.size()); + } + + // Test that 2 exclusive partition locks coalesce to one + @Test + public void testExExPart() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", "mypart"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", "mypart"); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + } + + // Test that existing exclusive partition with new shared_write coalesces to + // exclusive + @Test + public void testExSWPart() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", "mypart"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", "mytable", "mypart"); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.EXCLUSIVE, locks.get(0).getType()); + } + + // Test that existing exclusive partition with new shared_read coalesces to + // exclusive + @Test + public void testExSRPart() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", "mypart"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, + "mydb", "mytable", "mypart"); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.EXCLUSIVE, locks.get(0).getType()); + } + + // Test that existing shared_write partition with new exclusive coalesces to + // exclusive + @Test + public void testSWExPart() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", "mytable", "mypart"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", "mypart"); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.EXCLUSIVE, locks.get(0).getType()); + } + + // Test that existing shared_write partition with new shared_write coalesces to + // shared_write + @Test + public void testSWSWPart() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", "mytable", "mypart"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", "mytable", "mypart"); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.SHARED_WRITE, locks.get(0).getType()); + } + + // Test that existing shared_write partition with new shared_read coalesces to + // shared_write + @Test + public void testSWSRPart() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", "mytable", "mypart"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, + "mydb", "mytable", "mypart"); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.SHARED_WRITE, locks.get(0).getType()); + } + + // Test that existing shared_read partition with new exclusive coalesces to + // exclusive + @Test + public void testSRExPart() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, + "mydb", "mytable", "mypart"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", "mypart"); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.EXCLUSIVE, locks.get(0).getType()); + } + + // Test that existing shared_read partition with new shared_write coalesces to + // shared_write + @Test + public void testSRSWPart() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, + "mydb", "mytable", "mypart"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", "mytable", "mypart"); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.SHARED_WRITE, locks.get(0).getType()); + } + + // Test that existing shared_read partition with new shared_read coalesces to + // shared_read + @Test + public void testSRSRPart() { + LockRequestBuilder bldr = new LockRequestBuilder(); + LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, + "mydb", "mytable", "mypart"); + bldr.addLockComponent(comp); + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, + "mydb", "mytable", "mypart"); + bldr.addLockComponent(comp); + LockRequest req = bldr.build(); + List locks = req.getComponent(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(LockType.SHARED_READ, locks.get(0).getType()); + } +} diff --git metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java new file mode 100644 index 0000000..32df78e --- /dev/null +++ metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java @@ -0,0 +1,925 @@ +/** + * 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.txn; + +import junit.framework.Assert; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.*; +import org.apache.log4j.Level; +import org.apache.log4j.LogManager; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +/** + * Tests for TxnHandler. + */ +public class TestTxnHandler { + + private HiveConf conf = new HiveConf(); + private TxnHandler txnHandler; + + public TestTxnHandler() throws Exception { + TxnDbUtil.setConfValues(conf); + LogManager.getLogger(TxnHandler.class.getName()).setLevel(Level.DEBUG); + tearDown(); + } + + @Test + public void testValidTxnsEmpty() throws Exception { + GetOpenTxnsInfoResponse txnsInfo = txnHandler.getOpenTxnsInfo(); + Assert.assertEquals(0L, txnsInfo.getTxn_high_water_mark()); + Assert.assertTrue(txnsInfo.getOpen_txns().isEmpty()); + GetOpenTxnsResponse txns = txnHandler.getOpenTxns(); + Assert.assertEquals(0L, txns.getTxn_high_water_mark()); + Assert.assertTrue(txns.getOpen_txns().isEmpty()); + } + + @Test + public void testOpenTxn() throws Exception { + long first = openTxn(); + Assert.assertEquals(1L, first); + long second = openTxn(); + Assert.assertEquals(2L, second); + GetOpenTxnsInfoResponse txnsInfo = txnHandler.getOpenTxnsInfo(); + Assert.assertEquals(2L, txnsInfo.getTxn_high_water_mark()); + Assert.assertEquals(2, txnsInfo.getOpen_txns().size()); + Assert.assertEquals(1L, txnsInfo.getOpen_txns().get(0).getId()); + Assert.assertEquals(TxnState.OPEN, txnsInfo.getOpen_txns().get(0).getState()); + Assert.assertEquals(2L, txnsInfo.getOpen_txns().get(1).getId()); + Assert.assertEquals(TxnState.OPEN, txnsInfo.getOpen_txns().get(1).getState()); + + GetOpenTxnsResponse txns = txnHandler.getOpenTxns(); + Assert.assertEquals(2L, txns.getTxn_high_water_mark()); + Assert.assertEquals(2, txns.getOpen_txns().size()); + boolean[] saw = new boolean[3]; + for (int i = 0; i < saw.length; i++) saw[i] = false; + for (Long tid : txns.getOpen_txns()) { + saw[tid.intValue()] = true; + } + for (int i = 1; i < saw.length; i++) Assert.assertTrue(saw[i]); + } + + @Test + public void testAbortTxn() throws Exception { + OpenTxnsResponse openedTxns = txnHandler.openTxns(new OpenTxnRequest(2)); + List txnList = openedTxns.getTxn_ids(); + long first = txnList.get(0); + Assert.assertEquals(1L, first); + long second = txnList.get(1); + Assert.assertEquals(2L, second); + txnHandler.abortTxn(new AbortTxnRequest(1)); + GetOpenTxnsInfoResponse txnsInfo = txnHandler.getOpenTxnsInfo(); + Assert.assertEquals(2L, txnsInfo.getTxn_high_water_mark()); + Assert.assertEquals(2, txnsInfo.getOpen_txns().size()); + Assert.assertEquals(1L, txnsInfo.getOpen_txns().get(0).getId()); + Assert.assertEquals(TxnState.ABORTED, txnsInfo.getOpen_txns().get(0).getState()); + Assert.assertEquals(2L, txnsInfo.getOpen_txns().get(1).getId()); + Assert.assertEquals(TxnState.OPEN, txnsInfo.getOpen_txns().get(1).getState()); + + GetOpenTxnsResponse txns = txnHandler.getOpenTxns(); + Assert.assertEquals(2L, txns.getTxn_high_water_mark()); + Assert.assertEquals(2, txns.getOpen_txns().size()); + boolean[] saw = new boolean[3]; + for (int i = 0; i < saw.length; i++) saw[i] = false; + for (Long tid : txns.getOpen_txns()) { + saw[tid.intValue()] = true; + } + for (int i = 1; i < saw.length; i++) Assert.assertTrue(saw[i]); + } + + @Test + public void testAbortInvalidTxn() throws Exception { + boolean caught = false; + try { + txnHandler.abortTxn(new AbortTxnRequest(195L)); + } catch (NoSuchTxnException e) { + caught = true; + } + Assert.assertTrue(caught); + } + + @Test + public void testValidTxnsNoneOpen() throws Exception { + txnHandler.openTxns(new OpenTxnRequest(2)); + txnHandler.commitTxn(new CommitTxnRequest(1)); + txnHandler.commitTxn(new CommitTxnRequest(2)); + GetOpenTxnsInfoResponse txnsInfo = txnHandler.getOpenTxnsInfo(); + Assert.assertEquals(2L, txnsInfo.getTxn_high_water_mark()); + Assert.assertEquals(0, txnsInfo.getOpen_txns().size()); + GetOpenTxnsResponse txns = txnHandler.getOpenTxns(); + Assert.assertEquals(2L, txns.getTxn_high_water_mark()); + Assert.assertEquals(0, txns.getOpen_txns().size()); + } + + @Test + public void testValidTxnsSomeOpen() throws Exception { + txnHandler.openTxns(new OpenTxnRequest(3)); + txnHandler.abortTxn(new AbortTxnRequest(1)); + txnHandler.commitTxn(new CommitTxnRequest(2)); + GetOpenTxnsInfoResponse txnsInfo = txnHandler.getOpenTxnsInfo(); + Assert.assertEquals(3L, txnsInfo.getTxn_high_water_mark()); + Assert.assertEquals(2, txnsInfo.getOpen_txns().size()); + Assert.assertEquals(1L, txnsInfo.getOpen_txns().get(0).getId()); + Assert.assertEquals(TxnState.ABORTED, txnsInfo.getOpen_txns().get(0).getState()); + Assert.assertEquals(3L, txnsInfo.getOpen_txns().get(1).getId()); + Assert.assertEquals(TxnState.OPEN, txnsInfo.getOpen_txns().get(1).getState()); + + GetOpenTxnsResponse txns = txnHandler.getOpenTxns(); + Assert.assertEquals(3L, txns.getTxn_high_water_mark()); + Assert.assertEquals(2, txns.getOpen_txns().size()); + boolean[] saw = new boolean[4]; + for (int i = 0; i < saw.length; i++) saw[i] = false; + for (Long tid : txns.getOpen_txns()) { + saw[tid.intValue()] = true; + } + Assert.assertTrue(saw[1]); + Assert.assertFalse(saw[2]); + Assert.assertTrue(saw[3]); + } + + @Test + public void testLockDifferentDBs() throws Exception { + // Test that two different databases don't collide on their locks + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", null, null); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components); + LockResponse res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "yourdb", null, null); + components.clear(); + components.add(comp); + req = new LockRequest(components); + res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + } + + @Test + public void testLockSameDB() throws Exception { + // Test that two different databases don't collide on their locks + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", null, null); + List components = new ArrayList(1); + comp.setLock_object_data("Four score and seven years ago..."); + components.add(comp); + LockRequest req = new LockRequest(components); + LockResponse res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", null, null); + components.clear(); + components.add(comp); + req = new LockRequest(components); + res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.WAITING); + } + + @Test + public void testLockDbLocksTable() throws Exception { + // Test that locking a database prevents locking of tables in the database + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", null, null); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components); + LockResponse res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", null); + components.clear(); + components.add(comp); + req = new LockRequest(components); + res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.WAITING); + } + + @Test + public void testLockDbDoesNotLockTableInDifferentDB() throws Exception { + // Test that locking a database prevents locking of tables in the database + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", null, null); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components); + LockResponse res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "yourdb", "mytable", null); + components.clear(); + components.add(comp); + req = new LockRequest(components); + res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + } + + @Test + public void testLockDifferentTables() throws Exception { + // Test that two different tables don't collide on their locks + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", null); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components); + LockResponse res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "yourtable", null); + components.clear(); + components.add(comp); + req = new LockRequest(components); + res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + } + + @Test + public void testLockSameTable() throws Exception { + // Test that two different tables don't collide on their locks + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", null); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components); + LockResponse res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", null); + components.clear(); + components.add(comp); + req = new LockRequest(components); + res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.WAITING); + } + + @Test + public void testLockTableLocksPartition() throws Exception { + // Test that locking a table prevents locking of partitions of the table + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", null); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components); + LockResponse res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", "mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components); + res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.WAITING); + } + + @Test + public void testLockDifferentTableDoesntLockPartition() throws Exception { + // Test that locking a table prevents locking of partitions of the table + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", null); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components); + LockResponse res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "yourtable", "mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components); + res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + } + + @Test + public void testLockDifferentPartitions() throws Exception { + // Test that two different partitions don't collide on their locks + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", "mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components); + LockResponse res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", "yourpartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components); + res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + } + + @Test + public void testLockSamePartition() throws Exception { + // Test that two different partitions don't collide on their locks + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", "mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components); + LockResponse res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", "mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components); + res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.WAITING); + } + + @Test + public void testLockSRSR() throws Exception { + // Test that two shared read locks can share a partition + LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, + "mydb", "mytable", "mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components); + LockResponse res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, + "mydb", "mytable", "mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components); + res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + } + + @Test + public void testLockESRSR() throws Exception { + // Test that exclusive lock blocks shared reads + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", "mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components); + LockResponse res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, + "mydb", "mytable", "mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components); + res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.WAITING); + + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, + "mydb", "mytable", "mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components); + res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.WAITING); + } + + @Test + public void testLockSRSW() throws Exception { + // Test that write can acquire after read + LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, + "mydb", "mytable", "mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components); + LockResponse res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", "mytable", "mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components); + res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + } + + @Test + public void testLockESRSW() throws Exception { + // Test that exclusive lock blocks read and write + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", "mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components); + LockResponse res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, + "mydb", "mytable", "mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components); + res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.WAITING); + + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", "mytable", "mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components); + res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.WAITING); + } + + @Test + public void testLockSRE() throws Exception { + // Test that read blocks exclusive + LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, + "mydb", "mytable", "mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components); + LockResponse res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", "mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components); + res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.WAITING); + } + + @Test + public void testLockESRE() throws Exception { + // Test that exclusive blocks read and exclusive + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", "mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components); + LockResponse res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, + "mydb", "mytable", "mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components); + res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.WAITING); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", "mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components); + res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.WAITING); + } + + @Test + public void testLockSWSR() throws Exception { + // Test that read can acquire after write + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", "mytable", "mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components); + LockResponse res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, + "mydb", "mytable", "mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components); + res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + } + + @Test + public void testLockSWSWSR() throws Exception { + // Test that write blocks write but read can still acquire + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", "mytable", "mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components); + LockResponse res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", "mytable", "mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components); + res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.WAITING); + + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, + "mydb", "mytable", "mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components); + res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + } + + @Test + public void testLockSWSWSW() throws Exception { + // Test that write blocks two writes + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", "mytable", "mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components); + LockResponse res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", "mytable", "mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components); + res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.WAITING); + + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", "mytable", "mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components); + res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.WAITING); + } + + @Test + public void testLockEESW() throws Exception { + // Test that exclusive blocks exclusive and write + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", "mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components); + LockResponse res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", "mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components); + res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.WAITING); + + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", "mytable", "mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components); + res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.WAITING); + } + + @Test + public void testLockEESR() throws Exception { + // Test that exclusive blocks exclusive and read + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", "mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components); + LockResponse res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", "mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components); + res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.WAITING); + + comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, + "mydb", "mytable", "mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components); + res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.WAITING); + } + + @Test + public void testCheckLockAcquireAfterWaiting() throws Exception { + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", "mytable", "mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components); + LockResponse res = txnHandler.lock(req); + long lockid1 = res.getLockid(); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + + comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", "mytable", "mypartition"); + components.clear(); + components.add(comp); + req = new LockRequest(components); + res = txnHandler.lock(req); + long lockid2 = res.getLockid(); + Assert.assertTrue(res.getState() == LockState.WAITING); + + txnHandler.unlock(new UnlockRequest(lockid1)); + res = txnHandler.checkLock(new CheckLockRequest(lockid2)); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + } + + @Test + public void testCheckLockNoSuchLock() throws Exception { + try { + txnHandler.checkLock(new CheckLockRequest(23L)); + Assert.fail("Allowed to check lock on non-existent lock"); + } catch (NoSuchLockException e) { + } + } + + @Test + public void testCheckLockTxnAborted() throws Exception { + // Test that when a transaction is aborted, the heartbeat fails + long txnid = openTxn(); + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", "mytable", "mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components); + req.setTxnid(txnid); + LockResponse res = txnHandler.lock(req); + long lockid = res.getLockid(); + txnHandler.abortTxn(new AbortTxnRequest(txnid)); + try { + // This will throw NoSuchLockException (even though it's the + // transaction we've closed) because that will have deleted the lock. + txnHandler.checkLock(new CheckLockRequest(lockid)); + Assert.fail("Allowed to check lock on aborted transaction."); + } catch (NoSuchLockException e) { + } + } + + @Test + public void testMultipleLock() throws Exception { + // Test more than one lock can be handled in a lock request + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", "mypartition"); + List components = new ArrayList(2); + components.add(comp); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", "anotherpartition"); + components.add(comp); + LockRequest req = new LockRequest(components); + LockResponse res = txnHandler.lock(req); + long lockid = res.getLockid(); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + res = txnHandler.checkLock(new CheckLockRequest(lockid)); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + txnHandler.unlock(new UnlockRequest(lockid)); + Assert.assertEquals(0, txnHandler.numLocksInLockTable()); + } + + @Test + public void testMultipleLockWait() throws Exception { + // Test that two shared read locks can share a partition + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", "mypartition"); + List components = new ArrayList(2); + components.add(comp); + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", "anotherpartition"); + components.add(comp); + LockRequest req = new LockRequest(components); + LockResponse res = txnHandler.lock(req); + long lockid1 = res.getLockid(); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + + + comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", "mypartition"); + components = new ArrayList(1); + components.add(comp); + req = new LockRequest(components); + res = txnHandler.lock(req); + long lockid2 = res.getLockid(); + Assert.assertTrue(res.getState() == LockState.WAITING); + + txnHandler.unlock(new UnlockRequest(lockid1)); + + res = txnHandler.checkLock(new CheckLockRequest(lockid2)); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + } + + @Test + public void testUnlockOnCommit() throws Exception { + // Test that committing unlocks + long txnid = openTxn(); + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", "mytable", null); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components); + req.setTxnid(txnid); + LockResponse res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + txnHandler.commitTxn(new CommitTxnRequest(txnid)); + Assert.assertEquals(0, txnHandler.numLocksInLockTable()); + } + + @Test + public void testUnlockOnAbort() throws Exception { + // Test that committing unlocks + long txnid = openTxn(); + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", null, null); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components); + req.setTxnid(txnid); + LockResponse res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + txnHandler.abortTxn(new AbortTxnRequest(txnid)); + Assert.assertEquals(0, txnHandler.numLocksInLockTable()); + } + + @Test + public void testUnlockWithTxn() throws Exception { + // Test that attempting to unlock locks associated with a transaction + // generates an error + long txnid = openTxn(); + LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, + "mydb", "mytable", "mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components); + req.setTxnid(txnid); + LockResponse res = txnHandler.lock(req); + long lockid = res.getLockid(); + try { + txnHandler.unlock(new UnlockRequest(lockid)); + Assert.fail("Allowed to unlock lock associated with transaction."); + } catch (TxnOpenException e) { + } + } + + /* + public void testHeartbeatTxn() throws Exception { + // Test that our transaction doesn't get timedout when we heartbeat + // regularly. + // TODO, need to wait for the compactor + } + + public void testHeartbeatTxnTimeout() throws Exception { + // Test that our transaction doesn't get timedout when we heartbeat + // regularly. + // TODO, need to wait for the compactor + } + */ + + @Test + public void testHeartbeatTxnAborted() throws Exception { + // Test that when a transaction is aborted, the heartbeat fails + openTxn(); + txnHandler.abortTxn(new AbortTxnRequest(1)); + HeartbeatRequest h = new HeartbeatRequest(); + h.setTxnid(1); + try { + txnHandler.heartbeat(h); + Assert.fail("Told there was a txn, when it should have been aborted."); + } catch (TxnAbortedException e) { + } + } + + @Test + public void testHeartbeatNoTxn() throws Exception { + // Test that when a transaction is aborted, the heartbeat fails + HeartbeatRequest h = new HeartbeatRequest(); + h.setTxnid(939393L); + try { + txnHandler.heartbeat(h); + Assert.fail("Told there was a txn, when there wasn't."); + } catch (NoSuchTxnException e) { + } + } + + @Test + public void testHeartbeatLock() throws Exception { + conf.setIntVar(HiveConf.ConfVars.HIVE_TXN_TIMEOUT, 1); + HeartbeatRequest h = new HeartbeatRequest(); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", "mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components); + LockResponse res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + h.setLockid(res.getLockid()); + for (int i = 0; i < 30; i++) { + try { + txnHandler.heartbeat(h); + } catch (NoSuchLockException e) { + Assert.fail("Told there was no lock, when the heartbeat should have kept it."); + } + } + } + + @Test + public void testLockTimeout() throws Exception { + long timeout = txnHandler.setTimeout(1); + LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, + "mydb", "mytable", "mypartition"); + List components = new ArrayList(1); + components.add(comp); + LockRequest req = new LockRequest(components); + LockResponse res = txnHandler.lock(req); + Assert.assertTrue(res.getState() == LockState.ACQUIRED); + Thread.currentThread().sleep(10); + try { + txnHandler.checkLock(new CheckLockRequest(res.getLockid())); + Assert.fail("Told there was a lock, when it should have timed out."); + } catch (NoSuchLockException e) { + } finally { + txnHandler.setTimeout(timeout); + } + } + + @Test + public void testHeartbeatNoLock() throws Exception { + HeartbeatRequest h = new HeartbeatRequest(); + h.setLockid(29389839L); + try { + txnHandler.heartbeat(h); + Assert.fail("Told there was a lock, when there wasn't."); + } catch (NoSuchLockException e) { + } + } + + @Test + public void testNoJDBCDriver() throws Exception { + conf.unset(HiveConf.ConfVars.HIVE_TXN_JDBC_DRIVER.varname); + boolean sawException = false; + try { + txnHandler = new TxnHandler(conf); + } catch (Exception e) { + if (e instanceof RuntimeException && e.getMessage().contains("JDBC " + + "driver for transaction db not set")) { + sawException = true; + } else { + throw e; + } + } finally { + TxnDbUtil.setConfValues(conf); + } + Assert.assertTrue(sawException); + + } + + @Before + public void setUp() throws Exception { + TxnDbUtil.prepDb(); + txnHandler = new TxnHandler(conf); + } + + @After + public void tearDown() throws Exception { + TxnDbUtil.cleanDb(); + } + + private long openTxn() { + List txns = txnHandler.openTxns(new OpenTxnRequest(1)).getTxn_ids(); + return txns.get(0); + } + +} diff --git ql/src/java/org/apache/hadoop/hive/ql/Context.java ql/src/java/org/apache/hadoop/hive/ql/Context.java index ab97985..b7e1b00 100644 --- ql/src/java/org/apache/hadoop/hive/ql/Context.java +++ ql/src/java/org/apache/hadoop/hive/ql/Context.java @@ -18,18 +18,6 @@ package org.apache.hadoop.hive.ql; -import java.io.DataInput; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.net.URI; -import java.text.SimpleDateFormat; -import java.util.Date; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.concurrent.ConcurrentHashMap; - import org.antlr.runtime.TokenRewriteStream; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -46,10 +34,23 @@ import org.apache.hadoop.hive.ql.lockmgr.HiveLock; import org.apache.hadoop.hive.ql.lockmgr.HiveLockManager; import org.apache.hadoop.hive.ql.lockmgr.HiveLockObj; +import org.apache.hadoop.hive.ql.lockmgr.HiveTxnManager; import org.apache.hadoop.hive.ql.plan.LoadTableDesc; import org.apache.hadoop.hive.shims.ShimLoader; import org.apache.hadoop.util.StringUtils; +import java.io.DataInput; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.net.URI; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.ConcurrentHashMap; + /** * Context for Semantic Analyzers. Usage: not reusable - construct a new one for * each query should call clear() at end of use to remove temporary folders @@ -93,6 +94,9 @@ protected List hiveLocks; protected HiveLockManager hiveLockMgr; + // Transaction manager for this query + protected HiveTxnManager hiveTxnManager; + private boolean needLockMgr; // Keep track of the mapping from load table desc to the output and the lock @@ -523,15 +527,12 @@ public void setHiveLocks(List hiveLocks) { this.hiveLocks = hiveLocks; } - public HiveLockManager getHiveLockMgr() { - if (hiveLockMgr != null) { - hiveLockMgr.refresh(); - } - return hiveLockMgr; + public HiveTxnManager getHiveTxnManager() { + return hiveTxnManager; } - public void setHiveLockMgr(HiveLockManager hiveLockMgr) { - this.hiveLockMgr = hiveLockMgr; + public void setHiveTxnManager(HiveTxnManager txnMgr) { + hiveTxnManager = txnMgr; } public void setOriginalTracker(String originalTracker) { diff --git ql/src/java/org/apache/hadoop/hive/ql/Driver.java ql/src/java/org/apache/hadoop/hive/ql/Driver.java index 5af1ec6..f3758f7 100644 --- ql/src/java/org/apache/hadoop/hive/ql/Driver.java +++ ql/src/java/org/apache/hadoop/hive/ql/Driver.java @@ -19,21 +19,6 @@ package org.apache.hadoop.hive.ql; -import java.io.DataInput; -import java.io.IOException; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.LinkedHashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Queue; -import java.util.Set; -import java.util.concurrent.ConcurrentLinkedQueue; - import org.apache.commons.lang.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -44,15 +29,7 @@ import org.apache.hadoop.hive.metastore.api.Database; import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.Schema; -import org.apache.hadoop.hive.ql.exec.ConditionalTask; -import org.apache.hadoop.hive.ql.exec.FetchTask; -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.TableScanOperator; -import org.apache.hadoop.hive.ql.exec.Task; -import org.apache.hadoop.hive.ql.exec.TaskFactory; -import org.apache.hadoop.hive.ql.exec.TaskResult; -import org.apache.hadoop.hive.ql.exec.TaskRunner; -import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.exec.*; import org.apache.hadoop.hive.ql.history.HiveHistory.Keys; import org.apache.hadoop.hive.ql.hooks.Entity; import org.apache.hadoop.hive.ql.hooks.Entity.Type; @@ -64,39 +41,15 @@ import org.apache.hadoop.hive.ql.hooks.PreExecute; import org.apache.hadoop.hive.ql.hooks.ReadEntity; import org.apache.hadoop.hive.ql.hooks.WriteEntity; -import org.apache.hadoop.hive.ql.lockmgr.HiveLock; -import org.apache.hadoop.hive.ql.lockmgr.HiveLockManager; -import org.apache.hadoop.hive.ql.lockmgr.HiveLockManagerCtx; -import org.apache.hadoop.hive.ql.lockmgr.HiveLockMode; -import org.apache.hadoop.hive.ql.lockmgr.HiveLockObj; -import org.apache.hadoop.hive.ql.lockmgr.HiveLockObject; +import org.apache.hadoop.hive.ql.lockmgr.*; import org.apache.hadoop.hive.ql.lockmgr.HiveLockObject.HiveLockObjectData; -import org.apache.hadoop.hive.ql.lockmgr.LockException; import org.apache.hadoop.hive.ql.log.PerfLogger; -import org.apache.hadoop.hive.ql.metadata.AuthorizationException; -import org.apache.hadoop.hive.ql.metadata.DummyPartition; -import org.apache.hadoop.hive.ql.metadata.Hive; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.metadata.Partition; -import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.metadata.*; import org.apache.hadoop.hive.ql.metadata.formatting.JsonMetaDataFormatter; import org.apache.hadoop.hive.ql.metadata.formatting.MetaDataFormatUtils; import org.apache.hadoop.hive.ql.metadata.formatting.MetaDataFormatter; import org.apache.hadoop.hive.ql.optimizer.ppr.PartitionPruner; -import org.apache.hadoop.hive.ql.parse.ASTNode; -import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer; -import org.apache.hadoop.hive.ql.parse.HiveSemanticAnalyzerHook; -import org.apache.hadoop.hive.ql.parse.HiveSemanticAnalyzerHookContext; -import org.apache.hadoop.hive.ql.parse.HiveSemanticAnalyzerHookContextImpl; -import org.apache.hadoop.hive.ql.parse.ImportSemanticAnalyzer; -import org.apache.hadoop.hive.ql.parse.ParseContext; -import org.apache.hadoop.hive.ql.parse.ParseDriver; -import org.apache.hadoop.hive.ql.parse.ParseUtils; -import org.apache.hadoop.hive.ql.parse.PrunedPartitionList; -import org.apache.hadoop.hive.ql.parse.SemanticAnalyzer; -import org.apache.hadoop.hive.ql.parse.SemanticAnalyzerFactory; -import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.ql.parse.VariableSubstitution; +import org.apache.hadoop.hive.ql.parse.*; import org.apache.hadoop.hive.ql.plan.HiveOperation; import org.apache.hadoop.hive.ql.plan.OperatorDesc; import org.apache.hadoop.hive.ql.plan.TableDesc; @@ -112,7 +65,13 @@ import org.apache.hadoop.mapred.ClusterStatus; import org.apache.hadoop.mapred.JobClient; import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.util.ReflectionUtils; + +import java.io.DataInput; +import java.io.IOException; +import java.io.Serializable; +import java.util.*; +import java.util.concurrent.ConcurrentLinkedQueue; + public class Driver implements CommandProcessor { @@ -130,11 +89,10 @@ private Context ctx; private QueryPlan plan; private Schema schema; - private HiveLockManager hiveLockMgr; - private String errorMessage; private String SQLState; private Throwable downstreamError; + private HiveTxnManager txnMgr; // A limit on the number of threads that can be launched private int maxthreads; @@ -143,46 +101,35 @@ private String userName; + private boolean createTxnManager() { + if (txnMgr == null) { + try { + txnMgr = TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf); + } catch (LockException e) { + errorMessage = "FAILED: Error in semantic analysis: " + e.getMessage(); + SQLState = ErrorMsg.findSQLState(e.getMessage()); + downstreamError = e; + console.printError(errorMessage, "\n" + + org.apache.hadoop.util.StringUtils.stringifyException(e)); + return false; + } + } + return true; + } + private boolean checkConcurrency() throws SemanticException { boolean supportConcurrency = conf.getBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY); if (!supportConcurrency) { LOG.info("Concurrency mode is disabled, not creating a lock manager"); return false; } - createLockManager(); - // the reason that we set the lock manager for the cxt here is because each - // query has its own ctx object. The hiveLockMgr is shared accross the + createTxnManager(); + // the reason that we set the txn manager for the cxt here is because each + // query has its own ctx object. The txn mgr is shared across the // same instance of Driver, which can run multiple queries. - ctx.setHiveLockMgr(hiveLockMgr); - return true; - } + ctx.setHiveTxnManager(txnMgr); - private void createLockManager() throws SemanticException { - if (hiveLockMgr != null) { - return; - } - String lockMgr = conf.getVar(HiveConf.ConfVars.HIVE_LOCK_MANAGER); - LOG.info("Creating lock manager of type " + lockMgr); - if ((lockMgr == null) || (lockMgr.isEmpty())) { - throw new SemanticException(ErrorMsg.LOCKMGR_NOT_SPECIFIED.getMsg()); - } - try { - hiveLockMgr = (HiveLockManager) ReflectionUtils.newInstance(conf.getClassByName(lockMgr), - conf); - hiveLockMgr.setContext(new HiveLockManagerCtx(conf)); - } catch (Exception e1) { - // set hiveLockMgr to null just in case this invalid manager got set to - // next query's ctx. - if (hiveLockMgr != null) { - try { - hiveLockMgr.close(); - } catch (LockException e2) { - //nothing can do here - } - hiveLockMgr = null; - } - throw new SemanticException(ErrorMsg.LOCKMGR_NOT_INITIALIZED.getMsg() + e1.getMessage(), e1); - } + return true; } public void init() { @@ -833,123 +780,26 @@ public QueryPlan getPlan() { } /** - * Dedup the list of lock objects so that there is only one lock per table/partition. - * If there is both a shared and exclusive lock for the same object, this will deduped - * to just a single exclusive lock. - * @param lockObjects - */ - static void dedupLockObjects(List lockObjects) { - Map lockMap = new HashMap(); - for (HiveLockObj lockObj : lockObjects) { - String lockName = lockObj.getName(); - HiveLockObj foundLock = lockMap.get(lockName); - if (foundLock == null || lockObj.getMode() == HiveLockMode.EXCLUSIVE) { - lockMap.put(lockName, lockObj); - } - } - // copy set of deduped locks back to original list - lockObjects.clear(); - for (HiveLockObj lockObj : lockMap.values()) { - lockObjects.add(lockObj); - } - } - - /** * Acquire read and write locks needed by the statement. The list of objects to be locked are * obtained from he inputs and outputs populated by the compiler. The lock acuisition scheme is * pretty simple. If all the locks cannot be obtained, error out. Deadlock is avoided by making * sure that the locks are lexicographically sorted. **/ - public int acquireReadWriteLocks() { + private int acquireReadWriteLocks() { PerfLogger perfLogger = PerfLogger.getPerfLogger(); perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.ACQUIRE_READ_WRITE_LOCKS); - try { - boolean supportConcurrency = conf.getBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY); - if (!supportConcurrency) { - return 0; - } - - List lockObjects = new ArrayList(); - - // Sort all the inputs, outputs. - // If a lock needs to be acquired on any partition, a read lock needs to be acquired on all - // its parents also - for (ReadEntity input : plan.getInputs()) { - if (input.getType() == ReadEntity.Type.DATABASE) { - lockObjects.addAll(getLockObjects(input.getDatabase(), null, null, HiveLockMode.SHARED)); - } else if (input.getType() == ReadEntity.Type.TABLE) { - lockObjects.addAll(getLockObjects(null, input.getTable(), null, HiveLockMode.SHARED)); - } else { - lockObjects.addAll(getLockObjects(null, null, input.getPartition(), HiveLockMode.SHARED)); - } - } - - for (WriteEntity output : plan.getOutputs()) { - List lockObj = null; - if (output.getType() == WriteEntity.Type.DATABASE) { - lockObjects.addAll(getLockObjects(output.getDatabase(), null, null, - output.isComplete() ? HiveLockMode.EXCLUSIVE : HiveLockMode.SHARED)); - } else if (output.getTyp() == WriteEntity.Type.TABLE) { - lockObj = getLockObjects(null, output.getTable(), null, - output.isComplete() ? HiveLockMode.EXCLUSIVE : HiveLockMode.SHARED); - } else if (output.getTyp() == WriteEntity.Type.PARTITION) { - lockObj = getLockObjects(null, null, output.getPartition(), HiveLockMode.EXCLUSIVE); - } - // In case of dynamic queries, it is possible to have incomplete dummy partitions - else if (output.getTyp() == WriteEntity.Type.DUMMYPARTITION) { - lockObj = getLockObjects(null, null, output.getPartition(), HiveLockMode.SHARED); - } - - if(lockObj != null) { - lockObjects.addAll(lockObj); - ctx.getOutputLockObjects().put(output, lockObj); - } - } - - if (lockObjects.isEmpty() && !ctx.isNeedLockMgr()) { - return 0; - } - - HiveLockObjectData lockData = - new HiveLockObjectData(plan.getQueryId(), - String.valueOf(System.currentTimeMillis()), - "IMPLICIT", - plan.getQueryStr()); - - // Lock the database also - String currentDb = SessionState.get().getCurrentDatabase(); - lockObjects.add( - new HiveLockObj( - new HiveLockObject(currentDb, lockData), - HiveLockMode.SHARED - ) - ); - - dedupLockObjects(lockObjects); - List hiveLocks = ctx.getHiveLockMgr().lock(lockObjects, false); - - if (hiveLocks == null) { - throw new SemanticException(ErrorMsg.LOCK_CANNOT_BE_ACQUIRED.getMsg()); - } else { - ctx.setHiveLocks(hiveLocks); - } - return (0); - } catch (SemanticException e) { - errorMessage = "FAILED: Error in acquiring locks: " + e.getMessage(); - SQLState = ErrorMsg.findSQLState(e.getMessage()); - downstreamError = e; - console.printError(errorMessage, "\n" - + org.apache.hadoop.util.StringUtils.stringifyException(e)); - return (10); + try { + txnMgr.acquireLocks(plan, ctx); + return 0; } catch (LockException e) { errorMessage = "FAILED: Error in acquiring locks: " + e.getMessage(); SQLState = ErrorMsg.findSQLState(e.getMessage()); downstreamError = e; console.printError(errorMessage, "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e)); - return (10); + return 10; } finally { perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.ACQUIRE_READ_WRITE_LOCKS); } @@ -960,12 +810,12 @@ else if (output.getTyp() == WriteEntity.Type.DUMMYPARTITION) { * list of hive locks to be released Release all the locks specified. If some of the * locks have already been released, ignore them **/ - private void releaseLocks(List hiveLocks) { + private void releaseLocks(List hiveLocks) throws LockException { PerfLogger perfLogger = PerfLogger.getPerfLogger(); perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.RELEASE_LOCKS); if (hiveLocks != null) { - ctx.getHiveLockMgr().releaseLocks(hiveLocks); + ctx.getHiveTxnManager().getLockManager().releaseLocks(hiveLocks); } ctx.setHiveLocks(null); @@ -1051,7 +901,12 @@ private int compileInternal(String command) { ret = compile(command); } if (ret != 0) { - releaseLocks(ctx.getHiveLocks()); + try { + releaseLocks(ctx.getHiveLocks()); + } catch (LockException e) { + LOG.warn("Exception in releasing locks. " + + org.apache.hadoop.util.StringUtils.stringifyException(e)); + } } return ret; } @@ -1136,10 +991,15 @@ private CommandProcessorResponse runInternal(String command, boolean alreadyComp } } + if (requireLock) { ret = acquireReadWriteLocks(); if (ret != 0) { - releaseLocks(ctx.getHiveLocks()); + try { + releaseLocks(ctx.getHiveLocks()); + } catch (LockException e) { + // Not much to do here + } return new CommandProcessorResponse(ret, errorMessage, SQLState); } } @@ -1147,12 +1007,25 @@ private CommandProcessorResponse runInternal(String command, boolean alreadyComp ret = execute(); if (ret != 0) { //if needRequireLock is false, the release here will do nothing because there is no lock - releaseLocks(ctx.getHiveLocks()); + try { + releaseLocks(ctx.getHiveLocks()); + } catch (LockException e) { + // Nothing to do here + } return new CommandProcessorResponse(ret, errorMessage, SQLState); } //if needRequireLock is false, the release here will do nothing because there is no lock - releaseLocks(ctx.getHiveLocks()); + try { + releaseLocks(ctx.getHiveLocks()); + } catch (LockException e) { + errorMessage = "FAILED: Hive Internal Error: " + Utilities.getNameMessage(e); + SQLState = ErrorMsg.findSQLState(e.getMessage()); + downstreamError = e; + console.printError(errorMessage + "\n" + + org.apache.hadoop.util.StringUtils.stringifyException(e)); + return new CommandProcessorResponse(12, errorMessage, SQLState); + } perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.DRIVER_RUN); perfLogger.close(LOG, plan); @@ -1704,17 +1577,16 @@ public int close() { public void destroy() { if (ctx != null) { - releaseLocks(ctx.getHiveLocks()); - } - - if (hiveLockMgr != null) { try { - hiveLockMgr.close(); - } catch(LockException e) { - LOG.warn("Exception in closing hive lock manager. " - + org.apache.hadoop.util.StringUtils.stringifyException(e)); + releaseLocks(ctx.getHiveLocks()); + } catch (LockException e) { + LOG.warn("Exception when releasing locking in destroy: " + + e.getMessage()); } } + if (txnMgr != null) { + txnMgr.closeTxnManager(); + } } public org.apache.hadoop.hive.ql.plan.api.Query getQueryPlan() throws IOException { diff --git ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java index f8be581..12d5fef 100644 --- ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java +++ ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java @@ -18,17 +18,17 @@ package org.apache.hadoop.hive.ql; +import org.antlr.runtime.tree.Tree; +import org.apache.hadoop.hive.ql.metadata.HiveUtils; +import org.apache.hadoop.hive.ql.parse.ASTNode; +import org.apache.hadoop.hive.ql.parse.ASTNodeOrigin; + import java.text.MessageFormat; import java.util.HashMap; import java.util.Map; import java.util.regex.Matcher; import java.util.regex.Pattern; -import org.antlr.runtime.tree.Tree; -import org.apache.hadoop.hive.ql.metadata.HiveUtils; -import org.apache.hadoop.hive.ql.parse.ASTNode; -import org.apache.hadoop.hive.ql.parse.ASTNodeOrigin; - /** * List of all error messages. * This list contains both compile time and run-time errors. @@ -370,11 +370,29 @@ INVALID_HDFS_URI(10251, "{0} is not a hdfs uri", true), INVALID_DIR(10252, "{0} is not a directory", true), NO_VALID_LOCATIONS(10253, "Could not find any valid location to place the jars. " + - "Please update hive.jar.directory or hive.user.install.directory with a valid location", false), + "Please update hive.jar.directory or hive.user.install.directory with a valid location", false), UNNSUPPORTED_AUTHORIZATION_PRINCIPAL_TYPE_GROUP(10254, "Principal type GROUP is not supported in this authorization setting", "28000"), INVALID_TABLE_NAME(10255, "Invalid table name {0}", true), + TXNMGR_NOT_SPECIFIED(10260, "Transaction manager not specified correctly, " + + "set hive.txn.manager"), + TXNMGR_NOT_INSTANTIATED(10261, "Transaction manager could not be " + + "instantiated, check hive.txn.manager"), + TXN_NO_SUCH_TRANSACTION(10262, "No record of transaction could be found, " + + "may have timed out"), + TXN_ABORTED(10263, "Transaction manager has aborted the transaction."), + + LOCK_NO_SUCH_LOCK(10270, "No record of lock could be found, " + + "may have timed out"), + LOCK_REQUEST_UNSUPPORTED(10271, "Current transaction manager does not " + + "support explicit lock requests. Transaction manager: "), + + METASTORE_COMMUNICATION_FAILED(10280, "Error communicating with the " + + "metastore"), + METASTORE_COULD_NOT_INITIATE(10281, "Unable to initiate connection to the " + + "metastore."), + //========================== 20000 range starts here ========================// SCRIPT_INIT_ERROR(20000, "Unable to initialize custom script."), SCRIPT_IO_ERROR(20001, "An error occurred while reading or writing to your custom script. " diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java index 92ed55b..163b610 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java @@ -48,11 +48,8 @@ import org.apache.commons.lang.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.*; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FsShell; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.common.type.HiveDecimal; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; @@ -60,24 +57,7 @@ import org.apache.hadoop.hive.metastore.ProtectMode; import org.apache.hadoop.hive.metastore.TableType; import org.apache.hadoop.hive.metastore.Warehouse; -import org.apache.hadoop.hive.metastore.api.AlreadyExistsException; -import org.apache.hadoop.hive.metastore.api.Database; -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege; -import org.apache.hadoop.hive.metastore.api.HiveObjectRef; -import org.apache.hadoop.hive.metastore.api.HiveObjectType; -import org.apache.hadoop.hive.metastore.api.Index; -import org.apache.hadoop.hive.metastore.api.InvalidOperationException; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; -import org.apache.hadoop.hive.metastore.api.Order; -import org.apache.hadoop.hive.metastore.api.PrincipalType; -import org.apache.hadoop.hive.metastore.api.PrivilegeBag; -import org.apache.hadoop.hive.metastore.api.PrivilegeGrantInfo; -import org.apache.hadoop.hive.metastore.api.Role; -import org.apache.hadoop.hive.metastore.api.SerDeInfo; -import org.apache.hadoop.hive.metastore.api.SkewedInfo; -import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.*; import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.DriverContext; import org.apache.hadoop.hive.ql.ErrorMsg; @@ -89,71 +69,17 @@ import org.apache.hadoop.hive.ql.io.rcfile.merge.MergeWork; import org.apache.hadoop.hive.ql.io.rcfile.truncate.ColumnTruncateTask; import org.apache.hadoop.hive.ql.io.rcfile.truncate.ColumnTruncateWork; -import org.apache.hadoop.hive.ql.lockmgr.HiveLock; -import org.apache.hadoop.hive.ql.lockmgr.HiveLockManager; -import org.apache.hadoop.hive.ql.lockmgr.HiveLockMode; -import org.apache.hadoop.hive.ql.lockmgr.HiveLockObject; +import org.apache.hadoop.hive.ql.lockmgr.*; import org.apache.hadoop.hive.ql.lockmgr.HiveLockObject.HiveLockObjectData; -import org.apache.hadoop.hive.ql.metadata.CheckResult; -import org.apache.hadoop.hive.ql.metadata.Hive; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.metadata.HiveMetaStoreChecker; -import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler; -import org.apache.hadoop.hive.ql.metadata.HiveUtils; -import org.apache.hadoop.hive.ql.metadata.InvalidTableException; +import org.apache.hadoop.hive.ql.metadata.*; import org.apache.hadoop.hive.ql.metadata.Partition; import org.apache.hadoop.hive.ql.metadata.Table; import org.apache.hadoop.hive.ql.metadata.formatting.MetaDataFormatUtils; import org.apache.hadoop.hive.ql.metadata.formatting.MetaDataFormatter; import org.apache.hadoop.hive.ql.parse.AlterTablePartMergeFilesDesc; import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer; -import org.apache.hadoop.hive.ql.plan.AddPartitionDesc; -import org.apache.hadoop.hive.ql.plan.AlterDatabaseDesc; -import org.apache.hadoop.hive.ql.plan.AlterIndexDesc; -import org.apache.hadoop.hive.ql.plan.AlterTableAlterPartDesc; -import org.apache.hadoop.hive.ql.plan.AlterTableDesc; +import org.apache.hadoop.hive.ql.plan.*; import org.apache.hadoop.hive.ql.plan.AlterTableDesc.AlterTableTypes; -import org.apache.hadoop.hive.ql.plan.AlterTableExchangePartition; -import org.apache.hadoop.hive.ql.plan.AlterTableSimpleDesc; -import org.apache.hadoop.hive.ql.plan.CreateDatabaseDesc; -import org.apache.hadoop.hive.ql.plan.CreateIndexDesc; -import org.apache.hadoop.hive.ql.plan.CreateTableDesc; -import org.apache.hadoop.hive.ql.plan.CreateTableLikeDesc; -import org.apache.hadoop.hive.ql.plan.CreateViewDesc; -import org.apache.hadoop.hive.ql.plan.DDLWork; -import org.apache.hadoop.hive.ql.plan.DescDatabaseDesc; -import org.apache.hadoop.hive.ql.plan.DescFunctionDesc; -import org.apache.hadoop.hive.ql.plan.DescTableDesc; -import org.apache.hadoop.hive.ql.plan.DropDatabaseDesc; -import org.apache.hadoop.hive.ql.plan.DropIndexDesc; -import org.apache.hadoop.hive.ql.plan.DropTableDesc; -import org.apache.hadoop.hive.ql.plan.GrantDesc; -import org.apache.hadoop.hive.ql.plan.GrantRevokeRoleDDL; -import org.apache.hadoop.hive.ql.plan.LockDatabaseDesc; -import org.apache.hadoop.hive.ql.plan.LockTableDesc; -import org.apache.hadoop.hive.ql.plan.MsckDesc; -import org.apache.hadoop.hive.ql.plan.PartitionSpec; -import org.apache.hadoop.hive.ql.plan.PrincipalDesc; -import org.apache.hadoop.hive.ql.plan.PrivilegeDesc; -import org.apache.hadoop.hive.ql.plan.PrivilegeObjectDesc; -import org.apache.hadoop.hive.ql.plan.RenamePartitionDesc; -import org.apache.hadoop.hive.ql.plan.RevokeDesc; -import org.apache.hadoop.hive.ql.plan.RoleDDLDesc; -import org.apache.hadoop.hive.ql.plan.ShowColumnsDesc; -import org.apache.hadoop.hive.ql.plan.ShowCreateTableDesc; -import org.apache.hadoop.hive.ql.plan.ShowDatabasesDesc; -import org.apache.hadoop.hive.ql.plan.ShowFunctionsDesc; -import org.apache.hadoop.hive.ql.plan.ShowGrantDesc; -import org.apache.hadoop.hive.ql.plan.ShowIndexesDesc; -import org.apache.hadoop.hive.ql.plan.ShowLocksDesc; -import org.apache.hadoop.hive.ql.plan.ShowPartitionsDesc; -import org.apache.hadoop.hive.ql.plan.ShowTableStatusDesc; -import org.apache.hadoop.hive.ql.plan.ShowTablesDesc; -import org.apache.hadoop.hive.ql.plan.ShowTblPropertiesDesc; -import org.apache.hadoop.hive.ql.plan.SwitchDatabaseDesc; -import org.apache.hadoop.hive.ql.plan.TruncateTableDesc; -import org.apache.hadoop.hive.ql.plan.UnlockDatabaseDesc; -import org.apache.hadoop.hive.ql.plan.UnlockTableDesc; import org.apache.hadoop.hive.ql.plan.api.StageType; import org.apache.hadoop.hive.ql.security.authorization.Privilege; import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizer; @@ -177,6 +103,15 @@ import org.apache.hadoop.util.ToolRunner; import org.stringtemplate.v4.ST; +import java.io.*; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.*; +import java.util.Map.Entry; + +import static org.apache.commons.lang.StringUtils.join; +import static org.apache.hadoop.util.StringUtils.stringifyException; + /** * DDLTask implementation. * @@ -1012,7 +947,7 @@ private int createIndex(Hive db, CreateIndexDesc crtIndex) throws HiveException MetaStoreUtils.getIndexTableName(SessionState.get().getCurrentDatabase(), crtIndex.getTableName(), crtIndex.getIndexName()); Table indexTable = db.getTable(indexTableName); - work.getOutputs().add(new WriteEntity(indexTable)); + work.getOutputs().add(new WriteEntity(indexTable, WriteEntity.WriteType.DDL)); } return 0; } @@ -1105,7 +1040,7 @@ private int alterIndex(Hive db, AlterIndexDesc alterIndex) throws HiveException private int addPartitions(Hive db, AddPartitionDesc addPartitionDesc) throws HiveException { List parts = db.createPartitions(addPartitionDesc); for (Partition part : parts) { - work.getOutputs().add(new WriteEntity(part)); + work.getOutputs().add(new WriteEntity(part, WriteEntity.WriteType.INSERT)); } return 0; } @@ -1131,7 +1066,7 @@ private int renamePartition(Hive db, RenamePartitionDesc renamePartitionDesc) th Partition newPart = db .getPartition(tbl, renamePartitionDesc.getNewPartSpec(), false); work.getInputs().add(new ReadEntity(oldPart)); - work.getOutputs().add(new WriteEntity(newPart)); + work.getOutputs().add(new WriteEntity(newPart, WriteEntity.WriteType.DDL)); return 0; } @@ -1173,7 +1108,7 @@ private int alterTableAlterPart(Hive db, AlterTableAlterPartDesc alterPartitionD } work.getInputs().add(new ReadEntity(tbl)); - work.getOutputs().add(new WriteEntity(tbl)); + work.getOutputs().add(new WriteEntity(tbl, WriteEntity.WriteType.DDL)); return 0; } @@ -1202,7 +1137,7 @@ private int touch(Hive db, AlterTableSimpleDesc touchDesc) throw new HiveException("Uable to update table"); } work.getInputs().add(new ReadEntity(tbl)); - work.getOutputs().add(new WriteEntity(tbl)); + work.getOutputs().add(new WriteEntity(tbl, WriteEntity.WriteType.DDL_METADATA_ONLY)); } else { Partition part = db.getPartition(tbl, touchDesc.getPartSpec(), false); if (part == null) { @@ -1214,7 +1149,7 @@ private int touch(Hive db, AlterTableSimpleDesc touchDesc) throw new HiveException(e); } work.getInputs().add(new ReadEntity(part)); - work.getOutputs().add(new WriteEntity(part)); + work.getOutputs().add(new WriteEntity(part, WriteEntity.WriteType.DDL_METADATA_ONLY)); } return 0; } @@ -2436,7 +2371,7 @@ private int showFunctions(ShowFunctionsDesc showFuncs) throws HiveException { */ private int showLocks(ShowLocksDesc showLocks) throws HiveException { Context ctx = driverContext.getCtx(); - HiveLockManager lockMgr = ctx.getHiveLockMgr(); + HiveLockManager lockMgr = ctx.getHiveTxnManager().getLockManager(); boolean isExt = showLocks.isExt(); if (lockMgr == null) { throw new HiveException("show Locks LockManager not specified"); @@ -2451,9 +2386,12 @@ private int showLocks(ShowLocksDesc showLocks) throws HiveException { List locks = null; if (showLocks.getTableName() == null) { + // TODO should be doing security check here. Users should not be + // able to see each other's locks. locks = lockMgr.getLocks(false, isExt); } else { + // TODO make this work locks = lockMgr.getLocks(getHiveObject(showLocks.getTableName(), showLocks.getPartSpec()), true, isExt); @@ -2528,7 +2466,12 @@ public int compare(HiveLock o1, HiveLock o2) { */ private int lockTable(LockTableDesc lockTbl) throws HiveException { Context ctx = driverContext.getCtx(); - HiveLockManager lockMgr = ctx.getHiveLockMgr(); + HiveTxnManager txnManager = ctx.getHiveTxnManager(); + if (!txnManager.supportsExplicitLock()) { + throw new HiveException(ErrorMsg.LOCK_REQUEST_UNSUPPORTED, + conf.getVar(HiveConf.ConfVars.HIVE_TXN_MANAGER)); + } + HiveLockManager lockMgr = txnManager.getLockManager(); if (lockMgr == null) { throw new HiveException("lock Table LockManager not specified"); } @@ -2577,7 +2520,12 @@ private int lockTable(LockTableDesc lockTbl) throws HiveException { */ private int lockDatabase(LockDatabaseDesc lockDb) throws HiveException { Context ctx = driverContext.getCtx(); - HiveLockManager lockMgr = ctx.getHiveLockMgr(); + HiveTxnManager txnManager = ctx.getHiveTxnManager(); + if (!txnManager.supportsExplicitLock()) { + throw new HiveException(ErrorMsg.LOCK_REQUEST_UNSUPPORTED, + conf.getVar(HiveConf.ConfVars.HIVE_TXN_MANAGER)); + } + HiveLockManager lockMgr = txnManager.getLockManager(); if (lockMgr == null) { throw new HiveException("lock Database LockManager not specified"); } @@ -2613,7 +2561,12 @@ private int lockDatabase(LockDatabaseDesc lockDb) throws HiveException { */ private int unlockDatabase(UnlockDatabaseDesc unlockDb) throws HiveException { Context ctx = driverContext.getCtx(); - HiveLockManager lockMgr = ctx.getHiveLockMgr(); + HiveTxnManager txnManager = ctx.getHiveTxnManager(); + if (!txnManager.supportsExplicitLock()) { + throw new HiveException(ErrorMsg.LOCK_REQUEST_UNSUPPORTED, + conf.getVar(HiveConf.ConfVars.HIVE_TXN_MANAGER)); + } + HiveLockManager lockMgr = txnManager.getLockManager(); if (lockMgr == null) { throw new HiveException("unlock Database LockManager not specified"); } @@ -2671,7 +2624,12 @@ private HiveLockObject getHiveObject(String tabName, */ private int unlockTable(UnlockTableDesc unlockTbl) throws HiveException { Context ctx = driverContext.getCtx(); - HiveLockManager lockMgr = ctx.getHiveLockMgr(); + HiveTxnManager txnManager = ctx.getHiveTxnManager(); + if (!txnManager.supportsExplicitLock()) { + throw new HiveException(ErrorMsg.LOCK_REQUEST_UNSUPPORTED, + conf.getVar(HiveConf.ConfVars.HIVE_TXN_MANAGER)); + } + HiveLockManager lockMgr = txnManager.getLockManager(); if (lockMgr == null) { throw new HiveException("unlock Table LockManager not specified"); } @@ -3423,17 +3381,17 @@ private int alterTable(Hive db, AlterTableDesc alterTbl) throws HiveException { // passed if(part != null) { work.getInputs().add(new ReadEntity(part)); - work.getOutputs().add(new WriteEntity(part)); + work.getOutputs().add(new WriteEntity(part, WriteEntity.WriteType.DDL)); } else if (allPartitions != null ){ for (Partition tmpPart: allPartitions) { work.getInputs().add(new ReadEntity(tmpPart)); - work.getOutputs().add(new WriteEntity(tmpPart)); + work.getOutputs().add(new WriteEntity(tmpPart, WriteEntity.WriteType.DDL)); } } else { work.getInputs().add(new ReadEntity(oldTbl)); - work.getOutputs().add(new WriteEntity(tbl)); + work.getOutputs().add(new WriteEntity(tbl, WriteEntity.WriteType.DDL)); } return 0; } @@ -3511,7 +3469,7 @@ private int dropTable(Hive db, DropTableDesc dropTbl) // drop the table db.dropTable(dropTbl.getTableName()); if (tbl != null) { - work.getOutputs().add(new WriteEntity(tbl)); + work.getOutputs().add(new WriteEntity(tbl, WriteEntity.WriteType.DDL)); } } else { // This is actually an ALTER TABLE DROP PARTITION @@ -3565,7 +3523,7 @@ private int dropTable(Hive db, DropTableDesc dropTbl) for (Partition partition : partsToDelete) { console.printInfo("Dropping the partition " + partition.getName()); db.dropPartition(dropTbl.getTableName(), partition.getValues(), true); - work.getOutputs().add(new WriteEntity(partition)); + work.getOutputs().add(new WriteEntity(partition, WriteEntity.WriteType.DDL)); } } @@ -3846,7 +3804,7 @@ private int createTable(Hive db, CreateTableDesc crtTbl) throws HiveException { // create the table db.createTable(tbl, crtTbl.getIfNotExists()); - work.getOutputs().add(new WriteEntity(tbl)); + work.getOutputs().add(new WriteEntity(tbl, WriteEntity.WriteType.DDL)); return 0; } @@ -3954,7 +3912,7 @@ private int createTableLike(Hive db, CreateTableLikeDesc crtTbl) throws HiveExce // create the table db.createTable(tbl, crtTbl.getIfNotExists()); - work.getOutputs().add(new WriteEntity(tbl)); + work.getOutputs().add(new WriteEntity(tbl, WriteEntity.WriteType.DDL)); return 0; } @@ -3990,7 +3948,7 @@ private int createView(Hive db, CreateViewDesc crtView) throws HiveException { } catch (InvalidOperationException e) { throw new HiveException(e); } - work.getOutputs().add(new WriteEntity(oldview)); + work.getOutputs().add(new WriteEntity(oldview, WriteEntity.WriteType.DDL)); } else { // create new view Table tbl = db.newTable(crtView.getViewName()); @@ -4017,7 +3975,7 @@ private int createView(Hive db, CreateViewDesc crtView) throws HiveException { } db.createTable(tbl, crtView.getIfNotExists()); - work.getOutputs().add(new WriteEntity(tbl)); + work.getOutputs().add(new WriteEntity(tbl, WriteEntity.WriteType.DDL)); } return 0; } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java index ed7787d..a190155 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java @@ -18,16 +18,6 @@ package org.apache.hadoop.hive.ql.exec; -import java.io.IOException; -import java.io.Serializable; -import java.security.AccessControlException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashSet; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FileStatus; @@ -56,17 +46,16 @@ import org.apache.hadoop.hive.ql.optimizer.physical.BucketingSortingCtx.BucketCol; import org.apache.hadoop.hive.ql.optimizer.physical.BucketingSortingCtx.SortCol; import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer; -import org.apache.hadoop.hive.ql.plan.DynamicPartitionCtx; -import org.apache.hadoop.hive.ql.plan.LoadFileDesc; -import org.apache.hadoop.hive.ql.plan.LoadMultiFilesDesc; -import org.apache.hadoop.hive.ql.plan.LoadTableDesc; -import org.apache.hadoop.hive.ql.plan.MapWork; -import org.apache.hadoop.hive.ql.plan.MapredWork; -import org.apache.hadoop.hive.ql.plan.MoveWork; +import org.apache.hadoop.hive.ql.plan.*; import org.apache.hadoop.hive.ql.plan.api.StageType; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.util.StringUtils; +import java.io.IOException; +import java.io.Serializable; +import java.security.AccessControlException; +import java.util.*; + /** * MoveTask implementation. **/ @@ -175,7 +164,7 @@ private void releaseLocks(LoadTableDesc ltd) throws HiveException { } Context ctx = driverContext.getCtx(); - HiveLockManager lockMgr = ctx.getHiveLockMgr(); + HiveLockManager lockMgr = ctx.getHiveTxnManager().getLockManager(); WriteEntity output = ctx.getLoadTableOutputMap().get(ltd); List lockObjects = ctx.getOutputLockObjects().get(output); if (lockObjects == null) { @@ -284,7 +273,9 @@ public int execute(DriverContext driverContext) { db.loadTable(tbd.getSourcePath(), tbd.getTable() .getTableName(), tbd.getReplace(), tbd.getHoldDDLTime()); if (work.getOutputs() != null) { - work.getOutputs().add(new WriteEntity(table)); + work.getOutputs().add(new WriteEntity(table, + (tbd.getReplace() ? WriteEntity.WriteType.INSERT_OVERWRITE : + WriteEntity.WriteType.INSERT))); } } else { LOG.info("Partition is: " + tbd.getPartitionSpec().toString()); @@ -376,7 +367,9 @@ public int execute(DriverContext driverContext) { updatePartitionBucketSortColumns(table, partn, bucketCols, numBuckets, sortCols); } - WriteEntity enty = new WriteEntity(partn); + WriteEntity enty = new WriteEntity(partn, + (tbd.getReplace() ? WriteEntity.WriteType.INSERT_OVERWRITE : + WriteEntity.WriteType.INSERT)); if (work.getOutputs() != null) { work.getOutputs().add(enty); } @@ -417,7 +410,9 @@ public int execute(DriverContext driverContext) { dc = new DataContainer(table.getTTable(), partn.getTPartition()); // add this partition to post-execution hook if (work.getOutputs() != null) { - work.getOutputs().add(new WriteEntity(partn)); + work.getOutputs().add(new WriteEntity(partn, + (tbd.getReplace() ? WriteEntity.WriteType.INSERT_OVERWRITE + : WriteEntity.WriteType.INSERT))); } } } diff --git ql/src/java/org/apache/hadoop/hive/ql/hooks/WriteEntity.java ql/src/java/org/apache/hadoop/hive/ql/hooks/WriteEntity.java index 0493302..3e2241a 100644 --- ql/src/java/org/apache/hadoop/hive/ql/hooks/WriteEntity.java +++ ql/src/java/org/apache/hadoop/hive/ql/hooks/WriteEntity.java @@ -18,20 +18,32 @@ package org.apache.hadoop.hive.ql.hooks; -import java.io.Serializable; - import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.metastore.api.Database; -import org.apache.hadoop.hive.ql.metadata.Partition; import org.apache.hadoop.hive.ql.metadata.DummyPartition; +import org.apache.hadoop.hive.ql.metadata.Partition; import org.apache.hadoop.hive.ql.metadata.Table; +import java.io.Serializable; + /** * This class encapsulates an object that is being written to by the query. This * object may be a table, partition, dfs directory or a local directory. */ public class WriteEntity extends Entity implements Serializable { + public static enum WriteType { + DDL, // for use in DDL statements that will touch data, + // will result in an exclusive lock, + DDL_METADATA_ONLY, // for use in DDL statements that touch only + // metadata and don't need a lock + INSERT, + INSERT_OVERWRITE, + UPDATE, + DELETE}; + + private WriteType writeType; + /** * Only used by serialization. */ @@ -39,8 +51,9 @@ public WriteEntity() { super(); } - public WriteEntity(Database database) { + public WriteEntity(Database database, WriteType type) { super(database, true); + writeType = type; } /** @@ -49,12 +62,14 @@ public WriteEntity(Database database) { * @param t * Table that is written to. */ - public WriteEntity(Table t) { + public WriteEntity(Table t, WriteType type) { super(t, true); + writeType = type; } - public WriteEntity(Table t, boolean complete) { + public WriteEntity(Table t, WriteType type, boolean complete) { super(t, complete); + writeType = type; } /** @@ -63,12 +78,14 @@ public WriteEntity(Table t, boolean complete) { * @param p * Partition that is written to. */ - public WriteEntity(Partition p) { + public WriteEntity(Partition p, WriteType type) { super(p, true); + writeType = type; } - public WriteEntity(DummyPartition p, boolean complete) { + public WriteEntity(DummyPartition p, WriteType type, boolean complete) { super(p, complete); + writeType = type; } /** @@ -84,6 +101,15 @@ public WriteEntity(Path d, boolean islocal) { } /** + * Determine which type of write this is. This is needed by the lock + * manager so it can understand what kind of lock to acquire. + * @return write type + */ + public WriteType getWriteType() { + return writeType; + } + + /** * Equals function. */ @Override diff --git ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbLockManager.java ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbLockManager.java new file mode 100644 index 0000000..7355ef5 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbLockManager.java @@ -0,0 +1,212 @@ +/** + * 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.lockmgr; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.*; +import org.apache.hadoop.hive.ql.ErrorMsg; +import org.apache.thrift.TException; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * An implementation of HiveLockManager for use with {@link org.apache.hadoop.hive.ql.lockmgr.DbTxnManager}. + * Note, this lock manager is not meant to stand alone. It cannot be used + * without the DbTxnManager. + */ +public class DbLockManager implements HiveLockManager{ + + static final private String CLASS_NAME = DbLockManager.class.getName(); + static final private Log LOG = LogFactory.getLog(CLASS_NAME); + + private static final long MAX_SLEEP = 15000; + private HiveLockManagerCtx context; + private Set locks; + private HiveMetaStoreClient client; + private long nextSleep = 50; + + DbLockManager(HiveMetaStoreClient client) { + locks = new HashSet(); + this.client = client; + } + + @Override + public void setContext(HiveLockManagerCtx ctx) throws LockException { + context = ctx; + } + + @Override + public HiveLock lock(HiveLockObject key, HiveLockMode mode, + boolean keepAlive) throws LockException { + throw new UnsupportedOperationException(); + } + + @Override + public List lock(List objs, boolean keepAlive) throws + LockException { + throw new UnsupportedOperationException(); + } + + /** + * Send a lock request to the metastore. This is intended for use by + * {@link DbTxnManager}. + * @param lock lock request + * @throws LockException + */ + List lock(LockRequest lock) throws LockException { + try { + LOG.debug("Requesting lock"); + LockResponse res = client.lock(lock); + while (res.getState() == LockState.WAITING) { + backoff(); + res = client.checkLock(res.getLockid()); + + } + DbHiveLock hl = new DbHiveLock(res.getLockid()); + locks.add(hl); + if (res.getState() != LockState.ACQUIRED) { + throw new LockException(ErrorMsg.LOCK_CANNOT_BE_ACQUIRED.getMsg()); + } + List locks = new ArrayList(1); + locks.add(hl); + return locks; + } catch (NoSuchTxnException e) { + LOG.error("Metastore could not find txnid " + lock.getTxnid()); + throw new LockException(ErrorMsg.TXNMGR_NOT_INSTANTIATED.getMsg(), e); + } catch (TxnAbortedException e) { + LOG.error("Transaction " + lock.getTxnid() + " already aborted."); + throw new LockException(ErrorMsg.TXN_ABORTED.getMsg(), e); + } catch (TException e) { + throw new LockException(ErrorMsg.METASTORE_COMMUNICATION_FAILED.getMsg(), + e); + } + } + + @Override + public void unlock(HiveLock hiveLock) throws LockException { + long lockId = ((DbHiveLock)hiveLock).lockId; + try { + LOG.debug("Unlocking id:" + lockId); + client.unlock(lockId); + boolean removed = locks.remove((DbHiveLock)hiveLock); + LOG.debug("Removed a lock " + removed); + } catch (NoSuchLockException e) { + LOG.error("Metastore could find no record of lock " + lockId); + throw new LockException(ErrorMsg.LOCK_NO_SUCH_LOCK.getMsg(), e); + } catch (TxnOpenException e) { + throw new RuntimeException("Attempt to unlock lock " + lockId + + "associated with an open transaction, " + e.getMessage(), e); + } catch (TException e) { + throw new LockException(ErrorMsg.METASTORE_COMMUNICATION_FAILED.getMsg(), + e); + } + } + + @Override + public void releaseLocks(List hiveLocks) { + for (HiveLock lock : hiveLocks) { + try { + unlock(lock); + } catch (LockException e) { + // Not sure why this method doesn't throw any exceptions, + // but since the interface doesn't allow it we'll just swallow them and + // move on. + } + } + } + + @Override + public List getLocks(boolean verifyTablePartitions, + boolean fetchData) throws LockException { + return new ArrayList(locks); + } + + @Override + public List getLocks(HiveLockObject key, + boolean verifyTablePartitions, + boolean fetchData) throws LockException { + throw new UnsupportedOperationException(); + } + + @Override + public void close() throws LockException { + for (HiveLock lock : locks) { + unlock(lock); + } + locks.clear(); + } + + @Override + public void prepareRetry() throws LockException { + // NOP + } + + @Override + public void refresh() { + // NOP + } + + static class DbHiveLock extends HiveLock { + + long lockId; + + DbHiveLock(long id) { + lockId = id; + } + + @Override + public HiveLockObject getHiveLockObject() { + throw new UnsupportedOperationException(); + } + + @Override + public HiveLockMode getHiveLockMode() { + throw new UnsupportedOperationException(); + } + + @Override + public boolean equals(Object other) { + if (other instanceof DbHiveLock) { + return lockId == ((DbHiveLock)other).lockId; + } else { + return false; + } + } + + @Override + public int hashCode() { + return (int)(lockId % Integer.MAX_VALUE); + } + } + + // Sleep before we send checkLock again, but do it with a back off + // off so we don't sit and hammer the metastore in a tight loop + private void backoff() { + nextSleep *= 2; + if (nextSleep > MAX_SLEEP) nextSleep = MAX_SLEEP; + try { + Thread.sleep(nextSleep); + } catch (InterruptedException e) { + } + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java new file mode 100644 index 0000000..5697432 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java @@ -0,0 +1,324 @@ +/** + * 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.lockmgr; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.LockComponentBuilder; +import org.apache.hadoop.hive.metastore.LockRequestBuilder; +import org.apache.hadoop.hive.metastore.api.*; +import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.ErrorMsg; +import org.apache.hadoop.hive.ql.QueryPlan; +import org.apache.hadoop.hive.ql.hooks.Entity; +import org.apache.hadoop.hive.ql.hooks.ReadEntity; +import org.apache.hadoop.hive.ql.hooks.WriteEntity; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.thrift.TException; + +import java.util.List; + +/** + * An implementation of HiveTxnManager that stores the transactions in the + * metastore database. + */ +class DbTxnManager extends HiveTxnManagerImpl { + + static final private String CLASS_NAME = DbTxnManager.class.getName(); + static final private Log LOG = LogFactory.getLog(CLASS_NAME); + + private DbLockManager lockMgr = null; + private HiveMetaStoreClient client = null; + private long txnId = 0; + + DbTxnManager() { + } + + @Override + public void openTxn() throws LockException { + init(); + try { + txnId = client.openTxn(); + LOG.debug("Opened txn " + txnId); + } catch (TException e) { + throw new LockException(ErrorMsg.METASTORE_COMMUNICATION_FAILED.getMsg(), + e); + } + } + + @Override + public HiveLockManager getLockManager() throws LockException { + init(); + if (lockMgr == null) { + lockMgr = new DbLockManager(client); + } + return lockMgr; + } + + @Override + public void acquireLocks(QueryPlan plan, Context ctx) throws LockException { + init(); + // Make sure we've built the lock manager + getLockManager(); + + boolean atLeastOneLock = false; + + LockRequestBuilder rqstBuilder = new LockRequestBuilder(); + LOG.debug("Setting lock request transaction to " + txnId); + rqstBuilder.setTransactionId(txnId); + + // For each source to read, get a shared lock + for (ReadEntity input : plan.getInputs()) { + LockComponentBuilder compBuilder = new LockComponentBuilder(); + compBuilder.setShared(); + + Table t = null; + switch (input.getType()) { + case DATABASE: + compBuilder.setDbName(input.getDatabase().getName()); + break; + + case TABLE: + t = input.getTable(); + compBuilder.setDbName(t.getDbName()); + compBuilder.setTableName(t.getTableName()); + break; + + case PARTITION: + case DUMMYPARTITION: + compBuilder.setPartitionName(input.getPartition().getName()); + t = input.getPartition().getTable(); + compBuilder.setDbName(t.getDbName()); + compBuilder.setTableName(t.getTableName()); + break; + + default: + // This is a file or something we don't hold locks for. + continue; + } + LockComponent comp = compBuilder.build(); + LOG.debug("Adding lock component to lock request " + comp.toString()); + rqstBuilder.addLockComponent(comp); + atLeastOneLock = true; + } + + // For each source to write to, get the appropriate lock type. If it's + // an OVERWRITE, we need to get an exclusive lock. If it's an insert (no + // overwrite) than we need a shared. If it's update or delete then we + // need a SEMI-SHARED. + for (WriteEntity output : plan.getOutputs()) { + if (output.getType() == Entity.Type.DFS_DIR || output.getType() == + Entity.Type.LOCAL_DIR) { + // We don't lock files or directories. + continue; + } + LockComponentBuilder compBuilder = new LockComponentBuilder(); + Table t = null; + LOG.debug("output is null " + (output == null)); + switch (output.getWriteType()) { + case DDL: + case INSERT_OVERWRITE: + compBuilder.setExclusive(); + break; + + case INSERT: + case DDL_METADATA_ONLY: + compBuilder.setShared(); + break; + + case UPDATE: + case DELETE: + compBuilder.setSemiShared(); + break; + + default: + throw new RuntimeException("Unknown write type " + + output.getWriteType().toString()); + + } + switch (output.getType()) { + case DATABASE: + compBuilder.setDbName(output.getDatabase().getName()); + break; + + case TABLE: + t = output.getTable(); + compBuilder.setDbName(t.getDbName()); + compBuilder.setTableName(t.getTableName()); + break; + + case PARTITION: + case DUMMYPARTITION: + compBuilder.setPartitionName(output.getPartition().getName()); + t = output.getPartition().getTable(); + compBuilder.setDbName(t.getDbName()); + compBuilder.setTableName(t.getTableName()); + break; + + default: + // This is a file or something we don't hold locks for. + continue; + } + LockComponent comp = compBuilder.build(); + LOG.debug("Adding lock component to lock request " + comp.toString()); + rqstBuilder.addLockComponent(comp); + atLeastOneLock = true; + } + + // Make sure we need locks. It's possible there's nothing to lock in + // this operation. + if (!atLeastOneLock) return; + + List locks = lockMgr.lock(rqstBuilder.build()); + ctx.setHiveLocks(locks); + } + + @Override + public void commitTxn() throws LockException { + if (txnId == 0) { + throw new RuntimeException("Attempt to commit before opening a " + + "transaction"); + } + try { + LOG.debug("Committing txn " + txnId); + client.commitTxn(txnId); + } catch (NoSuchTxnException e) { + LOG.error("Metastore could not find txn " + txnId); + throw new LockException(ErrorMsg.TXN_NO_SUCH_TRANSACTION.getMsg() , e); + } catch (TxnAbortedException e) { + LOG.error("Transaction " + txnId + " aborted"); + throw new LockException(ErrorMsg.TXN_ABORTED.getMsg(), e); + } catch (TException e) { + throw new LockException(ErrorMsg.METASTORE_COMMUNICATION_FAILED.getMsg(), + e); + } finally { + txnId = 0; + } + } + + @Override + public void rollbackTxn() throws LockException { + if (txnId == 0) { + throw new RuntimeException("Attempt to rollback before opening a " + + "transaction"); + } + try { + LOG.debug("Rolling back txn " + txnId); + client.rollbackTxn(txnId); + } catch (NoSuchTxnException e) { + LOG.error("Metastore could not find txn " + txnId); + throw new LockException(ErrorMsg.TXN_NO_SUCH_TRANSACTION.getMsg() , e); + } catch (TException e) { + throw new LockException(ErrorMsg.METASTORE_COMMUNICATION_FAILED.getMsg(), + e); + } finally { + txnId = 0; + } + } + + @Override + public void heartbeat() throws LockException { + LOG.debug("Heartbeating lock and transaction " + txnId); + List locks = lockMgr.getLocks(false, false); + if (locks.size() == 0) { + if (txnId == 0) { + // No locks, no txn, we outta here. + return; + } else { + // Create one dummy lock so we can go through the loop below + DbLockManager.DbHiveLock dummyLock = new DbLockManager.DbHiveLock(0L); + locks.add(dummyLock); + } + } + for (HiveLock lock : locks) { + long lockId = ((DbLockManager.DbHiveLock)lock).lockId; + try { + client.heartbeat(txnId, lockId); + } catch (NoSuchLockException e) { + LOG.error("Unable to find lock " + lockId); + throw new LockException(ErrorMsg.LOCK_NO_SUCH_LOCK.getMsg(), e); + } catch (NoSuchTxnException e) { + LOG.error("Unable to find transaction " + txnId); + throw new LockException(ErrorMsg.TXN_NO_SUCH_TRANSACTION.getMsg(), e); + } catch (TxnAbortedException e) { + LOG.error("Transaction aborted " + txnId); + throw new LockException(ErrorMsg.TXN_ABORTED.getMsg(), e); + } catch (TException e) { + throw new LockException( + ErrorMsg.METASTORE_COMMUNICATION_FAILED.getMsg(), e); + } + } + } + + @Override + public IMetaStoreClient.ValidTxnList getValidTxns() throws LockException { + init(); + try { + return client.getValidTxns(); + } catch (TException e) { + throw new LockException(ErrorMsg.METASTORE_COMMUNICATION_FAILED.getMsg(), + e); + } + } + + @Override + public void timeoutTxns() throws LockException { + init(); + //TODO + } + + @Override + public void cleanAbortedTxns(String dbName, String tableName, + String partitionName) throws LockException { + init(); + //TODO + } + + @Override + public boolean supportsExplicitLock() { + return false; + } + + @Override + protected void destruct() { + try { + if (txnId > 0) rollbackTxn(); + if (lockMgr != null) lockMgr.close(); + } catch (Exception e) { + // Not much we can do about it here. + } + } + + private void init() throws LockException { + if (client == null) { + if (conf == null) { + throw new RuntimeException("Must call setHiveConf before any other " + + "methods."); + } + try { + client = new HiveMetaStoreClient(conf); + } catch (MetaException e) { + throw new LockException(ErrorMsg.METASTORE_COULD_NOT_INITIATE.getMsg(), + e); + } + } + } + +} diff --git ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java new file mode 100644 index 0000000..a664bc8 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java @@ -0,0 +1,334 @@ +/** + * 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.lockmgr; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hadoop.hive.metastore.api.GetOpenTxnsResponse; +import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.ErrorMsg; +import org.apache.hadoop.hive.ql.QueryPlan; +import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.hooks.ReadEntity; +import org.apache.hadoop.hive.ql.hooks.WriteEntity; +import org.apache.hadoop.hive.ql.metadata.DummyPartition; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.metadata.Partition; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hadoop.util.ReflectionUtils; + +import java.util.*; + +/** + * An implementation of {@link HiveTxnManager} that does not support + * transactions. This provides default Hive behavior. + */ +class DummyTxnManager extends HiveTxnManagerImpl { + private static final int separator = Utilities.tabCode; + private static final int terminator = Utilities.newLineCode; + static final private Log LOG = + LogFactory.getLog(DummyTxnManager.class.getName()); + + private HiveLockManager lockMgr; + + @Override + public void openTxn() throws LockException { + // No-op + } + + @Override + public HiveLockManager getLockManager() throws LockException { + if (lockMgr == null) { + boolean supportConcurrency = + conf.getBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY); + if (supportConcurrency) { + String lockMgrName = + conf.getVar(HiveConf.ConfVars.HIVE_LOCK_MANAGER); + if ((lockMgrName == null) || (lockMgrName.isEmpty())) { + throw new LockException(ErrorMsg.LOCKMGR_NOT_SPECIFIED.getMsg()); + } + + try { + LOG.info("Creating lock manager of type " + lockMgrName); + lockMgr = (HiveLockManager)ReflectionUtils.newInstance( + conf.getClassByName(lockMgrName), conf); + lockMgr.setContext(new HiveLockManagerCtx(conf)); + } catch (Exception e) { + // set hiveLockMgr to null just in case this invalid manager got set to + // next query's ctx. + if (lockMgr != null) { + try { + lockMgr.close(); + } catch (LockException e1) { + //nothing can do here + } + lockMgr = null; + } + throw new LockException(ErrorMsg.LOCKMGR_NOT_INITIALIZED.getMsg() + + e.getMessage()); + } + } else { + LOG.info("Concurrency mode is disabled, not creating a lock manager"); + return null; + } + } + // Force a re-read of the configuration file. This is done because + // different queries in the session may be using the same lock manager. + lockMgr.refresh(); + return lockMgr; + } + + @Override + public void acquireLocks(QueryPlan plan, Context ctx) throws LockException { + // Make sure we've built the lock manager + getLockManager(); + + // If the lock manager is still null, then it means we aren't using a + // lock manager + if (lockMgr == null) return; + + List lockObjects = new ArrayList(); + + // Sort all the inputs, outputs. + // If a lock needs to be acquired on any partition, a read lock needs to be acquired on all + // its parents also + for (ReadEntity input : plan.getInputs()) { + LOG.debug("Adding " + input.getName() + " to list of lock inputs"); + if (input.getType() == ReadEntity.Type.DATABASE) { + lockObjects.addAll(getLockObjects(plan, input.getDatabase(), null, + null, HiveLockMode.SHARED)); + } else if (input.getType() == ReadEntity.Type.TABLE) { + lockObjects.addAll(getLockObjects(plan, null, input.getTable(), null, + HiveLockMode.SHARED)); + } else { + lockObjects.addAll(getLockObjects(plan, null, null, + input.getPartition(), + HiveLockMode.SHARED)); + } + } + + for (WriteEntity output : plan.getOutputs()) { + LOG.debug("Adding " + output.getName() + " to list of lock outputs"); + List lockObj = null; + if (output.getType() == WriteEntity.Type.DATABASE) { + lockObjects.addAll(getLockObjects(plan, output.getDatabase(), null, + null, + output.isComplete() ? HiveLockMode.EXCLUSIVE : HiveLockMode.SHARED)); + } else if (output.getTyp() == WriteEntity.Type.TABLE) { + lockObj = getLockObjects(plan, null, output.getTable(), null, + output.isComplete() ? HiveLockMode.EXCLUSIVE : HiveLockMode.SHARED); + } else if (output.getTyp() == WriteEntity.Type.PARTITION) { + lockObj = getLockObjects(plan, null, null, output.getPartition(), + HiveLockMode.EXCLUSIVE); + } + // In case of dynamic queries, it is possible to have incomplete dummy partitions + else if (output.getTyp() == WriteEntity.Type.DUMMYPARTITION) { + lockObj = getLockObjects(plan, null, null, output.getPartition(), + HiveLockMode.SHARED); + } + + if(lockObj != null) { + lockObjects.addAll(lockObj); + ctx.getOutputLockObjects().put(output, lockObj); + } + } + + if (lockObjects.isEmpty() && !ctx.isNeedLockMgr()) { + return; + } + + HiveLockObject.HiveLockObjectData lockData = + new HiveLockObject.HiveLockObjectData(plan.getQueryId(), + String.valueOf(System.currentTimeMillis()), + "IMPLICIT", + plan.getQueryStr()); + + // Lock the database also + String currentDb = SessionState.get().getCurrentDatabase(); + lockObjects.add( + new HiveLockObj( + new HiveLockObject(currentDb, lockData), + HiveLockMode.SHARED + ) + ); + + dedupLockObjects(lockObjects); + List hiveLocks = lockMgr.lock(lockObjects, false); + + if (hiveLocks == null) { + throw new LockException(ErrorMsg.LOCK_CANNOT_BE_ACQUIRED.getMsg()); + } else { + ctx.setHiveLocks(hiveLocks); + } + } + + @Override + public void commitTxn() throws LockException { + // No-op + } + + @Override + public void rollbackTxn() throws LockException { + // No-op + } + + @Override + public void heartbeat() throws LockException { + // No-op + } + + @Override + public IMetaStoreClient.ValidTxnList getValidTxns() throws LockException { + return new IMetaStoreClient.ValidTxnList() { + @Override + public boolean isTxnCommitted(long txnid) { + return false; + } + + @Override + public GetOpenTxnsResponse getOpenTxns() { + return null; + } + }; + } + + @Override + public void timeoutTxns() { + // No-op + } + + @Override + public void cleanAbortedTxns(String dbName, String tableName, + String partitionName) { + // No-op + } + + @Override + public boolean supportsExplicitLock() { + return true; + } + + + protected void destruct() { + if (lockMgr != null) { + try { + lockMgr.close(); + } catch (LockException e) { + // Not much I can do about it. + LOG.warn("Got exception when closing lock manager " + e.getMessage()); + } + } + } + + /** + * Dedup the list of lock objects so that there is only one lock per table/partition. + * If there is both a shared and exclusive lock for the same object, this will deduped + * to just a single exclusive lock. Package level so that the unit tests + * can access it. Not intended for use outside this class. + * @param lockObjects + */ + static void dedupLockObjects(List lockObjects) { + Map lockMap = new HashMap(); + for (HiveLockObj lockObj : lockObjects) { + String lockName = lockObj.getName(); + HiveLockObj foundLock = lockMap.get(lockName); + if (foundLock == null || lockObj.getMode() == HiveLockMode.EXCLUSIVE) { + lockMap.put(lockName, lockObj); + } + } + // copy set of deduped locks back to original list + lockObjects.clear(); + for (HiveLockObj lockObj : lockMap.values()) { + lockObjects.add(lockObj); + } + } + + private List getLockObjects(QueryPlan plan, Database db, + Table t, Partition p, + HiveLockMode mode) + throws LockException { + List locks = new LinkedList(); + + HiveLockObject.HiveLockObjectData lockData = + new HiveLockObject.HiveLockObjectData(plan.getQueryId(), + String.valueOf(System.currentTimeMillis()), + "IMPLICIT", + plan.getQueryStr()); + + if (db != null) { + locks.add(new HiveLockObj(new HiveLockObject(db.getName(), lockData), + mode)); + return locks; + } + + if (t != null) { + locks.add(new HiveLockObj(new HiveLockObject(t, lockData), mode)); + mode = HiveLockMode.SHARED; + locks.add(new HiveLockObj(new HiveLockObject(t.getDbName(), lockData), mode)); + return locks; + } + + if (p != null) { + if (!(p instanceof DummyPartition)) { + locks.add(new HiveLockObj(new HiveLockObject(p, lockData), mode)); + } + + // All the parents are locked in shared mode + mode = HiveLockMode.SHARED; + + // For dummy partitions, only partition name is needed + String name = p.getName(); + + if (p instanceof DummyPartition) { + name = p.getName().split("@")[2]; + } + + String partialName = ""; + String[] partns = name.split("/"); + int len = p instanceof DummyPartition ? partns.length : partns.length - 1; + Map partialSpec = new LinkedHashMap(); + for (int idx = 0; idx < len; idx++) { + String partn = partns[idx]; + partialName += partn; + String[] nameValue = partn.split("="); + assert(nameValue.length == 2); + partialSpec.put(nameValue[0], nameValue[1]); + try { + locks.add(new HiveLockObj( + new HiveLockObject(new DummyPartition(p.getTable(), p.getTable().getDbName() + + "/" + p.getTable().getTableName() + + "/" + partialName, + partialSpec), lockData), mode)); + partialName += "/"; + } catch (HiveException e) { + throw new LockException(e.getMessage()); + } + } + + locks.add(new HiveLockObj(new HiveLockObject(p.getTable(), lockData), mode)); + locks.add(new HiveLockObj(new HiveLockObject(p.getTable().getDbName(), lockData), mode)); + } + return locks; + } + + +} diff --git ql/src/java/org/apache/hadoop/hive/ql/lockmgr/EmbeddedLockManager.java ql/src/java/org/apache/hadoop/hive/ql/lockmgr/EmbeddedLockManager.java index 9550001..11434a0 100644 --- ql/src/java/org/apache/hadoop/hive/ql/lockmgr/EmbeddedLockManager.java +++ ql/src/java/org/apache/hadoop/hive/ql/lockmgr/EmbeddedLockManager.java @@ -18,24 +18,14 @@ package org.apache.hadoop.hive.ql.lockmgr; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Stack; -import java.util.concurrent.locks.ReentrantLock; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.lockmgr.HiveLockObject.HiveLockObjectData; -import org.apache.hadoop.hive.ql.metadata.DummyPartition; -import org.apache.hadoop.hive.ql.metadata.Hive; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.metadata.Partition; -import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.metadata.*; + +import java.util.*; +import java.util.concurrent.locks.ReentrantLock; /** * shared lock manager for dedicated hive server. all locks are managed in memory diff --git ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockManager.java ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockManager.java index eb15993..b2eb997 100644 --- ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockManager.java +++ ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockManager.java @@ -20,6 +20,11 @@ import java.util.List; +/** + * Manager for locks in Hive. Users should not instantiate a lock manager + * directly. Instead they should get an instance from their instance of + * {@link HiveTxnManager}. + */ public interface HiveLockManager { public void setContext(HiveLockManagerCtx ctx) throws LockException; diff --git ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockMode.java ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockMode.java index a8fbf26..37af243 100644 --- ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockMode.java +++ ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveLockMode.java @@ -19,6 +19,8 @@ package org.apache.hadoop.hive.ql.lockmgr; public enum HiveLockMode { - SHARED, EXCLUSIVE; + SHARED, + EXCLUSIVE, + SEMI_SHARED; // SEMI_SHARED can share with SHARED but not with itself } diff --git ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManager.java ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManager.java new file mode 100644 index 0000000..4046efb --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManager.java @@ -0,0 +1,137 @@ +/** + * 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.lockmgr; + +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.QueryPlan; + +/** + * An interface that allows Hive to manage transactions. All classes + * implementing this should extend {@link HiveTxnManagerImpl} rather than + * implementing this directly. + */ +public interface HiveTxnManager { + + /** + * Open a new transaction. + * @throws LockException if a transaction is already open. + */ + void openTxn() throws LockException; + + /** + * Get the lock manager. This must be used rather than instantiating an + * instance of the lock manager directly as the transaction manager will + * choose which lock manager to instantiate. + * @return the instance of the lock manager + * @throws LockException if there is an issue obtaining the lock manager. + */ + HiveLockManager getLockManager() throws LockException; + + /** + * Acquire all of the locks needed by a query. If used with a query that + * requires transactions, this should be called after {@link #openTxn()}. + * A list of acquired locks will be stored in the + * {@link org.apache.hadoop.hive.ql.Context} object and can be retrieved + * via {@link org.apache.hadoop.hive.ql.Context#getHiveLocks}. + * @param plan query plan + * @param ctx Context for this query + * @throws LockException if there is an error getting the locks + */ + void acquireLocks(QueryPlan plan, Context ctx) throws LockException; + + /** + * Commit the current transaction. This will release all locks obtained in + * {@link #acquireLocks(org.apache.hadoop.hive.ql.QueryPlan, + * org.apache.hadoop.hive.ql.Context)}. + * @throws LockException if there is no current transaction or the + * transaction has already been committed or aborted. + */ + void commitTxn() throws LockException; + + /** + * Abort the current transaction. This will release all locks obtained in + * {@link #acquireLocks(org.apache.hadoop.hive.ql.QueryPlan, + * org.apache.hadoop.hive.ql.Context)}. + * @throws LockException if there is no current transaction or the + * transaction has already been committed or aborted. + */ + void rollbackTxn() throws LockException; + + /** + * Send a heartbeat to the transaction management storage so other Hive + * clients know that the transaction and locks held by this client are + * still valid. For implementations that do not require heartbeats this + * can be a no-op. + * @throws LockException If current transaction exists or the transaction + * has already been committed or aborted. + */ + void heartbeat() throws LockException; + + /** + * Get the transactions that are currently valid. The resulting + * {@link IMetaStoreClient.ValidTxnList} object is a thrift object and can + * be passed to the processing + * tasks for use in the reading the data. This call should be made once up + * front by the planner and should never be called on the backend, + * as this will violate the isolation level semantics. + * @return list of valid transactions. + * @throws LockException + */ + IMetaStoreClient.ValidTxnList getValidTxns() throws LockException; + + /** + * This will cause the transaction manager to inspect its list of open + * transactions and timeout ones that are old. This method should not be + * called by regular queries, but only by the compactor when it is running. + * @throws LockException + */ + void timeoutTxns() throws LockException; + + /** + * This call announces to the transaction manager that a compaction has + * been completed for a partition or table (in the case of + * a non-partitioned table) and any aborted transactions associated + * with that partition can potentially be removed from open transactions + * table. It is only potentially because other, as yet uncompacted, + * partitions may also be part of that transaction. + * @param dbName name of the database the table is in + * @param tableName name of the table that was compacted + * @param partitionName name of the partition that was compacted. If this + * is a non-partitioned table, this value will be null. + * @throws LockException + */ + void cleanAbortedTxns(String dbName, String tableName, String partitionName) + throws LockException; + + /** + * This call closes down the transaction manager. All open transactions + * are aborted. If no transactions are open but locks are held those locks + * are released. This method should be called if processing of a session + * is being halted in an abnormal way. It avoids locks and transactions + * timing out. + */ + void closeTxnManager(); + + /** + * Indicate whether this lock manager supports the use of lock + * database or lock table. + * @return + */ + boolean supportsExplicitLock(); +} diff --git ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManagerImpl.java ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManagerImpl.java new file mode 100644 index 0000000..eccb8d1 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManagerImpl.java @@ -0,0 +1,47 @@ +/** + * 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.lockmgr; + +import org.apache.hadoop.hive.conf.HiveConf; + +/** + * An implementation HiveTxnManager that includes internal methods that all + * transaction managers need to implement but that we don't want to expose to + * outside. + */ +abstract class HiveTxnManagerImpl implements HiveTxnManager { + + protected HiveConf conf; + + void setHiveConf(HiveConf c) { + conf = c; + } + + abstract protected void destruct(); + + @Override + public void closeTxnManager() { + destruct(); + } + + @Override + protected void finalize() throws Throwable { + destruct(); + } + +} diff --git ql/src/java/org/apache/hadoop/hive/ql/lockmgr/TxnManagerFactory.java ql/src/java/org/apache/hadoop/hive/ql/lockmgr/TxnManagerFactory.java new file mode 100644 index 0000000..b6d6704 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/lockmgr/TxnManagerFactory.java @@ -0,0 +1,83 @@ +/** + * 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.lockmgr; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.ErrorMsg; +import org.apache.hadoop.util.ReflectionUtils; + +/** + * A factory to get an instance of {@link HiveTxnManager}. This should + * always be called rather than building a transaction manager via reflection. + * This factory will read the configuration file to determine which + * transaction manager to instantiate. It will stash the chosen transaction + * manager into the Context object, and subsequently return it from there so + * that if there are multiple Hive threads running, + * each will get it's appropriate transaction manager. + */ +public class TxnManagerFactory { + + private static TxnManagerFactory self; + + /** + * Get the singleton instance of this factory. + * @return this factory + */ + public static synchronized TxnManagerFactory getTxnManagerFactory() { + if (self == null) { + self = new TxnManagerFactory(); + } + return self; + } + + /** + * Create a new transaction manager. The transaction manager to + * instantiate will be determined by the hive.txn.manager value in the + * configuration. This should not be called if a transaction manager has + * already been constructed and stored in the Context object. + * @param conf HiveConf object used to construct the transaction manager + * @return the transaction manager + * @throws LockException if there is an error constructing the transaction + * manager. + */ + public HiveTxnManager getTxnManager(HiveConf conf) throws + LockException { + // just return that. + HiveTxnManager txnMgr = null; + + // Determine the transaction manager to use from the configuration. + String txnMgrName = conf.getVar(HiveConf.ConfVars.HIVE_TXN_MANAGER); + if (txnMgrName == null || txnMgrName.isEmpty()) { + throw new LockException(ErrorMsg.TXNMGR_NOT_SPECIFIED.getMsg()); + } + + // Instantiate the chosen transaction manager + try { + HiveTxnManagerImpl impl = (HiveTxnManagerImpl)ReflectionUtils.newInstance( + conf.getClassByName(txnMgrName), conf); + impl.setHiveConf(conf); + txnMgr = impl; + } catch (ClassNotFoundException e) { + throw new LockException(ErrorMsg.TXNMGR_NOT_INSTANTIATED.getMsg()); + } + return txnMgr; + } + + private TxnManagerFactory() { + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/ZooKeeperHiveLockManager.java ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/ZooKeeperHiveLockManager.java index 093be45..65b9136 100644 --- ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/ZooKeeperHiveLockManager.java +++ ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/ZooKeeperHiveLockManager.java @@ -18,36 +18,14 @@ package org.apache.hadoop.hive.ql.lockmgr.zookeeper; -import java.io.IOException; -import java.net.InetAddress; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Queue; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - +import com.google.common.annotations.VisibleForTesting; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.ErrorMsg; -import org.apache.hadoop.hive.ql.lockmgr.HiveLock; -import org.apache.hadoop.hive.ql.lockmgr.HiveLockManager; -import org.apache.hadoop.hive.ql.lockmgr.HiveLockManagerCtx; -import org.apache.hadoop.hive.ql.lockmgr.HiveLockMode; -import org.apache.hadoop.hive.ql.lockmgr.HiveLockObj; -import org.apache.hadoop.hive.ql.lockmgr.HiveLockObject; +import org.apache.hadoop.hive.ql.lockmgr.*; import org.apache.hadoop.hive.ql.lockmgr.HiveLockObject.HiveLockObjectData; -import org.apache.hadoop.hive.ql.lockmgr.LockException; -import org.apache.hadoop.hive.ql.metadata.DummyPartition; -import org.apache.hadoop.hive.ql.metadata.Hive; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.metadata.Partition; -import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.metadata.*; import org.apache.hadoop.hive.ql.session.SessionState.LogHelper; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; @@ -55,7 +33,11 @@ import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.ZooKeeper; -import com.google.common.annotations.VisibleForTesting; +import java.io.IOException; +import java.net.InetAddress; +import java.util.*; +import java.util.regex.Matcher; +import java.util.regex.Pattern; public class ZooKeeperHiveLockManager implements HiveLockManager { HiveLockManagerCtx ctx; diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java index 0e2d555..b9639e9 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java @@ -18,24 +18,6 @@ package org.apache.hadoop.hive.ql.parse; -import static org.apache.hadoop.hive.ql.parse.HiveParser.TOK_DATABASELOCATION; -import static org.apache.hadoop.hive.ql.parse.HiveParser.TOK_DATABASEPROPERTIES; - -import java.io.Serializable; -import java.net.URI; -import java.net.URISyntaxException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.LinkedHashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Properties; -import java.util.Set; - import org.antlr.runtime.tree.CommonTree; import org.antlr.runtime.tree.Tree; import org.apache.commons.logging.Log; @@ -54,11 +36,7 @@ import org.apache.hadoop.hive.metastore.api.SkewedInfo; import org.apache.hadoop.hive.ql.Driver; import org.apache.hadoop.hive.ql.ErrorMsg; -import org.apache.hadoop.hive.ql.exec.ArchiveUtils; -import org.apache.hadoop.hive.ql.exec.FetchTask; -import org.apache.hadoop.hive.ql.exec.Task; -import org.apache.hadoop.hive.ql.exec.TaskFactory; -import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.exec.*; import org.apache.hadoop.hive.ql.hooks.ReadEntity; import org.apache.hadoop.hive.ql.hooks.WriteEntity; import org.apache.hadoop.hive.ql.index.HiveIndex; @@ -67,59 +45,16 @@ import org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat; import org.apache.hadoop.hive.ql.io.RCFileInputFormat; import org.apache.hadoop.hive.ql.lib.Node; -import org.apache.hadoop.hive.ql.metadata.Hive; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.metadata.HiveUtils; +import org.apache.hadoop.hive.ql.metadata.*; import org.apache.hadoop.hive.ql.metadata.Partition; import org.apache.hadoop.hive.ql.metadata.Table; import org.apache.hadoop.hive.ql.parse.authorization.HiveAuthorizationTaskFactory; import org.apache.hadoop.hive.ql.parse.authorization.HiveAuthorizationTaskFactoryImpl; -import org.apache.hadoop.hive.ql.plan.AddPartitionDesc; -import org.apache.hadoop.hive.ql.plan.AlterDatabaseDesc; -import org.apache.hadoop.hive.ql.plan.AlterIndexDesc; +import org.apache.hadoop.hive.ql.plan.*; import org.apache.hadoop.hive.ql.plan.AlterIndexDesc.AlterIndexTypes; -import org.apache.hadoop.hive.ql.plan.AlterTableAlterPartDesc; -import org.apache.hadoop.hive.ql.plan.AlterTableDesc; import org.apache.hadoop.hive.ql.plan.AlterTableDesc.AlterTableTypes; -import org.apache.hadoop.hive.ql.plan.AlterTableExchangePartition; -import org.apache.hadoop.hive.ql.plan.AlterTableSimpleDesc; -import org.apache.hadoop.hive.ql.plan.CreateDatabaseDesc; -import org.apache.hadoop.hive.ql.plan.CreateIndexDesc; -import org.apache.hadoop.hive.ql.plan.DDLWork; -import org.apache.hadoop.hive.ql.plan.DescDatabaseDesc; -import org.apache.hadoop.hive.ql.plan.DescFunctionDesc; -import org.apache.hadoop.hive.ql.plan.DescTableDesc; -import org.apache.hadoop.hive.ql.plan.DropDatabaseDesc; -import org.apache.hadoop.hive.ql.plan.DropIndexDesc; -import org.apache.hadoop.hive.ql.plan.DropTableDesc; -import org.apache.hadoop.hive.ql.plan.FetchWork; -import org.apache.hadoop.hive.ql.plan.ListBucketingCtx; -import org.apache.hadoop.hive.ql.plan.LoadTableDesc; -import org.apache.hadoop.hive.ql.plan.LockDatabaseDesc; -import org.apache.hadoop.hive.ql.plan.LockTableDesc; -import org.apache.hadoop.hive.ql.plan.MoveWork; -import org.apache.hadoop.hive.ql.plan.MsckDesc; -import org.apache.hadoop.hive.ql.plan.PartitionSpec; -import org.apache.hadoop.hive.ql.plan.PlanUtils; -import org.apache.hadoop.hive.ql.plan.RenamePartitionDesc; -import org.apache.hadoop.hive.ql.plan.RoleDDLDesc; -import org.apache.hadoop.hive.ql.plan.ShowColumnsDesc; -import org.apache.hadoop.hive.ql.plan.ShowCreateTableDesc; -import org.apache.hadoop.hive.ql.plan.ShowDatabasesDesc; -import org.apache.hadoop.hive.ql.plan.ShowFunctionsDesc; -import org.apache.hadoop.hive.ql.plan.ShowGrantDesc; -import org.apache.hadoop.hive.ql.plan.ShowIndexesDesc; -import org.apache.hadoop.hive.ql.plan.ShowLocksDesc; -import org.apache.hadoop.hive.ql.plan.ShowPartitionsDesc; -import org.apache.hadoop.hive.ql.plan.ShowTableStatusDesc; -import org.apache.hadoop.hive.ql.plan.ShowTablesDesc; -import org.apache.hadoop.hive.ql.plan.ShowTblPropertiesDesc; -import org.apache.hadoop.hive.ql.plan.StatsWork; -import org.apache.hadoop.hive.ql.plan.SwitchDatabaseDesc; -import org.apache.hadoop.hive.ql.plan.TableDesc; -import org.apache.hadoop.hive.ql.plan.TruncateTableDesc; -import org.apache.hadoop.hive.ql.plan.UnlockDatabaseDesc; -import org.apache.hadoop.hive.ql.plan.UnlockTableDesc; +import org.apache.hadoop.hive.ql.security.authorization.Privilege; +import org.apache.hadoop.hive.ql.security.authorization.PrivilegeRegistry; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.hive.serde.serdeConstants; import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe; @@ -129,6 +64,15 @@ import org.apache.hadoop.mapred.InputFormat; import org.apache.hadoop.mapred.TextInputFormat; +import java.io.Serializable; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.*; +import java.util.Map.Entry; + +import static org.apache.hadoop.hive.ql.parse.HiveParser.TOK_DATABASELOCATION; +import static org.apache.hadoop.hive.ql.parse.HiveParser.TOK_DATABASEPROPERTIES; + /** * DDLSemanticAnalyzer. * @@ -671,7 +615,7 @@ private void analyzeDropDatabase(ASTNode ast) throws SemanticException { } inputs.add(new ReadEntity(database)); - outputs.add(new WriteEntity(database)); + outputs.add(new WriteEntity(database, WriteEntity.WriteType.DDL)); DropDatabaseDesc dropDatabaseDesc = new DropDatabaseDesc(dbName, ifExists, ifCascade); rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), dropDatabaseDesc), conf)); @@ -697,7 +641,7 @@ private void analyzeDropTable(ASTNode ast, boolean expectView) Table tab = getTable(tableName, throwException); if (tab != null) { inputs.add(new ReadEntity(tab)); - outputs.add(new WriteEntity(tab)); + outputs.add(new WriteEntity(tab, WriteEntity.WriteType.DDL)); } DropTableDesc dropTblDesc = new DropTableDesc( @@ -723,19 +667,19 @@ private void analyzeTruncateTable(ASTNode ast) throws SemanticException { Map partSpec = getPartSpec((ASTNode) root.getChild(1)); if (partSpec == null) { if (!table.isPartitioned()) { - outputs.add(new WriteEntity(table)); + outputs.add(new WriteEntity(table, WriteEntity.WriteType.DDL)); } else { for (Partition partition : getPartitions(table, null, false)) { - outputs.add(new WriteEntity(partition)); + outputs.add(new WriteEntity(partition, WriteEntity.WriteType.DDL)); } } } else { if (isFullSpec(table, partSpec)) { Partition partition = getPartition(table, partSpec, true); - outputs.add(new WriteEntity(partition)); + outputs.add(new WriteEntity(partition, WriteEntity.WriteType.DDL)); } else { for (Partition partition : getPartitions(table, partSpec, false)) { - outputs.add(new WriteEntity(partition)); + outputs.add(new WriteEntity(partition, WriteEntity.WriteType.DDL)); } } } @@ -1273,17 +1217,17 @@ private void addInputsOutputsAlterTable(String tableName, Map pa Table tab = getTable(tableName, true); if (partSpec == null || partSpec.isEmpty()) { inputs.add(new ReadEntity(tab)); - outputs.add(new WriteEntity(tab)); + outputs.add(new WriteEntity(tab, WriteEntity.WriteType.DDL)); } else { inputs.add(new ReadEntity(tab)); if (desc == null || desc.getOp() != AlterTableDesc.AlterTableTypes.ALTERPROTECTMODE) { Partition part = getPartition(tab, partSpec, true); - outputs.add(new WriteEntity(part)); + outputs.add(new WriteEntity(part, WriteEntity.WriteType.DDL)); } else { for (Partition part : getPartitions(tab, partSpec, true)) { - outputs.add(new WriteEntity(part)); + outputs.add(new WriteEntity(part, WriteEntity.WriteType.DDL)); } } } @@ -2598,7 +2542,7 @@ private Partition getPartitionForOutput(Table tab, Map currentPa try { Partition partition = db.getPartition(tab, currentPart, false); if (partition != null) { - outputs.add(new WriteEntity(partition)); + outputs.add(new WriteEntity(partition, WriteEntity.WriteType.INSERT)); } return partition; } catch (HiveException e) { @@ -2632,7 +2576,7 @@ private void analyzeAlterTableTouch(CommonTree ast) AlterTableSimpleDesc touchDesc = new AlterTableSimpleDesc( SessionState.get().getCurrentDatabase(), tblName, null, AlterTableDesc.AlterTableTypes.TOUCH); - outputs.add(new WriteEntity(tab)); + outputs.add(new WriteEntity(tab, WriteEntity.WriteType.DDL_METADATA_ONLY)); rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), touchDesc), conf)); } else { @@ -2847,7 +2791,7 @@ private void addTablePartsOutputs(String tblName, List> part } } for (Partition p : parts) { - outputs.add(new WriteEntity(p)); + outputs.add(new WriteEntity(p, WriteEntity.WriteType.DDL)); } } } @@ -2892,7 +2836,7 @@ private void addTableDropPartsOutputs(String tblName, List partSp throw new SemanticException( ErrorMsg.DROP_COMMAND_NOT_ALLOWED_FOR_PARTITION.getMsg(p.getCompleteName())); } - outputs.add(new WriteEntity(p)); + outputs.add(new WriteEntity(p, WriteEntity.WriteType.DELETE)); } } } @@ -2915,7 +2859,7 @@ private void analyzeAltertableSkewedby(ASTNode ast) throws SemanticException { Table tab = getTable(tableName, true); inputs.add(new ReadEntity(tab)); - outputs.add(new WriteEntity(tab)); + outputs.add(new WriteEntity(tab, WriteEntity.WriteType.DDL)); validateAlterTableType(tab, AlterTableTypes.ADDSKEWEDBY); diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java index ceb4c8a..92ec334 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java @@ -18,18 +18,6 @@ package org.apache.hadoop.hive.ql.parse; -import java.io.IOException; -import java.net.URI; -import java.net.URISyntaxException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.Iterator; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.TreeMap; - import org.antlr.runtime.tree.Tree; import org.apache.commons.lang.ObjectUtils; import org.apache.hadoop.fs.FileStatus; @@ -50,15 +38,15 @@ import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.metadata.InvalidTableException; import org.apache.hadoop.hive.ql.metadata.Table; -import org.apache.hadoop.hive.ql.plan.AddPartitionDesc; -import org.apache.hadoop.hive.ql.plan.CopyWork; -import org.apache.hadoop.hive.ql.plan.CreateTableDesc; -import org.apache.hadoop.hive.ql.plan.DDLWork; -import org.apache.hadoop.hive.ql.plan.LoadTableDesc; -import org.apache.hadoop.hive.ql.plan.MoveWork; +import org.apache.hadoop.hive.ql.plan.*; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.hive.serde.serdeConstants; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.*; + /** * ImportSemanticAnalyzer. * @@ -237,7 +225,8 @@ public void analyzeInternal(ASTNode ast) throws SemanticException { .toString())); loadTable(fromURI, table); } - outputs.add(new WriteEntity(table)); + // Set this to read because we can't overwrite any existing partitions + outputs.add(new WriteEntity(table, WriteEntity.WriteType.DDL_METADATA_ONLY)); } catch (InvalidTableException e) { LOG.debug("table " + tblDesc.getTableName() + " does not exist"); diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java index a22a15f..c07e273 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/LoadSemanticAnalyzer.java @@ -18,14 +18,6 @@ package org.apache.hadoop.hive.ql.parse; -import java.io.IOException; -import java.io.Serializable; -import java.net.URI; -import java.net.URISyntaxException; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; - import org.antlr.runtime.tree.Tree; import org.apache.commons.lang.StringUtils; import org.apache.hadoop.fs.FileStatus; @@ -46,6 +38,14 @@ import org.apache.hadoop.hive.ql.plan.MoveWork; import org.apache.hadoop.hive.ql.plan.StatsWork; +import java.io.IOException; +import java.io.Serializable; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + /** * LoadSemanticAnalyzer. * @@ -237,7 +237,9 @@ public void analyzeInternal(ASTNode ast) throws SemanticException { Map partSpec = ts.getPartSpec(); if (partSpec == null) { partSpec = new LinkedHashMap(); - outputs.add(new WriteEntity(ts.tableHandle)); + outputs.add(new WriteEntity(ts.tableHandle, + (isOverWrite ? WriteEntity.WriteType.INSERT_OVERWRITE : + WriteEntity.WriteType.INSERT))); } else { try{ Partition part = Hive.get().getPartition(ts.tableHandle, partSpec, false); @@ -246,9 +248,13 @@ public void analyzeInternal(ASTNode ast) throws SemanticException { throw new SemanticException(ErrorMsg.OFFLINE_TABLE_OR_PARTITION. getMsg(ts.tableName + ":" + part.getName())); } - outputs.add(new WriteEntity(part)); + outputs.add(new WriteEntity(part, + (isOverWrite ? WriteEntity.WriteType.INSERT_OVERWRITE : + WriteEntity.WriteType.INSERT))); } else { - outputs.add(new WriteEntity(ts.tableHandle)); + outputs.add(new WriteEntity(ts.tableHandle, + (isOverWrite ? WriteEntity.WriteType.INSERT_OVERWRITE : + WriteEntity.WriteType.INSERT))); } } catch(HiveException e) { throw new SemanticException(e); diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java index 1f7aae0..27aacd5 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java @@ -5374,7 +5374,7 @@ private Operator genFileSinkPlan(String dest, QB qb, Operator input) // in the case of DP, we will register WriteEntity in MoveTask when the // list of dynamically created partitions are known. if ((dpCtx == null || dpCtx.getNumDPCols() == 0)) { - output = new WriteEntity(dest_tab); + output = new WriteEntity(dest_tab, determineWriteType(ltd, isNonNativeTable)); if (!outputs.add(output)) { throw new SemanticException(ErrorMsg.OUTPUT_SPECIFIED_MULTIPLE_TIMES .getMsg(dest_tab.getTableName())); @@ -5383,7 +5383,7 @@ private Operator genFileSinkPlan(String dest, QB qb, Operator input) if ((dpCtx != null) && (dpCtx.getNumDPCols() >= 0)) { // No static partition specified if (dpCtx.getNumSPCols() == 0) { - output = new WriteEntity(dest_tab, false); + output = new WriteEntity(dest_tab, determineWriteType(ltd, isNonNativeTable), false); outputs.add(output); } // part of the partition specified @@ -5397,7 +5397,7 @@ private Operator genFileSinkPlan(String dest, QB qb, Operator input) new DummyPartition(dest_tab, dest_tab.getDbName() + "@" + dest_tab.getTableName() + "@" + ppath, partSpec); - output = new WriteEntity(p, false); + output = new WriteEntity(p, WriteEntity.WriteType.INSERT, false); outputs.add(output); } catch (HiveException e) { throw new SemanticException(e.getMessage(), e); @@ -5459,7 +5459,9 @@ private Operator genFileSinkPlan(String dest, QB qb, Operator input) ltd.setHoldDDLTime(true); } loadTableWork.add(ltd); - if (!outputs.add(new WriteEntity(dest_part))) { + if (!outputs.add(new WriteEntity(dest_part, (ltd.getReplace() ? + WriteEntity.WriteType.INSERT_OVERWRITE : + WriteEntity.WriteType.INSERT)))) { throw new SemanticException(ErrorMsg.OUTPUT_SPECIFIED_MULTIPLE_TIMES .getMsg(dest_tab.getTableName() + "@" + dest_part.getName())); } @@ -8554,7 +8556,7 @@ private void setupStats(TableScanDesc tsDesc, QBParseInfo qbp, Table tab, String tsDesc.setStatsAggPrefix(tab.getDbName()+"."+k); // set up WritenEntity for replication - outputs.add(new WriteEntity(tab)); + outputs.add(new WriteEntity(tab, WriteEntity.WriteType.DDL_METADATA_ONLY)); // add WriteEntity for each matching partition if (tab.isPartitioned()) { @@ -8565,7 +8567,7 @@ private void setupStats(TableScanDesc tsDesc, QBParseInfo qbp, Table tab, String if (partitions != null) { for (Partition partn : partitions) { // inputs.add(new ReadEntity(partn)); // is this needed at all? - outputs.add(new WriteEntity(partn)); + outputs.add(new WriteEntity(partn, WriteEntity.WriteType.DDL_METADATA_ONLY)); } } } @@ -10941,4 +10943,13 @@ private void addAlternateGByKeyMappings(ASTNode gByExpr, ColumnInfo colInfo, gByRR.put(tab_alias, col_alias, colInfo); } } + + private WriteEntity.WriteType determineWriteType(LoadTableDesc ltd, boolean isNonNativeTable) { + // Don't know the characteristics of non-native tables, + // and don't have a rational way to guess, so assume the most + // conservative case. + if (isNonNativeTable) return WriteEntity.WriteType.INSERT_OVERWRITE; + else return (ltd.getReplace() ? WriteEntity.WriteType.INSERT_OVERWRITE : + WriteEntity.WriteType.INSERT); + } } diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactoryImpl.java ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactoryImpl.java index c41cd0f..87d649d 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactoryImpl.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/authorization/HiveAuthorizationTaskFactoryImpl.java @@ -260,9 +260,9 @@ private PrivilegeObjectDesc analyzePrivilegeObject(ASTNode ast, Table tbl = getTable(SessionState.get().getCurrentDatabase(), subject.getObject()); if (subject.getPartSpec() != null) { Partition part = getPartition(tbl, subject.getPartSpec()); - outputs.add(new WriteEntity(part)); + outputs.add(new WriteEntity(part, WriteEntity.WriteType.DDL_METADATA_ONLY)); } else { - outputs.add(new WriteEntity(tbl)); + outputs.add(new WriteEntity(tbl, WriteEntity.WriteType.DDL_METADATA_ONLY)); } } diff --git ql/src/test/org/apache/hadoop/hive/ql/TestDriver.java ql/src/test/org/apache/hadoop/hive/ql/TestDriver.java deleted file mode 100644 index e00a73d..0000000 --- ql/src/test/org/apache/hadoop/hive/ql/TestDriver.java +++ /dev/null @@ -1,57 +0,0 @@ -package org.apache.hadoop.hive.ql; - -import java.util.*; -import junit.framework.TestCase; - -import org.apache.hadoop.hive.ql.lockmgr.HiveLockMode; -import org.apache.hadoop.hive.ql.lockmgr.HiveLockObj; -import org.apache.hadoop.hive.ql.lockmgr.HiveLockObject; -import org.apache.hadoop.hive.ql.lockmgr.HiveLockObject.HiveLockObjectData; - -public class TestDriver extends TestCase { - public void testDedupLockObjects() { - List lockObjs = new ArrayList(); - String path1 = "path1"; - String path2 = "path2"; - HiveLockObjectData lockData1 = new HiveLockObjectData( - "query1", "1", "IMPLICIT", "drop table table1"); - HiveLockObjectData lockData2 = new HiveLockObjectData( - "query1", "1", "IMPLICIT", "drop table table1"); - - // Start with the following locks: - // [path1, shared] - // [path1, exclusive] - // [path2, shared] - // [path2, shared] - // [path2, shared] - lockObjs.add(new HiveLockObj(new HiveLockObject(path1, lockData1), HiveLockMode.SHARED)); - String name1 = lockObjs.get(lockObjs.size() - 1).getName(); - lockObjs.add(new HiveLockObj(new HiveLockObject(path1, lockData1), HiveLockMode.EXCLUSIVE)); - lockObjs.add(new HiveLockObj(new HiveLockObject(path2, lockData2), HiveLockMode.SHARED)); - String name2 = lockObjs.get(lockObjs.size() - 1).getName(); - lockObjs.add(new HiveLockObj(new HiveLockObject(path2, lockData2), HiveLockMode.SHARED)); - lockObjs.add(new HiveLockObj(new HiveLockObject(path2, lockData2), HiveLockMode.SHARED)); - - Driver.dedupLockObjects(lockObjs); - - // After dedup we should be left with 2 locks: - // [path1, exclusive] - // [path2, shared] - assertEquals("Locks should be deduped", 2, lockObjs.size()); - - Comparator cmp = new Comparator() { - public int compare(HiveLockObj lock1, HiveLockObj lock2) { - return lock1.getName().compareTo(lock2.getName()); - } - }; - Collections.sort(lockObjs, cmp); - - HiveLockObj lockObj = lockObjs.get(0); - assertEquals(name1, lockObj.getName()); - assertEquals(HiveLockMode.EXCLUSIVE, lockObj.getMode()); - - lockObj = lockObjs.get(1); - assertEquals(name2, lockObj.getName()); - assertEquals(HiveLockMode.SHARED, lockObj.getMode()); - } -} diff --git ql/src/test/org/apache/hadoop/hive/ql/TestHiveMetaStoreClient.java ql/src/test/org/apache/hadoop/hive/ql/TestHiveMetaStoreClient.java new file mode 100644 index 0000000..09b07d0 --- /dev/null +++ ql/src/test/org/apache/hadoop/hive/ql/TestHiveMetaStoreClient.java @@ -0,0 +1,154 @@ +/** + * 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 junit.framework.Assert; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.LockComponentBuilder; +import org.apache.hadoop.hive.metastore.LockRequestBuilder; +import org.apache.hadoop.hive.metastore.api.LockResponse; +import org.apache.hadoop.hive.metastore.api.LockState; +import org.apache.hadoop.hive.metastore.txn.TxnDbUtil; +import org.apache.log4j.Level; +import org.apache.log4j.LogManager; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.util.List; + +/** + * Unit tests for {@link org.apache.hadoop.hive.metastore.HiveMetaStoreClient}. For now this just has + * transaction and locking tests. The goal here is not to test all + * functionality possible through the interface, as all permutations of DB + * operations should be tested in the appropriate DB handler classes. The + * goal is to test that we can properly pass the messages through the thrift + * service. + * + * This is in the ql directory rather than the metastore directory because it + * required the hive-exec jar, and hive-exec jar already depends on + * hive-metastore jar, thus I can't make hive-metastore depend on hive-exec. + */ +public class TestHiveMetaStoreClient { + + private HiveConf conf = new HiveConf(); + private IMetaStoreClient client; + + public TestHiveMetaStoreClient() throws Exception { + TxnDbUtil.setConfValues(conf); + LogManager.getRootLogger().setLevel(Level.DEBUG); + tearDown(); + } + + @Test + public void testTxns() throws Exception { + List tids = client.openTxns(3).getTxn_ids(); + Assert.assertEquals(1L, (long) tids.get(0)); + Assert.assertEquals(2L, (long) tids.get(1)); + Assert.assertEquals(3L, (long) tids.get(2)); + client.rollbackTxn(1); + client.commitTxn(2); + IMetaStoreClient.ValidTxnList validTxns = client.getValidTxns(); + Assert.assertFalse(validTxns.isTxnCommitted(1)); + Assert.assertTrue(validTxns.isTxnCommitted(2)); + Assert.assertFalse(validTxns.isTxnCommitted(3)); + Assert.assertFalse(validTxns.isTxnCommitted(4)); + } + + @Test + public void testLocks() throws Exception { + LockRequestBuilder rqstBuilder = new LockRequestBuilder(); + rqstBuilder.addLockComponent(new LockComponentBuilder() + .setDbName("mydb") + .setTableName("mytable") + .setPartitionName("mypartition") + .setExclusive() + .build()); + rqstBuilder.addLockComponent(new LockComponentBuilder() + .setDbName("mydb") + .setTableName("yourtable") + .setSemiShared() + .setObjectData("Mary had a little lamb") + .build()); + rqstBuilder.addLockComponent(new LockComponentBuilder() + .setDbName("yourdb") + .setShared() + .build()); + + LockResponse res = client.lock(rqstBuilder.build()); + Assert.assertEquals(1L, res.getLockid()); + Assert.assertEquals(LockState.ACQUIRED, res.getState()); + + res = client.checkLock(1); + Assert.assertEquals(1L, res.getLockid()); + Assert.assertEquals(LockState.ACQUIRED, res.getState()); + + client.heartbeat(0, 1); + + client.unlock(1); + } + + @Test + public void testLocksWithTxn() throws Exception { + long txnid = client.openTxn(); + + LockRequestBuilder rqstBuilder = new LockRequestBuilder(); + rqstBuilder.setTransactionId(txnid) + .addLockComponent(new LockComponentBuilder() + .setDbName("mydb") + .setTableName("mytable") + .setPartitionName("mypartition") + .setExclusive() + .build()) + .addLockComponent(new LockComponentBuilder() + .setDbName("mydb") + .setTableName("yourtable") + .setSemiShared() + .setObjectData("Mary had a little lamb") + .build()) + .addLockComponent(new LockComponentBuilder() + .setDbName("yourdb") + .setShared() + .build()); + + LockResponse res = client.lock(rqstBuilder.build()); + Assert.assertEquals(1L, res.getLockid()); + Assert.assertEquals(LockState.ACQUIRED, res.getState()); + + res = client.checkLock(1); + Assert.assertEquals(1L, res.getLockid()); + Assert.assertEquals(LockState.ACQUIRED, res.getState()); + + client.heartbeat(txnid, 1); + + client.commitTxn(txnid); + } + + @Before + public void setUp() throws Exception { + TxnDbUtil.prepDb(); + client = new HiveMetaStoreClient(conf); + } + + @After + public void tearDown() throws Exception { + TxnDbUtil.cleanDb(); + } +} diff --git ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager.java ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager.java new file mode 100644 index 0000000..765c272 --- /dev/null +++ ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager.java @@ -0,0 +1,276 @@ +/** + * 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.lockmgr; + +import junit.framework.Assert; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.txn.TxnDbUtil; +import org.apache.hadoop.hive.ql.Context; +import org.apache.hadoop.hive.ql.QueryPlan; +import org.apache.hadoop.hive.ql.hooks.ReadEntity; +import org.apache.hadoop.hive.ql.hooks.WriteEntity; +import org.apache.hadoop.hive.ql.metadata.Partition; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.log4j.Level; +import org.apache.log4j.LogManager; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.util.*; + +/** + * Unit tests for {@link DbTxnManager}. + */ +public class TestDbTxnManager { + + private HiveConf conf = new HiveConf(); + private HiveTxnManager txnMgr; + private Context ctx; + private int nextInput; + private int nextOutput; + HashSet readEntities; + HashSet writeEntities; + + public TestDbTxnManager() throws Exception { + TxnDbUtil.setConfValues(conf); + ctx = new Context(conf); + LogManager.getRootLogger().setLevel(Level.DEBUG); + tearDown(); + } + + @Test + public void testSingleReadTable() throws Exception { + addTableInput(); + QueryPlan qp = new MockQueryPlan(this); + txnMgr.acquireLocks(qp, ctx); + List locks = ctx.getHiveLocks(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(1, + TxnDbUtil.countLockComponents(((DbLockManager.DbHiveLock) locks.get(0)).lockId)); + txnMgr.getLockManager().unlock(locks.get(0)); + locks = txnMgr.getLockManager().getLocks(false, false); + Assert.assertEquals(0, locks.size()); + } + + @Test + public void testSingleReadPartition() throws Exception { + addPartitionInput(newTable(true)); + QueryPlan qp = new MockQueryPlan(this); + txnMgr.acquireLocks(qp, ctx); + List locks = ctx.getHiveLocks(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(1, + TxnDbUtil.countLockComponents(((DbLockManager.DbHiveLock) locks.get(0)).lockId)); + txnMgr.getLockManager().unlock(locks.get(0)); + locks = txnMgr.getLockManager().getLocks(false, false); + Assert.assertEquals(0, locks.size()); + + } + + @Test + public void testSingleReadMultiPartition() throws Exception { + Table t = newTable(true); + addPartitionInput(t); + addPartitionInput(t); + addPartitionInput(t); + QueryPlan qp = new MockQueryPlan(this); + txnMgr.acquireLocks(qp, ctx); + List locks = ctx.getHiveLocks(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(3, + TxnDbUtil.countLockComponents(((DbLockManager.DbHiveLock) locks.get(0)).lockId)); + txnMgr.getLockManager().unlock(locks.get(0)); + locks = txnMgr.getLockManager().getLocks(false, false); + Assert.assertEquals(0, locks.size()); + } + + @Test + public void testJoin() throws Exception { + Table t = newTable(true); + addPartitionInput(t); + addPartitionInput(t); + addPartitionInput(t); + addTableInput(); + QueryPlan qp = new MockQueryPlan(this); + txnMgr.acquireLocks(qp, ctx); + List locks = ctx.getHiveLocks(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(4, + TxnDbUtil.countLockComponents(((DbLockManager.DbHiveLock) locks.get(0)).lockId)); + txnMgr.getLockManager().unlock(locks.get(0)); + locks = txnMgr.getLockManager().getLocks(false, false); + Assert.assertEquals(0, locks.size()); + } + + @Test + public void testSingleWriteTable() throws Exception { + WriteEntity we = addTableOutput(WriteEntity.WriteType.INSERT); + QueryPlan qp = new MockQueryPlan(this); + txnMgr.acquireLocks(qp, ctx); + List locks = ctx.getHiveLocks(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(1, + TxnDbUtil.countLockComponents(((DbLockManager.DbHiveLock) locks.get(0)).lockId)); + txnMgr.getLockManager().unlock(locks.get(0)); + locks = txnMgr.getLockManager().getLocks(false, false); + Assert.assertEquals(0, locks.size()); + } + + @Test + public void testReadWrite() throws Exception { + Table t = newTable(true); + addPartitionInput(t); + addPartitionInput(t); + addPartitionInput(t); + WriteEntity we = addTableOutput(WriteEntity.WriteType.INSERT); + QueryPlan qp = new MockQueryPlan(this); + txnMgr.acquireLocks(qp, ctx); + List locks = ctx.getHiveLocks(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(4, + TxnDbUtil.countLockComponents(((DbLockManager.DbHiveLock) locks.get(0)).lockId)); + txnMgr.getLockManager().unlock(locks.get(0)); + locks = txnMgr.getLockManager().getLocks(false, false); + Assert.assertEquals(0, locks.size()); + } + + @Test + public void testUpdate() throws Exception { + WriteEntity we = addTableOutput(WriteEntity.WriteType.UPDATE); + QueryPlan qp = new MockQueryPlan(this); + txnMgr.acquireLocks(qp, ctx); + List locks = ctx.getHiveLocks(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(1, + TxnDbUtil.countLockComponents(((DbLockManager.DbHiveLock) locks.get(0)).lockId)); + txnMgr.getLockManager().unlock(locks.get(0)); + locks = txnMgr.getLockManager().getLocks(false, false); + Assert.assertEquals(0, locks.size()); + } + + @Test + public void testDelete() throws Exception { + WriteEntity we = addTableOutput(WriteEntity.WriteType.DELETE); + QueryPlan qp = new MockQueryPlan(this); + txnMgr.acquireLocks(qp, ctx); + List locks = ctx.getHiveLocks(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(1, + TxnDbUtil.countLockComponents(((DbLockManager.DbHiveLock) locks.get(0)).lockId)); + txnMgr.getLockManager().unlock(locks.get(0)); + locks = txnMgr.getLockManager().getLocks(false, false); + Assert.assertEquals(0, locks.size()); + } + + @Test + public void testDDL() throws Exception { + WriteEntity we = addTableOutput(WriteEntity.WriteType.DDL); + QueryPlan qp = new MockQueryPlan(this); + txnMgr.acquireLocks(qp, ctx); + List locks = ctx.getHiveLocks(); + Assert.assertEquals(1, locks.size()); + Assert.assertEquals(1, + TxnDbUtil.countLockComponents(((DbLockManager.DbHiveLock) locks.get(0)).lockId)); + txnMgr.getLockManager().unlock(locks.get(0)); + locks = txnMgr.getLockManager().getLocks(false, false); + Assert.assertEquals(0, locks.size()); + } + + @Before + public void setUp() throws Exception { + TxnDbUtil.prepDb(); + txnMgr = TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf); + Assert.assertTrue(txnMgr instanceof DbTxnManager); + nextInput = 1; + nextOutput = 1; + readEntities = new HashSet(); + writeEntities = new HashSet(); + } + + @After + public void tearDown() throws Exception { + TxnDbUtil.cleanDb(); + } + + private static class MockQueryPlan extends QueryPlan { + private HashSet inputs; + private HashSet outputs; + + MockQueryPlan(TestDbTxnManager test) { + HashSet r = test.readEntities; + HashSet w = test.writeEntities; + inputs = (r == null) ? new HashSet() : r; + outputs = (w == null) ? new HashSet() : w; + } + + @Override + public HashSet getInputs() { + return inputs; + } + + @Override + public HashSet getOutputs() { + return outputs; + } + } + + private Table newTable(boolean isPartitioned) { + Table t = new Table("default", "table" + Integer.toString(nextInput++)); + if (isPartitioned) { + FieldSchema fs = new FieldSchema(); + fs.setName("version"); + fs.setType("String"); + List partCols = new ArrayList(1); + partCols.add(fs); + t.setPartCols(partCols); + } + return t; + } + + private void addTableInput() { + ReadEntity re = new ReadEntity(newTable(false)); + readEntities.add(re); + } + + private void addPartitionInput(Table t) throws Exception { + Map partSpec = new HashMap(); + partSpec.put("version", Integer.toString(nextInput++)); + Partition p = new Partition(t, partSpec, new Path("/dev/null")); + ReadEntity re = new ReadEntity(p); + readEntities.add(re); + } + + private WriteEntity addTableOutput(WriteEntity.WriteType writeType) { + WriteEntity we = new WriteEntity(newTable(false), writeType); + writeEntities.add(we); + return we; + } + + private WriteEntity addPartitionOutput(Table t, WriteEntity.WriteType writeType) + throws Exception { + Map partSpec = new HashMap(); + partSpec.put("version", Integer.toString(nextInput++)); + Partition p = new Partition(t, partSpec, new Path("/dev/null")); + WriteEntity we = new WriteEntity(p, writeType); + writeEntities.add(we); + return we; + } +} diff --git ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDummyTxnManager.java ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDummyTxnManager.java new file mode 100644 index 0000000..7e104c9 --- /dev/null +++ ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDummyTxnManager.java @@ -0,0 +1,60 @@ +package org.apache.hadoop.hive.ql.lockmgr; + +import junit.framework.Assert; +import org.apache.hadoop.hive.ql.lockmgr.HiveLockObject.HiveLockObjectData; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; + +public class TestDummyTxnManager { + + @Test + public void testDedupLockObjects() { + List lockObjs = new ArrayList(); + String path1 = "path1"; + String path2 = "path2"; + HiveLockObjectData lockData1 = new HiveLockObjectData( + "query1", "1", "IMPLICIT", "drop table table1"); + HiveLockObjectData lockData2 = new HiveLockObjectData( + "query1", "1", "IMPLICIT", "drop table table1"); + + // Start with the following locks: + // [path1, shared] + // [path1, exclusive] + // [path2, shared] + // [path2, shared] + // [path2, shared] + lockObjs.add(new HiveLockObj(new HiveLockObject(path1, lockData1), HiveLockMode.SHARED)); + String name1 = lockObjs.get(lockObjs.size() - 1).getName(); + lockObjs.add(new HiveLockObj(new HiveLockObject(path1, lockData1), HiveLockMode.EXCLUSIVE)); + lockObjs.add(new HiveLockObj(new HiveLockObject(path2, lockData2), HiveLockMode.SHARED)); + String name2 = lockObjs.get(lockObjs.size() - 1).getName(); + lockObjs.add(new HiveLockObj(new HiveLockObject(path2, lockData2), HiveLockMode.SHARED)); + lockObjs.add(new HiveLockObj(new HiveLockObject(path2, lockData2), HiveLockMode.SHARED)); + + DummyTxnManager.dedupLockObjects(lockObjs); + + // After dedup we should be left with 2 locks: + // [path1, exclusive] + // [path2, shared] + Assert.assertEquals("Locks should be deduped", 2, lockObjs.size()); + + Comparator cmp = new Comparator() { + public int compare(HiveLockObj lock1, HiveLockObj lock2) { + return lock1.getName().compareTo(lock2.getName()); + } + }; + Collections.sort(lockObjs, cmp); + + HiveLockObj lockObj = lockObjs.get(0); + Assert.assertEquals(name1, lockObj.getName()); + Assert.assertEquals(HiveLockMode.EXCLUSIVE, lockObj.getMode()); + + lockObj = lockObjs.get(1); + Assert.assertEquals(name2, lockObj.getName()); + Assert.assertEquals(HiveLockMode.SHARED, lockObj.getMode()); + } +}