You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@atlas.apache.org by ve...@apache.org on 2015/05/13 23:28:18 UTC

[37/50] [abbrv] incubator-atlas git commit: hive hook fixes for hdp 2.2.4

hive hook fixes for hdp 2.2.4


Project: http://git-wip-us.apache.org/repos/asf/incubator-atlas/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-atlas/commit/64c78442
Tree: http://git-wip-us.apache.org/repos/asf/incubator-atlas/tree/64c78442
Diff: http://git-wip-us.apache.org/repos/asf/incubator-atlas/diff/64c78442

Branch: refs/remotes/origin/master
Commit: 64c784428b7c87aab5e6a8311c9121de48e9183e
Parents: 4b2a4ca
Author: Shwetha GS <ss...@hortonworks.com>
Authored: Mon May 4 18:04:38 2015 +0530
Committer: Shwetha GS <ss...@hortonworks.com>
Committed: Mon May 4 18:04:38 2015 +0530

----------------------------------------------------------------------
 addons/hive-bridge/pom.xml                      |  8 +---
 .../hive/bridge/HiveMetaStoreBridge.java        | 41 +++++++++++---------
 .../hadoop/metadata/hive/hook/HiveHook.java     |  5 ++-
 .../hadoop/metadata/hive/hook/HiveHookIT.java   |  2 +-
 4 files changed, 28 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/64c78442/addons/hive-bridge/pom.xml
----------------------------------------------------------------------
diff --git a/addons/hive-bridge/pom.xml b/addons/hive-bridge/pom.xml
index b908847..03d2dcf 100755
--- a/addons/hive-bridge/pom.xml
+++ b/addons/hive-bridge/pom.xml
@@ -90,15 +90,9 @@
 
         <dependency>
             <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-common</artifactId>
-            <version>${hadoop.version}</version>
-            <scope>provided</scope>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-client</artifactId>
             <version>${hadoop.version}</version>
+            <scope>provided</scope>
         </dependency>
 
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/64c78442/addons/hive-bridge/src/main/java/org/apache/hadoop/metadata/hive/bridge/HiveMetaStoreBridge.java
----------------------------------------------------------------------
diff --git a/addons/hive-bridge/src/main/java/org/apache/hadoop/metadata/hive/bridge/HiveMetaStoreBridge.java b/addons/hive-bridge/src/main/java/org/apache/hadoop/metadata/hive/bridge/HiveMetaStoreBridge.java
index ab14744..6084a68 100755
--- a/addons/hive-bridge/src/main/java/org/apache/hadoop/metadata/hive/bridge/HiveMetaStoreBridge.java
+++ b/addons/hive-bridge/src/main/java/org/apache/hadoop/metadata/hive/bridge/HiveMetaStoreBridge.java
@@ -19,15 +19,15 @@
 package org.apache.hadoop.metadata.hive.bridge;
 
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Index;
 import org.apache.hadoop.hive.metastore.api.Order;
-import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
-import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.Partition;
+import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.metadata.MetadataServiceClient;
 import org.apache.hadoop.metadata.hive.model.HiveDataModelGenerator;
 import org.apache.hadoop.metadata.hive.model.HiveDataTypes;
@@ -40,6 +40,7 @@ import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Set;
 
 /**
  * A Bridge Utility that imports metadata from the Hive Meta Store
@@ -66,7 +67,7 @@ public class HiveMetaStoreBridge {
 
     private static final Logger LOG = LoggerFactory.getLogger(HiveMetaStoreBridge.class);
 
-    private final HiveMetaStoreClient hiveMetaStoreClient;
+    private final Hive hiveClient;
     private final MetadataServiceClient metadataServiceClient;
 
     /**
@@ -74,7 +75,7 @@ public class HiveMetaStoreBridge {
      * @param hiveConf
      */
     public HiveMetaStoreBridge(HiveConf hiveConf) throws Exception {
-        hiveMetaStoreClient = new HiveMetaStoreClient(hiveConf);
+        hiveClient = Hive.get(hiveConf);
         metadataServiceClient = new MetadataServiceClient(hiveConf.get(DGI_URL_PROPERTY, DEFAULT_DGI_URL));
     }
 
@@ -88,7 +89,7 @@ public class HiveMetaStoreBridge {
     }
 
     private void importDatabases() throws Exception {
-        List<String> databases = hiveMetaStoreClient.getAllDatabases();
+        List<String> databases = hiveClient.getAllDatabases();
         for (String databaseName : databases) {
             Referenceable dbReference = registerDatabase(databaseName);
 
@@ -99,7 +100,7 @@ public class HiveMetaStoreBridge {
     public Referenceable registerDatabase(String databaseName) throws Exception {
         LOG.info("Importing objects from databaseName : " + databaseName);
 
-        Database hiveDB = hiveMetaStoreClient.getDatabase(databaseName);
+        Database hiveDB = hiveClient.getDatabase(databaseName);
 
         Referenceable dbRef = new Referenceable(HiveDataTypes.HIVE_DB.getName());
         dbRef.set("name", hiveDB.getName());
@@ -128,7 +129,7 @@ public class HiveMetaStoreBridge {
     }
 
     private void importTables(String databaseName, Referenceable databaseReferenceable) throws Exception {
-        List<String> hiveTables = hiveMetaStoreClient.getAllTables(databaseName);
+        List<String> hiveTables = hiveClient.getAllTables(databaseName);
 
         for (String tableName : hiveTables) {
             Pair<Referenceable, Referenceable> tableReferenceable = registerTable(databaseReferenceable, databaseName, tableName);
@@ -144,12 +145,13 @@ public class HiveMetaStoreBridge {
     public Pair<Referenceable, Referenceable> registerTable(Referenceable dbReference, String dbName, String tableName) throws Exception {
         LOG.info("Importing objects from " + dbName + "." + tableName);
 
-        Table hiveTable = hiveMetaStoreClient.getTable(dbName, tableName);
+        Table hiveTable = hiveClient.getTable(dbName, tableName);
 
         Referenceable tableRef = new Referenceable(HiveDataTypes.HIVE_TABLE.getName());
         tableRef.set("tableName", hiveTable.getTableName());
         tableRef.set("owner", hiveTable.getOwner());
-        tableRef.set("createTime", hiveTable.getCreateTime());
+        //todo fix
+        tableRef.set("createTime", hiveTable.getLastAccessTime());
         tableRef.set("lastAccessTime", hiveTable.getLastAccessTime());
         tableRef.set("retention", hiveTable.getRetention());
 
@@ -164,7 +166,7 @@ public class HiveMetaStoreBridge {
         // add reference to the Partition Keys
         List<Referenceable> partKeys = new ArrayList<>();
         Referenceable colRef;
-        if (hiveTable.getPartitionKeysSize() > 0) {
+        if (hiveTable.getPartitionKeys().size() > 0) {
             for (FieldSchema fs : hiveTable.getPartitionKeys()) {
                 colRef = new Referenceable(HiveDataTypes.HIVE_COLUMN.getName());
                 colRef.set("name", fs.getName());
@@ -179,11 +181,11 @@ public class HiveMetaStoreBridge {
 
         tableRef.set("parameters", hiveTable.getParameters());
 
-        if (hiveTable.isSetViewOriginalText()) {
+        if (hiveTable.getViewOriginalText() != null) {
             tableRef.set("viewOriginalText", hiveTable.getViewOriginalText());
         }
 
-        if (hiveTable.isSetViewExpandedText()) {
+        if (hiveTable.getViewExpandedText() != null) {
             tableRef.set("viewExpandedText", hiveTable.getViewExpandedText());
         }
 
@@ -197,12 +199,14 @@ public class HiveMetaStoreBridge {
         return Pair.of(tableReferenceable, sdReferenceable);
     }
 
-    private void importPartitions(String db, String table,
+    private void importPartitions(String db, String tableName,
                                   Referenceable dbReferenceable,
                                   Referenceable tableReferenceable,
                                   Referenceable sdReferenceable) throws Exception {
-        List<Partition> tableParts = hiveMetaStoreClient.listPartitions(
-                db, table, Short.MAX_VALUE);
+        Table table = new Table();
+        table.setDbName(db);
+        table.setTableName(tableName);
+        Set<Partition> tableParts = hiveClient.getAllPartitionsOf(table);
 
         if (tableParts.size() > 0) {
             for (Partition hivePart : tableParts) {
@@ -221,7 +225,8 @@ public class HiveMetaStoreBridge {
         partRef.set("dbName", dbReferenceable);
         partRef.set("tableName", tableReferenceable);
 
-        partRef.set("createTime", hivePart.getCreateTime());
+        //todo fix
+        partRef.set("createTime", hivePart.getLastAccessTime());
         partRef.set("lastAccessTime", hivePart.getLastAccessTime());
 
         // sdStruct = fillStorageDescStruct(hivePart.getSd());
@@ -237,7 +242,7 @@ public class HiveMetaStoreBridge {
     private void importIndexes(String db, String table,
                                Referenceable dbReferenceable,
                                Referenceable tableReferenceable) throws Exception {
-        List<Index> indexes = hiveMetaStoreClient.listIndexes(db, table, Short.MAX_VALUE);
+        List<Index> indexes = hiveClient.getIndexes(db, table, Short.MAX_VALUE);
         if (indexes.size() > 0) {
             for (Index index : indexes) {
                 importIndex(index, dbReferenceable, tableReferenceable);

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/64c78442/addons/hive-bridge/src/main/java/org/apache/hadoop/metadata/hive/hook/HiveHook.java
----------------------------------------------------------------------
diff --git a/addons/hive-bridge/src/main/java/org/apache/hadoop/metadata/hive/hook/HiveHook.java b/addons/hive-bridge/src/main/java/org/apache/hadoop/metadata/hive/hook/HiveHook.java
index abffe2e..4af7178 100755
--- a/addons/hive-bridge/src/main/java/org/apache/hadoop/metadata/hive/hook/HiveHook.java
+++ b/addons/hive-bridge/src/main/java/org/apache/hadoop/metadata/hive/hook/HiveHook.java
@@ -155,7 +155,7 @@ public class HiveHook implements ExecuteWithHookContext, HiveSemanticAnalyzerHoo
 
         // clone to avoid concurrent access
         final HiveConf conf = new HiveConf(hookContext.getConf());
-        boolean debug = conf.get("debug", "false").equals("true");
+        boolean debug = conf.get("hive.hook.dgi.synchronous", "false").equals("true");
 
         if (debug) {
             fireAndForget(hookContext, conf);
@@ -178,8 +178,9 @@ public class HiveHook implements ExecuteWithHookContext, HiveSemanticAnalyzerHoo
     private void fireAndForget(HookContext hookContext, HiveConf conf) throws Exception {
         assert hookContext.getHookType() == HookContext.HookType.POST_EXEC_HOOK : "Non-POST_EXEC_HOOK not supported!";
 
+        LOG.info("Entered DGI hook for hook type {} operation {}", hookContext.getHookType(),
+                hookContext.getOperationName());
         HiveOperation operation = HiveOperation.valueOf(hookContext.getOperationName());
-        LOG.info("Entered DGI hook for hook type {} operation {}", hookContext.getHookType(), operation);
 
         HiveMetaStoreBridge dgiBridge = new HiveMetaStoreBridge(conf);
 

http://git-wip-us.apache.org/repos/asf/incubator-atlas/blob/64c78442/addons/hive-bridge/src/test/java/org/apache/hadoop/metadata/hive/hook/HiveHookIT.java
----------------------------------------------------------------------
diff --git a/addons/hive-bridge/src/test/java/org/apache/hadoop/metadata/hive/hook/HiveHookIT.java b/addons/hive-bridge/src/test/java/org/apache/hadoop/metadata/hive/hook/HiveHookIT.java
index c47bdd5..7b6ba1b 100755
--- a/addons/hive-bridge/src/test/java/org/apache/hadoop/metadata/hive/hook/HiveHookIT.java
+++ b/addons/hive-bridge/src/test/java/org/apache/hadoop/metadata/hive/hook/HiveHookIT.java
@@ -58,7 +58,7 @@ public class HiveHookIT {
         hiveConf.setVar(HiveConf.ConfVars.METASTOREWAREHOUSE, System.getProperty("user.dir") + "/target/metastore");
         hiveConf.set(HiveMetaStoreBridge.DGI_URL_PROPERTY, DGI_URL);
         hiveConf.set("javax.jdo.option.ConnectionURL", "jdbc:derby:./target/metastore_db;create=true");
-        hiveConf.set("debug", "true");
+        hiveConf.set("hive.hook.dgi.synchronous", "true");
         return hiveConf;
     }