You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by st...@apache.org on 2017/11/09 01:32:22 UTC
[01/14] hive git commit: HIVE-15761: ObjectStore.getNextNotification
could return an empty NotificationEventResponse causing TProtocolException
(Sergio Pena, reviewed by Aihua Xu)
Repository: hive
Updated Branches:
refs/heads/branch-2.3 a4e3e3592 -> 62d9b1981
HIVE-15761: ObjectStore.getNextNotification could return an empty NotificationEventResponse causing TProtocolException (Sergio Pena, reviewed by Aihua Xu)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/145ed20b
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/145ed20b
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/145ed20b
Branch: refs/heads/branch-2.3
Commit: 145ed20b04e4bf6bcb833e94efe71bcbe0851500
Parents: ecf9704
Author: Sergio Pena <se...@cloudera.com>
Authored: Fri Apr 21 15:23:52 2017 -0500
Committer: Sahil Takiar <st...@cloudera.com>
Committed: Tue Nov 7 08:15:47 2017 -0800
----------------------------------------------------------------------
.../java/org/apache/hadoop/hive/metastore/ObjectStore.java | 7 ++++---
.../org/apache/hadoop/hive/metastore/TestObjectStore.java | 5 +++++
2 files changed, 9 insertions(+), 3 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/145ed20b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index a63519a..358cf17 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -8260,6 +8260,9 @@ public class ObjectStore implements RawStore, Configurable {
public NotificationEventResponse getNextNotification(NotificationEventRequest rqst) {
boolean commited = false;
Query query = null;
+
+ NotificationEventResponse result = new NotificationEventResponse();
+ result.setEvents(new ArrayList<NotificationEvent>());
try {
openTransaction();
long lastEvent = rqst.getLastEvent();
@@ -8269,11 +8272,9 @@ public class ObjectStore implements RawStore, Configurable {
Collection<MNotificationLog> events = (Collection) query.execute(lastEvent);
commited = commitTransaction();
if (events == null) {
- return null;
+ return result;
}
Iterator<MNotificationLog> i = events.iterator();
- NotificationEventResponse result = new NotificationEventResponse();
- result.setEvents(new ArrayList<NotificationEvent>());
int maxEvents = rqst.getMaxEvents() > 0 ? rqst.getMaxEvents() : Integer.MAX_VALUE;
int numEvents = 0;
while (i.hasNext() && numEvents++ < maxEvents) {
http://git-wip-us.apache.org/repos/asf/hive/blob/145ed20b/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java b/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
index 9b8eaf2..6524ee7 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
@@ -153,11 +153,16 @@ public class TestObjectStore {
Assert.assertEquals(2, eventResponse.getEventsSize());
Assert.assertEquals(FIRST_EVENT_ID, eventResponse.getEvents().get(0).getEventId());
Assert.assertEquals(SECOND_EVENT_ID, eventResponse.getEvents().get(1).getEventId());
+
// Verify that getNextNotification(last) returns events after a specified event
eventResponse = objectStore.getNextNotification(new NotificationEventRequest(FIRST_EVENT_ID));
Assert.assertEquals(1, eventResponse.getEventsSize());
Assert.assertEquals(SECOND_EVENT_ID, eventResponse.getEvents().get(0).getEventId());
+ // Verify that getNextNotification(last) returns zero events if there are no more notifications available
+ eventResponse = objectStore.getNextNotification(new NotificationEventRequest(SECOND_EVENT_ID));
+ Assert.assertEquals(0, eventResponse.getEventsSize());
+
// Verify that cleanNotificationEvents() cleans up all old notifications
Thread.sleep(1);
objectStore.cleanNotificationEvents(1);
[04/14] hive git commit: HIVE-16646: Alias in transform ... as clause
shouldn't be case sensitive (Yibing Shi, reviewed by Yongzhi Chen)
Posted by st...@apache.org.
HIVE-16646: Alias in transform ... as clause shouldn't be case sensitive (Yibing Shi, reviewed by Yongzhi Chen)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/cf64bb9d
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/cf64bb9d
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/cf64bb9d
Branch: refs/heads/branch-2.3
Commit: cf64bb9d4b280b433db82a93ceace1a6091a0506
Parents: a4e3e35
Author: Yongzhi Chen <yc...@apache.org>
Authored: Thu May 11 14:53:57 2017 -0400
Committer: Sahil Takiar <st...@cloudera.com>
Committed: Tue Nov 7 08:15:47 2017 -0800
----------------------------------------------------------------------
.../hadoop/hive/ql/parse/SemanticAnalyzer.java | 4 +--
ql/src/test/queries/clientpositive/transform3.q | 6 +++++
.../results/clientpositive/transform3.q.out | 28 ++++++++++++++++++++
3 files changed, 36 insertions(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/cf64bb9d/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index d49708c..2a62062 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -3770,7 +3770,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
if (outputColNames) {
for (int i = 0; i < ccount; ++i) {
String colAlias = unescapeIdentifier(((ASTNode) collist.getChild(i))
- .getText());
+ .getText()).toLowerCase();
failIfColAliasExists(colAliasNamesDuplicateCheck, colAlias);
String intName = getColumnInternalName(i);
ColumnInfo colInfo = new ColumnInfo(intName,
@@ -3783,7 +3783,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
ASTNode child = (ASTNode) collist.getChild(i);
assert child.getType() == HiveParser.TOK_TABCOL;
String colAlias = unescapeIdentifier(((ASTNode) child.getChild(0))
- .getText());
+ .getText()).toLowerCase();
failIfColAliasExists(colAliasNamesDuplicateCheck, colAlias);
String intName = getColumnInternalName(i);
ColumnInfo colInfo = new ColumnInfo(intName, TypeInfoUtils
http://git-wip-us.apache.org/repos/asf/hive/blob/cf64bb9d/ql/src/test/queries/clientpositive/transform3.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/transform3.q b/ql/src/test/queries/clientpositive/transform3.q
new file mode 100644
index 0000000..4a2a368
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/transform3.q
@@ -0,0 +1,6 @@
+CREATE TABLE transform3_t1 (col string);
+INSERT OVERWRITE TABLE transform3_t1 VALUES('aaaa');
+
+SELECT t.newCol FROM (
+ SELECT TRANSFORM(col) USING 'cat' AS (NewCol string) FROM transform3_t1
+) t;
http://git-wip-us.apache.org/repos/asf/hive/blob/cf64bb9d/ql/src/test/results/clientpositive/transform3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/transform3.q.out b/ql/src/test/results/clientpositive/transform3.q.out
new file mode 100644
index 0000000..5f93ed8
--- /dev/null
+++ b/ql/src/test/results/clientpositive/transform3.q.out
@@ -0,0 +1,28 @@
+PREHOOK: query: CREATE TABLE transform3_t1 (col string)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@transform3_t1
+POSTHOOK: query: CREATE TABLE transform3_t1 (col string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@transform3_t1
+PREHOOK: query: INSERT OVERWRITE TABLE transform3_t1 VALUES('aaaa')
+PREHOOK: type: QUERY
+PREHOOK: Output: default@transform3_t1
+POSTHOOK: query: INSERT OVERWRITE TABLE transform3_t1 VALUES('aaaa')
+POSTHOOK: type: QUERY
+POSTHOOK: Output: default@transform3_t1
+POSTHOOK: Lineage: transform3_t1.col SIMPLE [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: SELECT t.newCol FROM (
+ SELECT TRANSFORM(col) USING 'cat' AS (NewCol string) FROM transform3_t1
+) t
+PREHOOK: type: QUERY
+PREHOOK: Input: default@transform3_t1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT t.newCol FROM (
+ SELECT TRANSFORM(col) USING 'cat' AS (NewCol string) FROM transform3_t1
+) t
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@transform3_t1
+#### A masked pattern was here ####
+aaaa
[02/14] hive git commit: HIVE-17640: Comparison of date return null
if time part is provided in string. (Yongzhi Chen, reviewed by Aihua Xu)
Posted by st...@apache.org.
HIVE-17640: Comparison of date return null if time part is provided in string. (Yongzhi Chen, reviewed by Aihua Xu)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/a2df8f48
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/a2df8f48
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/a2df8f48
Branch: refs/heads/branch-2.3
Commit: a2df8f48ee32008032102759d48b0172b949b494
Parents: f3dc6a9
Author: Yongzhi Chen <yc...@apache.org>
Authored: Fri Oct 20 10:29:49 2017 -0400
Committer: Sahil Takiar <st...@cloudera.com>
Committed: Tue Nov 7 08:15:47 2017 -0800
----------------------------------------------------------------------
.../queries/clientpositive/date_withtimestamp.q | 3 +++
.../clientpositive/annotate_stats_select.q.out | 4 +--
.../clientpositive/constantfolding.q.out | 4 +--
.../clientpositive/date_withtimestamp.q.out | 27 ++++++++++++++++++++
.../PrimitiveObjectInspectorUtils.java | 16 +++++++++---
5 files changed, 47 insertions(+), 7 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/a2df8f48/ql/src/test/queries/clientpositive/date_withtimestamp.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/date_withtimestamp.q b/ql/src/test/queries/clientpositive/date_withtimestamp.q
new file mode 100644
index 0000000..b6d04f5
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/date_withtimestamp.q
@@ -0,0 +1,3 @@
+select "2016-12-29 23:59:59" < cast("2016-12-30" as date);
+select "2016-12-30 00:00:00" = cast("2016-12-30" as date);
+select "2016-12-31 00:00:01" > cast("2016-12-30" as date);
http://git-wip-us.apache.org/repos/asf/hive/blob/a2df8f48/ql/src/test/results/clientpositive/annotate_stats_select.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/annotate_stats_select.q.out b/ql/src/test/results/clientpositive/annotate_stats_select.q.out
index 873f1ab..489f4d2 100644
--- a/ql/src/test/results/clientpositive/annotate_stats_select.q.out
+++ b/ql/src/test/results/clientpositive/annotate_stats_select.q.out
@@ -470,9 +470,9 @@ STAGE PLANS:
alias: alltypes_orc
Statistics: Num rows: 2 Data size: 1686 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
- expressions: null (type: date)
+ expressions: 1970-12-31 (type: date)
outputColumnNames: _col0
- Statistics: Num rows: 2 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 2 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE
ListSink
PREHOOK: query: explain select cast("58.174" as DECIMAL) from alltypes_orc
http://git-wip-us.apache.org/repos/asf/hive/blob/a2df8f48/ql/src/test/results/clientpositive/constantfolding.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/constantfolding.q.out b/ql/src/test/results/clientpositive/constantfolding.q.out
index 10e185f..f9a9d24 100644
--- a/ql/src/test/results/clientpositive/constantfolding.q.out
+++ b/ql/src/test/results/clientpositive/constantfolding.q.out
@@ -205,9 +205,9 @@ STAGE PLANS:
alias: src
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
- expressions: null (type: date)
+ expressions: 1970-12-31 (type: date)
outputColumnNames: _col0
- Statistics: Num rows: 500 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 500 Data size: 28000 Basic stats: COMPLETE Column stats: COMPLETE
ListSink
PREHOOK: query: CREATE TABLE dest1(c1 STRING) STORED AS TEXTFILE
http://git-wip-us.apache.org/repos/asf/hive/blob/a2df8f48/ql/src/test/results/clientpositive/date_withtimestamp.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/date_withtimestamp.q.out b/ql/src/test/results/clientpositive/date_withtimestamp.q.out
new file mode 100644
index 0000000..3661888
--- /dev/null
+++ b/ql/src/test/results/clientpositive/date_withtimestamp.q.out
@@ -0,0 +1,27 @@
+PREHOOK: query: select "2016-12-29 23:59:59" < cast("2016-12-30" as date)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+POSTHOOK: query: select "2016-12-29 23:59:59" < cast("2016-12-30" as date)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+true
+PREHOOK: query: select "2016-12-30 00:00:00" = cast("2016-12-30" as date)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+POSTHOOK: query: select "2016-12-30 00:00:00" = cast("2016-12-30" as date)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+true
+PREHOOK: query: select "2016-12-31 00:00:01" > cast("2016-12-30" as date)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+POSTHOOK: query: select "2016-12-31 00:00:01" > cast("2016-12-30" as date)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+#### A masked pattern was here ####
+true
http://git-wip-us.apache.org/repos/asf/hive/blob/a2df8f48/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java
index 9642a7e..27af6ec 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java
@@ -1072,16 +1072,26 @@ public final class PrimitiveObjectInspectorUtils {
try {
result = Date.valueOf(s);
} catch (IllegalArgumentException e) {
- result = null;
+ Timestamp ts = getTimestampFromString(s);
+ if (ts != null) {
+ result = new Date(ts.getTime());
+ } else {
+ result = null;
+ }
}
break;
case CHAR:
case VARCHAR: {
+ String val = getString(o, oi).trim();
try {
- String val = getString(o, oi).trim();
result = Date.valueOf(val);
} catch (IllegalArgumentException e) {
- result = null;
+ Timestamp ts = getTimestampFromString(val);
+ if (ts != null) {
+ result = new Date(ts.getTime());
+ } else {
+ result = null;
+ }
}
break;
}
[13/14] hive git commit: HIVE-16487: Serious Zookeeper exception is
logged when a race condition happens (Peter Vary via Chaoyu Tang)
Posted by st...@apache.org.
HIVE-16487: Serious Zookeeper exception is logged when a race condition happens (Peter Vary via Chaoyu Tang)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/3f0b696f
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/3f0b696f
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/3f0b696f
Branch: refs/heads/branch-2.3
Commit: 3f0b696f1badc10453dd59a65ce70d41420d02d1
Parents: c2b5dba
Author: Chaoyu Tang <ct...@cloudera.com>
Authored: Mon May 1 09:53:14 2017 -0400
Committer: Sahil Takiar <st...@cloudera.com>
Committed: Tue Nov 7 08:15:48 2017 -0800
----------------------------------------------------------------------
.../zookeeper/ZooKeeperHiveLockManager.java | 39 +++++++++++++-------
1 file changed, 26 insertions(+), 13 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/3f0b696f/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/ZooKeeperHiveLockManager.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/ZooKeeperHiveLockManager.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/ZooKeeperHiveLockManager.java
index c2a4806..9b46ae7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/ZooKeeperHiveLockManager.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/ZooKeeperHiveLockManager.java
@@ -285,8 +285,10 @@ public class ZooKeeperHiveLockManager implements HiveLockManager {
int tryNum = 0;
ZooKeeperHiveLock ret = null;
Set<String> conflictingLocks = new HashSet<String>();
+ Exception lastException = null;
do {
+ lastException = null;
tryNum++;
try {
if (tryNum > 1) {
@@ -298,26 +300,22 @@ public class ZooKeeperHiveLockManager implements HiveLockManager {
break;
}
} catch (Exception e1) {
+ lastException = e1;
if (e1 instanceof KeeperException) {
KeeperException e = (KeeperException) e1;
switch (e.code()) {
case CONNECTIONLOSS:
case OPERATIONTIMEOUT:
+ case NONODE:
+ case NODEEXISTS:
LOG.debug("Possibly transient ZooKeeper exception: ", e);
- continue;
+ break;
default:
LOG.error("Serious Zookeeper exception: ", e);
break;
}
- }
- if (tryNum >= numRetriesForLock) {
- console.printError("Unable to acquire " + key.getData().getLockMode()
- + ", " + mode + " lock " + key.getDisplayName() + " after "
- + tryNum + " attempts.");
- LOG.error("Exceeds maximum retries with errors: ", e1);
- printConflictingLocks(key,mode,conflictingLocks);
- conflictingLocks.clear();
- throw new LockException(e1);
+ } else {
+ LOG.error("Other unexpected exception: ", e1);
}
}
} while (tryNum < numRetriesForLock);
@@ -327,8 +325,11 @@ public class ZooKeeperHiveLockManager implements HiveLockManager {
+ ", " + mode + " lock " + key.getDisplayName() + " after "
+ tryNum + " attempts.");
printConflictingLocks(key,mode,conflictingLocks);
+ if (lastException != null) {
+ LOG.error("Exceeds maximum retries with errors: ", lastException);
+ throw new LockException(lastException);
+ }
}
- conflictingLocks.clear();
return ret;
}
@@ -350,6 +351,19 @@ public class ZooKeeperHiveLockManager implements HiveLockManager {
}
}
+ /**
+ * Creates a primitive lock object on ZooKeeper.
+ * @param key The lock data
+ * @param mode The lock mode (HiveLockMode - EXCLUSIVE/SHARED/SEMI_SHARED)
+ * @param keepAlive If true creating PERSISTENT ZooKeeper locks, otherwise EPHEMERAL ZooKeeper
+ * locks
+ * @param parentCreated If we expect, that the parent is already created then true, otherwise
+ * we will try to create the parents as well
+ * @param conflictingLocks The set where we should collect the conflicting locks when
+ * the logging level is set to DEBUG
+ * @return The created ZooKeeperHiveLock object, null if there was a conflicting lock
+ * @throws Exception If there was an unexpected Exception
+ */
private ZooKeeperHiveLock lockPrimitive(HiveLockObject key,
HiveLockMode mode, boolean keepAlive, boolean parentCreated,
Set<String> conflictingLocks)
@@ -390,7 +404,7 @@ public class ZooKeeperHiveLockManager implements HiveLockManager {
int seqNo = getSequenceNumber(res, getLockName(lastName, mode));
if (seqNo == -1) {
curatorFramework.delete().forPath(res);
- return null;
+ throw new LockException("The created node does not contain a sequence number: " + res);
}
List<String> children = curatorFramework.getChildren().forPath(lastName);
@@ -584,7 +598,6 @@ public class ZooKeeperHiveLockManager implements HiveLockManager {
/**
* @param conf Hive configuration
- * @param zkpClient The ZooKeeper client
* @param key The object to be compared against - if key is null, then get all locks
**/
private static List<HiveLock> getLocks(HiveConf conf,
[11/14] hive git commit: HIVE-17189 : Fix backwards incompatibility
in HiveMetaStoreClient (Vihang Karajgaonkar, reviewed by Alan Gates)
Posted by st...@apache.org.
HIVE-17189 : Fix backwards incompatibility in HiveMetaStoreClient (Vihang Karajgaonkar, reviewed by Alan Gates)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/b8bd4594
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/b8bd4594
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/b8bd4594
Branch: refs/heads/branch-2.3
Commit: b8bd4594bef718b1eeac9fceb437d7df7b480ed1
Parents: 0ff4d5b
Author: Vihang Karajgaonkar <vi...@cloudera.com>
Authored: Thu Jul 27 16:23:47 2017 -0700
Committer: Sahil Takiar <st...@cloudera.com>
Committed: Tue Nov 7 08:15:47 2017 -0800
----------------------------------------------------------------------
.../hive/metastore/TestHiveMetaStore.java | 49 ++++++++++++++++++
.../hive/metastore/HiveMetaStoreClient.java | 23 +++++++++
.../hadoop/hive/metastore/IMetaStoreClient.java | 52 +++++++++++++++++++-
.../ql/metadata/SessionHiveMetaStoreClient.java | 13 +++++
4 files changed, 136 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/b8bd4594/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
index 1b0b537..4b1df8e 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
@@ -608,6 +608,55 @@ public abstract class TestHiveMetaStore extends TestCase {
assertEquals(" should have returned 50 partitions", maxParts, partitions.size());
}
+ public void testAlterTableCascade() throws Throwable {
+ // create a table with multiple partitions
+ String dbName = "compdb";
+ String tblName = "comptbl";
+ String typeName = "Person";
+
+ cleanUp(dbName, tblName, typeName);
+
+ List<List<String>> values = new ArrayList<List<String>>();
+ values.add(makeVals("2008-07-01 14:13:12", "14"));
+ values.add(makeVals("2008-07-01 14:13:12", "15"));
+ values.add(makeVals("2008-07-02 14:13:12", "15"));
+ values.add(makeVals("2008-07-03 14:13:12", "151"));
+
+ createMultiPartitionTableSchema(dbName, tblName, typeName, values);
+ Table tbl = client.getTable(dbName, tblName);
+ List<FieldSchema> cols = tbl.getSd().getCols();
+ cols.add(new FieldSchema("new_col", serdeConstants.STRING_TYPE_NAME, ""));
+ tbl.getSd().setCols(cols);
+ //add new column with cascade option
+ client.alter_table(dbName, tblName, tbl, true);
+ //
+ Table tbl2 = client.getTable(dbName, tblName);
+ Assert.assertEquals("Unexpected number of cols", 3, tbl2.getSd().getCols().size());
+ Assert.assertEquals("Unexpected column name", "new_col", tbl2.getSd().getCols().get(2).getName());
+ //get a partition
+ List<String> pvalues = new ArrayList<>(2);
+ pvalues.add("2008-07-01 14:13:12");
+ pvalues.add("14");
+ Partition partition = client.getPartition(dbName, tblName, pvalues);
+ Assert.assertEquals("Unexpected number of cols", 3, partition.getSd().getCols().size());
+ Assert.assertEquals("Unexpected column name", "new_col", partition.getSd().getCols().get(2).getName());
+
+ //add another column
+ cols = tbl.getSd().getCols();
+ cols.add(new FieldSchema("new_col2", serdeConstants.STRING_TYPE_NAME, ""));
+ tbl.getSd().setCols(cols);
+ //add new column with no cascade option
+ client.alter_table(dbName, tblName, tbl, false);
+ tbl2 = client.getTable(dbName, tblName);
+ Assert.assertEquals("Unexpected number of cols", 4, tbl2.getSd().getCols().size());
+ Assert.assertEquals("Unexpected column name", "new_col2", tbl2.getSd().getCols().get(3).getName());
+ //get partition, this partition should not have the newly added column since cascade option
+ //was false
+ partition = client.getPartition(dbName, tblName, pvalues);
+ Assert.assertEquals("Unexpected number of cols", 3, partition.getSd().getCols().size());
+ }
+
+
public void testListPartitionNames() throws Throwable {
// create a table with multiple partitions
String dbName = "compdb";
http://git-wip-us.apache.org/repos/asf/hive/blob/b8bd4594/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index c5db77f..b1a9782 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -50,6 +50,7 @@ import java.security.PrivilegedExceptionAction;
import javax.security.auth.login.LoginException;
import org.apache.hadoop.hive.common.ObjectPair;
+import org.apache.hadoop.hive.common.StatsSetupConst;
import org.apache.hadoop.hive.common.ValidTxnList;
import org.apache.hadoop.hive.common.auth.HiveAuthUtils;
import org.apache.hadoop.hive.common.classification.InterfaceAudience;
@@ -359,6 +360,16 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
}
@Override
+ public void alter_table(String defaultDatabaseName, String tblName, Table table,
+ boolean cascade) throws InvalidOperationException, MetaException, TException {
+ EnvironmentContext environmentContext = new EnvironmentContext();
+ if (cascade) {
+ environmentContext.putToProperties(StatsSetupConst.CASCADE, StatsSetupConst.TRUE);
+ }
+ alter_table_with_environmentContext(defaultDatabaseName, tblName, table, environmentContext);
+ }
+
+ @Override
public void alter_table_with_environmentContext(String dbname, String tbl_name, Table new_tbl,
EnvironmentContext envContext) throws InvalidOperationException, MetaException, TException {
client.alter_table_with_environment_context(dbname, tbl_name, new_tbl, envContext);
@@ -1493,12 +1504,24 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
}
@Override
+ public void alter_partition(String dbName, String tblName, Partition newPart)
+ throws InvalidOperationException, MetaException, TException {
+ client.alter_partition_with_environment_context(dbName, tblName, newPart, null);
+ }
+
+ @Override
public void alter_partition(String dbName, String tblName, Partition newPart, EnvironmentContext environmentContext)
throws InvalidOperationException, MetaException, TException {
client.alter_partition_with_environment_context(dbName, tblName, newPart, environmentContext);
}
@Override
+ public void alter_partitions(String dbName, String tblName, List<Partition> newParts)
+ throws InvalidOperationException, MetaException, TException {
+ client.alter_partitions_with_environment_context(dbName, tblName, newParts, null);
+ }
+
+ @Override
public void alter_partitions(String dbName, String tblName, List<Partition> newParts, EnvironmentContext environmentContext)
throws InvalidOperationException, MetaException, TException {
client.alter_partitions_with_environment_context(dbName, tblName, newParts, environmentContext);
http://git-wip-us.apache.org/repos/asf/hive/blob/b8bd4594/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
index e9df1e1..e7ead6b 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -26,6 +26,7 @@ import java.util.Map;
import java.util.Map.Entry;
import org.apache.hadoop.hive.common.ObjectPair;
+import org.apache.hadoop.hive.common.StatsSetupConst;
import org.apache.hadoop.hive.common.ValidTxnList;
import org.apache.hadoop.hive.common.classification.InterfaceAudience;
import org.apache.hadoop.hive.common.classification.InterfaceAudience.Public;
@@ -707,6 +708,14 @@ public interface IMetaStoreClient {
void alter_table(String defaultDatabaseName, String tblName,
Table table) throws InvalidOperationException, MetaException, TException;
+ /**
+ * Use alter_table_with_environmentContext instead of alter_table with cascade option
+ * passed in EnvironmentContext using {@code StatsSetupConst.CASCADE}
+ */
+ @Deprecated
+ void alter_table(String defaultDatabaseName, String tblName, Table table,
+ boolean cascade) throws InvalidOperationException, MetaException, TException;
+
//wrapper of alter_table_with_cascade
void alter_table_with_environmentContext(String defaultDatabaseName, String tblName, Table table,
EnvironmentContext environmentContext) throws InvalidOperationException, MetaException,
@@ -780,6 +789,26 @@ public interface IMetaStoreClient {
boolean dropPartition(String db_name, String tbl_name,
String name, boolean deleteData) throws NoSuchObjectException,
MetaException, TException;
+
+ /**
+ * updates a partition to new partition
+ *
+ * @param dbName
+ * database of the old partition
+ * @param tblName
+ * table name of the old partition
+ * @param newPart
+ * new partition
+ * @throws InvalidOperationException
+ * if the old partition does not exist
+ * @throws MetaException
+ * if error in updating metadata
+ * @throws TException
+ * if error in communicating with metastore server
+ */
+ void alter_partition(String dbName, String tblName, Partition newPart)
+ throws InvalidOperationException, MetaException, TException;
+
/**
* updates a partition to new partition
*
@@ -815,7 +844,28 @@ public interface IMetaStoreClient {
* @throws TException
* if error in communicating with metastore server
*/
- void alter_partitions(String dbName, String tblName, List<Partition> newParts, EnvironmentContext environmentContext)
+ void alter_partitions(String dbName, String tblName, List<Partition> newParts)
+ throws InvalidOperationException, MetaException, TException;
+
+ /**
+ * updates a list of partitions
+ *
+ * @param dbName
+ * database of the old partition
+ * @param tblName
+ * table name of the old partition
+ * @param newParts
+ * list of partitions
+ * @param environmentContext
+ * @throws InvalidOperationException
+ * if the old partition does not exist
+ * @throws MetaException
+ * if error in updating metadata
+ * @throws TException
+ * if error in communicating with metastore server
+ */
+ void alter_partitions(String dbName, String tblName, List<Partition> newParts,
+ EnvironmentContext environmentContext)
throws InvalidOperationException, MetaException, TException;
/**
http://git-wip-us.apache.org/repos/asf/hive/blob/b8bd4594/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
index 8eb011e..109bc3a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
@@ -280,6 +280,19 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
return super.getSchema(dbName, tableName);
}
+ @Deprecated
+ @Override
+ public void alter_table(String dbname, String tbl_name, org.apache.hadoop.hive.metastore.api.Table new_tbl,
+ boolean cascade) throws InvalidOperationException, MetaException, TException {
+ org.apache.hadoop.hive.metastore.api.Table old_tbl = getTempTable(dbname, tbl_name);
+ if (old_tbl != null) {
+ //actually temp table does not support partitions, cascade is not applicable here
+ alterTempTable(dbname, tbl_name, old_tbl, new_tbl, null);
+ return;
+ }
+ super.alter_table(dbname, tbl_name, new_tbl, cascade);
+ }
+
@Override
public void alter_table(String dbname, String tbl_name,
org.apache.hadoop.hive.metastore.api.Table new_tbl) throws InvalidOperationException,
[10/14] hive git commit: HIVE-17169: Avoid extra call to
KeyProvider::getMetadata() (Mithun Radhakrishnan, reviewed by Owen O'Malley)
Posted by st...@apache.org.
HIVE-17169: Avoid extra call to KeyProvider::getMetadata() (Mithun Radhakrishnan, reviewed by Owen O'Malley)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/24dcccf3
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/24dcccf3
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/24dcccf3
Branch: refs/heads/branch-2.3
Commit: 24dcccf3030c5a6713dd901822dba9672f27de14
Parents: cd39cf3
Author: Mithun RK <mi...@apache.org>
Authored: Tue Aug 15 13:52:11 2017 -0700
Committer: Sahil Takiar <st...@cloudera.com>
Committed: Tue Nov 7 08:15:47 2017 -0800
----------------------------------------------------------------------
.../apache/hadoop/hive/shims/Hadoop23Shims.java | 39 ++++++++++++--------
1 file changed, 24 insertions(+), 15 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/24dcccf3/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
----------------------------------------------------------------------
diff --git a/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java b/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
index 0483e91..985a5bd 100644
--- a/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
+++ b/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
@@ -39,6 +39,7 @@ import java.util.TreeMap;
import javax.security.auth.Subject;
import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.CipherSuite;
import org.apache.hadoop.crypto.key.KeyProvider;
import org.apache.hadoop.crypto.key.KeyProvider.Options;
import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
@@ -1200,6 +1201,14 @@ public class Hadoop23Shims extends HadoopShimsSecure {
((HdfsEncryptionShim)encryptionShim2).hdfsAdmin.getEncryptionZoneForPath(path2));
}
+ /**
+ * Compares two encryption key strengths.
+ *
+ * @param path1 First path to compare
+ * @param path2 Second path to compare
+ * @return 1 if path1 is stronger; 0 if paths are equals; -1 if path1 is weaker.
+ * @throws IOException If an error occurred attempting to get key metadata
+ */
@Override
public int comparePathKeyStrength(Path path1, Path path2) throws IOException {
EncryptionZone zone1, zone2;
@@ -1215,7 +1224,7 @@ public class Hadoop23Shims extends HadoopShimsSecure {
return 1;
}
- return compareKeyStrength(zone1.getKeyName(), zone2.getKeyName());
+ return compareKeyStrength(zone1, zone2);
}
@Override
@@ -1267,28 +1276,28 @@ public class Hadoop23Shims extends HadoopShimsSecure {
/**
* Compares two encryption key strengths.
*
- * @param keyname1 Keyname to compare
- * @param keyname2 Keyname to compare
- * @return 1 if path1 is stronger; 0 if paths are equals; -1 if path1 is weaker.
+ * @param zone1 First EncryptionZone to compare
+ * @param zone2 Second EncryptionZone to compare
+ * @return 1 if zone1 is stronger; 0 if zones are equal; -1 if zone1 is weaker.
* @throws IOException If an error occurred attempting to get key metadata
*/
- private int compareKeyStrength(String keyname1, String keyname2) throws IOException {
- KeyProvider.Metadata meta1, meta2;
+ private int compareKeyStrength(EncryptionZone zone1, EncryptionZone zone2) throws IOException {
- if (keyProvider == null) {
- throw new IOException("HDFS security key provider is not configured on your server.");
- }
+ // zone1, zone2 should already have been checked for nulls.
+ assert zone1 != null && zone2 != null : "Neither EncryptionZone under comparison can be null.";
- meta1 = keyProvider.getMetadata(keyname1);
- meta2 = keyProvider.getMetadata(keyname2);
+ CipherSuite suite1 = zone1.getSuite();
+ CipherSuite suite2 = zone2.getSuite();
- if (meta1.getBitLength() < meta2.getBitLength()) {
- return -1;
- } else if (meta1.getBitLength() == meta2.getBitLength()) {
+ if (suite1 == null && suite2 == null) {
return 0;
- } else {
+ } else if (suite1 == null) {
+ return -1;
+ } else if (suite2 == null) {
return 1;
}
+
+ return Integer.compare(suite1.getAlgorithmBlockSize(), suite2.getAlgorithmBlockSize());
}
}
[14/14] hive git commit: HIVE-17008: Fix boolean flag switchup in
DropTableEvent (Dan Burkert, reviewed by Mohit Sabharwal and Peter Vary)
Posted by st...@apache.org.
HIVE-17008: Fix boolean flag switchup in DropTableEvent (Dan Burkert, reviewed by Mohit Sabharwal and Peter Vary)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/62d9b198
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/62d9b198
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/62d9b198
Branch: refs/heads/branch-2.3
Commit: 62d9b19810935b19f36bdb88baa1e980dd6068c8
Parents: 3f0b696
Author: Peter Vary <pv...@cloudera.com>
Authored: Fri Aug 11 12:00:33 2017 +0200
Committer: Sahil Takiar <st...@cloudera.com>
Committed: Tue Nov 7 08:16:09 2017 -0800
----------------------------------------------------------------------
.../src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java | 4 ++--
1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/62d9b198/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index 616314c..befd15e 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -1700,7 +1700,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
transactionalListenerResponses =
MetaStoreListenerNotifier.notifyEvent(transactionalListeners,
EventType.DROP_TABLE,
- new DropTableEvent(tbl, deleteData, true, this),
+ new DropTableEvent(tbl, true, deleteData, this),
envContext);
}
success = ms.commitTransaction();
@@ -1720,7 +1720,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
if (!listeners.isEmpty()) {
MetaStoreListenerNotifier.notifyEvent(listeners,
EventType.DROP_TABLE,
- new DropTableEvent(tbl, deleteData, success, this),
+ new DropTableEvent(tbl, success, deleteData, this),
envContext,
transactionalListenerResponses, ms);
}
[09/14] hive git commit: HIVE-17891 : HIVE-13076 uses create table if
not exists for the postgres script (Vihang Karajgaonkar,
reviewed by Naveen Gangam)
Posted by st...@apache.org.
HIVE-17891 : HIVE-13076 uses create table if not exists for the postgres script (Vihang Karajgaonkar, reviewed by Naveen Gangam)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/ecf97040
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/ecf97040
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/ecf97040
Branch: refs/heads/branch-2.3
Commit: ecf97040a4ff64cfcba37f7fc111cdd995ee8e0f
Parents: a2df8f4
Author: Vihang Karajgaonkar <vi...@cloudera.com>
Authored: Tue Oct 24 12:32:37 2017 -0700
Committer: Sahil Takiar <st...@cloudera.com>
Committed: Tue Nov 7 08:15:47 2017 -0800
----------------------------------------------------------------------
metastore/scripts/upgrade/postgres/033-HIVE-13076.postgres.sql | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/ecf97040/metastore/scripts/upgrade/postgres/033-HIVE-13076.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/033-HIVE-13076.postgres.sql b/metastore/scripts/upgrade/postgres/033-HIVE-13076.postgres.sql
index 9ee7c11..59e702b 100644
--- a/metastore/scripts/upgrade/postgres/033-HIVE-13076.postgres.sql
+++ b/metastore/scripts/upgrade/postgres/033-HIVE-13076.postgres.sql
@@ -1,4 +1,4 @@
-CREATE TABLE IF NOT EXISTS "KEY_CONSTRAINTS"
+CREATE TABLE "KEY_CONSTRAINTS"
(
"CHILD_CD_ID" BIGINT,
"CHILD_INTEGER_IDX" BIGINT,
[08/14] hive git commit: HIVE-17184 : Unexpected new line in beeline
when running with -f option (Vihang Karajgaonkar, reviewed by Peter Vary)
Posted by st...@apache.org.
HIVE-17184 : Unexpected new line in beeline when running with -f option (Vihang Karajgaonkar, reviewed by Peter Vary)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/0ff4d5b8
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/0ff4d5b8
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/0ff4d5b8
Branch: refs/heads/branch-2.3
Commit: 0ff4d5b8e28d142a3fe7b58126eb4c1a6a4d4bde
Parents: 24dcccf
Author: Vihang Karajgaonkar <vi...@cloudera.com>
Authored: Thu Jul 27 10:38:35 2017 -0700
Committer: Sahil Takiar <st...@cloudera.com>
Committed: Tue Nov 7 08:15:47 2017 -0800
----------------------------------------------------------------------
beeline/src/java/org/apache/hive/beeline/BeeLine.java | 1 -
1 file changed, 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/0ff4d5b8/beeline/src/java/org/apache/hive/beeline/BeeLine.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/BeeLine.java b/beeline/src/java/org/apache/hive/beeline/BeeLine.java
index 11526a7..0696345 100644
--- a/beeline/src/java/org/apache/hive/beeline/BeeLine.java
+++ b/beeline/src/java/org/apache/hive/beeline/BeeLine.java
@@ -1158,7 +1158,6 @@ public class BeeLine implements Closeable {
return ERRNO_OTHER;
} finally {
IOUtils.closeStream(fileStream);
- output(""); // dummy new line
}
}
[12/14] hive git commit: HIVE-16213: ObjectStore can leak Queries
when rollbackTransaction throws an exception (Vihang Karajgaonkar,
reviewed by Sergio Pena)
Posted by st...@apache.org.
HIVE-16213: ObjectStore can leak Queries when rollbackTransaction throws an exception (Vihang Karajgaonkar, reviewed by Sergio Pena)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/c2b5dba7
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/c2b5dba7
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/c2b5dba7
Branch: refs/heads/branch-2.3
Commit: c2b5dba78e2ad1c9884cd4a54f7113532614b605
Parents: 145ed20
Author: Vihang Karajgaonkar <vi...@cloudera.com>
Authored: Tue May 2 10:27:51 2017 -0500
Committer: Sahil Takiar <st...@cloudera.com>
Committed: Tue Nov 7 08:15:48 2017 -0800
----------------------------------------------------------------------
.../hadoop/hive/metastore/ObjectStore.java | 549 ++++---------------
.../hadoop/hive/metastore/TestObjectStore.java | 14 +
2 files changed, 131 insertions(+), 432 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/c2b5dba7/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index 358cf17..e6a918b 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -234,26 +234,22 @@ public class ObjectStore implements RawStore, Configurable {
private Pattern partitionValidationPattern;
/**
- * A class to pass the Query object to the caller to let the caller release
- * resources by calling QueryWrapper.query.closeAll() after consuming all the query results.
+ * A Autocloseable wrapper around Query class to pass the Query object to the caller and let the caller release
+ * the resources when the QueryWrapper goes out of scope
*/
- public static class QueryWrapper {
+ public static class QueryWrapper implements AutoCloseable {
public Query query;
/**
* Explicitly closes the query object to release the resources
*/
+ @Override
public void close() {
if (query != null) {
query.closeAll();
query = null;
}
}
-
- @Override
- protected void finalize() {
- this.close();
- }
}
public ObjectStore() {
@@ -700,12 +696,7 @@ public class ObjectStore implements RawStore, Configurable {
pm.retrieve(mdb);
commited = commitTransaction();
} finally {
- if (!commited) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(commited, query);
}
if (mdb == null) {
throw new NoSuchObjectException("There is no database named " + name);
@@ -824,10 +815,7 @@ public class ObjectStore implements RawStore, Configurable {
}
success = commitTransaction();
} finally {
- if (!success) {
- rollbackTransaction();
- }
- queryWrapper.close();
+ rollbackAndCleanup(success, queryWrapper);
}
return success;
}
@@ -858,12 +846,7 @@ public class ObjectStore implements RawStore, Configurable {
}
commited = commitTransaction();
} finally {
- if (!commited) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(commited, query);
}
return databases;
}
@@ -883,12 +866,7 @@ public class ObjectStore implements RawStore, Configurable {
databases = new ArrayList<String>((Collection<String>) query.execute());
commited = commitTransaction();
} finally {
- if (!commited) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(commited, query);
}
Collections.sort(databases);
return databases;
@@ -956,12 +934,7 @@ public class ObjectStore implements RawStore, Configurable {
}
commited = commitTransaction();
} finally {
- if (!commited) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(commited, query);
}
return type;
}
@@ -985,12 +958,7 @@ public class ObjectStore implements RawStore, Configurable {
success = commitTransaction();
LOG.debug("type not found " + typeName, e);
} finally {
- if (!success) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(success, query);
}
return success;
}
@@ -1231,12 +1199,7 @@ public class ObjectStore implements RawStore, Configurable {
}
commited = commitTransaction();
} finally {
- if (!commited) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(commited, query);
}
return tbls;
}
@@ -1268,12 +1231,7 @@ public class ObjectStore implements RawStore, Configurable {
result = (Long) query.execute();
commited = commitTransaction();
} finally {
- if (!commited) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(commited, query);
}
return result.intValue();
}
@@ -1311,12 +1269,7 @@ public class ObjectStore implements RawStore, Configurable {
}
commited = commitTransaction();
} finally {
- if (!commited) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(commited, query);
}
return metas;
}
@@ -1402,12 +1355,7 @@ public class ObjectStore implements RawStore, Configurable {
}
commited = commitTransaction();
} finally {
- if (!commited) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(commited, query);
}
nmtbl.mtbl = mtbl;
return nmtbl;
@@ -1450,15 +1398,10 @@ public class ObjectStore implements RawStore, Configurable {
}
committed = commitTransaction();
} finally {
- if (!committed) {
- rollbackTransaction();
- }
+ rollbackAndCleanup(committed, query);
if (dbExistsQuery != null) {
dbExistsQuery.closeAll();
}
- if (query != null) {
- query.closeAll();
- }
}
return tables;
}
@@ -1976,12 +1919,7 @@ public class ObjectStore implements RawStore, Configurable {
}
}
} finally {
- if (!commited) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(commited, query);
}
return ret;
}
@@ -2213,10 +2151,7 @@ public class ObjectStore implements RawStore, Configurable {
success = commitTransaction();
return parts;
} finally {
- if (!success) {
- rollbackTransaction();
- }
- queryWrapper.close();
+ rollbackAndCleanup(success, queryWrapper);
}
}
@@ -2318,6 +2253,7 @@ public class ObjectStore implements RawStore, Configurable {
for (Iterator i = names.iterator(); i.hasNext();) {
pns.add((String) i.next());
}
+
if (query != null) {
query.closeAll();
}
@@ -2412,10 +2348,7 @@ public class ObjectStore implements RawStore, Configurable {
}
success = commitTransaction();
} finally {
- if (!success) {
- rollbackTransaction();
- }
- queryWrapper.close();
+ rollbackAndCleanup(success, queryWrapper);
}
return partitions;
}
@@ -2437,10 +2370,7 @@ public class ObjectStore implements RawStore, Configurable {
}
success = commitTransaction();
} finally {
- if (!success) {
- rollbackTransaction();
- }
- queryWrapper.close();
+ rollbackAndCleanup(success, queryWrapper);
}
return partitionNames;
}
@@ -3206,12 +3136,7 @@ public class ObjectStore implements RawStore, Configurable {
success = commitTransaction();
LOG.debug("Done retrieving all objects for listTableNamesByFilter");
} finally {
- if (!success) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(success, query);
}
return tableNames;
}
@@ -3257,12 +3182,7 @@ public class ObjectStore implements RawStore, Configurable {
success = commitTransaction();
LOG.debug("Done retrieving all objects for listMPartitionNamesByFilter");
} finally {
- if (!success) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(success, query);
}
return partNames;
}
@@ -3481,10 +3401,7 @@ public class ObjectStore implements RawStore, Configurable {
success = commitTransaction();
LOG.debug("successfully deleted a CD in removeUnusedColumnDescriptor");
} finally {
- if (!success) {
- rollbackTransaction();
- }
- queryWrapper.close();
+ rollbackAndCleanup(success, queryWrapper);
}
}
@@ -3568,12 +3485,7 @@ public class ObjectStore implements RawStore, Configurable {
constraintNameIfExists = (String) constraintExistsQuery.execute(name);
commited = commitTransaction();
} finally {
- if (!commited) {
- rollbackTransaction();
- }
- if (constraintExistsQuery != null) {
- constraintExistsQuery.closeAll();
- }
+ rollbackAndCleanup(commited, constraintExistsQuery);
}
return constraintNameIfExists != null && !constraintNameIfExists.isEmpty();
}
@@ -3821,12 +3733,7 @@ public class ObjectStore implements RawStore, Configurable {
pm.retrieve(midx);
commited = commitTransaction();
} finally {
- if (!commited) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(commited, query);
}
return midx;
}
@@ -3889,12 +3796,7 @@ public class ObjectStore implements RawStore, Configurable {
return indexes;
} finally {
- if (!success) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(success, query);
}
}
@@ -3921,12 +3823,7 @@ public class ObjectStore implements RawStore, Configurable {
}
success = commitTransaction();
} finally {
- if (!success) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(success, query);
}
return pns;
}
@@ -4049,12 +3946,7 @@ public class ObjectStore implements RawStore, Configurable {
pm.retrieve(mRoleMember);
commited = commitTransaction();
} finally {
- if (!commited) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(commited, query);
}
return mRoleMember;
}
@@ -4123,11 +4015,7 @@ public class ObjectStore implements RawStore, Configurable {
}
success = commitTransaction();
} finally {
- if (!success) {
- rollbackTransaction();
- }
-
- queryWrapper.close();
+ rollbackAndCleanup(success, queryWrapper);
}
return success;
}
@@ -4197,12 +4085,7 @@ public class ObjectStore implements RawStore, Configurable {
LOG.debug("Done retrieving all objects for listRoles");
} finally {
- if (!success) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(success, query);
}
if (principalType == PrincipalType.USER) {
@@ -4268,7 +4151,6 @@ public class ObjectStore implements RawStore, Configurable {
mRoleMemebership = (List<MRoleMap>) query.execute(roleName, principalType.toString());
pm.retrieveAll(mRoleMemebership);
success = commitTransaction();
-
LOG.debug("Done retrieving all objects for listMSecurityPrincipalMembershipRole");
} finally {
if (!success) {
@@ -4302,12 +4184,7 @@ public class ObjectStore implements RawStore, Configurable {
pm.retrieve(mrole);
commited = commitTransaction();
} finally {
- if (!commited) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(commited, query);
}
return mrole;
}
@@ -4329,12 +4206,7 @@ public class ObjectStore implements RawStore, Configurable {
success = commitTransaction();
return roleNames;
} finally {
- if (!success) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(success, query);
}
}
@@ -5160,12 +5032,7 @@ public class ObjectStore implements RawStore, Configurable {
LOG.debug("Done retrieving all objects for listRoleMembers");
} finally {
- if (!success) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(success, query);
}
return mRoleMemeberList;
}
@@ -5216,12 +5083,7 @@ public class ObjectStore implements RawStore, Configurable {
userNameDbPriv.addAll(mPrivs);
}
} finally {
- if (!commited) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(commited, query);
}
return userNameDbPriv;
}
@@ -5261,12 +5123,7 @@ public class ObjectStore implements RawStore, Configurable {
commited = commitTransaction();
return convertGlobal(userNameDbPriv);
} finally {
- if (!commited) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(commited, query);
}
}
@@ -5309,12 +5166,7 @@ public class ObjectStore implements RawStore, Configurable {
mSecurityDBList.addAll(mPrivs);
LOG.debug("Done retrieving all objects for listPrincipalDBGrants");
} finally {
- if (!success) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(success, query);
}
return mSecurityDBList;
}
@@ -5437,12 +5289,7 @@ public class ObjectStore implements RawStore, Configurable {
LOG.debug("Done retrieving all objects for listAllTableGrants");
} finally {
- if (!success) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(success, query);
}
return mSecurityTabList;
}
@@ -5469,12 +5316,7 @@ public class ObjectStore implements RawStore, Configurable {
LOG.debug("Done retrieving all objects for listTableAllPartitionGrants");
} finally {
- if (!success) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(success, query);
}
return mSecurityTabPartList;
}
@@ -5502,12 +5344,7 @@ public class ObjectStore implements RawStore, Configurable {
LOG.debug("Done retrieving all objects for listTableAllColumnGrants");
} finally {
- if (!success) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(success, query);
}
return mTblColPrivilegeList;
}
@@ -5536,12 +5373,7 @@ public class ObjectStore implements RawStore, Configurable {
LOG.debug("Done retrieving all objects for listTableAllPartitionColumnGrants");
} finally {
- if (!success) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(success, query);
}
return mSecurityColList;
}
@@ -5584,7 +5416,6 @@ public class ObjectStore implements RawStore, Configurable {
private List<MDBPrivilege> listDatabaseGrants(String dbName, QueryWrapper queryWrapper) {
dbName = HiveStringUtils.normalizeIdentifier(dbName);
boolean success = false;
-
try {
LOG.debug("Executing listDatabaseGrants");
@@ -5692,12 +5523,7 @@ public class ObjectStore implements RawStore, Configurable {
LOG.debug("Done retrieving all objects for listAllTableGrants");
} finally {
- if (!success) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(success, query);
}
return mSecurityTabPartList;
}
@@ -5757,12 +5583,7 @@ public class ObjectStore implements RawStore, Configurable {
LOG.debug("Done retrieving all objects for listPrincipalPartitionGrants");
} finally {
- if (!success) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(success, query);
}
return mSecurityTabPartList;
}
@@ -5826,12 +5647,7 @@ public class ObjectStore implements RawStore, Configurable {
LOG.debug("Done retrieving all objects for listPrincipalTableColumnGrants");
} finally {
- if (!success) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(success, query);
}
return mSecurityColList;
}
@@ -5893,12 +5709,7 @@ public class ObjectStore implements RawStore, Configurable {
LOG.debug("Done retrieving all objects for listPrincipalPartitionColumnGrants");
} finally {
- if (!success) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(success, query);
}
return mSecurityColList;
}
@@ -5960,12 +5771,7 @@ public class ObjectStore implements RawStore, Configurable {
LOG.debug("Done retrieving all objects for listPrincipalPartitionColumnGrantsAll");
return result;
} finally {
- if (!success) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(success, query);
}
}
@@ -5993,12 +5799,7 @@ public class ObjectStore implements RawStore, Configurable {
LOG.debug("Done retrieving all objects for listPartitionColumnGrantsAll");
return result;
} finally {
- if (!success) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(success, query);
}
}
@@ -6073,12 +5874,7 @@ public class ObjectStore implements RawStore, Configurable {
LOG.debug("Done retrieving all objects for listPrincipalAllTableGrants");
return result;
} finally {
- if (!success) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(success, query);
}
}
@@ -6101,12 +5897,7 @@ public class ObjectStore implements RawStore, Configurable {
LOG.debug("Done retrieving all objects for listPrincipalAllTableGrants");
return result;
} finally {
- if (!success) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(success, query);
}
}
@@ -6146,7 +5937,7 @@ public class ObjectStore implements RawStore, Configurable {
LOG.debug("Done retrieving all objects for listPrincipalAllPartitionGrants");
} finally {
if (!success) {
- rollbackTransaction();
+ rollbackTransaction();
}
}
return mSecurityTabPartList;
@@ -6178,12 +5969,7 @@ public class ObjectStore implements RawStore, Configurable {
LOG.debug("Done retrieving all objects for listPrincipalPartitionGrantsAll");
return result;
} finally {
- if (!success) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(success, query);
}
}
@@ -6209,12 +5995,7 @@ public class ObjectStore implements RawStore, Configurable {
LOG.debug("Done retrieving all objects for listPrincipalPartitionGrantsAll");
return result;
} finally {
- if (!success) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(success, query);
}
}
@@ -6292,12 +6073,7 @@ public class ObjectStore implements RawStore, Configurable {
LOG.debug("Done retrieving all objects for listPrincipalTableColumnGrantsAll");
return result;
} finally {
- if (!success) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(success, query);
}
}
@@ -6322,12 +6098,7 @@ public class ObjectStore implements RawStore, Configurable {
LOG.debug("Done retrieving all objects for listPrincipalTableColumnGrantsAll");
return result;
} finally {
- if (!success) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(success, query);
}
}
@@ -6404,12 +6175,7 @@ public class ObjectStore implements RawStore, Configurable {
LOG.debug("Done executing isPartitionMarkedForEvent");
return (partEvents != null && !partEvents.isEmpty()) ? true : false;
} finally {
- if (!success) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(success, query);
}
}
@@ -6463,7 +6229,6 @@ public class ObjectStore implements RawStore, Configurable {
public Collection<?> executeJDOQLSelect(String queryStr, QueryWrapper queryWrapper) {
boolean committed = false;
Collection<?> result = null;
-
try {
openTransaction();
Query query = queryWrapper.query = pm.newQuery(queryStr);
@@ -6504,12 +6269,7 @@ public class ObjectStore implements RawStore, Configurable {
return -1;
}
} finally {
- if (!committed) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(committed, query);
}
}
@@ -6539,12 +6299,7 @@ public class ObjectStore implements RawStore, Configurable {
return null;
}
} finally {
- if (!committed) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(committed, query);
}
}
@@ -6655,12 +6410,7 @@ public class ObjectStore implements RawStore, Configurable {
}
return retVal;
} finally {
- if (!committed) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(committed, query);
}
}
@@ -6748,12 +6498,7 @@ public class ObjectStore implements RawStore, Configurable {
}
return retVal;
} finally {
- if (!committed) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(committed, query);
}
}
@@ -6787,12 +6532,7 @@ public class ObjectStore implements RawStore, Configurable {
}
return retVal;
} finally {
- if (!committed) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(committed, query);
}
}
@@ -6883,12 +6623,7 @@ public class ObjectStore implements RawStore, Configurable {
}
return retVal;
} finally {
- if (!committed) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(committed, query);
}
}
@@ -6965,12 +6700,7 @@ public class ObjectStore implements RawStore, Configurable {
}
return retVal;
} finally {
- if (!committed) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(committed, query);
}
}
@@ -7181,7 +6911,6 @@ public class ObjectStore implements RawStore, Configurable {
}
boolean committed = false;
-
try {
openTransaction();
@@ -7506,12 +7235,7 @@ public class ObjectStore implements RawStore, Configurable {
rollbackTransaction();
throw e;
} finally {
- if (!ret) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(ret, query);
}
return ret;
}
@@ -7581,12 +7305,7 @@ public class ObjectStore implements RawStore, Configurable {
rollbackTransaction();
throw e;
} finally {
- if (!ret) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(ret, query);
}
return ret;
}
@@ -7608,12 +7327,7 @@ public class ObjectStore implements RawStore, Configurable {
delCnt = query.deletePersistentAll(curTime, expiryTime);
commited = commitTransaction();
} finally {
- if (!commited) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(commited, query);
LOG.debug("Done executing cleanupEvents");
}
return delCnt;
@@ -7717,12 +7431,7 @@ public class ObjectStore implements RawStore, Configurable {
return tokenIdents;
} finally {
LOG.debug("Done executing getAllTokenIdentifers with status : " + committed);
- if (!committed) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(committed, query);
}
}
@@ -7765,12 +7474,7 @@ public class ObjectStore implements RawStore, Configurable {
}
committed = commitTransaction();
} finally {
- if (!committed) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(committed, query);
}
LOG.debug("Done executing updateMasterKey with status : " + committed);
if (null == masterKey) {
@@ -7798,12 +7502,7 @@ public class ObjectStore implements RawStore, Configurable {
}
success = commitTransaction();
} finally {
- if (!success) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(success, query);
}
LOG.debug("Done executing removeMasterKey with status : " + success);
return (null != masterKey) && success;
@@ -7829,12 +7528,7 @@ public class ObjectStore implements RawStore, Configurable {
return masterKeys;
} finally {
LOG.debug("Done executing getMasterKeys with status : " + committed);
- if (!committed) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(committed, query);
}
}
@@ -7946,12 +7640,7 @@ public class ObjectStore implements RawStore, Configurable {
}
return mVerTables.get(0);
} finally {
- if (!committed) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(committed, query);
}
}
@@ -8177,12 +7866,7 @@ public class ObjectStore implements RawStore, Configurable {
pm.retrieve(mfunc);
commited = commitTransaction();
} finally {
- if (!commited) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(commited, query);
}
return mfunc;
}
@@ -8246,12 +7930,7 @@ public class ObjectStore implements RawStore, Configurable {
}
commited = commitTransaction();
} finally {
- if (!commited) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(commited, query);
}
return funcs;
}
@@ -8282,11 +7961,8 @@ public class ObjectStore implements RawStore, Configurable {
}
return result;
} finally {
- if (query != null) {
- query.closeAll();
- }
if (!commited) {
- rollbackTransaction();
+ rollbackAndCleanup(commited, query);
return null;
}
}
@@ -8316,12 +7992,7 @@ public class ObjectStore implements RawStore, Configurable {
pm.makePersistent(translateThriftToDb(entry));
commited = commitTransaction();
} finally {
- if (!commited) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(commited, query);
}
}
@@ -8341,12 +8012,7 @@ public class ObjectStore implements RawStore, Configurable {
}
commited = commitTransaction();
} finally {
- if (!commited) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(commited, query);
}
}
@@ -8365,12 +8031,7 @@ public class ObjectStore implements RawStore, Configurable {
commited = commitTransaction();
return new CurrentNotificationEventId(id);
} finally {
- if (!commited) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(commited, query);
}
}
@@ -8587,12 +8248,7 @@ public class ObjectStore implements RawStore, Configurable {
}
commited = commitTransaction();
} finally {
- if (!commited) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(commited, query);
}
return primaryKeys;
}
@@ -8617,12 +8273,7 @@ public class ObjectStore implements RawStore, Configurable {
}
commited = commitTransaction();
} finally {
- if (!commited) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(commited, query);
}
return ret;
}
@@ -8741,12 +8392,7 @@ public class ObjectStore implements RawStore, Configurable {
}
commited = commitTransaction();
} finally {
- if (!commited) {
- rollbackTransaction();
- }
- if (query != null) {
- query.closeAll();
- }
+ rollbackAndCleanup(commited, query);
}
return foreignKeys;
}
@@ -8774,4 +8420,43 @@ public class ObjectStore implements RawStore, Configurable {
}
}
+ /**
+ * This is a cleanup method which is used to rollback a active transaction
+ * if the success flag is false and close the associated Query object. This method is used
+ * internally and visible for testing purposes only
+ * @param success Rollback the current active transaction if false
+ * @param query Query object which needs to be closed
+ */
+ @VisibleForTesting
+ void rollbackAndCleanup(boolean success, Query query) {
+ try {
+ if(!success) {
+ rollbackTransaction();
+ }
+ } finally {
+ if (query != null) {
+ query.closeAll();
+ }
+ }
+ }
+
+ /**
+ * This is a cleanup method which is used to rollback a active transaction
+ * if the success flag is false and close the associated QueryWrapper object. This method is used
+ * internally and visible for testing purposes only
+ * @param success Rollback the current active transaction if false
+ * @param queryWrapper QueryWrapper object which needs to be closed
+ */
+ @VisibleForTesting
+ void rollbackAndCleanup(boolean success, QueryWrapper queryWrapper) {
+ try {
+ if(!success) {
+ rollbackTransaction();
+ }
+ } finally {
+ if (queryWrapper != null) {
+ queryWrapper.close();
+ }
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/hive/blob/c2b5dba7/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java b/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
index 6524ee7..69e8826 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/TestObjectStore.java
@@ -59,9 +59,12 @@ import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
+import org.mockito.Mockito;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import javax.jdo.Query;
+
public class TestObjectStore {
private ObjectStore objectStore = null;
@@ -410,4 +413,15 @@ public class TestObjectStore {
} catch (NoSuchObjectException e) {
}
}
+
+ @Test
+ public void testQueryCloseOnError() throws Exception {
+ ObjectStore spy = Mockito.spy(objectStore);
+ spy.getAllDatabases();
+ spy.getAllFunctions();
+ spy.getAllTables(DB1);
+ spy.getPartitionCount();
+ Mockito.verify(spy, Mockito.times(3))
+ .rollbackAndCleanup(Mockito.anyBoolean(), Mockito.<Query>anyObject());
+ }
}
[06/14] hive git commit: HIVE-16991: HiveMetaStoreClient needs a
2-arg constructor for backwards compatibility (Andrew Sherman,
reviewed by Sergio Pena)
Posted by st...@apache.org.
HIVE-16991: HiveMetaStoreClient needs a 2-arg constructor for backwards compatibility (Andrew Sherman, reviewed by Sergio Pena)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/aef5ebb6
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/aef5ebb6
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/aef5ebb6
Branch: refs/heads/branch-2.3
Commit: aef5ebb678d37751546afbf6bca32d67e2f27dad
Parents: aaa440f
Author: Andrew Sherman <as...@cloudera.com>
Authored: Fri Jun 30 14:19:41 2017 -0500
Committer: Sahil Takiar <st...@cloudera.com>
Committed: Tue Nov 7 08:15:47 2017 -0800
----------------------------------------------------------------------
.../org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java | 4 ++++
1 file changed, 4 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/aef5ebb6/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index 7002620..c5db77f 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -128,6 +128,10 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
this(conf, null, true);
}
+ public HiveMetaStoreClient(HiveConf conf, HiveMetaHookLoader hookLoader) throws MetaException {
+ this(conf, hookLoader, true);
+ }
+
public HiveMetaStoreClient(HiveConf conf, HiveMetaHookLoader hookLoader, Boolean allowEmbedded)
throws MetaException {
[03/14] hive git commit: HIVE-17472: Drop-partition for multi-level
partition fails, if data does not exist. (Chris Drome,
reviewed by Mithun Radhakrishnan)
Posted by st...@apache.org.
HIVE-17472: Drop-partition for multi-level partition fails, if data does not exist. (Chris Drome, reviewed by Mithun Radhakrishnan)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/f3dc6a97
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/f3dc6a97
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/f3dc6a97
Branch: refs/heads/branch-2.3
Commit: f3dc6a9774912257bb9ecdc9fe19ee32698fded0
Parents: b8bd459
Author: Mithun RK <mi...@apache.org>
Authored: Wed Sep 6 17:25:04 2017 -0700
Committer: Sahil Takiar <st...@cloudera.com>
Committed: Tue Nov 7 08:15:47 2017 -0800
----------------------------------------------------------------------
.../hadoop/hive/metastore/HiveMetaStore.java | 6 +-
.../clientpositive/drop_deleted_partitions.q | 18 +++++
.../drop_deleted_partitions.q.out | 74 ++++++++++++++++++++
3 files changed, 96 insertions(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/f3dc6a97/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index 6f2b727..616314c 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -3158,8 +3158,10 @@ public class HiveMetaStore extends ThriftHiveMetastore {
}
private void deleteParentRecursive(Path parent, int depth, boolean mustPurge) throws IOException, MetaException {
- if (depth > 0 && parent != null && wh.isWritable(parent) && wh.isEmpty(parent)) {
- wh.deleteDir(parent, true, mustPurge);
+ if (depth > 0 && parent != null && wh.isWritable(parent)) {
+ if (wh.isDir(parent) && wh.isEmpty(parent)) {
+ wh.deleteDir(parent, true, mustPurge);
+ }
deleteParentRecursive(parent.getParent(), depth - 1, mustPurge);
}
}
http://git-wip-us.apache.org/repos/asf/hive/blob/f3dc6a97/ql/src/test/queries/clientpositive/drop_deleted_partitions.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/drop_deleted_partitions.q b/ql/src/test/queries/clientpositive/drop_deleted_partitions.q
new file mode 100644
index 0000000..a758b1b
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/drop_deleted_partitions.q
@@ -0,0 +1,18 @@
+create database dmp;
+
+create table dmp.mp (a string) partitioned by (b string, c string) location '/tmp/dmp_mp';
+
+alter table dmp.mp add partition (b='1', c='1');
+
+show partitions dmp.mp;
+
+dfs -rm -R /tmp/dmp_mp/b=1;
+
+explain extended alter table dmp.mp drop partition (b='1');
+alter table dmp.mp drop partition (b='1');
+
+show partitions dmp.mp;
+
+drop table dmp.mp;
+
+drop database dmp;
http://git-wip-us.apache.org/repos/asf/hive/blob/f3dc6a97/ql/src/test/results/clientpositive/drop_deleted_partitions.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/drop_deleted_partitions.q.out b/ql/src/test/results/clientpositive/drop_deleted_partitions.q.out
new file mode 100644
index 0000000..e543158
--- /dev/null
+++ b/ql/src/test/results/clientpositive/drop_deleted_partitions.q.out
@@ -0,0 +1,74 @@
+PREHOOK: query: create database dmp
+PREHOOK: type: CREATEDATABASE
+PREHOOK: Output: database:dmp
+POSTHOOK: query: create database dmp
+POSTHOOK: type: CREATEDATABASE
+POSTHOOK: Output: database:dmp
+#### A masked pattern was here ####
+PREHOOK: type: CREATETABLE
+#### A masked pattern was here ####
+PREHOOK: Output: database:dmp
+PREHOOK: Output: dmp@mp
+#### A masked pattern was here ####
+POSTHOOK: type: CREATETABLE
+#### A masked pattern was here ####
+POSTHOOK: Output: database:dmp
+POSTHOOK: Output: dmp@mp
+PREHOOK: query: alter table dmp.mp add partition (b='1', c='1')
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Output: dmp@mp
+POSTHOOK: query: alter table dmp.mp add partition (b='1', c='1')
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: dmp@mp
+POSTHOOK: Output: dmp@mp@b=1/c=1
+PREHOOK: query: show partitions dmp.mp
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: dmp@mp
+POSTHOOK: query: show partitions dmp.mp
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: dmp@mp
+b=1/c=1
+#### A masked pattern was here ####
+PREHOOK: query: explain extended alter table dmp.mp drop partition (b='1')
+PREHOOK: type: ALTERTABLE_DROPPARTS
+POSTHOOK: query: explain extended alter table dmp.mp drop partition (b='1')
+POSTHOOK: type: ALTERTABLE_DROPPARTS
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Drop Table Operator:
+ Drop Table
+ table: dmp.mp
+
+PREHOOK: query: alter table dmp.mp drop partition (b='1')
+PREHOOK: type: ALTERTABLE_DROPPARTS
+PREHOOK: Input: dmp@mp
+PREHOOK: Output: dmp@mp@b=1/c=1
+POSTHOOK: query: alter table dmp.mp drop partition (b='1')
+POSTHOOK: type: ALTERTABLE_DROPPARTS
+POSTHOOK: Input: dmp@mp
+POSTHOOK: Output: dmp@mp@b=1/c=1
+PREHOOK: query: show partitions dmp.mp
+PREHOOK: type: SHOWPARTITIONS
+PREHOOK: Input: dmp@mp
+POSTHOOK: query: show partitions dmp.mp
+POSTHOOK: type: SHOWPARTITIONS
+POSTHOOK: Input: dmp@mp
+PREHOOK: query: drop table dmp.mp
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: dmp@mp
+PREHOOK: Output: dmp@mp
+POSTHOOK: query: drop table dmp.mp
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: dmp@mp
+POSTHOOK: Output: dmp@mp
+PREHOOK: query: drop database dmp
+PREHOOK: type: DROPDATABASE
+PREHOOK: Input: database:dmp
+PREHOOK: Output: database:dmp
+POSTHOOK: query: drop database dmp
+POSTHOOK: type: DROPDATABASE
+POSTHOOK: Input: database:dmp
+POSTHOOK: Output: database:dmp
[07/14] hive git commit: HIVE-17150: CREATE INDEX execute HMS
out-of-transaction listener calls inside a transaction (Sergio Pena,
reviewed by Vihang Karajgaonkar)
Posted by st...@apache.org.
HIVE-17150: CREATE INDEX execute HMS out-of-transaction listener calls inside a transaction (Sergio Pena, reviewed by Vihang Karajgaonkar)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/cd39cf38
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/cd39cf38
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/cd39cf38
Branch: refs/heads/branch-2.3
Commit: cd39cf38aae32ac39cb1adf92966be32ab796a6a
Parents: aef5ebb
Author: Sergio Pena <se...@cloudera.com>
Authored: Mon Jul 24 16:52:49 2017 -0500
Committer: Sahil Takiar <st...@cloudera.com>
Committed: Tue Nov 7 08:15:47 2017 -0800
----------------------------------------------------------------------
.../listener/DbNotificationListener.java | 1 +
.../MetaStoreEventListenerConstants.java | 33 ----------------
.../listener/DummyRawStoreFailEvent.java | 5 +++
.../listener/TestDbNotificationListener.java | 1 +
.../hadoop/hive/metastore/HiveMetaStore.java | 40 ++++++++++---------
.../MetaStoreEventListenerConstants.java | 41 ++++++++++++++++++++
.../metastore/MetaStoreListenerNotifier.java | 15 ++++++-
.../apache/hadoop/hive/metastore/RawStore.java | 2 +
.../hadoop/hive/metastore/hbase/HBaseStore.java | 5 +++
.../DummyRawStoreControlledCommit.java | 5 +++
.../DummyRawStoreForJdoConnection.java | 5 +++
11 files changed, 100 insertions(+), 53 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/cd39cf38/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
index bbfbc36..f08b970 100644
--- a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
+++ b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
import org.apache.hadoop.hive.metastore.MetaStoreEventListener;
+import org.apache.hadoop.hive.metastore.MetaStoreEventListenerConstants;
import org.apache.hadoop.hive.metastore.RawStore;
import org.apache.hadoop.hive.metastore.RawStoreProxy;
import org.apache.hadoop.hive.metastore.ReplChangeManager;
http://git-wip-us.apache.org/repos/asf/hive/blob/cd39cf38/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/MetaStoreEventListenerConstants.java
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/MetaStoreEventListenerConstants.java b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/MetaStoreEventListenerConstants.java
deleted file mode 100644
index a4f2d59..0000000
--- a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/MetaStoreEventListenerConstants.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hive.hcatalog.listener;
-
-/**
- * Keeps a list of reserved keys used by Hive listeners when updating the ListenerEvent
- * parameters.
- */
-public class MetaStoreEventListenerConstants {
- /*
- * DbNotificationListener keys reserved for updating ListenerEvent parameters.
- *
- * DB_NOTIFICATION_EVENT_ID_KEY_NAME This key will have the event identifier that DbNotificationListener
- * processed during an event. This event identifier might be shared
- * across other MetaStoreEventListener implementations.
- */
- public static final String DB_NOTIFICATION_EVENT_ID_KEY_NAME = "DB_NOTIFICATION_EVENT_ID_KEY_NAME";
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hive/blob/cd39cf38/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
----------------------------------------------------------------------
diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
index 5282a5a..74d0efe 100644
--- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
+++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
@@ -93,6 +93,11 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
}
@Override
+ public boolean isActiveTransaction() {
+ return false;
+ }
+
+ @Override
public Configuration getConf() {
return objectStore.getConf();
}
http://git-wip-us.apache.org/repos/asf/hive/blob/cd39cf38/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
----------------------------------------------------------------------
diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
index 50d8878..976c3c5 100644
--- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
+++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
@@ -41,6 +41,7 @@ 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.MetaStoreEventListener;
+import org.apache.hadoop.hive.metastore.MetaStoreEventListenerConstants;
import org.apache.hadoop.hive.metastore.api.Database;
import org.apache.hadoop.hive.metastore.api.FieldSchema;
import org.apache.hadoop.hive.metastore.api.FireEventRequest;
http://git-wip-us.apache.org/repos/asf/hive/blob/cd39cf38/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index c4e45a1..6f2b727 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -904,7 +904,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
EventType.CREATE_DATABASE,
new CreateDatabaseEvent(db, success, this),
null,
- transactionalListenersResponses);
+ transactionalListenersResponses, ms);
}
}
}
@@ -1136,7 +1136,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
EventType.DROP_DATABASE,
new DropDatabaseEvent(db, success, this),
null,
- transactionalListenerResponses);
+ transactionalListenerResponses, ms);
}
}
}
@@ -1477,7 +1477,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
EventType.CREATE_TABLE,
new CreateTableEvent(tbl, success, this),
envContext,
- transactionalListenerResponses);
+ transactionalListenerResponses, ms);
}
}
}
@@ -1722,7 +1722,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
EventType.DROP_TABLE,
new DropTableEvent(tbl, deleteData, success, this),
envContext,
- transactionalListenerResponses);
+ transactionalListenerResponses, ms);
}
}
return success;
@@ -2266,7 +2266,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
EventType.ADD_PARTITION,
new AddPartitionEvent(tbl, part, success, this),
envContext,
- transactionalListenerResponses);
+ transactionalListenerResponses, ms);
}
}
return part;
@@ -2521,7 +2521,8 @@ public class HiveMetaStore extends ThriftHiveMetastore {
if (!listeners.isEmpty()) {
MetaStoreListenerNotifier.notifyEvent(listeners,
EventType.ADD_PARTITION,
- new AddPartitionEvent(tbl, parts, false, this));
+ new AddPartitionEvent(tbl, parts, false, this),
+ null, null, ms);
}
} else {
if (!listeners.isEmpty()) {
@@ -2529,13 +2530,14 @@ public class HiveMetaStore extends ThriftHiveMetastore {
EventType.ADD_PARTITION,
new AddPartitionEvent(tbl, newParts, true, this),
null,
- transactionalListenerResponses);
+ transactionalListenerResponses, ms);
if (!existingParts.isEmpty()) {
// The request has succeeded but we failed to add these partitions.
MetaStoreListenerNotifier.notifyEvent(listeners,
EventType.ADD_PARTITION,
- new AddPartitionEvent(tbl, existingParts, false, this));
+ new AddPartitionEvent(tbl, existingParts, false, this),
+ null, null, ms);
}
}
}
@@ -2723,7 +2725,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
EventType.ADD_PARTITION,
new AddPartitionEvent(tbl, partitionSpecProxy, true, this),
null,
- transactionalListenerResponses);
+ transactionalListenerResponses, ms);
}
}
}
@@ -2877,7 +2879,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
EventType.ADD_PARTITION,
new AddPartitionEvent(tbl, Arrays.asList(part), success, this),
envContext,
- transactionalListenerResponses);
+ transactionalListenerResponses, ms);
}
}
@@ -3031,7 +3033,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
EventType.ADD_PARTITION,
addPartitionEvent,
null,
- transactionalListenerResponsesForAddPartition);
+ transactionalListenerResponsesForAddPartition, ms);
i = 0;
for (Partition partition : partitionsToExchange) {
@@ -3046,7 +3048,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
EventType.DROP_PARTITION,
dropPartitionEvent,
null,
- parameters);
+ parameters, ms);
i++;
}
}
@@ -3137,7 +3139,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
EventType.DROP_PARTITION,
new DropPartitionEvent(tbl, part, success, deleteData, this),
envContext,
- transactionalListenerResponses);
+ transactionalListenerResponses, ms);
}
}
return true;
@@ -3334,7 +3336,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
EventType.DROP_PARTITION,
new DropPartitionEvent(tbl, part, success, deleteData, this),
envContext,
- parameters);
+ parameters, ms);
i++;
}
@@ -3926,7 +3928,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
EventType.ALTER_INDEX,
new AlterIndexEvent(oldIndex, newIndex, success, this),
null,
- transactionalListenerResponses);
+ transactionalListenerResponses, ms);
}
}
}
@@ -4629,7 +4631,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
EventType.CREATE_INDEX,
new AddIndexEvent(index, success, this),
null,
- transactionalListenerResponses);
+ transactionalListenerResponses, ms);
}
}
}
@@ -4722,7 +4724,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
EventType.DROP_INDEX,
new DropIndexEvent(index, success, this),
null,
- transactionalListenerResponses);
+ transactionalListenerResponses, ms);
}
}
return success;
@@ -6193,7 +6195,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
EventType.CREATE_FUNCTION,
new CreateFunctionEvent(func, success, this),
null,
- transactionalListenerResponses);
+ transactionalListenerResponses, ms);
}
}
}
@@ -6232,7 +6234,7 @@ public class HiveMetaStore extends ThriftHiveMetastore {
EventType.DROP_FUNCTION,
new DropFunctionEvent(func, success, this),
null,
- transactionalListenerResponses);
+ transactionalListenerResponses, ms);
}
}
}
http://git-wip-us.apache.org/repos/asf/hive/blob/cd39cf38/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreEventListenerConstants.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreEventListenerConstants.java b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreEventListenerConstants.java
new file mode 100644
index 0000000..79de79d
--- /dev/null
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreEventListenerConstants.java
@@ -0,0 +1,41 @@
+/**
+ * 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;
+
+/**
+ * Keeps a list of reserved keys used by Hive listeners when updating the ListenerEvent
+ * parameters.
+ */
+public class MetaStoreEventListenerConstants {
+ /*
+ * DbNotificationListener keys reserved for updating ListenerEvent parameters.
+ *
+ * DB_NOTIFICATION_EVENT_ID_KEY_NAME This key will have the event identifier that DbNotificationListener
+ * processed during an event. This event identifier might be shared
+ * across other MetaStoreEventListener implementations.
+ */
+ public static final String DB_NOTIFICATION_EVENT_ID_KEY_NAME = "DB_NOTIFICATION_EVENT_ID_KEY_NAME";
+
+ /*
+ * HiveMetaStore keys reserved for updating ListenerEvent parameters.
+ *
+ * HIVE_METASTORE_TRANSACTION_ACTIVE This key is used to check if a listener event is run inside a current
+ * transaction. A boolean value is used for active (true) or no active (false).
+ */
+ public static final String HIVE_METASTORE_TRANSACTION_ACTIVE = "HIVE_METASTORE_TRANSACTION_ACTIVE";
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hive/blob/cd39cf38/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java
index 20011cc..37327f8 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.hive.metastore.events.ListenerEvent;
import java.util.List;
import java.util.Map;
+import static org.apache.hadoop.hive.metastore.MetaStoreEventListenerConstants.HIVE_METASTORE_TRANSACTION_ACTIVE;
import static org.apache.hadoop.hive.metastore.messaging.EventMessage.EventType;
/**
@@ -201,11 +202,17 @@ public class MetaStoreListenerNotifier {
* the (ListenerEvent) event by setting a parameter key/value pair. These updated parameters will
* be returned to the caller.
*
+ * Sometimes these events are run inside a DB transaction and might cause issues with the listeners,
+ * for instance, Sentry blocks the HMS until an event is seen committed on the DB. To notify the listener about this,
+ * a new parameter to verify if a transaction is active is added to the ListenerEvent, and is up to the listener
+ * to skip this notification if so.
+ *
* @param listeners List of MetaStoreEventListener listeners.
* @param eventType Type of the notification event.
* @param event The ListenerEvent with information about the event.
* @param environmentContext An EnvironmentContext object with parameters sent by the HMS client.
* @param parameters A list of key/value pairs with the new parameters to add.
+ * @param ms The RawStore object from where to check if a transaction is active.
* @return A list of key/value pair parameters that the listeners set. The returned object will return an empty
* map if no parameters were updated or if no listeners were notified.
* @throws MetaException If an error occurred while calling the listeners.
@@ -214,11 +221,17 @@ public class MetaStoreListenerNotifier {
EventType eventType,
ListenerEvent event,
EnvironmentContext environmentContext,
- Map<String, String> parameters) throws MetaException {
+ Map<String, String> parameters,
+ final RawStore ms) throws MetaException {
Preconditions.checkNotNull(event, "The event must not be null.");
event.putParameters(parameters);
+
+ if (ms != null) {
+ event.putParameter(HIVE_METASTORE_TRANSACTION_ACTIVE, Boolean.toString(ms.isActiveTransaction()));
+ }
+
return notifyEvent(listeners, eventType, event, environmentContext);
}
}
http://git-wip-us.apache.org/repos/asf/hive/blob/cd39cf38/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
index 6f4f031..5b40835 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/RawStore.java
@@ -93,6 +93,8 @@ public interface RawStore extends Configurable {
@CanNotRetry
public abstract boolean commitTransaction();
+ public boolean isActiveTransaction();
+
/**
* Rolls back the current transaction if it is active
*/
http://git-wip-us.apache.org/repos/asf/hive/blob/cd39cf38/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
index 6593fa6..ecddb8a 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseStore.java
@@ -139,6 +139,11 @@ public class HBaseStore implements RawStore {
}
@Override
+ public boolean isActiveTransaction() {
+ return txnNestLevel != 0;
+ }
+
+ @Override
public void rollbackTransaction() {
txnNestLevel = 0;
LOG.debug("Rolling back HBase transaction");
http://git-wip-us.apache.org/repos/asf/hive/blob/cd39cf38/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
index f64b08d..275797e 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
@@ -95,6 +95,11 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
}
}
+ @Override
+ public boolean isActiveTransaction() {
+ return false;
+ }
+
// All remaining functions simply delegate to objectStore
@Override
http://git-wip-us.apache.org/repos/asf/hive/blob/cd39cf38/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
index 2682886..7f1784e 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
@@ -109,6 +109,11 @@ public class DummyRawStoreForJdoConnection implements RawStore {
}
@Override
+ public boolean isActiveTransaction() {
+ return false;
+ }
+
+ @Override
public void rollbackTransaction() {
[05/14] hive git commit: HIVE-16930: HoS should verify the value of
Kerberos principal and keytab file before adding them to spark-submit command
parameters (Yibing Shi via Chaoyu Tang)
Posted by st...@apache.org.
HIVE-16930: HoS should verify the value of Kerberos principal and keytab file before adding them to spark-submit command parameters (Yibing Shi via Chaoyu Tang)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/aaa440f3
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/aaa440f3
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/aaa440f3
Branch: refs/heads/branch-2.3
Commit: aaa440f35fb47648fa7b412280f01e939a83cb61
Parents: cf64bb9
Author: Chaoyu Tang <ct...@cloudera.com>
Authored: Thu Jun 22 16:53:55 2017 -0400
Committer: Sahil Takiar <st...@cloudera.com>
Committed: Tue Nov 7 08:15:47 2017 -0800
----------------------------------------------------------------------
.../hive/spark/client/SparkClientImpl.java | 35 +++++++++++---------
1 file changed, 19 insertions(+), 16 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/aaa440f3/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java
----------------------------------------------------------------------
diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java b/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java
index d4b63f0..5c42bcc 100644
--- a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java
+++ b/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java
@@ -54,6 +54,7 @@ import java.util.concurrent.Future;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.hive.conf.Constants;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
@@ -405,22 +406,24 @@ class SparkClientImpl implements SparkClient {
String principal = SecurityUtil.getServerPrincipal(hiveConf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL),
"0.0.0.0");
String keyTabFile = hiveConf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB);
- if (hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_SERVER2_ENABLE_DOAS)) {
- List<String> kinitArgv = Lists.newLinkedList();
- kinitArgv.add("kinit");
- kinitArgv.add(principal);
- kinitArgv.add("-k");
- kinitArgv.add("-t");
- kinitArgv.add(keyTabFile + ";");
- kinitArgv.addAll(argv);
- argv = kinitArgv;
- } else {
- // if doAs is not enabled, we pass the principal/keypad to spark-submit in order to
- // support the possible delegation token renewal in Spark
- argv.add("--principal");
- argv.add(principal);
- argv.add("--keytab");
- argv.add(keyTabFile);
+ if (StringUtils.isNotBlank(principal) && StringUtils.isNotBlank(keyTabFile)) {
+ if (hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_SERVER2_ENABLE_DOAS)) {
+ List<String> kinitArgv = Lists.newLinkedList();
+ kinitArgv.add("kinit");
+ kinitArgv.add(principal);
+ kinitArgv.add("-k");
+ kinitArgv.add("-t");
+ kinitArgv.add(keyTabFile + ";");
+ kinitArgv.addAll(argv);
+ argv = kinitArgv;
+ } else {
+ // if doAs is not enabled, we pass the principal/keypad to spark-submit in order to
+ // support the possible delegation token renewal in Spark
+ argv.add("--principal");
+ argv.add(principal);
+ argv.add("--keytab");
+ argv.add(keyTabFile);
+ }
}
}
if (hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_SERVER2_ENABLE_DOAS)) {