hive git commit: HIVE-18832: Support change management for trashing data files from ACID tables.(Anishek Agarwal, reviewed by Sankar Hariappan)

2018-03-12 Thread anishek
Repository: hive
Updated Branches:
  refs/heads/master d2cb97b6f -> 53df7e881


HIVE-18832: Support change management for trashing data files from ACID 
tables.(Anishek Agarwal, reviewed by Sankar Hariappan)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/53df7e88
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/53df7e88
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/53df7e88

Branch: refs/heads/master
Commit: 53df7e881723827a29a783fcbff67a16929950ec
Parents: d2cb97b
Author: Anishek Agarwal 
Authored: Mon Mar 12 11:55:42 2018 +0530
Committer: Anishek Agarwal 
Committed: Mon Mar 12 11:55:42 2018 +0530

--
 .../hadoop/hive/ql/parse/WarehouseInstance.java |   8 +-
 .../compactor/TestCleanerWithReplication.java   | 198 +++
 .../hadoop/hive/ql/txn/compactor/Cleaner.java   |  25 ++-
 .../hive/ql/txn/compactor/CompactorTest.java|   9 +-
 .../hive/ql/txn/compactor/TestCleaner.java  |   8 -
 .../hive/ql/txn/compactor/TestCleaner2.java |   3 -
 .../hive/ql/txn/compactor/TestInitiator.java|   8 -
 .../hive/ql/txn/compactor/TestWorker.java   |   4 -
 .../hive/ql/txn/compactor/TestWorker2.java  |   4 -
 .../hive/metastore/ReplChangeManager.java   | 163 ---
 .../apache/hadoop/hive/metastore/Warehouse.java |  20 +-
 11 files changed, 319 insertions(+), 131 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/53df7e88/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java
--
diff --git 
a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java
 
b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java
index 33e5157..feb1191 100644
--- 
a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java
+++ 
b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java
@@ -56,7 +56,7 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-class WarehouseInstance implements Closeable {
+public class WarehouseInstance implements Closeable {
   final String functionsRoot;
   private Logger logger;
   private IDriver driver;
@@ -85,7 +85,7 @@ class WarehouseInstance implements Closeable {
 initialize(cmRootPath.toString(), warehouseRoot.toString(), 
overridesForHiveConf);
   }
 
-  WarehouseInstance(Logger logger, MiniDFSCluster cluster,
+  public WarehouseInstance(Logger logger, MiniDFSCluster cluster,
   Map overridesForHiveConf) throws Exception {
 this(logger, cluster, overridesForHiveConf, null);
   }
@@ -165,7 +165,7 @@ class WarehouseInstance implements Closeable {
 return (lastResults.get(0).split("\\t"))[colNum];
   }
 
-  WarehouseInstance run(String command) throws Throwable {
+  public WarehouseInstance run(String command) throws Throwable {
 CommandProcessorResponse ret = driver.run(command);
 if (ret.getException() != null) {
   throw ret.getException();
@@ -257,7 +257,7 @@ class WarehouseInstance implements Closeable {
 return this;
   }
 
-  List getOutput() throws IOException {
+  public List getOutput() throws IOException {
 List results = new ArrayList<>();
 driver.getResults(results);
 return results;

http://git-wip-us.apache.org/repos/asf/hive/blob/53df7e88/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCleanerWithReplication.java
--
diff --git 
a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCleanerWithReplication.java
 
b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCleanerWithReplication.java
new file mode 100644
index 000..c0751a7
--- /dev/null
+++ 
b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCleanerWithReplication.java
@@ -0,0 +1,198 @@
+/*
+ * 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 

hive git commit: HIVE-16855: org.apache.hadoop.hive.ql.exec.mr.HashTableLoader Improvements (BELUGA BEHR, reviewed by Sahil Takiar)

2018-03-12 Thread stakiar
Repository: hive
Updated Branches:
  refs/heads/master 4cb24ce84 -> 0fe38792a


HIVE-16855: org.apache.hadoop.hive.ql.exec.mr.HashTableLoader Improvements 
(BELUGA BEHR, reviewed by Sahil Takiar)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/0fe38792
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/0fe38792
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/0fe38792

Branch: refs/heads/master
Commit: 0fe38792a0852dcfbf58438d3f28257aa332b07f
Parents: 4cb24ce
Author: BELUGA BEHR 
Authored: Mon Mar 12 12:29:00 2018 -0700
Committer: Sahil Takiar 
Committed: Mon Mar 12 12:29:36 2018 -0700

--
 .../hadoop/hive/ql/exec/mr/HashTableLoader.java  | 19 +--
 1 file changed, 9 insertions(+), 10 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/0fe38792/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HashTableLoader.java
--
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HashTableLoader.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HashTableLoader.java
index 3f3d8f2..3489843 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HashTableLoader.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HashTableLoader.java
@@ -26,6 +26,7 @@ import java.util.List;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.commons.collections.CollectionUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.filecache.DistributedCache;
 import org.apache.hadoop.fs.FileSystem;
@@ -53,7 +54,7 @@ import org.apache.hadoop.mapred.JobConf;
  */
 public class HashTableLoader implements 
org.apache.hadoop.hive.ql.exec.HashTableLoader {
 
-  private static final Logger LOG = 
LoggerFactory.getLogger(MapJoinOperator.class.getName());
+  private static final Logger LOG = 
LoggerFactory.getLogger(MapJoinOperator.class);
 
   private ExecMapperContext context;
   private Configuration hconf;
@@ -76,7 +77,7 @@ public class HashTableLoader implements 
org.apache.hadoop.hive.ql.exec.HashTable
   MapJoinTableContainerSerDe[] mapJoinTableSerdes) throws HiveException {
 
 String currentInputPath = context.getCurrentInputPath().toString();
-LOG.info("*** Load from HashTable for input file: " + 
currentInputPath);
+LOG.info("Load from HashTable for input file: {}", currentInputPath);
 MapredLocalWork localWork = context.getLocalWork();
 try {
   if (localWork.getDirectFetchOp() != null) {
@@ -92,9 +93,9 @@ public class HashTableLoader implements 
org.apache.hadoop.hive.ql.exec.HashTable
   continue;
 }
 Path path = Utilities.generatePath(baseDir, desc.getDumpFilePrefix(), 
(byte)pos, fileName);
-LOG.info("\tLoad back 1 hashtable file from tmp file uri:" + path);
+LOG.info("Load back 1 hashtable file from tmp file uri: {}", path);
 ObjectInputStream in = new ObjectInputStream(new BufferedInputStream(
-new FileInputStream(path.toUri().getPath()), 4096));
+new FileInputStream(path.toUri().getPath(;
 try{
   mapJoinTables[pos] = mapJoinTableSerdes[pos].load(in);
 } finally {
@@ -115,12 +116,10 @@ public class HashTableLoader implements 
org.apache.hadoop.hive.ql.exec.HashTable
   String stageID = localWork.getStageID();
   String suffix = Utilities.generateTarFileName(stageID);
   FileSystem localFs = FileSystem.getLocal(hconf);
-  for (int j = 0; j < localArchives.length; j++) {
-Path archive = localArchives[j];
-if (!archive.getName().endsWith(suffix)) {
-  continue;
+  for (Path archive : localArchives) {
+if (archive.getName().endsWith(suffix)) {
+  return archive.makeQualified(localFs);
 }
-return archive.makeQualified(localFs);
   }
 }
 return null;
@@ -130,7 +129,7 @@ public class HashTableLoader implements 
org.apache.hadoop.hive.ql.exec.HashTable
   throws Exception {
 MapredLocalWork localWork = context.getLocalWork();
 List directWorks = localWork.getDirectFetchOp().get(joinOp);
-if (directWorks == null || directWorks.isEmpty()) {
+if (CollectionUtils.isEmpty(directWorks)) {
   return;
 }
 JobConf job = new JobConf(hconf);



[1/2] hive git commit: Revert "HIVE-14792: AvroSerde reads the remote schema-file at least once per mapper, per table reference. (Addendum)"

2018-03-12 Thread aihuaxu
Repository: hive
Updated Branches:
  refs/heads/branch-2.2 e8e3974d1 -> 5949479f7


Revert "HIVE-14792: AvroSerde reads the remote schema-file at least once per 
mapper, per table reference. (Addendum)"

This reverts commit e8e3974d157a550ec0112258b59b603d9829.


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/220d1998
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/220d1998
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/220d1998

Branch: refs/heads/branch-2.2
Commit: 220d199859e373f99e314e50121a149c10351a67
Parents: e8e3974
Author: Aihua Xu 
Authored: Mon Mar 12 14:52:36 2018 -0700
Committer: Aihua Xu 
Committed: Mon Mar 12 14:52:36 2018 -0700

--
 .../TablePropertyEnrichmentOptimizer.java   |  45 +---
 .../avro_tableproperty_optimize.q   |  63 --
 .../avro_tableproperty_optimize.q.out   | 226 ---
 3 files changed, 10 insertions(+), 324 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/220d1998/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
--
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
index d313f7d..5824490 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
@@ -26,7 +26,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
 import org.apache.hadoop.hive.ql.lib.Dispatcher;
@@ -41,10 +40,8 @@ import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.TableScanDesc;
 import org.apache.hadoop.hive.serde2.Deserializer;
-import org.apache.hive.common.util.ReflectionUtil;
 
 import java.util.Arrays;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
@@ -81,51 +78,29 @@ class TablePropertyEnrichmentOptimizer extends Transform {
 }
   }
 
-  /**
-   * Retrieves the table properties as well as the properties from Serde.
-   */
-  private static Map getTableParameters(Table table) {
-Map originalTableParameters = new 
HashMap<>(table.getParameters());
-Properties tableMetadata = MetaStoreUtils.getTableMetadata(table);
-for (String property : tableMetadata.stringPropertyNames()) {
-  if (!originalTableParameters.containsKey(property)) {
-originalTableParameters.put(property, 
tableMetadata.getProperty(property));
-  }
-}
-return originalTableParameters;
-  }
-
   private static class Processor implements NodeProcessor {
 
 @Override
 public Object process(Node nd, Stack stack, NodeProcessorCtx 
procCtx, Object... nodeOutputs) throws SemanticException {
   TableScanOperator tsOp = (TableScanOperator) nd;
   WalkerCtx context = (WalkerCtx)procCtx;
+
   TableScanDesc tableScanDesc = tsOp.getConf();
   Table table = tsOp.getConf().getTableMetadata().getTTable();
+  Map tableParameters = table.getParameters();
+  Properties tableProperties = new Properties();
+  tableProperties.putAll(tableParameters);
 
-  Map originalTableParameters = getTableParameters(table);
-  if (LOG.isDebugEnabled()) {
-LOG.debug("Original Table parameters: " + originalTableParameters);
-  }
-  Properties clonedTableParameters = new Properties();
-  clonedTableParameters.putAll(originalTableParameters);
-
-  String deserializerClassName = null;
+  Deserializer deserializer = 
tableScanDesc.getTableMetadata().getDeserializer();
+  String deserializerClassName = deserializer.getClass().getName();
   try {
-deserializerClassName = 
tableScanDesc.getTableMetadata().getSd().getSerdeInfo().getSerializationLib();
-Deserializer deserializer = ReflectionUtil.newInstance(
-context.conf.getClassByName(deserializerClassName)
-.asSubclass(Deserializer.class),
-context.conf);
-
 if 
(context.serdeClassesUnderConsideration.contains(deserializerClassName)) {
-  deserializer.initialize(context.conf, clonedTableParameters);
+  

[06/18] hive git commit: HIVE-17990 Add Thrift and DB storage for Schema Registry objects (Alan Gates, reviewed by Thejas Nair)

2018-03-12 Thread gates
http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
--
diff --git 
a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
 
b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
index 516be77..30214d8 100644
--- 
a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ 
b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@ -1410,6 +1410,104 @@ class Iface(fb303.FacebookService.Iface):
 """
 pass
 
+  def create_ischema(self, schema):
+"""
+Parameters:
+ - schema
+"""
+pass
+
+  def alter_ischema(self, rqst):
+"""
+Parameters:
+ - rqst
+"""
+pass
+
+  def get_ischema(self, name):
+"""
+Parameters:
+ - name
+"""
+pass
+
+  def drop_ischema(self, name):
+"""
+Parameters:
+ - name
+"""
+pass
+
+  def add_schema_version(self, schemaVersion):
+"""
+Parameters:
+ - schemaVersion
+"""
+pass
+
+  def get_schema_version(self, schemaVersion):
+"""
+Parameters:
+ - schemaVersion
+"""
+pass
+
+  def get_schema_latest_version(self, schemaName):
+"""
+Parameters:
+ - schemaName
+"""
+pass
+
+  def get_schema_all_versions(self, schemaName):
+"""
+Parameters:
+ - schemaName
+"""
+pass
+
+  def drop_schema_version(self, schemaVersion):
+"""
+Parameters:
+ - schemaVersion
+"""
+pass
+
+  def get_schemas_by_cols(self, rqst):
+"""
+Parameters:
+ - rqst
+"""
+pass
+
+  def map_schema_version_to_serde(self, rqst):
+"""
+Parameters:
+ - rqst
+"""
+pass
+
+  def set_schema_version_state(self, rqst):
+"""
+Parameters:
+ - rqst
+"""
+pass
+
+  def add_serde(self, serde):
+"""
+Parameters:
+ - serde
+"""
+pass
+
+  def get_serde(self, rqst):
+"""
+Parameters:
+ - rqst
+"""
+pass
+
 
 class Client(fb303.FacebookService.Client, Iface):
   """
@@ -7888,6 +7986,486 @@ class Client(fb303.FacebookService.Client, Iface):
   raise result.o4
 raise TApplicationException(TApplicationException.MISSING_RESULT, 
"create_or_drop_wm_trigger_to_pool_mapping failed: unknown result")
 
+  def create_ischema(self, schema):
+"""
+Parameters:
+ - schema
+"""
+self.send_create_ischema(schema)
+self.recv_create_ischema()
+
+  def send_create_ischema(self, schema):
+self._oprot.writeMessageBegin('create_ischema', TMessageType.CALL, 
self._seqid)
+args = create_ischema_args()
+args.schema = schema
+args.write(self._oprot)
+self._oprot.writeMessageEnd()
+self._oprot.trans.flush()
+
+  def recv_create_ischema(self):
+iprot = self._iprot
+(fname, mtype, rseqid) = iprot.readMessageBegin()
+if mtype == TMessageType.EXCEPTION:
+  x = TApplicationException()
+  x.read(iprot)
+  iprot.readMessageEnd()
+  raise x
+result = create_ischema_result()
+result.read(iprot)
+iprot.readMessageEnd()
+if result.o1 is not None:
+  raise result.o1
+if result.o2 is not None:
+  raise result.o2
+if result.o3 is not None:
+  raise result.o3
+return
+
+  def alter_ischema(self, rqst):
+"""
+Parameters:
+ - rqst
+"""
+self.send_alter_ischema(rqst)
+self.recv_alter_ischema()
+
+  def send_alter_ischema(self, rqst):
+self._oprot.writeMessageBegin('alter_ischema', TMessageType.CALL, 
self._seqid)
+args = alter_ischema_args()
+args.rqst = rqst
+args.write(self._oprot)
+self._oprot.writeMessageEnd()
+self._oprot.trans.flush()
+
+  def recv_alter_ischema(self):
+iprot = self._iprot
+(fname, mtype, rseqid) = iprot.readMessageBegin()
+if mtype == TMessageType.EXCEPTION:
+  x = TApplicationException()
+  x.read(iprot)
+  iprot.readMessageEnd()
+  raise x
+result = alter_ischema_result()
+result.read(iprot)
+iprot.readMessageEnd()
+if result.o1 is not None:
+  raise result.o1
+if result.o2 is not None:
+  raise result.o2
+return
+
+  def get_ischema(self, name):
+"""
+Parameters:
+ - name
+"""
+self.send_get_ischema(name)
+return self.recv_get_ischema()
+
+  def send_get_ischema(self, name):
+self._oprot.writeMessageBegin('get_ischema', TMessageType.CALL, 
self._seqid)
+args = get_ischema_args()
+args.name = name
+args.write(self._oprot)
+self._oprot.writeMessageEnd()
+self._oprot.trans.flush()
+
+  def recv_get_ischema(self):
+iprot = self._iprot
+(fname, mtype, rseqid) = iprot.readMessageBegin()
+if mtype == TMessageType.EXCEPTION:
+  x = TApplicationException()
+  x.read(iprot)
+  iprot.readMessageEnd()
+  raise x
+result = 

[1/2] hive git commit: Revert "HIVE-14792: AvroSerde reads the remote schema-file at least once per mapper, per table reference. (Addendum)"

2018-03-12 Thread aihuaxu
Repository: hive
Updated Branches:
  refs/heads/branch-2.2 5949479f7 -> 9e10b88c3


Revert "HIVE-14792: AvroSerde reads the remote schema-file at least once per 
mapper, per table reference. (Addendum)"

This reverts commit 5949479f7e08987b67c4ee86c06c2d5949f75bee.


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/a20e63e1
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/a20e63e1
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/a20e63e1

Branch: refs/heads/branch-2.2
Commit: a20e63e1eab675fcf2fa85ec9a320fec4fec6886
Parents: 5949479
Author: Aihua Xu 
Authored: Mon Mar 12 15:00:53 2018 -0700
Committer: Aihua Xu 
Committed: Mon Mar 12 15:00:53 2018 -0700

--
 .../TablePropertyEnrichmentOptimizer.java   |  45 +---
 .../avro_tableproperty_optimize.q   |  63 --
 .../avro_tableproperty_optimize.q.out   | 226 ---
 3 files changed, 10 insertions(+), 324 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/a20e63e1/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
--
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
index d313f7d..5824490 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
@@ -26,7 +26,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
 import org.apache.hadoop.hive.ql.lib.Dispatcher;
@@ -41,10 +40,8 @@ import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.TableScanDesc;
 import org.apache.hadoop.hive.serde2.Deserializer;
-import org.apache.hive.common.util.ReflectionUtil;
 
 import java.util.Arrays;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
@@ -81,51 +78,29 @@ class TablePropertyEnrichmentOptimizer extends Transform {
 }
   }
 
-  /**
-   * Retrieves the table properties as well as the properties from Serde.
-   */
-  private static Map getTableParameters(Table table) {
-Map originalTableParameters = new 
HashMap<>(table.getParameters());
-Properties tableMetadata = MetaStoreUtils.getTableMetadata(table);
-for (String property : tableMetadata.stringPropertyNames()) {
-  if (!originalTableParameters.containsKey(property)) {
-originalTableParameters.put(property, 
tableMetadata.getProperty(property));
-  }
-}
-return originalTableParameters;
-  }
-
   private static class Processor implements NodeProcessor {
 
 @Override
 public Object process(Node nd, Stack stack, NodeProcessorCtx 
procCtx, Object... nodeOutputs) throws SemanticException {
   TableScanOperator tsOp = (TableScanOperator) nd;
   WalkerCtx context = (WalkerCtx)procCtx;
+
   TableScanDesc tableScanDesc = tsOp.getConf();
   Table table = tsOp.getConf().getTableMetadata().getTTable();
+  Map tableParameters = table.getParameters();
+  Properties tableProperties = new Properties();
+  tableProperties.putAll(tableParameters);
 
-  Map originalTableParameters = getTableParameters(table);
-  if (LOG.isDebugEnabled()) {
-LOG.debug("Original Table parameters: " + originalTableParameters);
-  }
-  Properties clonedTableParameters = new Properties();
-  clonedTableParameters.putAll(originalTableParameters);
-
-  String deserializerClassName = null;
+  Deserializer deserializer = 
tableScanDesc.getTableMetadata().getDeserializer();
+  String deserializerClassName = deserializer.getClass().getName();
   try {
-deserializerClassName = 
tableScanDesc.getTableMetadata().getSd().getSerdeInfo().getSerializationLib();
-Deserializer deserializer = ReflectionUtil.newInstance(
-context.conf.getClassByName(deserializerClassName)
-.asSubclass(Deserializer.class),
-context.conf);
-
 if 
(context.serdeClassesUnderConsideration.contains(deserializerClassName)) {
-  deserializer.initialize(context.conf, clonedTableParameters);
+  

hive git commit: HIVE-14792: AvroSerde reads the remote schema-file at least once per mapper, per table reference. (Addendum)

2018-03-12 Thread aihuaxu
Repository: hive
Updated Branches:
  refs/heads/master 190c72e77 -> 1629ec058


HIVE-14792: AvroSerde reads the remote schema-file at least once per mapper, 
per table reference. (Addendum)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/1629ec05
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/1629ec05
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/1629ec05

Branch: refs/heads/master
Commit: 1629ec058faf2dce581de4f393f0c6485c7425d7
Parents: 190c72e
Author: Aihua Xu 
Authored: Thu Mar 8 11:33:37 2018 -0800
Committer: Aihua Xu 
Committed: Mon Mar 12 14:16:38 2018 -0700

--
 .../TablePropertyEnrichmentOptimizer.java   |  45 +++-
 .../avro_tableproperty_optimize.q   |  63 ++
 .../avro_tableproperty_optimize.q.out   | 226 +++
 3 files changed, 324 insertions(+), 10 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/1629ec05/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
--
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
index d806775..bc17691 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
@@ -26,6 +26,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
 import org.apache.hadoop.hive.ql.lib.Dispatcher;
@@ -40,8 +41,10 @@ import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.TableScanDesc;
 import org.apache.hadoop.hive.serde2.Deserializer;
+import org.apache.hive.common.util.ReflectionUtil;
 
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
@@ -78,29 +81,51 @@ class TablePropertyEnrichmentOptimizer extends Transform {
 }
   }
 
+  /**
+   * Retrieves the table properties as well as the properties from Serde.
+   */
+  private static Map getTableParameters(Table table) {
+Map originalTableParameters = new 
HashMap<>(table.getParameters());
+Properties tableMetadata = MetaStoreUtils.getTableMetadata(table);
+for (String property : tableMetadata.stringPropertyNames()) {
+  if (!originalTableParameters.containsKey(property)) {
+originalTableParameters.put(property, 
tableMetadata.getProperty(property));
+  }
+}
+return originalTableParameters;
+  }
+
   private static class Processor implements NodeProcessor {
 
 @Override
 public Object process(Node nd, Stack stack, NodeProcessorCtx 
procCtx, Object... nodeOutputs) throws SemanticException {
   TableScanOperator tsOp = (TableScanOperator) nd;
   WalkerCtx context = (WalkerCtx)procCtx;
-
   TableScanDesc tableScanDesc = tsOp.getConf();
   Table table = tsOp.getConf().getTableMetadata().getTTable();
-  Map tableParameters = table.getParameters();
-  Properties tableProperties = new Properties();
-  tableProperties.putAll(tableParameters);
 
-  Deserializer deserializer = 
tableScanDesc.getTableMetadata().getDeserializer();
-  String deserializerClassName = deserializer.getClass().getName();
+  Map originalTableParameters = getTableParameters(table);
+  if (LOG.isDebugEnabled()) {
+LOG.debug("Original Table parameters: " + originalTableParameters);
+  }
+  Properties clonedTableParameters = new Properties();
+  clonedTableParameters.putAll(originalTableParameters);
+
+  String deserializerClassName = null;
   try {
+deserializerClassName = 
tableScanDesc.getTableMetadata().getSd().getSerdeInfo().getSerializationLib();
+Deserializer deserializer = ReflectionUtil.newInstance(
+context.conf.getClassByName(deserializerClassName)
+.asSubclass(Deserializer.class),
+context.conf);
+
 if 
(context.serdeClassesUnderConsideration.contains(deserializerClassName)) {
-  deserializer.initialize(context.conf, tableProperties);
+  deserializer.initialize(context.conf, clonedTableParameters);
   LOG.debug("SerDe init 

hive git commit: HIVE-18907: Create utility to fix acid key index issue from HIVE-18817 (Jason Dere, reviewed by Prasanth Jayachandran)

2018-03-12 Thread jdere
Repository: hive
Updated Branches:
  refs/heads/master 1629ec058 -> d717d3853


HIVE-18907: Create utility to fix acid key index issue from HIVE-18817 (Jason 
Dere, reviewed by Prasanth Jayachandran)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/d717d385
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/d717d385
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/d717d385

Branch: refs/heads/master
Commit: d717d3853469afc95dbb46a05f52e1b4c7304a89
Parents: 1629ec0
Author: Jason Dere 
Authored: Mon Mar 12 15:28:50 2018 -0700
Committer: Jason Dere 
Committed: Mon Mar 12 15:28:50 2018 -0700

--
 bin/ext/fixacidkeyindex.sh  |  32 ++
 .../hadoop/hive/ql/io/orc/FixAcidKeyIndex.java  | 374 +++
 .../hive/ql/io/orc/TestFixAcidKeyIndex.java | 301 +++
 3 files changed, 707 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/d717d385/bin/ext/fixacidkeyindex.sh
--
diff --git a/bin/ext/fixacidkeyindex.sh b/bin/ext/fixacidkeyindex.sh
new file mode 100644
index 000..28af2ea
--- /dev/null
+++ b/bin/ext/fixacidkeyindex.sh
@@ -0,0 +1,32 @@
+# 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.
+
+THISSERVICE=fixacidkeyindex
+export SERVICE_LIST="${SERVICE_LIST}${THISSERVICE} "
+
+fixacidkeyindex () {
+  CLASS=org.apache.hadoop.hive.ql.io.orc.FixAcidKeyIndex
+  HIVE_OPTS=''
+  execHiveCmd $CLASS "$@"
+}
+
+fixacidkeyindex_help () {
+  echo "usage ./hive fixacidkeyindex [-h] --check-only|--recover 
[--backup-path ] "
+  echo ""
+  echo "  --check-onlyCheck acid orc file for valid acid key 
index and exit without fixing"
+  echo "  --recover   Fix the acid key index for acid orc file 
if it requires fixing"
+  echo "  --backup-path   Specify a backup path to store the 
corrupted files (default: /tmp)"
+  echo "  --help (-h) Print help message"
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d717d385/ql/src/java/org/apache/hadoop/hive/ql/io/orc/FixAcidKeyIndex.java
--
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/FixAcidKeyIndex.java 
b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/FixAcidKeyIndex.java
new file mode 100644
index 000..6920938
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/FixAcidKeyIndex.java
@@ -0,0 +1,374 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.io.orc;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetDecoder;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;

[2/2] hive git commit: HIVE-14792: AvroSerde reads the remote schema-file at least once per mapper, per table reference. (Addendum)

2018-03-12 Thread aihuaxu
HIVE-14792: AvroSerde reads the remote schema-file at least once per mapper, 
per table reference. (Addendum)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/5949479f
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/5949479f
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/5949479f

Branch: refs/heads/branch-2.2
Commit: 5949479f7e08987b67c4ee86c06c2d5949f75bee
Parents: 220d199
Author: Aihua Xu 
Authored: Thu Mar 8 11:33:37 2018 -0800
Committer: Aihua Xu 
Committed: Mon Mar 12 14:52:53 2018 -0700

--
 .../TablePropertyEnrichmentOptimizer.java   |  45 +++-
 .../avro_tableproperty_optimize.q   |  63 ++
 .../avro_tableproperty_optimize.q.out   | 226 +++
 3 files changed, 324 insertions(+), 10 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/5949479f/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
--
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
index 5824490..d313f7d 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
@@ -26,6 +26,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
 import org.apache.hadoop.hive.ql.lib.Dispatcher;
@@ -40,8 +41,10 @@ import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.TableScanDesc;
 import org.apache.hadoop.hive.serde2.Deserializer;
+import org.apache.hive.common.util.ReflectionUtil;
 
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
@@ -78,29 +81,51 @@ class TablePropertyEnrichmentOptimizer extends Transform {
 }
   }
 
+  /**
+   * Retrieves the table properties as well as the properties from Serde.
+   */
+  private static Map getTableParameters(Table table) {
+Map originalTableParameters = new 
HashMap<>(table.getParameters());
+Properties tableMetadata = MetaStoreUtils.getTableMetadata(table);
+for (String property : tableMetadata.stringPropertyNames()) {
+  if (!originalTableParameters.containsKey(property)) {
+originalTableParameters.put(property, 
tableMetadata.getProperty(property));
+  }
+}
+return originalTableParameters;
+  }
+
   private static class Processor implements NodeProcessor {
 
 @Override
 public Object process(Node nd, Stack stack, NodeProcessorCtx 
procCtx, Object... nodeOutputs) throws SemanticException {
   TableScanOperator tsOp = (TableScanOperator) nd;
   WalkerCtx context = (WalkerCtx)procCtx;
-
   TableScanDesc tableScanDesc = tsOp.getConf();
   Table table = tsOp.getConf().getTableMetadata().getTTable();
-  Map tableParameters = table.getParameters();
-  Properties tableProperties = new Properties();
-  tableProperties.putAll(tableParameters);
 
-  Deserializer deserializer = 
tableScanDesc.getTableMetadata().getDeserializer();
-  String deserializerClassName = deserializer.getClass().getName();
+  Map originalTableParameters = getTableParameters(table);
+  if (LOG.isDebugEnabled()) {
+LOG.debug("Original Table parameters: " + originalTableParameters);
+  }
+  Properties clonedTableParameters = new Properties();
+  clonedTableParameters.putAll(originalTableParameters);
+
+  String deserializerClassName = null;
   try {
+deserializerClassName = 
tableScanDesc.getTableMetadata().getSd().getSerdeInfo().getSerializationLib();
+Deserializer deserializer = ReflectionUtil.newInstance(
+context.conf.getClassByName(deserializerClassName)
+.asSubclass(Deserializer.class),
+context.conf);
+
 if 
(context.serdeClassesUnderConsideration.contains(deserializerClassName)) {
-  deserializer.initialize(context.conf, tableProperties);
+  deserializer.initialize(context.conf, clonedTableParameters);
   LOG.debug("SerDe init succeeded for class: " + 
deserializerClassName);
-  for (Map.Entry 

[2/2] hive git commit: HIVE-14792: AvroSerde reads the remote schema-file at least once per mapper, per table reference. (Addendum)

2018-03-12 Thread aihuaxu
HIVE-14792: AvroSerde reads the remote schema-file at least once per mapper, 
per table reference. (Addendum)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/9e10b88c
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/9e10b88c
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/9e10b88c

Branch: refs/heads/branch-2.2
Commit: 9e10b88c33a3ae9cb09cb230c4ae09e442203ba9
Parents: a20e63e
Author: Aihua Xu 
Authored: Thu Mar 8 11:33:37 2018 -0800
Committer: Aihua Xu 
Committed: Mon Mar 12 15:01:25 2018 -0700

--
 .../TablePropertyEnrichmentOptimizer.java   |  45 +++-
 .../avro_tableproperty_optimize.q   |  63 ++
 .../avro_tableproperty_optimize.q.out   | 226 +++
 3 files changed, 324 insertions(+), 10 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/9e10b88c/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
--
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
index 5824490..154eb02 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
@@ -26,6 +26,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
 import org.apache.hadoop.hive.ql.lib.Dispatcher;
@@ -40,8 +41,10 @@ import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.TableScanDesc;
 import org.apache.hadoop.hive.serde2.Deserializer;
+import org.apache.hive.common.util.ReflectionUtil;
 
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
@@ -78,29 +81,51 @@ class TablePropertyEnrichmentOptimizer extends Transform {
 }
   }
 
+  /**
+   * Retrieves the table properties as well as the properties from Serde.
+   */
+  private static Map getTableParameters(Table table) {
+Map originalTableParameters = new 
HashMap<>(table.getParameters());
+Properties tableMetadata = MetaStoreUtils.getTableMetadata(table);
+for (String property : tableMetadata.stringPropertyNames()) {
+  if (!originalTableParameters.containsKey(property)) {
+originalTableParameters.put(property, 
tableMetadata.getProperty(property));
+  }
+}
+return originalTableParameters;
+  }
+
   private static class Processor implements NodeProcessor {
 
 @Override
 public Object process(Node nd, Stack stack, NodeProcessorCtx 
procCtx, Object... nodeOutputs) throws SemanticException {
   TableScanOperator tsOp = (TableScanOperator) nd;
   WalkerCtx context = (WalkerCtx)procCtx;
-
   TableScanDesc tableScanDesc = tsOp.getConf();
   Table table = tsOp.getConf().getTableMetadata().getTTable();
-  Map tableParameters = table.getParameters();
-  Properties tableProperties = new Properties();
-  tableProperties.putAll(tableParameters);
 
-  Deserializer deserializer = 
tableScanDesc.getTableMetadata().getDeserializer();
-  String deserializerClassName = deserializer.getClass().getName();
+  Map originalTableParameters = getTableParameters(table);
+  if (LOG.isDebugEnabled()) {
+LOG.debug("Original Table parameters: " + originalTableParameters);
+  }
+  Properties clonedTableParameters = new Properties();
+  clonedTableParameters.putAll(originalTableParameters);
+
+  String deserializerClassName = null;
   try {
+deserializerClassName = 
tableScanDesc.getTableMetadata().getSd().getSerdeInfo().getSerializationLib();
+Deserializer deserializer = ReflectionUtil.newInstance(
+context.conf.getClassByName(deserializerClassName)
+.asSubclass(Deserializer.class),
+context.conf);
+
 if 
(context.serdeClassesUnderConsideration.contains(deserializerClassName)) {
-  deserializer.initialize(context.conf, tableProperties);
+  deserializer.initialize(context.conf, clonedTableParameters);
   LOG.debug("SerDe init succeeded for class: " + 
deserializerClassName);
-  for (Map.Entry property 

[13/18] hive git commit: HIVE-17990 Add Thrift and DB storage for Schema Registry objects (Alan Gates, reviewed by Thejas Nair)

2018-03-12 Thread gates
http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
--
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h 
b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
index cee1382..ef8b694 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
@@ -132,6 +132,59 @@ struct EventRequestType {
 
 extern const std::map _EventRequestType_VALUES_TO_NAMES;
 
+struct SerdeType {
+  enum type {
+HIVE = 1,
+SCHEMA_REGISTRY = 2
+  };
+};
+
+extern const std::map _SerdeType_VALUES_TO_NAMES;
+
+struct SchemaType {
+  enum type {
+HIVE = 1,
+AVRO = 2
+  };
+};
+
+extern const std::map _SchemaType_VALUES_TO_NAMES;
+
+struct SchemaCompatibility {
+  enum type {
+NONE = 1,
+BACKWARD = 2,
+FORWARD = 3,
+BOTH = 4
+  };
+};
+
+extern const std::map _SchemaCompatibility_VALUES_TO_NAMES;
+
+struct SchemaValidation {
+  enum type {
+LATEST = 1,
+ALL = 2
+  };
+};
+
+extern const std::map _SchemaValidation_VALUES_TO_NAMES;
+
+struct SchemaVersionState {
+  enum type {
+INITIATED = 1,
+START_REVIEW = 2,
+CHANGES_REQUIRED = 3,
+REVIEWED = 4,
+ENABLED = 5,
+DISABLED = 6,
+ARCHIVED = 7,
+DELETED = 8
+  };
+};
+
+extern const std::map _SchemaVersionState_VALUES_TO_NAMES;
+
 struct FunctionType {
   enum type {
 JAVA = 1
@@ -566,6 +619,26 @@ class WMCreateOrDropTriggerToPoolMappingRequest;
 
 class WMCreateOrDropTriggerToPoolMappingResponse;
 
+class ISchema;
+
+class ISchemaName;
+
+class AlterISchemaRequest;
+
+class SchemaVersion;
+
+class SchemaVersionDescriptor;
+
+class FindSchemasByColsRqst;
+
+class FindSchemasByColsResp;
+
+class MapSchemaVersionToSerdeRequest;
+
+class SetSchemaVersionStateRequest;
+
+class GetSerdeRequest;
+
 class MetaException;
 
 class UnknownTableException;
@@ -2192,10 +2265,14 @@ inline std::ostream& operator<<(std::ostream& out, 
const Database& obj)
 }
 
 typedef struct _SerDeInfo__isset {
-  _SerDeInfo__isset() : name(false), serializationLib(false), 
parameters(false) {}
+  _SerDeInfo__isset() : name(false), serializationLib(false), 
parameters(false), description(false), serializerClass(false), 
deserializerClass(false), serdeType(false) {}
   bool name :1;
   bool serializationLib :1;
   bool parameters :1;
+  bool description :1;
+  bool serializerClass :1;
+  bool deserializerClass :1;
+  bool serdeType :1;
 } _SerDeInfo__isset;
 
 class SerDeInfo {
@@ -2203,13 +2280,17 @@ class SerDeInfo {
 
   SerDeInfo(const SerDeInfo&);
   SerDeInfo& operator=(const SerDeInfo&);
-  SerDeInfo() : name(), serializationLib() {
+  SerDeInfo() : name(), serializationLib(), description(), serializerClass(), 
deserializerClass(), serdeType((SerdeType::type)0) {
   }
 
   virtual ~SerDeInfo() throw();
   std::string name;
   std::string serializationLib;
   std::map  parameters;
+  std::string description;
+  std::string serializerClass;
+  std::string deserializerClass;
+  SerdeType::type serdeType;
 
   _SerDeInfo__isset __isset;
 
@@ -2219,6 +2300,14 @@ class SerDeInfo {
 
   void __set_parameters(const std::map & val);
 
+  void __set_description(const std::string& val);
+
+  void __set_serializerClass(const std::string& val);
+
+  void __set_deserializerClass(const std::string& val);
+
+  void __set_serdeType(const SerdeType::type val);
+
   bool operator == (const SerDeInfo & rhs) const
   {
 if (!(name == rhs.name))
@@ -2227,6 +2316,22 @@ class SerDeInfo {
   return false;
 if (!(parameters == rhs.parameters))
   return false;
+if (__isset.description != rhs.__isset.description)
+  return false;
+else if (__isset.description && !(description == rhs.description))
+  return false;
+if (__isset.serializerClass != rhs.__isset.serializerClass)
+  return false;
+else if (__isset.serializerClass && !(serializerClass == 
rhs.serializerClass))
+  return false;
+if (__isset.deserializerClass != rhs.__isset.deserializerClass)
+  return false;
+else if (__isset.deserializerClass && !(deserializerClass == 
rhs.deserializerClass))
+  return false;
+if (__isset.serdeType != rhs.__isset.serdeType)
+  return false;
+else if (__isset.serdeType && !(serdeType == rhs.serdeType))
+  return false;
 return true;
   }
   bool operator != (const SerDeInfo ) const {
@@ -11328,6 +11433,626 @@ inline std::ostream& operator<<(std::ostream& out, 
const WMCreateOrDropTriggerTo
   return out;
 }
 
+typedef struct _ISchema__isset {
+  _ISchema__isset() : schemaType(false), name(false), dbName(false), 
compatibility(false), validationLevel(false), 

[08/18] hive git commit: HIVE-17990 Add Thrift and DB storage for Schema Registry objects (Alan Gates, reviewed by Thejas Nair)

2018-03-12 Thread gates
http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
--
diff --git 
a/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php 
b/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
index 02242a7..efe693a 100644
--- 
a/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ 
b/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@ -1380,6 +1380,99 @@ interface ThriftHiveMetastoreIf extends 
\FacebookServiceIf {
* @throws \metastore\MetaException
*/
   public function 
create_or_drop_wm_trigger_to_pool_mapping(\metastore\WMCreateOrDropTriggerToPoolMappingRequest
 $request);
+  /**
+   * @param \metastore\ISchema $schema
+   * @throws \metastore\AlreadyExistsException
+   * @throws \metastore\NoSuchObjectException
+   * @throws \metastore\MetaException
+   */
+  public function create_ischema(\metastore\ISchema $schema);
+  /**
+   * @param \metastore\AlterISchemaRequest $rqst
+   * @throws \metastore\NoSuchObjectException
+   * @throws \metastore\MetaException
+   */
+  public function alter_ischema(\metastore\AlterISchemaRequest $rqst);
+  /**
+   * @param \metastore\ISchemaName $name
+   * @return \metastore\ISchema
+   * @throws \metastore\NoSuchObjectException
+   * @throws \metastore\MetaException
+   */
+  public function get_ischema(\metastore\ISchemaName $name);
+  /**
+   * @param \metastore\ISchemaName $name
+   * @throws \metastore\NoSuchObjectException
+   * @throws \metastore\InvalidOperationException
+   * @throws \metastore\MetaException
+   */
+  public function drop_ischema(\metastore\ISchemaName $name);
+  /**
+   * @param \metastore\SchemaVersion $schemaVersion
+   * @throws \metastore\AlreadyExistsException
+   * @throws \metastore\NoSuchObjectException
+   * @throws \metastore\MetaException
+   */
+  public function add_schema_version(\metastore\SchemaVersion $schemaVersion);
+  /**
+   * @param \metastore\SchemaVersionDescriptor $schemaVersion
+   * @return \metastore\SchemaVersion
+   * @throws \metastore\NoSuchObjectException
+   * @throws \metastore\MetaException
+   */
+  public function get_schema_version(\metastore\SchemaVersionDescriptor 
$schemaVersion);
+  /**
+   * @param \metastore\ISchemaName $schemaName
+   * @return \metastore\SchemaVersion
+   * @throws \metastore\NoSuchObjectException
+   * @throws \metastore\MetaException
+   */
+  public function get_schema_latest_version(\metastore\ISchemaName 
$schemaName);
+  /**
+   * @param \metastore\ISchemaName $schemaName
+   * @return \metastore\SchemaVersion[]
+   * @throws \metastore\NoSuchObjectException
+   * @throws \metastore\MetaException
+   */
+  public function get_schema_all_versions(\metastore\ISchemaName $schemaName);
+  /**
+   * @param \metastore\SchemaVersionDescriptor $schemaVersion
+   * @throws \metastore\NoSuchObjectException
+   * @throws \metastore\MetaException
+   */
+  public function drop_schema_version(\metastore\SchemaVersionDescriptor 
$schemaVersion);
+  /**
+   * @param \metastore\FindSchemasByColsRqst $rqst
+   * @return \metastore\FindSchemasByColsResp
+   * @throws \metastore\MetaException
+   */
+  public function get_schemas_by_cols(\metastore\FindSchemasByColsRqst $rqst);
+  /**
+   * @param \metastore\MapSchemaVersionToSerdeRequest $rqst
+   * @throws \metastore\NoSuchObjectException
+   * @throws \metastore\MetaException
+   */
+  public function 
map_schema_version_to_serde(\metastore\MapSchemaVersionToSerdeRequest $rqst);
+  /**
+   * @param \metastore\SetSchemaVersionStateRequest $rqst
+   * @throws \metastore\NoSuchObjectException
+   * @throws \metastore\InvalidOperationException
+   * @throws \metastore\MetaException
+   */
+  public function 
set_schema_version_state(\metastore\SetSchemaVersionStateRequest $rqst);
+  /**
+   * @param \metastore\SerDeInfo $serde
+   * @throws \metastore\AlreadyExistsException
+   * @throws \metastore\MetaException
+   */
+  public function add_serde(\metastore\SerDeInfo $serde);
+  /**
+   * @param \metastore\GetSerdeRequest $rqst
+   * @return \metastore\SerDeInfo
+   * @throws \metastore\NoSuchObjectException
+   * @throws \metastore\MetaException
+   */
+  public function get_serde(\metastore\GetSerdeRequest $rqst);
 }
 
 class ThriftHiveMetastoreClient extends \FacebookServiceClient implements 
\metastore\ThriftHiveMetastoreIf {
@@ -11633,322 +11726,4430 @@ class ThriftHiveMetastoreClient extends 
\FacebookServiceClient implements \metas
 throw new \Exception("create_or_drop_wm_trigger_to_pool_mapping failed: 
unknown result");
   }
 
-}
-
-// HELPER FUNCTIONS AND STRUCTURES
+  public function create_ischema(\metastore\ISchema $schema)
+  {
+$this->send_create_ischema($schema);
+$this->recv_create_ischema();
+  }
 
-class ThriftHiveMetastore_getMetaConf_args {
-  static $_TSPEC;

[02/18] hive git commit: HIVE-17990 Add Thrift and DB storage for Schema Registry objects (Alan Gates, reviewed by Thejas Nair)

2018-03-12 Thread gates
http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/main/thrift/hive_metastore.thrift
--
diff --git a/standalone-metastore/src/main/thrift/hive_metastore.thrift 
b/standalone-metastore/src/main/thrift/hive_metastore.thrift
index 65e2f78..b816eb6 100644
--- a/standalone-metastore/src/main/thrift/hive_metastore.thrift
+++ b/standalone-metastore/src/main/thrift/hive_metastore.thrift
@@ -183,6 +183,39 @@ enum EventRequestType {
 DELETE = 3,
 }
 
+enum SerdeType {
+  HIVE = 1,
+  SCHEMA_REGISTRY = 2,
+}
+
+enum SchemaType {
+  HIVE = 1,
+  AVRO = 2,
+}
+
+enum SchemaCompatibility {
+  NONE = 1,
+  BACKWARD = 2,
+  FORWARD = 3,
+  BOTH = 4
+}
+
+enum SchemaValidation {
+  LATEST = 1,
+  ALL = 2
+}
+
+enum SchemaVersionState {
+  INITIATED = 1,
+  START_REVIEW = 2,
+  CHANGES_REQUIRED = 3,
+  REVIEWED = 4,
+  ENABLED = 5,
+  DISABLED = 6,
+  ARCHIVED = 7,
+  DELETED = 8
+}
+
 struct HiveObjectRef{
   1: HiveObjectType objectType,
   2: string dbName,
@@ -289,7 +322,11 @@ struct Database {
 struct SerDeInfo {
   1: string name,   // name of the serde, table name by default
   2: string serializationLib,   // usually the class that implements the 
extractor & loader
-  3: map parameters // initialization parameters
+  3: map parameters, // initialization parameters
+  4: optional string description,
+  5: optional string serializerClass,
+  6: optional string deserializerClass,
+  7: optional SerdeType serdeType
 }
 
 // sort order of a column (column name along with asc(1)/desc(0))
@@ -1318,6 +1355,71 @@ struct WMCreateOrDropTriggerToPoolMappingRequest {
 struct WMCreateOrDropTriggerToPoolMappingResponse {
 }
 
+// Schema objects
+// Schema is already taken, so for the moment I'm calling it an ISchema for 
Independent Schema
+struct ISchema {
+  1: SchemaType schemaType,
+  2: string name,
+  3: string dbName,
+  4: SchemaCompatibility compatibility,
+  5: SchemaValidation validationLevel,
+  6: bool canEvolve,
+  7: optional string schemaGroup,
+  8: optional string description
+}
+
+struct ISchemaName {
+  1: string dbName,
+  2: string schemaName
+}
+
+struct AlterISchemaRequest {
+  1: ISchemaName name,
+  3: ISchema newSchema
+}
+
+struct SchemaVersion {
+  1:  ISchemaName schema,
+  2:  i32 version,
+  3:  i64 createdAt,
+  4:  list cols,
+  5:  optional SchemaVersionState state,
+  6:  optional string description,
+  7:  optional string schemaText,
+  8:  optional string fingerprint,
+  9:  optional string name,
+  10: optional SerDeInfo serDe
+}
+
+struct SchemaVersionDescriptor {
+  1: ISchemaName schema,
+  2: i32 version
+}
+
+struct FindSchemasByColsRqst {
+  1: optional string colName,
+  2: optional string colNamespace,
+  3: optional string type
+}
+
+struct FindSchemasByColsResp {
+  1: list schemaVersions
+}
+
+struct MapSchemaVersionToSerdeRequest {
+  1: SchemaVersionDescriptor schemaVersion,
+  2: string serdeName
+}
+
+struct SetSchemaVersionStateRequest {
+  1: SchemaVersionDescriptor schemaVersion,
+  2: SchemaVersionState state
+}
+
+struct GetSerdeRequest {
+  1: string serdeName
+}
+
 // Exceptions.
 
 exception MetaException {
@@ -1929,6 +2031,38 @@ service ThriftHiveMetastore extends fb303.FacebookService
 
   WMCreateOrDropTriggerToPoolMappingResponse 
create_or_drop_wm_trigger_to_pool_mapping(1:WMCreateOrDropTriggerToPoolMappingRequest
 request)
   throws(1:AlreadyExistsException o1, 2:NoSuchObjectException o2, 
3:InvalidObjectException o3, 4:MetaException o4)
+
+  // Schema calls
+  void create_ischema(1:ISchema schema) throws(1:AlreadyExistsException o1,
+NoSuchObjectException o2, 3:MetaException o3)
+  void alter_ischema(1:AlterISchemaRequest rqst)
+throws(1:NoSuchObjectException o1, 2:MetaException o2)
+  ISchema get_ischema(1:ISchemaName name) throws (1:NoSuchObjectException o1, 
2:MetaException o2)
+  void drop_ischema(1:ISchemaName name)
+throws(1:NoSuchObjectException o1, 2:InvalidOperationException o2, 
3:MetaException o3)
+
+  void add_schema_version(1:SchemaVersion schemaVersion)
+throws(1:AlreadyExistsException o1, 2:NoSuchObjectException o2, 
3:MetaException o3)
+  SchemaVersion get_schema_version(1: SchemaVersionDescriptor schemaVersion)
+throws (1:NoSuchObjectException o1, 2:MetaException o2)
+  SchemaVersion get_schema_latest_version(1: ISchemaName schemaName)
+throws (1:NoSuchObjectException o1, 2:MetaException o2)
+  list get_schema_all_versions(1: ISchemaName schemaName)
+throws (1:NoSuchObjectException o1, 2:MetaException o2)
+  void drop_schema_version(1: SchemaVersionDescriptor schemaVersion)
+throws(1:NoSuchObjectException o1, 2:MetaException o2)
+  FindSchemasByColsResp get_schemas_by_cols(1: FindSchemasByColsRqst rqst)
+throws(1:MetaException o1)
+  // There is no blanket update of SchemaVersion since it is (mostly) 
immutable.  The 

[05/18] hive git commit: HIVE-17990 Add Thrift and DB storage for Schema Registry objects (Alan Gates, reviewed by Thejas Nair)

2018-03-12 Thread gates
http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
--
diff --git 
a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py 
b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
index b24c5f9..4d4429f 100644
--- a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -211,6 +211,100 @@ class EventRequestType:
 "DELETE": 3,
   }
 
+class SerdeType:
+  HIVE = 1
+  SCHEMA_REGISTRY = 2
+
+  _VALUES_TO_NAMES = {
+1: "HIVE",
+2: "SCHEMA_REGISTRY",
+  }
+
+  _NAMES_TO_VALUES = {
+"HIVE": 1,
+"SCHEMA_REGISTRY": 2,
+  }
+
+class SchemaType:
+  HIVE = 1
+  AVRO = 2
+
+  _VALUES_TO_NAMES = {
+1: "HIVE",
+2: "AVRO",
+  }
+
+  _NAMES_TO_VALUES = {
+"HIVE": 1,
+"AVRO": 2,
+  }
+
+class SchemaCompatibility:
+  NONE = 1
+  BACKWARD = 2
+  FORWARD = 3
+  BOTH = 4
+
+  _VALUES_TO_NAMES = {
+1: "NONE",
+2: "BACKWARD",
+3: "FORWARD",
+4: "BOTH",
+  }
+
+  _NAMES_TO_VALUES = {
+"NONE": 1,
+"BACKWARD": 2,
+"FORWARD": 3,
+"BOTH": 4,
+  }
+
+class SchemaValidation:
+  LATEST = 1
+  ALL = 2
+
+  _VALUES_TO_NAMES = {
+1: "LATEST",
+2: "ALL",
+  }
+
+  _NAMES_TO_VALUES = {
+"LATEST": 1,
+"ALL": 2,
+  }
+
+class SchemaVersionState:
+  INITIATED = 1
+  START_REVIEW = 2
+  CHANGES_REQUIRED = 3
+  REVIEWED = 4
+  ENABLED = 5
+  DISABLED = 6
+  ARCHIVED = 7
+  DELETED = 8
+
+  _VALUES_TO_NAMES = {
+1: "INITIATED",
+2: "START_REVIEW",
+3: "CHANGES_REQUIRED",
+4: "REVIEWED",
+5: "ENABLED",
+6: "DISABLED",
+7: "ARCHIVED",
+8: "DELETED",
+  }
+
+  _NAMES_TO_VALUES = {
+"INITIATED": 1,
+"START_REVIEW": 2,
+"CHANGES_REQUIRED": 3,
+"REVIEWED": 4,
+"ENABLED": 5,
+"DISABLED": 6,
+"ARCHIVED": 7,
+"DELETED": 8,
+  }
+
 class FunctionType:
   JAVA = 1
 
@@ -3053,6 +3147,10 @@ class SerDeInfo:
- name
- serializationLib
- parameters
+   - description
+   - serializerClass
+   - deserializerClass
+   - serdeType
   """
 
   thrift_spec = (
@@ -3060,12 +3158,20 @@ class SerDeInfo:
 (1, TType.STRING, 'name', None, None, ), # 1
 (2, TType.STRING, 'serializationLib', None, None, ), # 2
 (3, TType.MAP, 'parameters', (TType.STRING,None,TType.STRING,None), None, 
), # 3
+(4, TType.STRING, 'description', None, None, ), # 4
+(5, TType.STRING, 'serializerClass', None, None, ), # 5
+(6, TType.STRING, 'deserializerClass', None, None, ), # 6
+(7, TType.I32, 'serdeType', None, None, ), # 7
   )
 
-  def __init__(self, name=None, serializationLib=None, parameters=None,):
+  def __init__(self, name=None, serializationLib=None, parameters=None, 
description=None, serializerClass=None, deserializerClass=None, 
serdeType=None,):
 self.name = name
 self.serializationLib = serializationLib
 self.parameters = parameters
+self.description = description
+self.serializerClass = serializerClass
+self.deserializerClass = deserializerClass
+self.serdeType = serdeType
 
   def read(self, iprot):
 if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and 
isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is 
not None and fastbinary is not None:
@@ -3097,6 +3203,26 @@ class SerDeInfo:
   iprot.readMapEnd()
 else:
   iprot.skip(ftype)
+  elif fid == 4:
+if ftype == TType.STRING:
+  self.description = iprot.readString()
+else:
+  iprot.skip(ftype)
+  elif fid == 5:
+if ftype == TType.STRING:
+  self.serializerClass = iprot.readString()
+else:
+  iprot.skip(ftype)
+  elif fid == 6:
+if ftype == TType.STRING:
+  self.deserializerClass = iprot.readString()
+else:
+  iprot.skip(ftype)
+  elif fid == 7:
+if ftype == TType.I32:
+  self.serdeType = iprot.readI32()
+else:
+  iprot.skip(ftype)
   else:
 iprot.skip(ftype)
   iprot.readFieldEnd()
@@ -3123,6 +3249,22 @@ class SerDeInfo:
 oprot.writeString(viter100)
   oprot.writeMapEnd()
   oprot.writeFieldEnd()
+if self.description is not None:
+  oprot.writeFieldBegin('description', TType.STRING, 4)
+  oprot.writeString(self.description)
+  oprot.writeFieldEnd()
+if self.serializerClass is not None:
+  oprot.writeFieldBegin('serializerClass', TType.STRING, 5)
+  oprot.writeString(self.serializerClass)
+  oprot.writeFieldEnd()
+if self.deserializerClass is not None:
+  oprot.writeFieldBegin('deserializerClass', TType.STRING, 6)
+  oprot.writeString(self.deserializerClass)
+  oprot.writeFieldEnd()
+if self.serdeType is not None:
+  oprot.writeFieldBegin('serdeType', TType.I32, 7)
+

[09/18] hive git commit: HIVE-17990 Add Thrift and DB storage for Schema Registry objects (Alan Gates, reviewed by Thejas Nair)

2018-03-12 Thread gates
http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
--
diff --git 
a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
 
b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
index 8f3b848..8c5ceaf 100644
--- 
a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
+++ 
b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
@@ -402,6 +402,34 @@ import org.slf4j.LoggerFactory;
 
 public WMCreateOrDropTriggerToPoolMappingResponse 
create_or_drop_wm_trigger_to_pool_mapping(WMCreateOrDropTriggerToPoolMappingRequest
 request) throws AlreadyExistsException, NoSuchObjectException, 
InvalidObjectException, MetaException, org.apache.thrift.TException;
 
+public void create_ischema(ISchema schema) throws AlreadyExistsException, 
NoSuchObjectException, MetaException, org.apache.thrift.TException;
+
+public void alter_ischema(AlterISchemaRequest rqst) throws 
NoSuchObjectException, MetaException, org.apache.thrift.TException;
+
+public ISchema get_ischema(ISchemaName name) throws NoSuchObjectException, 
MetaException, org.apache.thrift.TException;
+
+public void drop_ischema(ISchemaName name) throws NoSuchObjectException, 
InvalidOperationException, MetaException, org.apache.thrift.TException;
+
+public void add_schema_version(SchemaVersion schemaVersion) throws 
AlreadyExistsException, NoSuchObjectException, MetaException, 
org.apache.thrift.TException;
+
+public SchemaVersion get_schema_version(SchemaVersionDescriptor 
schemaVersion) throws NoSuchObjectException, MetaException, 
org.apache.thrift.TException;
+
+public SchemaVersion get_schema_latest_version(ISchemaName schemaName) 
throws NoSuchObjectException, MetaException, org.apache.thrift.TException;
+
+public List get_schema_all_versions(ISchemaName schemaName) 
throws NoSuchObjectException, MetaException, org.apache.thrift.TException;
+
+public void drop_schema_version(SchemaVersionDescriptor schemaVersion) 
throws NoSuchObjectException, MetaException, org.apache.thrift.TException;
+
+public FindSchemasByColsResp get_schemas_by_cols(FindSchemasByColsRqst 
rqst) throws MetaException, org.apache.thrift.TException;
+
+public void map_schema_version_to_serde(MapSchemaVersionToSerdeRequest 
rqst) throws NoSuchObjectException, MetaException, org.apache.thrift.TException;
+
+public void set_schema_version_state(SetSchemaVersionStateRequest rqst) 
throws NoSuchObjectException, InvalidOperationException, MetaException, 
org.apache.thrift.TException;
+
+public void add_serde(SerDeInfo serde) throws AlreadyExistsException, 
MetaException, org.apache.thrift.TException;
+
+public SerDeInfo get_serde(GetSerdeRequest rqst) throws 
NoSuchObjectException, MetaException, org.apache.thrift.TException;
+
   }
 
   @org.apache.hadoop.classification.InterfaceAudience.Public 
@org.apache.hadoop.classification.InterfaceStability.Stable public interface 
AsyncIface extends com.facebook.fb303.FacebookService .AsyncIface {
@@ -766,6 +794,34 @@ import org.slf4j.LoggerFactory;
 
 public void 
create_or_drop_wm_trigger_to_pool_mapping(WMCreateOrDropTriggerToPoolMappingRequest
 request, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws 
org.apache.thrift.TException;
 
+public void create_ischema(ISchema schema, 
org.apache.thrift.async.AsyncMethodCallback resultHandler) throws 
org.apache.thrift.TException;
+
+public void alter_ischema(AlterISchemaRequest rqst, 
org.apache.thrift.async.AsyncMethodCallback resultHandler) throws 
org.apache.thrift.TException;
+
+public void get_ischema(ISchemaName name, 
org.apache.thrift.async.AsyncMethodCallback resultHandler) throws 
org.apache.thrift.TException;
+
+public void drop_ischema(ISchemaName name, 
org.apache.thrift.async.AsyncMethodCallback resultHandler) throws 
org.apache.thrift.TException;
+
+public void add_schema_version(SchemaVersion schemaVersion, 
org.apache.thrift.async.AsyncMethodCallback resultHandler) throws 
org.apache.thrift.TException;
+
+public void get_schema_version(SchemaVersionDescriptor schemaVersion, 
org.apache.thrift.async.AsyncMethodCallback resultHandler) throws 
org.apache.thrift.TException;
+
+public void get_schema_latest_version(ISchemaName schemaName, 
org.apache.thrift.async.AsyncMethodCallback resultHandler) throws 
org.apache.thrift.TException;
+
+public void get_schema_all_versions(ISchemaName schemaName, 
org.apache.thrift.async.AsyncMethodCallback resultHandler) throws 
org.apache.thrift.TException;
+
+public void drop_schema_version(SchemaVersionDescriptor schemaVersion, 

[17/18] hive git commit: HIVE-17990 Add Thrift and DB storage for Schema Registry objects (Alan Gates, reviewed by Thejas Nair)

2018-03-12 Thread gates
http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
--
diff --git 
a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp 
b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
index 32a7086..b7a3b92 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
@@ -1240,14 +1240,14 @@ uint32_t 
ThriftHiveMetastore_get_databases_result::read(::apache::thrift::protoc
 if (ftype == ::apache::thrift::protocol::T_LIST) {
   {
 this->success.clear();
-uint32_t _size1099;
-::apache::thrift::protocol::TType _etype1102;
-xfer += iprot->readListBegin(_etype1102, _size1099);
-this->success.resize(_size1099);
-uint32_t _i1103;
-for (_i1103 = 0; _i1103 < _size1099; ++_i1103)
+uint32_t _size1137;
+::apache::thrift::protocol::TType _etype1140;
+xfer += iprot->readListBegin(_etype1140, _size1137);
+this->success.resize(_size1137);
+uint32_t _i1141;
+for (_i1141 = 0; _i1141 < _size1137; ++_i1141)
 {
-  xfer += iprot->readString(this->success[_i1103]);
+  xfer += iprot->readString(this->success[_i1141]);
 }
 xfer += iprot->readListEnd();
   }
@@ -1286,10 +1286,10 @@ uint32_t 
ThriftHiveMetastore_get_databases_result::write(::apache::thrift::proto
 xfer += oprot->writeFieldBegin("success", 
::apache::thrift::protocol::T_LIST, 0);
 {
   xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, 
static_cast(this->success.size()));
-  std::vector ::const_iterator _iter1104;
-  for (_iter1104 = this->success.begin(); _iter1104 != 
this->success.end(); ++_iter1104)
+  std::vector ::const_iterator _iter1142;
+  for (_iter1142 = this->success.begin(); _iter1142 != 
this->success.end(); ++_iter1142)
   {
-xfer += oprot->writeString((*_iter1104));
+xfer += oprot->writeString((*_iter1142));
   }
   xfer += oprot->writeListEnd();
 }
@@ -1334,14 +1334,14 @@ uint32_t 
ThriftHiveMetastore_get_databases_presult::read(::apache::thrift::proto
 if (ftype == ::apache::thrift::protocol::T_LIST) {
   {
 (*(this->success)).clear();
-uint32_t _size1105;
-::apache::thrift::protocol::TType _etype1108;
-xfer += iprot->readListBegin(_etype1108, _size1105);
-(*(this->success)).resize(_size1105);
-uint32_t _i1109;
-for (_i1109 = 0; _i1109 < _size1105; ++_i1109)
+uint32_t _size1143;
+::apache::thrift::protocol::TType _etype1146;
+xfer += iprot->readListBegin(_etype1146, _size1143);
+(*(this->success)).resize(_size1143);
+uint32_t _i1147;
+for (_i1147 = 0; _i1147 < _size1143; ++_i1147)
 {
-  xfer += iprot->readString((*(this->success))[_i1109]);
+  xfer += iprot->readString((*(this->success))[_i1147]);
 }
 xfer += iprot->readListEnd();
   }
@@ -1458,14 +1458,14 @@ uint32_t 
ThriftHiveMetastore_get_all_databases_result::read(::apache::thrift::pr
 if (ftype == ::apache::thrift::protocol::T_LIST) {
   {
 this->success.clear();
-uint32_t _size1110;
-::apache::thrift::protocol::TType _etype1113;
-xfer += iprot->readListBegin(_etype1113, _size1110);
-this->success.resize(_size1110);
-uint32_t _i1114;
-for (_i1114 = 0; _i1114 < _size1110; ++_i1114)
+uint32_t _size1148;
+::apache::thrift::protocol::TType _etype1151;
+xfer += iprot->readListBegin(_etype1151, _size1148);
+this->success.resize(_size1148);
+uint32_t _i1152;
+for (_i1152 = 0; _i1152 < _size1148; ++_i1152)
 {
-  xfer += iprot->readString(this->success[_i1114]);
+  xfer += iprot->readString(this->success[_i1152]);
 }
 xfer += iprot->readListEnd();
   }
@@ -1504,10 +1504,10 @@ uint32_t 
ThriftHiveMetastore_get_all_databases_result::write(::apache::thrift::p
 xfer += oprot->writeFieldBegin("success", 
::apache::thrift::protocol::T_LIST, 0);
 {
   xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, 
static_cast(this->success.size()));
-  std::vector ::const_iterator _iter1115;
-  for (_iter1115 = this->success.begin(); _iter1115 != 
this->success.end(); ++_iter1115)
+  std::vector ::const_iterator _iter1153;
+  for (_iter1153 = this->success.begin(); _iter1153 != 
this->success.end(); ++_iter1153)
   {
-xfer += 

[04/18] hive git commit: HIVE-17990 Add Thrift and DB storage for Schema Registry objects (Alan Gates, reviewed by Thejas Nair)

2018-03-12 Thread gates
http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
--
diff --git 
a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
 
b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index c99c3af..db09407 100644
--- 
a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ 
b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -53,6 +53,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
+import java.util.function.Consumer;
 import java.util.regex.Pattern;
 
 import javax.jdo.JDOException;
@@ -79,33 +80,47 @@ import 
org.apache.hadoop.hive.metastore.events.AddPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.AddPrimaryKeyEvent;
 import org.apache.hadoop.hive.metastore.events.AddUniqueConstraintEvent;
 import org.apache.hadoop.hive.metastore.events.AlterDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.AlterISchemaEvent;
 import org.apache.hadoop.hive.metastore.events.AlterPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.AlterSchemaVersionEvent;
 import org.apache.hadoop.hive.metastore.events.AlterTableEvent;
 import org.apache.hadoop.hive.metastore.events.ConfigChangeEvent;
 import org.apache.hadoop.hive.metastore.events.CreateDatabaseEvent;
 import org.apache.hadoop.hive.metastore.events.CreateFunctionEvent;
+import org.apache.hadoop.hive.metastore.events.CreateISchemaEvent;
+import org.apache.hadoop.hive.metastore.events.AddSchemaVersionEvent;
 import org.apache.hadoop.hive.metastore.events.CreateTableEvent;
 import org.apache.hadoop.hive.metastore.events.DropConstraintEvent;
 import org.apache.hadoop.hive.metastore.events.DropDatabaseEvent;
 import org.apache.hadoop.hive.metastore.events.DropFunctionEvent;
+import org.apache.hadoop.hive.metastore.events.DropISchemaEvent;
 import org.apache.hadoop.hive.metastore.events.DropPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.DropSchemaVersionEvent;
 import org.apache.hadoop.hive.metastore.events.DropTableEvent;
 import org.apache.hadoop.hive.metastore.events.InsertEvent;
 import org.apache.hadoop.hive.metastore.events.LoadPartitionDoneEvent;
 import org.apache.hadoop.hive.metastore.events.PreAddPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.PreAlterDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.PreAlterISchemaEvent;
 import org.apache.hadoop.hive.metastore.events.PreAlterPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.PreAlterSchemaVersionEvent;
 import org.apache.hadoop.hive.metastore.events.PreAlterTableEvent;
 import org.apache.hadoop.hive.metastore.events.PreAuthorizationCallEvent;
 import org.apache.hadoop.hive.metastore.events.PreCreateDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.PreCreateISchemaEvent;
+import org.apache.hadoop.hive.metastore.events.PreAddSchemaVersionEvent;
 import org.apache.hadoop.hive.metastore.events.PreCreateTableEvent;
 import org.apache.hadoop.hive.metastore.events.PreDropDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.PreDropISchemaEvent;
 import org.apache.hadoop.hive.metastore.events.PreDropPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.PreDropSchemaVersionEvent;
 import org.apache.hadoop.hive.metastore.events.PreDropTableEvent;
 import org.apache.hadoop.hive.metastore.events.PreEventContext;
 import org.apache.hadoop.hive.metastore.events.PreLoadPartitionDoneEvent;
 import org.apache.hadoop.hive.metastore.events.PreReadDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.PreReadISchemaEvent;
 import org.apache.hadoop.hive.metastore.events.PreReadTableEvent;
+import org.apache.hadoop.hive.metastore.events.PreReadhSchemaVersionEvent;
 import org.apache.hadoop.hive.metastore.messaging.EventMessage.EventType;
 import org.apache.hadoop.hive.metastore.metrics.JvmPauseMonitor;
 import org.apache.hadoop.hive.metastore.metrics.Metrics;
@@ -7106,7 +7121,6 @@ public class HiveMetaStore extends ThriftHiveMetastore {
   }
 }
 
-
 @Override
 public WMCreateResourcePlanResponse 
create_resource_plan(WMCreateResourcePlanRequest request)
 throws AlreadyExistsException, InvalidObjectException, MetaException, 
TException {
@@ -7254,7 +7268,8 @@ public class HiveMetaStore extends ThriftHiveMetastore {
 WMGetTriggersForResourePlanRequest request)
 throws NoSuchObjectException, MetaException, TException {
   try {
-List triggers = 
getMS().getTriggersForResourcePlan(request.getResourcePlanName());
+List triggers =
+

[14/18] hive git commit: HIVE-17990 Add Thrift and DB storage for Schema Registry objects (Alan Gates, reviewed by Thejas Nair)

2018-03-12 Thread gates
http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
--
diff --git 
a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp 
b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
index 244b7ab..4a7c6bf 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
@@ -149,6 +149,72 @@ const char* _kEventRequestTypeNames[] = {
 };
 const std::map 
_EventRequestType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(3, 
_kEventRequestTypeValues, _kEventRequestTypeNames), 
::apache::thrift::TEnumIterator(-1, NULL, NULL));
 
+int _kSerdeTypeValues[] = {
+  SerdeType::HIVE,
+  SerdeType::SCHEMA_REGISTRY
+};
+const char* _kSerdeTypeNames[] = {
+  "HIVE",
+  "SCHEMA_REGISTRY"
+};
+const std::map 
_SerdeType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(2, 
_kSerdeTypeValues, _kSerdeTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, 
NULL));
+
+int _kSchemaTypeValues[] = {
+  SchemaType::HIVE,
+  SchemaType::AVRO
+};
+const char* _kSchemaTypeNames[] = {
+  "HIVE",
+  "AVRO"
+};
+const std::map 
_SchemaType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(2, 
_kSchemaTypeValues, _kSchemaTypeNames), ::apache::thrift::TEnumIterator(-1, 
NULL, NULL));
+
+int _kSchemaCompatibilityValues[] = {
+  SchemaCompatibility::NONE,
+  SchemaCompatibility::BACKWARD,
+  SchemaCompatibility::FORWARD,
+  SchemaCompatibility::BOTH
+};
+const char* _kSchemaCompatibilityNames[] = {
+  "NONE",
+  "BACKWARD",
+  "FORWARD",
+  "BOTH"
+};
+const std::map 
_SchemaCompatibility_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(4, 
_kSchemaCompatibilityValues, _kSchemaCompatibilityNames), 
::apache::thrift::TEnumIterator(-1, NULL, NULL));
+
+int _kSchemaValidationValues[] = {
+  SchemaValidation::LATEST,
+  SchemaValidation::ALL
+};
+const char* _kSchemaValidationNames[] = {
+  "LATEST",
+  "ALL"
+};
+const std::map 
_SchemaValidation_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(2, 
_kSchemaValidationValues, _kSchemaValidationNames), 
::apache::thrift::TEnumIterator(-1, NULL, NULL));
+
+int _kSchemaVersionStateValues[] = {
+  SchemaVersionState::INITIATED,
+  SchemaVersionState::START_REVIEW,
+  SchemaVersionState::CHANGES_REQUIRED,
+  SchemaVersionState::REVIEWED,
+  SchemaVersionState::ENABLED,
+  SchemaVersionState::DISABLED,
+  SchemaVersionState::ARCHIVED,
+  SchemaVersionState::DELETED
+};
+const char* _kSchemaVersionStateNames[] = {
+  "INITIATED",
+  "START_REVIEW",
+  "CHANGES_REQUIRED",
+  "REVIEWED",
+  "ENABLED",
+  "DISABLED",
+  "ARCHIVED",
+  "DELETED"
+};
+const std::map 
_SchemaVersionState_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(8, 
_kSchemaVersionStateValues, _kSchemaVersionStateNames), 
::apache::thrift::TEnumIterator(-1, NULL, NULL));
+
 int _kFunctionTypeValues[] = {
   FunctionType::JAVA
 };
@@ -4235,6 +4301,26 @@ void SerDeInfo::__set_parameters(const 
std::map & val)
   this->parameters = val;
 }
 
+void SerDeInfo::__set_description(const std::string& val) {
+  this->description = val;
+__isset.description = true;
+}
+
+void SerDeInfo::__set_serializerClass(const std::string& val) {
+  this->serializerClass = val;
+__isset.serializerClass = true;
+}
+
+void SerDeInfo::__set_deserializerClass(const std::string& val) {
+  this->deserializerClass = val;
+__isset.deserializerClass = true;
+}
+
+void SerDeInfo::__set_serdeType(const SerdeType::type val) {
+  this->serdeType = val;
+__isset.serdeType = true;
+}
+
 uint32_t SerDeInfo::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
@@ -4295,6 +4381,40 @@ uint32_t 
SerDeInfo::read(::apache::thrift::protocol::TProtocol* iprot) {
   xfer += iprot->skip(ftype);
 }
 break;
+  case 4:
+if (ftype == ::apache::thrift::protocol::T_STRING) {
+  xfer += iprot->readString(this->description);
+  this->__isset.description = true;
+} else {
+  xfer += iprot->skip(ftype);
+}
+break;
+  case 5:
+if (ftype == ::apache::thrift::protocol::T_STRING) {
+  xfer += iprot->readString(this->serializerClass);
+  this->__isset.serializerClass = true;
+} else {
+  xfer += iprot->skip(ftype);
+}
+break;
+  case 6:
+if (ftype == ::apache::thrift::protocol::T_STRING) {
+  xfer += iprot->readString(this->deserializerClass);
+  this->__isset.deserializerClass = true;
+} else {
+  xfer += iprot->skip(ftype);
+}
+break;
+  case 7:
+if (ftype == ::apache::thrift::protocol::T_I32) {
+  int32_t ecast146;
+ 

[03/18] hive git commit: HIVE-17990 Add Thrift and DB storage for Schema Registry objects (Alan Gates, reviewed by Thejas Nair)

2018-03-12 Thread gates
http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SchemaVersionBuilder.java
--
diff --git 
a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SchemaVersionBuilder.java
 
b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SchemaVersionBuilder.java
new file mode 100644
index 000..ceb0f49
--- /dev/null
+++ 
b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SchemaVersionBuilder.java
@@ -0,0 +1,108 @@
+/*
+ * 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.client.builder;
+
+import org.apache.hadoop.hive.metastore.api.ISchema;
+import org.apache.hadoop.hive.metastore.api.ISchemaName;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.SchemaVersion;
+import org.apache.hadoop.hive.metastore.api.SchemaVersionState;
+
+public class SchemaVersionBuilder extends 
SerdeAndColsBuilder {
+  private String schemaName, dbName; // required
+  private int version; // required
+  private long createdAt; // required
+  private SchemaVersionState state; // optional
+  private String description; // optional
+  private String schemaText; // optional
+  private String fingerprint; // optional
+  private String name; // optional
+
+  public SchemaVersionBuilder() {
+createdAt = System.currentTimeMillis() / 1000;
+version = -1;
+super.setChild(this);
+  }
+
+  public SchemaVersionBuilder setSchemaName(String schemaName) {
+this.schemaName = schemaName;
+return this;
+  }
+
+  public SchemaVersionBuilder setDbName(String dbName) {
+this.dbName = dbName;
+return this;
+  }
+
+  public SchemaVersionBuilder versionOf(ISchema schema) {
+this.dbName = schema.getDbName();
+this.schemaName = schema.getName();
+return this;
+  }
+
+  public SchemaVersionBuilder setVersion(int version) {
+this.version = version;
+return this;
+  }
+
+  public SchemaVersionBuilder setCreatedAt(long createdAt) {
+this.createdAt = createdAt;
+return this;
+  }
+
+  public SchemaVersionBuilder setState(
+  SchemaVersionState state) {
+this.state = state;
+return this;
+  }
+
+  public SchemaVersionBuilder setDescription(String description) {
+this.description = description;
+return this;
+  }
+
+  public SchemaVersionBuilder setSchemaText(String schemaText) {
+this.schemaText = schemaText;
+return this;
+  }
+
+  public SchemaVersionBuilder setFingerprint(String fingerprint) {
+this.fingerprint = fingerprint;
+return this;
+  }
+
+  public SchemaVersionBuilder setName(String name) {
+this.name = name;
+return this;
+  }
+
+  public SchemaVersion build() throws MetaException {
+if (schemaName == null || dbName == null || version < 0) {
+  throw new MetaException("You must provide the database name, schema 
name, and schema version");
+}
+SchemaVersion schemaVersion =
+new SchemaVersion(new ISchemaName(dbName, schemaName), version, 
createdAt, getCols());
+if (state != null) schemaVersion.setState(state);
+if (description != null) schemaVersion.setDescription(description);
+if (schemaText != null) schemaVersion.setSchemaText(schemaText);
+if (fingerprint != null) schemaVersion.setFingerprint(fingerprint);
+if (name != null) schemaVersion.setName(name);
+schemaVersion.setSerDe(buildSerde());
+return schemaVersion;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SerdeAndColsBuilder.java
--
diff --git 
a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SerdeAndColsBuilder.java
 
b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/client/builder/SerdeAndColsBuilder.java
new file mode 100644
index 000..e1405d3
--- /dev/null
+++ 

[11/18] hive git commit: HIVE-17990 Add Thrift and DB storage for Schema Registry objects (Alan Gates, reviewed by Thejas Nair)

2018-03-12 Thread gates
http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java
--
diff --git 
a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java
 
b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java
new file mode 100644
index 000..fb0be40
--- /dev/null
+++ 
b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java
@@ -0,0 +1,1412 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public 
@org.apache.hadoop.classification.InterfaceStability.Stable public class 
SchemaVersion implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, 
Comparable {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new 
org.apache.thrift.protocol.TStruct("SchemaVersion");
+
+  private static final org.apache.thrift.protocol.TField SCHEMA_FIELD_DESC = 
new org.apache.thrift.protocol.TField("schema", 
org.apache.thrift.protocol.TType.STRUCT, (short)1);
+  private static final org.apache.thrift.protocol.TField VERSION_FIELD_DESC = 
new org.apache.thrift.protocol.TField("version", 
org.apache.thrift.protocol.TType.I32, (short)2);
+  private static final org.apache.thrift.protocol.TField CREATED_AT_FIELD_DESC 
= new org.apache.thrift.protocol.TField("createdAt", 
org.apache.thrift.protocol.TType.I64, (short)3);
+  private static final org.apache.thrift.protocol.TField COLS_FIELD_DESC = new 
org.apache.thrift.protocol.TField("cols", 
org.apache.thrift.protocol.TType.LIST, (short)4);
+  private static final org.apache.thrift.protocol.TField STATE_FIELD_DESC = 
new org.apache.thrift.protocol.TField("state", 
org.apache.thrift.protocol.TType.I32, (short)5);
+  private static final org.apache.thrift.protocol.TField 
DESCRIPTION_FIELD_DESC = new org.apache.thrift.protocol.TField("description", 
org.apache.thrift.protocol.TType.STRING, (short)6);
+  private static final org.apache.thrift.protocol.TField 
SCHEMA_TEXT_FIELD_DESC = new org.apache.thrift.protocol.TField("schemaText", 
org.apache.thrift.protocol.TType.STRING, (short)7);
+  private static final org.apache.thrift.protocol.TField 
FINGERPRINT_FIELD_DESC = new org.apache.thrift.protocol.TField("fingerprint", 
org.apache.thrift.protocol.TType.STRING, (short)8);
+  private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new 
org.apache.thrift.protocol.TField("name", 
org.apache.thrift.protocol.TType.STRING, (short)9);
+  private static final org.apache.thrift.protocol.TField SER_DE_FIELD_DESC = 
new org.apache.thrift.protocol.TField("serDe", 
org.apache.thrift.protocol.TType.STRUCT, (short)10);
+
+  private static final Map schemes = 
new HashMap();
+  static {
+schemes.put(StandardScheme.class, new 
SchemaVersionStandardSchemeFactory());
+schemes.put(TupleScheme.class, new SchemaVersionTupleSchemeFactory());
+  }
+
+  private ISchemaName schema; // required
+  private int version; // required
+  private long createdAt; // required
+  private List cols; // required
+  private SchemaVersionState state; // optional
+  private String description; // optional
+  private String schemaText; // optional
+  private String fingerprint; // optional
+  private String name; // optional
+  private SerDeInfo serDe; // optional
+
+  /** The set of fields this struct contains, along with convenience methods 
for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+SCHEMA((short)1, "schema"),

[18/18] hive git commit: HIVE-17990 Add Thrift and DB storage for Schema Registry objects (Alan Gates, reviewed by Thejas Nair)

2018-03-12 Thread gates
HIVE-17990 Add Thrift and DB storage for Schema Registry objects (Alan Gates, 
reviewed by Thejas Nair)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/12041d39
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/12041d39
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/12041d39

Branch: refs/heads/master
Commit: 12041d39f052dc8e4858815da15c967cb378fae9
Parents: d717d38
Author: Alan Gates 
Authored: Mon Mar 12 16:54:46 2018 -0700
Committer: Alan Gates 
Committed: Mon Mar 12 16:54:46 2018 -0700

--
 .../listener/DummyRawStoreFailEvent.java|75 +
 standalone-metastore/pom.xml| 3 +-
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 35798 ++---
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h|  2378 +-
 .../ThriftHiveMetastore_server.skeleton.cpp |70 +
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp |  6134 +--
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |   729 +-
 .../hive/metastore/api/AlterISchemaRequest.java |   509 +
 .../metastore/api/FindSchemasByColsResp.java|   449 +
 .../metastore/api/FindSchemasByColsRqst.java|   605 +
 .../hive/metastore/api/GetSerdeRequest.java |   395 +
 .../hadoop/hive/metastore/api/ISchema.java  |  1162 +
 .../hadoop/hive/metastore/api/ISchemaName.java  |   499 +
 .../api/MapSchemaVersionToSerdeRequest.java |   504 +
 .../hive/metastore/api/SchemaCompatibility.java |51 +
 .../hadoop/hive/metastore/api/SchemaType.java   |45 +
 .../hive/metastore/api/SchemaValidation.java|45 +
 .../hive/metastore/api/SchemaVersion.java   |  1412 +
 .../metastore/api/SchemaVersionDescriptor.java  |   502 +
 .../hive/metastore/api/SchemaVersionState.java  |63 +
 .../hadoop/hive/metastore/api/SerDeInfo.java|   443 +-
 .../hadoop/hive/metastore/api/SerdeType.java|45 +
 .../api/SetSchemaVersionStateRequest.java   |   516 +
 .../hive/metastore/api/ThriftHiveMetastore.java | 17889 +++-
 .../gen-php/metastore/ThriftHiveMetastore.php   | 27954 +++--
 .../src/gen/thrift/gen-php/metastore/Types.php  |  1523 +
 .../hive_metastore/ThriftHiveMetastore-remote   |98 +
 .../hive_metastore/ThriftHiveMetastore.py   |  5189 ++-
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |  1123 +-
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |   277 +-
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |   915 +
 .../hadoop/hive/metastore/HiveMetaStore.java|   472 +-
 .../hive/metastore/HiveMetaStoreClient.java |72 +
 .../hadoop/hive/metastore/IMetaStoreClient.java |   167 +
 .../hive/metastore/MetaStoreEventListener.java  |26 +
 .../metastore/MetaStoreListenerNotifier.java|42 +
 .../hadoop/hive/metastore/ObjectStore.java  |   426 +-
 .../apache/hadoop/hive/metastore/RawStore.java  |   135 +
 .../hive/metastore/cache/CachedStore.java   |77 +
 .../client/builder/DatabaseBuilder.java | 2 +-
 .../client/builder/ISchemaBuilder.java  |94 +
 .../client/builder/SchemaVersionBuilder.java|   108 +
 .../client/builder/SerdeAndColsBuilder.java |   124 +
 .../builder/StorageDescriptorBuilder.java   |57 +-
 .../metastore/events/AddSchemaVersionEvent.java |40 +
 .../metastore/events/AlterISchemaEvent.java |45 +
 .../events/AlterSchemaVersionEvent.java |46 +
 .../metastore/events/CreateISchemaEvent.java|39 +
 .../hive/metastore/events/DropISchemaEvent.java |39 +
 .../events/DropSchemaVersionEvent.java  |40 +
 .../events/PreAddSchemaVersionEvent.java|39 +
 .../metastore/events/PreAlterISchemaEvent.java  |44 +
 .../events/PreAlterSchemaVersionEvent.java  |45 +
 .../metastore/events/PreCreateISchemaEvent.java |39 +
 .../metastore/events/PreDropISchemaEvent.java   |39 +
 .../events/PreDropSchemaVersionEvent.java   |39 +
 .../hive/metastore/events/PreEventContext.java  |10 +-
 .../metastore/events/PreReadISchemaEvent.java   |39 +
 .../events/PreReadhSchemaVersionEvent.java  |36 +
 .../hive/metastore/messaging/EventMessage.java  | 8 +-
 .../metastore/messaging/MessageFactory.java | 7 +
 .../hadoop/hive/metastore/model/MISchema.java   |   107 +
 .../hive/metastore/model/MSchemaVersion.java|   127 +
 .../hadoop/hive/metastore/model/MSerDeInfo.java |48 +-
 .../hive/metastore/utils/MetaStoreUtils.java| 1 +
 .../main/resources/datanucleus-log4j.properties |17 +
 .../src/main/resources/package.jdo  |77 +
 .../main/sql/derby/hive-schema-3.0.0.derby.sql  |30 +-
 .../sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql  |34 +
 .../main/sql/mssql/hive-schema-3.0.0.mssql.sql  |33 +-
 .../sql/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql  |33 +
 .../main/sql/mysql/hive-schema-3.0.0.mysql.sql  |36 +

[16/18] hive git commit: HIVE-17990 Add Thrift and DB storage for Schema Registry objects (Alan Gates, reviewed by Thejas Nair)

2018-03-12 Thread gates
http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
--
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h 
b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
index 0f9ba5d..7206e29 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
@@ -202,6 +202,20 @@ class ThriftHiveMetastoreIf : virtual public  
::facebook::fb303::FacebookService
   virtual void create_or_update_wm_mapping(WMCreateOrUpdateMappingResponse& 
_return, const WMCreateOrUpdateMappingRequest& request) = 0;
   virtual void drop_wm_mapping(WMDropMappingResponse& _return, const 
WMDropMappingRequest& request) = 0;
   virtual void 
create_or_drop_wm_trigger_to_pool_mapping(WMCreateOrDropTriggerToPoolMappingResponse&
 _return, const WMCreateOrDropTriggerToPoolMappingRequest& request) = 0;
+  virtual void create_ischema(const ISchema& schema) = 0;
+  virtual void alter_ischema(const AlterISchemaRequest& rqst) = 0;
+  virtual void get_ischema(ISchema& _return, const ISchemaName& name) = 0;
+  virtual void drop_ischema(const ISchemaName& name) = 0;
+  virtual void add_schema_version(const SchemaVersion& schemaVersion) = 0;
+  virtual void get_schema_version(SchemaVersion& _return, const 
SchemaVersionDescriptor& schemaVersion) = 0;
+  virtual void get_schema_latest_version(SchemaVersion& _return, const 
ISchemaName& schemaName) = 0;
+  virtual void get_schema_all_versions(std::vector & _return, 
const ISchemaName& schemaName) = 0;
+  virtual void drop_schema_version(const SchemaVersionDescriptor& 
schemaVersion) = 0;
+  virtual void get_schemas_by_cols(FindSchemasByColsResp& _return, const 
FindSchemasByColsRqst& rqst) = 0;
+  virtual void map_schema_version_to_serde(const 
MapSchemaVersionToSerdeRequest& rqst) = 0;
+  virtual void set_schema_version_state(const SetSchemaVersionStateRequest& 
rqst) = 0;
+  virtual void add_serde(const SerDeInfo& serde) = 0;
+  virtual void get_serde(SerDeInfo& _return, const GetSerdeRequest& rqst) = 0;
 };
 
 class ThriftHiveMetastoreIfFactory : virtual public  
::facebook::fb303::FacebookServiceIfFactory {
@@ -798,6 +812,48 @@ class ThriftHiveMetastoreNull : virtual public 
ThriftHiveMetastoreIf , virtual p
   void 
create_or_drop_wm_trigger_to_pool_mapping(WMCreateOrDropTriggerToPoolMappingResponse&
 /* _return */, const WMCreateOrDropTriggerToPoolMappingRequest& /* request */) 
{
 return;
   }
+  void create_ischema(const ISchema& /* schema */) {
+return;
+  }
+  void alter_ischema(const AlterISchemaRequest& /* rqst */) {
+return;
+  }
+  void get_ischema(ISchema& /* _return */, const ISchemaName& /* name */) {
+return;
+  }
+  void drop_ischema(const ISchemaName& /* name */) {
+return;
+  }
+  void add_schema_version(const SchemaVersion& /* schemaVersion */) {
+return;
+  }
+  void get_schema_version(SchemaVersion& /* _return */, const 
SchemaVersionDescriptor& /* schemaVersion */) {
+return;
+  }
+  void get_schema_latest_version(SchemaVersion& /* _return */, const 
ISchemaName& /* schemaName */) {
+return;
+  }
+  void get_schema_all_versions(std::vector & /* _return */, 
const ISchemaName& /* schemaName */) {
+return;
+  }
+  void drop_schema_version(const SchemaVersionDescriptor& /* schemaVersion */) 
{
+return;
+  }
+  void get_schemas_by_cols(FindSchemasByColsResp& /* _return */, const 
FindSchemasByColsRqst& /* rqst */) {
+return;
+  }
+  void map_schema_version_to_serde(const MapSchemaVersionToSerdeRequest& /* 
rqst */) {
+return;
+  }
+  void set_schema_version_state(const SetSchemaVersionStateRequest& /* rqst 
*/) {
+return;
+  }
+  void add_serde(const SerDeInfo& /* serde */) {
+return;
+  }
+  void get_serde(SerDeInfo& /* _return */, const GetSerdeRequest& /* rqst */) {
+return;
+  }
 };
 
 typedef struct _ThriftHiveMetastore_getMetaConf_args__isset {
@@ -22973,228 +23029,1868 @@ class 
ThriftHiveMetastore_create_or_drop_wm_trigger_to_pool_mapping_presult {
 
 };
 
-class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public 
 ::facebook::fb303::FacebookServiceClient {
+typedef struct _ThriftHiveMetastore_create_ischema_args__isset {
+  _ThriftHiveMetastore_create_ischema_args__isset() : schema(false) {}
+  bool schema :1;
+} _ThriftHiveMetastore_create_ischema_args__isset;
+
+class ThriftHiveMetastore_create_ischema_args {
  public:
-  ThriftHiveMetastoreClient(boost::shared_ptr< 
::apache::thrift::protocol::TProtocol> prot) :
- ::facebook::fb303::FacebookServiceClient(prot, prot) {}
-  ThriftHiveMetastoreClient(boost::shared_ptr< 
::apache::thrift::protocol::TProtocol> iprot, boost::shared_ptr< 
::apache::thrift::protocol::TProtocol> oprot) : 
::facebook::fb303::FacebookServiceClient(iprot, oprot) {}
-  boost::shared_ptr< 

[07/18] hive git commit: HIVE-17990 Add Thrift and DB storage for Schema Registry objects (Alan Gates, reviewed by Thejas Nair)

2018-03-12 Thread gates
http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
--
diff --git a/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php 
b/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
index 0b11532..3be16de 100644
--- a/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
+++ b/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
@@ -142,6 +142,67 @@ final class EventRequestType {
   );
 }
 
+final class SerdeType {
+  const HIVE = 1;
+  const SCHEMA_REGISTRY = 2;
+  static public $__names = array(
+1 => 'HIVE',
+2 => 'SCHEMA_REGISTRY',
+  );
+}
+
+final class SchemaType {
+  const HIVE = 1;
+  const AVRO = 2;
+  static public $__names = array(
+1 => 'HIVE',
+2 => 'AVRO',
+  );
+}
+
+final class SchemaCompatibility {
+  const NONE = 1;
+  const BACKWARD = 2;
+  const FORWARD = 3;
+  const BOTH = 4;
+  static public $__names = array(
+1 => 'NONE',
+2 => 'BACKWARD',
+3 => 'FORWARD',
+4 => 'BOTH',
+  );
+}
+
+final class SchemaValidation {
+  const LATEST = 1;
+  const ALL = 2;
+  static public $__names = array(
+1 => 'LATEST',
+2 => 'ALL',
+  );
+}
+
+final class SchemaVersionState {
+  const INITIATED = 1;
+  const START_REVIEW = 2;
+  const CHANGES_REQUIRED = 3;
+  const REVIEWED = 4;
+  const ENABLED = 5;
+  const DISABLED = 6;
+  const ARCHIVED = 7;
+  const DELETED = 8;
+  static public $__names = array(
+1 => 'INITIATED',
+2 => 'START_REVIEW',
+3 => 'CHANGES_REQUIRED',
+4 => 'REVIEWED',
+5 => 'ENABLED',
+6 => 'DISABLED',
+7 => 'ARCHIVED',
+8 => 'DELETED',
+  );
+}
+
 final class FunctionType {
   const JAVA = 1;
   static public $__names = array(
@@ -4266,6 +4327,22 @@ class SerDeInfo {
* @var array
*/
   public $parameters = null;
+  /**
+   * @var string
+   */
+  public $description = null;
+  /**
+   * @var string
+   */
+  public $serializerClass = null;
+  /**
+   * @var string
+   */
+  public $deserializerClass = null;
+  /**
+   * @var int
+   */
+  public $serdeType = null;
 
   public function __construct($vals=null) {
 if (!isset(self::$_TSPEC)) {
@@ -4290,6 +4367,22 @@ class SerDeInfo {
 'type' => TType::STRING,
 ),
   ),
+4 => array(
+  'var' => 'description',
+  'type' => TType::STRING,
+  ),
+5 => array(
+  'var' => 'serializerClass',
+  'type' => TType::STRING,
+  ),
+6 => array(
+  'var' => 'deserializerClass',
+  'type' => TType::STRING,
+  ),
+7 => array(
+  'var' => 'serdeType',
+  'type' => TType::I32,
+  ),
 );
 }
 if (is_array($vals)) {
@@ -4302,6 +4395,18 @@ class SerDeInfo {
   if (isset($vals['parameters'])) {
 $this->parameters = $vals['parameters'];
   }
+  if (isset($vals['description'])) {
+$this->description = $vals['description'];
+  }
+  if (isset($vals['serializerClass'])) {
+$this->serializerClass = $vals['serializerClass'];
+  }
+  if (isset($vals['deserializerClass'])) {
+$this->deserializerClass = $vals['deserializerClass'];
+  }
+  if (isset($vals['serdeType'])) {
+$this->serdeType = $vals['serdeType'];
+  }
 }
   }
 
@@ -4358,6 +4463,34 @@ class SerDeInfo {
 $xfer += $input->skip($ftype);
   }
   break;
+case 4:
+  if ($ftype == TType::STRING) {
+$xfer += $input->readString($this->description);
+  } else {
+$xfer += $input->skip($ftype);
+  }
+  break;
+case 5:
+  if ($ftype == TType::STRING) {
+$xfer += $input->readString($this->serializerClass);
+  } else {
+$xfer += $input->skip($ftype);
+  }
+  break;
+case 6:
+  if ($ftype == TType::STRING) {
+$xfer += $input->readString($this->deserializerClass);
+  } else {
+$xfer += $input->skip($ftype);
+  }
+  break;
+case 7:
+  if ($ftype == TType::I32) {
+$xfer += $input->readI32($this->serdeType);
+  } else {
+$xfer += $input->skip($ftype);
+  }
+  break;
 default:
   $xfer += $input->skip($ftype);
   break;
@@ -4399,6 +4532,26 @@ class SerDeInfo {
   }
   $xfer += $output->writeFieldEnd();
 }
+if ($this->description !== null) {
+  $xfer += $output->writeFieldBegin('description', TType::STRING, 4);
+  $xfer += $output->writeString($this->description);
+  $xfer += $output->writeFieldEnd();
+}
+if ($this->serializerClass !== null) {
+  $xfer += $output->writeFieldBegin('serializerClass', TType::STRING, 5);
+  $xfer += $output->writeString($this->serializerClass);

[15/18] hive git commit: HIVE-17990 Add Thrift and DB storage for Schema Registry objects (Alan Gates, reviewed by Thejas Nair)

2018-03-12 Thread gates
http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
--
diff --git 
a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
 
b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
index 0290b7d..8d9ad25 100644
--- 
a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
+++ 
b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
@@ -922,6 +922,76 @@ class ThriftHiveMetastoreHandler : virtual public 
ThriftHiveMetastoreIf {
 printf("create_or_drop_wm_trigger_to_pool_mapping\n");
   }
 
+  void create_ischema(const ISchema& schema) {
+// Your implementation goes here
+printf("create_ischema\n");
+  }
+
+  void alter_ischema(const AlterISchemaRequest& rqst) {
+// Your implementation goes here
+printf("alter_ischema\n");
+  }
+
+  void get_ischema(ISchema& _return, const ISchemaName& name) {
+// Your implementation goes here
+printf("get_ischema\n");
+  }
+
+  void drop_ischema(const ISchemaName& name) {
+// Your implementation goes here
+printf("drop_ischema\n");
+  }
+
+  void add_schema_version(const SchemaVersion& schemaVersion) {
+// Your implementation goes here
+printf("add_schema_version\n");
+  }
+
+  void get_schema_version(SchemaVersion& _return, const 
SchemaVersionDescriptor& schemaVersion) {
+// Your implementation goes here
+printf("get_schema_version\n");
+  }
+
+  void get_schema_latest_version(SchemaVersion& _return, const ISchemaName& 
schemaName) {
+// Your implementation goes here
+printf("get_schema_latest_version\n");
+  }
+
+  void get_schema_all_versions(std::vector & _return, const 
ISchemaName& schemaName) {
+// Your implementation goes here
+printf("get_schema_all_versions\n");
+  }
+
+  void drop_schema_version(const SchemaVersionDescriptor& schemaVersion) {
+// Your implementation goes here
+printf("drop_schema_version\n");
+  }
+
+  void get_schemas_by_cols(FindSchemasByColsResp& _return, const 
FindSchemasByColsRqst& rqst) {
+// Your implementation goes here
+printf("get_schemas_by_cols\n");
+  }
+
+  void map_schema_version_to_serde(const MapSchemaVersionToSerdeRequest& rqst) 
{
+// Your implementation goes here
+printf("map_schema_version_to_serde\n");
+  }
+
+  void set_schema_version_state(const SetSchemaVersionStateRequest& rqst) {
+// Your implementation goes here
+printf("set_schema_version_state\n");
+  }
+
+  void add_serde(const SerDeInfo& serde) {
+// Your implementation goes here
+printf("add_serde\n");
+  }
+
+  void get_serde(SerDeInfo& _return, const GetSerdeRequest& rqst) {
+// Your implementation goes here
+printf("get_serde\n");
+  }
+
 };
 
 int main(int argc, char **argv) {



[12/18] hive git commit: HIVE-17990 Add Thrift and DB storage for Schema Registry objects (Alan Gates, reviewed by Thejas Nair)

2018-03-12 Thread gates
http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ISchema.java
--
diff --git 
a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ISchema.java
 
b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ISchema.java
new file mode 100644
index 000..92d8b52
--- /dev/null
+++ 
b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ISchema.java
@@ -0,0 +1,1162 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public 
@org.apache.hadoop.classification.InterfaceStability.Stable public class 
ISchema implements org.apache.thrift.TBase, 
java.io.Serializable, Cloneable, Comparable {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new 
org.apache.thrift.protocol.TStruct("ISchema");
+
+  private static final org.apache.thrift.protocol.TField 
SCHEMA_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("schemaType", 
org.apache.thrift.protocol.TType.I32, (short)1);
+  private static final org.apache.thrift.protocol.TField NAME_FIELD_DESC = new 
org.apache.thrift.protocol.TField("name", 
org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = 
new org.apache.thrift.protocol.TField("dbName", 
org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField 
COMPATIBILITY_FIELD_DESC = new 
org.apache.thrift.protocol.TField("compatibility", 
org.apache.thrift.protocol.TType.I32, (short)4);
+  private static final org.apache.thrift.protocol.TField 
VALIDATION_LEVEL_FIELD_DESC = new 
org.apache.thrift.protocol.TField("validationLevel", 
org.apache.thrift.protocol.TType.I32, (short)5);
+  private static final org.apache.thrift.protocol.TField CAN_EVOLVE_FIELD_DESC 
= new org.apache.thrift.protocol.TField("canEvolve", 
org.apache.thrift.protocol.TType.BOOL, (short)6);
+  private static final org.apache.thrift.protocol.TField 
SCHEMA_GROUP_FIELD_DESC = new org.apache.thrift.protocol.TField("schemaGroup", 
org.apache.thrift.protocol.TType.STRING, (short)7);
+  private static final org.apache.thrift.protocol.TField 
DESCRIPTION_FIELD_DESC = new org.apache.thrift.protocol.TField("description", 
org.apache.thrift.protocol.TType.STRING, (short)8);
+
+  private static final Map schemes = 
new HashMap();
+  static {
+schemes.put(StandardScheme.class, new ISchemaStandardSchemeFactory());
+schemes.put(TupleScheme.class, new ISchemaTupleSchemeFactory());
+  }
+
+  private SchemaType schemaType; // required
+  private String name; // required
+  private String dbName; // required
+  private SchemaCompatibility compatibility; // required
+  private SchemaValidation validationLevel; // required
+  private boolean canEvolve; // required
+  private String schemaGroup; // optional
+  private String description; // optional
+
+  /** The set of fields this struct contains, along with convenience methods 
for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+/**
+ * 
+ * @see SchemaType
+ */
+SCHEMA_TYPE((short)1, "schemaType"),
+NAME((short)2, "name"),
+DB_NAME((short)3, "dbName"),
+/**
+ * 
+ * @see SchemaCompatibility
+ */
+COMPATIBILITY((short)4, "compatibility"),
+/**
+ * 
+ * @see SchemaValidation
+ */
+VALIDATION_LEVEL((short)5, "validationLevel"),
+CAN_EVOLVE((short)6, "canEvolve"),
+SCHEMA_GROUP((short)7, "schemaGroup"),

[01/18] hive git commit: HIVE-17990 Add Thrift and DB storage for Schema Registry objects (Alan Gates, reviewed by Thejas Nair)

2018-03-12 Thread gates
Repository: hive
Updated Branches:
  refs/heads/master d717d3853 -> 12041d39f


http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStoreSchemaMethods.java
--
diff --git 
a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStoreSchemaMethods.java
 
b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStoreSchemaMethods.java
new file mode 100644
index 000..2b8fbd1
--- /dev/null
+++ 
b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStoreSchemaMethods.java
@@ -0,0 +1,573 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.metastore;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.annotation.MetastoreCheckinTest;
+import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.ISchema;
+import org.apache.hadoop.hive.metastore.api.ISchemaName;
+import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.SchemaCompatibility;
+import org.apache.hadoop.hive.metastore.api.SchemaType;
+import org.apache.hadoop.hive.metastore.api.SchemaValidation;
+import org.apache.hadoop.hive.metastore.api.SchemaVersion;
+import org.apache.hadoop.hive.metastore.api.SchemaVersionDescriptor;
+import org.apache.hadoop.hive.metastore.api.SchemaVersionState;
+import org.apache.hadoop.hive.metastore.api.SerDeInfo;
+import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.ISchemaBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.SchemaVersionBuilder;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.thrift.TException;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
+
+@Category(MetastoreCheckinTest.class)
+public class TestObjectStoreSchemaMethods {
+  private RawStore objectStore;
+
+  @Before
+  public void setUp() throws Exception {
+Configuration conf = MetastoreConf.newMetastoreConf();
+MetastoreConf.setVar(conf, MetastoreConf.ConfVars.EXPRESSION_PROXY_CLASS,
+DefaultPartitionExpressionProxy.class.getName());
+
+objectStore = new ObjectStore();
+objectStore.setConf(conf);
+  }
+
+  @Test
+  public void iSchema() throws TException {
+String dbName = createUniqueDatabaseForTest();
+ISchema schema = objectStore.getISchema(new ISchemaName(dbName, 
"no.such.schema"));
+Assert.assertNull(schema);
+
+String schemaName = "schema1";
+String schemaGroup = "group1";
+String description = "This is a description";
+schema = new ISchemaBuilder()
+.setSchemaType(SchemaType.AVRO)
+.setName(schemaName)
+.setDbName(dbName)
+.setCompatibility(SchemaCompatibility.FORWARD)
+.setValidationLevel(SchemaValidation.LATEST)
+.setCanEvolve(false)
+.setSchemaGroup(schemaGroup)
+.setDescription(description)
+.build();
+objectStore.createISchema(schema);
+
+schema = objectStore.getISchema(new ISchemaName(dbName, schemaName));
+Assert.assertNotNull(schema);
+
+Assert.assertEquals(SchemaType.AVRO, schema.getSchemaType());
+Assert.assertEquals(schemaName, schema.getName());
+Assert.assertEquals(SchemaCompatibility.FORWARD, 
schema.getCompatibility());
+Assert.assertEquals(SchemaValidation.LATEST, schema.getValidationLevel());
+Assert.assertFalse(schema.isCanEvolve());
+Assert.assertEquals(schemaGroup, schema.getSchemaGroup());
+

[10/18] hive git commit: HIVE-17990 Add Thrift and DB storage for Schema Registry objects (Alan Gates, reviewed by Thejas Nair)

2018-03-12 Thread gates
http://git-wip-us.apache.org/repos/asf/hive/blob/12041d39/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SetSchemaVersionStateRequest.java
--
diff --git 
a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SetSchemaVersionStateRequest.java
 
b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SetSchemaVersionStateRequest.java
new file mode 100644
index 000..23bf9a4
--- /dev/null
+++ 
b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SetSchemaVersionStateRequest.java
@@ -0,0 +1,516 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public 
@org.apache.hadoop.classification.InterfaceStability.Stable public class 
SetSchemaVersionStateRequest implements 
org.apache.thrift.TBase, java.io.Serializable, Cloneable, 
Comparable {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new 
org.apache.thrift.protocol.TStruct("SetSchemaVersionStateRequest");
+
+  private static final org.apache.thrift.protocol.TField 
SCHEMA_VERSION_FIELD_DESC = new 
org.apache.thrift.protocol.TField("schemaVersion", 
org.apache.thrift.protocol.TType.STRUCT, (short)1);
+  private static final org.apache.thrift.protocol.TField STATE_FIELD_DESC = 
new org.apache.thrift.protocol.TField("state", 
org.apache.thrift.protocol.TType.I32, (short)2);
+
+  private static final Map schemes = 
new HashMap();
+  static {
+schemes.put(StandardScheme.class, new 
SetSchemaVersionStateRequestStandardSchemeFactory());
+schemes.put(TupleScheme.class, new 
SetSchemaVersionStateRequestTupleSchemeFactory());
+  }
+
+  private SchemaVersionDescriptor schemaVersion; // required
+  private SchemaVersionState state; // required
+
+  /** The set of fields this struct contains, along with convenience methods 
for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+SCHEMA_VERSION((short)1, "schemaVersion"),
+/**
+ * 
+ * @see SchemaVersionState
+ */
+STATE((short)2, "state");
+
+private static final Map byName = new HashMap();
+
+static {
+  for (_Fields field : EnumSet.allOf(_Fields.class)) {
+byName.put(field.getFieldName(), field);
+  }
+}
+
+/**
+ * Find the _Fields constant that matches fieldId, or null if its not 
found.
+ */
+public static _Fields findByThriftId(int fieldId) {
+  switch(fieldId) {
+case 1: // SCHEMA_VERSION
+  return SCHEMA_VERSION;
+case 2: // STATE
+  return STATE;
+default:
+  return null;
+  }
+}
+
+/**
+ * Find the _Fields constant that matches fieldId, throwing an exception
+ * if it is not found.
+ */
+public static _Fields findByThriftIdOrThrow(int fieldId) {
+  _Fields fields = findByThriftId(fieldId);
+  if (fields == null) throw new IllegalArgumentException("Field " + 
fieldId + " doesn't exist!");
+  return fields;
+}
+
+/**
+ * Find the _Fields constant that matches name, or null if its not found.
+ */
+public static _Fields findByName(String name) {
+  return byName.get(name);
+}
+
+private final short _thriftId;
+private final String _fieldName;
+
+_Fields(short thriftId, String fieldName) {
+  _thriftId = thriftId;
+  _fieldName = fieldName;
+}
+
+public short getThriftFieldId() {
+  return _thriftId;
+}
+
+public String getFieldName() {
+  return 

hive git commit: HIVE-14792: AvroSerde reads the remote schema-file at least once per mapper, per table reference. (Addendum)

2018-03-12 Thread aihuaxu
Repository: hive
Updated Branches:
  refs/heads/branch-2 cbcd846b7 -> ad5a2fa0e


HIVE-14792: AvroSerde reads the remote schema-file at least once per mapper, 
per table reference. (Addendum)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/ad5a2fa0
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/ad5a2fa0
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/ad5a2fa0

Branch: refs/heads/branch-2
Commit: ad5a2fa0eed89bf88a4287965274c4988e71ee98
Parents: cbcd846
Author: Aihua Xu 
Authored: Thu Mar 8 11:33:37 2018 -0800
Committer: Aihua Xu 
Committed: Mon Mar 12 14:51:43 2018 -0700

--
 .../TablePropertyEnrichmentOptimizer.java   |  45 +++-
 .../avro_tableproperty_optimize.q   |  63 ++
 .../avro_tableproperty_optimize.q.out   | 226 +++
 3 files changed, 324 insertions(+), 10 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/ad5a2fa0/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
--
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
index 5824490..154eb02 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
@@ -26,6 +26,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
 import org.apache.hadoop.hive.ql.lib.Dispatcher;
@@ -40,8 +41,10 @@ import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.TableScanDesc;
 import org.apache.hadoop.hive.serde2.Deserializer;
+import org.apache.hive.common.util.ReflectionUtil;
 
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
@@ -78,29 +81,51 @@ class TablePropertyEnrichmentOptimizer extends Transform {
 }
   }
 
+  /**
+   * Retrieves the table properties as well as the properties from Serde.
+   */
+  private static Map getTableParameters(Table table) {
+Map originalTableParameters = new 
HashMap<>(table.getParameters());
+Properties tableMetadata = MetaStoreUtils.getTableMetadata(table);
+for (String property : tableMetadata.stringPropertyNames()) {
+  if (!originalTableParameters.containsKey(property)) {
+originalTableParameters.put(property, 
tableMetadata.getProperty(property));
+  }
+}
+return originalTableParameters;
+  }
+
   private static class Processor implements NodeProcessor {
 
 @Override
 public Object process(Node nd, Stack stack, NodeProcessorCtx 
procCtx, Object... nodeOutputs) throws SemanticException {
   TableScanOperator tsOp = (TableScanOperator) nd;
   WalkerCtx context = (WalkerCtx)procCtx;
-
   TableScanDesc tableScanDesc = tsOp.getConf();
   Table table = tsOp.getConf().getTableMetadata().getTTable();
-  Map tableParameters = table.getParameters();
-  Properties tableProperties = new Properties();
-  tableProperties.putAll(tableParameters);
 
-  Deserializer deserializer = 
tableScanDesc.getTableMetadata().getDeserializer();
-  String deserializerClassName = deserializer.getClass().getName();
+  Map originalTableParameters = getTableParameters(table);
+  if (LOG.isDebugEnabled()) {
+LOG.debug("Original Table parameters: " + originalTableParameters);
+  }
+  Properties clonedTableParameters = new Properties();
+  clonedTableParameters.putAll(originalTableParameters);
+
+  String deserializerClassName = null;
   try {
+deserializerClassName = 
tableScanDesc.getTableMetadata().getSd().getSerdeInfo().getSerializationLib();
+Deserializer deserializer = ReflectionUtil.newInstance(
+context.conf.getClassByName(deserializerClassName)
+.asSubclass(Deserializer.class),
+context.conf);
+
 if 
(context.serdeClassesUnderConsideration.contains(deserializerClassName)) {
-  deserializer.initialize(context.conf, tableProperties);
+  deserializer.initialize(context.conf, clonedTableParameters);
   LOG.debug("SerDe init 

hive git commit: HIVE-14792: AvroSerde reads the remote schema-file at least once per mapper, per table reference. (Addendum)

2018-03-12 Thread aihuaxu
Repository: hive
Updated Branches:
  refs/heads/branch-2.2 5fee15988 -> e8e3974d1


HIVE-14792: AvroSerde reads the remote schema-file at least once per mapper, 
per table reference. (Addendum)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/e8e3974d
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/e8e3974d
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/e8e3974d

Branch: refs/heads/branch-2.2
Commit: e8e3974d157a550ec0112258b59b603d9829
Parents: 5fee159
Author: Aihua Xu 
Authored: Thu Mar 8 11:33:37 2018 -0800
Committer: Aihua Xu 
Committed: Mon Mar 12 14:39:50 2018 -0700

--
 .../TablePropertyEnrichmentOptimizer.java   |  45 +++-
 .../avro_tableproperty_optimize.q   |  63 ++
 .../avro_tableproperty_optimize.q.out   | 226 +++
 3 files changed, 324 insertions(+), 10 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/e8e3974d/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
--
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
index 5824490..d313f7d 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/TablePropertyEnrichmentOptimizer.java
@@ -26,6 +26,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
 import org.apache.hadoop.hive.ql.lib.Dispatcher;
@@ -40,8 +41,10 @@ import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.TableScanDesc;
 import org.apache.hadoop.hive.serde2.Deserializer;
+import org.apache.hive.common.util.ReflectionUtil;
 
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
@@ -78,29 +81,51 @@ class TablePropertyEnrichmentOptimizer extends Transform {
 }
   }
 
+  /**
+   * Retrieves the table properties as well as the properties from Serde.
+   */
+  private static Map getTableParameters(Table table) {
+Map originalTableParameters = new 
HashMap<>(table.getParameters());
+Properties tableMetadata = MetaStoreUtils.getTableMetadata(table);
+for (String property : tableMetadata.stringPropertyNames()) {
+  if (!originalTableParameters.containsKey(property)) {
+originalTableParameters.put(property, 
tableMetadata.getProperty(property));
+  }
+}
+return originalTableParameters;
+  }
+
   private static class Processor implements NodeProcessor {
 
 @Override
 public Object process(Node nd, Stack stack, NodeProcessorCtx 
procCtx, Object... nodeOutputs) throws SemanticException {
   TableScanOperator tsOp = (TableScanOperator) nd;
   WalkerCtx context = (WalkerCtx)procCtx;
-
   TableScanDesc tableScanDesc = tsOp.getConf();
   Table table = tsOp.getConf().getTableMetadata().getTTable();
-  Map tableParameters = table.getParameters();
-  Properties tableProperties = new Properties();
-  tableProperties.putAll(tableParameters);
 
-  Deserializer deserializer = 
tableScanDesc.getTableMetadata().getDeserializer();
-  String deserializerClassName = deserializer.getClass().getName();
+  Map originalTableParameters = getTableParameters(table);
+  if (LOG.isDebugEnabled()) {
+LOG.debug("Original Table parameters: " + originalTableParameters);
+  }
+  Properties clonedTableParameters = new Properties();
+  clonedTableParameters.putAll(originalTableParameters);
+
+  String deserializerClassName = null;
   try {
+deserializerClassName = 
tableScanDesc.getTableMetadata().getSd().getSerdeInfo().getSerializationLib();
+Deserializer deserializer = ReflectionUtil.newInstance(
+context.conf.getClassByName(deserializerClassName)
+.asSubclass(Deserializer.class),
+context.conf);
+
 if 
(context.serdeClassesUnderConsideration.contains(deserializerClassName)) {
-  deserializer.initialize(context.conf, tableProperties);
+  deserializer.initialize(context.conf, clonedTableParameters);
   LOG.debug("SerDe 

hive git commit: HIVE-18675 - make HIVE_LOCKS.HL_TXNID NOT NULL (Igor Kryvenko via Eugene Koifman)

2018-03-12 Thread ekoifman
Repository: hive
Updated Branches:
  refs/heads/master 0fe38792a -> 190c72e77


HIVE-18675 - make HIVE_LOCKS.HL_TXNID NOT NULL (Igor Kryvenko via Eugene 
Koifman)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/190c72e7
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/190c72e7
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/190c72e7

Branch: refs/heads/master
Commit: 190c72e7793d80da48b5c71b5ff0a9547334bedf
Parents: 0fe3879
Author: Eugene Koifman 
Authored: Mon Mar 12 14:05:29 2018 -0700
Committer: Eugene Koifman 
Committed: Mon Mar 12 14:05:46 2018 -0700

--
 metastore/scripts/upgrade/derby/051-HIVE-18675.derby.sql  |  1 +
 .../scripts/upgrade/derby/hive-txn-schema-3.0.0.derby.sql |  6 +++---
 .../upgrade/derby/upgrade-2.3.0-to-3.0.0.derby.sql|  1 +
 .../org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java   |  8 
 .../src/main/sql/derby/hive-schema-3.0.0.derby.sql|  4 ++--
 .../src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql   |  2 ++
 .../src/main/sql/mssql/hive-schema-3.0.0.mssql.sql| 10 +-
 .../src/main/sql/mssql/upgrade-2.3.0-to-3.0.0.mssql.sql   |  4 +++-
 .../src/main/sql/mysql/hive-schema-3.0.0.mysql.sql|  4 ++--
 .../src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql   |  4 +++-
 .../src/main/sql/oracle/hive-schema-3.0.0.oracle.sql  |  2 +-
 .../src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql |  2 ++
 .../src/main/sql/postgres/hive-schema-3.0.0.postgres.sql  |  4 ++--
 .../main/sql/postgres/upgrade-2.3.0-to-3.0.0.postgres.sql |  4 +++-
 14 files changed, 34 insertions(+), 22 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/hive/blob/190c72e7/metastore/scripts/upgrade/derby/051-HIVE-18675.derby.sql
--
diff --git a/metastore/scripts/upgrade/derby/051-HIVE-18675.derby.sql 
b/metastore/scripts/upgrade/derby/051-HIVE-18675.derby.sql
new file mode 100644
index 000..9754292
--- /dev/null
+++ b/metastore/scripts/upgrade/derby/051-HIVE-18675.derby.sql
@@ -0,0 +1 @@
+ALTER TABLE "APP"."HIVE_LOCKS" ALTER COLUMN "HL_TXNID" NOT NULL;

http://git-wip-us.apache.org/repos/asf/hive/blob/190c72e7/metastore/scripts/upgrade/derby/hive-txn-schema-3.0.0.derby.sql
--
diff --git a/metastore/scripts/upgrade/derby/hive-txn-schema-3.0.0.derby.sql 
b/metastore/scripts/upgrade/derby/hive-txn-schema-3.0.0.derby.sql
index 2033bdc..6001262 100644
--- a/metastore/scripts/upgrade/derby/hive-txn-schema-3.0.0.derby.sql
+++ b/metastore/scripts/upgrade/derby/hive-txn-schema-3.0.0.derby.sql
@@ -15,7 +15,7 @@
 
 --
 -- Tables for transaction management
--- 
+--
 CREATE TABLE TXNS (
   TXN_ID bigint PRIMARY KEY,
   TXN_STATE char(1) NOT NULL,
@@ -75,7 +75,7 @@ CREATE UNIQUE INDEX NEXT_WRITE_ID_IDX ON NEXT_WRITE_ID 
(NWI_DATABASE, NWI_TABLE)
 CREATE TABLE HIVE_LOCKS (
   HL_LOCK_EXT_ID bigint NOT NULL,
   HL_LOCK_INT_ID bigint NOT NULL,
-  HL_TXNID bigint,
+  HL_TXNID bigint NOT NULL,
   HL_DB varchar(128) NOT NULL,
   HL_TABLE varchar(128),
   HL_PARTITION varchar(767),
@@ -90,7 +90,7 @@ CREATE TABLE HIVE_LOCKS (
   HL_BLOCKEDBY_EXT_ID bigint,
   HL_BLOCKEDBY_INT_ID bigint,
   PRIMARY KEY(HL_LOCK_EXT_ID, HL_LOCK_INT_ID)
-); 
+);
 
 CREATE INDEX HL_TXNID_INDEX ON HIVE_LOCKS (HL_TXNID);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/190c72e7/metastore/scripts/upgrade/derby/upgrade-2.3.0-to-3.0.0.derby.sql
--
diff --git a/metastore/scripts/upgrade/derby/upgrade-2.3.0-to-3.0.0.derby.sql 
b/metastore/scripts/upgrade/derby/upgrade-2.3.0-to-3.0.0.derby.sql
index 55b89e7..c2dcb83 100644
--- a/metastore/scripts/upgrade/derby/upgrade-2.3.0-to-3.0.0.derby.sql
+++ b/metastore/scripts/upgrade/derby/upgrade-2.3.0-to-3.0.0.derby.sql
@@ -8,5 +8,6 @@ RUN '046-HIVE-17566.derby.sql';
 RUN '048-HIVE-14498.derby.sql';
 RUN '049-HIVE-18489.derby.sql';
 RUN '050-HIVE-18192.derby.sql';
+RUN '051-HIVE-18675.derby.sql';
 
 UPDATE "APP".VERSION SET SCHEMA_VERSION='3.0.0', VERSION_COMMENT='Hive release 
version 3.0.0' where VER_ID=1;

http://git-wip-us.apache.org/repos/asf/hive/blob/190c72e7/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
--
diff --git 
a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
 
b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
index 88f6346..588f335 100644
--- 
a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
+++